From 6b800e679c02321200244b18fd9687247631dd77 Mon Sep 17 00:00:00 2001 From: Alex Barreto Date: Thu, 3 Dec 2020 10:33:26 -0600 Subject: [PATCH] [SPARK-30098][SQL] Add a configuration to use default datasource as provider for CREATE TABLE command ### What changes were proposed in this pull request? For CRETE TABLE [AS SELECT] command, creates native Parquet table if neither USING nor STORE AS is specified and `spark.sql.legacy.createHiveTableByDefault` is false. This is a retry after we unify the CREATE TABLE syntax. It partially reverts https://github.com/apache/spark/commit/d2bec5e265e0aa4fa527c3f43cfe738cdbdc4598 This PR allows `CREATE EXTERNAL TABLE` when `LOCATION` is present. This was not allowed for data source tables before, which is an unnecessary behavior different with hive tables. ### Why are the changes needed? Changing from Hive text table to native Parquet table has many benefits: 1. be consistent with `DataFrameWriter.saveAsTable`. 2. better performance 3. better support for nested types (Hive text table doesn't work well with nested types, e.g. `insert into t values struct(null)` actually inserts a null value not `struct(null)` if `t` is a Hive text table, which leads to wrong result) 4. better interoperability as Parquet is a more popular open file format. ### Does this PR introduce _any_ user-facing change? No by default. If the config is set, the behavior change is described below: Behavior-wise, the change is very small as the native Parquet table is also Hive-compatible. All the Spark DDL commands that works for hive tables also works for native Parquet tables, with two exceptions: `ALTER TABLE SET [SERDE | SERDEPROPERTIES]` and `LOAD DATA`. char/varchar behavior has been taken care by https://github.com/apache/spark/pull/30412, and there is no behavior difference between data source and hive tables. One potential issue is `CREATE TABLE ... LOCATION ...` while users want to directly access the files later. It's more like a corner case and the legacy config should be good enough. Another potential issue is users may use Spark to create the table and then use Hive to add partitions with different serde. This is not allowed for Spark native tables. ### How was this patch tested? Re-enable the tests Closes #30554 from cloud-fan/create-table. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../BlockManagerReplicationSuite.scala | 2 - dev/.rat-excludes | 1 + docs/configuration.md | 11 -- .../org/apache/spark/ml/feature/Imputer.scala | 29 +-- python/pyspark/ml/feature.py | 12 ++ python/pyspark/ml/tests/test_feature.py | 1 + .../catalyst/analysis/FunctionRegistry.scala | 5 + .../spark/sql/catalyst/expressions/Cast.scala | 52 ++---- .../expressions/datetimeExpressions.scala | 119 +++++++++++++ .../apache/spark/sql/internal/SQLConf.scala | 9 + .../sql/catalyst/expressions/CastSuite.scala | 5 + .../expressions/DateExpressionsSuite.scala | 69 ++++++++ ...apache.spark.status.AppHistoryServerPlugin | 1 + .../analysis/ResolveSessionCatalog.scala | 13 +- .../streaming/StreamingQueryListenerBus.scala | 26 ++- .../StreamingQueryHistoryServerPlugin.scala | 43 +++++ .../ui/StreamingQueryStatusStore.scala | 53 ++++++ .../spark/sql/internal/SharedState.scala | 8 +- .../sql/streaming/DataStreamWriter.scala | 2 +- .../sql/streaming/StreamingQueryManager.scala | 3 +- .../sql/streaming/ui/StreamingQueryPage.scala | 44 ++--- .../ui/StreamingQueryStatisticsPage.scala | 27 +-- .../ui/StreamingQueryStatusListener.scala | 166 +++++++++++------- .../sql/streaming/ui/StreamingQueryTab.scala | 3 +- .../spark/sql/streaming/ui/UIUtils.scala | 12 +- .../spark-events/local-1596020211915 | 160 +++++++++++++++++ .../sql-functions/sql-expression-schema.md | 7 +- .../resources/sql-tests/inputs/datetime.sql | 9 +- .../sql-tests/results/ansi/datetime.sql.out | 42 ++++- .../sql-tests/results/datetime-legacy.sql.out | 42 ++++- .../sql-tests/results/datetime.sql.out | 42 ++++- .../apache/spark/deploy/history/Utils.scala | 40 +++++ .../sql/connector/DataSourceV2SQLSuite.scala | 33 ++-- .../command/PlanResolutionSuite.scala | 6 +- .../test/DataStreamTableAPISuite.scala | 2 +- .../ui/StreamingQueryHistorySuite.scala | 63 +++++++ .../ui/StreamingQueryPageSuite.scala | 42 +++-- .../StreamingQueryStatusListenerSuite.scala | 159 ++++++++++++++--- .../execution/HiveCompatibilitySuite.scala | 4 + .../sql/hive/HiveShowCreateTableSuite.scala | 18 +- .../apache/spark/sql/hive/InsertSuite.scala | 3 +- .../spark/sql/hive/QueryPartitionSuite.scala | 5 +- .../spark/sql/hive/StatisticsSuite.scala | 27 ++- .../spark/sql/hive/client/VersionsSuite.scala | 1 + .../sql/hive/execution/HiveDDLSuite.scala | 2 +- .../sql/hive/execution/HiveSerDeSuite.scala | 5 +- .../hive/execution/HiveTableScanSuite.scala | 5 +- .../sql/hive/execution/SQLQuerySuite.scala | 1 + .../apache/spark/sql/hive/test/TestHive.scala | 13 +- 49 files changed, 1179 insertions(+), 268 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala create mode 100644 sql/core/src/test/resources/spark-events/local-1596020211915 create mode 100644 sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 0b673c580d..1e9b481026 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -95,8 +95,6 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite conf.set(MEMORY_STORAGE_FRACTION, 0.999) conf.set(STORAGE_UNROLL_MEMORY_THRESHOLD, 512L) - // to make a replication attempt to inactive store fail fast - conf.set("spark.core.connection.ack.wait.timeout", "1s") // to make cached peers refresh frequently conf.set(STORAGE_CACHED_PEERS_TTL, 10) diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 7da330dfe1..167cf224f9 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -123,6 +123,7 @@ SessionHandler.java GangliaReporter.java application_1578436911597_0052 config.properties +local-1596020211915 app-20200706201101-0003 py.typed _metadata diff --git a/docs/configuration.md b/docs/configuration.md index d4d8e47645..21506e6901 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1919,7 +1919,6 @@ Apart from these, the following properties are also available, and may be useful 120s Default timeout for all network interactions. This config will be used in place of - spark.core.connection.ack.wait.timeout, spark.storage.blockManagerHeartbeatTimeoutMs, spark.shuffle.io.connectionTimeout, spark.rpc.askTimeout or spark.rpc.lookupTimeout if they are not configured. @@ -1982,16 +1981,6 @@ Apart from these, the following properties are also available, and may be useful 1.4.0 - - spark.core.connection.ack.wait.timeout - spark.network.timeout - - How long for the connection to wait for ack to occur before timing - out and giving up. To avoid unwilling timeout caused by long pause like GC, - you can set larger value. - - 1.1.1 - spark.network.maxRemoteBlockSizeFetchToMem 200m diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala index 03ebe0299f..d0b6ab1ef2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala @@ -254,20 +254,25 @@ class ImputerModel private[ml] ( /** @group setParam */ def setOutputCols(value: Array[String]): this.type = set(outputCols, value) + @transient private lazy val surrogates = { + val row = surrogateDF.head() + row.schema.fieldNames.zipWithIndex + .map { case (name, index) => (name, row.getDouble(index)) } + .toMap + } + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val (inputColumns, outputColumns) = getInOutCols - val surrogates = surrogateDF.select(inputColumns.map(col): _*).head().toSeq - - - val newCols = inputColumns.zip(outputColumns).zip(surrogates).map { - case ((inputCol, outputCol), surrogate) => - val inputType = dataset.schema(inputCol).dataType - val ic = col(inputCol).cast(DoubleType) - when(ic.isNull, surrogate) - .when(ic === $(missingValue), surrogate) - .otherwise(ic) - .cast(inputType) + val (inputColumns, outputColumns) = getInOutCols() + + val newCols = inputColumns.map { inputCol => + val surrogate = surrogates(inputCol) + val inputType = dataset.schema(inputCol).dataType + val ic = col(inputCol).cast(DoubleType) + when(ic.isNull, surrogate) + .when(ic === $(missingValue), surrogate) + .otherwise(ic) + .cast(inputType) } dataset.withColumns(outputColumns, newCols).toDF() } diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 8138f34d7a..546c46383d 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -3852,9 +3852,21 @@ def from_arrays_of_labels(cls, arrayOfLabels, inputCols, outputCols=None, def labels(self): """ Ordered list of labels, corresponding to indices to be assigned. + + .. deprecated:: 3.1.0 + It will be removed in future versions. Use `labelsArray` method instead. """ return self._call_java("labels") + @property + @since("3.0.2") + def labelsArray(self): + """ + Array of ordered list of labels, corresponding to indices to be assigned + for each input column. + """ + return self._call_java("labelsArray") + @inherit_doc class IndexToString(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): diff --git a/python/pyspark/ml/tests/test_feature.py b/python/pyspark/ml/tests/test_feature.py index 98b8ce6dfb..2cceb04338 100644 --- a/python/pyspark/ml/tests/test_feature.py +++ b/python/pyspark/ml/tests/test_feature.py @@ -232,6 +232,7 @@ def test_string_indexer_from_labels(self): model = StringIndexerModel.from_labels(["a", "b", "c"], inputCol="label", outputCol="indexed", handleInvalid="keep") self.assertEqual(model.labels, ["a", "b", "c"]) + self.assertEqual(model.labelsArray, [("a", "b", "c")]) df1 = self.spark.createDataFrame([ (0, "a"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 6fb9bed962..3b46de539c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -427,9 +427,14 @@ object FunctionRegistry { expression[MakeInterval]("make_interval"), expression[DatePart]("date_part"), expression[Extract]("extract"), + expression[DateFromUnixDate]("date_from_unix_date"), + expression[UnixDate]("unix_date"), expression[SecondsToTimestamp]("timestamp_seconds"), expression[MillisToTimestamp]("timestamp_millis"), expression[MicrosToTimestamp]("timestamp_micros"), + expression[UnixSeconds]("unix_seconds"), + expression[UnixMillis]("unix_millis"), + expression[UnixMicros]("unix_micros"), // collection functions expression[CreateArray]("array"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 95f09d64c4..1b2e2db932 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -1393,25 +1393,19 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit """ } - private[this] def lowerAndUpperBound( - fractionType: String, - integralType: String): (String, String) = { - assert(fractionType == "float" || fractionType == "double") - val typeIndicator = fractionType.charAt(0) - val (min, max) = integralType.toLowerCase(Locale.ROOT) match { - case "long" => (Long.MinValue, Long.MaxValue) - case "int" => (Int.MinValue, Int.MaxValue) - case "short" => (Short.MinValue, Short.MaxValue) - case "byte" => (Byte.MinValue, Byte.MaxValue) + private[this] def lowerAndUpperBound(integralType: String): (String, String) = { + val (min, max, typeIndicator) = integralType.toLowerCase(Locale.ROOT) match { + case "long" => (Long.MinValue, Long.MaxValue, "L") + case "int" => (Int.MinValue, Int.MaxValue, "") + case "short" => (Short.MinValue, Short.MaxValue, "") + case "byte" => (Byte.MinValue, Byte.MaxValue, "") } (min.toString + typeIndicator, max.toString + typeIndicator) } - private[this] def castFractionToIntegralTypeCode( - fractionType: String, - integralType: String): CastFunction = { + private[this] def castFractionToIntegralTypeCode(integralType: String): CastFunction = { assert(ansiEnabled) - val (min, max) = lowerAndUpperBound(fractionType, integralType) + val (min, max) = lowerAndUpperBound(integralType) val mathClass = classOf[Math].getName // When casting floating values to integral types, Spark uses the method `Numeric.toInt` // Or `Numeric.toLong` directly. For positive floating values, it is equivalent to `Math.floor`; @@ -1449,12 +1443,10 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit (c, evPrim, evNull) => code"$evNull = true;" case TimestampType => castTimestampToIntegralTypeCode(ctx, "byte") case DecimalType() => castDecimalToIntegralTypeCode(ctx, "byte") - case _: ShortType | _: IntegerType | _: LongType if ansiEnabled => + case ShortType | IntegerType | LongType if ansiEnabled => castIntegralTypeToIntegralTypeExactCode("byte") - case _: FloatType if ansiEnabled => - castFractionToIntegralTypeCode("float", "byte") - case _: DoubleType if ansiEnabled => - castFractionToIntegralTypeCode("double", "byte") + case FloatType | DoubleType if ansiEnabled => + castFractionToIntegralTypeCode("byte") case x: NumericType => (c, evPrim, evNull) => code"$evPrim = (byte) $c;" } @@ -1482,12 +1474,10 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit (c, evPrim, evNull) => code"$evNull = true;" case TimestampType => castTimestampToIntegralTypeCode(ctx, "short") case DecimalType() => castDecimalToIntegralTypeCode(ctx, "short") - case _: IntegerType | _: LongType if ansiEnabled => + case IntegerType | LongType if ansiEnabled => castIntegralTypeToIntegralTypeExactCode("short") - case _: FloatType if ansiEnabled => - castFractionToIntegralTypeCode("float", "short") - case _: DoubleType if ansiEnabled => - castFractionToIntegralTypeCode("double", "short") + case FloatType | DoubleType if ansiEnabled => + castFractionToIntegralTypeCode("short") case x: NumericType => (c, evPrim, evNull) => code"$evPrim = (short) $c;" } @@ -1513,11 +1503,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit (c, evPrim, evNull) => code"$evNull = true;" case TimestampType => castTimestampToIntegralTypeCode(ctx, "int") case DecimalType() => castDecimalToIntegralTypeCode(ctx, "int") - case _: LongType if ansiEnabled => castIntegralTypeToIntegralTypeExactCode("int") - case _: FloatType if ansiEnabled => - castFractionToIntegralTypeCode("float", "int") - case _: DoubleType if ansiEnabled => - castFractionToIntegralTypeCode("double", "int") + case LongType if ansiEnabled => castIntegralTypeToIntegralTypeExactCode("int") + case FloatType | DoubleType if ansiEnabled => + castFractionToIntegralTypeCode("int") case x: NumericType => (c, evPrim, evNull) => code"$evPrim = (int) $c;" } @@ -1544,10 +1532,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case TimestampType => (c, evPrim, evNull) => code"$evPrim = (long) ${timestampToLongCode(c)};" case DecimalType() => castDecimalToIntegralTypeCode(ctx, "long") - case _: FloatType if ansiEnabled => - castFractionToIntegralTypeCode("float", "long") - case _: DoubleType if ansiEnabled => - castFractionToIntegralTypeCode("double", "long") + case FloatType | DoubleType if ansiEnabled => + castFractionToIntegralTypeCode("long") case x: NumericType => (c, evPrim, evNull) => code"$evPrim = (long) $c;" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 424887a13c..c20dd6148b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -400,6 +400,52 @@ case class DayOfYear(child: Expression) extends GetDateField { override val funcName = "getDayInYear" } +@ExpressionDescription( + usage = "_FUNC_(days) - Create date from the number of days since 1970-01-01.", + examples = """ + Examples: + > SELECT _FUNC_(1); + 1970-01-02 + """, + group = "datetime_funcs", + since = "3.1.0") +case class DateFromUnixDate(child: Expression) extends UnaryExpression + with ImplicitCastInputTypes with NullIntolerant { + override def inputTypes: Seq[AbstractDataType] = Seq(IntegerType) + + override def dataType: DataType = DateType + + override def nullSafeEval(input: Any): Any = input.asInstanceOf[Int] + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + defineCodeGen(ctx, ev, c => c) + + override def prettyName: String = "date_from_unix_date" +} + +@ExpressionDescription( + usage = "_FUNC_(date) - Returns the number of days since 1970-01-01.", + examples = """ + Examples: + > SELECT _FUNC_(DATE("1970-01-02")); + 1 + """, + group = "datetime_funcs", + since = "3.1.0") +case class UnixDate(child: Expression) extends UnaryExpression + with ExpectsInputTypes with NullIntolerant { + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) + + override def dataType: DataType = IntegerType + + override def nullSafeEval(input: Any): Any = input.asInstanceOf[Int] + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + defineCodeGen(ctx, ev, c => c) + + override def prettyName: String = "unix_date" +} + abstract class IntegralToTimestampBase extends UnaryExpression with ExpectsInputTypes with NullIntolerant { @@ -524,6 +570,79 @@ case class MicrosToTimestamp(child: Expression) override def prettyName: String = "timestamp_micros" } +abstract class TimestampToLongBase extends UnaryExpression + with ExpectsInputTypes with NullIntolerant { + + protected def scaleFactor: Long + + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + + override def dataType: DataType = LongType + + override def nullSafeEval(input: Any): Any = { + Math.floorDiv(input.asInstanceOf[Number].longValue(), scaleFactor) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + if (scaleFactor == 1) { + defineCodeGen(ctx, ev, c => c) + } else { + defineCodeGen(ctx, ev, c => s"java.lang.Math.floorDiv($c, ${scaleFactor}L)") + } + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(timestamp) - Returns the number of seconds since 1970-01-01 00:00:00 UTC. Truncates higher levels of precision.", + examples = """ + Examples: + > SELECT _FUNC_(TIMESTAMP('1970-01-01 00:00:01Z')); + 1 + """, + group = "datetime_funcs", + since = "3.1.0") +// scalastyle:on line.size.limit +case class UnixSeconds(child: Expression) extends TimestampToLongBase { + override def scaleFactor: Long = MICROS_PER_SECOND + + override def prettyName: String = "unix_seconds" +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(timestamp) - Returns the number of milliseconds since 1970-01-01 00:00:00 UTC. Truncates higher levels of precision.", + examples = """ + Examples: + > SELECT _FUNC_(TIMESTAMP('1970-01-01 00:00:01Z')); + 1000 + """, + group = "datetime_funcs", + since = "3.1.0") +// scalastyle:on line.size.limit +case class UnixMillis(child: Expression) extends TimestampToLongBase { + override def scaleFactor: Long = MICROS_PER_MILLIS + + override def prettyName: String = "unix_millis" +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(timestamp) - Returns the number of microseconds since 1970-01-01 00:00:00 UTC.", + examples = """ + Examples: + > SELECT _FUNC_(TIMESTAMP('1970-01-01 00:00:01Z')); + 1000000 + """, + group = "datetime_funcs", + since = "3.1.0") +// scalastyle:on line.size.limit +case class UnixMicros(child: Expression) extends TimestampToLongBase { + override def scaleFactor: Long = 1L + + override def prettyName: String = "unix_micros" +} + @ExpressionDescription( usage = "_FUNC_(date) - Returns the year component of the date/timestamp.", examples = """ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index a1d6f9f608..b32476a5af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2921,6 +2921,15 @@ object SQLConf { .stringConf .createWithDefault("") + val LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT = + buildConf("spark.sql.legacy.createHiveTableByDefault") + .internal() + .doc("When set to true, CREATE TABLE syntax without USING or STORED AS will use Hive " + + s"instead of the value of ${DEFAULT_DATA_SOURCE_NAME.key} as the table provider.") + .version("3.1.0") + .booleanConf + .createWithDefault(true) + /** * Holds information about keys that have been deprecated. * diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index d284c41704..35db25ec93 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -975,6 +975,11 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase { } } } + + test("SPARK-26218: Fix the corner case of codegen when casting float to Integer") { + checkExceptionInExpression[ArithmeticException]( + cast(cast(Literal("2147483648"), FloatType), IntegerType), "overflow") + } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 587ca0cdbe..79770505ec 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -1245,6 +1245,75 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkResult(Int.MinValue.toLong - 100) } + test("DATE_FROM_UNIX_DATE") { + def testIntegralFunc(value: Number): Unit = { + checkEvaluation( + DateFromUnixDate(Literal(value.intValue())), + LocalDate.ofEpochDay(value.intValue())) + } + // test null input + checkEvaluation(DateFromUnixDate(Literal(null, IntegerType)), null) + // test integral input + testIntegralInput(testIntegralFunc) + } + + test("UNIX_DATE") { + def testIntegralFunc(value: Number): Unit = { + checkEvaluation( + UnixDate(Literal(LocalDate.ofEpochDay(value.intValue()))), + value.intValue()) + } + // test null input + checkEvaluation(UnixDate(Literal(null, DateType)), null) + // test various inputs + testIntegralInput(testIntegralFunc) + } + + test("UNIX_SECONDS") { + checkEvaluation(UnixSeconds(Literal(null, TimestampType)), null) + var timestamp = Literal(new Timestamp(0L)) + checkEvaluation(UnixSeconds(timestamp), 0L) + timestamp = Literal(new Timestamp(1000L)) + checkEvaluation(UnixSeconds(timestamp), 1L) + timestamp = Literal(new Timestamp(-1000L)) + checkEvaluation(UnixSeconds(timestamp), -1L) + // -1ms is considered to be in -1st second, as 0-999ms is in 0th second. + timestamp = Literal(new Timestamp(-1L)) + checkEvaluation(UnixSeconds(timestamp), -1L) + timestamp = Literal(new Timestamp(-1000L)) + checkEvaluation(UnixSeconds(timestamp), -1L) + // Truncates higher levels of precision + timestamp = Literal(new Timestamp(1999L)) + checkEvaluation(UnixSeconds(timestamp), 1L) + } + + test("UNIX_MILLIS") { + checkEvaluation(UnixMillis(Literal(null, TimestampType)), null) + var timestamp = Literal(new Timestamp(0L)) + checkEvaluation(UnixMillis(timestamp), 0L) + timestamp = Literal(new Timestamp(1000L)) + checkEvaluation(UnixMillis(timestamp), 1000L) + timestamp = Literal(new Timestamp(-1000L)) + checkEvaluation(UnixMillis(timestamp), -1000L) + // Truncates higher levels of precision + val timestampWithNanos = new Timestamp(1000L) + timestampWithNanos.setNanos(999999) + checkEvaluation(UnixMillis(Literal(timestampWithNanos)), 1000L) + } + + test("UNIX_MICROS") { + checkEvaluation(UnixMicros(Literal(null, TimestampType)), null) + var timestamp = Literal(new Timestamp(0L)) + checkEvaluation(UnixMicros(timestamp), 0L) + timestamp = Literal(new Timestamp(1000L)) + checkEvaluation(UnixMicros(timestamp), 1000000L) + timestamp = Literal(new Timestamp(-1000L)) + checkEvaluation(UnixMicros(timestamp), -1000000L) + val timestampWithNanos = new Timestamp(1000L) + timestampWithNanos.setNanos(1000) // 1 microsecond + checkEvaluation(UnixMicros(Literal(timestampWithNanos)), 1000001L) + } + test("TIMESTAMP_SECONDS") { def testIntegralFunc(value: Number): Unit = { checkEvaluation( diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin b/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin index 0bba2f88b9..6771eef525 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin @@ -1 +1,2 @@ org.apache.spark.sql.execution.ui.SQLHistoryServerPlugin +org.apache.spark.sql.execution.ui.StreamingQueryHistoryServerPlugin diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index f6005f4b41..f35fcdc07c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 -import org.apache.spark.sql.internal.HiveSerDe +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.types.{MetadataBuilder, StructField, StructType} /** @@ -636,11 +636,16 @@ class ResolveSessionCatalog( (storageFormat, DDLUtils.HIVE_PROVIDER) } else { // If neither USING nor STORED AS/ROW FORMAT is specified, we create native data source - // tables if it's a CTAS and `conf.convertCTAS` is true. - // TODO: create native data source table by default for non-CTAS. - if (ctas && conf.convertCTAS) { + // tables if: + // 1. `LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT` is false, or + // 2. It's a CTAS and `conf.convertCTAS` is true. + val createHiveTableByDefault = conf.getConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT) + if (!createHiveTableByDefault || (ctas && conf.convertCTAS)) { (nonHiveStorageFormat, conf.defaultDataSourceName) } else { + logWarning("A Hive serde table will be created as there is no table provider " + + s"specified. You can set ${SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT.key} to false " + + "so that native data source table will be created instead.") (defaultHiveStorage, DDLUtils.HIVE_PROVIDER) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala index 1b8d69ffb7..4b98acd16f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala @@ -31,16 +31,21 @@ import org.apache.spark.util.ListenerBus * Spark listener bus, so that it can receive [[StreamingQueryListener.Event]]s and dispatch them * to StreamingQueryListeners. * - * Note that each bus and its registered listeners are associated with a single SparkSession + * Note 1: Each bus and its registered listeners are associated with a single SparkSession * and StreamingQueryManager. So this bus will dispatch events to registered listeners for only * those queries that were started in the associated SparkSession. + * + * Note 2: To rebuild Structured Streaming UI in SHS, this bus will be registered into + * [[org.apache.spark.scheduler.ReplayListenerBus]]. We check `sparkListenerBus` defined or not to + * determine how to process [[StreamingQueryListener.Event]]. If false, it means this bus is used to + * replay all streaming query event from eventLog. */ -class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) +class StreamingQueryListenerBus(sparkListenerBus: Option[LiveListenerBus]) extends SparkListener with ListenerBus[StreamingQueryListener, StreamingQueryListener.Event] { import StreamingQueryListener._ - sparkListenerBus.addToQueue(this, StreamingQueryListenerBus.STREAM_EVENT_QUERY) + sparkListenerBus.foreach(_.addToQueue(this, StreamingQueryListenerBus.STREAM_EVENT_QUERY)) /** * RunIds of active queries whose events are supposed to be forwarded by this ListenerBus @@ -67,11 +72,11 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) event match { case s: QueryStartedEvent => activeQueryRunIds.synchronized { activeQueryRunIds += s.runId } - sparkListenerBus.post(s) + sparkListenerBus.foreach(bus => bus.post(s)) // post to local listeners to trigger callbacks postToAll(s) case _ => - sparkListenerBus.post(event) + sparkListenerBus.foreach(bus => bus.post(event)) } } @@ -95,7 +100,11 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) // synchronously and the ones attached to LiveListenerBus asynchronously. Therefore, // we need to ignore QueryStartedEvent if this method is called within SparkListenerBus // thread - if (!LiveListenerBus.withinListenerThread.value || !e.isInstanceOf[QueryStartedEvent]) { + // + // When loaded by Spark History Server, we should process all event coming from replay + // listener bus. + if (sparkListenerBus.isEmpty || !LiveListenerBus.withinListenerThread.value || + !e.isInstanceOf[QueryStartedEvent]) { postToAll(e) } case _ => @@ -110,7 +119,10 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) listener: StreamingQueryListener, event: StreamingQueryListener.Event): Unit = { def shouldReport(runId: UUID): Boolean = { - activeQueryRunIds.synchronized { activeQueryRunIds.contains(runId) } + // When loaded by Spark History Server, we should process all event coming from replay + // listener bus. + sparkListenerBus.isEmpty || + activeQueryRunIds.synchronized { activeQueryRunIds.contains(runId) } } event match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala new file mode 100644 index 0000000000..a127fa59b7 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala @@ -0,0 +1,43 @@ +/* + * 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.ui + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SparkListener +import org.apache.spark.sql.execution.streaming.StreamingQueryListenerBus +import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, StreamingQueryTab} +import org.apache.spark.status.{AppHistoryServerPlugin, ElementTrackingStore} +import org.apache.spark.ui.SparkUI + +class StreamingQueryHistoryServerPlugin extends AppHistoryServerPlugin { + + override def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener] = { + val listenerBus = new StreamingQueryListenerBus(None) + listenerBus.addListener(new StreamingQueryStatusListener(conf, store)) + Seq(listenerBus) + } + + override def setupUI(ui: SparkUI): Unit = { + val streamingQueryStatusStore = new StreamingQueryStatusStore(ui.store.store) + if (streamingQueryStatusStore.allQueryUIData.nonEmpty) { + new StreamingQueryTab(streamingQueryStatusStore, ui) + } + } + + override def displayOrder: Int = 1 +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala new file mode 100644 index 0000000000..9eb14a6a63 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryStatusStore.scala @@ -0,0 +1,53 @@ +/* + * 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.ui + +import java.util.UUID + +import org.apache.spark.sql.streaming.ui.{StreamingQueryData, StreamingQueryProgressWrapper, StreamingQueryUIData} +import org.apache.spark.status.KVUtils +import org.apache.spark.util.kvstore.KVStore + +/** + * Provides a view of a KVStore with methods that make it easy to query Streaming Query state. + * There's no state kept in this class, so it's ok to have multiple instances of it in an + * application. + */ +class StreamingQueryStatusStore(store: KVStore) { + + def allQueryUIData: Seq[StreamingQueryUIData] = { + val view = store.view(classOf[StreamingQueryData]).index("startTimestamp").first(0L) + KVUtils.viewToSeq(view, Int.MaxValue)(_ => true).map(makeUIData) + } + + // visible for test + private[sql] def getQueryProgressData(runId: UUID): Seq[StreamingQueryProgressWrapper] = { + val view = store.view(classOf[StreamingQueryProgressWrapper]) + .index("runId").first(runId.toString).last(runId.toString) + KVUtils.viewToSeq(view, Int.MaxValue)(_ => true) + } + + private def makeUIData(summary: StreamingQueryData): StreamingQueryUIData = { + val runId = summary.runId.toString + val view = store.view(classOf[StreamingQueryProgressWrapper]) + .index("runId").first(runId).last(runId) + val recentProgress = KVUtils.viewToSeq(view, Int.MaxValue)(_ => true) + .map(_.progress).sortBy(_.timestamp).toArray + StreamingQueryUIData(summary, recentProgress) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 89aceacac6..ea430db9f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -34,7 +34,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.streaming.StreamExecution -import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} +import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab, StreamingQueryStatusStore} import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, StreamingQueryTab} import org.apache.spark.status.ElementTrackingStore @@ -111,9 +111,9 @@ private[sql] class SharedState( lazy val streamingQueryStatusListener: Option[StreamingQueryStatusListener] = { sparkContext.ui.flatMap { ui => if (conf.get(STREAMING_UI_ENABLED)) { - val statusListener = new StreamingQueryStatusListener(conf) - new StreamingQueryTab(statusListener, ui) - Some(statusListener) + val kvStore = sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore] + new StreamingQueryTab(new StreamingQueryStatusStore(kvStore), ui) + Some(new StreamingQueryStatusListener(conf, kvStore)) } else { None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index d67e175c24..9e3599712f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -304,7 +304,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * @since 3.1.0 */ @throws[TimeoutException] - def saveAsTable(tableName: String): StreamingQuery = { + def toTable(tableName: String): StreamingQuery = { this.source = SOURCE_NAME_TABLE this.tableName = tableName startInternal(None) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index ffdbe9d4e4..b66037d009 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -49,7 +49,8 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo private[sql] val stateStoreCoordinator = StateStoreCoordinatorRef.forDriver(sparkSession.sparkContext.env) - private val listenerBus = new StreamingQueryListenerBus(sparkSession.sparkContext.listenerBus) + private val listenerBus = + new StreamingQueryListenerBus(Some(sparkSession.sparkContext.listenerBus)) @GuardedBy("activeQueriesSharedLock") private val activeQueries = new mutable.HashMap[UUID, StreamingQuery] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala index b98fdf16ee..96e498991e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala @@ -40,8 +40,8 @@ private[ui] class StreamingQueryPage(parent: StreamingQueryTab) } private def generateStreamingQueryTable(request: HttpServletRequest): Seq[Node] = { - val (activeQueries, inactiveQueries) = parent.statusListener.allQueryStatus - .partition(_.isActive) + val (activeQueries, inactiveQueries) = + parent.store.allQueryUIData.partition(_.summary.isActive) val content = mutable.ListBuffer[Node]() // show active queries table only if there is at least one active query @@ -176,7 +176,7 @@ class StreamingQueryPagedTable( val streamingQuery = query.streamingUIData val statisticsLink = "%s/%s/statistics?id=%s" .format(SparkUIUtils.prependBaseUri(request, parent.basePath), parent.prefix, - streamingQuery.runId) + streamingQuery.summary.runId) def details(detail: Any): Seq[Node] = { if (isActive) { @@ -194,14 +194,14 @@ class StreamingQueryPagedTable( {UIUtils.getQueryName(streamingQuery)} {UIUtils.getQueryStatus(streamingQuery)} - {streamingQuery.id} - {streamingQuery.runId} - {SparkUIUtils.formatDate(streamingQuery.startTimestamp)} + {streamingQuery.summary.id} + {streamingQuery.summary.runId} + {SparkUIUtils.formatDate(streamingQuery.summary.startTimestamp)} {SparkUIUtils.formatDurationVerbose(query.duration)} {withNoProgress(streamingQuery, {query.avgInput.formatted("%.2f")}, "NaN")} {withNoProgress(streamingQuery, {query.avgProcess.formatted("%.2f")}, "NaN")} {withNoProgress(streamingQuery, {streamingQuery.lastProgress.batchId}, "NaN")} - {details(streamingQuery.exception.getOrElse("-"))} + {details(streamingQuery.summary.exception.getOrElse("-"))} } } @@ -222,32 +222,32 @@ class StreamingQueryDataSource(uiData: Seq[StreamingQueryUIData], sortColumn: St override def sliceData(from: Int, to: Int): Seq[StructuredStreamingRow] = data.slice(from, to) - private def streamingRow(query: StreamingQueryUIData): StructuredStreamingRow = { + private def streamingRow(uiData: StreamingQueryUIData): StructuredStreamingRow = { val duration = if (isActive) { - System.currentTimeMillis() - query.startTimestamp + System.currentTimeMillis() - uiData.summary.startTimestamp } else { - withNoProgress(query, { - val endTimeMs = query.lastProgress.timestamp - parseProgressTimestamp(endTimeMs) - query.startTimestamp + withNoProgress(uiData, { + val endTimeMs = uiData.lastProgress.timestamp + parseProgressTimestamp(endTimeMs) - uiData.summary.startTimestamp }, 0) } - val avgInput = (query.recentProgress.map(p => withNumberInvalid(p.inputRowsPerSecond)).sum / - query.recentProgress.length) + val avgInput = (uiData.recentProgress.map(p => withNumberInvalid(p.inputRowsPerSecond)).sum / + uiData.recentProgress.length) - val avgProcess = (query.recentProgress.map(p => - withNumberInvalid(p.processedRowsPerSecond)).sum / query.recentProgress.length) + val avgProcess = (uiData.recentProgress.map(p => + withNumberInvalid(p.processedRowsPerSecond)).sum / uiData.recentProgress.length) - StructuredStreamingRow(duration, avgInput, avgProcess, query) + StructuredStreamingRow(duration, avgInput, avgProcess, uiData) } private def ordering(sortColumn: String, desc: Boolean): Ordering[StructuredStreamingRow] = { val ordering: Ordering[StructuredStreamingRow] = sortColumn match { - case "Name" => Ordering.by(q => UIUtils.getQueryName(q.streamingUIData)) - case "Status" => Ordering.by(q => UIUtils.getQueryStatus(q.streamingUIData)) - case "ID" => Ordering.by(_.streamingUIData.id) - case "Run ID" => Ordering.by(_.streamingUIData.runId) - case "Start Time" => Ordering.by(_.streamingUIData.startTimestamp) + case "Name" => Ordering.by(row => UIUtils.getQueryName(row.streamingUIData)) + case "Status" => Ordering.by(row => UIUtils.getQueryStatus(row.streamingUIData)) + case "ID" => Ordering.by(_.streamingUIData.summary.id) + case "Run ID" => Ordering.by(_.streamingUIData.summary.runId) + case "Start Time" => Ordering.by(_.streamingUIData.summary.startTimestamp) case "Duration" => Ordering.by(_.duration) case "Avg Input /sec" => Ordering.by(_.avgInput) case "Avg Process /sec" => Ordering.by(_.avgProcess) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala index 24709ba470..97691d9d7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala @@ -58,8 +58,8 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab) val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val query = parent.statusListener.allQueryStatus.find { case q => - q.runId.equals(UUID.fromString(parameterId)) + val query = parent.store.allQueryUIData.find { uiData => + uiData.summary.runId.equals(UUID.fromString(parameterId)) }.getOrElse(throw new IllegalArgumentException(s"Failed to find streaming query $parameterId")) val resources = generateLoadResources(request) @@ -109,34 +109,35 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab) } - def generateBasicInfo(query: StreamingQueryUIData): Seq[Node] = { - val duration = if (query.isActive) { - SparkUIUtils.formatDurationVerbose(System.currentTimeMillis() - query.startTimestamp) + def generateBasicInfo(uiData: StreamingQueryUIData): Seq[Node] = { + val duration = if (uiData.summary.isActive) { + val durationMs = System.currentTimeMillis() - uiData.summary.startTimestamp + SparkUIUtils.formatDurationVerbose(durationMs) } else { - withNoProgress(query, { - val end = query.lastProgress.timestamp - val start = query.recentProgress.head.timestamp + withNoProgress(uiData, { + val end = uiData.lastProgress.timestamp + val start = uiData.recentProgress.head.timestamp SparkUIUtils.formatDurationVerbose( parseProgressTimestamp(end) - parseProgressTimestamp(start)) }, "-") } - val name = UIUtils.getQueryName(query) - val numBatches = withNoProgress(query, { query.lastProgress.batchId + 1L }, 0) + val name = UIUtils.getQueryName(uiData) + val numBatches = withNoProgress(uiData, { uiData.lastProgress.batchId + 1L }, 0)
Running batches for {duration} since - {SparkUIUtils.formatDate(query.startTimestamp)} + {SparkUIUtils.formatDate(uiData.summary.startTimestamp)} ({numBatches} completed batches)

