From a6efa7380591498a000ee89eff8460e9e6463f9d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 31 May 2017 08:03:08 +0000 Subject: [PATCH 01/22] Support to show data writing metrics. --- .../scala/org/apache/spark/util/Utils.scala | 9 ++ .../spark/sql/execution/QueryExecution.scala | 4 +- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../sql/execution/command/commands.scala | 111 +++++++++++++++--- .../datasources/FileFormatWriter.scala | 89 +++++++++++--- .../InsertIntoHadoopFsRelationCommand.scala | 21 +++- .../execution/metric/SQLMetricsSuite.scala | 109 ++++++++++++++++- .../sql/sources/PartitionedWriteSuite.scala | 21 ++-- .../hive/execution/InsertIntoHiveTable.scala | 13 +- 9 files changed, 319 insertions(+), 62 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index bbb7999e2a144..58884a1c777b9 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1002,6 +1002,15 @@ private[spark] object Utils extends Logging { } } + /** + * Lists files recursively. + */ + def recursiveList(f: File): Array[File] = { + require(f.isDirectory) + val current = f.listFiles + current ++ current.filter(_.isDirectory).flatMap(recursiveList) + } + /** * Delete a file or directory and its contents recursively. * Don't follow directories if they are symlinks. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 1ba9a79446aad..02421418c7b5a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -117,7 +117,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { * `SparkSQLDriver` for CLI applications. */ def hiveResultString(): Seq[String] = executedPlan match { - case ExecutedCommandExec(desc: DescribeTableCommand, _) => + case ExecutedCommandExec(desc: DescribeTableCommand) => // If it is a describe command for a Hive table, we want to have the output format // be similar with Hive. desc.run(sparkSession).map { @@ -128,7 +128,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { .mkString("\t") } // SHOW TABLES in Hive only output table names, while ours output database, table name, isTemp. - case command @ ExecutedCommandExec(s: ShowTablesCommand, _) if !s.isExtended => + case command @ ExecutedCommandExec(s: ShowTablesCommand) if !s.isExtended => command.executeCollect().map(_.getString(1)) case other => val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index f13294c925e36..4fb906478db0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -346,7 +346,9 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case r: RunnableCommand => ExecutedCommandExec(r, r.children.map(planLater)) :: Nil + case w: WriteOutFileCommand => WrittenFileCommandExec(w, w.children.map(planLater)) :: Nil + + case r: RunnableCommand => ExecutedCommandExec(r) :: Nil case MemoryPlan(sink, output) => val encoder = RowEncoder(sink.schema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 99d81c49f1e3b..63da4219837c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -24,31 +24,44 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary import org.apache.spark.sql.execution.debug._ +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types._ /** - * A logical command that is executed for its side-effects. `RunnableCommand`s are - * wrapped in `ExecutedCommand` during execution. + * A logical command specialized for writing data out. `WriteOutFileCommand`s are + * wrapped in `WrittenFileCommandExec` during execution. */ -trait RunnableCommand extends logical.Command { - def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { +trait WriteOutFileCommand extends logical.Command { + def run( + sparkSession: SparkSession, + children: Seq[SparkPlan], + metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { throw new NotImplementedError } +} +/** + * A logical command that is executed for its side-effects. `RunnableCommand`s are + * wrapped in `ExecutedCommand` during execution. + */ +trait RunnableCommand extends logical.Command { def run(sparkSession: SparkSession): Seq[Row] = { throw new NotImplementedError } } /** - * A physical operator that executes the run method of a `RunnableCommand` and + * A physical operator that executes the run method of a `logical.Command` and * saves the result to prevent multiple executions. */ -case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) extends SparkPlan { +trait CommandExec extends SparkPlan { + val cmd: logical.Command + /** * A concrete command should override this lazy field to wrap up any side effects caused by the * command or any other computation that should be evaluated exactly once. The value of this field @@ -58,15 +71,7 @@ case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) e * The `execute()` method of all the physical command classes should reference `sideEffectResult` * so that the command can be executed eagerly right after the command query is created. */ - protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { - val converter = CatalystTypeConverters.createToCatalystConverter(schema) - val rows = if (children.isEmpty) { - cmd.run(sqlContext.sparkSession) - } else { - cmd.run(sqlContext.sparkSession, children) - } - rows.map(converter(_).asInstanceOf[InternalRow]) - } + protected[sql] val sideEffectResult: Seq[InternalRow] override def innerChildren: Seq[QueryPlan[_]] = cmd.innerChildren @@ -85,6 +90,80 @@ case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) e } } +/** + * A physical operator specialized to execute the run method of a `WriteOutFileCommand`, + * save the result to prevent multiple executions, and record necessary metrics for UI. + */ +case class WrittenFileCommandExec( + cmd: WriteOutFileCommand, + children: Seq[SparkPlan]) extends CommandExec { + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of output rows"), + "writingTime" -> SQLMetrics.createMetric(sqlContext.sparkContext, "writing data out time (ms)"), + "dynamicPartNum" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of dynamic part"), + "fileNum" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of written files"), + "fileBytes" -> SQLMetrics.createMetric(sqlContext.sparkContext, "bytes of written files")) + + // Callback used to update metrics returned from the operation of writing data out. + private def updateDriverMetrics(writeTaskSummary: Seq[ExecutedWriteSummary]): Unit = { + var partitionNum = 0 + var fileNum = 0 + var fileBytes: Long = 0L + var numOutput: Long = 0L + + writeTaskSummary.foreach { summary => + partitionNum += summary.updatedPartitions.size + fileNum += summary.writtenFileNum + fileBytes += summary.writtenBytes + numOutput += summary.numOutputRows + } + + val partitionMetric = metrics("dynamicPartNum") + val fileNumMetric = metrics("fileNum") + val fileBytesMetric = metrics("fileBytes") + val numOutputRows = metrics("numOutputRows") + partitionMetric.add(partitionNum) + fileNumMetric.add(fileNum) + fileBytesMetric.add(fileBytes) + numOutputRows.add(numOutput) + + val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, + partitionMetric :: fileNumMetric :: fileBytesMetric :: numOutputRows :: Nil) + } + + protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { + assert(children.nonEmpty) + + val converter = CatalystTypeConverters.createToCatalystConverter(schema) + val startTime = System.nanoTime() + val rows = cmd.run(sqlContext.sparkSession, children, updateDriverMetrics) + val timeTakenMs = (System.nanoTime() - startTime) / 1000 / 1000 + val writingTime = metrics("writingTime") + writingTime.add(timeTakenMs) + + val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, + writingTime :: Nil) + rows.map(converter(_).asInstanceOf[InternalRow]) + } +} + +/** + * A physical operator specialized to execute the run method of a `RunnableCommand` and + * save the result to prevent multiple executions. + */ +case class ExecutedCommandExec(cmd: RunnableCommand) extends CommandExec { + override protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { + val converter = CatalystTypeConverters.createToCatalystConverter(schema) + val rows = cmd.run(sqlContext.sparkSession) + rows.map(converter(_).asInstanceOf[InternalRow]) + } + + override def children: Seq[SparkPlan] = Nil +} + /** * An explain command for users to see how a command will be executed. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index afe454f714c47..31d60bd31a640 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -22,7 +22,7 @@ import java.util.{Date, UUID} import scala.collection.mutable import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl @@ -81,7 +81,7 @@ object FileFormatWriter extends Logging { } /** The result of a successful write task. */ - private case class WriteTaskResult(commitMsg: TaskCommitMessage, updatedPartitions: Set[String]) + private case class WriteTaskResult(commitMsg: TaskCommitMessage, summary: ExecutedWriteSummary) /** * Basic work flow of this command is: @@ -103,7 +103,7 @@ object FileFormatWriter extends Logging { hadoopConf: Configuration, partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], - refreshFunction: (Seq[TablePartitionSpec]) => Unit, + refreshFunction: (Seq[ExecutedWriteSummary]) => Unit, options: Map[String, String]): Unit = { val job = Job.getInstance(hadoopConf) @@ -195,12 +195,10 @@ object FileFormatWriter extends Logging { }) val commitMsgs = ret.map(_.commitMsg) - val updatedPartitions = ret.flatMap(_.updatedPartitions) - .distinct.map(PartitioningUtils.parsePathFragment) committer.commitJob(job, commitMsgs) logInfo(s"Job ${job.getJobID} committed.") - refreshFunction(updatedPartitions) + refreshFunction(ret.map(_.summary)) } catch { case cause: Throwable => logError(s"Aborting job ${job.getJobID}.", cause) committer.abortJob(job) @@ -246,9 +244,9 @@ object FileFormatWriter extends Logging { try { Utils.tryWithSafeFinallyAndFailureCallbacks(block = { // Execute the task to write rows out and commit the task. - val outputPartitions = writeTask.execute(iterator) + val summary = writeTask.execute(iterator) writeTask.releaseResources() - WriteTaskResult(committer.commitTask(taskAttemptContext), outputPartitions) + WriteTaskResult(committer.commitTask(taskAttemptContext), summary) })(catchBlock = { // If there is an error, release resource and then abort the task try { @@ -271,11 +269,26 @@ object FileFormatWriter extends Logging { */ private trait ExecuteWriteTask { /** - * Writes data out to files, and then returns the list of partition strings written out. - * The list of partitions is sent back to the driver and used to update the catalog. + * Writes data out to files, and then returns the summary of relative information which + * includes the list of partition strings written out. The list of partitions is sent back + * to the driver and used to update the catalog. Other information will be sent back to the + * driver too and used to update the metrics in UI. */ - def execute(iterator: Iterator[InternalRow]): Set[String] + def execute(iterator: Iterator[InternalRow]): ExecutedWriteSummary def releaseResources(): Unit + + /** + * A helper function used to determine the size in bytes of a written file. + */ + protected def getFileSize(conf: Configuration, filePath: String): Long = { + if (filePath != null) { + val path = new Path(filePath) + val fs = path.getFileSystem(conf) + fs.getFileStatus(path).getLen() + } else { + 0L + } + } } /** Writes data to a single directory (used for non-dynamic-partition writes). */ @@ -285,23 +298,26 @@ object FileFormatWriter extends Logging { committer: FileCommitProtocol) extends ExecuteWriteTask { private[this] var currentWriter: OutputWriter = _ + private[this] var currentPath: String = _ private def newOutputWriter(fileCounter: Int): Unit = { val ext = description.outputWriterFactory.getFileExtension(taskAttemptContext) - val tmpFilePath = committer.newTaskTempFile( + currentPath = committer.newTaskTempFile( taskAttemptContext, None, f"-c$fileCounter%03d" + ext) currentWriter = description.outputWriterFactory.newInstance( - path = tmpFilePath, + path = currentPath, dataSchema = description.dataColumns.toStructType, context = taskAttemptContext) } - override def execute(iter: Iterator[InternalRow]): Set[String] = { + override def execute(iter: Iterator[InternalRow]): ExecutedWriteSummary = { var fileCounter = 0 var recordsInFile: Long = 0L + var totalFileSize: Long = 0L + var totalOutputRows: Long = 0L newOutputWriter(fileCounter) while (iter.hasNext) { if (description.maxRecordsPerFile > 0 && recordsInFile >= description.maxRecordsPerFile) { @@ -311,15 +327,19 @@ object FileFormatWriter extends Logging { recordsInFile = 0 releaseResources() + totalFileSize += getFileSize(taskAttemptContext.getConfiguration, currentPath) newOutputWriter(fileCounter) } val internalRow = iter.next() currentWriter.write(internalRow) + totalOutputRows += 1 recordsInFile += 1 } releaseResources() - Set.empty + totalFileSize += getFileSize(taskAttemptContext.getConfiguration, currentPath) + ExecutedWriteSummary(updatedPartitions = Set.empty, writtenFileNum = fileCounter + 1, + writtenBytes = totalFileSize, numOutputRows = totalOutputRows) } override def releaseResources(): Unit = { @@ -345,6 +365,8 @@ object FileFormatWriter extends Logging { // currentWriter is initialized whenever we see a new key private var currentWriter: OutputWriter = _ + private var currentPath: String = _ + /** Expressions that given partition columns build a path string like: col1=val/col2=val/... */ private def partitionPathExpression: Seq[Expression] = { desc.partitionColumns.zipWithIndex.flatMap { case (c, i) => @@ -400,19 +422,19 @@ object FileFormatWriter extends Logging { case _ => None } - val path = if (customPath.isDefined) { + currentPath = if (customPath.isDefined) { committer.newTaskTempFileAbsPath(taskAttemptContext, customPath.get, ext) } else { committer.newTaskTempFile(taskAttemptContext, partDir, ext) } currentWriter = desc.outputWriterFactory.newInstance( - path = path, + path = currentPath, dataSchema = desc.dataColumns.toStructType, context = taskAttemptContext) } - override def execute(iter: Iterator[InternalRow]): Set[String] = { + override def execute(iter: Iterator[InternalRow]): ExecutedWriteSummary = { val getPartitionColsAndBucketId = UnsafeProjection.create( desc.partitionColumns ++ desc.bucketIdExpression, desc.allColumns) @@ -426,11 +448,17 @@ object FileFormatWriter extends Logging { // If anything below fails, we should abort the task. var recordsInFile: Long = 0L var fileCounter = 0 + var totalFileCounter = 0 + var totalFileSize: Long = 0L + var totalOutputRows: Long = 0L var currentPartColsAndBucketId: UnsafeRow = null val updatedPartitions = mutable.Set[String]() for (row <- iter) { val nextPartColsAndBucketId = getPartitionColsAndBucketId(row) if (currentPartColsAndBucketId != nextPartColsAndBucketId) { + if (currentPartColsAndBucketId != null) { + totalFileCounter += (fileCounter + 1) + } // See a new partition or bucket - write to a new partition dir (or a new bucket file). currentPartColsAndBucketId = nextPartColsAndBucketId.copy() logDebug(s"Writing partition: $currentPartColsAndBucketId") @@ -439,6 +467,7 @@ object FileFormatWriter extends Logging { fileCounter = 0 releaseResources() + totalFileSize += getFileSize(taskAttemptContext.getConfiguration, currentPath) newOutputWriter(currentPartColsAndBucketId, getPartPath, fileCounter, updatedPartitions) } else if (desc.maxRecordsPerFile > 0 && recordsInFile >= desc.maxRecordsPerFile) { @@ -450,14 +479,23 @@ object FileFormatWriter extends Logging { s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") releaseResources() + totalFileSize += getFileSize(taskAttemptContext.getConfiguration, currentPath) newOutputWriter(currentPartColsAndBucketId, getPartPath, fileCounter, updatedPartitions) } currentWriter.write(getOutputRow(row)) + totalOutputRows += 1 recordsInFile += 1 } + if (currentPartColsAndBucketId != null) { + totalFileCounter += (fileCounter + 1) + } releaseResources() updatedPartitions.toSet + totalFileSize += getFileSize(taskAttemptContext.getConfiguration, currentPath) + ExecutedWriteSummary(updatedPartitions = updatedPartitions.toSet, + writtenFileNum = totalFileCounter, writtenBytes = totalFileSize, + numOutputRows = totalOutputRows) } override def releaseResources(): Unit = { @@ -471,3 +509,18 @@ object FileFormatWriter extends Logging { } } } + +/** + * Wrapper class for the information of writing data out. + * + * @param updatedPartitions the partitions updated during writing data out. Only valid + * for dynamic partition. + * @param numOutputRows the number of output rows. + * @param writtenFileNum the total number of files written out. + * @param writtenBytes the total bytes written out. + */ +case class ExecutedWriteSummary( + updatedPartitions: Set[String], + numOutputRows: Long, + writtenFileNum: Int, + writtenBytes: Long) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 00aa1240886e4..a6b556bb61e31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -53,12 +53,15 @@ case class InsertIntoHadoopFsRelationCommand( mode: SaveMode, catalogTable: Option[CatalogTable], fileIndex: Option[FileIndex]) - extends RunnableCommand { + extends WriteOutFileCommand { import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName override def children: Seq[LogicalPlan] = query :: Nil - override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { + override def run( + sparkSession: SparkSession, + children: Seq[SparkPlan], + metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { assert(children.length == 1) // Most formats don't do well with duplicate columns, so lets not allow that @@ -123,8 +126,16 @@ case class InsertIntoHadoopFsRelationCommand( if (doInsertion) { - // Callback for updating metastore partition metadata after the insertion job completes. - def refreshPartitionsCallback(updatedPartitions: Seq[TablePartitionSpec]): Unit = { + // Callback for updating metric and metastore partition metadata + // after the insertion job completes. + def refreshCallback(summary: Seq[ExecutedWriteSummary]): Unit = { + val updatedPartitions = summary.flatMap(_.updatedPartitions) + .distinct.map(PartitioningUtils.parsePathFragment) + + // Updating metrics. + metricsCallback(summary) + + // Updating metastore partition metadata. if (partitionsTrackedByCatalog) { val newPartitions = updatedPartitions.toSet -- initialMatchingPartitions if (newPartitions.nonEmpty) { @@ -154,7 +165,7 @@ case class InsertIntoHadoopFsRelationCommand( hadoopConf = hadoopConf, partitionColumns = partitionColumns, bucketSpec = bucketSpec, - refreshFunction = refreshPartitionsCallback, + refreshFunction = refreshCallback, options = options) // refresh cached files in FileIndex diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index a4e62f1d16792..315a668d0d747 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.ui.SparkPlanGraph import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.util.{AccumulatorContext, JsonProtocol} +import org.apache.spark.util.{AccumulatorContext, JsonProtocol, Utils} class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ @@ -91,6 +91,51 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } } + /** + * Run the given function and return latest execution id. + * + * @param func the given function to run. + */ + private def getLatestExecutionId(func: () => Unit): Long = { + val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet + // Run the given function to trigger query execution. + func() + sparkContext.listenerBus.waitUntilEmpty(10000) + val executionIds = + spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) + assert(executionIds.size === 1) + executionIds.head + } + + /** + * Get execution metrics for the given execution id and verify metrics values. + * + * @param executionId the given execution id. + * @param verifyFuncs functions used to verify the values of metrics. + */ + private def verifyWriteDataMetrics(executionId: Long, verifyFuncs: Seq[Int => Boolean]): Unit = { + val executionData = spark.sharedState.listener.getExecution(executionId).get + val executedNode = executionData.physicalPlanGraph.nodes.head + + val metricsNames = Seq( + "number of written files", + "number of dynamic part", + "bytes of written files", + "number of output rows", + "writing data out time (ms)") + + val metrics = + spark.sharedState.listener.getExecutionMetrics(executionId) + + metricsNames.zip(verifyFuncs).foreach { case (metricsName, verifyFunc) => + val sqlMetric = executedNode.metrics.find(_.name == metricsName) + assert(sqlMetric.isDefined) + val accumulatorId = sqlMetric.get.accumulatorId + val metricValue = metrics(accumulatorId).replaceAll(",", "").toInt + assert(verifyFunc(metricValue)) + } + } + test("LocalTableScanExec computes metrics in collect and take") { val df1 = spark.createDataset(Seq(1, 2, 3)) val logical = df1.queryExecution.logical @@ -288,6 +333,68 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } } + test("writing data out metrics") { + withTable("writeToTable") { + // Create the table. + Seq.empty[(Int, Int)].toDF("i", "j").write.mode("overwrite").saveAsTable("writeToTable") + + val executionId1 = getLatestExecutionId { () => + Seq((1, 2)).toDF("i", "j").write.insertInto("writeToTable") + } + // written 1 file, 1 row, 0 dynamic partition. + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 1, _ > 0) + verifyWriteDataMetrics(executionId1, verifyFuncs1) + + val executionId2 = getLatestExecutionId { () => + Seq((3, 4), (5, 6), (7, 8)).toDF("i", "j").repartition(1) + .write.insertInto("writeToTable") + } + // written 1 file, 3 rows, 0 dynamic partition. + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 3, _ > 0) + verifyWriteDataMetrics(executionId2, verifyFuncs2) + + val executionId3 = getLatestExecutionId { () => + Seq((9, 10), (11, 12)).toDF("i", "j").repartition(2) + .write.insertInto("writeToTable") + } + // written 2 files, 2 rows, 0 dynamic partition. + val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ > 0, _ == 2, _ > 0) + verifyWriteDataMetrics(executionId3, verifyFuncs3) + } + } + + test("writing data out metrics: dynamic partition") { + withTempDir { f => + val df = + spark.range(start = 0, end = 4, step = 1, numPartitions = 1).selectExpr("id", "id id1") + val executionId1 = getLatestExecutionId { () => + df + .write + .partitionBy("id") + .option("maxRecordsPerFile", 1) + .mode("overwrite") + .parquet(f.getAbsolutePath) + } + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) + // written 4 files, 4 rows, 4 dynamic partitions. + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 4, _ > 0) + verifyWriteDataMetrics(executionId1, verifyFuncs1) + + val executionId2 = getLatestExecutionId { () => + df.union(df).repartition(2, $"id") + .write + .partitionBy("id") + .option("maxRecordsPerFile", 2) + .mode("overwrite") + .parquet(f.getAbsolutePath) + } + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) + // written 4 files, 8 rows, 4 dynamic partitions. + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 8, _ > 0) + verifyWriteDataMetrics(executionId2, verifyFuncs2) + } + } + test("save metrics") { withTempPath { file => // person creates a temporary view. get the DF before listing previous execution IDs diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala index a2f3afe3ce236..6f998aa60faf5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala @@ -91,15 +91,15 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext { withTempDir { f => spark.range(start = 0, end = 4, step = 1, numPartitions = 1) .write.option("maxRecordsPerFile", 1).mode("overwrite").parquet(f.getAbsolutePath) - assert(recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) spark.range(start = 0, end = 4, step = 1, numPartitions = 1) .write.option("maxRecordsPerFile", 2).mode("overwrite").parquet(f.getAbsolutePath) - assert(recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 2) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 2) spark.range(start = 0, end = 4, step = 1, numPartitions = 1) .write.option("maxRecordsPerFile", -1).mode("overwrite").parquet(f.getAbsolutePath) - assert(recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 1) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 1) } } @@ -111,7 +111,7 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext { .option("maxRecordsPerFile", 1) .mode("overwrite") .parquet(f.getAbsolutePath) - assert(recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) } } @@ -138,14 +138,14 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext { val df = Seq((1, ts)).toDF("i", "ts") withTempPath { f => df.write.partitionBy("ts").parquet(f.getAbsolutePath) - val files = recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) + val files = Utils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) assert(files.length == 1) checkPartitionValues(files.head, "2016-12-01 00:00:00") } withTempPath { f => df.write.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") .partitionBy("ts").parquet(f.getAbsolutePath) - val files = recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) + val files = Utils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) assert(files.length == 1) // use timeZone option "GMT" to format partition value. checkPartitionValues(files.head, "2016-12-01 08:00:00") @@ -153,18 +153,11 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext { withTempPath { f => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") { df.write.partitionBy("ts").parquet(f.getAbsolutePath) - val files = recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) + val files = Utils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) assert(files.length == 1) // if there isn't timeZone option, then use session local timezone. checkPartitionValues(files.head, "2016-12-01 08:00:00") } } } - - /** Lists files recursively. */ - private def recursiveList(f: File): Array[File] = { - require(f.isDirectory) - val current = f.listFiles - current ++ current.filter(_.isDirectory).flatMap(recursiveList) - } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 392b7cfaa8eff..594f348d27d68 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -37,8 +37,8 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.execution.datasources.FileFormatWriter +import org.apache.spark.sql.execution.command.WriteOutFileCommand +import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, FileFormatWriter} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.sql.hive.client.{HiveClientImpl, HiveVersion} @@ -80,7 +80,7 @@ case class InsertIntoHiveTable( partition: Map[String, Option[String]], query: LogicalPlan, overwrite: Boolean, - ifPartitionNotExists: Boolean) extends RunnableCommand { + ifPartitionNotExists: Boolean) extends WriteOutFileCommand { override def children: Seq[LogicalPlan] = query :: Nil @@ -231,7 +231,10 @@ case class InsertIntoHiveTable( * `org.apache.hadoop.hive.serde2.SerDe` and the * `org.apache.hadoop.mapred.OutputFormat` provided by the table definition. */ - override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { + override def run( + sparkSession: SparkSession, + children: Seq[SparkPlan], + metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { assert(children.length == 1) val sessionState = sparkSession.sessionState @@ -354,7 +357,7 @@ case class InsertIntoHiveTable( hadoopConf = hadoopConf, partitionColumns = partitionAttributes, bucketSpec = None, - refreshFunction = _ => (), + refreshFunction = metricsCallback, options = Map.empty) if (partition.nonEmpty) { From 2ff2fa13bc0b46a1074c2814ad69f23516664968 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 31 May 2017 14:05:16 +0000 Subject: [PATCH 02/22] Update metrics for CreateHiveTableAsSelectCommand. --- .../sql/execution/command/commands.scala | 2 - .../execution/metric/SQLMetricsSuite.scala | 123 ++++++++++-------- .../CreateHiveTableAsSelectCommand.scala | 27 +++- 3 files changed, 88 insertions(+), 64 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 63da4219837c5..27d169f8abe2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -134,8 +134,6 @@ case class WrittenFileCommandExec( } protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { - assert(children.nonEmpty) - val converter = CatalystTypeConverters.createToCatalystConverter(schema) val startTime = System.nanoTime() val rows = cmd.run(sqlContext.sparkSession, children, updateDriverMetrics) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 315a668d0d747..fdd6fe6a7c403 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -91,51 +91,6 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } } - /** - * Run the given function and return latest execution id. - * - * @param func the given function to run. - */ - private def getLatestExecutionId(func: () => Unit): Long = { - val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet - // Run the given function to trigger query execution. - func() - sparkContext.listenerBus.waitUntilEmpty(10000) - val executionIds = - spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) - assert(executionIds.size === 1) - executionIds.head - } - - /** - * Get execution metrics for the given execution id and verify metrics values. - * - * @param executionId the given execution id. - * @param verifyFuncs functions used to verify the values of metrics. - */ - private def verifyWriteDataMetrics(executionId: Long, verifyFuncs: Seq[Int => Boolean]): Unit = { - val executionData = spark.sharedState.listener.getExecution(executionId).get - val executedNode = executionData.physicalPlanGraph.nodes.head - - val metricsNames = Seq( - "number of written files", - "number of dynamic part", - "bytes of written files", - "number of output rows", - "writing data out time (ms)") - - val metrics = - spark.sharedState.listener.getExecutionMetrics(executionId) - - metricsNames.zip(verifyFuncs).foreach { case (metricsName, verifyFunc) => - val sqlMetric = executedNode.metrics.find(_.name == metricsName) - assert(sqlMetric.isDefined) - val accumulatorId = sqlMetric.get.accumulatorId - val metricValue = metrics(accumulatorId).replaceAll(",", "").toInt - assert(verifyFunc(metricValue)) - } - } - test("LocalTableScanExec computes metrics in collect and take") { val df1 = spark.createDataset(Seq(1, 2, 3)) val logical = df1.queryExecution.logical @@ -334,40 +289,44 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } test("writing data out metrics") { + import SQLMetricsSuite._ + withTable("writeToTable") { // Create the table. Seq.empty[(Int, Int)].toDF("i", "j").write.mode("overwrite").saveAsTable("writeToTable") - val executionId1 = getLatestExecutionId { () => + val executionId1 = getLatestExecutionId(spark) { () => Seq((1, 2)).toDF("i", "j").write.insertInto("writeToTable") } // written 1 file, 1 row, 0 dynamic partition. val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 1, _ > 0) - verifyWriteDataMetrics(executionId1, verifyFuncs1) + verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) - val executionId2 = getLatestExecutionId { () => + val executionId2 = getLatestExecutionId(spark) { () => Seq((3, 4), (5, 6), (7, 8)).toDF("i", "j").repartition(1) .write.insertInto("writeToTable") } // written 1 file, 3 rows, 0 dynamic partition. val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 3, _ > 0) - verifyWriteDataMetrics(executionId2, verifyFuncs2) + verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) - val executionId3 = getLatestExecutionId { () => + val executionId3 = getLatestExecutionId(spark) { () => Seq((9, 10), (11, 12)).toDF("i", "j").repartition(2) .write.insertInto("writeToTable") } // written 2 files, 2 rows, 0 dynamic partition. val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ > 0, _ == 2, _ > 0) - verifyWriteDataMetrics(executionId3, verifyFuncs3) + verifyWriteDataMetrics(spark, executionId3, verifyFuncs3) } } test("writing data out metrics: dynamic partition") { + import SQLMetricsSuite._ + withTempDir { f => val df = spark.range(start = 0, end = 4, step = 1, numPartitions = 1).selectExpr("id", "id id1") - val executionId1 = getLatestExecutionId { () => + val executionId1 = getLatestExecutionId(spark) { () => df .write .partitionBy("id") @@ -378,9 +337,9 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) // written 4 files, 4 rows, 4 dynamic partitions. val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 4, _ > 0) - verifyWriteDataMetrics(executionId1, verifyFuncs1) + verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) - val executionId2 = getLatestExecutionId { () => + val executionId2 = getLatestExecutionId(spark) { () => df.union(df).repartition(2, $"id") .write .partitionBy("id") @@ -391,7 +350,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) // written 4 files, 8 rows, 4 dynamic partitions. val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 8, _ > 0) - verifyWriteDataMetrics(executionId2, verifyFuncs2) + verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) } } @@ -542,3 +501,57 @@ object InputOutputMetricsHelper { listener.getResults() } } + +/** + * This object wraps few methods used to verify SQL metrics when writing data out. The methods are + * also used in the tests in Hive. + */ +object SQLMetricsSuite { + /** + * Run the given function and return latest execution id. + * + * @param func the given function to run. + */ + def getLatestExecutionId(spark: SparkSession)(func: () => Unit): Long = { + val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet + // Run the given function to trigger query execution. + func() + spark.sparkContext.listenerBus.waitUntilEmpty(10000) + val executionIds = + spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) + assert(executionIds.size == 1) + executionIds.head + } + + /** + * Get execution metrics for the given execution id and verify metrics values. + * + * @param executionId the given execution id. + * @param verifyFuncs functions used to verify the values of metrics. + */ + def verifyWriteDataMetrics( + spark: SparkSession, + executionId: Long, + verifyFuncs: Seq[Int => Boolean]): Unit = { + val executionData = spark.sharedState.listener.getExecution(executionId).get + val executedNode = executionData.physicalPlanGraph.nodes.head + + val metricsNames = Seq( + "number of written files", + "number of dynamic part", + "bytes of written files", + "number of output rows", + "writing data out time (ms)") + + val metrics = + spark.sharedState.listener.getExecutionMetrics(executionId) + + metricsNames.zip(verifyFuncs).foreach { case (metricsName, verifyFunc) => + val sqlMetric = executedNode.metrics.find(_.name == metricsName) + assert(sqlMetric.isDefined) + val accumulatorId = sqlMetric.get.accumulatorId + val metricValue = metrics(accumulatorId).replaceAll(",", "").toInt + assert(verifyFunc(metricValue)) + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 65e8b4e3c725c..5aa1524310326 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -23,7 +23,9 @@ import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} -import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.command.{WriteOutFileCommand, WrittenFileCommandExec} +import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary /** @@ -37,13 +39,16 @@ case class CreateHiveTableAsSelectCommand( tableDesc: CatalogTable, query: LogicalPlan, mode: SaveMode) - extends RunnableCommand { + extends WriteOutFileCommand { private val tableIdentifier = tableDesc.identifier override def innerChildren: Seq[LogicalPlan] = Seq(query) - override def run(sparkSession: SparkSession): Seq[Row] = { + override def run( + sparkSession: SparkSession, + children: Seq[SparkPlan], + metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { if (sparkSession.sessionState.catalog.tableExists(tableIdentifier)) { assert(mode != SaveMode.Overwrite, s"Expect the table $tableIdentifier has been dropped when the save mode is Overwrite") @@ -56,13 +61,17 @@ case class CreateHiveTableAsSelectCommand( return Seq.empty } - sparkSession.sessionState.executePlan( + val qe = sparkSession.sessionState.executePlan( InsertIntoTable( UnresolvedRelation(tableIdentifier), Map(), query, overwrite = false, - ifPartitionNotExists = false)).toRdd + ifPartitionNotExists = false)) + val insertCommand = qe.executedPlan.collect { + case w: WrittenFileCommandExec => w + }.head + insertCommand.cmd.run(sparkSession, insertCommand.children, metricsCallback) } else { // TODO ideally, we should get the output data ready first and then // add the relation into catalog, just in case of failure occurs while data @@ -72,13 +81,17 @@ case class CreateHiveTableAsSelectCommand( tableDesc.copy(schema = query.schema), ignoreIfExists = false) try { - sparkSession.sessionState.executePlan( + val qe = sparkSession.sessionState.executePlan( InsertIntoTable( UnresolvedRelation(tableIdentifier), Map(), query, overwrite = true, - ifPartitionNotExists = false)).toRdd + ifPartitionNotExists = false)) + val insertCommand = qe.executedPlan.collect { + case w: WrittenFileCommandExec => w + }.head + insertCommand.cmd.run(sparkSession, insertCommand.children, metricsCallback) } catch { case NonFatal(e) => // drop the created table. From 143764cf266b4cec4e1d2c529a3673cb95fcc523 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 1 Jun 2017 06:58:30 +0000 Subject: [PATCH 03/22] Support metrics of CreateDataSourceTableAsSelectCommand. Add test cases for hive. --- .../sql/execution/command/commands.scala | 35 +++--- .../command/createDataSourceTables.scala | 37 +++++- .../execution/datasources/DataSource.scala | 22 +++- .../execution/metric/SQLMetricsSuite.scala | 8 +- .../sql/hive/execution/SQLMetricsSuite.scala | 106 ++++++++++++++++++ 5 files changed, 182 insertions(+), 26 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 27d169f8abe2f..b9da1e21063d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.{Row, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary import org.apache.spark.sql.execution.debug._ -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types._ @@ -37,6 +37,18 @@ import org.apache.spark.sql.types._ * wrapped in `WrittenFileCommandExec` during execution. */ trait WriteOutFileCommand extends logical.Command { + + /** + * Those metrics will be updated once the command finishes writing data out. Those metrics will + * be taken by `WrittenFileCommandExe` as its metrics when showing in UI. + */ + def metrics(sqlContext: SQLContext): Map[String, SQLMetric] = Map( + "numOutputRows" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of output rows"), + "writingTime" -> SQLMetrics.createMetric(sqlContext.sparkContext, "writing data out time (ms)"), + "dynamicPartNum" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of dynamic part"), + "fileNum" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of written files"), + "fileBytes" -> SQLMetrics.createMetric(sqlContext.sparkContext, "bytes of written files")) + def run( sparkSession: SparkSession, children: Seq[SparkPlan], @@ -98,12 +110,7 @@ case class WrittenFileCommandExec( cmd: WriteOutFileCommand, children: Seq[SparkPlan]) extends CommandExec { - override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of output rows"), - "writingTime" -> SQLMetrics.createMetric(sqlContext.sparkContext, "writing data out time (ms)"), - "dynamicPartNum" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of dynamic part"), - "fileNum" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of written files"), - "fileBytes" -> SQLMetrics.createMetric(sqlContext.sparkContext, "bytes of written files")) + override lazy val metrics = cmd.metrics(sqlContext) // Callback used to update metrics returned from the operation of writing data out. private def updateDriverMetrics(writeTaskSummary: Seq[ExecutedWriteSummary]): Unit = { @@ -138,12 +145,12 @@ case class WrittenFileCommandExec( val startTime = System.nanoTime() val rows = cmd.run(sqlContext.sparkSession, children, updateDriverMetrics) val timeTakenMs = (System.nanoTime() - startTime) / 1000 / 1000 - val writingTime = metrics("writingTime") - writingTime.add(timeTakenMs) - - val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, - writingTime :: Nil) + metrics.get("writingTime").foreach { writingTime => + writingTime.add(timeTakenMs) + val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, + writingTime :: Nil) + } rows.map(converter(_).asInstanceOf[InternalRow]) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 729bd39d821c9..237a22fdd6bab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -24,7 +24,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.sources.BaseRelation /** @@ -120,11 +122,31 @@ case class CreateDataSourceTableAsSelectCommand( table: CatalogTable, mode: SaveMode, query: LogicalPlan) - extends RunnableCommand { + extends WriteOutFileCommand { + + /** + * The code path this command writes data out depends on the type of data source: + * + * For FileFormat-based data sources, `InsertIntoHadoopFsRelationCommand` is invoked and we + * can update metrics. + * + * For other data sources, `CreatableRelationProvider.createRelation` will be called. We can't + * record metrics for that. So we will return empty metrics map. + */ + override def metrics(sqlContext: SQLContext): Map[String, SQLMetric] = { + if (DataSource.isFileFormat(table.provider.get)) { + super.metrics(sqlContext) + } else { + Map.empty + } + } override def innerChildren: Seq[LogicalPlan] = Seq(query) - override def run(sparkSession: SparkSession): Seq[Row] = { + override def run( + sparkSession: SparkSession, + children: Seq[SparkPlan], + metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { assert(table.tableType != CatalogTableType.VIEW) assert(table.provider.isDefined) @@ -146,7 +168,8 @@ case class CreateDataSourceTableAsSelectCommand( } saveDataIntoTable( - sparkSession, table, table.storage.locationUri, query, SaveMode.Append, tableExists = true) + sparkSession, table, table.storage.locationUri, query, SaveMode.Append, tableExists = true, + metricsCallback = metricsCallback) } else { assert(table.schema.isEmpty) @@ -156,7 +179,8 @@ case class CreateDataSourceTableAsSelectCommand( table.storage.locationUri } val result = saveDataIntoTable( - sparkSession, table, tableLocation, query, SaveMode.Overwrite, tableExists = false) + sparkSession, table, tableLocation, query, SaveMode.Overwrite, tableExists = false, + metricsCallback = metricsCallback) val newTable = table.copy( storage = table.storage.copy(locationUri = tableLocation), // We will use the schema of resolved.relation as the schema of the table (instead of @@ -183,7 +207,8 @@ case class CreateDataSourceTableAsSelectCommand( tableLocation: Option[URI], data: LogicalPlan, mode: SaveMode, - tableExists: Boolean): BaseRelation = { + tableExists: Boolean, + metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): BaseRelation = { // Create the relation based on the input logical plan: `data`. val pathOption = tableLocation.map("path" -> CatalogUtils.URIToString(_)) val dataSource = DataSource( @@ -195,7 +220,7 @@ case class CreateDataSourceTableAsSelectCommand( catalogTable = if (tableExists) Some(table) else None) try { - dataSource.writeAndRead(mode, query) + dataSource.writeAndRead(mode, query, Some(metricsCallback)) } catch { case ex: AnalysisException => logError(s"Failed to write to table ${table.identifier.unquotedString}", ex) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 958715eefa0a2..6d10c2ec4be11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution.command.WrittenFileCommandExec import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider import org.apache.spark.sql.execution.datasources.json.JsonFileFormat @@ -436,7 +437,10 @@ case class DataSource( * Writes the given [[LogicalPlan]] out to this [[DataSource]] and returns a [[BaseRelation]] for * the following reading. */ - def writeAndRead(mode: SaveMode, data: LogicalPlan): BaseRelation = { + def writeAndRead( + mode: SaveMode, + data: LogicalPlan, + metricsCallback: Option[Seq[ExecutedWriteSummary] => Unit] = None): BaseRelation = { if (data.schema.map(_.dataType).exists(_.isInstanceOf[CalendarIntervalType])) { throw new AnalysisException("Cannot save interval data type into external storage.") } @@ -446,7 +450,12 @@ case class DataSource( dataSource.createRelation( sparkSession.sqlContext, mode, caseInsensitiveOptions, Dataset.ofRows(sparkSession, data)) case format: FileFormat => - sparkSession.sessionState.executePlan(planForWritingFileFormat(format, mode, data)).toRdd + val qe = sparkSession.sessionState.executePlan(planForWritingFileFormat(format, mode, data)) + val insertCommand = qe.executedPlan.collect { + case w: WrittenFileCommandExec => w + }.head + insertCommand.cmd.run(sparkSession, insertCommand.children, + metricsCallback.getOrElse(_ => ())) // Replace the schema with that of the DataFrame we just wrote out to avoid re-inferring copy(userSpecifiedSchema = Some(data.schema.asNullable)).resolveRelation() case _ => @@ -590,6 +599,15 @@ object DataSource extends Logging { } } + /** Given a provider name, returns if the provider is a `FileFormat`-based data source. */ + def isFileFormat(provider: String): Boolean = { + if (classOf[FileFormat].isAssignableFrom(lookupDataSource(provider))) { + true + } else { + false + } + } + /** * When creating a data source table, the `path` option has a special meaning: the table location. * This method extracts the `path` option and treat it as table location to build a diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index fdd6fe6a7c403..40b5d9c83c776 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -292,16 +292,15 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { import SQLMetricsSuite._ withTable("writeToTable") { - // Create the table. - Seq.empty[(Int, Int)].toDF("i", "j").write.mode("overwrite").saveAsTable("writeToTable") - + // Verifies the metrics of CreateDataSourceTableAsSelectCommand val executionId1 = getLatestExecutionId(spark) { () => - Seq((1, 2)).toDF("i", "j").write.insertInto("writeToTable") + Seq((1, 2)).toDF("i", "j").write.mode("overwrite").saveAsTable("writeToTable") } // written 1 file, 1 row, 0 dynamic partition. val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 1, _ > 0) verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) + // Verifies the metrics of InsertIntoHadoopFsRelationCommand val executionId2 = getLatestExecutionId(spark) { () => Seq((3, 4), (5, 6), (7, 8)).toDF("i", "j").repartition(1) .write.insertInto("writeToTable") @@ -326,6 +325,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { withTempDir { f => val df = spark.range(start = 0, end = 4, step = 1, numPartitions = 1).selectExpr("id", "id id1") + // Verifies the metrics of InsertIntoHadoopFsRelationCommand val executionId1 = getLatestExecutionId(spark) { () => df .write diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala new file mode 100644 index 0000000000000..dedf2ac104442 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.util.Utils + +class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + import spark.implicits._ + import org.apache.spark.sql.execution.metric.SQLMetricsSuite._ + + test("writing data out metrics") { + withTable("writeToHiveTable") { + // Verifies the metrics of CreateHiveTableAsSelectCommand + val executionId1 = getLatestExecutionId(spark) { () => + Seq((1, 2)).toDF("i", "j").write.format("hive").saveAsTable("writeToHiveTable") + } + // written 1 file, 1 row, 0 dynamic partition. + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 1, _ > 0) + verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) + + // Verifies the metrics of InsertIntoHiveTable + val executionId2 = getLatestExecutionId(spark) { () => + Seq((3, 4), (5, 6), (7, 8)).toDF("i", "j").repartition(1) + .write.format("hive").insertInto("writeToHiveTable") + } + // written 1 file, 3 rows, 0 dynamic partition. + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 3, _ > 0) + verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) + + val executionId3 = getLatestExecutionId(spark) { () => + Seq((9, 10), (11, 12)).toDF("i", "j").repartition(2) + .write.format("hive").insertInto("writeToHiveTable") + } + // written 2 files, 2 rows, 0 dynamic partition. + val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ > 0, _ == 2, _ > 0) + verifyWriteDataMetrics(spark, executionId3, verifyFuncs3) + } + } + + test("writing data out metrics: dynamic partition") { + withTempPath { dir => + spark.sql( + s""" + |CREATE TABLE t1(a int, b int) + |USING hive + |PARTITIONED BY(a) + |LOCATION '${dir.toURI}' + """.stripMargin) + + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + val df = spark.range(start = 0, end = 4, step = 1, numPartitions = 1) + .selectExpr("id a", "id b") + sql("SET hive.exec.dynamic.partition.mode=nonstrict") + + // Verifies the metrics of InsertIntoHiveTable + val executionId1 = getLatestExecutionId(spark) { () => + df + .write + .format("hive") + .option("fileFormat", "parquet") + .option("maxRecordsPerFile", 1) + .mode("overwrite") + .insertInto("t1") + } + assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 4) + // written 4 files, 4 rows, 4 dynamic partitions. + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 4, _ > 0) + verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) + + val executionId2 = getLatestExecutionId(spark) { () => + df.union(df).repartition(2, $"a") + .write + .format("hive") + .option("fileFormat", "parquet") + .option("maxRecordsPerFile", 2) + .mode("overwrite") + .insertInto("t1") + } + assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 4) + // written 4 files, 8 rows, 4 dynamic partitions. + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 8, _ > 0) + verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) + } + } +} From 43c17c08cf326fe59d583afcb070100771f1239d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 2 Jun 2017 07:32:03 +0000 Subject: [PATCH 04/22] Support detailed metrics per file/partition. --- .../sql/execution/command/commands.scala | 147 ++++++++++++----- .../datasources/FileFormatWriter.scala | 150 ++++++++++++++---- .../execution/metric/SQLMetricsSuite.scala | 29 +++- .../sql/hive/execution/SQLMetricsSuite.scala | 10 +- 4 files changed, 259 insertions(+), 77 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index b9da1e21063d3..3fc09f20330de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.command +import scala.collection.mutable + import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} @@ -42,12 +44,44 @@ trait WriteOutFileCommand extends logical.Command { * Those metrics will be updated once the command finishes writing data out. Those metrics will * be taken by `WrittenFileCommandExe` as its metrics when showing in UI. */ - def metrics(sqlContext: SQLContext): Map[String, SQLMetric] = Map( - "numOutputRows" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of output rows"), - "writingTime" -> SQLMetrics.createMetric(sqlContext.sparkContext, "writing data out time (ms)"), - "dynamicPartNum" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of dynamic part"), - "fileNum" -> SQLMetrics.createMetric(sqlContext.sparkContext, "number of written files"), - "fileBytes" -> SQLMetrics.createMetric(sqlContext.sparkContext, "bytes of written files")) + def metrics(sqlContext: SQLContext): Map[String, SQLMetric] = { + val sparkContext = sqlContext.sparkContext + + Map( + // General metrics. + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numDynamicParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part"), + "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), + "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), + "writingTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), + // Detailed metrics per file. + "minOutputRowsPerFile" -> + SQLMetrics.createMetric(sparkContext, "minimum number of output rows per file"), + "maxOutputRowsPerFile" -> + SQLMetrics.createMetric(sparkContext, "maximum number of output rows per file"), + "medOutputRowsPerFile" -> + SQLMetrics.createMetric(sparkContext, "median number of output rows per file"), + "minOutputBytesPerFile" -> + SQLMetrics.createMetric(sparkContext, "minimum written bytes per file"), + "maxOutputBytesPerFile" -> + SQLMetrics.createMetric(sparkContext, "maximum written bytes per file"), + "medOutputBytesPerFile" -> + SQLMetrics.createMetric(sparkContext, "median written bytes per file"), + // Detailed metrics per partition. + "minOutputRowsPerPart" -> + SQLMetrics.createMetric(sparkContext, "minimum number of output rows per partition"), + "maxOutputRowsPerPart" -> + SQLMetrics.createMetric(sparkContext, "maximum number of output rows per partition"), + "medOutputRowsPerPart" -> + SQLMetrics.createMetric(sparkContext, "median number of output rows per partition"), + "minOutputBytesPerPart" -> + SQLMetrics.createMetric(sparkContext, "minimum written bytes per partition"), + "maxOutputBytesPerPart" -> + SQLMetrics.createMetric(sparkContext, "maximum written bytes per partition"), + "medOutputBytesPerPart" -> + SQLMetrics.createMetric(sparkContext, "median written bytes per partition") + ) + } def run( sparkSession: SparkSession, @@ -112,45 +146,86 @@ case class WrittenFileCommandExec( override lazy val metrics = cmd.metrics(sqlContext) - // Callback used to update metrics returned from the operation of writing data out. - private def updateDriverMetrics(writeTaskSummary: Seq[ExecutedWriteSummary]): Unit = { - var partitionNum = 0 - var fileNum = 0 - var fileBytes: Long = 0L - var numOutput: Long = 0L - - writeTaskSummary.foreach { summary => - partitionNum += summary.updatedPartitions.size - fileNum += summary.writtenFileNum - fileBytes += summary.writtenBytes - numOutput += summary.numOutputRows + /** + * A helper function used to compute median/minimum/maximum values and update metrics based + * on the given metric name. + */ + private def setMetrics(values: Seq[Long], metricName: String): Seq[SQLMetric] = { + val sorted = values.sorted + val metricValues = Seq(sorted(0), sorted(values.length / 2), sorted(values.length - 1)) + Seq("min", "med", "max").zip(metricValues).map { case (prefix, metricValue) => + val metric = metrics(prefix + metricName) + metric.add(metricValue) + metric } + } + + /** + * The callback function used to update metrics returned from the operation of writing data out. + */ + private def updateDriverMetrics(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { + var numPartitions = 0 + var numFiles = 0 + + val (numBytesPerFilePart, numOutputRowsPerFilePart) = writeSummaries.map { summary => + numPartitions += summary.updatedPartitions.size + numFiles += summary.writtenFileNum + + (summary.writtenBytesPerPart.flatten, summary.numOutputRowsPerPart.flatten) + }.unzip - val partitionMetric = metrics("dynamicPartNum") - val fileNumMetric = metrics("fileNum") - val fileBytesMetric = metrics("fileBytes") - val numOutputRows = metrics("numOutputRows") - partitionMetric.add(partitionNum) - fileNumMetric.add(fileNum) - fileBytesMetric.add(fileBytes) - numOutputRows.add(numOutput) + val numBytesPerFile = numBytesPerFilePart.flatten + val numOutputRowsPerFile = numOutputRowsPerFilePart.flatten + + val totalNumBytes: Long = numBytesPerFile.sum + val totalNumOutput: Long = numOutputRowsPerFile.sum + + // Computes number of bytes/rows per file. + val numBytesPerFileMetrics = setMetrics(numBytesPerFile, "OutputBytesPerFile") + val numOutputsPerFileMetrics = setMetrics(numOutputRowsPerFile, "OutputRowsPerFile") + + // Computes number of bytes/rows per partition. + val numBytesPerPart = numBytesPerFilePart.map(_.sum) + val numOutputRowsPerPart = numOutputRowsPerFilePart.map(_.sum) + + val numBytesPerPartMetrics = setMetrics(numBytesPerPart, "OutputBytesPerPart") + val numOutputsPerPartMetrics = setMetrics(numOutputRowsPerPart, "OutputRowsPerPart") + + // Metrics of writing time in ms. + // We only count for the non-zero writing time when computing average. + val writingTimePerFile: Seq[Long] = writeSummaries.flatMap(_.writingTimePerPart.flatten) + val nonZeroCount = writingTimePerFile.filter(_ > 0).size + val avgWritingTime = if (nonZeroCount == 0) { + 0 + } else { + writingTimePerFile.sum / nonZeroCount + } + + // Updates metrics. + val numDynamicPartsMetric = metrics("numDynamicParts") + val fileNumMetric = metrics("numFiles") + val numBytesMetric = metrics("numOutputBytes") + val numOutputRowsMetric = metrics("numOutputRows") + val writingTimeMetric = metrics("writingTime") + + numDynamicPartsMetric.add(numPartitions) + fileNumMetric.add(numFiles) + numBytesMetric.add(totalNumBytes) + numOutputRowsMetric.add(totalNumOutput) + writingTimeMetric.add(avgWritingTime) + + val updatedMetrics = + Seq(numDynamicPartsMetric, fileNumMetric, numBytesMetric, numOutputRowsMetric, + writingTimeMetric) ++ numBytesPerFileMetrics ++ numOutputsPerFileMetrics ++ + numBytesPerPartMetrics ++ numOutputsPerPartMetrics val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, - partitionMetric :: fileNumMetric :: fileBytesMetric :: numOutputRows :: Nil) + SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, updatedMetrics) } protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { val converter = CatalystTypeConverters.createToCatalystConverter(schema) - val startTime = System.nanoTime() val rows = cmd.run(sqlContext.sparkSession, children, updateDriverMetrics) - val timeTakenMs = (System.nanoTime() - startTime) / 1000 / 1000 - metrics.get("writingTime").foreach { writingTime => - writingTime.add(timeTakenMs) - val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, - writingTime :: Nil) - } rows.map(converter(_).asInstanceOf[InternalRow]) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 31d60bd31a640..1e49238177622 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -268,6 +268,44 @@ object FileFormatWriter extends Logging { * automatically trigger task aborts. */ private trait ExecuteWriteTask { + /** + * The data structures used to measure metrics during writing. + */ + protected var fileSizesPerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty + protected var numOutputRowsPerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty + protected var writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty + + protected val fileSizesPerPart: mutable.ArrayBuffer[Seq[Long]] = mutable.ArrayBuffer.empty + protected val numOutputRowsPerPart: mutable.ArrayBuffer[Seq[Long]] = mutable.ArrayBuffer.empty + protected val writingTimePerPart: mutable.ArrayBuffer[Seq[Long]] = mutable.ArrayBuffer.empty + + /** + * Commits metrics for current file. + */ + protected def commitMetricsPerFile(): Unit = { + fileSizesPerPart += fileSizesPerFile.toSeq + numOutputRowsPerPart += numOutputRowsPerFile.toSeq + writingTimePerPart += writingTimePerFile.toSeq + + fileSizesPerFile = mutable.ArrayBuffer.empty + numOutputRowsPerFile = mutable.ArrayBuffer.empty + writingTimePerFile = mutable.ArrayBuffer.empty + } + + /** + * Constructs the `ExecutedWriteSummary` for returning the metrics of writing. + */ + protected def prepareSummary( + updatedPartitions: Set[String], + totalNumFiles: Int): ExecutedWriteSummary = { + ExecutedWriteSummary( + updatedPartitions = updatedPartitions, + writtenFileNum = totalNumFiles, + writtenBytesPerPart = fileSizesPerPart.toSeq, + numOutputRowsPerPart = numOutputRowsPerPart.toSeq, + writingTimePerPart = writingTimePerPart.toSeq) + } + /** * Writes data out to files, and then returns the summary of relative information which * includes the list of partition strings written out. The list of partitions is sent back @@ -289,6 +327,15 @@ object FileFormatWriter extends Logging { 0L } } + + /** + * Runs the given function and measures the time spent in ms. + */ + protected def measureTimeInMs(f: => Unit): Long = { + val startTime = System.nanoTime() + f + (System.nanoTime() - startTime) / 1000 / 1000 + } } /** Writes data to a single directory (used for non-dynamic-partition writes). */ @@ -316,30 +363,44 @@ object FileFormatWriter extends Logging { override def execute(iter: Iterator[InternalRow]): ExecutedWriteSummary = { var fileCounter = 0 var recordsInFile: Long = 0L - var totalFileSize: Long = 0L - var totalOutputRows: Long = 0L + var writingTime: Long = 0L newOutputWriter(fileCounter) + while (iter.hasNext) { if (description.maxRecordsPerFile > 0 && recordsInFile >= description.maxRecordsPerFile) { fileCounter += 1 assert(fileCounter < MAX_FILE_COUNTER, s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") + numOutputRowsPerFile += recordsInFile recordsInFile = 0 - releaseResources() - totalFileSize += getFileSize(taskAttemptContext.getConfiguration, currentPath) + + writingTime += measureTimeInMs { + releaseResources() + } + writingTimePerFile += writingTime + writingTime = 0 + + fileSizesPerFile += getFileSize(taskAttemptContext.getConfiguration, currentPath) newOutputWriter(fileCounter) } val internalRow = iter.next() - currentWriter.write(internalRow) - totalOutputRows += 1 + writingTime += measureTimeInMs { + currentWriter.write(internalRow) + } recordsInFile += 1 } - releaseResources() - totalFileSize += getFileSize(taskAttemptContext.getConfiguration, currentPath) - ExecutedWriteSummary(updatedPartitions = Set.empty, writtenFileNum = fileCounter + 1, - writtenBytes = totalFileSize, numOutputRows = totalOutputRows) + writingTime += measureTimeInMs { + releaseResources() + } + // Record the metrics for last file. + writingTimePerFile += writingTime + numOutputRowsPerFile += recordsInFile + fileSizesPerFile += getFileSize(taskAttemptContext.getConfiguration, currentPath) + + commitMetricsPerFile() + prepareSummary(Set.empty, fileCounter + 1) } override def releaseResources(): Unit = { @@ -449,53 +510,79 @@ object FileFormatWriter extends Logging { var recordsInFile: Long = 0L var fileCounter = 0 var totalFileCounter = 0 - var totalFileSize: Long = 0L - var totalOutputRows: Long = 0L + var writingTime: Long = 0L var currentPartColsAndBucketId: UnsafeRow = null val updatedPartitions = mutable.Set[String]() + for (row <- iter) { val nextPartColsAndBucketId = getPartitionColsAndBucketId(row) if (currentPartColsAndBucketId != nextPartColsAndBucketId) { + writingTime += measureTimeInMs { + releaseResources() + } + val lastFileSize = getFileSize(taskAttemptContext.getConfiguration, currentPath) + + // Only update metrics when this is not the first partition. Otherwise, it generates + // a metrics record with zero values for a non-existing file. if (currentPartColsAndBucketId != null) { totalFileCounter += (fileCounter + 1) + numOutputRowsPerFile += recordsInFile + writingTimePerFile += writingTime + fileSizesPerFile += lastFileSize } + // See a new partition or bucket - write to a new partition dir (or a new bucket file). currentPartColsAndBucketId = nextPartColsAndBucketId.copy() logDebug(s"Writing partition: $currentPartColsAndBucketId") + writingTime = 0 recordsInFile = 0 fileCounter = 0 - releaseResources() - totalFileSize += getFileSize(taskAttemptContext.getConfiguration, currentPath) + val prevUpdatePartNum = updatedPartitions.size newOutputWriter(currentPartColsAndBucketId, getPartPath, fileCounter, updatedPartitions) + + // If this is a new partition, committing the metrics for previous partition. + if (updatedPartitions.size > prevUpdatePartNum) { + commitMetricsPerFile() + } } else if (desc.maxRecordsPerFile > 0 && recordsInFile >= desc.maxRecordsPerFile) { // Exceeded the threshold in terms of the number of records per file. // Create a new file by increasing the file counter. + + writingTime += measureTimeInMs { + releaseResources() + } + writingTimePerFile += writingTime + numOutputRowsPerFile += recordsInFile + recordsInFile = 0 fileCounter += 1 assert(fileCounter < MAX_FILE_COUNTER, s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") - releaseResources() - totalFileSize += getFileSize(taskAttemptContext.getConfiguration, currentPath) + fileSizesPerFile += getFileSize(taskAttemptContext.getConfiguration, currentPath) newOutputWriter(currentPartColsAndBucketId, getPartPath, fileCounter, updatedPartitions) } - - currentWriter.write(getOutputRow(row)) - totalOutputRows += 1 + writingTime += measureTimeInMs { + currentWriter.write(getOutputRow(row)) + } recordsInFile += 1 } + + writingTime += measureTimeInMs { + releaseResources() + } + // Record the metrics of last file if any. if (currentPartColsAndBucketId != null) { totalFileCounter += (fileCounter + 1) + writingTimePerFile += writingTime + numOutputRowsPerFile += recordsInFile + fileSizesPerFile += getFileSize(taskAttemptContext.getConfiguration, currentPath) + commitMetricsPerFile() } - releaseResources() - updatedPartitions.toSet - totalFileSize += getFileSize(taskAttemptContext.getConfiguration, currentPath) - ExecutedWriteSummary(updatedPartitions = updatedPartitions.toSet, - writtenFileNum = totalFileCounter, writtenBytes = totalFileSize, - numOutputRows = totalOutputRows) + prepareSummary(updatedPartitions.toSet, totalFileCounter) } override def releaseResources(): Unit = { @@ -515,12 +602,17 @@ object FileFormatWriter extends Logging { * * @param updatedPartitions the partitions updated during writing data out. Only valid * for dynamic partition. - * @param numOutputRows the number of output rows. * @param writtenFileNum the total number of files written out. - * @param writtenBytes the total bytes written out. + * @param numOutputRowsPerPart the number of output rows per file for partitions. For + * non-dynamic-partition, keeping the number of output rows per file. + * @param writtenBytesPerPart the bytes written out per file for partitions. For + * non-dynamic-partition, keeping the bytes written out per file. + * @param writingTimePerPart the writing time in ms per file for partitions. For + * non-dynamic-partition, keeping the writing time in ms per file. */ case class ExecutedWriteSummary( updatedPartitions: Set[String], - numOutputRows: Long, writtenFileNum: Int, - writtenBytes: Long) + numOutputRowsPerPart: Seq[Seq[Long]], + writtenBytesPerPart: Seq[Seq[Long]], + writingTimePerPart: Seq[Seq[Long]]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 40b5d9c83c776..3655a6de20a5c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -288,6 +288,21 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } } + test("writing data out metrics: empty dataset") { + import SQLMetricsSuite._ + + withTable("writeToTable") { + // Verifies the metrics of CreateDataSourceTableAsSelectCommand + val executionId1 = getLatestExecutionId(spark) { () => + Seq.empty[(Int, Int)].toDF("i", "j").write.mode("overwrite").saveAsTable("writeToTable") + } + // For non-dynamic-partition, an empty dataset will still generate file. The size of this file + // isn't zero because there's cost of storage format. + val verifyFuncs: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 0, _ >= 0) + verifyWriteDataMetrics(spark, executionId1, verifyFuncs) + } + } + test("writing data out metrics") { import SQLMetricsSuite._ @@ -297,7 +312,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { Seq((1, 2)).toDF("i", "j").write.mode("overwrite").saveAsTable("writeToTable") } // written 1 file, 1 row, 0 dynamic partition. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 1, _ > 0) + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 1, _ >= 0) verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) // Verifies the metrics of InsertIntoHadoopFsRelationCommand @@ -306,7 +321,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { .write.insertInto("writeToTable") } // written 1 file, 3 rows, 0 dynamic partition. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 3, _ > 0) + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 3, _ >= 0) verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) val executionId3 = getLatestExecutionId(spark) { () => @@ -314,7 +329,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { .write.insertInto("writeToTable") } // written 2 files, 2 rows, 0 dynamic partition. - val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ > 0, _ == 2, _ > 0) + val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ > 0, _ == 2, _ >= 0) verifyWriteDataMetrics(spark, executionId3, verifyFuncs3) } } @@ -336,7 +351,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) // written 4 files, 4 rows, 4 dynamic partitions. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 4, _ > 0) + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 4, _ >= 0) verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) val executionId2 = getLatestExecutionId(spark) { () => @@ -349,7 +364,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) // written 4 files, 8 rows, 4 dynamic partitions. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 8, _ > 0) + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 8, _ >= 0) verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) } } @@ -539,9 +554,9 @@ object SQLMetricsSuite { val metricsNames = Seq( "number of written files", "number of dynamic part", - "bytes of written files", + "bytes of written output", "number of output rows", - "writing data out time (ms)") + "average writing time (ms)") val metrics = spark.sharedState.listener.getExecutionMetrics(executionId) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala index dedf2ac104442..f8b6bdc9e1b2b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -34,7 +34,7 @@ class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton Seq((1, 2)).toDF("i", "j").write.format("hive").saveAsTable("writeToHiveTable") } // written 1 file, 1 row, 0 dynamic partition. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 1, _ > 0) + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 1, _ >= 0) verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) // Verifies the metrics of InsertIntoHiveTable @@ -43,7 +43,7 @@ class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton .write.format("hive").insertInto("writeToHiveTable") } // written 1 file, 3 rows, 0 dynamic partition. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 3, _ > 0) + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 3, _ >= 0) verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) val executionId3 = getLatestExecutionId(spark) { () => @@ -51,7 +51,7 @@ class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton .write.format("hive").insertInto("writeToHiveTable") } // written 2 files, 2 rows, 0 dynamic partition. - val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ > 0, _ == 2, _ > 0) + val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ > 0, _ == 2, _ >= 0) verifyWriteDataMetrics(spark, executionId3, verifyFuncs3) } } @@ -85,7 +85,7 @@ class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton } assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 4) // written 4 files, 4 rows, 4 dynamic partitions. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 4, _ > 0) + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 4, _ >= 0) verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) val executionId2 = getLatestExecutionId(spark) { () => @@ -99,7 +99,7 @@ class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton } assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 4) // written 4 files, 8 rows, 4 dynamic partitions. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 8, _ > 0) + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 8, _ >= 0) verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) } } From 8bb02d972c0f5606ef9870ec4d515b721275f653 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 2 Jun 2017 15:44:38 +0000 Subject: [PATCH 05/22] Improve test cases. --- .../sql/execution/command/commands.scala | 33 ++++++++----- .../execution/metric/SQLMetricsSuite.scala | 46 ++++++++++++++----- .../sql/hive/execution/SQLMetricsSuite.scala | 10 ++-- 3 files changed, 62 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 3fc09f20330de..33036e4dcfcc2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -151,12 +151,16 @@ case class WrittenFileCommandExec( * on the given metric name. */ private def setMetrics(values: Seq[Long], metricName: String): Seq[SQLMetric] = { - val sorted = values.sorted - val metricValues = Seq(sorted(0), sorted(values.length / 2), sorted(values.length - 1)) - Seq("min", "med", "max").zip(metricValues).map { case (prefix, metricValue) => - val metric = metrics(prefix + metricName) - metric.add(metricValue) - metric + if (values.nonEmpty) { + val sorted = values.sorted + val metricValues = Seq(sorted(0), sorted(values.length / 2), sorted(values.length - 1)) + Seq("min", "med", "max").zip(metricValues).map { case (prefix, metricValue) => + val metric = metrics(prefix + metricName) + metric.add(metricValue) + metric + } + } else { + Seq.empty } } @@ -214,13 +218,20 @@ case class WrittenFileCommandExec( numOutputRowsMetric.add(totalNumOutput) writingTimeMetric.add(avgWritingTime) - val updatedMetrics = - Seq(numDynamicPartsMetric, fileNumMetric, numBytesMetric, numOutputRowsMetric, - writingTimeMetric) ++ numBytesPerFileMetrics ++ numOutputsPerFileMetrics ++ - numBytesPerPartMetrics ++ numOutputsPerPartMetrics + val generalMetrics = Seq(numDynamicPartsMetric, fileNumMetric, numBytesMetric, + numOutputRowsMetric, writingTimeMetric) + val metricsPerFile = numBytesPerFileMetrics ++ numOutputsPerFileMetrics + val metricsPerPart = numBytesPerPartMetrics ++ numOutputsPerPartMetrics + + val finalMetrics = if (numPartitions == 0) { + // For non-dynamic partition, we don't need to update the metrics per partition. + generalMetrics ++ metricsPerFile + } else { + generalMetrics ++ metricsPerFile ++ metricsPerPart + } val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, updatedMetrics) + SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, finalMetrics) } protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 3655a6de20a5c..99af2affe65ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -298,7 +298,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } // For non-dynamic-partition, an empty dataset will still generate file. The size of this file // isn't zero because there's cost of storage format. - val verifyFuncs: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 0, _ >= 0) + val verifyFuncs: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 0) verifyWriteDataMetrics(spark, executionId1, verifyFuncs) } } @@ -312,7 +312,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { Seq((1, 2)).toDF("i", "j").write.mode("overwrite").saveAsTable("writeToTable") } // written 1 file, 1 row, 0 dynamic partition. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 1, _ >= 0) + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 1) verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) // Verifies the metrics of InsertIntoHadoopFsRelationCommand @@ -321,7 +321,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { .write.insertInto("writeToTable") } // written 1 file, 3 rows, 0 dynamic partition. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 3, _ >= 0) + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 3) verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) val executionId3 = getLatestExecutionId(spark) { () => @@ -329,7 +329,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { .write.insertInto("writeToTable") } // written 2 files, 2 rows, 0 dynamic partition. - val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ > 0, _ == 2, _ >= 0) + val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ == 2) verifyWriteDataMetrics(spark, executionId3, verifyFuncs3) } } @@ -351,7 +351,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) // written 4 files, 4 rows, 4 dynamic partitions. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 4, _ >= 0) + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ == 4) verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) val executionId2 = getLatestExecutionId(spark) { () => @@ -364,7 +364,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) // written 4 files, 8 rows, 4 dynamic partitions. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 8, _ >= 0) + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ == 8) verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) } } @@ -522,9 +522,11 @@ object InputOutputMetricsHelper { * also used in the tests in Hive. */ object SQLMetricsSuite { + /** * Run the given function and return latest execution id. * + * @param spark the given `SparkSession` used to get execution data. * @param func the given function to run. */ def getLatestExecutionId(spark: SparkSession)(func: () => Unit): Long = { @@ -541,6 +543,7 @@ object SQLMetricsSuite { /** * Get execution metrics for the given execution id and verify metrics values. * + * @param spark the given `SparkSession` used to get execution data and metrics. * @param executionId the given execution id. * @param verifyFuncs functions used to verify the values of metrics. */ @@ -554,12 +557,9 @@ object SQLMetricsSuite { val metricsNames = Seq( "number of written files", "number of dynamic part", - "bytes of written output", - "number of output rows", - "average writing time (ms)") + "number of output rows") - val metrics = - spark.sharedState.listener.getExecutionMetrics(executionId) + val metrics = spark.sharedState.listener.getExecutionMetrics(executionId) metricsNames.zip(verifyFuncs).foreach { case (metricsName, verifyFunc) => val sqlMetric = executedNode.metrics.find(_.name == metricsName) @@ -568,5 +568,29 @@ object SQLMetricsSuite { val metricValue = metrics(accumulatorId).replaceAll(",", "").toInt assert(verifyFunc(metricValue)) } + + // Sanity check. + val numDynamicPartMetric = executedNode.metrics.find(_.name == "number of dynamic part").get + val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get + val numDynamicPart = metrics(numDynamicPartMetric.accumulatorId).toInt + val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt + val maxOutputBytesPerPartMetric = + executedNode.metrics.find(_.name == "maximum written bytes per partition").get + if (numDynamicPart == 0) { + // For non-dynamic-partition, we won't update the metrics per partition. + val maxOutputBytesPerPart = metrics.get(maxOutputBytesPerPartMetric.accumulatorId) + assert(maxOutputBytesPerPart.isEmpty && totalNumBytes > 0) + } else { + val maxOutputBytesPerPart = metrics(maxOutputBytesPerPartMetric.accumulatorId) + .replaceAll(",", "").toInt + + // Even number of output rows is zero, the output bytes still can't be zero. + // So if there's dynamic partitions, maximum output bytes per partition should be more than 0. + assert(maxOutputBytesPerPart > 0 && maxOutputBytesPerPart <= totalNumBytes) + } + // Check if the metric of writing time is updated. + val writingTimeMetric = executedNode.metrics.find(_.name == "average writing time (ms)").get + val writingTime = metrics(writingTimeMetric.accumulatorId).replaceAll(",", "").toInt + assert(writingTime >= 0) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala index f8b6bdc9e1b2b..1b7beb30641e6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -34,7 +34,7 @@ class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton Seq((1, 2)).toDF("i", "j").write.format("hive").saveAsTable("writeToHiveTable") } // written 1 file, 1 row, 0 dynamic partition. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 1, _ >= 0) + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 1) verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) // Verifies the metrics of InsertIntoHiveTable @@ -43,7 +43,7 @@ class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton .write.format("hive").insertInto("writeToHiveTable") } // written 1 file, 3 rows, 0 dynamic partition. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ > 0, _ == 3, _ >= 0) + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 3) verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) val executionId3 = getLatestExecutionId(spark) { () => @@ -51,7 +51,7 @@ class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton .write.format("hive").insertInto("writeToHiveTable") } // written 2 files, 2 rows, 0 dynamic partition. - val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ > 0, _ == 2, _ >= 0) + val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ == 2) verifyWriteDataMetrics(spark, executionId3, verifyFuncs3) } } @@ -85,7 +85,7 @@ class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton } assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 4) // written 4 files, 4 rows, 4 dynamic partitions. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 4, _ >= 0) + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ == 4) verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) val executionId2 = getLatestExecutionId(spark) { () => @@ -99,7 +99,7 @@ class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton } assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 4) // written 4 files, 8 rows, 4 dynamic partitions. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ > 0, _ == 8, _ >= 0) + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ == 8) verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) } } From 778007d354030648e379040d7bb647553422a031 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 4 Jun 2017 04:11:09 +0000 Subject: [PATCH 06/22] Simplify test cases. --- .../execution/metric/SQLMetricsSuite.scala | 161 +---------------- .../metric/WritingDataMetricsSuite.scala | 167 ++++++++++++++++++ .../sql/hive/execution/SQLMetricsSuite.scala | 83 +-------- 3 files changed, 172 insertions(+), 239 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 99af2affe65ed..a4e62f1d16792 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.ui.SparkPlanGraph import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.util.{AccumulatorContext, JsonProtocol, Utils} +import org.apache.spark.util.{AccumulatorContext, JsonProtocol} class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ @@ -288,87 +288,6 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } } - test("writing data out metrics: empty dataset") { - import SQLMetricsSuite._ - - withTable("writeToTable") { - // Verifies the metrics of CreateDataSourceTableAsSelectCommand - val executionId1 = getLatestExecutionId(spark) { () => - Seq.empty[(Int, Int)].toDF("i", "j").write.mode("overwrite").saveAsTable("writeToTable") - } - // For non-dynamic-partition, an empty dataset will still generate file. The size of this file - // isn't zero because there's cost of storage format. - val verifyFuncs: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 0) - verifyWriteDataMetrics(spark, executionId1, verifyFuncs) - } - } - - test("writing data out metrics") { - import SQLMetricsSuite._ - - withTable("writeToTable") { - // Verifies the metrics of CreateDataSourceTableAsSelectCommand - val executionId1 = getLatestExecutionId(spark) { () => - Seq((1, 2)).toDF("i", "j").write.mode("overwrite").saveAsTable("writeToTable") - } - // written 1 file, 1 row, 0 dynamic partition. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 1) - verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) - - // Verifies the metrics of InsertIntoHadoopFsRelationCommand - val executionId2 = getLatestExecutionId(spark) { () => - Seq((3, 4), (5, 6), (7, 8)).toDF("i", "j").repartition(1) - .write.insertInto("writeToTable") - } - // written 1 file, 3 rows, 0 dynamic partition. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 3) - verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) - - val executionId3 = getLatestExecutionId(spark) { () => - Seq((9, 10), (11, 12)).toDF("i", "j").repartition(2) - .write.insertInto("writeToTable") - } - // written 2 files, 2 rows, 0 dynamic partition. - val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ == 2) - verifyWriteDataMetrics(spark, executionId3, verifyFuncs3) - } - } - - test("writing data out metrics: dynamic partition") { - import SQLMetricsSuite._ - - withTempDir { f => - val df = - spark.range(start = 0, end = 4, step = 1, numPartitions = 1).selectExpr("id", "id id1") - // Verifies the metrics of InsertIntoHadoopFsRelationCommand - val executionId1 = getLatestExecutionId(spark) { () => - df - .write - .partitionBy("id") - .option("maxRecordsPerFile", 1) - .mode("overwrite") - .parquet(f.getAbsolutePath) - } - assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) - // written 4 files, 4 rows, 4 dynamic partitions. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ == 4) - verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) - - val executionId2 = getLatestExecutionId(spark) { () => - df.union(df).repartition(2, $"id") - .write - .partitionBy("id") - .option("maxRecordsPerFile", 2) - .mode("overwrite") - .parquet(f.getAbsolutePath) - } - assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) - // written 4 files, 8 rows, 4 dynamic partitions. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ == 8) - verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) - } - } - test("save metrics") { withTempPath { file => // person creates a temporary view. get the DF before listing previous execution IDs @@ -516,81 +435,3 @@ object InputOutputMetricsHelper { listener.getResults() } } - -/** - * This object wraps few methods used to verify SQL metrics when writing data out. The methods are - * also used in the tests in Hive. - */ -object SQLMetricsSuite { - - /** - * Run the given function and return latest execution id. - * - * @param spark the given `SparkSession` used to get execution data. - * @param func the given function to run. - */ - def getLatestExecutionId(spark: SparkSession)(func: () => Unit): Long = { - val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet - // Run the given function to trigger query execution. - func() - spark.sparkContext.listenerBus.waitUntilEmpty(10000) - val executionIds = - spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) - assert(executionIds.size == 1) - executionIds.head - } - - /** - * Get execution metrics for the given execution id and verify metrics values. - * - * @param spark the given `SparkSession` used to get execution data and metrics. - * @param executionId the given execution id. - * @param verifyFuncs functions used to verify the values of metrics. - */ - def verifyWriteDataMetrics( - spark: SparkSession, - executionId: Long, - verifyFuncs: Seq[Int => Boolean]): Unit = { - val executionData = spark.sharedState.listener.getExecution(executionId).get - val executedNode = executionData.physicalPlanGraph.nodes.head - - val metricsNames = Seq( - "number of written files", - "number of dynamic part", - "number of output rows") - - val metrics = spark.sharedState.listener.getExecutionMetrics(executionId) - - metricsNames.zip(verifyFuncs).foreach { case (metricsName, verifyFunc) => - val sqlMetric = executedNode.metrics.find(_.name == metricsName) - assert(sqlMetric.isDefined) - val accumulatorId = sqlMetric.get.accumulatorId - val metricValue = metrics(accumulatorId).replaceAll(",", "").toInt - assert(verifyFunc(metricValue)) - } - - // Sanity check. - val numDynamicPartMetric = executedNode.metrics.find(_.name == "number of dynamic part").get - val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get - val numDynamicPart = metrics(numDynamicPartMetric.accumulatorId).toInt - val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt - val maxOutputBytesPerPartMetric = - executedNode.metrics.find(_.name == "maximum written bytes per partition").get - if (numDynamicPart == 0) { - // For non-dynamic-partition, we won't update the metrics per partition. - val maxOutputBytesPerPart = metrics.get(maxOutputBytesPerPartMetric.accumulatorId) - assert(maxOutputBytesPerPart.isEmpty && totalNumBytes > 0) - } else { - val maxOutputBytesPerPart = metrics(maxOutputBytesPerPartMetric.accumulatorId) - .replaceAll(",", "").toInt - - // Even number of output rows is zero, the output bytes still can't be zero. - // So if there's dynamic partitions, maximum output bytes per partition should be more than 0. - assert(maxOutputBytesPerPart > 0 && maxOutputBytesPerPart <= totalNumBytes) - } - // Check if the metric of writing time is updated. - val writingTimeMetric = executedNode.metrics.find(_.name == "average writing time (ms)").get - val writingTime = metrics(writingTimeMetric.accumulatorId).replaceAll(",", "").toInt - assert(writingTime >= 0) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala new file mode 100644 index 0000000000000..3635a3043f6f4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala @@ -0,0 +1,167 @@ +/* + * 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.metric + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.util.Utils + +trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { + import testImplicits._ + + /** + * Run the given function and return latest execution id. + * + * @param func the given function to run. + */ + def getLatestExecutionId(spark: SparkSession)(func: () => Unit): Long = { + val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet + // Run the given function to trigger query execution. + func() + spark.sparkContext.listenerBus.waitUntilEmpty(10000) + val executionIds = + spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) + assert(executionIds.size == 1) + executionIds.head + } + + /** + * Get execution metrics for the given execution id and verify metrics values. + * + * @param executionId the given execution id. + * @param verifyFuncs functions used to verify the values of metrics. + */ + def verifyWriteDataMetrics( + spark: SparkSession, + executionId: Long, + verifyFuncs: Seq[Int => Boolean]): Unit = { + val executionData = spark.sharedState.listener.getExecution(executionId).get + val executedNode = executionData.physicalPlanGraph.nodes.head + + val metricsNames = Seq( + "number of written files", + "number of dynamic part", + "number of output rows") + + val metrics = spark.sharedState.listener.getExecutionMetrics(executionId) + + metricsNames.zip(verifyFuncs).foreach { case (metricsName, verifyFunc) => + val sqlMetric = executedNode.metrics.find(_.name == metricsName) + assert(sqlMetric.isDefined) + val accumulatorId = sqlMetric.get.accumulatorId + val metricValue = metrics(accumulatorId).replaceAll(",", "").toInt + assert(verifyFunc(metricValue)) + } + + // Sanity check. + val numDynamicPartMetric = executedNode.metrics.find(_.name == "number of dynamic part").get + val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get + val numDynamicPart = metrics(numDynamicPartMetric.accumulatorId).toInt + val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt + val maxOutputBytesPerPartMetric = + executedNode.metrics.find(_.name == "maximum written bytes per partition").get + if (numDynamicPart == 0) { + // For non-dynamic-partition, we won't update the metrics per partition. + val maxOutputBytesPerPart = metrics.get(maxOutputBytesPerPartMetric.accumulatorId) + assert(maxOutputBytesPerPart.isEmpty && totalNumBytes > 0) + } else { + val maxOutputBytesPerPart = metrics(maxOutputBytesPerPartMetric.accumulatorId) + .replaceAll(",", "").toInt + + // Even number of output rows is zero, the output bytes still can't be zero. + // So if there's dynamic partitions, maximum output bytes per partition should be more than 0. + assert(maxOutputBytesPerPart > 0 && maxOutputBytesPerPart <= totalNumBytes) + } + // Check if the metric of writing time is updated. + val writingTimeMetric = executedNode.metrics.find(_.name == "average writing time (ms)").get + val writingTime = metrics(writingTimeMetric.accumulatorId).replaceAll(",", "").toInt + assert(writingTime >= 0) + } + + protected def testMetricsNonDynamicPartition( + spark: SparkSession, + dataFormat: String, + tableName: String): Unit = { + withTable(tableName) { + val executionId1 = getLatestExecutionId(spark) { () => + Seq((1, 2)).toDF("i", "j") + .write.format(dataFormat).mode("overwrite").saveAsTable(tableName) + } + // written 1 file, 1 row, 0 dynamic partition. + val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 1) + verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) + + val executionId2 = getLatestExecutionId(spark) { () => + Seq((9, 10), (11, 12)).toDF("i", "j").repartition(2) + .write.format(dataFormat).insertInto(tableName) + } + // written 2 files, 2 rows, 0 dynamic partition. + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ == 2) + verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) + } + } + + protected def testMetricsDynamicPartition( + spark: SparkSession, + provider: String, + dataFormat: String, + tableName: String): Unit = { + withTempPath { dir => + spark.sql( + s""" + |CREATE TABLE t1(a int, b int) + |USING $provider + |PARTITIONED BY(a) + |LOCATION '${dir.toURI}' + """.stripMargin) + + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + val df = spark.range(start = 0, end = 4, step = 1, numPartitions = 1) + .selectExpr("id a", "id b") + sql("SET hive.exec.dynamic.partition.mode=nonstrict") + + val executionId = getLatestExecutionId(spark) { () => + df.union(df).repartition(2, $"a") + .write + .format(dataFormat) + .option("maxRecordsPerFile", 2) + .mode("overwrite") + .insertInto(tableName) + } + assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 4) + // written 4 files, 8 rows, 4 dynamic partitions. + val verifyFuncs: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ == 8) + verifyWriteDataMetrics(spark, executionId, verifyFuncs) + } + } +} + +class WritingDataMetricsSuite extends SharedSQLContext with BaseWritingDataMetricsSuite { + test("writing data out metrics") { + testMetricsNonDynamicPartition(spark, "parquet", "t1") + } + + test("writing data out metrics: dynamic partition") { + testMetricsDynamicPartition(spark, "parquet", "parquet", "t1") + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala index 1b7beb30641e6..d9b86833220dd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -17,90 +17,15 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.metric.BaseWritingDataMetricsSuite import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.util.Utils - -class SQLMetricsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { - import spark.implicits._ - import org.apache.spark.sql.execution.metric.SQLMetricsSuite._ +class SQLMetricsSuite extends BaseWritingDataMetricsSuite with TestHiveSingleton { test("writing data out metrics") { - withTable("writeToHiveTable") { - // Verifies the metrics of CreateHiveTableAsSelectCommand - val executionId1 = getLatestExecutionId(spark) { () => - Seq((1, 2)).toDF("i", "j").write.format("hive").saveAsTable("writeToHiveTable") - } - // written 1 file, 1 row, 0 dynamic partition. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 1) - verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) - - // Verifies the metrics of InsertIntoHiveTable - val executionId2 = getLatestExecutionId(spark) { () => - Seq((3, 4), (5, 6), (7, 8)).toDF("i", "j").repartition(1) - .write.format("hive").insertInto("writeToHiveTable") - } - // written 1 file, 3 rows, 0 dynamic partition. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 3) - verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) - - val executionId3 = getLatestExecutionId(spark) { () => - Seq((9, 10), (11, 12)).toDF("i", "j").repartition(2) - .write.format("hive").insertInto("writeToHiveTable") - } - // written 2 files, 2 rows, 0 dynamic partition. - val verifyFuncs3: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ == 2) - verifyWriteDataMetrics(spark, executionId3, verifyFuncs3) - } + testMetricsNonDynamicPartition(spark, "hive", "t1") } test("writing data out metrics: dynamic partition") { - withTempPath { dir => - spark.sql( - s""" - |CREATE TABLE t1(a int, b int) - |USING hive - |PARTITIONED BY(a) - |LOCATION '${dir.toURI}' - """.stripMargin) - - val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) - assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) - - val df = spark.range(start = 0, end = 4, step = 1, numPartitions = 1) - .selectExpr("id a", "id b") - sql("SET hive.exec.dynamic.partition.mode=nonstrict") - - // Verifies the metrics of InsertIntoHiveTable - val executionId1 = getLatestExecutionId(spark) { () => - df - .write - .format("hive") - .option("fileFormat", "parquet") - .option("maxRecordsPerFile", 1) - .mode("overwrite") - .insertInto("t1") - } - assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 4) - // written 4 files, 4 rows, 4 dynamic partitions. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ == 4) - verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) - - val executionId2 = getLatestExecutionId(spark) { () => - df.union(df).repartition(2, $"a") - .write - .format("hive") - .option("fileFormat", "parquet") - .option("maxRecordsPerFile", 2) - .mode("overwrite") - .insertInto("t1") - } - assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 4) - // written 4 files, 8 rows, 4 dynamic partitions. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ == 8) - verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) - } + testMetricsDynamicPartition(spark, "hive", "hive", "t1") } } From 9819f0103a15dd948c049eb7130f577f084b28e4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 4 Jun 2017 04:49:31 +0000 Subject: [PATCH 07/22] Revert the specified metrics per file/partition. --- .../command/WriteOutFileCommand.scala | 55 ++++++ .../command/WrittenFileCommandExec.scala | 86 +++++++++ .../sql/execution/command/commands.scala | 170 +----------------- .../command/createDataSourceTables.scala | 5 +- .../datasources/FileFormatWriter.scala | 105 ++++------- .../metric/WritingDataMetricsSuite.scala | 19 +- 6 files changed, 186 insertions(+), 254 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala new file mode 100644 index 0000000000000..8bb04ec071bf5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import scala.collection.mutable + +import org.apache.spark.SparkContext +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * A logical command specialized for writing data out. `WriteOutFileCommand`s are + * wrapped in `WrittenFileCommandExec` during execution. + */ +trait WriteOutFileCommand extends logical.Command { + + /** + * Those metrics will be updated once the command finishes writing data out. Those metrics will + * be taken by `WrittenFileCommandExe` as its metrics when showing in UI. + */ + def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = + Map( + // General metrics. + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numDynamicParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part"), + "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), + "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), + "writingTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)") + ) + + def run( + sparkSession: SparkSession, + children: Seq[SparkPlan], + metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { + throw new NotImplementedError + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala new file mode 100644 index 0000000000000..b349ed21b8937 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary +import org.apache.spark.sql.execution.metric.SQLMetrics + +/** + * A physical operator specialized to execute the run method of a `WriteOutFileCommand`, + * save the result to prevent multiple executions, and record necessary metrics for UI. + */ +case class WrittenFileCommandExec( + cmd: WriteOutFileCommand, + children: Seq[SparkPlan]) extends CommandExec { + + override lazy val metrics = cmd.metrics(sqlContext.sparkContext) + + /** + * The callback function used to update metrics returned from the operation of writing data out. + */ + private def updateDriverMetrics(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { + var numPartitions = 0 + var numFiles = 0 + var totalNumBytes: Long = 0L + var totalNumOutput: Long = 0L + + writeSummaries.foreach { summary => + numPartitions += summary.updatedPartitions.size + numFiles += summary.numOutputFile + totalNumBytes += summary.numOutputBytes + totalNumOutput += summary.numOutputRows + } + + // Metrics of writing time in ms. + // We only count for the non-zero writing time when computing average. + val writingTimePerFile: Seq[Long] = writeSummaries.flatMap(_.writingTimePerFile) + val nonZeroCount = writingTimePerFile.filter(_ > 0).size + val avgWritingTime = if (nonZeroCount == 0) { + 0 + } else { + writingTimePerFile.sum / nonZeroCount + } + + // Updates metrics. + val numDynamicPartsMetric = metrics("numDynamicParts") + val fileNumMetric = metrics("numFiles") + val numBytesMetric = metrics("numOutputBytes") + val numOutputRowsMetric = metrics("numOutputRows") + val writingTimeMetric = metrics("writingTime") + + numDynamicPartsMetric.add(numPartitions) + fileNumMetric.add(numFiles) + numBytesMetric.add(totalNumBytes) + numOutputRowsMetric.add(totalNumOutput) + writingTimeMetric.add(avgWritingTime) + + val generalMetrics = Seq(numDynamicPartsMetric, fileNumMetric, numBytesMetric, + numOutputRowsMetric, writingTimeMetric) + + val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, generalMetrics) + } + + protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { + val converter = CatalystTypeConverters.createToCatalystConverter(schema) + val rows = cmd.run(sqlContext.sparkSession, children, updateDriverMetrics) + rows.map(converter(_).asInstanceOf[InternalRow]) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 33036e4dcfcc2..be1113cb97eb4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -17,80 +17,19 @@ package org.apache.spark.sql.execution.command -import scala.collection.mutable - import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SparkSession, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} -import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.debug._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types._ -/** - * A logical command specialized for writing data out. `WriteOutFileCommand`s are - * wrapped in `WrittenFileCommandExec` during execution. - */ -trait WriteOutFileCommand extends logical.Command { - - /** - * Those metrics will be updated once the command finishes writing data out. Those metrics will - * be taken by `WrittenFileCommandExe` as its metrics when showing in UI. - */ - def metrics(sqlContext: SQLContext): Map[String, SQLMetric] = { - val sparkContext = sqlContext.sparkContext - - Map( - // General metrics. - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numDynamicParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part"), - "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), - "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), - "writingTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), - // Detailed metrics per file. - "minOutputRowsPerFile" -> - SQLMetrics.createMetric(sparkContext, "minimum number of output rows per file"), - "maxOutputRowsPerFile" -> - SQLMetrics.createMetric(sparkContext, "maximum number of output rows per file"), - "medOutputRowsPerFile" -> - SQLMetrics.createMetric(sparkContext, "median number of output rows per file"), - "minOutputBytesPerFile" -> - SQLMetrics.createMetric(sparkContext, "minimum written bytes per file"), - "maxOutputBytesPerFile" -> - SQLMetrics.createMetric(sparkContext, "maximum written bytes per file"), - "medOutputBytesPerFile" -> - SQLMetrics.createMetric(sparkContext, "median written bytes per file"), - // Detailed metrics per partition. - "minOutputRowsPerPart" -> - SQLMetrics.createMetric(sparkContext, "minimum number of output rows per partition"), - "maxOutputRowsPerPart" -> - SQLMetrics.createMetric(sparkContext, "maximum number of output rows per partition"), - "medOutputRowsPerPart" -> - SQLMetrics.createMetric(sparkContext, "median number of output rows per partition"), - "minOutputBytesPerPart" -> - SQLMetrics.createMetric(sparkContext, "minimum written bytes per partition"), - "maxOutputBytesPerPart" -> - SQLMetrics.createMetric(sparkContext, "maximum written bytes per partition"), - "medOutputBytesPerPart" -> - SQLMetrics.createMetric(sparkContext, "median written bytes per partition") - ) - } - - def run( - sparkSession: SparkSession, - children: Seq[SparkPlan], - metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { - throw new NotImplementedError - } -} - /** * A logical command that is executed for its side-effects. `RunnableCommand`s are * wrapped in `ExecutedCommand` during execution. @@ -136,111 +75,6 @@ trait CommandExec extends SparkPlan { } } -/** - * A physical operator specialized to execute the run method of a `WriteOutFileCommand`, - * save the result to prevent multiple executions, and record necessary metrics for UI. - */ -case class WrittenFileCommandExec( - cmd: WriteOutFileCommand, - children: Seq[SparkPlan]) extends CommandExec { - - override lazy val metrics = cmd.metrics(sqlContext) - - /** - * A helper function used to compute median/minimum/maximum values and update metrics based - * on the given metric name. - */ - private def setMetrics(values: Seq[Long], metricName: String): Seq[SQLMetric] = { - if (values.nonEmpty) { - val sorted = values.sorted - val metricValues = Seq(sorted(0), sorted(values.length / 2), sorted(values.length - 1)) - Seq("min", "med", "max").zip(metricValues).map { case (prefix, metricValue) => - val metric = metrics(prefix + metricName) - metric.add(metricValue) - metric - } - } else { - Seq.empty - } - } - - /** - * The callback function used to update metrics returned from the operation of writing data out. - */ - private def updateDriverMetrics(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { - var numPartitions = 0 - var numFiles = 0 - - val (numBytesPerFilePart, numOutputRowsPerFilePart) = writeSummaries.map { summary => - numPartitions += summary.updatedPartitions.size - numFiles += summary.writtenFileNum - - (summary.writtenBytesPerPart.flatten, summary.numOutputRowsPerPart.flatten) - }.unzip - - val numBytesPerFile = numBytesPerFilePart.flatten - val numOutputRowsPerFile = numOutputRowsPerFilePart.flatten - - val totalNumBytes: Long = numBytesPerFile.sum - val totalNumOutput: Long = numOutputRowsPerFile.sum - - // Computes number of bytes/rows per file. - val numBytesPerFileMetrics = setMetrics(numBytesPerFile, "OutputBytesPerFile") - val numOutputsPerFileMetrics = setMetrics(numOutputRowsPerFile, "OutputRowsPerFile") - - // Computes number of bytes/rows per partition. - val numBytesPerPart = numBytesPerFilePart.map(_.sum) - val numOutputRowsPerPart = numOutputRowsPerFilePart.map(_.sum) - - val numBytesPerPartMetrics = setMetrics(numBytesPerPart, "OutputBytesPerPart") - val numOutputsPerPartMetrics = setMetrics(numOutputRowsPerPart, "OutputRowsPerPart") - - // Metrics of writing time in ms. - // We only count for the non-zero writing time when computing average. - val writingTimePerFile: Seq[Long] = writeSummaries.flatMap(_.writingTimePerPart.flatten) - val nonZeroCount = writingTimePerFile.filter(_ > 0).size - val avgWritingTime = if (nonZeroCount == 0) { - 0 - } else { - writingTimePerFile.sum / nonZeroCount - } - - // Updates metrics. - val numDynamicPartsMetric = metrics("numDynamicParts") - val fileNumMetric = metrics("numFiles") - val numBytesMetric = metrics("numOutputBytes") - val numOutputRowsMetric = metrics("numOutputRows") - val writingTimeMetric = metrics("writingTime") - - numDynamicPartsMetric.add(numPartitions) - fileNumMetric.add(numFiles) - numBytesMetric.add(totalNumBytes) - numOutputRowsMetric.add(totalNumOutput) - writingTimeMetric.add(avgWritingTime) - - val generalMetrics = Seq(numDynamicPartsMetric, fileNumMetric, numBytesMetric, - numOutputRowsMetric, writingTimeMetric) - val metricsPerFile = numBytesPerFileMetrics ++ numOutputsPerFileMetrics - val metricsPerPart = numBytesPerPartMetrics ++ numOutputsPerPartMetrics - - val finalMetrics = if (numPartitions == 0) { - // For non-dynamic partition, we don't need to update the metrics per partition. - generalMetrics ++ metricsPerFile - } else { - generalMetrics ++ metricsPerFile ++ metricsPerPart - } - - val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, finalMetrics) - } - - protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { - val converter = CatalystTypeConverters.createToCatalystConverter(schema) - val rows = cmd.run(sqlContext.sparkSession, children, updateDriverMetrics) - rows.map(converter(_).asInstanceOf[InternalRow]) - } -} - /** * A physical operator specialized to execute the run method of a `RunnableCommand` and * save the result to prevent multiple executions. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 237a22fdd6bab..d6a01b65c2019 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -21,6 +21,7 @@ import java.net.URI import org.apache.hadoop.fs.Path +import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -133,9 +134,9 @@ case class CreateDataSourceTableAsSelectCommand( * For other data sources, `CreatableRelationProvider.createRelation` will be called. We can't * record metrics for that. So we will return empty metrics map. */ - override def metrics(sqlContext: SQLContext): Map[String, SQLMetric] = { + override def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = { if (DataSource.isFileFormat(table.provider.get)) { - super.metrics(sqlContext) + super.metrics(sparkContext) } else { Map.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 1e49238177622..4823ba3dda36a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -268,43 +268,11 @@ object FileFormatWriter extends Logging { * automatically trigger task aborts. */ private trait ExecuteWriteTask { - /** - * The data structures used to measure metrics during writing. - */ - protected var fileSizesPerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty - protected var numOutputRowsPerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty - protected var writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty - - protected val fileSizesPerPart: mutable.ArrayBuffer[Seq[Long]] = mutable.ArrayBuffer.empty - protected val numOutputRowsPerPart: mutable.ArrayBuffer[Seq[Long]] = mutable.ArrayBuffer.empty - protected val writingTimePerPart: mutable.ArrayBuffer[Seq[Long]] = mutable.ArrayBuffer.empty - - /** - * Commits metrics for current file. - */ - protected def commitMetricsPerFile(): Unit = { - fileSizesPerPart += fileSizesPerFile.toSeq - numOutputRowsPerPart += numOutputRowsPerFile.toSeq - writingTimePerPart += writingTimePerFile.toSeq - - fileSizesPerFile = mutable.ArrayBuffer.empty - numOutputRowsPerFile = mutable.ArrayBuffer.empty - writingTimePerFile = mutable.ArrayBuffer.empty - } /** - * Constructs the `ExecutedWriteSummary` for returning the metrics of writing. + * The data structures used to measure metrics during writing. */ - protected def prepareSummary( - updatedPartitions: Set[String], - totalNumFiles: Int): ExecutedWriteSummary = { - ExecutedWriteSummary( - updatedPartitions = updatedPartitions, - writtenFileNum = totalNumFiles, - writtenBytesPerPart = fileSizesPerPart.toSeq, - numOutputRowsPerPart = numOutputRowsPerPart.toSeq, - writingTimePerPart = writingTimePerPart.toSeq) - } + protected val writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty /** * Writes data out to files, and then returns the summary of relative information which @@ -364,6 +332,8 @@ object FileFormatWriter extends Logging { var fileCounter = 0 var recordsInFile: Long = 0L var writingTime: Long = 0L + var numOutputRows: Long = 0L + var numOutputBytes: Long = 0L newOutputWriter(fileCounter) while (iter.hasNext) { @@ -372,7 +342,7 @@ object FileFormatWriter extends Logging { assert(fileCounter < MAX_FILE_COUNTER, s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") - numOutputRowsPerFile += recordsInFile + numOutputRows += recordsInFile recordsInFile = 0 writingTime += measureTimeInMs { @@ -381,7 +351,7 @@ object FileFormatWriter extends Logging { writingTimePerFile += writingTime writingTime = 0 - fileSizesPerFile += getFileSize(taskAttemptContext.getConfiguration, currentPath) + numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) newOutputWriter(fileCounter) } @@ -396,11 +366,15 @@ object FileFormatWriter extends Logging { } // Record the metrics for last file. writingTimePerFile += writingTime - numOutputRowsPerFile += recordsInFile - fileSizesPerFile += getFileSize(taskAttemptContext.getConfiguration, currentPath) + numOutputRows += recordsInFile + numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) - commitMetricsPerFile() - prepareSummary(Set.empty, fileCounter + 1) + ExecutedWriteSummary( + updatedPartitions = Set.empty, + numOutputFile = fileCounter + 1, + numOutputBytes = numOutputBytes, + numOutputRows = numOutputRows, + writingTimePerFile = writingTimePerFile) } override def releaseResources(): Unit = { @@ -509,6 +483,8 @@ object FileFormatWriter extends Logging { // If anything below fails, we should abort the task. var recordsInFile: Long = 0L var fileCounter = 0 + var numOutputBytes: Long = 0L + var numOutputRows: Long = 0L var totalFileCounter = 0 var writingTime: Long = 0L var currentPartColsAndBucketId: UnsafeRow = null @@ -520,15 +496,14 @@ object FileFormatWriter extends Logging { writingTime += measureTimeInMs { releaseResources() } - val lastFileSize = getFileSize(taskAttemptContext.getConfiguration, currentPath) // Only update metrics when this is not the first partition. Otherwise, it generates // a metrics record with zero values for a non-existing file. if (currentPartColsAndBucketId != null) { totalFileCounter += (fileCounter + 1) - numOutputRowsPerFile += recordsInFile + numOutputRows += recordsInFile writingTimePerFile += writingTime - fileSizesPerFile += lastFileSize + numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } // See a new partition or bucket - write to a new partition dir (or a new bucket file). @@ -539,13 +514,7 @@ object FileFormatWriter extends Logging { recordsInFile = 0 fileCounter = 0 - val prevUpdatePartNum = updatedPartitions.size newOutputWriter(currentPartColsAndBucketId, getPartPath, fileCounter, updatedPartitions) - - // If this is a new partition, committing the metrics for previous partition. - if (updatedPartitions.size > prevUpdatePartNum) { - commitMetricsPerFile() - } } else if (desc.maxRecordsPerFile > 0 && recordsInFile >= desc.maxRecordsPerFile) { // Exceeded the threshold in terms of the number of records per file. @@ -555,14 +524,15 @@ object FileFormatWriter extends Logging { releaseResources() } writingTimePerFile += writingTime - numOutputRowsPerFile += recordsInFile + numOutputRows += recordsInFile + numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) recordsInFile = 0 + writingTime = 0 fileCounter += 1 assert(fileCounter < MAX_FILE_COUNTER, s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") - fileSizesPerFile += getFileSize(taskAttemptContext.getConfiguration, currentPath) newOutputWriter(currentPartColsAndBucketId, getPartPath, fileCounter, updatedPartitions) } writingTime += measureTimeInMs { @@ -574,15 +544,19 @@ object FileFormatWriter extends Logging { writingTime += measureTimeInMs { releaseResources() } - // Record the metrics of last file if any. + // Record the metrics of last file/partition if any. if (currentPartColsAndBucketId != null) { totalFileCounter += (fileCounter + 1) writingTimePerFile += writingTime - numOutputRowsPerFile += recordsInFile - fileSizesPerFile += getFileSize(taskAttemptContext.getConfiguration, currentPath) - commitMetricsPerFile() + numOutputRows += recordsInFile + numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } - prepareSummary(updatedPartitions.toSet, totalFileCounter) + ExecutedWriteSummary( + updatedPartitions = updatedPartitions.toSet, + numOutputFile = totalFileCounter, + numOutputBytes = numOutputBytes, + numOutputRows = numOutputRows, + writingTimePerFile = writingTimePerFile) } override def releaseResources(): Unit = { @@ -602,17 +576,14 @@ object FileFormatWriter extends Logging { * * @param updatedPartitions the partitions updated during writing data out. Only valid * for dynamic partition. - * @param writtenFileNum the total number of files written out. - * @param numOutputRowsPerPart the number of output rows per file for partitions. For - * non-dynamic-partition, keeping the number of output rows per file. - * @param writtenBytesPerPart the bytes written out per file for partitions. For - * non-dynamic-partition, keeping the bytes written out per file. - * @param writingTimePerPart the writing time in ms per file for partitions. For - * non-dynamic-partition, keeping the writing time in ms per file. + * @param numOutputFile the total number of files. + * @param numOutputRows the number of output rows. + * @param numOutputBytes the bytes of output data. + * @param writingTimePerFile the writing time in ms per file. */ case class ExecutedWriteSummary( updatedPartitions: Set[String], - writtenFileNum: Int, - numOutputRowsPerPart: Seq[Seq[Long]], - writtenBytesPerPart: Seq[Seq[Long]], - writingTimePerPart: Seq[Seq[Long]]) + numOutputFile: Int, + numOutputRows: Long, + numOutputBytes: Long, + writingTimePerFile: Seq[Long]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala index 3635a3043f6f4..d3d340d182a31 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala @@ -72,25 +72,10 @@ trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { } // Sanity check. - val numDynamicPartMetric = executedNode.metrics.find(_.name == "number of dynamic part").get val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get - val numDynamicPart = metrics(numDynamicPartMetric.accumulatorId).toInt val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt - val maxOutputBytesPerPartMetric = - executedNode.metrics.find(_.name == "maximum written bytes per partition").get - if (numDynamicPart == 0) { - // For non-dynamic-partition, we won't update the metrics per partition. - val maxOutputBytesPerPart = metrics.get(maxOutputBytesPerPartMetric.accumulatorId) - assert(maxOutputBytesPerPart.isEmpty && totalNumBytes > 0) - } else { - val maxOutputBytesPerPart = metrics(maxOutputBytesPerPartMetric.accumulatorId) - .replaceAll(",", "").toInt - - // Even number of output rows is zero, the output bytes still can't be zero. - // So if there's dynamic partitions, maximum output bytes per partition should be more than 0. - assert(maxOutputBytesPerPart > 0 && maxOutputBytesPerPart <= totalNumBytes) - } - // Check if the metric of writing time is updated. + assert(totalNumBytes > 0) + val writingTimeMetric = executedNode.metrics.find(_.name == "average writing time (ms)").get val writingTime = metrics(writingTimeMetric.accumulatorId).replaceAll(",", "").toInt assert(writingTime >= 0) From 6e61bd64f99d27b71c0c32b867dc02f66df2def7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 4 Jun 2017 08:55:16 +0000 Subject: [PATCH 08/22] Simplify code change. --- .../command/WriteOutFileCommand.scala | 4 +-- .../command/WrittenFileCommandExec.scala | 31 ++++--------------- 2 files changed, 8 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala index 8bb04ec071bf5..ac9b7a9f6432a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala @@ -40,10 +40,10 @@ trait WriteOutFileCommand extends logical.Command { Map( // General metrics. "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numDynamicParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part"), + "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part"), "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), - "writingTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)") + "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)") ) def run( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala index b349ed21b8937..9cad3ea525116 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala @@ -48,34 +48,15 @@ case class WrittenFileCommandExec( totalNumOutput += summary.numOutputRows } - // Metrics of writing time in ms. - // We only count for the non-zero writing time when computing average. - val writingTimePerFile: Seq[Long] = writeSummaries.flatMap(_.writingTimePerFile) - val nonZeroCount = writingTimePerFile.filter(_ > 0).size - val avgWritingTime = if (nonZeroCount == 0) { - 0 - } else { - writingTimePerFile.sum / nonZeroCount - } - - // Updates metrics. - val numDynamicPartsMetric = metrics("numDynamicParts") - val fileNumMetric = metrics("numFiles") - val numBytesMetric = metrics("numOutputBytes") - val numOutputRowsMetric = metrics("numOutputRows") - val writingTimeMetric = metrics("writingTime") - - numDynamicPartsMetric.add(numPartitions) - fileNumMetric.add(numFiles) - numBytesMetric.add(totalNumBytes) - numOutputRowsMetric.add(totalNumOutput) - writingTimeMetric.add(avgWritingTime) + val avgWritingTime = writeSummaries.flatMap(_.writingTimePerFile).sum / numFiles - val generalMetrics = Seq(numDynamicPartsMetric, fileNumMetric, numBytesMetric, - numOutputRowsMetric, writingTimeMetric) + val metricsNames = Seq("numParts", "numFiles", "numOutputBytes", "numOutputRows", "avgTime") + val metricsValues = Seq(numPartitions, numFiles, totalNumBytes, totalNumOutput, avgWritingTime) + metricsNames.zip(metricsValues).foreach(x => metrics(x._1).add(x._2)) val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, generalMetrics) + SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, + metricsNames.map(metrics(_))) } protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { From e8acb49300a2cd497e5eacd36945ddaf04ebdaa8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 4 Jun 2017 08:55:48 +0000 Subject: [PATCH 09/22] Simplify code change again. --- .../command/WrittenFileCommandExec.scala | 7 +- .../datasources/FileFormatWriter.scala | 89 ++++++------------- .../metric/WritingDataMetricsSuite.scala | 31 ++++--- 3 files changed, 51 insertions(+), 76 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala index 9cad3ea525116..0c70621ea9fe2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala @@ -48,7 +48,12 @@ case class WrittenFileCommandExec( totalNumOutput += summary.numOutputRows } - val avgWritingTime = writeSummaries.flatMap(_.writingTimePerFile).sum / numFiles + val times = writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0)) + val avgWritingTime = if (times.size > 0) { + times.sum / times.size + } else { + 0 + } val metricsNames = Seq("numParts", "numFiles", "numOutputBytes", "numOutputRows", "avgTime") val metricsValues = Seq(numPartitions, numFiles, totalNumBytes, totalNumOutput, avgWritingTime) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 4823ba3dda36a..33bfc866de402 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -273,6 +273,9 @@ object FileFormatWriter extends Logging { * The data structures used to measure metrics during writing. */ protected val writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty + protected var timeOnCurrentFile: Long = 0L + protected var numOutputRows: Long = 0L + protected var numOutputBytes: Long = 0L /** * Writes data out to files, and then returns the summary of relative information which @@ -295,15 +298,6 @@ object FileFormatWriter extends Logging { 0L } } - - /** - * Runs the given function and measures the time spent in ms. - */ - protected def measureTimeInMs(f: => Unit): Long = { - val startTime = System.nanoTime() - f - (System.nanoTime() - startTime) / 1000 / 1000 - } } /** Writes data to a single directory (used for non-dynamic-partition writes). */ @@ -331,9 +325,6 @@ object FileFormatWriter extends Logging { override def execute(iter: Iterator[InternalRow]): ExecutedWriteSummary = { var fileCounter = 0 var recordsInFile: Long = 0L - var writingTime: Long = 0L - var numOutputRows: Long = 0L - var numOutputBytes: Long = 0L newOutputWriter(fileCounter) while (iter.hasNext) { @@ -342,32 +333,20 @@ object FileFormatWriter extends Logging { assert(fileCounter < MAX_FILE_COUNTER, s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") - numOutputRows += recordsInFile recordsInFile = 0 - - writingTime += measureTimeInMs { - releaseResources() - } - writingTimePerFile += writingTime - writingTime = 0 - - numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) + releaseResources() + numOutputRows += recordsInFile newOutputWriter(fileCounter) } val internalRow = iter.next() - writingTime += measureTimeInMs { - currentWriter.write(internalRow) - } + val startTime = System.nanoTime() + currentWriter.write(internalRow) + timeOnCurrentFile += (System.nanoTime() - startTime) recordsInFile += 1 } - writingTime += measureTimeInMs { - releaseResources() - } - // Record the metrics for last file. - writingTimePerFile += writingTime + releaseResources() numOutputRows += recordsInFile - numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) ExecutedWriteSummary( updatedPartitions = Set.empty, @@ -380,7 +359,11 @@ object FileFormatWriter extends Logging { override def releaseResources(): Unit = { if (currentWriter != null) { try { + val startTime = System.nanoTime() currentWriter.close() + writingTimePerFile += (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 + timeOnCurrentFile = 0 + numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } finally { currentWriter = null } @@ -483,74 +466,52 @@ object FileFormatWriter extends Logging { // If anything below fails, we should abort the task. var recordsInFile: Long = 0L var fileCounter = 0 - var numOutputBytes: Long = 0L - var numOutputRows: Long = 0L var totalFileCounter = 0 - var writingTime: Long = 0L var currentPartColsAndBucketId: UnsafeRow = null val updatedPartitions = mutable.Set[String]() for (row <- iter) { val nextPartColsAndBucketId = getPartitionColsAndBucketId(row) if (currentPartColsAndBucketId != nextPartColsAndBucketId) { - writingTime += measureTimeInMs { - releaseResources() - } - - // Only update metrics when this is not the first partition. Otherwise, it generates - // a metrics record with zero values for a non-existing file. if (currentPartColsAndBucketId != null) { totalFileCounter += (fileCounter + 1) - numOutputRows += recordsInFile - writingTimePerFile += writingTime - numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } // See a new partition or bucket - write to a new partition dir (or a new bucket file). currentPartColsAndBucketId = nextPartColsAndBucketId.copy() logDebug(s"Writing partition: $currentPartColsAndBucketId") - writingTime = 0 + numOutputRows += recordsInFile recordsInFile = 0 fileCounter = 0 + releaseResources() newOutputWriter(currentPartColsAndBucketId, getPartPath, fileCounter, updatedPartitions) } else if (desc.maxRecordsPerFile > 0 && recordsInFile >= desc.maxRecordsPerFile) { // Exceeded the threshold in terms of the number of records per file. // Create a new file by increasing the file counter. - writingTime += measureTimeInMs { - releaseResources() - } - writingTimePerFile += writingTime numOutputRows += recordsInFile - numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) - recordsInFile = 0 - writingTime = 0 fileCounter += 1 assert(fileCounter < MAX_FILE_COUNTER, s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") + releaseResources() newOutputWriter(currentPartColsAndBucketId, getPartPath, fileCounter, updatedPartitions) } - writingTime += measureTimeInMs { - currentWriter.write(getOutputRow(row)) - } + val startTime = System.nanoTime() + currentWriter.write(getOutputRow(row)) + timeOnCurrentFile += (System.nanoTime() - startTime) recordsInFile += 1 } - - writingTime += measureTimeInMs { - releaseResources() - } - // Record the metrics of last file/partition if any. if (currentPartColsAndBucketId != null) { totalFileCounter += (fileCounter + 1) - writingTimePerFile += writingTime - numOutputRows += recordsInFile - numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } + releaseResources() + numOutputRows += recordsInFile + ExecutedWriteSummary( updatedPartitions = updatedPartitions.toSet, numOutputFile = totalFileCounter, @@ -562,7 +523,11 @@ object FileFormatWriter extends Logging { override def releaseResources(): Unit = { if (currentWriter != null) { try { + val startTime = System.nanoTime() currentWriter.close() + writingTimePerFile += (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 + timeOnCurrentFile = 0 + numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } finally { currentWriter = null } @@ -572,7 +537,7 @@ object FileFormatWriter extends Logging { } /** - * Wrapper class for the information of writing data out. + * Wrapper class for the metrics of writing data out. * * @param updatedPartitions the partitions updated during writing data out. Only valid * for dynamic partition. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala index d3d340d182a31..6e2a79e7148ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.metric +import java.io.File + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier @@ -75,7 +77,6 @@ trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt assert(totalNumBytes > 0) - val writingTimeMetric = executedNode.metrics.find(_.name == "average writing time (ms)").get val writingTime = metrics(writingTimeMetric.accumulatorId).replaceAll(",", "").toInt assert(writingTime >= 0) @@ -90,16 +91,21 @@ trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { Seq((1, 2)).toDF("i", "j") .write.format(dataFormat).mode("overwrite").saveAsTable(tableName) } - // written 1 file, 1 row, 0 dynamic partition. + val tableLocation = + new File(spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location) + + assert(Utils.recursiveList(tableLocation).count(_.getName.startsWith("part-")) == 1) + // 1 file, 1 row, 0 dynamic partition. val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 1) verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) val executionId2 = getLatestExecutionId(spark) { () => - Seq((9, 10), (11, 12)).toDF("i", "j").repartition(2) - .write.format(dataFormat).insertInto(tableName) + (0 until 100).map(i => (i, i + 1)).toDF("i", "j").repartition(2) + .write.format(dataFormat).mode("overwrite").insertInto(tableName) } - // written 2 files, 2 rows, 0 dynamic partition. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ == 2) + assert(Utils.recursiveList(tableLocation).count(_.getName.startsWith("part-")) == 2) + // 2 files, 100 rows, 0 dynamic partition. + val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ == 100) verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) } } @@ -112,16 +118,16 @@ trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { withTempPath { dir => spark.sql( s""" - |CREATE TABLE t1(a int, b int) + |CREATE TABLE $tableName(a int, b int) |USING $provider |PARTITIONED BY(a) |LOCATION '${dir.toURI}' """.stripMargin) - val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) - val df = spark.range(start = 0, end = 4, step = 1, numPartitions = 1) + val df = spark.range(start = 0, end = 40, step = 1, numPartitions = 1) .selectExpr("id a", "id b") sql("SET hive.exec.dynamic.partition.mode=nonstrict") @@ -129,13 +135,12 @@ trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { df.union(df).repartition(2, $"a") .write .format(dataFormat) - .option("maxRecordsPerFile", 2) .mode("overwrite") .insertInto(tableName) } - assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 4) - // written 4 files, 8 rows, 4 dynamic partitions. - val verifyFuncs: Seq[Int => Boolean] = Seq(_ == 4, _ == 4, _ == 8) + assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 40) + // 40 files, 80 rows, 40 dynamic partitions. + val verifyFuncs: Seq[Int => Boolean] = Seq(_ == 40, _ == 40, _ == 80) verifyWriteDataMetrics(spark, executionId, verifyFuncs) } } From de069e757d81419aee21748d3aa605ee7a75cf33 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 4 Jun 2017 13:50:52 +0000 Subject: [PATCH 10/22] Simplify test cases. --- .../metric/WritingDataMetricsSuite.scala | 50 ++++++------------- .../sql/hive/execution/SQLMetricsSuite.scala | 1 + 2 files changed, 17 insertions(+), 34 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala index 6e2a79e7148ba..c91c2766829b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala @@ -30,31 +30,23 @@ trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { import testImplicits._ /** - * Run the given function and return latest execution id. + * Get execution metrics for the SQL execution and verify metrics values. * - * @param func the given function to run. + * @param metricsValues the expected metric values (numFiles, numPartitions, numOutputRows). + * @param func the function can produce execution id after running. */ - def getLatestExecutionId(spark: SparkSession)(func: () => Unit): Long = { + def verifyWriteDataMetrics( + spark: SparkSession, + metricsValues: Seq[Int])(func: => Unit): Unit = { val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet // Run the given function to trigger query execution. - func() + func spark.sparkContext.listenerBus.waitUntilEmpty(10000) val executionIds = spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) assert(executionIds.size == 1) - executionIds.head - } + val executionId = executionIds.head - /** - * Get execution metrics for the given execution id and verify metrics values. - * - * @param executionId the given execution id. - * @param verifyFuncs functions used to verify the values of metrics. - */ - def verifyWriteDataMetrics( - spark: SparkSession, - executionId: Long, - verifyFuncs: Seq[Int => Boolean]): Unit = { val executionData = spark.sharedState.listener.getExecution(executionId).get val executedNode = executionData.physicalPlanGraph.nodes.head @@ -65,15 +57,14 @@ trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { val metrics = spark.sharedState.listener.getExecutionMetrics(executionId) - metricsNames.zip(verifyFuncs).foreach { case (metricsName, verifyFunc) => + metricsNames.zip(metricsValues).foreach { case (metricsName, expected) => val sqlMetric = executedNode.metrics.find(_.name == metricsName) assert(sqlMetric.isDefined) val accumulatorId = sqlMetric.get.accumulatorId val metricValue = metrics(accumulatorId).replaceAll(",", "").toInt - assert(verifyFunc(metricValue)) + assert(metricValue == expected) } - // Sanity check. val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt assert(totalNumBytes > 0) @@ -87,26 +78,21 @@ trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { dataFormat: String, tableName: String): Unit = { withTable(tableName) { - val executionId1 = getLatestExecutionId(spark) { () => + // 1 file, 1 row, 0 dynamic partition. + verifyWriteDataMetrics(spark, Seq(1, 0, 1)) { Seq((1, 2)).toDF("i", "j") .write.format(dataFormat).mode("overwrite").saveAsTable(tableName) } val tableLocation = new File(spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location) - assert(Utils.recursiveList(tableLocation).count(_.getName.startsWith("part-")) == 1) - // 1 file, 1 row, 0 dynamic partition. - val verifyFuncs1: Seq[Int => Boolean] = Seq(_ == 1, _ == 0, _ == 1) - verifyWriteDataMetrics(spark, executionId1, verifyFuncs1) - val executionId2 = getLatestExecutionId(spark) { () => + // 2 files, 100 rows, 0 dynamic partition. + verifyWriteDataMetrics(spark, Seq(2, 0, 100)) { (0 until 100).map(i => (i, i + 1)).toDF("i", "j").repartition(2) .write.format(dataFormat).mode("overwrite").insertInto(tableName) } assert(Utils.recursiveList(tableLocation).count(_.getName.startsWith("part-")) == 2) - // 2 files, 100 rows, 0 dynamic partition. - val verifyFuncs2: Seq[Int => Boolean] = Seq(_ == 2, _ == 0, _ == 100) - verifyWriteDataMetrics(spark, executionId2, verifyFuncs2) } } @@ -123,15 +109,14 @@ trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { |PARTITIONED BY(a) |LOCATION '${dir.toURI}' """.stripMargin) - val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) val df = spark.range(start = 0, end = 40, step = 1, numPartitions = 1) .selectExpr("id a", "id b") - sql("SET hive.exec.dynamic.partition.mode=nonstrict") - val executionId = getLatestExecutionId(spark) { () => + // 40 files, 80 rows, 40 dynamic partitions. + verifyWriteDataMetrics(spark, Seq(40, 40, 80)) { df.union(df).repartition(2, $"a") .write .format(dataFormat) @@ -139,9 +124,6 @@ trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { .insertInto(tableName) } assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 40) - // 40 files, 80 rows, 40 dynamic partitions. - val verifyFuncs: Seq[Int => Boolean] = Seq(_ == 40, _ == 40, _ == 80) - verifyWriteDataMetrics(spark, executionId, verifyFuncs) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala index d9b86833220dd..7fd6773b0bbb2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -26,6 +26,7 @@ class SQLMetricsSuite extends BaseWritingDataMetricsSuite with TestHiveSingleton } test("writing data out metrics: dynamic partition") { + sql("SET hive.exec.dynamic.partition.mode=nonstrict") testMetricsDynamicPartition(spark, "hive", "hive", "t1") } } From 775d2c284666238e25cbe4cdaf95677fde358eb4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 4 Jun 2017 14:25:43 +0000 Subject: [PATCH 11/22] Simplify test cases again. --- .../command/WriteOutFileCommand.scala | 55 ------- .../command/WrittenFileCommandExec.scala | 33 ++++- .../command/createDataSourceTables.scala | 2 +- .../execution/datasources/DataSource.scala | 9 -- .../metric/WritingDataMetricsSuite.scala | 139 ------------------ .../sql/hive/execution/SQLMetricsSuite.scala | 124 +++++++++++++++- 6 files changed, 150 insertions(+), 212 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala deleted file mode 100644 index ac9b7a9f6432a..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WriteOutFileCommand.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.command - -import scala.collection.mutable - -import org.apache.spark.SparkContext -import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} - -/** - * A logical command specialized for writing data out. `WriteOutFileCommand`s are - * wrapped in `WrittenFileCommandExec` during execution. - */ -trait WriteOutFileCommand extends logical.Command { - - /** - * Those metrics will be updated once the command finishes writing data out. Those metrics will - * be taken by `WrittenFileCommandExe` as its metrics when showing in UI. - */ - def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = - Map( - // General metrics. - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part"), - "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), - "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), - "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)") - ) - - def run( - sparkSession: SparkSession, - children: Seq[SparkPlan], - metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { - throw new NotImplementedError - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala index 0c70621ea9fe2..8347034573624 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala @@ -17,10 +17,41 @@ package org.apache.spark.sql.execution.command +import org.apache.spark.SparkContext +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * A logical command specialized for writing data out. `WriteOutFileCommand`s are + * wrapped in `WrittenFileCommandExec` during execution. + */ +trait WriteOutFileCommand extends logical.Command { + + /** + * Those metrics will be updated once the command finishes writing data out. Those metrics will + * be taken by `WrittenFileCommandExe` as its metrics when showing in UI. + */ + def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = + Map( + // General metrics. + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part"), + "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), + "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), + "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)") + ) + + def run( + sparkSession: SparkSession, + children: Seq[SparkPlan], + metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { + throw new NotImplementedError + } +} /** * A physical operator specialized to execute the run method of a `WriteOutFileCommand`, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index d6a01b65c2019..86a449fe106af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -135,7 +135,7 @@ case class CreateDataSourceTableAsSelectCommand( * record metrics for that. So we will return empty metrics map. */ override def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = { - if (DataSource.isFileFormat(table.provider.get)) { + if (classOf[FileFormat].isAssignableFrom(DataSource.lookupDataSource(table.provider.get))) { super.metrics(sparkContext) } else { Map.empty diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 6d10c2ec4be11..bc3421e6c09af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -599,15 +599,6 @@ object DataSource extends Logging { } } - /** Given a provider name, returns if the provider is a `FileFormat`-based data source. */ - def isFileFormat(provider: String): Boolean = { - if (classOf[FileFormat].isAssignableFrom(lookupDataSource(provider))) { - true - } else { - false - } - } - /** * When creating a data source table, the `path` option has a special meaning: the table location. * This method extracts the `path` option and treat it as table location to build a diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala deleted file mode 100644 index c91c2766829b9..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/WritingDataMetricsSuite.scala +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.metric - -import java.io.File - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} -import org.apache.spark.util.Utils - -trait BaseWritingDataMetricsSuite extends SparkFunSuite with SQLTestUtils { - import testImplicits._ - - /** - * Get execution metrics for the SQL execution and verify metrics values. - * - * @param metricsValues the expected metric values (numFiles, numPartitions, numOutputRows). - * @param func the function can produce execution id after running. - */ - def verifyWriteDataMetrics( - spark: SparkSession, - metricsValues: Seq[Int])(func: => Unit): Unit = { - val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet - // Run the given function to trigger query execution. - func - spark.sparkContext.listenerBus.waitUntilEmpty(10000) - val executionIds = - spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) - assert(executionIds.size == 1) - val executionId = executionIds.head - - val executionData = spark.sharedState.listener.getExecution(executionId).get - val executedNode = executionData.physicalPlanGraph.nodes.head - - val metricsNames = Seq( - "number of written files", - "number of dynamic part", - "number of output rows") - - val metrics = spark.sharedState.listener.getExecutionMetrics(executionId) - - metricsNames.zip(metricsValues).foreach { case (metricsName, expected) => - val sqlMetric = executedNode.metrics.find(_.name == metricsName) - assert(sqlMetric.isDefined) - val accumulatorId = sqlMetric.get.accumulatorId - val metricValue = metrics(accumulatorId).replaceAll(",", "").toInt - assert(metricValue == expected) - } - - val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get - val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt - assert(totalNumBytes > 0) - val writingTimeMetric = executedNode.metrics.find(_.name == "average writing time (ms)").get - val writingTime = metrics(writingTimeMetric.accumulatorId).replaceAll(",", "").toInt - assert(writingTime >= 0) - } - - protected def testMetricsNonDynamicPartition( - spark: SparkSession, - dataFormat: String, - tableName: String): Unit = { - withTable(tableName) { - // 1 file, 1 row, 0 dynamic partition. - verifyWriteDataMetrics(spark, Seq(1, 0, 1)) { - Seq((1, 2)).toDF("i", "j") - .write.format(dataFormat).mode("overwrite").saveAsTable(tableName) - } - val tableLocation = - new File(spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location) - assert(Utils.recursiveList(tableLocation).count(_.getName.startsWith("part-")) == 1) - - // 2 files, 100 rows, 0 dynamic partition. - verifyWriteDataMetrics(spark, Seq(2, 0, 100)) { - (0 until 100).map(i => (i, i + 1)).toDF("i", "j").repartition(2) - .write.format(dataFormat).mode("overwrite").insertInto(tableName) - } - assert(Utils.recursiveList(tableLocation).count(_.getName.startsWith("part-")) == 2) - } - } - - protected def testMetricsDynamicPartition( - spark: SparkSession, - provider: String, - dataFormat: String, - tableName: String): Unit = { - withTempPath { dir => - spark.sql( - s""" - |CREATE TABLE $tableName(a int, b int) - |USING $provider - |PARTITIONED BY(a) - |LOCATION '${dir.toURI}' - """.stripMargin) - val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) - assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) - - val df = spark.range(start = 0, end = 40, step = 1, numPartitions = 1) - .selectExpr("id a", "id b") - - // 40 files, 80 rows, 40 dynamic partitions. - verifyWriteDataMetrics(spark, Seq(40, 40, 80)) { - df.union(df).repartition(2, $"a") - .write - .format(dataFormat) - .mode("overwrite") - .insertInto(tableName) - } - assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 40) - } - } -} - -class WritingDataMetricsSuite extends SharedSQLContext with BaseWritingDataMetricsSuite { - test("writing data out metrics") { - testMetricsNonDynamicPartition(spark, "parquet", "t1") - } - - test("writing data out metrics: dynamic partition") { - testMetricsDynamicPartition(spark, "parquet", "parquet", "t1") - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala index 7fd6773b0bbb2..0c900ec1ffd22 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -17,16 +17,126 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.execution.metric.BaseWritingDataMetricsSuite +import java.io.File + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.util.Utils + +class SQLMetricsSuite extends SQLTestUtils with TestHiveSingleton { + import spark.implicits._ + + /** + * Get execution metrics for the SQL execution and verify metrics values. + * + * @param metricsValues the expected metric values (numFiles, numPartitions, numOutputRows). + * @param func the function can produce execution id after running. + */ + private def verifyWriteDataMetrics(metricsValues: Seq[Int])(func: => Unit): Unit = { + val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet + // Run the given function to trigger query execution. + func + spark.sparkContext.listenerBus.waitUntilEmpty(10000) + val executionIds = + spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) + assert(executionIds.size == 1) + val executionId = executionIds.head + + val executionData = spark.sharedState.listener.getExecution(executionId).get + val executedNode = executionData.physicalPlanGraph.nodes.head + + val metricsNames = Seq( + "number of written files", + "number of dynamic part", + "number of output rows") + + val metrics = spark.sharedState.listener.getExecutionMetrics(executionId) + + metricsNames.zip(metricsValues).foreach { case (metricsName, expected) => + val sqlMetric = executedNode.metrics.find(_.name == metricsName) + assert(sqlMetric.isDefined) + val accumulatorId = sqlMetric.get.accumulatorId + val metricValue = metrics(accumulatorId).replaceAll(",", "").toInt + assert(metricValue == expected) + } + + val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get + val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt + assert(totalNumBytes > 0) + val writingTimeMetric = executedNode.metrics.find(_.name == "average writing time (ms)").get + val writingTime = metrics(writingTimeMetric.accumulatorId).replaceAll(",", "").toInt + assert(writingTime >= 0) + } + + private def testMetricsNonDynamicPartition( + dataFormat: String, + tableName: String): Unit = { + withTable(tableName) { + // 1 file, 1 row, 0 dynamic partition. + verifyWriteDataMetrics(Seq(1, 0, 1)) { + Seq((1, 2)).toDF("i", "j") + .write.format(dataFormat).mode("overwrite").saveAsTable(tableName) + } + val tableLocation = + new File(spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location) + assert(Utils.recursiveList(tableLocation).count(_.getName.startsWith("part-")) == 1) + + // 2 files, 100 rows, 0 dynamic partition. + verifyWriteDataMetrics(Seq(2, 0, 100)) { + (0 until 100).map(i => (i, i + 1)).toDF("i", "j").repartition(2) + .write.format(dataFormat).mode("overwrite").insertInto(tableName) + } + assert(Utils.recursiveList(tableLocation).count(_.getName.startsWith("part-")) == 2) + } + } + + private def testMetricsDynamicPartition( + provider: String, + dataFormat: String, + tableName: String): Unit = { + withTempPath { dir => + spark.sql( + s""" + |CREATE TABLE $tableName(a int, b int) + |USING $provider + |PARTITIONED BY(a) + |LOCATION '${dir.toURI}' + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + val df = spark.range(start = 0, end = 40, step = 1, numPartitions = 1) + .selectExpr("id a", "id b") + + // 40 files, 80 rows, 40 dynamic partitions. + verifyWriteDataMetrics(Seq(40, 40, 80)) { + df.union(df).repartition(2, $"a") + .write + .format(dataFormat) + .mode("overwrite") + .insertInto(tableName) + } + assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 40) + } + } + + test("writing data out metrics: parquet") { + testMetricsNonDynamicPartition("parquet", "t1") + } + + test("writing data out metrics with dynamic partition: parquet") { + testMetricsDynamicPartition("parquet", "parquet", "t1") + } -class SQLMetricsSuite extends BaseWritingDataMetricsSuite with TestHiveSingleton { - test("writing data out metrics") { - testMetricsNonDynamicPartition(spark, "hive", "t1") + test("writing data out metrics: hive") { + testMetricsNonDynamicPartition("hive", "t1") } - test("writing data out metrics: dynamic partition") { - sql("SET hive.exec.dynamic.partition.mode=nonstrict") - testMetricsDynamicPartition(spark, "hive", "hive", "t1") + test("writing data out metrics dynamic partition: hive") { + withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { + testMetricsDynamicPartition("hive", "hive", "t1") + } } } From f670ae32c58e9dbe060e0e62aaf3e440ff7a9c4e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 6 Jun 2017 01:58:39 +0000 Subject: [PATCH 12/22] Address comments. --- .../scala/org/apache/spark/util/Utils.scala | 7 ++ .../spark/sql/execution/SparkStrategies.scala | 2 +- ...andExec.scala => FileWritingCommand.scala} | 86 +++++++++---------- .../sql/execution/command/commands.scala | 18 ++-- .../command/createDataSourceTables.scala | 2 +- .../execution/datasources/DataSource.scala | 4 +- .../InsertIntoHadoopFsRelationCommand.scala | 8 +- .../CreateHiveTableAsSelectCommand.scala | 8 +- .../hive/execution/InsertIntoHiveTable.scala | 7 +- 9 files changed, 77 insertions(+), 65 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/command/{WrittenFileCommandExec.scala => FileWritingCommand.scala} (59%) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 58884a1c777b9..250256479e496 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2668,6 +2668,13 @@ private[spark] object Utils extends Logging { redact(redactionPattern, kvs.toArray) } + def average[T](ts: Iterable[T])(implicit num: Numeric[T]): Double = { + if (ts.size > 0) { + num.toDouble(ts.sum) / ts.size + } else { + 0 + } + } } private[util] object CallerContext extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 4fb906478db0a..0ef3a95149d47 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -346,7 +346,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case w: WriteOutFileCommand => WrittenFileCommandExec(w, w.children.map(planLater)) :: Nil + case f: FileWritingCommand => FileWritingCommandExec(f, f.children.map(planLater)) :: Nil case r: RunnableCommand => ExecutedCommandExec(r) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala similarity index 59% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala index 8347034573624..954c678c37aaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala @@ -24,49 +24,38 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.util.Utils /** - * A logical command specialized for writing data out. `WriteOutFileCommand`s are - * wrapped in `WrittenFileCommandExec` during execution. + * A logical command specialized for writing data out. `FileWritingCommand`s are + * wrapped in `FileWritingCommandExec` during execution. */ -trait WriteOutFileCommand extends logical.Command { +trait FileWritingCommand extends logical.Command { + + // The caller of `FileWritingCommand` can replace the metrics location by providing this external + // metrics structure. + val externalMetrics: Option[Map[String, SQLMetric]] = None /** * Those metrics will be updated once the command finishes writing data out. Those metrics will - * be taken by `WrittenFileCommandExe` as its metrics when showing in UI. + * be taken by `FileWritingCommandExec` as its metrics when showing in UI. */ - def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = + def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = externalMetrics.getOrElse { Map( // General metrics. - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part"), + "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), - "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)") + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") ) - - def run( - sparkSession: SparkSession, - children: Seq[SparkPlan], - metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { - throw new NotImplementedError } -} - -/** - * A physical operator specialized to execute the run method of a `WriteOutFileCommand`, - * save the result to prevent multiple executions, and record necessary metrics for UI. - */ -case class WrittenFileCommandExec( - cmd: WriteOutFileCommand, - children: Seq[SparkPlan]) extends CommandExec { - - override lazy val metrics = cmd.metrics(sqlContext.sparkContext) /** - * The callback function used to update metrics returned from the operation of writing data out. + * Callback function that update metrics collected from the writing operation. */ - private def updateDriverMetrics(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { + private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: Map[String, SQLMetric]) + (writeSummaries: Seq[ExecutedWriteSummary]): Unit = { var numPartitions = 0 var numFiles = 0 var totalNumBytes: Long = 0L @@ -79,25 +68,36 @@ case class WrittenFileCommandExec( totalNumOutput += summary.numOutputRows } - val times = writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0)) - val avgWritingTime = if (times.size > 0) { - times.sum / times.size - } else { - 0 - } + // The time for writing individual file can be zero if it's less than 1 ms. Zero values can + // lower actual time of writing when calculating average, so excluding them. + val writingTime = + Utils.average(writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0))).toLong - val metricsNames = Seq("numParts", "numFiles", "numOutputBytes", "numOutputRows", "avgTime") - val metricsValues = Seq(numPartitions, numFiles, totalNumBytes, totalNumOutput, avgWritingTime) + val metricsNames = metrics.keys.toSeq.sorted + val metricsValues = Seq(writingTime, numFiles, totalNumBytes, totalNumOutput, numPartitions) metricsNames.zip(metricsValues).foreach(x => metrics(x._1).add(x._2)) - val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, - metricsNames.map(metrics(_))) + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metricsNames.map(metrics(_))) } - protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { - val converter = CatalystTypeConverters.createToCatalystConverter(schema) - val rows = cmd.run(sqlContext.sparkSession, children, updateDriverMetrics) - rows.map(converter(_).asInstanceOf[InternalRow]) - } + def run( + sparkSession: SparkSession, + children: Seq[SparkPlan], + metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] +} + +/** + * A physical operator specialized to execute the run method of a `FileWritingCommand`, + * save the result to prevent multiple executions, and record necessary metrics for UI. + */ +case class FileWritingCommandExec( + cmd: FileWritingCommand, + children: Seq[SparkPlan]) extends CommandExec { + + override lazy val metrics = cmd.metrics(sqlContext.sparkContext) + + protected[sql] lazy val invokeCommand: Seq[Row] = + cmd.run(sqlContext.sparkSession, children, + cmd.postDriverMetrics(sqlContext.sparkContext, metrics)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index be1113cb97eb4..2de334822eb32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -35,9 +35,7 @@ import org.apache.spark.sql.types._ * wrapped in `ExecutedCommand` during execution. */ trait RunnableCommand extends logical.Command { - def run(sparkSession: SparkSession): Seq[Row] = { - throw new NotImplementedError - } + def run(sparkSession: SparkSession): Seq[Row] } /** @@ -56,7 +54,12 @@ trait CommandExec extends SparkPlan { * The `execute()` method of all the physical command classes should reference `sideEffectResult` * so that the command can be executed eagerly right after the command query is created. */ - protected[sql] val sideEffectResult: Seq[InternalRow] + protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { + val converter = CatalystTypeConverters.createToCatalystConverter(schema) + invokeCommand.map(converter(_).asInstanceOf[InternalRow]) + } + + protected[sql] val invokeCommand: Seq[Row] override def innerChildren: Seq[QueryPlan[_]] = cmd.innerChildren @@ -80,12 +83,7 @@ trait CommandExec extends SparkPlan { * save the result to prevent multiple executions. */ case class ExecutedCommandExec(cmd: RunnableCommand) extends CommandExec { - override protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { - val converter = CatalystTypeConverters.createToCatalystConverter(schema) - val rows = cmd.run(sqlContext.sparkSession) - rows.map(converter(_).asInstanceOf[InternalRow]) - } - + protected[sql] lazy val invokeCommand: Seq[Row] = cmd.run(sqlContext.sparkSession) override def children: Seq[SparkPlan] = Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 86a449fe106af..528e7bf1f2b4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -123,7 +123,7 @@ case class CreateDataSourceTableAsSelectCommand( table: CatalogTable, mode: SaveMode, query: LogicalPlan) - extends WriteOutFileCommand { + extends FileWritingCommand { /** * The code path this command writes data out depends on the type of data source: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index bc3421e6c09af..215ac078c108d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.execution.command.WrittenFileCommandExec +import org.apache.spark.sql.execution.command.FileWritingCommandExec import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider import org.apache.spark.sql.execution.datasources.json.JsonFileFormat @@ -452,7 +452,7 @@ case class DataSource( case format: FileFormat => val qe = sparkSession.sessionState.executePlan(planForWritingFileFormat(format, mode, data)) val insertCommand = qe.executedPlan.collect { - case w: WrittenFileCommandExec => w + case f: FileWritingCommandExec => f }.head insertCommand.cmd.run(sparkSession, insertCommand.children, metricsCallback.getOrElse(_ => ())) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index a6b556bb61e31..000921455f9b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.metric.SQLMetric /** * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. @@ -40,6 +41,8 @@ import org.apache.spark.sql.execution.command._ * the prefix are overwritten. * @param ifPartitionNotExists If true, only write if the partition does not exist. * Only valid for static partitions. + * @param externalMetrics If given, it will replace the inherited `metrics` as the actual metrics + * destination when running this command. */ case class InsertIntoHadoopFsRelationCommand( outputPath: Path, @@ -52,8 +55,9 @@ case class InsertIntoHadoopFsRelationCommand( query: LogicalPlan, mode: SaveMode, catalogTable: Option[CatalogTable], - fileIndex: Option[FileIndex]) - extends WriteOutFileCommand { + fileIndex: Option[FileIndex], + override val externalMetrics: Option[Map[String, SQLMetric]] = None) + extends FileWritingCommand { import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName override def children: Seq[LogicalPlan] = query :: Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 5aa1524310326..ea8a8d557e727 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.{WriteOutFileCommand, WrittenFileCommandExec} +import org.apache.spark.sql.execution.command.{FileWritingCommand, FileWritingCommandExec} import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary @@ -39,7 +39,7 @@ case class CreateHiveTableAsSelectCommand( tableDesc: CatalogTable, query: LogicalPlan, mode: SaveMode) - extends WriteOutFileCommand { + extends FileWritingCommand { private val tableIdentifier = tableDesc.identifier @@ -69,7 +69,7 @@ case class CreateHiveTableAsSelectCommand( overwrite = false, ifPartitionNotExists = false)) val insertCommand = qe.executedPlan.collect { - case w: WrittenFileCommandExec => w + case f: FileWritingCommandExec => f }.head insertCommand.cmd.run(sparkSession, insertCommand.children, metricsCallback) } else { @@ -89,7 +89,7 @@ case class CreateHiveTableAsSelectCommand( overwrite = true, ifPartitionNotExists = false)) val insertCommand = qe.executedPlan.collect { - case w: WrittenFileCommandExec => w + case f: FileWritingCommandExec => f }.head insertCommand.cmd.run(sparkSession, insertCommand.children, metricsCallback) } catch { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 594f348d27d68..c9526fd377bb2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -37,8 +37,9 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.WriteOutFileCommand +import org.apache.spark.sql.execution.command.FileWritingCommand import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, FileFormatWriter} +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.sql.hive.client.{HiveClientImpl, HiveVersion} @@ -80,7 +81,9 @@ case class InsertIntoHiveTable( partition: Map[String, Option[String]], query: LogicalPlan, overwrite: Boolean, - ifPartitionNotExists: Boolean) extends WriteOutFileCommand { + ifPartitionNotExists: Boolean, + override val externalMetrics: Option[Map[String, SQLMetric]] = None) + extends FileWritingCommand { override def children: Seq[LogicalPlan] = query :: Nil From 0af718d15ed9c6bcf4e8de19528affdc492d1257 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 6 Jun 2017 02:45:40 +0000 Subject: [PATCH 13/22] Remove hacky part. --- .../command/FileWritingCommand.scala | 11 +++++++--- .../command/createDataSourceTables.scala | 12 +++++------ .../execution/datasources/DataSource.scala | 13 ++++++------ .../InsertIntoHadoopFsRelationCommand.scala | 7 ++----- .../CreateHiveTableAsSelectCommand.scala | 21 +++++++++++-------- .../hive/execution/InsertIntoHiveTable.scala | 5 ++--- 6 files changed, 37 insertions(+), 32 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala index 954c678c37aaa..768309c22827f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala @@ -34,13 +34,17 @@ trait FileWritingCommand extends logical.Command { // The caller of `FileWritingCommand` can replace the metrics location by providing this external // metrics structure. - val externalMetrics: Option[Map[String, SQLMetric]] = None + private var _externalMetrics: Option[Map[String, SQLMetric]] = None + private[sql] def withExternalMetrics(map: Map[String, SQLMetric]): this.type = { + _externalMetrics = Option(map) + this + } /** * Those metrics will be updated once the command finishes writing data out. Those metrics will * be taken by `FileWritingCommandExec` as its metrics when showing in UI. */ - def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = externalMetrics.getOrElse { + def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = _externalMetrics.getOrElse { Map( // General metrics. "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), @@ -84,6 +88,7 @@ trait FileWritingCommand extends logical.Command { def run( sparkSession: SparkSession, children: Seq[SparkPlan], + metrics: Map[String, SQLMetric], metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] } @@ -98,6 +103,6 @@ case class FileWritingCommandExec( override lazy val metrics = cmd.metrics(sqlContext.sparkContext) protected[sql] lazy val invokeCommand: Seq[Row] = - cmd.run(sqlContext.sparkSession, children, + cmd.run(sqlContext.sparkSession, children, metrics, cmd.postDriverMetrics(sqlContext.sparkContext, metrics)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 528e7bf1f2b4c..939c0abbae367 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -134,19 +134,19 @@ case class CreateDataSourceTableAsSelectCommand( * For other data sources, `CreatableRelationProvider.createRelation` will be called. We can't * record metrics for that. So we will return empty metrics map. */ - override def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = { + override def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = if (classOf[FileFormat].isAssignableFrom(DataSource.lookupDataSource(table.provider.get))) { super.metrics(sparkContext) } else { Map.empty } - } override def innerChildren: Seq[LogicalPlan] = Seq(query) override def run( sparkSession: SparkSession, children: Seq[SparkPlan], + metrics: Map[String, SQLMetric], metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { assert(table.tableType != CatalogTableType.VIEW) assert(table.provider.isDefined) @@ -170,7 +170,7 @@ case class CreateDataSourceTableAsSelectCommand( saveDataIntoTable( sparkSession, table, table.storage.locationUri, query, SaveMode.Append, tableExists = true, - metricsCallback = metricsCallback) + metrics = metrics) } else { assert(table.schema.isEmpty) @@ -181,7 +181,7 @@ case class CreateDataSourceTableAsSelectCommand( } val result = saveDataIntoTable( sparkSession, table, tableLocation, query, SaveMode.Overwrite, tableExists = false, - metricsCallback = metricsCallback) + metrics = metrics) val newTable = table.copy( storage = table.storage.copy(locationUri = tableLocation), // We will use the schema of resolved.relation as the schema of the table (instead of @@ -209,7 +209,7 @@ case class CreateDataSourceTableAsSelectCommand( data: LogicalPlan, mode: SaveMode, tableExists: Boolean, - metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): BaseRelation = { + metrics: Map[String, SQLMetric]): BaseRelation = { // Create the relation based on the input logical plan: `data`. val pathOption = tableLocation.map("path" -> CatalogUtils.URIToString(_)) val dataSource = DataSource( @@ -221,7 +221,7 @@ case class CreateDataSourceTableAsSelectCommand( catalogTable = if (tableExists) Some(table) else None) try { - dataSource.writeAndRead(mode, query, Some(metricsCallback)) + dataSource.writeAndRead(mode, query, Some(metrics)) } catch { case ex: AnalysisException => logError(s"Failed to write to table ${table.identifier.unquotedString}", ex) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 215ac078c108d..3f502d275bee1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.streaming.OutputMode @@ -440,7 +441,7 @@ case class DataSource( def writeAndRead( mode: SaveMode, data: LogicalPlan, - metricsCallback: Option[Seq[ExecutedWriteSummary] => Unit] = None): BaseRelation = { + externalMetrics: Option[Map[String, SQLMetric]] = None): BaseRelation = { if (data.schema.map(_.dataType).exists(_.isInstanceOf[CalendarIntervalType])) { throw new AnalysisException("Cannot save interval data type into external storage.") } @@ -451,11 +452,11 @@ case class DataSource( sparkSession.sqlContext, mode, caseInsensitiveOptions, Dataset.ofRows(sparkSession, data)) case format: FileFormat => val qe = sparkSession.sessionState.executePlan(planForWritingFileFormat(format, mode, data)) - val insertCommand = qe.executedPlan.collect { - case f: FileWritingCommandExec => f - }.head - insertCommand.cmd.run(sparkSession, insertCommand.children, - metricsCallback.getOrElse(_ => ())) + qe.executedPlan.transform { + case f: FileWritingCommandExec => + val newCmd = f.cmd.withExternalMetrics(externalMetrics.getOrElse(null)) + FileWritingCommandExec(newCmd, f.children) + }.execute() // Replace the schema with that of the DataFrame we just wrote out to avoid re-inferring copy(userSpecifiedSchema = Some(data.schema.asNullable)).resolveRelation() case _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 000921455f9b9..2cb6c80e45957 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -41,8 +41,6 @@ import org.apache.spark.sql.execution.metric.SQLMetric * the prefix are overwritten. * @param ifPartitionNotExists If true, only write if the partition does not exist. * Only valid for static partitions. - * @param externalMetrics If given, it will replace the inherited `metrics` as the actual metrics - * destination when running this command. */ case class InsertIntoHadoopFsRelationCommand( outputPath: Path, @@ -55,9 +53,7 @@ case class InsertIntoHadoopFsRelationCommand( query: LogicalPlan, mode: SaveMode, catalogTable: Option[CatalogTable], - fileIndex: Option[FileIndex], - override val externalMetrics: Option[Map[String, SQLMetric]] = None) - extends FileWritingCommand { + fileIndex: Option[FileIndex]) extends FileWritingCommand { import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName override def children: Seq[LogicalPlan] = query :: Nil @@ -65,6 +61,7 @@ case class InsertIntoHadoopFsRelationCommand( override def run( sparkSession: SparkSession, children: Seq[SparkPlan], + metrics: Map[String, SQLMetric], metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { assert(children.length == 1) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index ea8a8d557e727..1d94d93761398 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.{FileWritingCommand, FileWritingCommandExec} import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary - +import org.apache.spark.sql.execution.metric.SQLMetric /** * Create table and insert the query result into it. @@ -48,6 +48,7 @@ case class CreateHiveTableAsSelectCommand( override def run( sparkSession: SparkSession, children: Seq[SparkPlan], + metrics: Map[String, SQLMetric], metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { if (sparkSession.sessionState.catalog.tableExists(tableIdentifier)) { assert(mode != SaveMode.Overwrite, @@ -68,10 +69,11 @@ case class CreateHiveTableAsSelectCommand( query, overwrite = false, ifPartitionNotExists = false)) - val insertCommand = qe.executedPlan.collect { - case f: FileWritingCommandExec => f - }.head - insertCommand.cmd.run(sparkSession, insertCommand.children, metricsCallback) + qe.executedPlan.transform { + case f: FileWritingCommandExec => + val newCmd = f.cmd.withExternalMetrics(metrics) + FileWritingCommandExec(newCmd, f.children) + }.execute() } else { // TODO ideally, we should get the output data ready first and then // add the relation into catalog, just in case of failure occurs while data @@ -88,10 +90,11 @@ case class CreateHiveTableAsSelectCommand( query, overwrite = true, ifPartitionNotExists = false)) - val insertCommand = qe.executedPlan.collect { - case f: FileWritingCommandExec => f - }.head - insertCommand.cmd.run(sparkSession, insertCommand.children, metricsCallback) + qe.executedPlan.transform { + case f: FileWritingCommandExec => + val newCmd = f.cmd.withExternalMetrics(metrics) + FileWritingCommandExec(newCmd, f.children) + }.execute() } catch { case NonFatal(e) => // drop the created table. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index c9526fd377bb2..9475736cd0101 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -81,9 +81,7 @@ case class InsertIntoHiveTable( partition: Map[String, Option[String]], query: LogicalPlan, overwrite: Boolean, - ifPartitionNotExists: Boolean, - override val externalMetrics: Option[Map[String, SQLMetric]] = None) - extends FileWritingCommand { + ifPartitionNotExists: Boolean) extends FileWritingCommand { override def children: Seq[LogicalPlan] = query :: Nil @@ -237,6 +235,7 @@ case class InsertIntoHiveTable( override def run( sparkSession: SparkSession, children: Seq[SparkPlan], + metrics: Map[String, SQLMetric], metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { assert(children.length == 1) From 9b87b4f306ff5f46f0795ff40712e71698cddab7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 6 Jun 2017 23:17:31 +0000 Subject: [PATCH 14/22] A bit refactoring to simplify the API. --- .../spark/sql/execution/SparkStrategies.scala | 7 ++ .../command/FileWritingCommand.scala | 65 +++++++------------ .../command/createDataSourceTables.scala | 23 +------ .../execution/datasources/DataSource.scala | 7 +- .../InsertIntoHadoopFsRelationCommand.scala | 5 +- .../CreateHiveTableAsSelectCommand.scala | 18 ++--- .../hive/execution/InsertIntoHiveTable.scala | 7 +- 7 files changed, 53 insertions(+), 79 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 0ef3a95149d47..53220c199f606 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.datasources.{DataSource, FileFormat} import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight} import org.apache.spark.sql.execution.streaming._ @@ -346,6 +347,12 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + // For non `FileFormat` datasource table, don't need to update the writing metrics. + case c @ CreateDataSourceTableAsSelectCommand(table, _, _) + if (!classOf[FileFormat].isAssignableFrom( + DataSource.lookupDataSource(table.provider.get))) => + FileWritingCommandExec(c, Seq.empty, Some(Map.empty)) :: Nil + case f: FileWritingCommand => FileWritingCommandExec(f, f.children.map(planLater)) :: Nil case r: RunnableCommand => ExecutedCommandExec(r) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala index 768309c22827f..6262922516bc1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala @@ -31,20 +31,23 @@ import org.apache.spark.util.Utils * wrapped in `FileWritingCommandExec` during execution. */ trait FileWritingCommand extends logical.Command { + def run( + sparkSession: SparkSession, + children: Seq[SparkPlan], + fileCommandExec: FileWritingCommandExec): Seq[Row] +} - // The caller of `FileWritingCommand` can replace the metrics location by providing this external - // metrics structure. - private var _externalMetrics: Option[Map[String, SQLMetric]] = None - private[sql] def withExternalMetrics(map: Map[String, SQLMetric]): this.type = { - _externalMetrics = Option(map) - this - } +/** + * A physical operator specialized to execute the run method of a `FileWritingCommand`, + * save the result to prevent multiple executions, and record necessary metrics for UI. + */ +case class FileWritingCommandExec( + cmd: FileWritingCommand, + children: Seq[SparkPlan], + givenMetrics: Option[Map[String, SQLMetric]] = None) extends CommandExec { - /** - * Those metrics will be updated once the command finishes writing data out. Those metrics will - * be taken by `FileWritingCommandExec` as its metrics when showing in UI. - */ - def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = _externalMetrics.getOrElse { + override val metrics = givenMetrics.getOrElse { + val sparkContext = sqlContext.sparkContext Map( // General metrics. "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), @@ -58,8 +61,7 @@ trait FileWritingCommand extends logical.Command { /** * Callback function that update metrics collected from the writing operation. */ - private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: Map[String, SQLMetric]) - (writeSummaries: Seq[ExecutedWriteSummary]): Unit = { + private[sql] def postDriverMetrics(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { var numPartitions = 0 var numFiles = 0 var totalNumBytes: Long = 0L @@ -73,36 +75,19 @@ trait FileWritingCommand extends logical.Command { } // The time for writing individual file can be zero if it's less than 1 ms. Zero values can - // lower actual time of writing when calculating average, so excluding them. - val writingTime = + // lower actual time of writing to zero when calculating average, so excluding them. + val avgWritingTime = Utils.average(writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0))).toLong - + // Note: for simplifying metric values assignment, we put the values as the alphabetically + // sorted of the metric keys. val metricsNames = metrics.keys.toSeq.sorted - val metricsValues = Seq(writingTime, numFiles, totalNumBytes, totalNumOutput, numPartitions) + val metricsValues = Seq(avgWritingTime, numFiles, totalNumBytes, totalNumOutput, numPartitions) metricsNames.zip(metricsValues).foreach(x => metrics(x._1).add(x._2)) - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metricsNames.map(metrics(_))) + val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, + metricsNames.map(metrics(_))) } - def run( - sparkSession: SparkSession, - children: Seq[SparkPlan], - metrics: Map[String, SQLMetric], - metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] -} - -/** - * A physical operator specialized to execute the run method of a `FileWritingCommand`, - * save the result to prevent multiple executions, and record necessary metrics for UI. - */ -case class FileWritingCommandExec( - cmd: FileWritingCommand, - children: Seq[SparkPlan]) extends CommandExec { - - override lazy val metrics = cmd.metrics(sqlContext.sparkContext) - - protected[sql] lazy val invokeCommand: Seq[Row] = - cmd.run(sqlContext.sparkSession, children, metrics, - cmd.postDriverMetrics(sqlContext.sparkContext, metrics)) + protected[sql] lazy val invokeCommand: Seq[Row] = cmd.run(sqlContext.sparkSession, children, this) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 939c0abbae367..3d93d6157b1e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -125,29 +125,12 @@ case class CreateDataSourceTableAsSelectCommand( query: LogicalPlan) extends FileWritingCommand { - /** - * The code path this command writes data out depends on the type of data source: - * - * For FileFormat-based data sources, `InsertIntoHadoopFsRelationCommand` is invoked and we - * can update metrics. - * - * For other data sources, `CreatableRelationProvider.createRelation` will be called. We can't - * record metrics for that. So we will return empty metrics map. - */ - override def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = - if (classOf[FileFormat].isAssignableFrom(DataSource.lookupDataSource(table.provider.get))) { - super.metrics(sparkContext) - } else { - Map.empty - } - override def innerChildren: Seq[LogicalPlan] = Seq(query) override def run( sparkSession: SparkSession, children: Seq[SparkPlan], - metrics: Map[String, SQLMetric], - metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { + fileCommandExec: FileWritingCommandExec): Seq[Row] = { assert(table.tableType != CatalogTableType.VIEW) assert(table.provider.isDefined) @@ -170,7 +153,7 @@ case class CreateDataSourceTableAsSelectCommand( saveDataIntoTable( sparkSession, table, table.storage.locationUri, query, SaveMode.Append, tableExists = true, - metrics = metrics) + metrics = fileCommandExec.metrics) } else { assert(table.schema.isEmpty) @@ -181,7 +164,7 @@ case class CreateDataSourceTableAsSelectCommand( } val result = saveDataIntoTable( sparkSession, table, tableLocation, query, SaveMode.Overwrite, tableExists = false, - metrics = metrics) + metrics = fileCommandExec.metrics) val newTable = table.copy( storage = table.storage.copy(locationUri = tableLocation), // We will use the schema of resolved.relation as the schema of the table (instead of diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 3f502d275bee1..6e59bddd6df5b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -452,10 +452,11 @@ case class DataSource( sparkSession.sqlContext, mode, caseInsensitiveOptions, Dataset.ofRows(sparkSession, data)) case format: FileFormat => val qe = sparkSession.sessionState.executePlan(planForWritingFileFormat(format, mode, data)) + // We need to replace the invoked command's metrics with the caller's if any. So we can + // update the correct metrics for showing on UI. qe.executedPlan.transform { - case f: FileWritingCommandExec => - val newCmd = f.cmd.withExternalMetrics(externalMetrics.getOrElse(null)) - FileWritingCommandExec(newCmd, f.children) + case FileWritingCommandExec(cmd, children, None) => + FileWritingCommandExec(cmd, children, externalMetrics) }.execute() // Replace the schema with that of the DataFrame we just wrote out to avoid re-inferring copy(userSpecifiedSchema = Some(data.schema.asNullable)).resolveRelation() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 2cb6c80e45957..03a0e4519ad9e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -61,8 +61,7 @@ case class InsertIntoHadoopFsRelationCommand( override def run( sparkSession: SparkSession, children: Seq[SparkPlan], - metrics: Map[String, SQLMetric], - metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { + fileCommandExec: FileWritingCommandExec): Seq[Row] = { assert(children.length == 1) // Most formats don't do well with duplicate columns, so lets not allow that @@ -134,7 +133,7 @@ case class InsertIntoHadoopFsRelationCommand( .distinct.map(PartitioningUtils.parsePathFragment) // Updating metrics. - metricsCallback(summary) + fileCommandExec.postDriverMetrics(summary) // Updating metastore partition metadata. if (partitionsTrackedByCatalog) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 1d94d93761398..d7293af4a1ee1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -48,8 +48,7 @@ case class CreateHiveTableAsSelectCommand( override def run( sparkSession: SparkSession, children: Seq[SparkPlan], - metrics: Map[String, SQLMetric], - metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { + fileCommandExec: FileWritingCommandExec): Seq[Row] = { if (sparkSession.sessionState.catalog.tableExists(tableIdentifier)) { assert(mode != SaveMode.Overwrite, s"Expect the table $tableIdentifier has been dropped when the save mode is Overwrite") @@ -61,7 +60,6 @@ case class CreateHiveTableAsSelectCommand( // Since the table already exists and the save mode is Ignore, we will just return. return Seq.empty } - val qe = sparkSession.sessionState.executePlan( InsertIntoTable( UnresolvedRelation(tableIdentifier), @@ -69,10 +67,11 @@ case class CreateHiveTableAsSelectCommand( query, overwrite = false, ifPartitionNotExists = false)) + // We need to replace the invoked command's metrics with the caller's. So we can update + // the correct metrics for showing on UI. qe.executedPlan.transform { - case f: FileWritingCommandExec => - val newCmd = f.cmd.withExternalMetrics(metrics) - FileWritingCommandExec(newCmd, f.children) + case FileWritingCommandExec(cmd, children, None) => + FileWritingCommandExec(cmd, children, Some(fileCommandExec.metrics)) }.execute() } else { // TODO ideally, we should get the output data ready first and then @@ -90,10 +89,11 @@ case class CreateHiveTableAsSelectCommand( query, overwrite = true, ifPartitionNotExists = false)) + // We need to replace the invoked command's metrics with the caller's. So we can update + // the correct metrics for showing on UI. qe.executedPlan.transform { - case f: FileWritingCommandExec => - val newCmd = f.cmd.withExternalMetrics(metrics) - FileWritingCommandExec(newCmd, f.children) + case FileWritingCommandExec(cmd, children, None) => + FileWritingCommandExec(cmd, children, Some(fileCommandExec.metrics)) }.execute() } catch { case NonFatal(e) => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 9475736cd0101..9eb4ceda966ae 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.FileWritingCommand +import org.apache.spark.sql.execution.command.{FileWritingCommand, FileWritingCommandExec} import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, FileFormatWriter} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.hive._ @@ -235,8 +235,7 @@ case class InsertIntoHiveTable( override def run( sparkSession: SparkSession, children: Seq[SparkPlan], - metrics: Map[String, SQLMetric], - metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row] = { + fileCommandExec: FileWritingCommandExec): Seq[Row] = { assert(children.length == 1) val sessionState = sparkSession.sessionState @@ -359,7 +358,7 @@ case class InsertIntoHiveTable( hadoopConf = hadoopConf, partitionColumns = partitionAttributes, bucketSpec = None, - refreshFunction = metricsCallback, + refreshFunction = fileCommandExec.postDriverMetrics, options = Map.empty) if (partition.nonEmpty) { From 6e8ffc5b847905dcd703b1614858693c94f5fdae Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 14 Jun 2017 13:20:31 +0000 Subject: [PATCH 15/22] Address comments. --- .../spark/sql/execution/SparkStrategies.scala | 7 --- .../command/FileWritingCommand.scala | 56 ++++++++++++------- .../sql/execution/command/commands.scala | 6 +- .../command/createDataSourceTables.scala | 15 +++-- .../execution/datasources/DataSource.scala | 4 +- .../InsertIntoHadoopFsRelationCommand.scala | 4 +- .../CreateHiveTableAsSelectCommand.scala | 6 +- .../hive/execution/InsertIntoHiveTable.scala | 4 +- 8 files changed, 57 insertions(+), 45 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 53220c199f606..0ef3a95149d47 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.{DataSource, FileFormat} import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight} import org.apache.spark.sql.execution.streaming._ @@ -347,12 +346,6 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - // For non `FileFormat` datasource table, don't need to update the writing metrics. - case c @ CreateDataSourceTableAsSelectCommand(table, _, _) - if (!classOf[FileFormat].isAssignableFrom( - DataSource.lookupDataSource(table.provider.get))) => - FileWritingCommandExec(c, Seq.empty, Some(Map.empty)) :: Nil - case f: FileWritingCommand => FileWritingCommandExec(f, f.children.map(planLater)) :: Nil case r: RunnableCommand => ExecutedCommandExec(r) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala index 6262922516bc1..46b5c8f56e9a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala @@ -31,10 +31,13 @@ import org.apache.spark.util.Utils * wrapped in `FileWritingCommandExec` during execution. */ trait FileWritingCommand extends logical.Command { + // Whether this command will update the wrapping `FileWritingCommandExec`'s metrics. + protected[sql] val canUpdateMetrics: Boolean = true + def run( sparkSession: SparkSession, children: Seq[SparkPlan], - fileCommandExec: FileWritingCommandExec): Seq[Row] + fileCommandExec: Option[FileWritingCommandExec]): Seq[Row] } /** @@ -44,24 +47,32 @@ trait FileWritingCommand extends logical.Command { case class FileWritingCommandExec( cmd: FileWritingCommand, children: Seq[SparkPlan], - givenMetrics: Option[Map[String, SQLMetric]] = None) extends CommandExec { + parentCommandExec: Option[FileWritingCommandExec] = None) extends CommandExec { - override val metrics = givenMetrics.getOrElse { - val sparkContext = sqlContext.sparkContext - Map( - // General metrics. - "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), - "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), - "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") - ) - } + override val metrics: Map[String, SQLMetric] = + if (parentCommandExec.isDefined && cmd.canUpdateMetrics) { + // When this operator is invoked by another `FileWritingCommandExec`, we need to update + // the caller's metrics. + parentCommandExec.get.metrics + } else if (cmd.canUpdateMetrics) { + val sparkContext = sqlContext.sparkContext + Map( + "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), + "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), + "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") + ) + } else { + // The command declares it won't update metrics. Just use an empty map. + Map.empty + } /** * Callback function that update metrics collected from the writing operation. */ private[sql] def postDriverMetrics(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { + assert(cmd.canUpdateMetrics, s"This $cmd shouldn't update metrics") var numPartitions = 0 var numFiles = 0 var totalNumBytes: Long = 0L @@ -78,16 +89,19 @@ case class FileWritingCommandExec( // lower actual time of writing to zero when calculating average, so excluding them. val avgWritingTime = Utils.average(writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0))).toLong - // Note: for simplifying metric values assignment, we put the values as the alphabetically - // sorted of the metric keys. - val metricsNames = metrics.keys.toSeq.sorted - val metricsValues = Seq(avgWritingTime, numFiles, totalNumBytes, totalNumOutput, numPartitions) - metricsNames.zip(metricsValues).foreach(x => metrics(x._1).add(x._2)) + + metrics("avgTime").add(avgWritingTime) + metrics("numFiles").add(numFiles) + metrics("numOutputBytes").add(totalNumBytes) + metrics("numOutputRows").add(totalNumOutput) + metrics("numParts").add(numPartitions) val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, - metricsNames.map(metrics(_))) + SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, metrics.values.toList) } - protected[sql] lazy val invokeCommand: Seq[Row] = cmd.run(sqlContext.sparkSession, children, this) + protected[sql] def invokeCommand(): Seq[Row] = { + val exec = if (cmd.canUpdateMetrics) Some(this) else None + cmd.run(sqlContext.sparkSession, children, exec) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 2de334822eb32..1d62ad547b35f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -56,10 +56,10 @@ trait CommandExec extends SparkPlan { */ protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { val converter = CatalystTypeConverters.createToCatalystConverter(schema) - invokeCommand.map(converter(_).asInstanceOf[InternalRow]) + invokeCommand().map(converter(_).asInstanceOf[InternalRow]) } - protected[sql] val invokeCommand: Seq[Row] + protected[sql] def invokeCommand(): Seq[Row] override def innerChildren: Seq[QueryPlan[_]] = cmd.innerChildren @@ -83,7 +83,7 @@ trait CommandExec extends SparkPlan { * save the result to prevent multiple executions. */ case class ExecutedCommandExec(cmd: RunnableCommand) extends CommandExec { - protected[sql] lazy val invokeCommand: Seq[Row] = cmd.run(sqlContext.sparkSession) + protected[sql] def invokeCommand: Seq[Row] = cmd.run(sqlContext.sparkSession) override def children: Seq[SparkPlan] = Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 3d93d6157b1e1..0262534d19232 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -127,10 +127,15 @@ case class CreateDataSourceTableAsSelectCommand( override def innerChildren: Seq[LogicalPlan] = Seq(query) + // This command updates parent `FileWritingCommandExec`'s metrics only if the table is + // `FileFormat`-based. + override protected[sql] val canUpdateMetrics = classOf[FileFormat].isAssignableFrom( + DataSource.lookupDataSource(table.provider.get)) + override def run( sparkSession: SparkSession, children: Seq[SparkPlan], - fileCommandExec: FileWritingCommandExec): Seq[Row] = { + fileCommandExec: Option[FileWritingCommandExec]): Seq[Row] = { assert(table.tableType != CatalogTableType.VIEW) assert(table.provider.isDefined) @@ -153,7 +158,7 @@ case class CreateDataSourceTableAsSelectCommand( saveDataIntoTable( sparkSession, table, table.storage.locationUri, query, SaveMode.Append, tableExists = true, - metrics = fileCommandExec.metrics) + fileCommandExec = fileCommandExec) } else { assert(table.schema.isEmpty) @@ -164,7 +169,7 @@ case class CreateDataSourceTableAsSelectCommand( } val result = saveDataIntoTable( sparkSession, table, tableLocation, query, SaveMode.Overwrite, tableExists = false, - metrics = fileCommandExec.metrics) + fileCommandExec = fileCommandExec) val newTable = table.copy( storage = table.storage.copy(locationUri = tableLocation), // We will use the schema of resolved.relation as the schema of the table (instead of @@ -192,7 +197,7 @@ case class CreateDataSourceTableAsSelectCommand( data: LogicalPlan, mode: SaveMode, tableExists: Boolean, - metrics: Map[String, SQLMetric]): BaseRelation = { + fileCommandExec: Option[FileWritingCommandExec]): BaseRelation = { // Create the relation based on the input logical plan: `data`. val pathOption = tableLocation.map("path" -> CatalogUtils.URIToString(_)) val dataSource = DataSource( @@ -204,7 +209,7 @@ case class CreateDataSourceTableAsSelectCommand( catalogTable = if (tableExists) Some(table) else None) try { - dataSource.writeAndRead(mode, query, Some(metrics)) + dataSource.writeAndRead(mode, query, fileCommandExec) } catch { case ex: AnalysisException => logError(s"Failed to write to table ${table.identifier.unquotedString}", ex) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 6e59bddd6df5b..6ace32e9a66a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -441,7 +441,7 @@ case class DataSource( def writeAndRead( mode: SaveMode, data: LogicalPlan, - externalMetrics: Option[Map[String, SQLMetric]] = None): BaseRelation = { + fileCommandExec: Option[FileWritingCommandExec] = None): BaseRelation = { if (data.schema.map(_.dataType).exists(_.isInstanceOf[CalendarIntervalType])) { throw new AnalysisException("Cannot save interval data type into external storage.") } @@ -456,7 +456,7 @@ case class DataSource( // update the correct metrics for showing on UI. qe.executedPlan.transform { case FileWritingCommandExec(cmd, children, None) => - FileWritingCommandExec(cmd, children, externalMetrics) + FileWritingCommandExec(cmd, children, fileCommandExec) }.execute() // Replace the schema with that of the DataFrame we just wrote out to avoid re-inferring copy(userSpecifiedSchema = Some(data.schema.asNullable)).resolveRelation() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 03a0e4519ad9e..b3816bb354f7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -61,7 +61,7 @@ case class InsertIntoHadoopFsRelationCommand( override def run( sparkSession: SparkSession, children: Seq[SparkPlan], - fileCommandExec: FileWritingCommandExec): Seq[Row] = { + fileCommandExec: Option[FileWritingCommandExec]): Seq[Row] = { assert(children.length == 1) // Most formats don't do well with duplicate columns, so lets not allow that @@ -133,7 +133,7 @@ case class InsertIntoHadoopFsRelationCommand( .distinct.map(PartitioningUtils.parsePathFragment) // Updating metrics. - fileCommandExec.postDriverMetrics(summary) + fileCommandExec.get.postDriverMetrics(summary) // Updating metastore partition metadata. if (partitionsTrackedByCatalog) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index d7293af4a1ee1..1b9e2778a7b85 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -48,7 +48,7 @@ case class CreateHiveTableAsSelectCommand( override def run( sparkSession: SparkSession, children: Seq[SparkPlan], - fileCommandExec: FileWritingCommandExec): Seq[Row] = { + fileCommandExec: Option[FileWritingCommandExec]): Seq[Row] = { if (sparkSession.sessionState.catalog.tableExists(tableIdentifier)) { assert(mode != SaveMode.Overwrite, s"Expect the table $tableIdentifier has been dropped when the save mode is Overwrite") @@ -71,7 +71,7 @@ case class CreateHiveTableAsSelectCommand( // the correct metrics for showing on UI. qe.executedPlan.transform { case FileWritingCommandExec(cmd, children, None) => - FileWritingCommandExec(cmd, children, Some(fileCommandExec.metrics)) + FileWritingCommandExec(cmd, children, fileCommandExec) }.execute() } else { // TODO ideally, we should get the output data ready first and then @@ -93,7 +93,7 @@ case class CreateHiveTableAsSelectCommand( // the correct metrics for showing on UI. qe.executedPlan.transform { case FileWritingCommandExec(cmd, children, None) => - FileWritingCommandExec(cmd, children, Some(fileCommandExec.metrics)) + FileWritingCommandExec(cmd, children, fileCommandExec) }.execute() } catch { case NonFatal(e) => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 9eb4ceda966ae..6fbafd5e42344 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -235,7 +235,7 @@ case class InsertIntoHiveTable( override def run( sparkSession: SparkSession, children: Seq[SparkPlan], - fileCommandExec: FileWritingCommandExec): Seq[Row] = { + fileCommandExec: Option[FileWritingCommandExec]): Seq[Row] = { assert(children.length == 1) val sessionState = sparkSession.sessionState @@ -358,7 +358,7 @@ case class InsertIntoHiveTable( hadoopConf = hadoopConf, partitionColumns = partitionAttributes, bucketSpec = None, - refreshFunction = fileCommandExec.postDriverMetrics, + refreshFunction = fileCommandExec.get.postDriverMetrics, options = Map.empty) if (partition.nonEmpty) { From 9d8d77f2390e19432881f10c099fbb3cddd6476d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 28 Jun 2017 07:23:19 +0000 Subject: [PATCH 16/22] Revert change for CTAS operations. --- .../spark/sql/execution/QueryExecution.scala | 4 +- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../command/FileWritingCommand.scala | 107 ------------------ .../sql/execution/command/commands.scala | 85 +++++++++++--- .../command/createDataSourceTables.scala | 26 +---- .../execution/datasources/DataSource.scala | 15 +-- .../InsertIntoHadoopFsRelationCommand.scala | 19 +++- .../CreateHiveTableAsSelectCommand.scala | 34 ++---- .../hive/execution/InsertIntoHiveTable.scala | 20 +++- .../sql/hive/execution/SQLMetricsSuite.scala | 9 +- 10 files changed, 121 insertions(+), 202 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 02421418c7b5a..1ba9a79446aad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -117,7 +117,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { * `SparkSQLDriver` for CLI applications. */ def hiveResultString(): Seq[String] = executedPlan match { - case ExecutedCommandExec(desc: DescribeTableCommand) => + case ExecutedCommandExec(desc: DescribeTableCommand, _) => // If it is a describe command for a Hive table, we want to have the output format // be similar with Hive. desc.run(sparkSession).map { @@ -128,7 +128,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { .mkString("\t") } // SHOW TABLES in Hive only output table names, while ours output database, table name, isTemp. - case command @ ExecutedCommandExec(s: ShowTablesCommand) if !s.isExtended => + case command @ ExecutedCommandExec(s: ShowTablesCommand, _) if !s.isExtended => command.executeCollect().map(_.getString(1)) case other => val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 0ef3a95149d47..f13294c925e36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -346,9 +346,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case f: FileWritingCommand => FileWritingCommandExec(f, f.children.map(planLater)) :: Nil - - case r: RunnableCommand => ExecutedCommandExec(r) :: Nil + case r: RunnableCommand => ExecutedCommandExec(r, r.children.map(planLater)) :: Nil case MemoryPlan(sink, output) => val encoder = RowEncoder(sink.schema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala deleted file mode 100644 index 46b5c8f56e9a9..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.command - -import org.apache.spark.SparkContext -import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} -import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.util.Utils - -/** - * A logical command specialized for writing data out. `FileWritingCommand`s are - * wrapped in `FileWritingCommandExec` during execution. - */ -trait FileWritingCommand extends logical.Command { - // Whether this command will update the wrapping `FileWritingCommandExec`'s metrics. - protected[sql] val canUpdateMetrics: Boolean = true - - def run( - sparkSession: SparkSession, - children: Seq[SparkPlan], - fileCommandExec: Option[FileWritingCommandExec]): Seq[Row] -} - -/** - * A physical operator specialized to execute the run method of a `FileWritingCommand`, - * save the result to prevent multiple executions, and record necessary metrics for UI. - */ -case class FileWritingCommandExec( - cmd: FileWritingCommand, - children: Seq[SparkPlan], - parentCommandExec: Option[FileWritingCommandExec] = None) extends CommandExec { - - override val metrics: Map[String, SQLMetric] = - if (parentCommandExec.isDefined && cmd.canUpdateMetrics) { - // When this operator is invoked by another `FileWritingCommandExec`, we need to update - // the caller's metrics. - parentCommandExec.get.metrics - } else if (cmd.canUpdateMetrics) { - val sparkContext = sqlContext.sparkContext - Map( - "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), - "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), - "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") - ) - } else { - // The command declares it won't update metrics. Just use an empty map. - Map.empty - } - - /** - * Callback function that update metrics collected from the writing operation. - */ - private[sql] def postDriverMetrics(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { - assert(cmd.canUpdateMetrics, s"This $cmd shouldn't update metrics") - var numPartitions = 0 - var numFiles = 0 - var totalNumBytes: Long = 0L - var totalNumOutput: Long = 0L - - writeSummaries.foreach { summary => - numPartitions += summary.updatedPartitions.size - numFiles += summary.numOutputFile - totalNumBytes += summary.numOutputBytes - totalNumOutput += summary.numOutputRows - } - - // The time for writing individual file can be zero if it's less than 1 ms. Zero values can - // lower actual time of writing to zero when calculating average, so excluding them. - val avgWritingTime = - Utils.average(writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0))).toLong - - metrics("avgTime").add(avgWritingTime) - metrics("numFiles").add(numFiles) - metrics("numOutputBytes").add(totalNumBytes) - metrics("numOutputRows").add(totalNumOutput) - metrics("numParts").add(numPartitions) - - val executionId = sqlContext.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sqlContext.sparkContext, executionId, metrics.values.toList) - } - - protected[sql] def invokeCommand(): Seq[Row] = { - val exec = if (cmd.canUpdateMetrics) Some(this) else None - cmd.run(sqlContext.sparkSession, children, exec) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 1d62ad547b35f..7495486dbe9da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.command +import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} @@ -24,26 +25,82 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary import org.apache.spark.sql.execution.debug._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils /** * A logical command that is executed for its side-effects. `RunnableCommand`s are * wrapped in `ExecutedCommand` during execution. */ trait RunnableCommand extends logical.Command { - def run(sparkSession: SparkSession): Seq[Row] + + def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = Map.empty + + /** + * Callback function that update metrics collected from the writing operation. + */ + private[sql] def prepareMetricsUpdater( + metrics: Map[String, SQLMetric], + sparkContext: SparkContext): (Seq[ExecutedWriteSummary] => Unit) = { + (writeSummaries) => { + var numPartitions = 0 + var numFiles = 0 + var totalNumBytes: Long = 0L + var totalNumOutput: Long = 0L + + writeSummaries.foreach { summary => + numPartitions += summary.updatedPartitions.size + numFiles += summary.numOutputFile + totalNumBytes += summary.numOutputBytes + totalNumOutput += summary.numOutputRows + } + + // The time for writing individual file can be zero if it's less than 1 ms. Zero values can + // lower actual time of writing to zero when calculating average, so excluding them. + val avgWritingTime = + Utils.average(writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0))).toLong + + metrics("avgTime").add(avgWritingTime) + metrics("numFiles").add(numFiles) + metrics("numOutputBytes").add(totalNumBytes) + metrics("numOutputRows").add(totalNumOutput) + metrics("numParts").add(numPartitions) + + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toList) + } + } + + def run( + sparkSession: SparkSession, + children: Seq[SparkPlan], + metricsUpdater: (Seq[ExecutedWriteSummary] => Unit)): Seq[Row] = { + throw new NotImplementedError + } + + def run(sparkSession: SparkSession): Seq[Row] = { + throw new NotImplementedError + } } /** - * A physical operator that executes the run method of a `logical.Command` and + * A physical operator that executes the run method of a `RunnableCommand` and * saves the result to prevent multiple executions. + * + * @param cmd the `RunnableCommand` this operator will run. + * @param children the children physical plans ran by the `RunnableCommand`. */ -trait CommandExec extends SparkPlan { - val cmd: logical.Command +case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) extends SparkPlan { + + override lazy val metrics: Map[String, SQLMetric] = cmd.metrics(sparkContext) + + private lazy val metricsUpdater = cmd.prepareMetricsUpdater(metrics, sparkContext) /** * A concrete command should override this lazy field to wrap up any side effects caused by the @@ -56,11 +113,14 @@ trait CommandExec extends SparkPlan { */ protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { val converter = CatalystTypeConverters.createToCatalystConverter(schema) - invokeCommand().map(converter(_).asInstanceOf[InternalRow]) + val rows = if (children.isEmpty) { + cmd.run(sqlContext.sparkSession) + } else { + cmd.run(sqlContext.sparkSession, children, metricsUpdater) + } + rows.map(converter(_).asInstanceOf[InternalRow]) } - protected[sql] def invokeCommand(): Seq[Row] - override def innerChildren: Seq[QueryPlan[_]] = cmd.innerChildren override def output: Seq[Attribute] = cmd.output @@ -78,15 +138,6 @@ trait CommandExec extends SparkPlan { } } -/** - * A physical operator specialized to execute the run method of a `RunnableCommand` and - * save the result to prevent multiple executions. - */ -case class ExecutedCommandExec(cmd: RunnableCommand) extends CommandExec { - protected[sql] def invokeCommand: Seq[Row] = cmd.run(sqlContext.sparkSession) - override def children: Seq[SparkPlan] = Nil -} - /** * An explain command for users to see how a command will be executed. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 0262534d19232..729bd39d821c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -21,13 +21,10 @@ import java.net.URI import org.apache.hadoop.fs.Path -import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.sources.BaseRelation /** @@ -123,19 +120,11 @@ case class CreateDataSourceTableAsSelectCommand( table: CatalogTable, mode: SaveMode, query: LogicalPlan) - extends FileWritingCommand { + extends RunnableCommand { override def innerChildren: Seq[LogicalPlan] = Seq(query) - // This command updates parent `FileWritingCommandExec`'s metrics only if the table is - // `FileFormat`-based. - override protected[sql] val canUpdateMetrics = classOf[FileFormat].isAssignableFrom( - DataSource.lookupDataSource(table.provider.get)) - - override def run( - sparkSession: SparkSession, - children: Seq[SparkPlan], - fileCommandExec: Option[FileWritingCommandExec]): Seq[Row] = { + override def run(sparkSession: SparkSession): Seq[Row] = { assert(table.tableType != CatalogTableType.VIEW) assert(table.provider.isDefined) @@ -157,8 +146,7 @@ case class CreateDataSourceTableAsSelectCommand( } saveDataIntoTable( - sparkSession, table, table.storage.locationUri, query, SaveMode.Append, tableExists = true, - fileCommandExec = fileCommandExec) + sparkSession, table, table.storage.locationUri, query, SaveMode.Append, tableExists = true) } else { assert(table.schema.isEmpty) @@ -168,8 +156,7 @@ case class CreateDataSourceTableAsSelectCommand( table.storage.locationUri } val result = saveDataIntoTable( - sparkSession, table, tableLocation, query, SaveMode.Overwrite, tableExists = false, - fileCommandExec = fileCommandExec) + sparkSession, table, tableLocation, query, SaveMode.Overwrite, tableExists = false) val newTable = table.copy( storage = table.storage.copy(locationUri = tableLocation), // We will use the schema of resolved.relation as the schema of the table (instead of @@ -196,8 +183,7 @@ case class CreateDataSourceTableAsSelectCommand( tableLocation: Option[URI], data: LogicalPlan, mode: SaveMode, - tableExists: Boolean, - fileCommandExec: Option[FileWritingCommandExec]): BaseRelation = { + tableExists: Boolean): BaseRelation = { // Create the relation based on the input logical plan: `data`. val pathOption = tableLocation.map("path" -> CatalogUtils.URIToString(_)) val dataSource = DataSource( @@ -209,7 +195,7 @@ case class CreateDataSourceTableAsSelectCommand( catalogTable = if (tableExists) Some(table) else None) try { - dataSource.writeAndRead(mode, query, fileCommandExec) + dataSource.writeAndRead(mode, query) } catch { case ex: AnalysisException => logError(s"Failed to write to table ${table.identifier.unquotedString}", ex) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 6ace32e9a66a6..958715eefa0a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -32,12 +32,10 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.execution.command.FileWritingCommandExec import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.streaming.OutputMode @@ -438,10 +436,7 @@ case class DataSource( * Writes the given [[LogicalPlan]] out to this [[DataSource]] and returns a [[BaseRelation]] for * the following reading. */ - def writeAndRead( - mode: SaveMode, - data: LogicalPlan, - fileCommandExec: Option[FileWritingCommandExec] = None): BaseRelation = { + def writeAndRead(mode: SaveMode, data: LogicalPlan): BaseRelation = { if (data.schema.map(_.dataType).exists(_.isInstanceOf[CalendarIntervalType])) { throw new AnalysisException("Cannot save interval data type into external storage.") } @@ -451,13 +446,7 @@ case class DataSource( dataSource.createRelation( sparkSession.sqlContext, mode, caseInsensitiveOptions, Dataset.ofRows(sparkSession, data)) case format: FileFormat => - val qe = sparkSession.sessionState.executePlan(planForWritingFileFormat(format, mode, data)) - // We need to replace the invoked command's metrics with the caller's if any. So we can - // update the correct metrics for showing on UI. - qe.executedPlan.transform { - case FileWritingCommandExec(cmd, children, None) => - FileWritingCommandExec(cmd, children, fileCommandExec) - }.execute() + sparkSession.sessionState.executePlan(planForWritingFileFormat(format, mode, data)).toRdd // Replace the schema with that of the DataFrame we just wrote out to avoid re-inferring copy(userSpecifiedSchema = Some(data.schema.asNullable)).resolveRelation() case _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index b3816bb354f7e..c9681eb26bf4b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -21,6 +21,7 @@ import java.io.IOException import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.SparkContext import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTablePartition} @@ -29,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} /** * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. @@ -53,15 +54,25 @@ case class InsertIntoHadoopFsRelationCommand( query: LogicalPlan, mode: SaveMode, catalogTable: Option[CatalogTable], - fileIndex: Option[FileIndex]) extends FileWritingCommand { + fileIndex: Option[FileIndex]) + extends RunnableCommand { import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName override def children: Seq[LogicalPlan] = query :: Nil + override def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = + Map( + "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), + "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), + "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") + ) + override def run( sparkSession: SparkSession, children: Seq[SparkPlan], - fileCommandExec: Option[FileWritingCommandExec]): Seq[Row] = { + metricsUpdater: (Seq[ExecutedWriteSummary] => Unit)): Seq[Row] = { assert(children.length == 1) // Most formats don't do well with duplicate columns, so lets not allow that @@ -133,7 +144,7 @@ case class InsertIntoHadoopFsRelationCommand( .distinct.map(PartitioningUtils.parsePathFragment) // Updating metrics. - fileCommandExec.get.postDriverMetrics(summary) + metricsUpdater(summary) // Updating metastore partition metadata. if (partitionsTrackedByCatalog) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 1b9e2778a7b85..65e8b4e3c725c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -23,10 +23,8 @@ import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.{FileWritingCommand, FileWritingCommandExec} -import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary -import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.command.RunnableCommand + /** * Create table and insert the query result into it. @@ -39,16 +37,13 @@ case class CreateHiveTableAsSelectCommand( tableDesc: CatalogTable, query: LogicalPlan, mode: SaveMode) - extends FileWritingCommand { + extends RunnableCommand { private val tableIdentifier = tableDesc.identifier override def innerChildren: Seq[LogicalPlan] = Seq(query) - override def run( - sparkSession: SparkSession, - children: Seq[SparkPlan], - fileCommandExec: Option[FileWritingCommandExec]): Seq[Row] = { + override def run(sparkSession: SparkSession): Seq[Row] = { if (sparkSession.sessionState.catalog.tableExists(tableIdentifier)) { assert(mode != SaveMode.Overwrite, s"Expect the table $tableIdentifier has been dropped when the save mode is Overwrite") @@ -60,19 +55,14 @@ case class CreateHiveTableAsSelectCommand( // Since the table already exists and the save mode is Ignore, we will just return. return Seq.empty } - val qe = sparkSession.sessionState.executePlan( + + sparkSession.sessionState.executePlan( InsertIntoTable( UnresolvedRelation(tableIdentifier), Map(), query, overwrite = false, - ifPartitionNotExists = false)) - // We need to replace the invoked command's metrics with the caller's. So we can update - // the correct metrics for showing on UI. - qe.executedPlan.transform { - case FileWritingCommandExec(cmd, children, None) => - FileWritingCommandExec(cmd, children, fileCommandExec) - }.execute() + ifPartitionNotExists = false)).toRdd } else { // TODO ideally, we should get the output data ready first and then // add the relation into catalog, just in case of failure occurs while data @@ -82,19 +72,13 @@ case class CreateHiveTableAsSelectCommand( tableDesc.copy(schema = query.schema), ignoreIfExists = false) try { - val qe = sparkSession.sessionState.executePlan( + sparkSession.sessionState.executePlan( InsertIntoTable( UnresolvedRelation(tableIdentifier), Map(), query, overwrite = true, - ifPartitionNotExists = false)) - // We need to replace the invoked command's metrics with the caller's. So we can update - // the correct metrics for showing on UI. - qe.executedPlan.transform { - case FileWritingCommandExec(cmd, children, None) => - FileWritingCommandExec(cmd, children, fileCommandExec) - }.execute() + ifPartitionNotExists = false)).toRdd } catch { case NonFatal(e) => // drop the created table. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 6fbafd5e42344..422f88968c9a3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -31,15 +31,16 @@ import org.apache.hadoop.hive.ql.exec.TaskRunner import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.spark.SparkContext import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.{FileWritingCommand, FileWritingCommandExec} +import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, FileFormatWriter} -import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.sql.hive.client.{HiveClientImpl, HiveVersion} @@ -81,10 +82,19 @@ case class InsertIntoHiveTable( partition: Map[String, Option[String]], query: LogicalPlan, overwrite: Boolean, - ifPartitionNotExists: Boolean) extends FileWritingCommand { + ifPartitionNotExists: Boolean) extends RunnableCommand { override def children: Seq[LogicalPlan] = query :: Nil + override def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = + Map( + "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), + "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), + "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") + ) + var createdTempDir: Option[Path] = None private def executionId: String = { @@ -235,7 +245,7 @@ case class InsertIntoHiveTable( override def run( sparkSession: SparkSession, children: Seq[SparkPlan], - fileCommandExec: Option[FileWritingCommandExec]): Seq[Row] = { + metricsUpdater: (Seq[ExecutedWriteSummary] => Unit)): Seq[Row] = { assert(children.length == 1) val sessionState = sparkSession.sessionState @@ -358,7 +368,7 @@ case class InsertIntoHiveTable( hadoopConf = hadoopConf, partitionColumns = partitionAttributes, bucketSpec = None, - refreshFunction = fileCommandExec.get.postDriverMetrics, + refreshFunction = metricsUpdater, options = Map.empty) if (partition.nonEmpty) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala index 0c900ec1ffd22..1ef1988d4c605 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -74,14 +74,11 @@ class SQLMetricsSuite extends SQLTestUtils with TestHiveSingleton { dataFormat: String, tableName: String): Unit = { withTable(tableName) { - // 1 file, 1 row, 0 dynamic partition. - verifyWriteDataMetrics(Seq(1, 0, 1)) { - Seq((1, 2)).toDF("i", "j") - .write.format(dataFormat).mode("overwrite").saveAsTable(tableName) - } + Seq((1, 2)).toDF("i", "j") + .write.format(dataFormat).mode("overwrite").saveAsTable(tableName) + val tableLocation = new File(spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location) - assert(Utils.recursiveList(tableLocation).count(_.getName.startsWith("part-")) == 1) // 2 files, 100 rows, 0 dynamic partition. verifyWriteDataMetrics(Seq(2, 0, 100)) { From 32f4abf64263a16ef9c0cefd55990d30cd93ce38 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 30 Jun 2017 07:35:42 +0000 Subject: [PATCH 17/22] Remove useless import. --- .../apache/spark/sql/hive/execution/InsertIntoHiveTable.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 5b3a88adc61d5..10d0d3125db2a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.{CommandUtils, RunnableCommand} -import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, FileFormatWriter} +import org.apache.spark.sql.execution.datasources.FileFormatWriter import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} From 6ea6bbb819a2ed90f86bde7415a8fada809ce4c9 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 1 Jul 2017 08:45:56 +0000 Subject: [PATCH 18/22] Create new trait for classes that can update metrics. --- .../sql/execution/command/commands.scala | 26 ++++++++++++------- .../InsertIntoHadoopFsRelationCommand.scala | 4 +-- .../hive/execution/InsertIntoHiveTable.scala | 6 ++--- 3 files changed, 22 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 25dfa6c3a2134..8ae621e7d6a3c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -44,7 +44,23 @@ trait RunnableCommand extends logical.Command { // The map used to record the metrics of running the command. This will be passed to // `ExecutedCommand` during query planning. - private[sql] lazy val metrics: Map[String, SQLMetric] = Map.empty + lazy val metrics: Map[String, SQLMetric] = Map.empty + + def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { + throw new NotImplementedError + } + + def run(sparkSession: SparkSession): Seq[Row] = { + throw new NotImplementedError + } +} + +/** + * A trait for classes that can update its metrics of data writing operation. + */ +trait MetricUpdater { + + val metrics: Map[String, SQLMetric] /** * Callback function that update metrics collected from the writing operation. @@ -77,14 +93,6 @@ trait RunnableCommand extends logical.Command { val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toList) } - - def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { - throw new NotImplementedError - } - - def run(sparkSession: SparkSession): Seq[Row] = { - throw new NotImplementedError - } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 856c29899ec6b..c16d4f65305f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -55,12 +55,12 @@ case class InsertIntoHadoopFsRelationCommand( mode: SaveMode, catalogTable: Option[CatalogTable], fileIndex: Option[FileIndex]) - extends RunnableCommand { + extends RunnableCommand with MetricUpdater { import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName override def children: Seq[LogicalPlan] = query :: Nil - override private[sql] lazy val metrics: Map[String, SQLMetric] = { + override lazy val metrics: Map[String, SQLMetric] = { val sparkContext = SparkContext.getActive.get Map( "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 10d0d3125db2a..e66fc8454f949 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.{CommandUtils, RunnableCommand} +import org.apache.spark.sql.execution.command.{CommandUtils, MetricUpdater, RunnableCommand} import org.apache.spark.sql.execution.datasources.FileFormatWriter import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.hive._ @@ -82,11 +82,11 @@ case class InsertIntoHiveTable( partition: Map[String, Option[String]], query: LogicalPlan, overwrite: Boolean, - ifPartitionNotExists: Boolean) extends RunnableCommand { + ifPartitionNotExists: Boolean) extends RunnableCommand with MetricUpdater { override def children: Seq[LogicalPlan] = query :: Nil - override private[sql] lazy val metrics: Map[String, SQLMetric] = { + override lazy val metrics: Map[String, SQLMetric] = { val sparkContext = SparkContext.getActive.get Map( "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), From f575f0c999647d1f7fda22652ea480f7a0d07047 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 3 Jul 2017 06:40:09 +0000 Subject: [PATCH 19/22] Address comments. --- .../spark/sql/execution/command/commands.scala | 17 +++++++++++++---- .../InsertIntoHadoopFsRelationCommand.scala | 15 ++------------- .../hive/execution/InsertIntoHiveTable.scala | 17 +++-------------- 3 files changed, 18 insertions(+), 31 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 8ae621e7d6a3c..752c7e1951f74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -56,16 +56,25 @@ trait RunnableCommand extends logical.Command { } /** - * A trait for classes that can update its metrics of data writing operation. + * A special `RunnableCommand` which writes data out and updates metrics. */ -trait MetricUpdater { +trait DataWritingCommand extends RunnableCommand { - val metrics: Map[String, SQLMetric] + override lazy val metrics: Map[String, SQLMetric] = { + val sparkContext = SparkContext.getActive.get + Map( + "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), + "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), + "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") + ) + } /** * Callback function that update metrics collected from the writing operation. */ - protected def callbackMetricsUpdater(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { + protected def updateWritingMetrics(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { val sparkContext = SparkContext.getActive.get var numPartitions = 0 var numFiles = 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index c16d4f65305f7..0031567d3d288 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -55,22 +55,11 @@ case class InsertIntoHadoopFsRelationCommand( mode: SaveMode, catalogTable: Option[CatalogTable], fileIndex: Option[FileIndex]) - extends RunnableCommand with MetricUpdater { + extends DataWritingCommand { import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName override def children: Seq[LogicalPlan] = query :: Nil - override lazy val metrics: Map[String, SQLMetric] = { - val sparkContext = SparkContext.getActive.get - Map( - "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), - "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), - "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") - ) - } - override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { assert(children.length == 1) @@ -143,7 +132,7 @@ case class InsertIntoHadoopFsRelationCommand( .distinct.map(PartitioningUtils.parsePathFragment) // Updating metrics. - callbackMetricsUpdater(summary) + updateWritingMetrics(summary) // Updating metastore partition metadata. if (partitionsTrackedByCatalog) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index e66fc8454f949..cd263e8b6df8e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.{CommandUtils, MetricUpdater, RunnableCommand} +import org.apache.spark.sql.execution.command.{CommandUtils, DataWritingCommand} import org.apache.spark.sql.execution.datasources.FileFormatWriter import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.hive._ @@ -82,21 +82,10 @@ case class InsertIntoHiveTable( partition: Map[String, Option[String]], query: LogicalPlan, overwrite: Boolean, - ifPartitionNotExists: Boolean) extends RunnableCommand with MetricUpdater { + ifPartitionNotExists: Boolean) extends DataWritingCommand { override def children: Seq[LogicalPlan] = query :: Nil - override lazy val metrics: Map[String, SQLMetric] = { - val sparkContext = SparkContext.getActive.get - Map( - "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), - "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), - "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") - ) - } - var createdTempDir: Option[Path] = None private def executionId: String = { @@ -367,7 +356,7 @@ case class InsertIntoHiveTable( hadoopConf = hadoopConf, partitionColumns = partitionAttributes, bucketSpec = None, - refreshFunction = callbackMetricsUpdater, + refreshFunction = updateWritingMetrics, options = Map.empty) if (partition.nonEmpty) { From 8380023ab96fa6febe19918da9284fe14aea1f53 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 3 Jul 2017 07:19:57 +0000 Subject: [PATCH 20/22] Don't send back writing time per file. --- .../sql/execution/command/commands.scala | 12 +++++++-- .../datasources/FileFormatWriter.scala | 27 ++++++++++++++----- 2 files changed, 30 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 752c7e1951f74..9a5f514ccf50b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -80,18 +80,26 @@ trait DataWritingCommand extends RunnableCommand { var numFiles = 0 var totalNumBytes: Long = 0L var totalNumOutput: Long = 0L + var totalWritingTime: Long = 0L + var numFilesNonZeroWritingTime = 0 writeSummaries.foreach { summary => numPartitions += summary.updatedPartitions.size numFiles += summary.numOutputFile totalNumBytes += summary.numOutputBytes totalNumOutput += summary.numOutputRows + totalWritingTime += summary.totalWritingTime + numFilesNonZeroWritingTime += summary.numFilesWithNonZeroWritingTime } + // We only count non-zero writing time when averaging total writing time. // The time for writing individual file can be zero if it's less than 1 ms. Zero values can // lower actual time of writing to zero when calculating average, so excluding them. - val avgWritingTime = - Utils.average(writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0))).toLong + val avgWritingTime = if (numFilesNonZeroWritingTime > 0) { + (totalWritingTime / numFilesNonZeroWritingTime).toLong + } else { + 0L + } metrics("avgTime").add(avgWritingTime) metrics("numFiles").add(numFiles) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index b0c234923ea08..de4b3f833f0d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -275,7 +275,8 @@ object FileFormatWriter extends Logging { /** * The data structures used to measure metrics during writing. */ - protected val writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty + protected var totalWritingTime: Long = 0L + protected var numFilesWithNonZeroWritingTime = 0 protected var timeOnCurrentFile: Long = 0L protected var numOutputRows: Long = 0L protected var numOutputBytes: Long = 0L @@ -356,7 +357,8 @@ object FileFormatWriter extends Logging { numOutputFile = fileCounter + 1, numOutputBytes = numOutputBytes, numOutputRows = numOutputRows, - writingTimePerFile = writingTimePerFile) + numFilesWithNonZeroWritingTime = numFilesWithNonZeroWritingTime, + totalWritingTime = totalWritingTime) } override def releaseResources(): Unit = { @@ -364,7 +366,11 @@ object FileFormatWriter extends Logging { try { val startTime = System.nanoTime() currentWriter.close() - writingTimePerFile += (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 + val writingTime = (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 + if (writingTime > 0) { + numFilesWithNonZeroWritingTime += 1 + totalWritingTime += writingTime + } timeOnCurrentFile = 0 numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } finally { @@ -520,7 +526,8 @@ object FileFormatWriter extends Logging { numOutputFile = totalFileCounter, numOutputBytes = numOutputBytes, numOutputRows = numOutputRows, - writingTimePerFile = writingTimePerFile) + numFilesWithNonZeroWritingTime = numFilesWithNonZeroWritingTime, + totalWritingTime = totalWritingTime) } override def releaseResources(): Unit = { @@ -528,7 +535,11 @@ object FileFormatWriter extends Logging { try { val startTime = System.nanoTime() currentWriter.close() - writingTimePerFile += (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 + val writingTime = (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 + if (writingTime > 0) { + numFilesWithNonZeroWritingTime += 1 + totalWritingTime += writingTime + } timeOnCurrentFile = 0 numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } finally { @@ -547,11 +558,13 @@ object FileFormatWriter extends Logging { * @param numOutputFile the total number of files. * @param numOutputRows the number of output rows. * @param numOutputBytes the bytes of output data. - * @param writingTimePerFile the writing time in ms per file. + * @param numFilesWithNonZeroWritingTime the number of files with non zero writing time. + * @param totalWritingTime the total writing time in ms. */ case class ExecutedWriteSummary( updatedPartitions: Set[String], numOutputFile: Int, numOutputRows: Long, numOutputBytes: Long, - writingTimePerFile: Seq[Long]) + numFilesWithNonZeroWritingTime: Int, + totalWritingTime: Long) From dfaa4f11f35e2615ce627f29b1a60ab27dfd5477 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 3 Jul 2017 07:21:28 +0000 Subject: [PATCH 21/22] Remove unused method. --- .../main/scala/org/apache/spark/util/Utils.scala | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 28f87bb6633e0..58884a1c777b9 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2668,21 +2668,6 @@ private[spark] object Utils extends Logging { redact(redactionPattern, kvs.toArray) } - /** - * Computes the average of all elements in an `Iterable`. If there is no element, returns 0. - */ - def average[T](ts: Iterable[T])(implicit num: Numeric[T]): Double = { - if (ts.isEmpty) { - 0.0 - } else { - var count = 0 - val sum = ts.reduce { (sum, ele) => - count += 1 - num.plus(sum, ele) - } - num.toDouble(sum) / (count + 1) - } - } } private[util] object CallerContext extends Logging { From 04e79d9255f923b9e5a740d20e110b62280310f6 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 4 Jul 2017 08:42:55 +0000 Subject: [PATCH 22/22] Address comments. --- .../command/DataWritingCommand.scala | 75 +++++++++++++++++++ .../sql/execution/command/commands.scala | 64 +--------------- .../datasources/FileFormatWriter.scala | 17 +---- 3 files changed, 79 insertions(+), 77 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala new file mode 100644 index 0000000000000..0c381a2c02986 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala @@ -0,0 +1,75 @@ +/* + * 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.command + +import org.apache.spark.SparkContext +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * A special `RunnableCommand` which writes data out and updates metrics. + */ +trait DataWritingCommand extends RunnableCommand { + + override lazy val metrics: Map[String, SQLMetric] = { + val sparkContext = SparkContext.getActive.get + Map( + "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), + "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), + "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") + ) + } + + /** + * Callback function that update metrics collected from the writing operation. + */ + protected def updateWritingMetrics(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { + val sparkContext = SparkContext.getActive.get + var numPartitions = 0 + var numFiles = 0 + var totalNumBytes: Long = 0L + var totalNumOutput: Long = 0L + var totalWritingTime: Long = 0L + + writeSummaries.foreach { summary => + numPartitions += summary.updatedPartitions.size + numFiles += summary.numOutputFile + totalNumBytes += summary.numOutputBytes + totalNumOutput += summary.numOutputRows + totalWritingTime += summary.totalWritingTime + } + + val avgWritingTime = if (numFiles > 0) { + (totalWritingTime / numFiles).toLong + } else { + 0L + } + + metrics("avgTime").add(avgWritingTime) + metrics("numFiles").add(numFiles) + metrics("numOutputBytes").add(totalNumBytes) + metrics("numOutputRows").add(totalNumOutput) + metrics("numParts").add(numPartitions) + + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toList) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 9a5f514ccf50b..7cd4baef89e75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.command import java.util.UUID -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} @@ -27,14 +26,12 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} -import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.debug._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils /** * A logical command that is executed for its side-effects. `RunnableCommand`s are @@ -55,63 +52,6 @@ trait RunnableCommand extends logical.Command { } } -/** - * A special `RunnableCommand` which writes data out and updates metrics. - */ -trait DataWritingCommand extends RunnableCommand { - - override lazy val metrics: Map[String, SQLMetric] = { - val sparkContext = SparkContext.getActive.get - Map( - "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing time (ms)"), - "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), - "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numParts" -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") - ) - } - - /** - * Callback function that update metrics collected from the writing operation. - */ - protected def updateWritingMetrics(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { - val sparkContext = SparkContext.getActive.get - var numPartitions = 0 - var numFiles = 0 - var totalNumBytes: Long = 0L - var totalNumOutput: Long = 0L - var totalWritingTime: Long = 0L - var numFilesNonZeroWritingTime = 0 - - writeSummaries.foreach { summary => - numPartitions += summary.updatedPartitions.size - numFiles += summary.numOutputFile - totalNumBytes += summary.numOutputBytes - totalNumOutput += summary.numOutputRows - totalWritingTime += summary.totalWritingTime - numFilesNonZeroWritingTime += summary.numFilesWithNonZeroWritingTime - } - - // We only count non-zero writing time when averaging total writing time. - // The time for writing individual file can be zero if it's less than 1 ms. Zero values can - // lower actual time of writing to zero when calculating average, so excluding them. - val avgWritingTime = if (numFilesNonZeroWritingTime > 0) { - (totalWritingTime / numFilesNonZeroWritingTime).toLong - } else { - 0L - } - - metrics("avgTime").add(avgWritingTime) - metrics("numFiles").add(numFiles) - metrics("numOutputBytes").add(totalNumBytes) - metrics("numOutputRows").add(totalNumOutput) - metrics("numParts").add(numPartitions) - - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toList) - } -} - /** * A physical operator that executes the run method of a `RunnableCommand` and * saves the result to prevent multiple executions. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index de4b3f833f0d8..64866630623ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -276,7 +276,6 @@ object FileFormatWriter extends Logging { * The data structures used to measure metrics during writing. */ protected var totalWritingTime: Long = 0L - protected var numFilesWithNonZeroWritingTime = 0 protected var timeOnCurrentFile: Long = 0L protected var numOutputRows: Long = 0L protected var numOutputBytes: Long = 0L @@ -357,7 +356,6 @@ object FileFormatWriter extends Logging { numOutputFile = fileCounter + 1, numOutputBytes = numOutputBytes, numOutputRows = numOutputRows, - numFilesWithNonZeroWritingTime = numFilesWithNonZeroWritingTime, totalWritingTime = totalWritingTime) } @@ -366,11 +364,7 @@ object FileFormatWriter extends Logging { try { val startTime = System.nanoTime() currentWriter.close() - val writingTime = (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 - if (writingTime > 0) { - numFilesWithNonZeroWritingTime += 1 - totalWritingTime += writingTime - } + totalWritingTime += (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 timeOnCurrentFile = 0 numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } finally { @@ -526,7 +520,6 @@ object FileFormatWriter extends Logging { numOutputFile = totalFileCounter, numOutputBytes = numOutputBytes, numOutputRows = numOutputRows, - numFilesWithNonZeroWritingTime = numFilesWithNonZeroWritingTime, totalWritingTime = totalWritingTime) } @@ -535,11 +528,7 @@ object FileFormatWriter extends Logging { try { val startTime = System.nanoTime() currentWriter.close() - val writingTime = (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 - if (writingTime > 0) { - numFilesWithNonZeroWritingTime += 1 - totalWritingTime += writingTime - } + totalWritingTime += (timeOnCurrentFile + System.nanoTime() - startTime) / 1000 / 1000 timeOnCurrentFile = 0 numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) } finally { @@ -558,7 +547,6 @@ object FileFormatWriter extends Logging { * @param numOutputFile the total number of files. * @param numOutputRows the number of output rows. * @param numOutputBytes the bytes of output data. - * @param numFilesWithNonZeroWritingTime the number of files with non zero writing time. * @param totalWritingTime the total writing time in ms. */ case class ExecutedWriteSummary( @@ -566,5 +554,4 @@ case class ExecutedWriteSummary( numOutputFile: Int, numOutputRows: Long, numOutputBytes: Long, - numFilesWithNonZeroWritingTime: Int, totalWritingTime: Long)