Name: {name}
-
Id: {query.id}
-
RunId: {query.runId}
+
Id: {uiData.summary.id}
+
RunId: {uiData.summary.runId}

} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala index e331083b30..fdd3754344 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala @@ -20,102 +20,144 @@ package org.apache.spark.sql.streaming.ui import java.util.UUID import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConverters._ import scala.collection.mutable +import com.fasterxml.jackson.annotation.JsonIgnore + import org.apache.spark.SparkConf import org.apache.spark.sql.internal.StaticSQLConf import org.apache.spark.sql.streaming.{StreamingQueryListener, StreamingQueryProgress} +import org.apache.spark.sql.streaming.ui.StreamingQueryProgressWrapper._ import org.apache.spark.sql.streaming.ui.UIUtils.parseProgressTimestamp +import org.apache.spark.status.{ElementTrackingStore, KVUtils} +import org.apache.spark.status.KVUtils.KVIndexParam +import org.apache.spark.util.kvstore.KVIndex /** * A customized StreamingQueryListener used in structured streaming UI, which contains all * UI data for both active and inactive query. - * TODO: Add support for history server. */ -private[sql] class StreamingQueryStatusListener(conf: SparkConf) extends StreamingQueryListener { - - /** - * We use runId as the key here instead of id in active query status map, - * because the runId is unique for every started query, even it its a restart. - */ - private[ui] val activeQueryStatus = new ConcurrentHashMap[UUID, StreamingQueryUIData]() - private[ui] val inactiveQueryStatus = new mutable.Queue[StreamingQueryUIData]() +private[sql] class StreamingQueryStatusListener( + conf: SparkConf, + store: ElementTrackingStore) extends StreamingQueryListener { private val streamingProgressRetention = conf.get(StaticSQLConf.STREAMING_UI_RETAINED_PROGRESS_UPDATES) private val inactiveQueryStatusRetention = conf.get(StaticSQLConf.STREAMING_UI_RETAINED_QUERIES) + store.addTrigger(classOf[StreamingQueryData], inactiveQueryStatusRetention) { count => + cleanupInactiveQueries(count) + } + + // Events from the same query run will never be processed concurrently, so it's safe to + // access `progressIds` without any protection. + private val queryToProgress = new ConcurrentHashMap[UUID, mutable.Queue[String]]() + + private def cleanupInactiveQueries(count: Long): Unit = { + val view = store.view(classOf[StreamingQueryData]).index("active").first(false).last(false) + val inactiveQueries = KVUtils.viewToSeq(view, Int.MaxValue)(_ => true) + val numInactiveQueries = inactiveQueries.size + if (numInactiveQueries <= inactiveQueryStatusRetention) { + return + } + val toDelete = inactiveQueries.sortBy(_.endTimestamp.get) + .take(numInactiveQueries - inactiveQueryStatusRetention) + val runIds = toDelete.map { e => + store.delete(e.getClass, e.runId) + e.runId.toString + } + // Delete wrappers in one pass, as deleting them for each summary is slow + store.removeAllByIndexValues(classOf[StreamingQueryProgressWrapper], "runId", runIds) + } + override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = { val startTimestamp = parseProgressTimestamp(event.timestamp) - activeQueryStatus.putIfAbsent(event.runId, - new StreamingQueryUIData(event.name, event.id, event.runId, startTimestamp)) + store.write(new StreamingQueryData( + event.name, + event.id, + event.runId, + isActive = true, + None, + startTimestamp + ), checkTriggers = true) } override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = { - val batchTimestamp = parseProgressTimestamp(event.progress.timestamp) - val queryStatus = activeQueryStatus.getOrDefault( - event.progress.runId, - new StreamingQueryUIData(event.progress.name, event.progress.id, event.progress.runId, - batchTimestamp)) - queryStatus.updateProcess(event.progress, streamingProgressRetention) - } - - override def onQueryTerminated( - event: StreamingQueryListener.QueryTerminatedEvent): Unit = synchronized { - val queryStatus = activeQueryStatus.remove(event.runId) - if (queryStatus != null) { - queryStatus.queryTerminated(event) - inactiveQueryStatus += queryStatus - while (inactiveQueryStatus.length >= inactiveQueryStatusRetention) { - inactiveQueryStatus.dequeue() - } + val runId = event.progress.runId + val batchId = event.progress.batchId + val timestamp = event.progress.timestamp + if (!queryToProgress.containsKey(runId)) { + queryToProgress.put(runId, mutable.Queue.empty[String]) + } + val progressIds = queryToProgress.get(runId) + progressIds.enqueue(getUniqueId(runId, batchId, timestamp)) + store.write(new StreamingQueryProgressWrapper(event.progress)) + while (progressIds.length > streamingProgressRetention) { + val uniqueId = progressIds.dequeue + store.delete(classOf[StreamingQueryProgressWrapper], uniqueId) } } - def allQueryStatus: Seq[StreamingQueryUIData] = synchronized { - activeQueryStatus.values().asScala.toSeq ++ inactiveQueryStatus + override def onQueryTerminated( + event: StreamingQueryListener.QueryTerminatedEvent): Unit = { + val querySummary = store.read(classOf[StreamingQueryData], event.runId) + val curTime = System.currentTimeMillis() + store.write(new StreamingQueryData( + querySummary.name, + querySummary.id, + querySummary.runId, + isActive = false, + querySummary.exception, + querySummary.startTimestamp, + Some(curTime) + ), checkTriggers = true) + queryToProgress.remove(event.runId) } } +private[sql] class StreamingQueryData( + val name: String, + val id: UUID, + @KVIndexParam val runId: UUID, + @KVIndexParam("active") val isActive: Boolean, + val exception: Option[String], + @KVIndexParam("startTimestamp") val startTimestamp: Long, + val endTimestamp: Option[Long] = None) + /** * This class contains all message related to UI display, each instance corresponds to a single * [[org.apache.spark.sql.streaming.StreamingQuery]]. */ -private[ui] class StreamingQueryUIData( - val name: String, - val id: UUID, - val runId: UUID, - val startTimestamp: Long) { - - /** Holds the most recent query progress updates. */ - private val progressBuffer = new mutable.Queue[StreamingQueryProgress]() - - private var _isActive = true - private var _exception: Option[String] = None - - def isActive: Boolean = synchronized { _isActive } - - def exception: Option[String] = synchronized { _exception } - - def queryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = synchronized { - _isActive = false - _exception = event.exception - } - - def updateProcess( - newProgress: StreamingQueryProgress, retentionNum: Int): Unit = progressBuffer.synchronized { - progressBuffer += newProgress - while (progressBuffer.length >= retentionNum) { - progressBuffer.dequeue() +private[sql] case class StreamingQueryUIData( + summary: StreamingQueryData, + recentProgress: Array[StreamingQueryProgress]) { + + def lastProgress: StreamingQueryProgress = { + if (recentProgress.nonEmpty) { + recentProgress.last + } else { + null } } +} - def recentProgress: Array[StreamingQueryProgress] = progressBuffer.synchronized { - progressBuffer.toArray - } +private[sql] class StreamingQueryProgressWrapper(val progress: StreamingQueryProgress) { + @JsonIgnore @KVIndex + private val uniqueId: String = getUniqueId(progress.runId, progress.batchId, progress.timestamp) - def lastProgress: StreamingQueryProgress = progressBuffer.synchronized { - progressBuffer.lastOption.orNull + @JsonIgnore @KVIndex("runId") + private def runIdIndex: String = progress.runId.toString +} + +private[sql] object StreamingQueryProgressWrapper { + /** + * Adding `timestamp` into unique id to support reporting `empty` query progress + * in which no data comes but with the same batchId. + */ + def getUniqueId( + runId: UUID, + batchId: Long, + timestamp: String): String = { + s"${runId}_${batchId}_$timestamp" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala index bb097ffc06..65cad8f06c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.streaming.ui import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore import org.apache.spark.ui.{SparkUI, SparkUITab} private[sql] class StreamingQueryTab( - val statusListener: StreamingQueryStatusListener, + val store: StreamingQueryStatusStore, sparkUI: SparkUI) extends SparkUITab(sparkUI, "StreamingQuery") with Logging { override val name = "Structured Streaming" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala index 1f7e65dede..88a110fa9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala @@ -46,19 +46,19 @@ private[ui] object UIUtils { } } - def getQueryName(query: StreamingQueryUIData): String = { - if (query.name == null || query.name.isEmpty) { + def getQueryName(uiData: StreamingQueryUIData): String = { + if (uiData.summary.name == null || uiData.summary.name.isEmpty) { "" } else { - query.name + uiData.summary.name } } - def getQueryStatus(query: StreamingQueryUIData): String = { - if (query.isActive) { + def getQueryStatus(uiData: StreamingQueryUIData): String = { + if (uiData.summary.isActive) { "RUNNING" } else { - query.exception.map(_ => "FAILED").getOrElse("FINISHED") + uiData.summary.exception.map(_ => "FAILED").getOrElse("FINISHED") } } diff --git a/sql/core/src/test/resources/spark-events/local-1596020211915 b/sql/core/src/test/resources/spark-events/local-1596020211915 new file mode 100644 index 0000000000..ff34bbc16e --- /dev/null +++ b/sql/core/src/test/resources/spark-events/local-1596020211915 @@ -0,0 +1,160 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.1.0-SNAPSHOT"} +{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":0,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":1,"Discovery Script":"","Vendor":""},"memory":{"Resource Name":"memory","Amount":1024,"Discovery Script":"","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":1.0}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1596020212090,"Executor ID":"driver","Executor Info":{"Host":"iZbp19vpr16ix621sdw476Z","Total Cores":4,"Log Urls":{},"Attributes":{},"Resources":{},"Resource Profile Id":0}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Port":39845},"Maximum Memory":384093388,"Timestamp":1596020212109,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre","Java Version":"1.8.0_252 (Oracle Corporation)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.driver.port":"46309","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","spark.app.name":"StructuredKafkaWordCount","spark.scheduler.mode":"FIFO","spark.submit.pyFiles":"","spark.executor.id":"driver","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"},"Hadoop Properties":{"yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms":"1000","hadoop.security.kms.client.timeout":"60","hadoop.http.authentication.kerberos.principal":"HTTP/_HOST@LOCALHOST","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern":"^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$","tfile.fs.output.buffer.size":"262144","yarn.app.mapreduce.am.job.task.listener.thread-count":"30","hadoop.security.groups.cache.background.reload.threads":"3","yarn.resourcemanager.webapp.cross-origin.enabled":"false","fs.AbstractFileSystem.ftp.impl":"org.apache.hadoop.fs.ftp.FtpFs","fs.s3.block.size":"67108864","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","fs.s3.buffer.dir":"${hadoop.tmp.dir}/s3","mapreduce.job.acl-view-job":" ","mapreduce.jobhistory.loadedjobs.cache.size":"5","mapreduce.input.fileinputformat.split.minsize":"0","yarn.resourcemanager.container.liveness-monitor.interval-ms":"600000","yarn.resourcemanager.client.thread-count":"50","io.seqfile.compress.blocksize":"1000000","yarn.sharedcache.checksum.algo.impl":"org.apache.hadoop.yarn.sharedcache.ChecksumSHA256Impl","yarn.nodemanager.amrmproxy.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.nodemanager.amrmproxy.DefaultRequestInterceptor","yarn.timeline-service.entity-group-fs-store.leveldb-cache-read-cache-size":"10485760","mapreduce.reduce.shuffle.fetch.retry.interval-ms":"1000","mapreduce.task.profile.maps":"0-2","yarn.scheduler.include-port-in-node-name":"false","yarn.nodemanager.admin-env":"MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX","yarn.resourcemanager.node-removal-untracked.timeout-ms":"60000","mapreduce.am.max-attempts":"2","hadoop.security.kms.client.failover.sleep.base.millis":"100","mapreduce.jobhistory.webapp.https.address":"0.0.0.0:19890","yarn.node-labels.fs-store.impl.class":"org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore","fs.trash.checkpoint.interval":"0","mapreduce.job.map.output.collector.class":"org.apache.hadoop.mapred.MapTask$MapOutputBuffer","yarn.resourcemanager.node-ip-cache.expiry-interval-secs":"-1","hadoop.http.authentication.signature.secret.file":"*********(redacted)","hadoop.jetty.logs.serve.aliases":"true","yarn.timeline-service.handler-thread-count":"10","yarn.resourcemanager.max-completed-applications":"10000","yarn.resourcemanager.system-metrics-publisher.enabled":"false","yarn.sharedcache.webapp.address":"0.0.0.0:8788","yarn.resourcemanager.delegation.token.renew-interval":"*********(redacted)","yarn.sharedcache.nm.uploader.replication.factor":"10","hadoop.security.groups.negative-cache.secs":"30","yarn.app.mapreduce.task.container.log.backups":"0","mapreduce.reduce.skip.proc-count.auto-incr":"true","hadoop.security.group.mapping.ldap.posix.attr.gid.name":"gidNumber","ipc.client.fallback-to-simple-auth-allowed":"false","yarn.client.failover-proxy-provider":"org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider","yarn.timeline-service.http-authentication.simple.anonymous.allowed":"true","ha.health-monitor.check-interval.ms":"1000","yarn.acl.reservation-enable":"false","yarn.resourcemanager.store.class":"org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore","yarn.app.mapreduce.am.hard-kill-timeout-ms":"10000","yarn.nodemanager.container-metrics.enable":"true","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.nodemanager.docker-container-executor.exec-name":"/usr/bin/docker","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","mapred.child.java.opts":"-Xmx200m","hadoop.common.configuration.version":"0.23.0","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","hadoop.registry.zk.retry.ceiling.ms":"60000","yarn.sharedcache.store.in-memory.initial-delay-mins":"10","mapreduce.jobhistory.principal":"jhs/_HOST@REALM.TLD","mapreduce.map.skip.proc-count.auto-incr":"true","mapreduce.task.profile.reduces":"0-2","yarn.timeline-service.webapp.https.address":"${yarn.timeline-service.hostname}:8190","yarn.resourcemanager.scheduler.address":"${yarn.resourcemanager.hostname}:8030","yarn.node-labels.enabled":"false","yarn.resourcemanager.webapp.ui-actions.enabled":"true","mapreduce.task.timeout":"600000","yarn.sharedcache.client-server.thread-count":"50","hadoop.security.crypto.cipher.suite":"AES/CTR/NoPadding","yarn.resourcemanager.connect.max-wait.ms":"900000","fs.defaultFS":"file:///","yarn.minicluster.use-rpc":"false","fs.har.impl.disable.cache":"true","io.compression.codec.bzip2.library":"system-native","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","mapreduce.jobhistory.address":"0.0.0.0:10020","yarn.resourcemanager.nm-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.is.minicluster":"false","yarn.nodemanager.address":"${yarn.nodemanager.hostname}:0","fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A","mapreduce.task.combine.progress.records":"10000","yarn.resourcemanager.am.max-attempts":"2","yarn.nodemanager.linux-container-executor.cgroups.hierarchy":"/hadoop-yarn","ipc.server.log.slow.rpc":"false","yarn.resourcemanager.node-labels.provider.fetch-interval-ms":"1800000","yarn.nodemanager.webapp.cross-origin.enabled":"false","yarn.app.mapreduce.am.job.committer.cancel-timeout":"60000","ftp.bytes-per-checksum":"512","yarn.nodemanager.resource.memory-mb":"-1","fs.s3a.fast.upload.active.blocks":"4","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","yarn.resourcemanager.fs.state-store.num-retries":"0","yarn.resourcemanager.nodemanager-connect-retries":"10","hadoop.security.kms.client.encrypted.key.cache.low-watermark":"0.3f","yarn.timeline-service.client.max-retries":"30","dfs.ha.fencing.ssh.connect-timeout":"30000","yarn.log-aggregation-enable":"false","mapreduce.reduce.markreset.buffer.percent":"0.0","fs.AbstractFileSystem.viewfs.impl":"org.apache.hadoop.fs.viewfs.ViewFs","mapreduce.task.io.sort.factor":"10","yarn.nodemanager.amrmproxy.client.thread-count":"25","ha.failover-controller.new-active.rpc-timeout.ms":"60000","yarn.nodemanager.container-localizer.java.opts":"-Xmx256m","mapreduce.jobhistory.datestring.cache.size":"200000","mapreduce.job.acl-modify-job":" ","yarn.nodemanager.windows-container.memory-limit.enabled":"false","yarn.timeline-service.webapp.address":"${yarn.timeline-service.hostname}:8188","yarn.app.mapreduce.am.job.committer.commit-window":"10000","yarn.nodemanager.container-manager.thread-count":"20","yarn.minicluster.fixed.ports":"false","yarn.cluster.max-application-priority":"0","yarn.timeline-service.ttl-enable":"true","mapreduce.jobhistory.recovery.store.fs.uri":"${hadoop.tmp.dir}/mapred/history/recoverystore","hadoop.caller.context.signature.max.size":"40","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","mapreduce.job.speculative.slowtaskthreshold":"1.0","io.serializations":"org.apache.hadoop.io.serializer.WritableSerialization, org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.hadoop.io.serializer.avro.AvroReflectSerialization","hadoop.security.kms.client.failover.sleep.max.millis":"2000","hadoop.security.group.mapping.ldap.directory.search.timeout":"10000","fs.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","yarn.nodemanager.local-cache.max-files-per-directory":"8192","hadoop.http.cross-origin.enabled":"false","mapreduce.map.sort.spill.percent":"0.80","yarn.timeline-service.entity-group-fs-store.scan-interval-seconds":"60","yarn.timeline-service.client.best-effort":"false","yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled":"*********(redacted)","hadoop.security.group.mapping.ldap.posix.attr.uid.name":"uidNumber","fs.AbstractFileSystem.swebhdfs.impl":"org.apache.hadoop.fs.SWebHdfs","mapreduce.ifile.readahead":"true","yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms":"300000","hadoop.security.kms.client.encrypted.key.cache.num.refill.threads":"2","yarn.resourcemanager.scheduler.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler","yarn.app.mapreduce.am.command-opts":"-Xmx1024m","mapreduce.cluster.local.dir":"${hadoop.tmp.dir}/mapred/local","io.mapfile.bloom.error.rate":"0.005","yarn.nodemanager.runtime.linux.allowed-runtimes":"default","yarn.sharedcache.store.class":"org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore","ha.failover-controller.graceful-fence.rpc-timeout.ms":"5000","ftp.replication":"3","hadoop.security.uid.cache.secs":"14400","mapreduce.job.maxtaskfailures.per.tracker":"3","io.skip.checksum.errors":"false","yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts":"3","fs.s3a.connection.timeout":"200000","mapreduce.job.max.split.locations":"10","hadoop.registry.zk.session.timeout.ms":"60000","mapreduce.jvm.system-properties-to-log":"os.name,os.version,java.home,java.runtime.version,java.vendor,java.version,java.vm.name,java.class.path,java.io.tmpdir,user.dir,user.name","yarn.timeline-service.entity-group-fs-store.active-dir":"/tmp/entity-file-history/active","mapreduce.shuffle.transfer.buffer.size":"131072","yarn.timeline-service.client.retry-interval-ms":"1000","yarn.http.policy":"HTTP_ONLY","fs.s3a.socket.send.buffer":"8192","yarn.sharedcache.uploader.server.address":"0.0.0.0:8046","hadoop.http.authentication.token.validity":"*********(redacted)","mapreduce.shuffle.max.connections":"0","yarn.minicluster.yarn.nodemanager.resource.memory-mb":"4096","mapreduce.job.emit-timeline-data":"false","yarn.nodemanager.resource.system-reserved-memory-mb":"-1","hadoop.kerberos.min.seconds.before.relogin":"60","mapreduce.jobhistory.move.thread-count":"3","yarn.resourcemanager.admin.client.thread-count":"1","yarn.dispatcher.drain-events.timeout":"300000","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","mapreduce.shuffle.port":"13562","yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory":"10","yarn.nodemanager.health-checker.interval-ms":"600000","ftp.blocksize":"67108864","yarn.nodemanager.log-container-debug-info.enabled":"false","yarn.client.max-cached-nodemanagers-proxies":"0","yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms":"20","yarn.nodemanager.delete.debug-delay-sec":"0","yarn.nodemanager.pmem-check-enabled":"true","yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage":"90.0","mapreduce.app-submission.cross-platform":"false","yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms":"10000","hadoop.security.groups.cache.secs":"300","yarn.resourcemanager.zk-retry-interval-ms":"1000","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","hadoop.security.authorization":"false","mapreduce.job.complete.cancel.delegation.tokens":"*********(redacted)","fs.s3a.paging.maximum":"5000","nfs.exports.allowed.hosts":"* rw","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","s3native.replication":"3","hadoop.security.group.mapping.ldap.ssl":"false","yarn.client.application-client-protocol.poll-interval-ms":"200","ha.zookeeper.parent-znode":"/hadoop-ha","yarn.nodemanager.log-aggregation.policy.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AllContainerLogAggregationPolicy","mapreduce.reduce.shuffle.merge.percent":"0.66","hadoop.security.group.mapping.ldap.search.filter.group":"(objectClass=group)","yarn.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","yarn.resourcemanager.address":"${yarn.resourcemanager.hostname}:8032","ipc.client.ping":"true","mapreduce.task.local-fs.write-limit.bytes":"-1","fs.adl.oauth2.access.token.provider.type":"*********(redacted)","mapreduce.shuffle.ssl.file.buffer.size":"65536","yarn.resourcemanager.ha.automatic-failover.embedded":"true","hadoop.ssl.enabled":"false","fs.s3a.multipart.purge":"false","mapreduce.job.end-notification.max.attempts":"5","mapreduce.output.fileoutputformat.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled":"false","ha.health-monitor.connect-retry-interval.ms":"1000","yarn.nodemanager.keytab":"/etc/krb5.keytab","mapreduce.jobhistory.keytab":"/etc/security/keytab/jhs.service.keytab","fs.s3a.threads.max":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","mapreduce.cluster.temp.dir":"${hadoop.tmp.dir}/mapred/temp","s3.replication":"3","yarn.nodemanager.node-labels.resync-interval-ms":"120000","hadoop.tmp.dir":"/tmp/hadoop-${user.name}","mapreduce.job.maps":"2","mapreduce.job.end-notification.max.retry.interval":"5000","yarn.log-aggregation.retain-check-interval-seconds":"-1","yarn.resourcemanager.resource-tracker.client.thread-count":"50","yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size":"10000","yarn.resourcemanager.ha.automatic-failover.zk-base-path":"/yarn-leader-election","io.seqfile.local.dir":"${hadoop.tmp.dir}/io/local","mapreduce.client.submit.file.replication":"10","mapreduce.jobhistory.minicluster.fixed.ports":"false","fs.s3a.multipart.threshold":"2147483647","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","yarn.resourcemanager.zk-acl":"world:anyone:rwcda","ipc.client.idlethreshold":"4000","yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage":"false","mapreduce.reduce.input.buffer.percent":"0.0","yarn.nodemanager.amrmproxy.enable":"false","fs.ftp.host.port":"21","ipc.ping.interval":"60000","yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size":"10","yarn.resourcemanager.admin.address":"${yarn.resourcemanager.hostname}:8033","file.client-write-packet-size":"65536","ipc.client.kill.max":"10","mapreduce.reduce.speculative":"true","mapreduce.local.clientfactory.class.name":"org.apache.hadoop.mapred.LocalClientFactory","mapreduce.job.reducer.unconditional-preempt.delay.sec":"300","yarn.nodemanager.disk-health-checker.interval-ms":"120000","yarn.nodemanager.log.deletion-threads-count":"4","ipc.client.connection.maxidletime":"10000","mapreduce.task.io.sort.mb":"100","yarn.nodemanager.localizer.client.thread-count":"5","yarn.sharedcache.admin.address":"0.0.0.0:8047","yarn.nodemanager.localizer.cache.cleanup.interval-ms":"600000","hadoop.security.crypto.codec.classes.aes.ctr.nopadding":"org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.crypto.JceAesCtrCryptoCodec","fs.s3a.connection.ssl.enabled":"true","yarn.nodemanager.process-kill-wait.ms":"2000","mapreduce.job.hdfs-servers":"${fs.defaultFS}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","fs.s3.sleepTimeSeconds":"10","fs.s3a.multiobjectdelete.enable":"true","yarn.sharedcache.cleaner.resource-sleep-ms":"0","yarn.nodemanager.disk-health-checker.min-healthy-disks":"0.25","hadoop.shell.missing.defaultFs.warning":"false","io.file.buffer.size":"65536","hadoop.security.group.mapping.ldap.search.attr.member":"member","hadoop.security.random.device.file.path":"/dev/urandom","hadoop.security.sensitive-config-keys":"*********(redacted)","hadoop.rpc.socket.factory.class.default":"org.apache.hadoop.net.StandardSocketFactory","yarn.intermediate-data-encryption.enable":"false","yarn.resourcemanager.connect.retry-interval.ms":"30000","yarn.scheduler.minimum-allocation-mb":"1024","yarn.app.mapreduce.am.staging-dir":"/tmp/hadoop-yarn/staging","mapreduce.reduce.shuffle.read.timeout":"180000","hadoop.http.cross-origin.max-age":"1800","fs.s3a.connection.establish.timeout":"5000","mapreduce.job.running.map.limit":"0","yarn.minicluster.control-resource-monitoring":"false","hadoop.ssl.require.client.cert":"false","hadoop.kerberos.kinit.command":"kinit","mapreduce.reduce.log.level":"INFO","hadoop.security.dns.log-slow-lookups.threshold.ms":"1000","mapreduce.job.ubertask.enable":"false","hadoop.caller.context.enabled":"false","yarn.nodemanager.vmem-pmem-ratio":"2.1","hadoop.rpc.protection":"authentication","ha.health-monitor.rpc-timeout.ms":"45000","s3native.stream-buffer-size":"4096","yarn.nodemanager.remote-app-log-dir":"/tmp/logs","yarn.nodemanager.resource.pcores-vcores-multiplier":"1.0","yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size":"10","fs.s3n.multipart.uploads.enabled":"false","hadoop.security.crypto.buffer.size":"8192","yarn.nodemanager.node-labels.provider.fetch-interval-ms":"600000","mapreduce.jobhistory.recovery.store.leveldb.path":"${hadoop.tmp.dir}/mapred/history/recoverystore","yarn.client.failover-retries-on-socket-timeouts":"0","hadoop.security.instrumentation.requires.admin":"false","yarn.nodemanager.delete.thread-count":"4","mapreduce.job.finish-when-all-reducers-done":"false","hadoop.registry.jaas.context":"Client","yarn.timeline-service.leveldb-timeline-store.path":"${hadoop.tmp.dir}/yarn/timeline","s3.blocksize":"67108864","io.map.index.interval":"128","mapreduce.job.counters.max":"120","yarn.timeline-service.store-class":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.jobhistory.move.interval-ms":"180000","yarn.nodemanager.localizer.fetch.thread-count":"4","yarn.resourcemanager.scheduler.client.thread-count":"50","hadoop.ssl.hostname.verifier":"DEFAULT","yarn.timeline-service.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/timeline","mapreduce.job.classloader":"false","mapreduce.task.profile.map.params":"${mapreduce.task.profile.params}","ipc.client.connect.timeout":"20000","s3.stream-buffer-size":"4096","yarn.nm.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.reservation-system.planfollower.time-step":"1000","s3native.bytes-per-checksum":"512","mapreduce.jobtracker.address":"local","yarn.nodemanager.recovery.enabled":"false","mapreduce.job.end-notification.retry.interval":"1000","fs.du.interval":"600000","hadoop.security.group.mapping.ldap.read.timeout.ms":"60000","hadoop.security.groups.cache.warn.after.ms":"5000","file.bytes-per-checksum":"512","yarn.node-labels.fs-store.retry-policy-spec":"2000, 500","hadoop.security.groups.cache.background.reload":"false","net.topology.script.number.args":"100","mapreduce.task.merge.progress.records":"10000","yarn.nodemanager.localizer.address":"${yarn.nodemanager.hostname}:8040","yarn.timeline-service.keytab":"/etc/krb5.keytab","mapreduce.reduce.shuffle.fetch.retry.timeout-ms":"30000","yarn.resourcemanager.rm.container-allocation.expiry-interval-ms":"600000","mapreduce.fileoutputcommitter.algorithm.version":"1","yarn.resourcemanager.work-preserving-recovery.enabled":"true","mapreduce.map.skip.maxrecords":"0","yarn.sharedcache.root-dir":"/sharedcache","hadoop.http.authentication.type":"simple","mapreduce.task.userlog.limit.kb":"0","yarn.resourcemanager.scheduler.monitor.enable":"false","fs.s3n.block.size":"67108864","ipc.client.connect.max.retries":"10","hadoop.registry.zk.retry.times":"5","mapreduce.jobtracker.staging.root.dir":"${hadoop.tmp.dir}/mapred/staging","yarn.nodemanager.resource-monitor.interval-ms":"3000","mapreduce.shuffle.listen.queue.size":"128","mapreduce.map.cpu.vcores":"1","yarn.timeline-service.client.fd-retain-secs":"300","hadoop.user.group.static.mapping.overrides":"dr.who=;","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","yarn.resourcemanager.fail-fast":"${yarn.fail-fast}","yarn.resourcemanager.proxy-user-privileges.enabled":"false","mapreduce.job.reducer.preempt.delay.sec":"0","hadoop.util.hash.type":"murmur","yarn.app.mapreduce.client.job.max-retries":"0","mapreduce.reduce.shuffle.retry-delay.max.ms":"60000","hadoop.security.group.mapping.ldap.connection.timeout.ms":"60000","mapreduce.task.profile.params":"-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s","yarn.app.mapreduce.shuffle.log.backups":"0","hadoop.registry.zk.retry.interval.ms":"1000","yarn.nodemanager.linux-container-executor.cgroups.delete-timeout-ms":"1000","fs.AbstractFileSystem.file.impl":"org.apache.hadoop.fs.local.LocalFs","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds":"-1","mapreduce.jobhistory.cleaner.interval-ms":"86400000","hadoop.registry.zk.quorum":"localhost:2181","mapreduce.output.fileoutputformat.compress":"false","yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs":"*********(redacted)","hadoop.ssl.server.conf":"ssl-server.xml","yarn.sharedcache.cleaner.initial-delay-mins":"10","mapreduce.client.completion.pollinterval":"5000","hadoop.ssl.keystores.factory.class":"org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory","yarn.app.mapreduce.am.resource.cpu-vcores":"1","yarn.timeline-service.enabled":"false","yarn.nodemanager.runtime.linux.docker.capabilities":"CHOWN,DAC_OVERRIDE,FSETID,FOWNER,MKNOD,NET_RAW,SETGID,SETUID,SETFCAP,SETPCAP,NET_BIND_SERVICE,SYS_CHROOT,KILL,AUDIT_WRITE","yarn.acl.enable":"false","yarn.timeline-service.entity-group-fs-store.done-dir":"/tmp/entity-file-history/done/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user":"nobody","yarn.resourcemanager.configuration.provider-class":"org.apache.hadoop.yarn.LocalConfigurationProvider","yarn.resourcemanager.configuration.file-system-based-store":"/yarn/conf","yarn.nodemanager.resource.percentage-physical-cpu-limit":"100","mapreduce.jobhistory.client.thread-count":"10","tfile.fs.input.buffer.size":"262144","mapreduce.client.progressmonitor.pollinterval":"1000","yarn.nodemanager.log-dirs":"${yarn.log.dir}/userlogs","fs.automatic.close":"true","fs.s3n.multipart.copy.block.size":"5368709120","yarn.nodemanager.hostname":"0.0.0.0","yarn.resourcemanager.zk-timeout-ms":"10000","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","yarn.timeline-service.address":"${yarn.timeline-service.hostname}:10200","mapreduce.job.ubertask.maxmaps":"9","fs.s3a.threads.keepalivetime":"60","mapreduce.task.files.preserve.failedtasks":"false","yarn.app.mapreduce.client.job.retry-interval":"2000","ha.failover-controller.graceful-fence.connection.retries":"1","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","yarn.timeline-service.entity-group-fs-store.summary-store":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.reduce.cpu.vcores":"1","fs.client.resolve.remote.symlinks":"true","yarn.nodemanager.webapp.https.address":"0.0.0.0:8044","hadoop.http.cross-origin.allowed-origins":"*","yarn.timeline-service.entity-group-fs-store.retain-seconds":"604800","yarn.resourcemanager.metrics.runtime.buckets":"60,300,1440","yarn.timeline-service.generic-application-history.max-applications":"10000","yarn.nodemanager.local-dirs":"${hadoop.tmp.dir}/nm-local-dir","mapreduce.shuffle.connection-keep-alive.enable":"false","yarn.node-labels.configuration-type":"centralized","fs.s3a.path.style.access":"false","yarn.nodemanager.aux-services.mapreduce_shuffle.class":"org.apache.hadoop.mapred.ShuffleHandler","yarn.sharedcache.store.in-memory.staleness-period-mins":"10080","fs.adl.impl":"org.apache.hadoop.fs.adl.AdlFileSystem","yarn.resourcemanager.nodemanager.minimum.version":"NONE","net.topology.impl":"org.apache.hadoop.net.NetworkTopology","io.map.index.skip":"0","yarn.scheduler.maximum-allocation-vcores":"4","hadoop.http.cross-origin.allowed-headers":"X-Requested-With,Content-Type,Accept,Origin","yarn.nodemanager.log-aggregation.compression-type":"none","yarn.timeline-service.version":"1.0f","yarn.ipc.rpc.class":"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC","mapreduce.reduce.maxattempts":"4","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"0","ipc.maximum.response.length":"134217728","mapreduce.job.token.tracking.ids.enabled":"*********(redacted)","hadoop.caller.context.max.size":"128","hadoop.registry.system.acls":"sasl:yarn@, sasl:mapred@, sasl:hdfs@","yarn.nodemanager.recovery.dir":"${hadoop.tmp.dir}/yarn-nm-recovery","fs.s3a.fast.upload.buffer":"disk","mapreduce.jobhistory.intermediate-done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done_intermediate","yarn.app.mapreduce.shuffle.log.separate":"true","fs.s3a.max.total.tasks":"5","fs.s3a.readahead.range":"64K","hadoop.http.authentication.simple.anonymous.allowed":"true","fs.s3a.fast.upload":"false","fs.s3a.attempts.maximum":"20","hadoop.registry.zk.connection.timeout.ms":"15000","yarn.resourcemanager.delegation-token-renewer.thread-count":"*********(redacted)","yarn.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","mapreduce.map.log.level":"INFO","mapreduce.output.fileoutputformat.compress.type":"RECORD","yarn.resourcemanager.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/system/rmstore","hadoop.registry.rm.enabled":"false","mapreduce.ifile.readahead.bytes":"4194304","yarn.resourcemanager.fs.state-store.retry-policy-spec":"2000, 500","yarn.sharedcache.app-checker.class":"org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker","yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users":"true","yarn.nodemanager.resource.detect-hardware-capabilities":"false","mapreduce.cluster.acls.enabled":"false","mapreduce.job.speculative.retry-after-no-speculate":"1000","yarn.resourcemanager.fs.state-store.retry-interval-ms":"1000","file.stream-buffer-size":"4096","mapreduce.map.output.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","mapreduce.map.speculative":"true","mapreduce.job.speculative.retry-after-speculate":"15000","yarn.nodemanager.linux-container-executor.cgroups.mount":"false","yarn.app.mapreduce.am.container.log.backups":"0","yarn.app.mapreduce.am.log.level":"INFO","mapreduce.job.reduce.slowstart.completedmaps":"0.05","yarn.timeline-service.http-authentication.type":"simple","hadoop.security.group.mapping.ldap.search.attr.group.name":"cn","yarn.timeline-service.client.internal-timers-ttl-secs":"420","fs.s3a.block.size":"32M","yarn.sharedcache.client-server.address":"0.0.0.0:8045","yarn.resourcemanager.hostname":"0.0.0.0","yarn.resourcemanager.delegation.key.update-interval":"86400000","mapreduce.reduce.shuffle.fetch.retry.enabled":"${yarn.nodemanager.recovery.enabled}","mapreduce.map.memory.mb":"1024","mapreduce.task.skip.start.attempts":"2","fs.AbstractFileSystem.hdfs.impl":"org.apache.hadoop.fs.Hdfs","yarn.nodemanager.disk-health-checker.enable":"true","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","fs.s3.maxRetries":"4","ipc.client.low-latency":"false","mapreduce.input.lineinputformat.linespermap":"1","ipc.client.connect.max.retries.on.timeouts":"45","yarn.timeline-service.leveldb-timeline-store.read-cache-size":"104857600","fs.AbstractFileSystem.har.impl":"org.apache.hadoop.fs.HarFs","mapreduce.job.split.metainfo.maxsize":"10000000","yarn.am.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.timeline-service.entity-group-fs-store.app-cache-size":"10","fs.s3a.socket.recv.buffer":"8192","fs.s3n.multipart.uploads.block.size":"67108864","yarn.resourcemanager.resource-tracker.address":"${yarn.resourcemanager.hostname}:8031","yarn.nodemanager.node-labels.provider.fetch-timeout-ms":"1200000","yarn.resourcemanager.leveldb-state-store.compaction-interval-secs":"3600","mapreduce.client.output.filter":"FAILED","hadoop.http.filter.initializers":"org.apache.hadoop.http.lib.StaticUserWebFilter","mapreduce.reduce.memory.mb":"1024","s3native.client-write-packet-size":"65536","yarn.timeline-service.hostname":"0.0.0.0","file.replication":"1","yarn.nodemanager.container-metrics.unregister-delay-ms":"10000","yarn.nodemanager.container-metrics.period-ms":"-1","yarn.nodemanager.log.retain-seconds":"10800","yarn.timeline-service.entity-group-fs-store.cleaner-interval-seconds":"3600","yarn.resourcemanager.keytab":"/etc/krb5.keytab","hadoop.security.group.mapping.providers.combined":"true","mapreduce.reduce.merge.inmem.threshold":"1000","yarn.timeline-service.recovery.enabled":"false","yarn.sharedcache.nm.uploader.thread-count":"20","mapreduce.shuffle.ssl.enabled":"false","yarn.resourcemanager.state-store.max-completed-applications":"${yarn.resourcemanager.max-completed-applications}","mapreduce.job.speculative.minimum-allowed-tasks":"10","yarn.log-aggregation.retain-seconds":"-1","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb":"0","mapreduce.jobhistory.max-age-ms":"604800000","hadoop.http.cross-origin.allowed-methods":"GET,POST,HEAD","mapreduce.jobhistory.webapp.address":"0.0.0.0:19888","mapreduce.jobtracker.system.dir":"${hadoop.tmp.dir}/mapred/system","yarn.client.nodemanager-connect.max-wait-ms":"180000","yarn.resourcemanager.webapp.address":"${yarn.resourcemanager.hostname}:8088","mapreduce.jobhistory.recovery.enable":"false","mapreduce.reduce.shuffle.parallelcopies":"5","fs.AbstractFileSystem.webhdfs.impl":"org.apache.hadoop.fs.WebHdfs","fs.trash.interval":"0","yarn.app.mapreduce.client.max-retries":"3","hadoop.security.authentication":"simple","mapreduce.task.profile.reduce.params":"${mapreduce.task.profile.params}","yarn.app.mapreduce.am.resource.mb":"1536","mapreduce.input.fileinputformat.list-status.num-threads":"1","yarn.nodemanager.container-executor.class":"org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor","io.mapfile.bloom.size":"1048576","yarn.timeline-service.ttl-ms":"604800000","yarn.nodemanager.resource.cpu-vcores":"-1","mapreduce.job.reduces":"1","fs.s3a.multipart.size":"100M","yarn.scheduler.minimum-allocation-vcores":"1","mapreduce.job.speculative.speculative-cap-total-tasks":"0.01","hadoop.ssl.client.conf":"ssl-client.xml","mapreduce.job.queuename":"default","ha.health-monitor.sleep-after-disconnect.ms":"1000","s3.bytes-per-checksum":"512","yarn.app.mapreduce.shuffle.log.limit.kb":"0","hadoop.security.group.mapping":"org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback","yarn.client.application-client-protocol.poll-timeout-ms":"-1","mapreduce.jobhistory.jhist.format":"json","yarn.resourcemanager.ha.enabled":"false","hadoop.http.staticuser.user":"dr.who","mapreduce.task.exit.timeout.check-interval-ms":"20000","mapreduce.task.exit.timeout":"60000","yarn.nodemanager.linux-container-executor.resources-handler.class":"org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler","mapreduce.reduce.shuffle.memory.limit.percent":"0.25","yarn.resourcemanager.reservation-system.enable":"false","s3.client-write-packet-size":"65536","mapreduce.map.output.compress":"false","ha.zookeeper.acl":"world:anyone:rwcda","ipc.server.max.connections":"0","yarn.scheduler.maximum-allocation-mb":"8192","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","yarn.sharedcache.cleaner.period-mins":"1440","yarn.app.mapreduce.am.container.log.limit.kb":"0","s3native.blocksize":"67108864","ipc.client.connect.retry.interval":"1000","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","mapreduce.jobhistory.cleaner.enable":"true","yarn.timeline-service.client.fd-flush-interval-secs":"10","hadoop.security.kms.client.encrypted.key.cache.expiry":"43200000","yarn.client.nodemanager-client-async.thread-pool-max-size":"500","mapreduce.map.maxattempts":"4","yarn.nodemanager.sleep-delay-before-sigkill.ms":"250","mapreduce.job.end-notification.retry.attempts":"0","yarn.nodemanager.resource.count-logical-processors-as-cores":"false","yarn.resourcemanager.zk-num-retries":"1000","hadoop.registry.zk.root":"/registry","adl.feature.ownerandgroup.enableupn":"false","mapreduce.job.reduce.shuffle.consumer.plugin.class":"org.apache.hadoop.mapreduce.task.reduce.Shuffle","yarn.resourcemanager.delayed.delegation-token.removal-interval-ms":"*********(redacted)","yarn.nodemanager.localizer.cache.target-size-mb":"10240","ftp.client-write-packet-size":"65536","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","yarn.client.failover-retries":"0","fs.s3a.multipart.purge.age":"86400","io.native.lib.available":"true","net.topology.node.switch.mapping.impl":"org.apache.hadoop.net.ScriptBasedMapping","yarn.nodemanager.amrmproxy.address":"0.0.0.0:8048","ipc.server.listen.queue.size":"128","map.sort.class":"org.apache.hadoop.util.QuickSort","fs.viewfs.rename.strategy":"SAME_MOUNTPOINT","hadoop.security.kms.client.authentication.retry-count":"1","fs.permissions.umask-mode":"022","yarn.nodemanager.vmem-check-enabled":"true","yarn.nodemanager.recovery.compaction-interval-secs":"3600","yarn.app.mapreduce.client-am.ipc.max-retries":"3","mapreduce.job.ubertask.maxreduces":"1","hadoop.security.kms.client.encrypted.key.cache.size":"500","hadoop.security.java.secure.random.algorithm":"SHA1PRNG","ha.failover-controller.cli-check.rpc-timeout.ms":"20000","mapreduce.jobhistory.jobname.limit":"50","yarn.client.nodemanager-connect.retry-interval-ms":"10000","yarn.timeline-service.state-store-class":"org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME","yarn.sharedcache.nested-level":"3","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","yarn.resourcemanager.recovery.enabled":"false"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/root","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/amd64","user.dir":"/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.252-b09","jetty.git.hash":"ab228fde9e55e9164c738d7fa121f8ac5acd51c9","java.endorsed.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/endorsed","java.runtime.version":"1.8.0_252-b09","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/resources.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/rt.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/jsse.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/jce.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/charsets.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/lib/jfr.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre/classes","file.encoding":"UTF-8","user.timezone":"Asia/Shanghai","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-1127.10.1.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"root","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local[*] --conf spark.eventLog.dir=/tmp/spark-history --conf spark.eventLog.enabled=true --conf spark.sql.shuffle.partitions=2 --class org.apache.spark.examples.sql.streaming.StructuredKafkaWordCount ./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar 192.168.130.97:9092 subscribe test5","java.home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.252.b09-2.el7_8.x86_64/jre","java.version":"1.8.0_252","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-graphite-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-vector-code-gen-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-server-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/pyrolite-4.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/conf/":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json-smart-2.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/objenesis-2.5.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-auth-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jsp-api-2.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-unsafe_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-codec-1.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/protobuf-java-2.5.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/avro-1.8.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/guice-3.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/aopalliance-repackaged-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/transaction-api-1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/gson-2.2.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/datanucleus-rdbms-4.1.19.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-module-paranamer-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/libfb303-0.9.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-cli-1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-tags_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-library-2.12.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/xbean-asm7-shaded-4.15.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-container-servlet-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hk2-api-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/okhttp-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/derby-10.12.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-collections-3.2.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/httpcore-4.4.12.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-beanutils-1.9.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire-util_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-crypto-1.0.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-launcher_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/stax-api-1.0-2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-ast_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/lz4-java-1.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-format-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-column-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-logging-1.1.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/audience-annotations-0.5.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-jdbc-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-hive-thriftserver_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-cli-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javolution-5.5.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/JLargeArrays-1.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-api-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/algebra_2.12-2.0.0-M2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-dbcp-1.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/threeten-extra-1.5.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-io-2.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-json-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/libthrift-0.12.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/compress-lzf-1.0.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-jmx-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.inject-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/stax-api-1.0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-common-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/curator-recipes-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/antlr4-runtime-4.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/slf4j-api-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/oro-2.0.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arrow-memory-0.15.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jpam-1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/velocity-1.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/orc-core-1.5.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-sql_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-databind-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-text-1.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-client-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/htrace-core4-4.0.1-incubating.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json-1.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-graphx_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/avro-ipc-1.8.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jetty-util-6.1.26.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/core-1.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/joda-time-2.10.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-encoding-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-llap-common-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-network-common_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/datanucleus-api-jdo-4.2.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/paranamer-2.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-0.23-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/activation-1.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/curator-framework-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-compress-1.8.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/RoaringBitmap-0.7.45.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/ivy-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-core-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-client-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-httpclient-3.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-yarn_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-common-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/zstd-jni-1.4.5-2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-container-servlet-core-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/snappy-java-1.1.7.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/shapeless_2.12-2.3.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-pool-1.5.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-core_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/httpclient-4.5.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/api-util-1.0.0-M20.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/aircompressor-0.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-repl_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/leveldbjni-all-1.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-hk2-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jta-1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-net-3.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/datanucleus-core-4.1.17.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-server-web-proxy-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/breeze_2.12-1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/orc-mapreduce-1.5.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jetty-6.1.26.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-core_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/xz-1.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javax.inject-1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-compiler-2.12.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-jvm-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/orc-shims-1.5.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jaxb-api-2.2.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.validation-api-2.0.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/janino-3.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/osgi-resource-locator-1.0.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jcl-over-slf4j-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-app-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hk2-utils-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-sketch_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/JTransforms-3.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/kafka-clients-2.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/guice-servlet-3.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/okio-1.4.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-annotations-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-math3-3.4.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-scalap_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-streaming_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/accessors-smart-1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/guava-14.0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/shims-0.7.45.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/ST4-4.0.4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-xml_2.12-1.2.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/chill-java-0.9.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-shuffle-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/stream-2.9.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-configuration-1.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jodd-core-3.5.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-pool2-2.6.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jul-to-slf4j-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/xmlenc-0.52.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-token-provider-kafka-0-10_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/json4s-jackson_2.12-3.6.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-compiler-3.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jline-2.14.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/breeze-macros_2.12-1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/kryo-shaded-4.0.2.jar":"System Classpath","spark://iZbp19vpr16ix621sdw476Z:46309/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar":"Added By User","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-hive_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-common-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/istack-commons-runtime-3.0.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/curator-client-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-xc-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/zookeeper-3.4.14.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jakarta.annotation-api-1.3.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-shims-scheduler-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/univocity-parsers-2.8.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-digester-1.8.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-mllib_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arpack_combined_all-0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-sql-kafka-0-10_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-annotations-2.10.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hk2-locator-2.6.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-core-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-yarn-server-common-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/scala-reflect-2.12.10.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/super-csv-2.2.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-mapreduce-client-jobclient-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-client-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-common-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-exec-2.3.7-core.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/opencsv-2.3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-storage-api-2.7.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/aopalliance-1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/HikariCP-2.5.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-metastore-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/minlog-1.3.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arrow-format-0.15.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jsr305-3.0.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-lang-2.6.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/commons-lang3-3.9.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javax.jdo-3.2.0-m3.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javassist-3.25.0-GA.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/antlr-runtime-3.5.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/log4j-1.2.17.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-beeline-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/chill_2.12-0.9.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jdo-api-3.0.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-kvstore_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/metrics-core-4.1.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jaxb-runtime-2.3.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-mllib-local_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/py4j-0.10.9.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hive-serde-2.3.7.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/hadoop-hdfs-client-2.8.5.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-network-shuffle_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jcip-annotations-1.0-1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/jersey-media-jaxb-2.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/parquet-jackson-1.10.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/slf4j-log4j12-1.7.30.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/netty-all-4.1.47.Final.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/arrow-vector-0.15.1.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/spark-catalyst_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/jars/machinist_2.12-0.6.8.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"StructuredKafkaWordCount","App ID":"local-1596020211915","Timestamp":1596020210919,"User":"root"} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryStartedEvent","id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:56:55.947Z"} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48276}}, {\"test5\":{\"0\":48279}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#142]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = f7faa1e9-69d9-41b4-9d77-919795af2413, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = f7faa1e9-69d9-41b4-9d77-919795af2413, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#66]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":80,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":79,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":76,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":77,"metricType":"timing"},{"name":"peak memory","accumulatorId":75,"metricType":"size"},{"name":"number of output rows","accumulatorId":74,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":78,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":71,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":72,"metricType":"timing"},{"name":"peak memory","accumulatorId":70,"metricType":"size"},{"name":"number of output rows","accumulatorId":69,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":73,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":68,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":20,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":21,"metricType":"nsTiming"},{"name":"records read","accumulatorId":18,"metricType":"sum"},{"name":"local bytes read","accumulatorId":16,"metricType":"size"},{"name":"fetch wait time","accumulatorId":17,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":14,"metricType":"size"},{"name":"local blocks read","accumulatorId":13,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":12,"metricType":"sum"},{"name":"data size","accumulatorId":11,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":15,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":19,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":67,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":64,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":65,"metricType":"timing"},{"name":"peak memory","accumulatorId":63,"metricType":"size"},{"name":"number of output rows","accumulatorId":62,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":66,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":61,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":51,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":52,"metricType":"sum"},{"name":"memory used by state","accumulatorId":57,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":59,"metricType":"sum"},{"name":"number of output rows","accumulatorId":50,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":58,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":60,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":56,"metricType":"timing"},{"name":"time to remove","accumulatorId":55,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":53,"metricType":"sum"},{"name":"time to update","accumulatorId":54,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":47,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":48,"metricType":"timing"},{"name":"peak memory","accumulatorId":46,"metricType":"size"},{"name":"number of output rows","accumulatorId":45,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":49,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":44,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020220179} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48276}}, {\"test5\":{\"0\":48279}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#218]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 64a4779b-846a-4f20-9f5c-899a8dbf68d8, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 64a4779b-846a-4f20-9f5c-899a8dbf68d8, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27fafcca","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 0, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#66]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":80,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":79,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":76,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":77,"metricType":"timing"},{"name":"peak memory","accumulatorId":75,"metricType":"size"},{"name":"number of output rows","accumulatorId":74,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":78,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":71,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":72,"metricType":"timing"},{"name":"peak memory","accumulatorId":70,"metricType":"size"},{"name":"number of output rows","accumulatorId":69,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":73,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":68,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":20,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":21,"metricType":"nsTiming"},{"name":"records read","accumulatorId":18,"metricType":"sum"},{"name":"local bytes read","accumulatorId":16,"metricType":"size"},{"name":"fetch wait time","accumulatorId":17,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":14,"metricType":"size"},{"name":"local blocks read","accumulatorId":13,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":12,"metricType":"sum"},{"name":"data size","accumulatorId":11,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":15,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":19,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":67,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":64,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":65,"metricType":"timing"},{"name":"peak memory","accumulatorId":63,"metricType":"size"},{"name":"number of output rows","accumulatorId":62,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":66,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":61,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":51,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":52,"metricType":"sum"},{"name":"memory used by state","accumulatorId":57,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":59,"metricType":"sum"},{"name":"number of output rows","accumulatorId":50,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":58,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":60,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":56,"metricType":"timing"},{"name":"time to remove","accumulatorId":55,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":53,"metricType":"sum"},{"name":"time to update","accumulatorId":54,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":47,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":48,"metricType":"timing"},{"name":"peak memory","accumulatorId":46,"metricType":"size"},{"name":"number of output rows","accumulatorId":45,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":49,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":44,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020220258} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1596020221633,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"DataSourceRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"StateStoreRDD","Scope":"{\"id\":\"3\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"StateStoreRDD","Scope":"{\"id\":\"7\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[0],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0,1],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"0","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"1","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"DataSourceRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020221656,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"0","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"1","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1596020221738,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1596020221738,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020222649,"Failed":false,"Killed":false,"Accumulables":[{"ID":21,"Name":"shuffle write time","Update":"9599308","Value":"9599308","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"duration","Update":"296","Value":"296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":69,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":70,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":72,"Name":"time in aggregation build","Update":"200","Value":"200","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":75,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":77,"Name":"time in aggregation build","Update":"190","Value":"190","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":79,"Name":"duration","Update":"336","Value":"336","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":80,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":125,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":123,"Name":"internal.metrics.shuffle.write.writeTime","Update":9599308,"Value":9599308,"Internal":true,"Count Failed Values":true},{"ID":122,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":17,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.resultSize","Update":2630,"Value":2630,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.executorCpuTime","Update":466139164,"Value":466139164,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorRunTime","Update":503,"Value":503,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorDeserializeCpuTime","Update":301869581,"Value":301869581,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorDeserializeTime","Update":361,"Value":361,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":361,"Executor Deserialize CPU Time":301869581,"Executor Run Time":503,"Executor CPU Time":466139164,"Peak Execution Memory":524288,"Result Size":2630,"JVM GC Time":17,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":9599308,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"16\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"DataSourceRDD","Scope":"{\"id\":\"20\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020221656,"Completion Time":1596020222661,"Accumulables":[{"ID":104,"Name":"internal.metrics.executorDeserializeCpuTime","Value":301869581,"Internal":true,"Count Failed Values":true},{"ID":122,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"time in aggregation build","Value":"190","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"duration","Value":"296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":80,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":125,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":107,"Name":"internal.metrics.resultSize","Value":2630,"Internal":true,"Count Failed Values":true},{"ID":74,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":106,"Name":"internal.metrics.executorCpuTime","Value":466139164,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorDeserializeTime","Value":361,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"duration","Value":"336","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":70,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":123,"Name":"internal.metrics.shuffle.write.writeTime","Value":9599308,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.executorRunTime","Value":503,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":72,"Name":"time in aggregation build","Value":"200","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":108,"Name":"internal.metrics.jvmGCTime","Value":17,"Internal":true,"Count Failed Values":true},{"ID":21,"Name":"shuffle write time","Value":"9599308","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":75,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"StateStoreRDD","Scope":"{\"id\":\"3\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"StateStoreRDD","Scope":"{\"id\":\"7\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[0],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020222688,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"0","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"1","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1596020222709,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1596020222713,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1596020222713,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020222954,"Failed":false,"Killed":false,"Accumulables":[{"ID":44,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":46,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":48,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":54,"Name":"time to update","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":56,"Name":"time to commit changes","Update":"50","Value":"50","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":58,"Name":"estimated size of state only on current version","Update":"64","Value":"64","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":57,"Name":"memory used by state","Update":"208","Value":"208","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":61,"Name":"duration","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":145,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":143,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":142,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":141,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":140,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":139,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":137,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":132,"Name":"internal.metrics.resultSize","Update":5354,"Value":5354,"Internal":true,"Count Failed Values":true},{"ID":131,"Name":"internal.metrics.executorCpuTime","Update":93367533,"Value":93367533,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.executorRunTime","Update":203,"Value":203,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10308753,"Value":10308753,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorDeserializeTime","Update":23,"Value":23,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":23,"Executor Deserialize CPU Time":10308753,"Executor Run Time":203,"Executor CPU Time":93367533,"Peak Execution Memory":524288,"Result Size":5354,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1596020222709,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020222965,"Failed":false,"Killed":false,"Accumulables":[{"ID":44,"Name":"duration","Update":"33","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":45,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":46,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":48,"Name":"time in aggregation build","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":54,"Name":"time to update","Update":"28","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":53,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":56,"Name":"time to commit changes","Update":"31","Value":"81","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":58,"Name":"estimated size of state only on current version","Update":"424","Value":"488","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":50,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":57,"Name":"memory used by state","Update":"568","Value":"776","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":52,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":61,"Name":"duration","Update":"28","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":17,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":18,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":145,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":143,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":142,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":141,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":140,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":139,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":137,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":132,"Name":"internal.metrics.resultSize","Update":5574,"Value":10928,"Internal":true,"Count Failed Values":true},{"ID":131,"Name":"internal.metrics.executorCpuTime","Update":91355172,"Value":184722705,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.executorRunTime","Update":205,"Value":408,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorDeserializeCpuTime","Update":21029530,"Value":31338283,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorDeserializeTime","Update":34,"Value":57,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":34,"Executor Deserialize CPU Time":21029530,"Executor Run Time":205,"Executor CPU Time":91355172,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"8\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"StateStoreRDD","Scope":"{\"id\":\"3\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"StateStoreRDD","Scope":"{\"id\":\"7\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[0],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020222688,"Completion Time":1596020222967,"Accumulables":[{"ID":137,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorDeserializeTime","Value":57,"Internal":true,"Count Failed Values":true},{"ID":131,"Name":"internal.metrics.executorCpuTime","Value":184722705,"Internal":true,"Count Failed Values":true},{"ID":50,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":140,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":17,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":134,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"duration","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":56,"Name":"time to commit changes","Value":"81","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":142,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":145,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":139,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"estimated size of state only on current version","Value":"488","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":130,"Name":"internal.metrics.executorRunTime","Value":408,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":52,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":61,"Name":"duration","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":129,"Name":"internal.metrics.executorDeserializeCpuTime","Value":31338283,"Internal":true,"Count Failed Values":true},{"ID":132,"Name":"internal.metrics.resultSize","Value":10928,"Internal":true,"Count Failed Values":true},{"ID":141,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":54,"Name":"time to update","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":144,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":57,"Name":"memory used by state","Value":"776","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":48,"Name":"time in aggregation build","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":143,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1596020222973,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 0","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#46, count#47]\nArguments: [value#46, count#47]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#46, count#47]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":153,"metricType":"sum"}]},"time":1596020223028} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1596020223062} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1596020223069} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1596020223069} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:56:56.015Z","batchId":0,"batchDuration":7110,"durationMs":{"triggerExecution":7109,"queryPlanning":439,"getBatch":21,"latestOffset":3524,"addBatch":3011,"walCommit":35},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":776,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":488,"loadedMapCacheHitCount":0,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":null,"endOffset":"{\"test5\":{\"0\":48279}}","numInputRows":3,"inputRowsPerSecond":"NaN","processedRowsPerSecond":0.42194092827004215}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":3,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48279}}, {\"test5\":{\"0\":48642}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#373]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 1fb6b6c6-ced8-4f85-80af-1f3f4c424457, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 1fb6b6c6-ced8-4f85-80af-1f3f4c424457, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#297]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":237,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":236,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":233,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":234,"metricType":"timing"},{"name":"peak memory","accumulatorId":232,"metricType":"size"},{"name":"number of output rows","accumulatorId":231,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":235,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":228,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":229,"metricType":"timing"},{"name":"peak memory","accumulatorId":227,"metricType":"size"},{"name":"number of output rows","accumulatorId":226,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":230,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":225,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":177,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":178,"metricType":"nsTiming"},{"name":"records read","accumulatorId":175,"metricType":"sum"},{"name":"local bytes read","accumulatorId":173,"metricType":"size"},{"name":"fetch wait time","accumulatorId":174,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":171,"metricType":"size"},{"name":"local blocks read","accumulatorId":170,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":169,"metricType":"sum"},{"name":"data size","accumulatorId":168,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":172,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":176,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":224,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":221,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":222,"metricType":"timing"},{"name":"peak memory","accumulatorId":220,"metricType":"size"},{"name":"number of output rows","accumulatorId":219,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":223,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":218,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":208,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":209,"metricType":"sum"},{"name":"memory used by state","accumulatorId":214,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":216,"metricType":"sum"},{"name":"number of output rows","accumulatorId":207,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":215,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":217,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":213,"metricType":"timing"},{"name":"time to remove","accumulatorId":212,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":210,"metricType":"sum"},{"name":"time to update","accumulatorId":211,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":204,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":205,"metricType":"timing"},{"name":"peak memory","accumulatorId":203,"metricType":"size"},{"name":"number of output rows","accumulatorId":202,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":206,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":201,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020223333} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":4,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48279}}, {\"test5\":{\"0\":48642}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#449]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 7992c0a8-0641-440d-aaf7-ad453fe25c0a, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 7992c0a8-0641-440d-aaf7-ad453fe25c0a, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@3a1eb73c","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 1, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#297]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":237,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":236,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":233,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":234,"metricType":"timing"},{"name":"peak memory","accumulatorId":232,"metricType":"size"},{"name":"number of output rows","accumulatorId":231,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":235,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":228,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":229,"metricType":"timing"},{"name":"peak memory","accumulatorId":227,"metricType":"size"},{"name":"number of output rows","accumulatorId":226,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":230,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":225,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":177,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":178,"metricType":"nsTiming"},{"name":"records read","accumulatorId":175,"metricType":"sum"},{"name":"local bytes read","accumulatorId":173,"metricType":"size"},{"name":"fetch wait time","accumulatorId":174,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":171,"metricType":"size"},{"name":"local blocks read","accumulatorId":170,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":169,"metricType":"sum"},{"name":"data size","accumulatorId":168,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":172,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":176,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":224,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":221,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":222,"metricType":"timing"},{"name":"peak memory","accumulatorId":220,"metricType":"size"},{"name":"number of output rows","accumulatorId":219,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":223,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":218,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":208,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":209,"metricType":"sum"},{"name":"memory used by state","accumulatorId":214,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":216,"metricType":"sum"},{"name":"number of output rows","accumulatorId":207,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":215,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":217,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":213,"metricType":"timing"},{"name":"time to remove","accumulatorId":212,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":210,"metricType":"sum"},{"name":"time to update","accumulatorId":211,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":204,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":205,"metricType":"timing"},{"name":"peak memory","accumulatorId":203,"metricType":"size"},{"name":"number of output rows","accumulatorId":202,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":206,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":201,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020223382} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1596020223482,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"48\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"StateStoreRDD","Scope":"{\"id\":\"36\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"StateStoreRDD","Scope":"{\"id\":\"40\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[2],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[2,3],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"1","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"4","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"48\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223485,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"1","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"4","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":0,"Attempt":0,"Launch Time":1596020223493,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":0,"Attempt":0,"Launch Time":1596020223493,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020223601,"Failed":false,"Killed":false,"Accumulables":[{"ID":178,"Name":"shuffle write time","Update":"837580","Value":"837580","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":177,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":176,"Name":"shuffle bytes written","Update":"169","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":168,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"duration","Update":"84","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":226,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":229,"Name":"time in aggregation build","Update":"74","Value":"74","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":231,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":232,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":234,"Name":"time in aggregation build","Update":"68","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":236,"Name":"duration","Update":"84","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":237,"Name":"number of output rows","Update":"363","Value":"363","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":282,"Name":"internal.metrics.input.recordsRead","Update":363,"Value":363,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.shuffle.write.writeTime","Update":837580,"Value":837580,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":169,"Value":169,"Internal":true,"Count Failed Values":true},{"ID":269,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.executorCpuTime","Update":95945587,"Value":95945587,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.executorRunTime","Update":96,"Value":96,"Internal":true,"Count Failed Values":true},{"ID":261,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7437557,"Value":7437557,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":7,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7437557,"Executor Run Time":96,"Executor CPU Time":95945587,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":169,"Shuffle Write Time":837580,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":363},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"DataSourceRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"53\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"48\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223485,"Completion Time":1596020223603,"Accumulables":[{"ID":227,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":236,"Name":"duration","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":176,"Name":"shuffle bytes written","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":262,"Name":"internal.metrics.executorRunTime","Value":96,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":280,"Name":"internal.metrics.shuffle.write.writeTime","Value":837580,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"time in aggregation build","Value":"74","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":232,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":178,"Name":"shuffle write time","Value":"837580","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"duration","Value":"84","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":261,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7437557,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"time in aggregation build","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":264,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":282,"Name":"internal.metrics.input.recordsRead","Value":363,"Internal":true,"Count Failed Values":true},{"ID":237,"Name":"number of output rows","Value":"363","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":177,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":168,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":231,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":263,"Name":"internal.metrics.executorCpuTime","Value":95945587,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.executorDeserializeTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":269,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":169,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"StateStoreRDD","Scope":"{\"id\":\"36\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"StateStoreRDD","Scope":"{\"id\":\"40\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[2],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223613,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"1","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"4","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1596020223625,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1596020223626,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1596020223625,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020223717,"Failed":false,"Killed":false,"Accumulables":[{"ID":201,"Name":"duration","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":203,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":205,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"time to update","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":212,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":213,"Name":"time to commit changes","Update":"38","Value":"38","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":215,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":216,"Name":"count of cache hit on states cache in provider","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":214,"Name":"memory used by state","Update":"376","Value":"376","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":218,"Name":"duration","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":220,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":222,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":302,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":299,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":288,"Name":"internal.metrics.executorCpuTime","Update":22954307,"Value":22954307,"Internal":true,"Count Failed Values":true},{"ID":287,"Name":"internal.metrics.executorRunTime","Update":77,"Value":77,"Internal":true,"Count Failed Values":true},{"ID":286,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6627382,"Value":6627382,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6627382,"Executor Run Time":77,"Executor CPU Time":22954307,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1596020223626,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020223720,"Failed":false,"Killed":false,"Accumulables":[{"ID":201,"Name":"duration","Update":"4","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":206,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":202,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":203,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":205,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"time to update","Update":"18","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":210,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":212,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":213,"Name":"time to commit changes","Update":"30","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":215,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":207,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":216,"Name":"count of cache hit on states cache in provider","Update":"2","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":214,"Name":"memory used by state","Update":"840","Value":"1216","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":218,"Name":"duration","Update":"19","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":219,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":220,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":222,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":170,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":174,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":173,"Name":"local bytes read","Update":"169","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":175,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":302,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":169,"Value":169,"Internal":true,"Count Failed Values":true},{"ID":299,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":288,"Name":"internal.metrics.executorCpuTime","Update":25907369,"Value":48861676,"Internal":true,"Count Failed Values":true},{"ID":287,"Name":"internal.metrics.executorRunTime","Update":82,"Value":159,"Internal":true,"Count Failed Values":true},{"ID":286,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7573630,"Value":14201012,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":13,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7573630,"Executor Run Time":82,"Executor CPU Time":25907369,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":169,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"StateStoreRDD","Scope":"{\"id\":\"36\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"41\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"StateStoreRDD","Scope":"{\"id\":\"40\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[2],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020223613,"Completion Time":1596020223724,"Accumulables":[{"ID":218,"Name":"duration","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":173,"Name":"local bytes read","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":286,"Name":"internal.metrics.executorDeserializeCpuTime","Value":14201012,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":175,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"time to update","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":202,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":220,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":214,"Name":"memory used by state","Value":"1216","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":205,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":300,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":169,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.executorDeserializeTime","Value":13,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":297,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":288,"Name":"internal.metrics.executorCpuTime","Value":48861676,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"count of cache hit on states cache in provider","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":174,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":210,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":219,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":201,"Name":"duration","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":222,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":213,"Name":"time to commit changes","Value":"68","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":299,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":203,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":170,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":215,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":287,"Name":"internal.metrics.executorRunTime","Value":159,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":296,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1596020223725,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":5,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 1","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#60, count#61]\nArguments: [value#60, count#61]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#60, count#61]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":310,"metricType":"sum"}]},"time":1596020223752} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":5,"time":1596020223761} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":4,"time":1596020223762} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":3,"time":1596020223762} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:03.168Z","batchId":1,"batchDuration":622,"durationMs":{"triggerExecution":622,"queryPlanning":47,"getBatch":0,"latestOffset":7,"addBatch":478,"walCommit":59},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1216,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":4,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48279}}","endOffset":"{\"test5\":{\"0\":48642}}","numInputRows":363,"inputRowsPerSecond":50.74793792814204,"processedRowsPerSecond":583.6012861736334}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":6,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48642}}, {\"test5\":{\"0\":48705}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#604]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 39c861a0-0e30-4ca2-b363-495aff0f3f93, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 39c861a0-0e30-4ca2-b363-495aff0f3f93, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#528]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":394,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":393,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":390,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":391,"metricType":"timing"},{"name":"peak memory","accumulatorId":389,"metricType":"size"},{"name":"number of output rows","accumulatorId":388,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":392,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":385,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":386,"metricType":"timing"},{"name":"peak memory","accumulatorId":384,"metricType":"size"},{"name":"number of output rows","accumulatorId":383,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":387,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":382,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":334,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":335,"metricType":"nsTiming"},{"name":"records read","accumulatorId":332,"metricType":"sum"},{"name":"local bytes read","accumulatorId":330,"metricType":"size"},{"name":"fetch wait time","accumulatorId":331,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":328,"metricType":"size"},{"name":"local blocks read","accumulatorId":327,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":326,"metricType":"sum"},{"name":"data size","accumulatorId":325,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":329,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":333,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":381,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":378,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":379,"metricType":"timing"},{"name":"peak memory","accumulatorId":377,"metricType":"size"},{"name":"number of output rows","accumulatorId":376,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":380,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":375,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":365,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":366,"metricType":"sum"},{"name":"memory used by state","accumulatorId":371,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":373,"metricType":"sum"},{"name":"number of output rows","accumulatorId":364,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":372,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":374,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":370,"metricType":"timing"},{"name":"time to remove","accumulatorId":369,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":367,"metricType":"sum"},{"name":"time to update","accumulatorId":368,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":361,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":362,"metricType":"timing"},{"name":"peak memory","accumulatorId":360,"metricType":"size"},{"name":"number of output rows","accumulatorId":359,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":363,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":358,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020223909} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":7,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48642}}, {\"test5\":{\"0\":48705}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#680]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c2fd3b95-1ba6-4d3e-8b9c-0256dfd90973, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c2fd3b95-1ba6-4d3e-8b9c-0256dfd90973, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@52d6c50a","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 2, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#528]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":394,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":393,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":390,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":391,"metricType":"timing"},{"name":"peak memory","accumulatorId":389,"metricType":"size"},{"name":"number of output rows","accumulatorId":388,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":392,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":385,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":386,"metricType":"timing"},{"name":"peak memory","accumulatorId":384,"metricType":"size"},{"name":"number of output rows","accumulatorId":383,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":387,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":382,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":334,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":335,"metricType":"nsTiming"},{"name":"records read","accumulatorId":332,"metricType":"sum"},{"name":"local bytes read","accumulatorId":330,"metricType":"size"},{"name":"fetch wait time","accumulatorId":331,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":328,"metricType":"size"},{"name":"local blocks read","accumulatorId":327,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":326,"metricType":"sum"},{"name":"data size","accumulatorId":325,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":329,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":333,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":381,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":378,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":379,"metricType":"timing"},{"name":"peak memory","accumulatorId":377,"metricType":"size"},{"name":"number of output rows","accumulatorId":376,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":380,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":375,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":365,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":366,"metricType":"sum"},{"name":"memory used by state","accumulatorId":371,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":373,"metricType":"sum"},{"name":"number of output rows","accumulatorId":364,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":372,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":374,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":370,"metricType":"timing"},{"name":"time to remove","accumulatorId":369,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":367,"metricType":"sum"},{"name":"time to update","accumulatorId":368,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":361,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":362,"metricType":"timing"},{"name":"peak memory","accumulatorId":360,"metricType":"size"},{"name":"number of output rows","accumulatorId":359,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":363,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":358,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224006} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1596020224100,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"StateStoreRDD","Scope":"{\"id\":\"73\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"StateStoreRDD","Scope":"{\"id\":\"69\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[33],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"70\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"81\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"82\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"DataSourceRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[5,4],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"2","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"7","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"81\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"82\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"DataSourceRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224103,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"2","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"7","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":0,"Attempt":0,"Launch Time":1596020224113,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":0,"Attempt":0,"Launch Time":1596020224113,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224174,"Failed":false,"Killed":false,"Accumulables":[{"ID":335,"Name":"shuffle write time","Update":"686296","Value":"686296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":334,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":333,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":325,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":382,"Name":"duration","Update":"39","Value":"39","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":383,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":384,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":386,"Name":"time in aggregation build","Update":"32","Value":"32","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":388,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":389,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":391,"Name":"time in aggregation build","Update":"26","Value":"26","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":393,"Name":"duration","Update":"40","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":394,"Name":"number of output rows","Update":"63","Value":"63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":439,"Name":"internal.metrics.input.recordsRead","Update":63,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":437,"Name":"internal.metrics.shuffle.write.writeTime","Update":686296,"Value":686296,"Internal":true,"Count Failed Values":true},{"ID":436,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":435,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":426,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":421,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":420,"Name":"internal.metrics.executorCpuTime","Update":33390843,"Value":33390843,"Internal":true,"Count Failed Values":true},{"ID":419,"Name":"internal.metrics.executorRunTime","Update":49,"Value":49,"Internal":true,"Count Failed Values":true},{"ID":418,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4867521,"Value":4867521,"Internal":true,"Count Failed Values":true},{"ID":417,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":8,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":4867521,"Executor Run Time":49,"Executor CPU Time":33390843,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":686296,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":63},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"81\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"82\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"DataSourceRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"86\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224103,"Completion Time":1596020224175,"Accumulables":[{"ID":436,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":391,"Name":"time in aggregation build","Value":"26","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":382,"Name":"duration","Value":"39","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":418,"Name":"internal.metrics.executorDeserializeCpuTime","Value":4867521,"Internal":true,"Count Failed Values":true},{"ID":421,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":394,"Name":"number of output rows","Value":"63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":439,"Name":"internal.metrics.input.recordsRead","Value":63,"Internal":true,"Count Failed Values":true},{"ID":388,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":334,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":325,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":420,"Name":"internal.metrics.executorCpuTime","Value":33390843,"Internal":true,"Count Failed Values":true},{"ID":426,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":417,"Name":"internal.metrics.executorDeserializeTime","Value":8,"Internal":true,"Count Failed Values":true},{"ID":435,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":384,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":393,"Name":"duration","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":333,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":383,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":437,"Name":"internal.metrics.shuffle.write.writeTime","Value":686296,"Internal":true,"Count Failed Values":true},{"ID":419,"Name":"internal.metrics.executorRunTime","Value":49,"Internal":true,"Count Failed Values":true},{"ID":386,"Name":"time in aggregation build","Value":"32","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":335,"Name":"shuffle write time","Value":"686296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":389,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"StateStoreRDD","Scope":"{\"id\":\"73\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"StateStoreRDD","Scope":"{\"id\":\"69\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[33],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"70\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224179,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"2","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"7","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224256,"Failed":false,"Killed":false,"Accumulables":[{"ID":358,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":360,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":362,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":368,"Name":"time to update","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":369,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":370,"Name":"time to commit changes","Update":"32","Value":"32","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":372,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":373,"Name":"count of cache hit on states cache in provider","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":371,"Name":"memory used by state","Update":"400","Value":"400","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":375,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":377,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":379,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":459,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":458,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":457,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":456,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":455,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":454,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":453,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":451,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":445,"Name":"internal.metrics.executorCpuTime","Update":17230622,"Value":17230622,"Internal":true,"Count Failed Values":true},{"ID":444,"Name":"internal.metrics.executorRunTime","Update":56,"Value":56,"Internal":true,"Count Failed Values":true},{"ID":443,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5948051,"Value":5948051,"Internal":true,"Count Failed Values":true},{"ID":442,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5948051,"Executor Run Time":56,"Executor CPU Time":17230622,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1596020224187,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224257,"Failed":false,"Killed":false,"Accumulables":[{"ID":358,"Name":"duration","Update":"4","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":363,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":359,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":360,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":362,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":368,"Name":"time to update","Update":"21","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":367,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":369,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":370,"Name":"time to commit changes","Update":"18","Value":"50","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":372,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":364,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":373,"Name":"count of cache hit on states cache in provider","Update":"4","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":371,"Name":"memory used by state","Update":"784","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":366,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":375,"Name":"duration","Update":"22","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":376,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":377,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":379,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":327,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":330,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":332,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":459,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":458,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":457,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":456,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":455,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":454,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":453,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":451,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":445,"Name":"internal.metrics.executorCpuTime","Update":23808555,"Value":41039177,"Internal":true,"Count Failed Values":true},{"ID":444,"Name":"internal.metrics.executorRunTime","Update":56,"Value":112,"Internal":true,"Count Failed Values":true},{"ID":443,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6247106,"Value":12195157,"Internal":true,"Count Failed Values":true},{"ID":442,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":12,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6247106,"Executor Run Time":56,"Executor CPU Time":23808555,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"StateStoreRDD","Scope":"{\"id\":\"73\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"StateStoreRDD","Scope":"{\"id\":\"69\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[33],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"70\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"74\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[4],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224179,"Completion Time":1596020224259,"Accumulables":[{"ID":442,"Name":"internal.metrics.executorDeserializeTime","Value":12,"Internal":true,"Count Failed Values":true},{"ID":451,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":445,"Name":"internal.metrics.executorCpuTime","Value":41039177,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":454,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"count of cache hit on states cache in provider","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":367,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":376,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":358,"Name":"duration","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":457,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":370,"Name":"time to commit changes","Value":"50","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":456,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":369,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":459,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":360,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":453,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":372,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":363,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":327,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":444,"Name":"internal.metrics.executorRunTime","Value":112,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"duration","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":366,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":330,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":443,"Name":"internal.metrics.executorDeserializeCpuTime","Value":12195157,"Internal":true,"Count Failed Values":true},{"ID":455,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":332,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":377,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":359,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":458,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"time to update","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":362,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":371,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1596020224259,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":8,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 2","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#74, count#75]\nArguments: [value#74, count#75]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#74, count#75]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":467,"metricType":"sum"}]},"time":1596020224278} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":8,"time":1596020224287} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":7,"time":1596020224287} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":6,"time":1596020224288} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:03.793Z","batchId":2,"batchDuration":522,"durationMs":{"triggerExecution":522,"queryPlanning":41,"getBatch":1,"latestOffset":3,"addBatch":421,"walCommit":27},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":8,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48642}}","endOffset":"{\"test5\":{\"0\":48705}}","numInputRows":63,"inputRowsPerSecond":100.8,"processedRowsPerSecond":120.6896551724138}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":9,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48705}}, {\"test5\":{\"0\":48757}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#835]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 8bb5d8a6-42f8-4141-8f25-e1b98f81aac4, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 8bb5d8a6-42f8-4141-8f25-e1b98f81aac4, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#759]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":551,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":550,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":547,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":548,"metricType":"timing"},{"name":"peak memory","accumulatorId":546,"metricType":"size"},{"name":"number of output rows","accumulatorId":545,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":549,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":542,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":543,"metricType":"timing"},{"name":"peak memory","accumulatorId":541,"metricType":"size"},{"name":"number of output rows","accumulatorId":540,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":544,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":539,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":491,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":492,"metricType":"nsTiming"},{"name":"records read","accumulatorId":489,"metricType":"sum"},{"name":"local bytes read","accumulatorId":487,"metricType":"size"},{"name":"fetch wait time","accumulatorId":488,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":485,"metricType":"size"},{"name":"local blocks read","accumulatorId":484,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":483,"metricType":"sum"},{"name":"data size","accumulatorId":482,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":486,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":490,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":538,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":535,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":536,"metricType":"timing"},{"name":"peak memory","accumulatorId":534,"metricType":"size"},{"name":"number of output rows","accumulatorId":533,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":537,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":532,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":522,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":523,"metricType":"sum"},{"name":"memory used by state","accumulatorId":528,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":530,"metricType":"sum"},{"name":"number of output rows","accumulatorId":521,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":529,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":531,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":527,"metricType":"timing"},{"name":"time to remove","accumulatorId":526,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":524,"metricType":"sum"},{"name":"time to update","accumulatorId":525,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":518,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":519,"metricType":"timing"},{"name":"peak memory","accumulatorId":517,"metricType":"size"},{"name":"number of output rows","accumulatorId":516,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":520,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":515,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224419} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":10,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48705}}, {\"test5\":{\"0\":48757}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#911]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 29402d2a-a5da-4bb1-8d1a-c6d1c2d998d5, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 29402d2a-a5da-4bb1-8d1a-c6d1c2d998d5, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@59b7c509","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 3, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#759]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":551,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":550,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":547,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":548,"metricType":"timing"},{"name":"peak memory","accumulatorId":546,"metricType":"size"},{"name":"number of output rows","accumulatorId":545,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":549,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":542,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":543,"metricType":"timing"},{"name":"peak memory","accumulatorId":541,"metricType":"size"},{"name":"number of output rows","accumulatorId":540,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":544,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":539,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":491,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":492,"metricType":"nsTiming"},{"name":"records read","accumulatorId":489,"metricType":"sum"},{"name":"local bytes read","accumulatorId":487,"metricType":"size"},{"name":"fetch wait time","accumulatorId":488,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":485,"metricType":"size"},{"name":"local blocks read","accumulatorId":484,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":483,"metricType":"sum"},{"name":"data size","accumulatorId":482,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":486,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":490,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":538,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":535,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":536,"metricType":"timing"},{"name":"peak memory","accumulatorId":534,"metricType":"size"},{"name":"number of output rows","accumulatorId":533,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":537,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":532,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":522,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":523,"metricType":"sum"},{"name":"memory used by state","accumulatorId":528,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":530,"metricType":"sum"},{"name":"number of output rows","accumulatorId":521,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":529,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":531,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":527,"metricType":"timing"},{"name":"time to remove","accumulatorId":526,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":524,"metricType":"sum"},{"name":"time to update","accumulatorId":525,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":518,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":519,"metricType":"timing"},{"name":"peak memory","accumulatorId":517,"metricType":"size"},{"name":"number of output rows","accumulatorId":516,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":520,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":515,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224452} +{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1596020224533,"Stage Infos":[{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":42,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[41],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"115\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[37],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"DataSourceRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"108\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"113\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"114\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"99\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"StateStoreRDD","Scope":"{\"id\":\"102\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[44],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":44,"Name":"StateStoreRDD","Scope":"{\"id\":\"106\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[6],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[6,7],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"3","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"10","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":42,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[41],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"115\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[37],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"DataSourceRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"108\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"113\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"114\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224535,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"3","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"10","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1596020224541,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1596020224541,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224581,"Failed":false,"Killed":false,"Accumulables":[{"ID":492,"Name":"shuffle write time","Update":"643278","Value":"643278","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":491,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":490,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":482,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":539,"Name":"duration","Update":"20","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":540,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":541,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":543,"Name":"time in aggregation build","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":545,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":546,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":548,"Name":"time in aggregation build","Update":"9","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":550,"Name":"duration","Update":"20","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":551,"Name":"number of output rows","Update":"52","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":596,"Name":"internal.metrics.input.recordsRead","Update":52,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":594,"Name":"internal.metrics.shuffle.write.writeTime","Update":643278,"Value":643278,"Internal":true,"Count Failed Values":true},{"ID":593,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":592,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":583,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":578,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":577,"Name":"internal.metrics.executorCpuTime","Update":29099071,"Value":29099071,"Internal":true,"Count Failed Values":true},{"ID":576,"Name":"internal.metrics.executorRunTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true},{"ID":575,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3091128,"Value":3091128,"Internal":true,"Count Failed Values":true},{"ID":574,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3091128,"Executor Run Time":29,"Executor CPU Time":29099071,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":643278,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":52},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":42,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[41],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"115\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[37],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"DataSourceRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"108\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"113\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"114\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224535,"Completion Time":1596020224582,"Accumulables":[{"ID":550,"Name":"duration","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":541,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":577,"Name":"internal.metrics.executorCpuTime","Value":29099071,"Internal":true,"Count Failed Values":true},{"ID":490,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":576,"Name":"internal.metrics.executorRunTime","Value":29,"Internal":true,"Count Failed Values":true},{"ID":540,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":594,"Name":"internal.metrics.shuffle.write.writeTime","Value":643278,"Internal":true,"Count Failed Values":true},{"ID":543,"Name":"time in aggregation build","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":492,"Name":"shuffle write time","Value":"643278","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":546,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":539,"Name":"duration","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":575,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3091128,"Internal":true,"Count Failed Values":true},{"ID":593,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":548,"Name":"time in aggregation build","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":578,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":596,"Name":"internal.metrics.input.recordsRead","Value":52,"Internal":true,"Count Failed Values":true},{"ID":551,"Name":"number of output rows","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":482,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":491,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":545,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":592,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":574,"Name":"internal.metrics.executorDeserializeTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":583,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"99\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"StateStoreRDD","Scope":"{\"id\":\"102\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[44],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":44,"Name":"StateStoreRDD","Scope":"{\"id\":\"106\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[6],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224588,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"3","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"10","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":0,"Attempt":0,"Launch Time":1596020224596,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":1,"Attempt":0,"Launch Time":1596020224597,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":0,"Attempt":0,"Launch Time":1596020224596,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224670,"Failed":false,"Killed":false,"Accumulables":[{"ID":515,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":517,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":519,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":525,"Name":"time to update","Update":"5","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":526,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":527,"Name":"time to commit changes","Update":"27","Value":"27","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":529,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":530,"Name":"count of cache hit on states cache in provider","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":528,"Name":"memory used by state","Update":"400","Value":"400","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":532,"Name":"duration","Update":"5","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":534,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":536,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":616,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":615,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":614,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":613,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":612,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":611,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":610,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":608,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":603,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":602,"Name":"internal.metrics.executorCpuTime","Update":19967906,"Value":19967906,"Internal":true,"Count Failed Values":true},{"ID":601,"Name":"internal.metrics.executorRunTime","Update":62,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":600,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4899567,"Value":4899567,"Internal":true,"Count Failed Values":true},{"ID":599,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":4,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4899567,"Executor Run Time":62,"Executor CPU Time":19967906,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":1,"Attempt":0,"Launch Time":1596020224597,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224687,"Failed":false,"Killed":false,"Accumulables":[{"ID":515,"Name":"duration","Update":"4","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":520,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":516,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":517,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":519,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":525,"Name":"time to update","Update":"17","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":524,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":526,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":527,"Name":"time to commit changes","Update":"26","Value":"53","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":529,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":521,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":530,"Name":"count of cache hit on states cache in provider","Update":"6","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":528,"Name":"memory used by state","Update":"784","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":523,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":532,"Name":"duration","Update":"17","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":533,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":534,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":536,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":538,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":484,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":488,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":487,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":489,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":616,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":615,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":614,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":613,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":612,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":611,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":610,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":608,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":603,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":602,"Name":"internal.metrics.executorCpuTime","Update":22402538,"Value":42370444,"Internal":true,"Count Failed Values":true},{"ID":601,"Name":"internal.metrics.executorRunTime","Update":79,"Value":141,"Internal":true,"Count Failed Values":true},{"ID":600,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4671511,"Value":9571078,"Internal":true,"Count Failed Values":true},{"ID":599,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":8,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4671511,"Executor Run Time":79,"Executor CPU Time":22402538,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"99\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"StateStoreRDD","Scope":"{\"id\":\"102\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[44],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":44,"Name":"StateStoreRDD","Scope":"{\"id\":\"106\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"107\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[6],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224588,"Completion Time":1596020224688,"Accumulables":[{"ID":523,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":532,"Name":"duration","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":487,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":517,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":600,"Name":"internal.metrics.executorDeserializeCpuTime","Value":9571078,"Internal":true,"Count Failed Values":true},{"ID":603,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":612,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":516,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":615,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":534,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":525,"Name":"time to update","Value":"22","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":489,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":528,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":519,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":608,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":599,"Name":"internal.metrics.executorDeserializeTime","Value":8,"Internal":true,"Count Failed Values":true},{"ID":521,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":530,"Name":"count of cache hit on states cache in provider","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":611,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":602,"Name":"internal.metrics.executorCpuTime","Value":42370444,"Internal":true,"Count Failed Values":true},{"ID":488,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":515,"Name":"duration","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":524,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":533,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":614,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":536,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":527,"Name":"time to commit changes","Value":"53","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":613,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":616,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":526,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":520,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":610,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":601,"Name":"internal.metrics.executorRunTime","Value":141,"Internal":true,"Count Failed Values":true},{"ID":484,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":538,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":529,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1596020224689,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":11,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 3","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#88, count#89]\nArguments: [value#88, count#89]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#88, count#89]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":624,"metricType":"sum"}]},"time":1596020224709} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":11,"time":1596020224713} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":10,"time":1596020224714} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":9,"time":1596020224714} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:04.317Z","batchId":3,"batchDuration":415,"durationMs":{"triggerExecution":415,"queryPlanning":38,"getBatch":1,"latestOffset":3,"addBatch":332,"walCommit":21},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":12,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48705}}","endOffset":"{\"test5\":{\"0\":48757}}","numInputRows":52,"inputRowsPerSecond":99.23664122137404,"processedRowsPerSecond":125.30120481927712}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":12,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48757}}, {\"test5\":{\"0\":48799}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1066]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 42efe357-12ef-4061-9b83-20bf4c29a257, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 42efe357-12ef-4061-9b83-20bf4c29a257, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#990]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":708,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":707,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":704,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":705,"metricType":"timing"},{"name":"peak memory","accumulatorId":703,"metricType":"size"},{"name":"number of output rows","accumulatorId":702,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":706,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":699,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":700,"metricType":"timing"},{"name":"peak memory","accumulatorId":698,"metricType":"size"},{"name":"number of output rows","accumulatorId":697,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":701,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":696,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":648,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":649,"metricType":"nsTiming"},{"name":"records read","accumulatorId":646,"metricType":"sum"},{"name":"local bytes read","accumulatorId":644,"metricType":"size"},{"name":"fetch wait time","accumulatorId":645,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":642,"metricType":"size"},{"name":"local blocks read","accumulatorId":641,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":640,"metricType":"sum"},{"name":"data size","accumulatorId":639,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":643,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":647,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":695,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":692,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":693,"metricType":"timing"},{"name":"peak memory","accumulatorId":691,"metricType":"size"},{"name":"number of output rows","accumulatorId":690,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":694,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":689,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":679,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":680,"metricType":"sum"},{"name":"memory used by state","accumulatorId":685,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":687,"metricType":"sum"},{"name":"number of output rows","accumulatorId":678,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":686,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":688,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":684,"metricType":"timing"},{"name":"time to remove","accumulatorId":683,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":681,"metricType":"sum"},{"name":"time to update","accumulatorId":682,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":675,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":676,"metricType":"timing"},{"name":"peak memory","accumulatorId":674,"metricType":"size"},{"name":"number of output rows","accumulatorId":673,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":677,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":672,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224817} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":13,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48757}}, {\"test5\":{\"0\":48799}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1142]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6fa28bd2-2924-4e01-8bbe-128888d2669b, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6fa28bd2-2924-4e01-8bbe-128888d2669b, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@1717338b","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 4, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#990]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":708,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":707,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":704,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":705,"metricType":"timing"},{"name":"peak memory","accumulatorId":703,"metricType":"size"},{"name":"number of output rows","accumulatorId":702,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":706,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":699,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":700,"metricType":"timing"},{"name":"peak memory","accumulatorId":698,"metricType":"size"},{"name":"number of output rows","accumulatorId":697,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":701,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":696,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":648,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":649,"metricType":"nsTiming"},{"name":"records read","accumulatorId":646,"metricType":"sum"},{"name":"local bytes read","accumulatorId":644,"metricType":"size"},{"name":"fetch wait time","accumulatorId":645,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":642,"metricType":"size"},{"name":"local blocks read","accumulatorId":641,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":640,"metricType":"sum"},{"name":"data size","accumulatorId":639,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":643,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":647,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":695,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":692,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":693,"metricType":"timing"},{"name":"peak memory","accumulatorId":691,"metricType":"size"},{"name":"number of output rows","accumulatorId":690,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":694,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":689,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":679,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":680,"metricType":"sum"},{"name":"memory used by state","accumulatorId":685,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":687,"metricType":"sum"},{"name":"number of output rows","accumulatorId":678,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":686,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":688,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":684,"metricType":"timing"},{"name":"time to remove","accumulatorId":683,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":681,"metricType":"sum"},{"name":"time to update","accumulatorId":682,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":675,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":676,"metricType":"timing"},{"name":"peak memory","accumulatorId":674,"metricType":"size"},{"name":"number of output rows","accumulatorId":673,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":677,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":672,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020224849} +{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1596020224928,"Stage Infos":[{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"132\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":56,"Name":"StateStoreRDD","Scope":"{\"id\":\"139\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"136\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[56],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"StateStoreRDD","Scope":"{\"id\":\"135\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[8],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"141\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[48],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"148\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":48,"Name":"DataSourceRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[9,8],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"4","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"13","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"141\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[48],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"148\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":48,"Name":"DataSourceRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224929,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"4","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"13","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1596020224941,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":0,"Attempt":0,"Launch Time":1596020224941,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020224979,"Failed":false,"Killed":false,"Accumulables":[{"ID":649,"Name":"shuffle write time","Update":"572754","Value":"572754","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":648,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":647,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":639,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":696,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":697,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":698,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":700,"Name":"time in aggregation build","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":702,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":703,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":705,"Name":"time in aggregation build","Update":"9","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":707,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":708,"Name":"number of output rows","Update":"42","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":753,"Name":"internal.metrics.input.recordsRead","Update":42,"Value":42,"Internal":true,"Count Failed Values":true},{"ID":751,"Name":"internal.metrics.shuffle.write.writeTime","Update":572754,"Value":572754,"Internal":true,"Count Failed Values":true},{"ID":750,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":749,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":740,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":735,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":734,"Name":"internal.metrics.executorCpuTime","Update":27800373,"Value":27800373,"Internal":true,"Count Failed Values":true},{"ID":733,"Name":"internal.metrics.executorRunTime","Update":28,"Value":28,"Internal":true,"Count Failed Values":true},{"ID":732,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4768103,"Value":4768103,"Internal":true,"Count Failed Values":true},{"ID":731,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":4,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4768103,"Executor Run Time":28,"Executor CPU Time":27800373,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":572754,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":42},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"141\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[48],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"148\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":48,"Name":"DataSourceRDD","Scope":"{\"id\":\"152\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224929,"Completion Time":1596020224979,"Accumulables":[{"ID":732,"Name":"internal.metrics.executorDeserializeCpuTime","Value":4768103,"Internal":true,"Count Failed Values":true},{"ID":696,"Name":"duration","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":750,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":705,"Name":"time in aggregation build","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":735,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":708,"Name":"number of output rows","Value":"42","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":753,"Name":"internal.metrics.input.recordsRead","Value":42,"Internal":true,"Count Failed Values":true},{"ID":648,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":639,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":702,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":740,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":731,"Name":"internal.metrics.executorDeserializeTime","Value":4,"Internal":true,"Count Failed Values":true},{"ID":749,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":698,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":734,"Name":"internal.metrics.executorCpuTime","Value":27800373,"Internal":true,"Count Failed Values":true},{"ID":707,"Name":"duration","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":647,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":733,"Name":"internal.metrics.executorRunTime","Value":28,"Internal":true,"Count Failed Values":true},{"ID":697,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":751,"Name":"internal.metrics.shuffle.write.writeTime","Value":572754,"Internal":true,"Count Failed Values":true},{"ID":700,"Name":"time in aggregation build","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":649,"Name":"shuffle write time","Value":"572754","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":703,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"132\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":56,"Name":"StateStoreRDD","Scope":"{\"id\":\"139\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"136\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[56],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"StateStoreRDD","Scope":"{\"id\":\"135\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[8],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224987,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"4","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"13","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":1,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":1,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225056,"Failed":false,"Killed":false,"Accumulables":[{"ID":672,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":677,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":673,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":674,"Name":"peak memory","Update":"4456448","Value":"4456448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":676,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"time to update","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":681,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":683,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":684,"Name":"time to commit changes","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":686,"Name":"estimated size of state only on current version","Update":"368","Value":"368","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":678,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":687,"Name":"count of cache hit on states cache in provider","Update":"8","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":685,"Name":"memory used by state","Update":"784","Value":"784","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":680,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":689,"Name":"duration","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":690,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":691,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":693,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":695,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":641,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":645,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":644,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":646,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":773,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":771,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":770,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":769,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":768,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":767,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":765,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":4718592,"Internal":true,"Count Failed Values":true},{"ID":760,"Name":"internal.metrics.resultSize","Update":5574,"Value":5574,"Internal":true,"Count Failed Values":true},{"ID":759,"Name":"internal.metrics.executorCpuTime","Update":19548688,"Value":19548688,"Internal":true,"Count Failed Values":true},{"ID":758,"Name":"internal.metrics.executorRunTime","Update":52,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":757,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5622533,"Value":5622533,"Internal":true,"Count Failed Values":true},{"ID":756,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":5,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5622533,"Executor Run Time":52,"Executor CPU Time":19548688,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1596020224994,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225058,"Failed":false,"Killed":false,"Accumulables":[{"ID":672,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":674,"Name":"peak memory","Update":"262144","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":676,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"time to update","Update":"4","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":683,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":684,"Name":"time to commit changes","Update":"35","Value":"46","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":686,"Name":"estimated size of state only on current version","Update":"88","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":687,"Name":"count of cache hit on states cache in provider","Update":"8","Value":"16","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":685,"Name":"memory used by state","Update":"400","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":689,"Name":"duration","Update":"4","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":691,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":693,"Name":"time in aggregation build","Update":"0","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":773,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":771,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":770,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":769,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":768,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":767,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":765,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":760,"Name":"internal.metrics.resultSize","Update":5311,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":759,"Name":"internal.metrics.executorCpuTime","Update":16813539,"Value":36362227,"Internal":true,"Count Failed Values":true},{"ID":758,"Name":"internal.metrics.executorRunTime","Update":55,"Value":107,"Internal":true,"Count Failed Values":true},{"ID":757,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4322992,"Value":9945525,"Internal":true,"Count Failed Values":true},{"ID":756,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":9,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4322992,"Executor Run Time":55,"Executor CPU Time":16813539,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"132\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"140\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":56,"Name":"StateStoreRDD","Scope":"{\"id\":\"139\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"136\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[56],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"StateStoreRDD","Scope":"{\"id\":\"135\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[8],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020224987,"Completion Time":1596020225059,"Accumulables":[{"ID":765,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":756,"Name":"internal.metrics.executorDeserializeTime","Value":9,"Internal":true,"Count Failed Values":true},{"ID":678,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":759,"Name":"internal.metrics.executorCpuTime","Value":36362227,"Internal":true,"Count Failed Values":true},{"ID":768,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":687,"Name":"count of cache hit on states cache in provider","Value":"16","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":681,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":771,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":690,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":672,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":645,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":684,"Name":"time to commit changes","Value":"46","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":693,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":770,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":683,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":773,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":686,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":695,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":677,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":767,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":641,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":758,"Name":"internal.metrics.executorRunTime","Value":107,"Internal":true,"Count Failed Values":true},{"ID":644,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":680,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":689,"Name":"duration","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":674,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":757,"Name":"internal.metrics.executorDeserializeCpuTime","Value":9945525,"Internal":true,"Count Failed Values":true},{"ID":769,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":760,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":646,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"time to update","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":691,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":673,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":676,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":685,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1596020225059,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":14,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 4","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#102, count#103]\nArguments: [value#102, count#103]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#102, count#103]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":781,"metricType":"sum"}]},"time":1596020225079} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":14,"time":1596020225087} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":13,"time":1596020225087} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":12,"time":1596020225087} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:04.734Z","batchId":4,"batchDuration":387,"durationMs":{"triggerExecution":387,"queryPlanning":30,"getBatch":1,"latestOffset":3,"addBatch":306,"walCommit":12},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":16,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48757}}","endOffset":"{\"test5\":{\"0\":48799}}","numInputRows":42,"inputRowsPerSecond":100.71942446043165,"processedRowsPerSecond":108.52713178294573}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":15,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48799}}, {\"test5\":{\"0\":48837}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1297]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 9579cc6c-8827-43f7-9678-7747602e493e, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 9579cc6c-8827-43f7-9678-7747602e493e, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1221]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":865,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":864,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":861,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":862,"metricType":"timing"},{"name":"peak memory","accumulatorId":860,"metricType":"size"},{"name":"number of output rows","accumulatorId":859,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":863,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":856,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":857,"metricType":"timing"},{"name":"peak memory","accumulatorId":855,"metricType":"size"},{"name":"number of output rows","accumulatorId":854,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":858,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":853,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":805,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":806,"metricType":"nsTiming"},{"name":"records read","accumulatorId":803,"metricType":"sum"},{"name":"local bytes read","accumulatorId":801,"metricType":"size"},{"name":"fetch wait time","accumulatorId":802,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":799,"metricType":"size"},{"name":"local blocks read","accumulatorId":798,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":797,"metricType":"sum"},{"name":"data size","accumulatorId":796,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":800,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":804,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":852,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":849,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":850,"metricType":"timing"},{"name":"peak memory","accumulatorId":848,"metricType":"size"},{"name":"number of output rows","accumulatorId":847,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":851,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":846,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":836,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":837,"metricType":"sum"},{"name":"memory used by state","accumulatorId":842,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":844,"metricType":"sum"},{"name":"number of output rows","accumulatorId":835,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":843,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":845,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":841,"metricType":"timing"},{"name":"time to remove","accumulatorId":840,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":838,"metricType":"sum"},{"name":"time to update","accumulatorId":839,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":832,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":833,"metricType":"timing"},{"name":"peak memory","accumulatorId":831,"metricType":"size"},{"name":"number of output rows","accumulatorId":830,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":834,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":829,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225211} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":16,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48799}}, {\"test5\":{\"0\":48837}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1373]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = b800d96e-7584-4e8d-8df8-c9b901b7f2e2, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = b800d96e-7584-4e8d-8df8-c9b901b7f2e2, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@2c214312","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 5, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1221]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":865,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":864,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":861,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":862,"metricType":"timing"},{"name":"peak memory","accumulatorId":860,"metricType":"size"},{"name":"number of output rows","accumulatorId":859,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":863,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":856,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":857,"metricType":"timing"},{"name":"peak memory","accumulatorId":855,"metricType":"size"},{"name":"number of output rows","accumulatorId":854,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":858,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":853,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":805,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":806,"metricType":"nsTiming"},{"name":"records read","accumulatorId":803,"metricType":"sum"},{"name":"local bytes read","accumulatorId":801,"metricType":"size"},{"name":"fetch wait time","accumulatorId":802,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":799,"metricType":"size"},{"name":"local blocks read","accumulatorId":798,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":797,"metricType":"sum"},{"name":"data size","accumulatorId":796,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":800,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":804,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":852,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":849,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":850,"metricType":"timing"},{"name":"peak memory","accumulatorId":848,"metricType":"size"},{"name":"number of output rows","accumulatorId":847,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":851,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":846,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":836,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":837,"metricType":"sum"},{"name":"memory used by state","accumulatorId":842,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":844,"metricType":"sum"},{"name":"number of output rows","accumulatorId":835,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":843,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":845,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":841,"metricType":"timing"},{"name":"time to remove","accumulatorId":840,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":838,"metricType":"sum"},{"name":"time to update","accumulatorId":839,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":832,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":833,"metricType":"timing"},{"name":"peak memory","accumulatorId":831,"metricType":"size"},{"name":"number of output rows","accumulatorId":830,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":834,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":829,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225270} +{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1596020225342,"Stage Infos":[{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"181\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[63],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[60],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"174\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":60,"Name":"DataSourceRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"180\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"165\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"StateStoreRDD","Scope":"{\"id\":\"168\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"StateStoreRDD","Scope":"{\"id\":\"172\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[10],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[10,11],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"5","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"16","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"181\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[63],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[60],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"174\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":60,"Name":"DataSourceRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"180\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225343,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"5","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"16","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":0,"Attempt":0,"Launch Time":1596020225359,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":0,"Attempt":0,"Launch Time":1596020225359,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225400,"Failed":false,"Killed":false,"Accumulables":[{"ID":806,"Name":"shuffle write time","Update":"530930","Value":"530930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":805,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":804,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":796,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":853,"Name":"duration","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":854,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":855,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":857,"Name":"time in aggregation build","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":859,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":860,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":862,"Name":"time in aggregation build","Update":"9","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":864,"Name":"duration","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":865,"Name":"number of output rows","Update":"38","Value":"38","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":910,"Name":"internal.metrics.input.recordsRead","Update":38,"Value":38,"Internal":true,"Count Failed Values":true},{"ID":908,"Name":"internal.metrics.shuffle.write.writeTime","Update":530930,"Value":530930,"Internal":true,"Count Failed Values":true},{"ID":907,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":906,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":897,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":892,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":891,"Name":"internal.metrics.executorCpuTime","Update":22440089,"Value":22440089,"Internal":true,"Count Failed Values":true},{"ID":890,"Name":"internal.metrics.executorRunTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true},{"ID":889,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6808170,"Value":6808170,"Internal":true,"Count Failed Values":true},{"ID":888,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6808170,"Executor Run Time":29,"Executor CPU Time":22440089,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":530930,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":38},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"181\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[63],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[60],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"174\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":60,"Name":"DataSourceRDD","Scope":"{\"id\":\"185\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"180\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225343,"Completion Time":1596020225401,"Accumulables":[{"ID":855,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":891,"Name":"internal.metrics.executorCpuTime","Value":22440089,"Internal":true,"Count Failed Values":true},{"ID":864,"Name":"duration","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":804,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":908,"Name":"internal.metrics.shuffle.write.writeTime","Value":530930,"Internal":true,"Count Failed Values":true},{"ID":890,"Name":"internal.metrics.executorRunTime","Value":29,"Internal":true,"Count Failed Values":true},{"ID":857,"Name":"time in aggregation build","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":860,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":806,"Name":"shuffle write time","Value":"530930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":854,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":853,"Name":"duration","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":862,"Name":"time in aggregation build","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":889,"Name":"internal.metrics.executorDeserializeCpuTime","Value":6808170,"Internal":true,"Count Failed Values":true},{"ID":907,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":892,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":910,"Name":"internal.metrics.input.recordsRead","Value":38,"Internal":true,"Count Failed Values":true},{"ID":865,"Name":"number of output rows","Value":"38","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":805,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":796,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":859,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":888,"Name":"internal.metrics.executorDeserializeTime","Value":6,"Internal":true,"Count Failed Values":true},{"ID":897,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":906,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"165\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"StateStoreRDD","Scope":"{\"id\":\"168\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"StateStoreRDD","Scope":"{\"id\":\"172\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[10],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225410,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"5","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"16","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225498,"Failed":false,"Killed":false,"Accumulables":[{"ID":829,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":834,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":830,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":831,"Name":"peak memory","Update":"4456448","Value":"4456448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":833,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":839,"Name":"time to update","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":838,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":840,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":841,"Name":"time to commit changes","Update":"37","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":843,"Name":"estimated size of state only on current version","Update":"368","Value":"368","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":835,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":844,"Name":"count of cache hit on states cache in provider","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":842,"Name":"memory used by state","Update":"784","Value":"784","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":837,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":846,"Name":"duration","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":847,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":848,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":850,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":852,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":798,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":802,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":801,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":803,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":930,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":929,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":928,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":927,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":926,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":925,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":924,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":922,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":4718592,"Internal":true,"Count Failed Values":true},{"ID":917,"Name":"internal.metrics.resultSize","Update":5574,"Value":5574,"Internal":true,"Count Failed Values":true},{"ID":916,"Name":"internal.metrics.executorCpuTime","Update":17945299,"Value":17945299,"Internal":true,"Count Failed Values":true},{"ID":915,"Name":"internal.metrics.executorRunTime","Update":68,"Value":68,"Internal":true,"Count Failed Values":true},{"ID":914,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3451032,"Value":3451032,"Internal":true,"Count Failed Values":true},{"ID":913,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3451032,"Executor Run Time":68,"Executor CPU Time":17945299,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1596020225417,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225509,"Failed":false,"Killed":false,"Accumulables":[{"ID":829,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":831,"Name":"peak memory","Update":"262144","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":833,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":839,"Name":"time to update","Update":"4","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":840,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":841,"Name":"time to commit changes","Update":"50","Value":"87","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":843,"Name":"estimated size of state only on current version","Update":"88","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":844,"Name":"count of cache hit on states cache in provider","Update":"10","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":842,"Name":"memory used by state","Update":"400","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":846,"Name":"duration","Update":"4","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":848,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":850,"Name":"time in aggregation build","Update":"0","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":930,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":929,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":928,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":927,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":926,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":925,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":924,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":922,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":917,"Name":"internal.metrics.resultSize","Update":5311,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":916,"Name":"internal.metrics.executorCpuTime","Update":15599091,"Value":33544390,"Internal":true,"Count Failed Values":true},{"ID":915,"Name":"internal.metrics.executorRunTime","Update":84,"Value":152,"Internal":true,"Count Failed Values":true},{"ID":914,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4357806,"Value":7808838,"Internal":true,"Count Failed Values":true},{"ID":913,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":7,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4357806,"Executor Run Time":84,"Executor CPU Time":15599091,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"165\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"StateStoreRDD","Scope":"{\"id\":\"168\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"173\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"StateStoreRDD","Scope":"{\"id\":\"172\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[10],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225410,"Completion Time":1596020225514,"Accumulables":[{"ID":846,"Name":"duration","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":837,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":801,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":831,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":926,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":917,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":830,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":848,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":803,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":839,"Name":"time to update","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":929,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":833,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":842,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":914,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7808838,"Internal":true,"Count Failed Values":true},{"ID":922,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":913,"Name":"internal.metrics.executorDeserializeTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":925,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":844,"Name":"count of cache hit on states cache in provider","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":835,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":916,"Name":"internal.metrics.executorCpuTime","Value":33544390,"Internal":true,"Count Failed Values":true},{"ID":829,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":928,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":802,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":838,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":847,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":850,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":841,"Name":"time to commit changes","Value":"87","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":927,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":930,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":840,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":834,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":852,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":798,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":843,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":915,"Name":"internal.metrics.executorRunTime","Value":152,"Internal":true,"Count Failed Values":true},{"ID":924,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1596020225514,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":17,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 5","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#116, count#117]\nArguments: [value#116, count#117]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#116, count#117]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":938,"metricType":"sum"}]},"time":1596020225536} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":17,"time":1596020225541} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":16,"time":1596020225542} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":15,"time":1596020225542} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:05.123Z","batchId":5,"batchDuration":437,"durationMs":{"triggerExecution":437,"queryPlanning":35,"getBatch":1,"latestOffset":3,"addBatch":361,"walCommit":18},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":20,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48799}}","endOffset":"{\"test5\":{\"0\":48837}}","numInputRows":38,"inputRowsPerSecond":97.68637532133675,"processedRowsPerSecond":86.95652173913044}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":18,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48837}}, {\"test5\":{\"0\":48881}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1528]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6a12c2d9-8d02-4241-93fc-f53da01bb454, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 6a12c2d9-8d02-4241-93fc-f53da01bb454, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1452]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1022,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1021,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1018,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1019,"metricType":"timing"},{"name":"peak memory","accumulatorId":1017,"metricType":"size"},{"name":"number of output rows","accumulatorId":1016,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1020,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1013,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1014,"metricType":"timing"},{"name":"peak memory","accumulatorId":1012,"metricType":"size"},{"name":"number of output rows","accumulatorId":1011,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1015,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1010,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":962,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":963,"metricType":"nsTiming"},{"name":"records read","accumulatorId":960,"metricType":"sum"},{"name":"local bytes read","accumulatorId":958,"metricType":"size"},{"name":"fetch wait time","accumulatorId":959,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":956,"metricType":"size"},{"name":"local blocks read","accumulatorId":955,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":954,"metricType":"sum"},{"name":"data size","accumulatorId":953,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":957,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":961,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1009,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1006,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1007,"metricType":"timing"},{"name":"peak memory","accumulatorId":1005,"metricType":"size"},{"name":"number of output rows","accumulatorId":1004,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1008,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1003,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":993,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":994,"metricType":"sum"},{"name":"memory used by state","accumulatorId":999,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1001,"metricType":"sum"},{"name":"number of output rows","accumulatorId":992,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1000,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1002,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":998,"metricType":"timing"},{"name":"time to remove","accumulatorId":997,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":995,"metricType":"sum"},{"name":"time to update","accumulatorId":996,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":989,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":990,"metricType":"timing"},{"name":"peak memory","accumulatorId":988,"metricType":"size"},{"name":"number of output rows","accumulatorId":987,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":991,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":986,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225657} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":19,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48837}}, {\"test5\":{\"0\":48881}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1604]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 96456757-8d0b-46da-a006-9fe2cb6fc936, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = 96456757-8d0b-46da-a006-9fe2cb6fc936, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@27ec018d","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 6, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1452]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1022,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1021,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1018,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1019,"metricType":"timing"},{"name":"peak memory","accumulatorId":1017,"metricType":"size"},{"name":"number of output rows","accumulatorId":1016,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1020,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1013,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1014,"metricType":"timing"},{"name":"peak memory","accumulatorId":1012,"metricType":"size"},{"name":"number of output rows","accumulatorId":1011,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1015,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1010,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":962,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":963,"metricType":"nsTiming"},{"name":"records read","accumulatorId":960,"metricType":"sum"},{"name":"local bytes read","accumulatorId":958,"metricType":"size"},{"name":"fetch wait time","accumulatorId":959,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":956,"metricType":"size"},{"name":"local blocks read","accumulatorId":955,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":954,"metricType":"sum"},{"name":"data size","accumulatorId":953,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":957,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":961,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1009,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1006,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1007,"metricType":"timing"},{"name":"peak memory","accumulatorId":1005,"metricType":"size"},{"name":"number of output rows","accumulatorId":1004,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1008,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1003,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":993,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":994,"metricType":"sum"},{"name":"memory used by state","accumulatorId":999,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1001,"metricType":"sum"},{"name":"number of output rows","accumulatorId":992,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1000,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1002,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":998,"metricType":"timing"},{"name":"time to remove","accumulatorId":997,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":995,"metricType":"sum"},{"name":"time to update","accumulatorId":996,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":989,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":990,"metricType":"timing"},{"name":"peak memory","accumulatorId":988,"metricType":"size"},{"name":"number of output rows","accumulatorId":987,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":991,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":986,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225687} +{"Event":"SparkListenerJobStart","Job ID":6,"Submission Time":1596020225759,"Stage Infos":[{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":78,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[77],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":75,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"213\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[74],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":74,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"214\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[73],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":77,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"207\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[76],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":72,"Name":"DataSourceRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":73,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[72],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":76,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"212\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[75],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":83,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"198\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[82],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":81,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"202\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[80],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":79,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[78],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":80,"Name":"StateStoreRDD","Scope":"{\"id\":\"205\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[79],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":82,"Name":"StateStoreRDD","Scope":"{\"id\":\"201\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[81],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[12],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[12,13],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"6","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"19","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":78,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[77],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":75,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"213\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[74],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":74,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"214\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[73],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":77,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"207\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[76],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":72,"Name":"DataSourceRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":73,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[72],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":76,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"212\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[75],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225760,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"6","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"19","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":0,"Attempt":0,"Launch Time":1596020225766,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":0,"Attempt":0,"Launch Time":1596020225766,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225796,"Failed":false,"Killed":false,"Accumulables":[{"ID":963,"Name":"shuffle write time","Update":"543836","Value":"543836","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":962,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":961,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":953,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1010,"Name":"duration","Update":"17","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1011,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1012,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1014,"Name":"time in aggregation build","Update":"11","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1016,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1017,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1019,"Name":"time in aggregation build","Update":"8","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1021,"Name":"duration","Update":"17","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1022,"Name":"number of output rows","Update":"44","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1067,"Name":"internal.metrics.input.recordsRead","Update":44,"Value":44,"Internal":true,"Count Failed Values":true},{"ID":1065,"Name":"internal.metrics.shuffle.write.writeTime","Update":543836,"Value":543836,"Internal":true,"Count Failed Values":true},{"ID":1064,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1063,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1054,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1049,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1048,"Name":"internal.metrics.executorCpuTime","Update":23733439,"Value":23733439,"Internal":true,"Count Failed Values":true},{"ID":1047,"Name":"internal.metrics.executorRunTime","Update":23,"Value":23,"Internal":true,"Count Failed Values":true},{"ID":1046,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3714406,"Value":3714406,"Internal":true,"Count Failed Values":true},{"ID":1045,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3714406,"Executor Run Time":23,"Executor CPU Time":23733439,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":543836,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":44},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":78,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[77],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":75,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"213\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[74],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":74,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"214\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[73],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":77,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"207\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[76],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":72,"Name":"DataSourceRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":73,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"218\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[72],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":76,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"212\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[75],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225760,"Completion Time":1596020225797,"Accumulables":[{"ID":1064,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1010,"Name":"duration","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1046,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3714406,"Internal":true,"Count Failed Values":true},{"ID":1019,"Name":"time in aggregation build","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1067,"Name":"internal.metrics.input.recordsRead","Value":44,"Internal":true,"Count Failed Values":true},{"ID":1022,"Name":"number of output rows","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1049,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1016,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":962,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":953,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1054,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1045,"Name":"internal.metrics.executorDeserializeTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":1063,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true},{"ID":1048,"Name":"internal.metrics.executorCpuTime","Value":23733439,"Internal":true,"Count Failed Values":true},{"ID":1012,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1021,"Name":"duration","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":961,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1065,"Name":"internal.metrics.shuffle.write.writeTime","Value":543836,"Internal":true,"Count Failed Values":true},{"ID":1047,"Name":"internal.metrics.executorRunTime","Value":23,"Internal":true,"Count Failed Values":true},{"ID":1014,"Name":"time in aggregation build","Value":"11","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":963,"Name":"shuffle write time","Value":"543836","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1017,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1011,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":83,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"198\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[82],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":81,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"202\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[80],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":79,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[78],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":80,"Name":"StateStoreRDD","Scope":"{\"id\":\"205\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[79],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":82,"Name":"StateStoreRDD","Scope":"{\"id\":\"201\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[81],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[12],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225801,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"6","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"19","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":0,"Attempt":0,"Launch Time":1596020225808,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":1,"Attempt":0,"Launch Time":1596020225809,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":0,"Attempt":0,"Launch Time":1596020225808,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225868,"Failed":false,"Killed":false,"Accumulables":[{"ID":986,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":988,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":990,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":996,"Name":"time to update","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":997,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":998,"Name":"time to commit changes","Update":"26","Value":"26","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1000,"Name":"estimated size of state only on current version","Update":"88","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1001,"Name":"count of cache hit on states cache in provider","Update":"12","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":999,"Name":"memory used by state","Update":"400","Value":"400","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1003,"Name":"duration","Update":"4","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1005,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1007,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1087,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1086,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1085,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1084,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1083,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1082,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1081,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1079,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1074,"Name":"internal.metrics.resultSize","Update":5311,"Value":5311,"Internal":true,"Count Failed Values":true},{"ID":1073,"Name":"internal.metrics.executorCpuTime","Update":17503528,"Value":17503528,"Internal":true,"Count Failed Values":true},{"ID":1072,"Name":"internal.metrics.executorRunTime","Update":50,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":1071,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4255703,"Value":4255703,"Internal":true,"Count Failed Values":true},{"ID":1070,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":4,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4255703,"Executor Run Time":50,"Executor CPU Time":17503528,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":1,"Attempt":0,"Launch Time":1596020225809,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020225874,"Failed":false,"Killed":false,"Accumulables":[{"ID":986,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":991,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":987,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":988,"Name":"peak memory","Update":"4456448","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":990,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":996,"Name":"time to update","Update":"15","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":995,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":997,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":998,"Name":"time to commit changes","Update":"23","Value":"49","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1000,"Name":"estimated size of state only on current version","Update":"368","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":992,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1001,"Name":"count of cache hit on states cache in provider","Update":"12","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":999,"Name":"memory used by state","Update":"784","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":994,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1003,"Name":"duration","Update":"15","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1004,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1005,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1007,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1009,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":955,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":959,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":958,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":960,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1087,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1086,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1085,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1084,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1083,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1082,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1081,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1079,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":1074,"Name":"internal.metrics.resultSize","Update":5574,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1073,"Name":"internal.metrics.executorCpuTime","Update":17516707,"Value":35020235,"Internal":true,"Count Failed Values":true},{"ID":1072,"Name":"internal.metrics.executorRunTime","Update":56,"Value":106,"Internal":true,"Count Failed Values":true},{"ID":1071,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3750230,"Value":8005933,"Internal":true,"Count Failed Values":true},{"ID":1070,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":7,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3750230,"Executor Run Time":56,"Executor CPU Time":17516707,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":83,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"198\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[82],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":81,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"202\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[80],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":79,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"206\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[78],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":80,"Name":"StateStoreRDD","Scope":"{\"id\":\"205\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[79],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":82,"Name":"StateStoreRDD","Scope":"{\"id\":\"201\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[81],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[12],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020225801,"Completion Time":1596020225874,"Accumulables":[{"ID":1070,"Name":"internal.metrics.executorDeserializeTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":1079,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":992,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1073,"Name":"internal.metrics.executorCpuTime","Value":35020235,"Internal":true,"Count Failed Values":true},{"ID":1082,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1001,"Name":"count of cache hit on states cache in provider","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":995,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1004,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":986,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":959,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1085,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":1007,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":998,"Name":"time to commit changes","Value":"49","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1084,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":997,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1087,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":955,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1081,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":991,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1009,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1072,"Name":"internal.metrics.executorRunTime","Value":106,"Internal":true,"Count Failed Values":true},{"ID":1000,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":994,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1003,"Name":"duration","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":958,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":988,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1074,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1083,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":960,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1086,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":987,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1005,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":996,"Name":"time to update","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1071,"Name":"internal.metrics.executorDeserializeCpuTime","Value":8005933,"Internal":true,"Count Failed Values":true},{"ID":999,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":990,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":6,"Completion Time":1596020225875,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":20,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 6","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#130, count#131]\nArguments: [value#130, count#131]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#130, count#131]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1095,"metricType":"sum"}]},"time":1596020225891} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":20,"time":1596020225896} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":19,"time":1596020225897} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":18,"time":1596020225897} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:05.562Z","batchId":6,"batchDuration":351,"durationMs":{"triggerExecution":351,"queryPlanning":28,"getBatch":1,"latestOffset":6,"addBatch":273,"walCommit":25},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":24,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48837}}","endOffset":"{\"test5\":{\"0\":48881}}","numInputRows":44,"inputRowsPerSecond":100.22779043280183,"processedRowsPerSecond":125.35612535612536}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":21,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48881}}, {\"test5\":{\"0\":48917}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1759]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c0968891-bf48-4112-a19b-444014085d1d, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = c0968891-bf48-4112-a19b-444014085d1d, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1683]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1179,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1178,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1175,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1176,"metricType":"timing"},{"name":"peak memory","accumulatorId":1174,"metricType":"size"},{"name":"number of output rows","accumulatorId":1173,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1177,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1170,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1171,"metricType":"timing"},{"name":"peak memory","accumulatorId":1169,"metricType":"size"},{"name":"number of output rows","accumulatorId":1168,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1172,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1167,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":1119,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":1120,"metricType":"nsTiming"},{"name":"records read","accumulatorId":1117,"metricType":"sum"},{"name":"local bytes read","accumulatorId":1115,"metricType":"size"},{"name":"fetch wait time","accumulatorId":1116,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":1113,"metricType":"size"},{"name":"local blocks read","accumulatorId":1112,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":1111,"metricType":"sum"},{"name":"data size","accumulatorId":1110,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":1114,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":1118,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1166,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1163,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1164,"metricType":"timing"},{"name":"peak memory","accumulatorId":1162,"metricType":"size"},{"name":"number of output rows","accumulatorId":1161,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1165,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1160,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":1150,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":1151,"metricType":"sum"},{"name":"memory used by state","accumulatorId":1156,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1158,"metricType":"sum"},{"name":"number of output rows","accumulatorId":1149,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1157,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1159,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":1155,"metricType":"timing"},{"name":"time to remove","accumulatorId":1154,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":1152,"metricType":"sum"},{"name":"time to update","accumulatorId":1153,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1146,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1147,"metricType":"timing"},{"name":"peak memory","accumulatorId":1145,"metricType":"size"},{"name":"number of output rows","accumulatorId":1144,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1148,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1143,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020225988} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":22,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nWriteToDataSourceV2 (14)\n+- * HashAggregate (13)\n +- StateStoreSave (12)\n +- * HashAggregate (11)\n +- StateStoreRestore (10)\n +- Exchange (9)\n +- * HashAggregate (8)\n +- * HashAggregate (7)\n +- * SerializeFromObject (6)\n +- MapPartitions (5)\n +- DeserializeToObject (4)\n +- * Project (3)\n +- * Project (2)\n +- MicroBatchScan (1)\n\n\n(1) MicroBatchScan\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nArguments: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13], org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan@7e7b182c, KafkaV2[Subscribe[test5]], {\"test5\":{\"0\":48881}}, {\"test5\":{\"0\":48917}}\n\n(2) Project [codegen id : 1]\nOutput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(3) Project [codegen id : 1]\nOutput [1]: [cast(value#8 as string) AS value#21]\nInput [7]: [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]\n\n(4) DeserializeToObject\nInput [1]: [value#21]\nArguments: value#21.toString, obj#27: java.lang.String\n\n(5) MapPartitions\nInput [1]: [obj#27]\nArguments: org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String\n\n(6) SerializeFromObject [codegen id : 2]\nInput [1]: [obj#28]\nArguments: [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]\n\n(7) HashAggregate [codegen id : 2]\nInput [1]: [value#29]\nKeys [1]: [value#29]\nFunctions [1]: [partial_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(8) HashAggregate [codegen id : 2]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(9) Exchange\nInput [2]: [value#29, count#38L]\nArguments: hashpartitioning(value#29, 2), true, [id=#1835]\n\n(10) StateStoreRestore\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = e165b23b-1a6f-459f-9c51-288922bb2647, opId = 0, ver = 0, numPartitions = 2], 2\n\n(11) HashAggregate [codegen id : 3]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [merge_count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count#38L]\n\n(12) StateStoreSave\nInput [2]: [value#29, count#38L]\nArguments: [value#29], state info [ checkpoint = , runId = e165b23b-1a6f-459f-9c51-288922bb2647, opId = 0, ver = 0, numPartitions = 2], Append, 0, 2\n\n(13) HashAggregate [codegen id : 4]\nInput [2]: [value#29, count#38L]\nKeys [1]: [value#29]\nFunctions [1]: [count(1)]\nAggregate Attributes [1]: [count(1)#31L]\nResults [2]: [value#29, count(1)#31L AS count#32L]\n\n(14) WriteToDataSourceV2\nInput [2]: [value#29, count#32L]\nArguments: org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e\n\n","sparkPlanInfo":{"nodeName":"WriteToDataSourceV2","simpleString":"WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6313b68e","children":[{"nodeName":"WholeStageCodegen (4)","simpleString":"WholeStageCodegen (4)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreSave","simpleString":"StateStoreSave [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], Complete, 0, 2","children":[{"nodeName":"WholeStageCodegen (3)","simpleString":"WholeStageCodegen (3)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"StateStoreRestore","simpleString":"StateStoreRestore [value#29], state info [ checkpoint = file:/tmp/temporary-025d7997-5b66-4def-abbf-bdcca57312b9/state, runId = e225d92f-2545-48f8-87a2-9c0309580f8a, opId = 0, ver = 7, numPartitions = 2], 2","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(value#29, 2), true, [id=#1683]","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[merge_count(1)])","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[value#29], functions=[partial_count(1)])","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#29]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MapPartitions","simpleString":"MapPartitions org.apache.spark.sql.Dataset$$Lambda$1321/872917583@67b99068, obj#28: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#21.toString, obj#27: java.lang.String","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"Project","simpleString":"Project [cast(value#8 as string) AS value#21]","children":[{"nodeName":"Project","simpleString":"Project [key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"MicroBatchScan","simpleString":"MicroBatchScan[key#7, value#8, topic#9, partition#10, offset#11L, timestamp#12, timestampType#13] class org.apache.spark.sql.kafka010.KafkaSourceProvider$KafkaScan","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1179,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1178,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1175,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1176,"metricType":"timing"},{"name":"peak memory","accumulatorId":1174,"metricType":"size"},{"name":"number of output rows","accumulatorId":1173,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1177,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1170,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1171,"metricType":"timing"},{"name":"peak memory","accumulatorId":1169,"metricType":"size"},{"name":"number of output rows","accumulatorId":1168,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1172,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1167,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":1119,"metricType":"sum"},{"name":"shuffle write time","accumulatorId":1120,"metricType":"nsTiming"},{"name":"records read","accumulatorId":1117,"metricType":"sum"},{"name":"local bytes read","accumulatorId":1115,"metricType":"size"},{"name":"fetch wait time","accumulatorId":1116,"metricType":"timing"},{"name":"remote bytes read","accumulatorId":1113,"metricType":"size"},{"name":"local blocks read","accumulatorId":1112,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":1111,"metricType":"sum"},{"name":"data size","accumulatorId":1110,"metricType":"size"},{"name":"remote bytes read to disk","accumulatorId":1114,"metricType":"size"},{"name":"shuffle bytes written","accumulatorId":1118,"metricType":"size"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1166,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1163,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1164,"metricType":"timing"},{"name":"peak memory","accumulatorId":1162,"metricType":"size"},{"name":"number of output rows","accumulatorId":1161,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1165,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1160,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of inputs which are later than watermark ('inputs' are relative to operators)","accumulatorId":1150,"metricType":"sum"},{"name":"number of total state rows","accumulatorId":1151,"metricType":"sum"},{"name":"memory used by state","accumulatorId":1156,"metricType":"size"},{"name":"count of cache hit on states cache in provider","accumulatorId":1158,"metricType":"sum"},{"name":"number of output rows","accumulatorId":1149,"metricType":"sum"},{"name":"estimated size of state only on current version","accumulatorId":1157,"metricType":"size"},{"name":"count of cache miss on states cache in provider","accumulatorId":1159,"metricType":"sum"},{"name":"time to commit changes","accumulatorId":1155,"metricType":"timing"},{"name":"time to remove","accumulatorId":1154,"metricType":"timing"},{"name":"number of updated state rows","accumulatorId":1152,"metricType":"sum"},{"name":"time to update","accumulatorId":1153,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size","accumulatorId":1146,"metricType":"size"},{"name":"time in aggregation build","accumulatorId":1147,"metricType":"timing"},{"name":"peak memory","accumulatorId":1145,"metricType":"size"},{"name":"number of output rows","accumulatorId":1144,"metricType":"sum"},{"name":"avg hash probe bucket list iters","accumulatorId":1148,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":1143,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1596020226019} +{"Event":"SparkListenerJobStart","Job ID":7,"Submission Time":1596020226076,"Stage Infos":[{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":95,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"231\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[94],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":93,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"235\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[92],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":91,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[90],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":94,"Name":"StateStoreRDD","Scope":"{\"id\":\"234\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[93],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":92,"Name":"StateStoreRDD","Scope":"{\"id\":\"238\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[91],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[14],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0},{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":90,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[89],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":88,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"245\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[87],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":84,"Name":"DataSourceRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":85,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[84],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":89,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"240\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[88],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":86,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"247\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[85],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":87,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"246\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[86],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[15,14],"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"7","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"22","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":90,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[89],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":88,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"245\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[87],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":84,"Name":"DataSourceRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":85,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[84],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":89,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"240\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[88],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":86,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"247\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[85],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":87,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"246\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[86],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226077,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"7","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"22","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":14,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":0,"Attempt":0,"Launch Time":1596020226086,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":0,"Attempt":0,"Launch Time":1596020226086,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020226116,"Failed":false,"Killed":false,"Accumulables":[{"ID":1120,"Name":"shuffle write time","Update":"543034","Value":"543034","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1119,"Name":"shuffle records written","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1118,"Name":"shuffle bytes written","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1110,"Name":"data size","Update":"128","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1167,"Name":"duration","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1168,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1169,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1171,"Name":"time in aggregation build","Update":"8","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1173,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1174,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1176,"Name":"time in aggregation build","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1178,"Name":"duration","Update":"13","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1179,"Name":"number of output rows","Update":"36","Value":"36","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1224,"Name":"internal.metrics.input.recordsRead","Update":36,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":1222,"Name":"internal.metrics.shuffle.write.writeTime","Update":543034,"Value":543034,"Internal":true,"Count Failed Values":true},{"ID":1221,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1220,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1211,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1206,"Name":"internal.metrics.resultSize","Update":2544,"Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1205,"Name":"internal.metrics.executorCpuTime","Update":19652237,"Value":19652237,"Internal":true,"Count Failed Values":true},{"ID":1204,"Name":"internal.metrics.executorRunTime","Update":19,"Value":19,"Internal":true,"Count Failed Values":true},{"ID":1203,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2829254,"Value":2829254,"Internal":true,"Count Failed Values":true},{"ID":1202,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2829254,"Executor Run Time":19,"Executor CPU Time":19652237,"Peak Execution Memory":524288,"Result Size":2544,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":168,"Shuffle Write Time":543034,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":0,"Records Read":36},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":1,"RDD Info":[{"RDD ID":90,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[89],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":88,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"245\",\"name\":\"MapPartitions\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[87],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":84,"Name":"DataSourceRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":85,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"251\",\"name\":\"MicroBatchScan\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[84],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":89,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"240\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[88],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":86,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"247\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[85],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":87,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"246\",\"name\":\"DeserializeToObject\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[86],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226077,"Completion Time":1596020226117,"Accumulables":[{"ID":1205,"Name":"internal.metrics.executorCpuTime","Value":19652237,"Internal":true,"Count Failed Values":true},{"ID":1178,"Name":"duration","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1169,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1118,"Name":"shuffle bytes written","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1204,"Name":"internal.metrics.executorRunTime","Value":19,"Internal":true,"Count Failed Values":true},{"ID":1222,"Name":"internal.metrics.shuffle.write.writeTime","Value":543034,"Internal":true,"Count Failed Values":true},{"ID":1171,"Name":"time in aggregation build","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1120,"Name":"shuffle write time","Value":"543034","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1174,"Name":"peak memory","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1168,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1203,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2829254,"Internal":true,"Count Failed Values":true},{"ID":1167,"Name":"duration","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1221,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1176,"Name":"time in aggregation build","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1206,"Name":"internal.metrics.resultSize","Value":2544,"Internal":true,"Count Failed Values":true},{"ID":1224,"Name":"internal.metrics.input.recordsRead","Value":36,"Internal":true,"Count Failed Values":true},{"ID":1179,"Name":"number of output rows","Value":"36","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1119,"Name":"shuffle records written","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1110,"Name":"data size","Value":"128","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1173,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1202,"Name":"internal.metrics.executorDeserializeTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":1211,"Name":"internal.metrics.peakExecutionMemory","Value":524288,"Internal":true,"Count Failed Values":true},{"ID":1220,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":168,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":95,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"231\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[94],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":93,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"235\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[92],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":91,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[90],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":94,"Name":"StateStoreRDD","Scope":"{\"id\":\"234\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[93],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":92,"Name":"StateStoreRDD","Scope":"{\"id\":\"238\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[91],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[14],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226120,"Accumulables":[],"Resource Profile Id":0},"Properties":{"sql.streaming.queryId":"8d268dc2-bc9c-4be8-97a9-b135d2943028","spark.driver.host":"iZbp19vpr16ix621sdw476Z","spark.eventLog.enabled":"true","spark.sql.adaptive.enabled":"false","spark.job.interruptOnCancel":"true","spark.driver.port":"46309","__fetch_continuous_blocks_in_batch_enabled":"true","spark.jars":"file:/root/spark-3.1.0-SNAPSHOT-bin-hadoop2.8/./examples/jars/spark-examples_2.12-3.1.0-SNAPSHOT.jar","__is_continuous_processing":"false","spark.app.name":"StructuredKafkaWordCount","callSite.long":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","callSite.short":"start at StructuredKafkaWordCount.scala:86","spark.submit.pyFiles":"","spark.job.description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","spark.executor.id":"driver","spark.sql.cbo.enabled":"false","streaming.sql.batchId":"7","spark.jobGroup.id":"e225d92f-2545-48f8-87a2-9c0309580f8a","spark.submit.deployMode":"client","spark.master":"local[*]","spark.eventLog.dir":"/tmp/spark-history","spark.sql.execution.id":"22","spark.app.id":"local-1596020211915","spark.sql.shuffle.partitions":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":15,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":0,"Attempt":0,"Launch Time":1596020226128,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":15,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":1,"Attempt":0,"Launch Time":1596020226129,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":15,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":1,"Attempt":0,"Launch Time":1596020226129,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020226196,"Failed":false,"Killed":false,"Accumulables":[{"ID":1143,"Name":"duration","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1148,"Name":"avg hash probe bucket list iters","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1144,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1145,"Name":"peak memory","Update":"4456448","Value":"4456448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1147,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1153,"Name":"time to update","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1152,"Name":"number of updated state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1154,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1155,"Name":"time to commit changes","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1157,"Name":"estimated size of state only on current version","Update":"368","Value":"368","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1149,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1158,"Name":"count of cache hit on states cache in provider","Update":"14","Value":"14","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1156,"Name":"memory used by state","Update":"784","Value":"784","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1151,"Name":"number of total state rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1160,"Name":"duration","Update":"21","Value":"21","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1161,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1162,"Name":"peak memory","Update":"262144","Value":"262144","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1164,"Name":"time in aggregation build","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1166,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1112,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1116,"Name":"fetch wait time","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1115,"Name":"local bytes read","Update":"168","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1117,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1244,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1243,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1242,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":168,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1241,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1240,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1239,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1238,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1236,"Name":"internal.metrics.peakExecutionMemory","Update":4718592,"Value":4718592,"Internal":true,"Count Failed Values":true},{"ID":1231,"Name":"internal.metrics.resultSize","Update":5574,"Value":5574,"Internal":true,"Count Failed Values":true},{"ID":1230,"Name":"internal.metrics.executorCpuTime","Update":19415818,"Value":19415818,"Internal":true,"Count Failed Values":true},{"ID":1229,"Name":"internal.metrics.executorRunTime","Update":60,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":1228,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3845429,"Value":3845429,"Internal":true,"Count Failed Values":true},{"ID":1227,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3845429,"Executor Run Time":60,"Executor CPU Time":19415818,"Peak Execution Memory":4718592,"Result Size":5574,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":168,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":15,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":0,"Attempt":0,"Launch Time":1596020226128,"Executor ID":"driver","Host":"iZbp19vpr16ix621sdw476Z","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1596020226204,"Failed":false,"Killed":false,"Accumulables":[{"ID":1143,"Name":"duration","Update":"2","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1145,"Name":"peak memory","Update":"262144","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1147,"Name":"time in aggregation build","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1153,"Name":"time to update","Update":"3","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1154,"Name":"time to remove","Update":"0","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1155,"Name":"time to commit changes","Update":"48","Value":"67","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1157,"Name":"estimated size of state only on current version","Update":"88","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1158,"Name":"count of cache hit on states cache in provider","Update":"14","Value":"28","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1156,"Name":"memory used by state","Update":"400","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1160,"Name":"duration","Update":"3","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1162,"Name":"peak memory","Update":"262144","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1164,"Name":"time in aggregation build","Update":"0","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1244,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1243,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1242,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":168,"Internal":true,"Count Failed Values":true},{"ID":1241,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1240,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1239,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":1238,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":1236,"Name":"internal.metrics.peakExecutionMemory","Update":524288,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":1231,"Name":"internal.metrics.resultSize","Update":5311,"Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1230,"Name":"internal.metrics.executorCpuTime","Update":14652861,"Value":34068679,"Internal":true,"Count Failed Values":true},{"ID":1229,"Name":"internal.metrics.executorRunTime","Update":65,"Value":125,"Internal":true,"Count Failed Values":true},{"ID":1228,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3933877,"Value":7779306,"Internal":true,"Count Failed Values":true},{"ID":1227,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3933877,"Executor Run Time":65,"Executor CPU Time":14652861,"Peak Execution Memory":524288,"Result Size":5311,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"start at StructuredKafkaWordCount.scala:86","Number of Tasks":2,"RDD Info":[{"RDD ID":95,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"231\",\"name\":\"WholeStageCodegen (4)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[94],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":93,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"235\",\"name\":\"WholeStageCodegen (3)\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[92],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":91,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"239\",\"name\":\"Exchange\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[90],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":94,"Name":"StateStoreRDD","Scope":"{\"id\":\"234\",\"name\":\"StateStoreSave\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[93],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":92,"Name":"StateStoreRDD","Scope":"{\"id\":\"238\",\"name\":\"StateStoreRestore\"}","Callsite":"start at StructuredKafkaWordCount.scala:86","Parent IDs":[91],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[14],"Details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1596020226120,"Completion Time":1596020226204,"Accumulables":[{"ID":1115,"Name":"local bytes read","Value":"168","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1160,"Name":"duration","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1151,"Name":"number of total state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1145,"Name":"peak memory","Value":"4718592","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1154,"Name":"time to remove","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1231,"Name":"internal.metrics.resultSize","Value":10885,"Internal":true,"Count Failed Values":true},{"ID":1240,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1153,"Name":"time to update","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1162,"Name":"peak memory","Value":"524288","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1144,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1243,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1117,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1228,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7779306,"Internal":true,"Count Failed Values":true},{"ID":1147,"Name":"time in aggregation build","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1156,"Name":"memory used by state","Value":"1184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1236,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":1227,"Name":"internal.metrics.executorDeserializeTime","Value":6,"Internal":true,"Count Failed Values":true},{"ID":1158,"Name":"count of cache hit on states cache in provider","Value":"28","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1149,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1239,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1230,"Name":"internal.metrics.executorCpuTime","Value":34068679,"Internal":true,"Count Failed Values":true},{"ID":1152,"Name":"number of updated state rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1242,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":168,"Internal":true,"Count Failed Values":true},{"ID":1116,"Name":"fetch wait time","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1161,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1143,"Name":"duration","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1155,"Name":"time to commit changes","Value":"67","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1164,"Name":"time in aggregation build","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1241,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1244,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1148,"Name":"avg hash probe bucket list iters","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1157,"Name":"estimated size of state only on current version","Value":"456","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1166,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1238,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":1229,"Name":"internal.metrics.executorRunTime","Value":125,"Internal":true,"Count Failed Values":true},{"ID":1112,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}} +{"Event":"SparkListenerJobEnd","Job ID":7,"Completion Time":1596020226204,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":23,"description":"\nid = 8d268dc2-bc9c-4be8-97a9-b135d2943028\nrunId = e225d92f-2545-48f8-87a2-9c0309580f8a\nbatch = 7","details":"org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:366)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount$.main(StructuredKafkaWordCount.scala:86)\norg.apache.spark.examples.sql.streaming.StructuredKafkaWordCount.main(StructuredKafkaWordCount.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","physicalPlanDescription":"== Physical Plan ==\nLocalTableScan (1)\n\n\n(1) LocalTableScan\nOutput [2]: [value#144, count#145]\nArguments: [value#144, count#145]\n\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#144, count#145]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1252,"metricType":"sum"}]},"time":1596020226221} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":23,"time":1596020226230} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":22,"time":1596020226231} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":21,"time":1596020226231} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","progress":{"id":"8d268dc2-bc9c-4be8-97a9-b135d2943028","runId":"e225d92f-2545-48f8-87a2-9c0309580f8a","name":null,"timestamp":"2020-07-29T10:57:05.916Z","batchId":7,"batchDuration":341,"durationMs":{"triggerExecution":341,"queryPlanning":24,"getBatch":0,"latestOffset":3,"addBatch":271,"walCommit":14},"eventTime":{},"stateOperators":[{"numRowsTotal":1,"numRowsUpdated":1,"memoryUsedBytes":1184,"numLateInputs":0,"customMetrics":{"stateOnCurrentVersionSizeBytes":456,"loadedMapCacheHitCount":28,"loadedMapCacheMissCount":0}}],"sources":[{"description":"KafkaV2[Subscribe[test5]]","startOffset":"{\"test5\":{\"0\":48881}}","endOffset":"{\"test5\":{\"0\":48917}}","numInputRows":36,"inputRowsPerSecond":101.69491525423729,"processedRowsPerSecond":105.57184750733137}],"sink":{"description":"org.apache.spark.sql.execution.streaming.ConsoleTable$@514ba885","numOutputRows":1},"observedMetrics":{}}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1596020226301} diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 0a54dff3a1..a6d041a588 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -1,6 +1,6 @@ ## Summary - - Number of queries: 342 + - Number of queries: 347 - Number of expressions that missing example: 13 - Expressions missing examples: bigint,binary,boolean,date,decimal,double,float,int,smallint,string,timestamp,tinyint,window ## Schema of Built-in Functions @@ -91,6 +91,7 @@ | org.apache.spark.sql.catalyst.expressions.DateAdd | date_add | SELECT date_add('2016-07-30', 1) | struct | | org.apache.spark.sql.catalyst.expressions.DateDiff | datediff | SELECT datediff('2009-07-31', '2009-07-30') | struct | | org.apache.spark.sql.catalyst.expressions.DateFormatClass | date_format | SELECT date_format('2016-04-08', 'y') | struct | +| org.apache.spark.sql.catalyst.expressions.DateFromUnixDate | date_from_unix_date | SELECT date_from_unix_date(1) | struct | | org.apache.spark.sql.catalyst.expressions.DatePart | date_part | SELECT date_part('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456') | struct | | org.apache.spark.sql.catalyst.expressions.DateSub | date_sub | SELECT date_sub('2016-07-30', 1) | struct | | org.apache.spark.sql.catalyst.expressions.DayOfMonth | day | SELECT day('2009-07-30') | struct | @@ -289,6 +290,10 @@ | org.apache.spark.sql.catalyst.expressions.UnaryMinus | negative | SELECT negative(1) | struct | | org.apache.spark.sql.catalyst.expressions.UnaryPositive | positive | SELECT positive(1) | struct<(+ 1):int> | | org.apache.spark.sql.catalyst.expressions.Unhex | unhex | SELECT decode(unhex('537061726B2053514C'), 'UTF-8') | struct | +| org.apache.spark.sql.catalyst.expressions.UnixDate | unix_date | SELECT unix_date(DATE("1970-01-02")) | struct | +| org.apache.spark.sql.catalyst.expressions.UnixMicros | unix_micros | SELECT unix_micros(TIMESTAMP('1970-01-01 00:00:01Z')) | struct | +| org.apache.spark.sql.catalyst.expressions.UnixMillis | unix_millis | SELECT unix_millis(TIMESTAMP('1970-01-01 00:00:01Z')) | struct | +| org.apache.spark.sql.catalyst.expressions.UnixSeconds | unix_seconds | SELECT unix_seconds(TIMESTAMP('1970-01-01 00:00:01Z')) | struct | | org.apache.spark.sql.catalyst.expressions.UnixTimestamp | unix_timestamp | SELECT unix_timestamp() | struct | | org.apache.spark.sql.catalyst.expressions.Upper | ucase | SELECT ucase('SparkSql') | struct | | org.apache.spark.sql.catalyst.expressions.Upper | upper | SELECT upper('SparkSql') | struct | diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index 534e222b7c..e35266a85d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -14,7 +14,14 @@ select TIMESTAMP_MILLIS(-92233720368547758); select TIMESTAMP_SECONDS(0.1234567); -- truncation is OK for float/double select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)); - +-- UNIX_SECONDS, UNIX_MILLISECONDS and UNIX_MICROSECONDS +select UNIX_SECONDS(TIMESTAMP('2020-12-01 14:30:08Z')), UNIX_SECONDS(TIMESTAMP('2020-12-01 14:30:08.999999Z')), UNIX_SECONDS(null); +select UNIX_MILLIS(TIMESTAMP('2020-12-01 14:30:08Z')), UNIX_MILLIS(TIMESTAMP('2020-12-01 14:30:08.999999Z')), UNIX_MILLIS(null); +select UNIX_MICROS(TIMESTAMP('2020-12-01 14:30:08Z')), UNIX_MICROS(TIMESTAMP('2020-12-01 14:30:08.999999Z')), UNIX_MICROS(null); +-- DATE_FROM_UNIX_DATE +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null); +-- UNIX_DATE +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null); -- [SPARK-16836] current_date and current_timestamp literals select current_date = current_date(), current_timestamp = current_timestamp(); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index 10669f14aa..18a751f573 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 117 +-- Number of queries: 122 -- !query @@ -87,6 +87,46 @@ struct +-- !query output +1606833008 1606833008 NULL + + +-- !query +select UNIX_MILLIS(TIMESTAMP('2020-12-01 14:30:08Z')), UNIX_MILLIS(TIMESTAMP('2020-12-01 14:30:08.999999Z')), UNIX_MILLIS(null) +-- !query schema +struct +-- !query output +1606833008000 1606833008999 NULL + + +-- !query +select UNIX_MICROS(TIMESTAMP('2020-12-01 14:30:08Z')), UNIX_MICROS(TIMESTAMP('2020-12-01 14:30:08.999999Z')), UNIX_MICROS(null) +-- !query schema +struct +-- !query output +1606833008000000 1606833008999999 NULL + + +-- !query +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null) +-- !query schema +struct +-- !query output +1970-01-01 1972-09-27 NULL + + +-- !query +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null) +-- !query schema +struct +-- !query output +0 18600 NULL + + -- !query select current_date = current_date(), current_timestamp = current_timestamp() -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 7c2c62a2db..be75f6fb99 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 117 +-- Number of queries: 122 -- !query @@ -87,6 +87,46 @@ struct +-- !query output +1606833008 1606833008 NULL + + +-- !query +select UNIX_MILLIS(TIMESTAMP('2020-12-01 14:30:08Z')), UNIX_MILLIS(TIMESTAMP('2020-12-01 14:30:08.999999Z')), UNIX_MILLIS(null) +-- !query schema +struct +-- !query output +1606833008000 1606833008999 NULL + + +-- !query +select UNIX_MICROS(TIMESTAMP('2020-12-01 14:30:08Z')), UNIX_MICROS(TIMESTAMP('2020-12-01 14:30:08.999999Z')), UNIX_MICROS(null) +-- !query schema +struct +-- !query output +1606833008000000 1606833008999999 NULL + + +-- !query +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null) +-- !query schema +struct +-- !query output +1970-01-01 1972-09-27 NULL + + +-- !query +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null) +-- !query schema +struct +-- !query output +0 18600 NULL + + -- !query select current_date = current_date(), current_timestamp = current_timestamp() -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index 810ab6ef0c..1e963ed16f 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 117 +-- Number of queries: 122 -- !query @@ -87,6 +87,46 @@ struct +-- !query output +1606833008 1606833008 NULL + + +-- !query +select UNIX_MILLIS(TIMESTAMP('2020-12-01 14:30:08Z')), UNIX_MILLIS(TIMESTAMP('2020-12-01 14:30:08.999999Z')), UNIX_MILLIS(null) +-- !query schema +struct +-- !query output +1606833008000 1606833008999 NULL + + +-- !query +select UNIX_MICROS(TIMESTAMP('2020-12-01 14:30:08Z')), UNIX_MICROS(TIMESTAMP('2020-12-01 14:30:08.999999Z')), UNIX_MICROS(null) +-- !query schema +struct +-- !query output +1606833008000000 1606833008999999 NULL + + +-- !query +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null) +-- !query schema +struct +-- !query output +1970-01-01 1972-09-27 NULL + + +-- !query +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null) +-- !query schema +struct +-- !query output +0 18600 NULL + + -- !query select current_date = current_date(), current_timestamp = current_timestamp() -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala b/sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala new file mode 100644 index 0000000000..f73305b1b0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/deploy/history/Utils.scala @@ -0,0 +1,40 @@ +/* + * 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.deploy.history + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.History.HISTORY_LOG_DIR +import org.apache.spark.util.ManualClock + +object Utils { + def withFsHistoryProvider(logDir: String)(fn: FsHistoryProvider => Unit): Unit = { + var provider: FsHistoryProvider = null + try { + val clock = new ManualClock() + val conf = new SparkConf().set(HISTORY_LOG_DIR, logDir) + val provider = new FsHistoryProvider(conf, clock) + provider.checkForLogs() + fn(provider) + } finally { + if (provider != null) { + provider.stop() + provider = null + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 583bc694dc..7635590ab4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -266,22 +266,23 @@ class DataSourceV2SQLSuite checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), Seq.empty) } - // TODO: ignored by SPARK-31707, restore the test after create table syntax unification - ignore("CreateTable: without USING clause") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) - val testCatalog = catalog("testcat").asTableCatalog - - sql("CREATE TABLE testcat.t1 (id int)") - val t1 = testCatalog.loadTable(Identifier.of(Array(), "t1")) - // Spark shouldn't set the default provider for catalog plugins. - assert(!t1.properties.containsKey(TableCatalog.PROP_PROVIDER)) - - sql("CREATE TABLE t2 (id int)") - val t2 = spark.sessionState.catalogManager.v2SessionCatalog.asTableCatalog - .loadTable(Identifier.of(Array("default"), "t2")).asInstanceOf[V1Table] - // Spark should set the default provider as DEFAULT_DATA_SOURCE_NAME for the session catalog. - assert(t2.v1Table.provider == Some(conf.defaultDataSourceName)) + test("CreateTable: without USING clause") { + withSQLConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT.key -> "false") { + // unset this config to use the default v2 session catalog. + spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + val testCatalog = catalog("testcat").asTableCatalog + + sql("CREATE TABLE testcat.t1 (id int)") + val t1 = testCatalog.loadTable(Identifier.of(Array(), "t1")) + // Spark shouldn't set the default provider for catalog plugins. + assert(!t1.properties.containsKey(TableCatalog.PROP_PROVIDER)) + + sql("CREATE TABLE t2 (id int)") + val t2 = spark.sessionState.catalogManager.v2SessionCatalog.asTableCatalog + .loadTable(Identifier.of(Array("default"), "t2")).asInstanceOf[V1Table] + // Spark should set the default provider as DEFAULT_DATA_SOURCE_NAME for the session catalog. + assert(t2.v1Table.provider == Some(conf.defaultDataSourceName)) + } } test("CreateTable/RepalceTable: invalid schema if has interval type") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 20cad721d3..33515ad41e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -1588,7 +1588,7 @@ class PlanResolutionSuite extends AnalysisTest { .add("b", StringType) ) ) - compare("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) " + + compare("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) STORED AS textfile " + "PARTITIONED BY (c INT, d STRING COMMENT 'test2')", createTable( table = "my_tab", @@ -1616,7 +1616,7 @@ class PlanResolutionSuite extends AnalysisTest { ) // Partitioned by a StructType should be accepted by `SparkSqlParser` but will fail an analyze // rule in `AnalyzeCreateTable`. - compare("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) " + + compare("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) STORED AS textfile " + "PARTITIONED BY (nested STRUCT)", createTable( table = "my_tab", @@ -1890,7 +1890,7 @@ class PlanResolutionSuite extends AnalysisTest { } test("Test CTAS #3") { - val s3 = """CREATE TABLE page_view AS SELECT * FROM src""" + val s3 = """CREATE TABLE page_view STORED AS textfile AS SELECT * FROM src""" val (desc, exists) = extractTableDesc(s3) assert(exists == false) assert(desc.identifier.database == Some("default")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index 062b1060bc..bf850432d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala @@ -291,7 +291,7 @@ class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter { val query = inputDF .writeStream .option("checkpointLocation", checkpointDir.getAbsolutePath) - .saveAsTable(tableIdentifier) + .toTable(tableIdentifier) inputData.addData(newInputs: _*) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala new file mode 100644 index 0000000000..160535ea4d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryHistorySuite.scala @@ -0,0 +1,63 @@ +/* + * 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.streaming.ui + +import java.util.Locale +import javax.servlet.http.HttpServletRequest + +import org.mockito.Mockito.{mock, when} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.deploy.history.{Utils => HsUtils} +import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore +import org.apache.spark.sql.test.SharedSparkSession + +class StreamingQueryHistorySuite extends SharedSparkSession with BeforeAndAfter { + + test("support streaming query events") { + val logDir = Thread.currentThread().getContextClassLoader.getResource("spark-events").toString + HsUtils.withFsHistoryProvider(logDir) { provider => + val appUi = provider.getAppUI("local-1596020211915", None).getOrElse { + assert(false, "Failed to load event log of local-1596020211915.") + null + } + assert(appUi.ui.appName == "StructuredKafkaWordCount") + assert(appUi.ui.store.store.count(classOf[StreamingQueryData]) == 1) + assert(appUi.ui.store.store.count(classOf[StreamingQueryProgressWrapper]) == 8) + + val store = new StreamingQueryStatusStore(appUi.ui.store.store) + val tab = new StreamingQueryTab(store, appUi.ui) + val request = mock(classOf[HttpServletRequest]) + var html = new StreamingQueryPage(tab).render(request) + .toString().toLowerCase(Locale.ROOT) + // 81.39: Avg Input /sec + assert(html.contains("81.39")) + // 157.05: Avg Process /sec + assert(html.contains("157.05")) + + val id = "8d268dc2-bc9c-4be8-97a9-b135d2943028" + val runId = "e225d92f-2545-48f8-87a2-9c0309580f8a" + when(request.getParameter("id")).thenReturn(runId) + html = new StreamingQueryStatisticsPage(tab).render(request) + .toString().toLowerCase(Locale.ROOT) + assert(html.contains("8 completed batches")) + assert(html.contains(id)) + assert(html.contains(runId)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala index c2b6688faf..246fa1f7c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala @@ -20,11 +20,13 @@ package org.apache.spark.sql.streaming.ui import java.util.{Locale, UUID} import javax.servlet.http.HttpServletRequest +import scala.xml.Node + import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.scalatest.BeforeAndAfter -import scala.xml.Node import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore import org.apache.spark.sql.streaming.StreamingQueryProgress import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.ui.SparkUI @@ -35,26 +37,26 @@ class StreamingQueryPageSuite extends SharedSparkSession with BeforeAndAfter { val id = UUID.randomUUID() val request = mock(classOf[HttpServletRequest]) val tab = mock(classOf[StreamingQueryTab], RETURNS_SMART_NULLS) - val statusListener = mock(classOf[StreamingQueryStatusListener], RETURNS_SMART_NULLS) + val store = mock(classOf[StreamingQueryStatusStore], RETURNS_SMART_NULLS) when(tab.appName).thenReturn("testing") when(tab.headerTabs).thenReturn(Seq.empty) - when(tab.statusListener).thenReturn(statusListener) + when(tab.store).thenReturn(store) val streamQuery = createStreamQueryUIData(id) - when(statusListener.allQueryStatus).thenReturn(Seq(streamQuery)) + when(store.allQueryUIData).thenReturn(Seq(streamQuery)) var html = renderStreamingQueryPage(request, tab) .toString().toLowerCase(Locale.ROOT) assert(html.contains("active streaming queries (1)")) - when(streamQuery.isActive).thenReturn(false) - when(streamQuery.exception).thenReturn(None) + when(streamQuery.summary.isActive).thenReturn(false) + when(streamQuery.summary.exception).thenReturn(None) html = renderStreamingQueryPage(request, tab) .toString().toLowerCase(Locale.ROOT) assert(html.contains("completed streaming queries (1)")) assert(html.contains("finished")) - when(streamQuery.isActive).thenReturn(false) - when(streamQuery.exception).thenReturn(Option("exception in query")) + when(streamQuery.summary.isActive).thenReturn(false) + when(streamQuery.summary.exception).thenReturn(Option("exception in query")) html = renderStreamingQueryPage(request, tab) .toString().toLowerCase(Locale.ROOT) assert(html.contains("completed streaming queries (1)")) @@ -66,17 +68,20 @@ class StreamingQueryPageSuite extends SharedSparkSession with BeforeAndAfter { val id = UUID.randomUUID() val request = mock(classOf[HttpServletRequest]) val tab = mock(classOf[StreamingQueryTab], RETURNS_SMART_NULLS) - val statusListener = mock(classOf[StreamingQueryStatusListener], RETURNS_SMART_NULLS) + val store = mock(classOf[StreamingQueryStatusStore], RETURNS_SMART_NULLS) + when(request.getParameter("id")).thenReturn(id.toString) + when(tab.appName).thenReturn("testing") + when(tab.headerTabs).thenReturn(Seq.empty) + when(tab.store).thenReturn(store) val ui = mock(classOf[SparkUI]) when(request.getParameter("id")).thenReturn(id.toString) when(tab.appName).thenReturn("testing") when(tab.headerTabs).thenReturn(Seq.empty) - when(tab.statusListener).thenReturn(statusListener) when(ui.conf).thenReturn(new SparkConf()) when(tab.parent).thenReturn(ui) val streamQuery = createStreamQueryUIData(id) - when(statusListener.allQueryStatus).thenReturn(Seq(streamQuery)) + when(store.allQueryUIData).thenReturn(Seq(streamQuery)) val html = renderStreamingQueryStatisticsPage(request, tab) .toString().toLowerCase(Locale.ROOT) @@ -94,15 +99,18 @@ class StreamingQueryPageSuite extends SharedSparkSession with BeforeAndAfter { when(progress.batchId).thenReturn(2) when(progress.prettyJson).thenReturn("""{"a":1}""") + val summary = mock(classOf[StreamingQueryData], RETURNS_SMART_NULLS) + when(summary.isActive).thenReturn(true) + when(summary.name).thenReturn("query") + when(summary.id).thenReturn(id) + when(summary.runId).thenReturn(id) + when(summary.startTimestamp).thenReturn(1L) + when(summary.exception).thenReturn(None) + val streamQuery = mock(classOf[StreamingQueryUIData], RETURNS_SMART_NULLS) - when(streamQuery.isActive).thenReturn(true) - when(streamQuery.name).thenReturn("query") - when(streamQuery.id).thenReturn(id) - when(streamQuery.runId).thenReturn(id) - when(streamQuery.startTimestamp).thenReturn(1L) + when(streamQuery.summary).thenReturn(summary) when(streamQuery.lastProgress).thenReturn(progress) when(streamQuery.recentProgress).thenReturn(Array(progress)) - when(streamQuery.exception).thenReturn(None) streamQuery } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala index 6aa440e560..91c55d5598 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala @@ -17,19 +17,28 @@ package org.apache.spark.sql.streaming.ui -import java.util.UUID +import java.text.SimpleDateFormat +import java.util.{Date, UUID} import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} +import org.scalatest.time.SpanSugar._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils.getTimeZone +import org.apache.spark.sql.execution.ui.StreamingQueryStatusStore +import org.apache.spark.sql.internal.StaticSQLConf import org.apache.spark.sql.streaming.{StreamingQueryListener, StreamingQueryProgress, StreamTest} import org.apache.spark.sql.streaming +import org.apache.spark.status.ElementTrackingStore +import org.apache.spark.util.kvstore.InMemoryStore class StreamingQueryStatusListenerSuite extends StreamTest { test("onQueryStarted, onQueryProgress, onQueryTerminated") { - val listener = new StreamingQueryStatusListener(spark.sparkContext.conf) + val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) + val listener = new StreamingQueryStatusListener(spark.sparkContext.conf, kvStore) + val queryStore = new StreamingQueryStatusStore(kvStore) - // hanlde query started event + // handle query started event val id = UUID.randomUUID() val runId = UUID.randomUUID() val startEvent = new StreamingQueryListener.QueryStartedEvent( @@ -37,8 +46,9 @@ class StreamingQueryStatusListenerSuite extends StreamTest { listener.onQueryStarted(startEvent) // result checking - assert(listener.activeQueryStatus.size() == 1) - assert(listener.activeQueryStatus.get(runId).name == "test") + assert(queryStore.allQueryUIData.count(_.summary.isActive) == 1) + assert(queryStore.allQueryUIData.filter(_.summary.isActive).exists(uiData => + uiData.summary.runId == runId && uiData.summary.name.equals("test"))) // handle query progress event val progress = mock(classOf[StreamingQueryProgress], RETURNS_SMART_NULLS) @@ -53,28 +63,32 @@ class StreamingQueryStatusListenerSuite extends StreamTest { listener.onQueryProgress(processEvent) // result checking - val activeQuery = listener.activeQueryStatus.get(runId) - assert(activeQuery.isActive) - assert(activeQuery.recentProgress.length == 1) - assert(activeQuery.lastProgress.id == id) - assert(activeQuery.lastProgress.runId == runId) - assert(activeQuery.lastProgress.timestamp == "2001-10-01T01:00:00.100Z") - assert(activeQuery.lastProgress.inputRowsPerSecond == 10.0) - assert(activeQuery.lastProgress.processedRowsPerSecond == 12.0) - assert(activeQuery.lastProgress.batchId == 2) - assert(activeQuery.lastProgress.prettyJson == """{"a":1}""") + val activeQuery = + queryStore.allQueryUIData.filter(_.summary.isActive).find(_.summary.runId == runId) + assert(activeQuery.isDefined) + assert(activeQuery.get.summary.isActive) + assert(activeQuery.get.recentProgress.length == 1) + assert(activeQuery.get.lastProgress.id == id) + assert(activeQuery.get.lastProgress.runId == runId) + assert(activeQuery.get.lastProgress.timestamp == "2001-10-01T01:00:00.100Z") + assert(activeQuery.get.lastProgress.inputRowsPerSecond == 10.0) + assert(activeQuery.get.lastProgress.processedRowsPerSecond == 12.0) + assert(activeQuery.get.lastProgress.batchId == 2) + assert(activeQuery.get.lastProgress.prettyJson == """{"a":1}""") // handle terminate event val terminateEvent = new StreamingQueryListener.QueryTerminatedEvent(id, runId, None) listener.onQueryTerminated(terminateEvent) - assert(!listener.inactiveQueryStatus.head.isActive) - assert(listener.inactiveQueryStatus.head.runId == runId) - assert(listener.inactiveQueryStatus.head.id == id) + assert(!queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.isActive) + assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.runId == runId) + assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.id == id) } test("same query start multiple times") { - val listener = new StreamingQueryStatusListener(spark.sparkContext.conf) + val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) + val listener = new StreamingQueryStatusListener(spark.sparkContext.conf, kvStore) + val queryStore = new StreamingQueryStatusStore(kvStore) // handle first time start val id = UUID.randomUUID() @@ -94,11 +108,106 @@ class StreamingQueryStatusListenerSuite extends StreamTest { listener.onQueryStarted(startEvent1) // result checking - assert(listener.activeQueryStatus.size() == 1) - assert(listener.inactiveQueryStatus.length == 1) - assert(listener.activeQueryStatus.containsKey(runId1)) - assert(listener.activeQueryStatus.get(runId1).id == id) - assert(listener.inactiveQueryStatus.head.runId == runId0) - assert(listener.inactiveQueryStatus.head.id == id) + assert(queryStore.allQueryUIData.count(_.summary.isActive) == 1) + assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).length == 1) + assert(queryStore.allQueryUIData.filter(_.summary.isActive).exists(_.summary.runId == runId1)) + assert(queryStore.allQueryUIData.filter(_.summary.isActive).exists(uiData => + uiData.summary.runId == runId1 && uiData.summary.id == id)) + assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.runId == runId0) + assert(queryStore.allQueryUIData.filterNot(_.summary.isActive).head.summary.id == id) + } + + test("test small retained queries") { + val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) + val conf = spark.sparkContext.conf + conf.set(StaticSQLConf.STREAMING_UI_RETAINED_QUERIES.key, "2") + val listener = new StreamingQueryStatusListener(conf, kvStore) + val queryStore = new StreamingQueryStatusStore(kvStore) + + def addNewQuery(): (UUID, UUID) = { + val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 + format.setTimeZone(getTimeZone("UTC")) + val id = UUID.randomUUID() + val runId = UUID.randomUUID() + val startEvent = new StreamingQueryListener.QueryStartedEvent( + id, runId, "test1", format.format(new Date(System.currentTimeMillis()))) + listener.onQueryStarted(startEvent) + (id, runId) + } + + def checkInactiveQueryStatus(numInactives: Int, targetInactives: Seq[UUID]): Unit = { + eventually(timeout(10.seconds)) { + val inactiveQueries = queryStore.allQueryUIData.filter(!_.summary.isActive) + assert(inactiveQueries.size == numInactives) + assert(inactiveQueries.map(_.summary.id).toSet == targetInactives.toSet) + } + } + + val (id1, runId1) = addNewQuery() + val (id2, runId2) = addNewQuery() + val (id3, runId3) = addNewQuery() + assert(queryStore.allQueryUIData.count(!_.summary.isActive) == 0) + + val terminateEvent1 = new StreamingQueryListener.QueryTerminatedEvent(id1, runId1, None) + listener.onQueryTerminated(terminateEvent1) + checkInactiveQueryStatus(1, Seq(id1)) + val terminateEvent2 = new StreamingQueryListener.QueryTerminatedEvent(id2, runId2, None) + listener.onQueryTerminated(terminateEvent2) + checkInactiveQueryStatus(2, Seq(id1, id2)) + val terminateEvent3 = new StreamingQueryListener.QueryTerminatedEvent(id3, runId3, None) + listener.onQueryTerminated(terminateEvent3) + checkInactiveQueryStatus(2, Seq(id2, id3)) + } + + test("test small retained progress") { + val kvStore = new ElementTrackingStore(new InMemoryStore(), sparkConf) + val conf = spark.sparkContext.conf + conf.set(StaticSQLConf.STREAMING_UI_RETAINED_PROGRESS_UPDATES.key, "5") + val listener = new StreamingQueryStatusListener(conf, kvStore) + val queryStore = new StreamingQueryStatusStore(kvStore) + + val id = UUID.randomUUID() + val runId = UUID.randomUUID() + val startEvent = new StreamingQueryListener.QueryStartedEvent( + id, runId, "test", "2016-12-05T20:54:20.827Z") + listener.onQueryStarted(startEvent) + + var batchId: Int = 0 + + def addQueryProgress(): Unit = { + val progress = mockProgressData(id, runId) + val processEvent = new streaming.StreamingQueryListener.QueryProgressEvent(progress) + listener.onQueryProgress(processEvent) + } + + def mockProgressData(id: UUID, runId: UUID): StreamingQueryProgress = { + val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 + format.setTimeZone(getTimeZone("UTC")) + + val progress = mock(classOf[StreamingQueryProgress], RETURNS_SMART_NULLS) + when(progress.id).thenReturn(id) + when(progress.runId).thenReturn(runId) + when(progress.timestamp).thenReturn(format.format(new Date(System.currentTimeMillis()))) + when(progress.inputRowsPerSecond).thenReturn(10.0) + when(progress.processedRowsPerSecond).thenReturn(12.0) + when(progress.batchId).thenReturn(batchId) + when(progress.prettyJson).thenReturn("""{"a":1}""") + + batchId += 1 + progress + } + + def checkQueryProcessData(targetNum: Int): Unit = { + eventually(timeout(10.seconds)) { + assert(queryStore.getQueryProgressData(runId).size == targetNum) + } + } + + Array.tabulate(4) { _ => addQueryProgress() } + checkQueryProcessData(4) + addQueryProgress() + checkQueryProcessData(5) + addQueryProgress() + checkQueryProcessData(5) } } 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 462206d8c5..4ce1964a19 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 @@ -40,6 +40,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled private val originalSessionLocalTimeZone = TestHive.conf.sessionLocalTimeZone + private val originalCreateHiveTable = + TestHive.conf.getConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT) def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -59,6 +61,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Fix session local timezone to America/Los_Angeles for those timezone sensitive tests // (timestamp_*) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, "America/Los_Angeles") + TestHive.setConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT, true) RuleExecutor.resetMetrics() } @@ -69,6 +72,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, originalSessionLocalTimeZone) + TestHive.setConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT, originalCreateHiveTable) // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala index 3e7c3e6799..2fb67c793d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala @@ -21,10 +21,26 @@ import org.apache.spark.sql.{AnalysisException, ShowCreateTableSuite} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.internal.HiveSerDe +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSingleton { + private var origCreateHiveTableConfig = false + + protected override def beforeAll(): Unit = { + super.beforeAll() + origCreateHiveTableConfig = + spark.conf.get(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT) + spark.conf.set(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT.key, true) + } + + protected override def afterAll(): Unit = { + spark.conf.set( + SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT.key, + origCreateHiveTableConfig) + super.afterAll() + } + test("view") { Seq(true, false).foreach { serde => withView("v1") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index ebc6cfb77d..71750e6b3a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -277,7 +277,8 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter test("Test partition mode = strict") { withSQLConf(("hive.exec.dynamic.partition.mode", "strict")) { withTable("partitioned") { - sql("CREATE TABLE partitioned (id bigint, data string) PARTITIONED BY (part string)") + sql("CREATE TABLE partitioned (id bigint, data string) USING hive " + + "PARTITIONED BY (part string)") val data = (1 to 10).map(i => (i, s"data-$i", if ((i % 2) == 0) "even" else "odd")) .toDF("id", "data", "part") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala index 483622b167..cec6ec1ee1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala @@ -38,7 +38,7 @@ class QueryPartitionSuite extends QueryTest with SQLTestUtils with TestHiveSingl testData.createOrReplaceTempView("testData") // create the table for test - sql(s"CREATE TABLE table_with_partition(key int,value string) " + + sql(s"CREATE TABLE table_with_partition(key int,value string) USING hive " + s"PARTITIONED by (ds string) location '${tmpDir.toURI}' ") sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') " + "SELECT key,value FROM testData") @@ -81,7 +81,8 @@ class QueryPartitionSuite extends QueryTest with SQLTestUtils with TestHiveSingl test("SPARK-21739: Cast expression should initialize timezoneId") { withTable("table_with_timestamp_partition") { - sql("CREATE TABLE table_with_timestamp_partition(value int) PARTITIONED BY (ts TIMESTAMP)") + sql("CREATE TABLE table_with_timestamp_partition(value int) USING hive " + + "PARTITIONED BY (ts TIMESTAMP)") sql("INSERT OVERWRITE TABLE table_with_timestamp_partition " + "PARTITION (ts = '2010-01-01 00:00:00.000') VALUES (1)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 43d1ba04c5..2ea9894301 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -165,7 +165,8 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto // Partitioned table val partTable = "part_table" withTable(partTable) { - sql(s"CREATE TABLE $partTable (key STRING, value STRING) PARTITIONED BY (ds STRING)") + sql(s"CREATE TABLE $partTable (key STRING, value STRING) USING hive " + + "PARTITIONED BY (ds STRING)") sql(s"INSERT INTO TABLE $partTable PARTITION (ds='2010-01-01') SELECT * FROM src") sql(s"INSERT INTO TABLE $partTable PARTITION (ds='2010-01-02') SELECT * FROM src") sql(s"INSERT INTO TABLE $partTable PARTITION (ds='2010-01-03') SELECT * FROM src") @@ -191,7 +192,8 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto SQLConf.PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION.key -> "True") { val checkSizeTable = "checkSizeTable" withTable(checkSizeTable) { - sql(s"CREATE TABLE $checkSizeTable (key STRING, value STRING) PARTITIONED BY (ds STRING)") + sql(s"CREATE TABLE $checkSizeTable (key STRING, value STRING) USING hive " + + "PARTITIONED BY (ds STRING)") sql(s"INSERT INTO TABLE $checkSizeTable PARTITION (ds='2010-01-01') SELECT * FROM src") sql(s"INSERT INTO TABLE $checkSizeTable PARTITION (ds='2010-01-02') SELECT * FROM src") sql(s"INSERT INTO TABLE $checkSizeTable PARTITION (ds='2010-01-03') SELECT * FROM src") @@ -274,7 +276,8 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto test("SPARK-22745 - read Hive's statistics for partition") { val tableName = "hive_stats_part_table" withTable(tableName) { - sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING)") + sql(s"CREATE TABLE $tableName (key STRING, value STRING) USING hive " + + "PARTITIONED BY (ds STRING)") sql(s"INSERT INTO TABLE $tableName PARTITION (ds='2017-01-01') SELECT * FROM src") var partition = spark.sessionState.catalog .getPartition(TableIdentifier(tableName), Map("ds" -> "2017-01-01")) @@ -296,7 +299,8 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto val tableName = "analyzeTable_part" withTable(tableName) { withTempPath { path => - sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING)") + sql(s"CREATE TABLE $tableName (key STRING, value STRING) USING hive " + + "PARTITIONED BY (ds STRING)") val partitionDates = List("2010-01-01", "2010-01-02", "2010-01-03") partitionDates.foreach { ds => @@ -321,6 +325,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto sql( s""" |CREATE TABLE $sourceTableName (key STRING, value STRING) + |USING hive |PARTITIONED BY (ds STRING) |LOCATION '${path.toURI}' """.stripMargin) @@ -338,6 +343,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto sql( s""" |CREATE TABLE $tableName (key STRING, value STRING) + |USING hive |PARTITIONED BY (ds STRING) |LOCATION '${path.toURI}' """.stripMargin) @@ -371,7 +377,8 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto } withTable(tableName) { - sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING)") + sql(s"CREATE TABLE $tableName (key STRING, value STRING) USING hive " + + "PARTITIONED BY (ds STRING)") createPartition("2010-01-01", "SELECT '1', 'A' from src") createPartition("2010-01-02", "SELECT '1', 'A' from src UNION ALL SELECT '1', 'A' from src") @@ -424,7 +431,8 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto } withTable(tableName) { - sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING, hr INT)") + sql(s"CREATE TABLE $tableName (key STRING, value STRING) USING hive " + + "PARTITIONED BY (ds STRING, hr INT)") createPartition("2010-01-01", 10, "SELECT '1', 'A' from src") createPartition("2010-01-01", 11, "SELECT '1', 'A' from src") @@ -472,7 +480,8 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto } withTable(tableName) { - sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING, hr INT)") + sql(s"CREATE TABLE $tableName (key STRING, value STRING) USING hive " + + "PARTITIONED BY (ds STRING, hr INT)") createPartition("2010-01-01", 10, "SELECT '1', 'A' from src") createPartition("2010-01-01", 11, "SELECT '1', 'A' from src") @@ -961,7 +970,8 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto Seq(false, true).foreach { autoUpdate => withSQLConf(SQLConf.AUTO_SIZE_UPDATE_ENABLED.key -> autoUpdate.toString) { withTable(table) { - sql(s"CREATE TABLE $table (i INT, j STRING) PARTITIONED BY (ds STRING, hr STRING)") + sql(s"CREATE TABLE $table (i INT, j STRING) USING hive " + + "PARTITIONED BY (ds STRING, hr STRING)") // table has two partitions initially for (ds <- Seq("2008-04-08"); hr <- Seq("11", "12")) { sql(s"INSERT OVERWRITE TABLE $table PARTITION (ds='$ds',hr='$hr') SELECT 1, 'a'") @@ -1034,6 +1044,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto sql( s""" |CREATE TABLE $managedTable (key INT, value STRING) + |USING hive |PARTITIONED BY (ds STRING, hr STRING) """.stripMargin) 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 d9ba6dd80e..684529aa33 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 @@ -798,6 +798,7 @@ class VersionsSuite extends SparkFunSuite with Logging { versionSpark.sql( """ |CREATE TABLE tbl(c1 string) + |USING hive |PARTITIONED BY (ds STRING) """.stripMargin) versionSpark.sql("INSERT OVERWRITE TABLE tbl partition (ds='2') SELECT '1'") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 2dfb8bb552..ce31e39985 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -983,7 +983,7 @@ class HiveDDLSuite } test("alter table partition - storage information") { - sql("CREATE TABLE boxes (height INT, length INT) PARTITIONED BY (width INT)") + sql("CREATE TABLE boxes (height INT, length INT) STORED AS textfile PARTITIONED BY (width INT)") sql("INSERT OVERWRITE TABLE boxes PARTITION (width=4) SELECT 4, 4") val catalog = spark.sessionState.catalog val expectedSerde = "com.sparkbricks.serde.ColumnarSerDe" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index f723c9f80c..d7129bcb37 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -88,7 +88,7 @@ class HiveSerDeSuite extends HiveComparisonTest with PlanTest with BeforeAndAfte test("Test the default fileformat for Hive-serde tables") { withSQLConf("hive.default.fileformat" -> "orc") { val (desc, exists) = extractTableDesc( - "CREATE TABLE IF NOT EXISTS fileformat_test (id int)") + "CREATE TABLE IF NOT EXISTS fileformat_test (id int) USING hive") assert(exists) assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")) assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) @@ -96,7 +96,8 @@ class HiveSerDeSuite extends HiveComparisonTest with PlanTest with BeforeAndAfte } withSQLConf("hive.default.fileformat" -> "parquet") { - val (desc, exists) = extractTableDesc("CREATE TABLE IF NOT EXISTS fileformat_test (id int)") + val (desc, exists) = extractTableDesc( + "CREATE TABLE IF NOT EXISTS fileformat_test (id int) USING hive") assert(exists) val input = desc.storage.inputFormat val output = desc.storage.outputFormat diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 4a50621d89..5b43f82f25 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -113,6 +113,7 @@ class HiveTableScanSuite extends HiveComparisonTest with SQLTestUtils with TestH sql( s""" |CREATE TABLE $table(id string) + |USING hive |PARTITIONED BY (p1 string,p2 string,p3 string,p4 string,p5 string) """.stripMargin) sql( @@ -157,6 +158,7 @@ class HiveTableScanSuite extends HiveComparisonTest with SQLTestUtils with TestH sql( s""" |CREATE TABLE $table(id string) + |USING hive |PARTITIONED BY (p1 string,p2 string,p3 string,p4 string,p5 string) """.stripMargin) sql( @@ -182,6 +184,7 @@ class HiveTableScanSuite extends HiveComparisonTest with SQLTestUtils with TestH sql( s""" |CREATE TABLE $table (id int) + |USING hive |PARTITIONED BY (a int, b int) """.stripMargin) val scan1 = getHiveTableScanExec(s"SELECT * FROM $table WHERE a = 1 AND b = 2") @@ -252,7 +255,7 @@ class HiveTableScanSuite extends HiveComparisonTest with SQLTestUtils with TestH test("SPARK-32069: Improve error message on reading unexpected directory") { withTable("t") { withTempDir { f => - sql(s"CREATE TABLE t(i LONG) LOCATION '${f.getAbsolutePath}'") + sql(s"CREATE TABLE t(i LONG) USING hive LOCATION '${f.getAbsolutePath}'") sql("INSERT INTO t VALUES(1)") val dir = new File(f.getCanonicalPath + "/data") dir.mkdir() 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 79b3c3efe5..6b82b1267b 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 @@ -2026,6 +2026,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi sql( """ |CREATE TABLE part_table (c STRING) + |STORED AS textfile |PARTITIONED BY (d STRING) """.stripMargin) sql(s"LOAD DATA LOCAL INPATH '$path/part-r-000011' " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index a25c61c96f..e996f2c6ec 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -327,20 +327,22 @@ private[hive] class TestHiveSparkSession( } if (loadTestTables) { + def createTableSQL(tblName: String): String = { + s"CREATE TABLE $tblName (key INT, value STRING) STORED AS textfile" + } // 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] = Seq( TestTable("src", - "CREATE TABLE src (key INT, value STRING) STORED AS TEXTFILE".cmd, + createTableSQL("src").cmd, s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), TestTable("src1", - "CREATE TABLE src1 (key INT, value STRING) STORED AS TEXTFILE".cmd, + createTableSQL("src1").cmd, s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), TestTable("srcpart", () => { - "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)" - .cmd.apply() + s"${createTableSQL("srcpart")} PARTITIONED BY (ds STRING, hr STRING)".cmd.apply() for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { s""" |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' @@ -349,8 +351,7 @@ private[hive] class TestHiveSparkSession( } }), TestTable("srcpart1", () => { - "CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)" - .cmd.apply() + s"${createTableSQL("srcpart1")} PARTITIONED BY (ds STRING, hr INT)".cmd.apply() for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { s""" |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}'