You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by viirya <gi...@git.apache.org> on 2017/05/31 08:52:02 UTC

[GitHub] spark pull request #18159: [SPARK-20703][SQL][WIP] Associate metrics with da...

GitHub user viirya opened a pull request:

    https://github.com/apache/spark/pull/18159

    [SPARK-20703][SQL][WIP] Associate metrics with data writes onto DataFrameWriter operations

    ## What changes were proposed in this pull request?
    
    Right now in the UI, after SPARK-20213, we can show the operations to write data out. However, there is no way to associate metrics with data writes. We should show relative metrics on the operations.
    
    ### The Approach
    
    We have several paths for writing data out through some commands.
    
    #### File-based: `InsertIntoHadoopFsRelationCommand`, `InsertIntoHiveTable`
    
    The two commands use `FileFormatWriter` to write out data files. This patch record some metrics in `FileFormatWriter` and pass into the callback function for updating metrics in `SparkPlan`.
    
    * number of written files
    * number of dynamic partitions
    * bytes of written files
    * number of output rows
    * writing data out time (ms)
    
    #### Other datasources: `InsertIntoDataSourceCommand`, `SaveIntoDataSourceCommand`
    
    For other datasource relations, the logic of writing data out is delegated to the datasource implementations, e.g., `InsertableRelation.insert`, `CreatableRelationProvider.createRelation`. So we can't obtain metrics from delegated methods for now.
    
    #### `CreateDataSourceTableAsSelectCommand` and `CreateHiveTableAsSelectCommand`: 
    
    The two commands creates and invokes other commands (`InsertIntoHadoopFsRelationCommand`, `InsertIntoHiveTable`). Although we support recording metrics for the invoked commands, however, the metrics are recorded in the invoked `SparkPlan`, instead of the commands invoking them. So we can't show metrics for the two commands for now.
    
    ## How was this patch tested?
    
    Updated unit tests.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/viirya/spark-1 SPARK-20703-2

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/18159.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #18159
    
----
commit a6efa7380591498a000ee89eff8460e9e6463f9d
Author: Liang-Chi Hsieh <vi...@gmail.com>
Date:   2017-05-31T08:03:08Z

    Support to show data writing metrics.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125389877
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -47,10 +56,73 @@ 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
    --- End diff --
    
    This only happens if a partition is very small, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77639 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77639/testReport)** for PR 18159 at commit [`143764c`](https://github.com/apache/spark/commit/143764cf266b4cec4e1d2c529a3673cb95fcc523).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r121889358
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,93 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.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 {
    +  def run(
    +    sparkSession: SparkSession,
    +    children: Seq[SparkPlan],
    +    fileCommandExec: 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],
    +    givenMetrics: Option[Map[String, SQLMetric]] = 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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(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
    +    }
    +
    +    // 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
    +    // Note: for simplifying metric values assignment, we put the values as the alphabetically
    +    // sorted of the metric keys.
    --- End diff --
    
    I think it's simpler to just write 4 lines to set these 4 metrics


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77592/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120121507
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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}
    +
    +/**
    + * 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`,
    + * 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
    +    }
    +
    +    val times = writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0))
    --- End diff --
    
    leave a note explaining how a `writeTime` of <=0 can occur and why we're excluding it here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77708/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120359550
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -446,7 +451,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))
    +        qe.executedPlan.transform {
    +          case f: FileWritingCommandExec =>
    +            val newCmd = f.cmd.withExternalMetrics(externalMetrics.getOrElse(null))
    +            FileWritingCommandExec(newCmd, f.children)
    +        }.execute()
    --- End diff --
    
    Hm..
    
    So it looks like you do this in 2 places: `CreateHiveTableAsSelectCommand` and `CreateDataSourceTableAsSelectCommand.saveDataIntoTable() -> DataSource.writeAndRead()`
    
    For the latter case, I'd pass `CreateDataSourceTableAsSelectCommand.metrics` through `planForWritingFileFormat()` to `InsertIntoHadoopFsRelationCommand` which already extends `FileWritingCommand` and could just take those existing metrics as a constructor argument.
    
    For the former, I'm not sure, as I see `InsertIntoTable` does not extend `FileWritingCommand`. Maybe it should? Then a similar solution would apply there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125218214
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -273,12 +271,36 @@ object FileFormatWriter extends Logging {
        * automatically trigger task aborts.
        */
       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.
    +     * The data structures used to measure metrics during writing.
          */
    -    def execute(iterator: Iterator[InternalRow]): Set[String]
    +    protected val writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty
    --- End diff --
    
    So I send back `writingTimePerFile` and filter with time values greater than 0 before counting average on them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r124982973
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -19,25 +19,70 @@ 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}
     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, children: Seq[SparkPlan]): 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(
    --- End diff --
    
    Sounds good.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77714/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/78969/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Ok. Thanks! @cloud-fan @rxin @adrian-ionescu 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    ping @cloud-fan @rxin Do you have more comments on this? Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77678 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77678/testReport)** for PR 18159 at commit [`184b50b`](https://github.com/apache/spark/commit/184b50be96dcc7166025c6d1a8cbf140e3ad73e5).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #79078 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79078/testReport)** for PR 18159 at commit [`f575f0c`](https://github.com/apache/spark/commit/f575f0c999647d1f7fda22652ea480f7a0d07047).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait DataWritingCommand extends RunnableCommand `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r126048467
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -314,21 +339,40 @@ object FileFormatWriter extends Logging {
     
               recordsInFile = 0
               releaseResources()
    +          numOutputRows += recordsInFile
               newOutputWriter(fileCounter)
             }
     
             val internalRow = iter.next()
    +        val startTime = System.nanoTime()
             currentWriter.write(internalRow)
    +        timeOnCurrentFile += (System.nanoTime() - startTime)
    --- End diff --
    
    seems it's impossible to track writing time without per-row tracking, shall we just remove the `writeTime` metrics?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    After think about it more, I think the problem we need to solve here is to support metrics for `RunnableCommand`. My previous refactor fixed the problem to support children in `RunnableCommand`.
    
    My proposal:
    ```
    trait RunnableCommand {
      val metrics: Map[String, SQLMetric] = Map.empty
      ...
    }
    
    case class ExecutedCommandExec(
        cmd: RunnableCommand,
        children: Seq[SparkPlan],
        override val metrics: Map[String, SQLMetric])
    ```
    
    In `InsertIntoHadoopFsRelationCommand` we can override the  `metrics` and update it according to `FileFormatWritter`. And in the planner we pass `RunnableCommand.metrics` to `ExecutedCommandExec`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77717 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77717/testReport)** for PR 18159 at commit [`de069e7`](https://github.com/apache/spark/commit/de069e757d81419aee21748d3aa605ee7a75cf33).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    That sounds good. Then I will simplify this. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78779 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78779/testReport)** for PR 18159 at commit [`9d8d77f`](https://github.com/apache/spark/commit/9d8d77f2390e19432881f10c099fbb3cddd6476d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125400761
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -314,21 +339,40 @@ object FileFormatWriter extends Logging {
     
               recordsInFile = 0
               releaseResources()
    +          numOutputRows += recordsInFile
               newOutputWriter(fileCounter)
             }
     
             val internalRow = iter.next()
    +        val startTime = System.nanoTime()
             currentWriter.write(internalRow)
    +        timeOnCurrentFile += (System.nanoTime() - startTime)
    --- End diff --
    
    If we track the time in `newOutputWriter`, the time will not only contain writing time, but also the time pulling the rows when iterating. If the query plan is complicated, I guess that makes inaccurate writing time?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/78803/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77678/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/78779/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125217300
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala ---
    @@ -53,11 +55,22 @@ 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 lazy val metrics: Map[String, SQLMetric] = {
    --- End diff --
    
    Sure.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77632 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77632/testReport)** for PR 18159 at commit [`15fb308`](https://github.com/apache/spark/commit/15fb3082fdfa4e149c715193f3421b893f851d96).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #79078 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79078/testReport)** for PR 18159 at commit [`f575f0c`](https://github.com/apache/spark/commit/f575f0c999647d1f7fda22652ea480f7a0d07047).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    [Here](https://gist.github.com/adrian-ionescu/bc68783e01071419ec483bd8216757e8#file-gistfile1-txt-L20)'s a small test that shows that your `writingTime` metric actually includes the whole query processing time. It would be more useful if it only showed actual write time.
    
    I suggest you just wrap calls to `currentWriter.write()` from `FileFormatWriter.scala`, where you've already added some instrumentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77709 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77709/testReport)** for PR 18159 at commit [`a6438ef`](https://github.com/apache/spark/commit/a6438ef30ce058905baf8614f3e17eaa41f5a4c3).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120339182
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -446,7 +451,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))
    +        qe.executedPlan.transform {
    +          case f: FileWritingCommandExec =>
    +            val newCmd = f.cmd.withExternalMetrics(externalMetrics.getOrElse(null))
    +            FileWritingCommandExec(newCmd, f.children)
    +        }.execute()
    --- End diff --
    
    I don't think this is much of an improvement over the previous hack.. See above comments, try to avoid changing this code at all.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77639/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125217353
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -47,10 +56,56 @@ trait RunnableCommand extends logical.Command {
     }
     
     /**
    + * 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.
    +   */
    +  protected def callbackMetricsUpdater(writeSummaries: Seq[ExecutedWriteSummary]): Unit = {
    --- End diff --
    
    Good for me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125218623
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -273,12 +271,36 @@ object FileFormatWriter extends Logging {
        * automatically trigger task aborts.
        */
       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.
    +     * The data structures used to measure metrics during writing.
          */
    -    def execute(iterator: Iterator[InternalRow]): Set[String]
    +    protected val writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty
    --- End diff --
    
    Yap.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #79019 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79019/testReport)** for PR 18159 at commit [`6ea6bbb`](https://github.com/apache/spark/commit/6ea6bbb819a2ed90f86bde7415a8fada809ce4c9).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait MetricUpdater `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #79084 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79084/testReport)** for PR 18159 at commit [`8380023`](https://github.com/apache/spark/commit/8380023ab96fa6febe19918da9284fe14aea1f53).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125218995
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -273,12 +271,36 @@ object FileFormatWriter extends Logging {
        * automatically trigger task aborts.
        */
       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.
    +     * The data structures used to measure metrics during writing.
          */
    -    def execute(iterator: Iterator[InternalRow]): Set[String]
    +    protected val writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty
    --- End diff --
    
    Ok. Will update accordingly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r121927605
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,93 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.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 {
    +  def run(
    +    sparkSession: SparkSession,
    +    children: Seq[SparkPlan],
    +    fileCommandExec: 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],
    +    givenMetrics: Option[Map[String, SQLMetric]] = 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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(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
    +    }
    +
    +    // 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
    +    // Note: for simplifying metric values assignment, we put the values as the alphabetically
    +    // sorted of the metric keys.
    --- End diff --
    
    Ok.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/78780/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77789 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77789/testReport)** for PR 18159 at commit [`9b87b4f`](https://github.com/apache/spark/commit/9b87b4f306ff5f46f0795ff40712e71698cddab7).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r126056717
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -314,21 +339,40 @@ object FileFormatWriter extends Logging {
     
               recordsInFile = 0
               releaseResources()
    +          numOutputRows += recordsInFile
               newOutputWriter(fileCounter)
             }
     
             val internalRow = iter.next()
    +        val startTime = System.nanoTime()
             currentWriter.write(internalRow)
    +        timeOnCurrentFile += (System.nanoTime() - startTime)
    --- End diff --
    
    Yeah, I also considered this option.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77707/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77718 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77718/testReport)** for PR 18159 at commit [`82de2a3`](https://github.com/apache/spark/commit/82de2a375fd8bc6ecb22a8529261c87540b2ccce).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class SQLMetricsSuite extends SQLTestUtils with TestHiveSingleton `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120368953
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    --- End diff --
    
    Indeed, I missed it at first, but then I got it. See below: https://github.com/apache/spark/pull/18159#discussion_r120359550


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120134028
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -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(_ => ()))
    --- End diff --
    
    Because here we need to invoke the command and pass the metrics-updating callback function. `.execute()` on the plan can't allow us do this. I'd think if there is a better way to do this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77718/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77717 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77717/testReport)** for PR 18159 at commit [`de069e7`](https://github.com/apache/spark/commit/de069e757d81419aee21748d3aa605ee7a75cf33).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    can you also post some screenshots?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77698/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    retest this please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77697 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77697/testReport)** for PR 18159 at commit [`e8e952b`](https://github.com/apache/spark/commit/e8e952bf50af8d636cec04d855540840347b6eda).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125213358
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -273,12 +271,36 @@ object FileFormatWriter extends Logging {
        * automatically trigger task aborts.
        */
       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.
    +     * The data structures used to measure metrics during writing.
          */
    -    def execute(iterator: Iterator[InternalRow]): Set[String]
    +    protected val writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty
    --- End diff --
    
    Since we only care about average writing time, why we send back `writingTimePerFile`? Can we just send back total writing time and numFiles?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77713 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77713/testReport)** for PR 18159 at commit [`3137566`](https://github.com/apache/spark/commit/3137566ee6dd6fc9cc17abd1085229e1d781bb32).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125218806
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -273,12 +271,36 @@ object FileFormatWriter extends Logging {
        * automatically trigger task aborts.
        */
       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.
    +     * The data structures used to measure metrics during writing.
          */
    -    def execute(iterator: Iterator[InternalRow]): Set[String]
    +    protected val writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty
    --- End diff --
    
    I think we should do so, sending `writingTimePerFile` seems expensive.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r121927602
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -60,14 +56,11 @@ case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) e
        */
       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])
    +    invokeCommand.map(converter(_).asInstanceOf[InternalRow])
       }
     
    +  protected[sql] val invokeCommand: Seq[Row]
    --- End diff --
    
    Sure. Sounds good.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125212516
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -47,10 +56,56 @@ trait RunnableCommand extends logical.Command {
     }
     
     /**
    + * 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.
    +   */
    +  protected def callbackMetricsUpdater(writeSummaries: Seq[ExecutedWriteSummary]): Unit = {
    --- End diff --
    
    how about `updateWritingMetrics`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    @adrian-ionescu I've included the suggested metrics per file/partition. The `writingTime` metric is also changed to record actual time on writing data.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Looks like a large code change, but there are 200+ codes for test cases.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77698 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77698/testReport)** for PR 18159 at commit [`8bb02d9`](https://github.com/apache/spark/commit/8bb02d972c0f5606ef9870ec4d515b721275f653).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    LGTM except some minor comments, thanks for working on it!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78969 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78969/testReport)** for PR 18159 at commit [`32f4abf`](https://github.com/apache/spark/commit/32f4abf64263a16ef9c0cefd55990d30cd93ce38).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79019/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79078/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125217322
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -47,10 +56,56 @@ trait RunnableCommand extends logical.Command {
     }
     
     /**
    + * A trait for classes that can update its metrics of data writing operation.
    + */
    +trait MetricUpdater {
    --- End diff --
    
    Sounds good.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120350843
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    +
    +  /**
    +   * 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 {
    +    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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: Map[String, SQLMetric])
    +      (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
    +    }
    +
    +    // 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 = 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 = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
    +    SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metricsNames.map(metrics(_)))
    +  }
    +
    +  def run(
    +    sparkSession: SparkSession,
    +    children: Seq[SparkPlan],
    +    metrics: Map[String, SQLMetric],
    +    metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row]
    --- End diff --
    
    The case is more complicated...
    
    We have those commands which don't write the data but invoke another commands to do that. The execution data for showing on UI is bound to the original commands, but the invoked commands. That's why we need to pass metrics or callback function here...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77721 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77721/testReport)** for PR 18159 at commit [`775d2c2`](https://github.com/apache/spark/commit/775d2c284666238e25cbe4cdaf95677fde358eb4).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #79085 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79085/testReport)** for PR 18159 at commit [`dfaa4f1`](https://github.com/apache/spark/commit/dfaa4f11f35e2615ce627f29b1a60ab27dfd5477).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77713/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79085/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/18159


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77616 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77616/testReport)** for PR 18159 at commit [`2ff2fa1`](https://github.com/apache/spark/commit/2ff2fa13bc0b46a1074c2814ad69f23516664968).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r124740444
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -19,25 +19,70 @@ 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}
     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, children: Seq[SparkPlan]): 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(
    --- End diff --
    
    We can remove this abstraction if we can get the `SparkContext` in logical plan instead of passing it via `RunnaleCommand.run`. I think this is doable via `SparkContext.getActive.get`
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #79085 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79085/testReport)** for PR 18159 at commit [`dfaa4f1`](https://github.com/apache/spark/commit/dfaa4f11f35e2615ce627f29b1a60ab27dfd5477).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78780 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78780/testReport)** for PR 18159 at commit [`8926156`](https://github.com/apache/spark/commit/89261564a6251b4eeebff6f0455536c20499c248).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r124979471
  
    --- Diff: core/src/main/scala/org/apache/spark/util/Utils.scala ---
    @@ -2659,6 +2668,21 @@ 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)
    --- End diff --
    
    Is this better? Now we don't call `size` on the `Iterable`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #79142 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79142/testReport)** for PR 18159 at commit [`04e79d9`](https://github.com/apache/spark/commit/04e79d9255f923b9e5a740d20e110b62280310f6).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait DataWritingCommand extends RunnableCommand `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77719 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77719/testReport)** for PR 18159 at commit [`ba1070b`](https://github.com/apache/spark/commit/ba1070b20aa2eb507ca859f58468e6b19709364b).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77678 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77678/testReport)** for PR 18159 at commit [`184b50b`](https://github.com/apache/spark/commit/184b50be96dcc7166025c6d1a8cbf140e3ad73e5).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78779 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78779/testReport)** for PR 18159 at commit [`9d8d77f`](https://github.com/apache/spark/commit/9d8d77f2390e19432881f10c099fbb3cddd6476d).
     * This patch passes all tests.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77707 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77707/testReport)** for PR 18159 at commit [`30f202e`](https://github.com/apache/spark/commit/30f202e37fc1d27d5752918f1886d500ea46aa11).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    @adrian-ionescu In order to reduce the size of this PR, I've removed the metrics per file/partition. They can be todo items and maybe we can add them later in other PRs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120506519
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    +
    +  /**
    +   * 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 {
    +    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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: Map[String, SQLMetric])
    +      (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
    +    }
    +
    +    // 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 = metrics.keys.toSeq.sorted
    +    val metricsValues = Seq(writingTime, numFiles, totalNumBytes, totalNumOutput, numPartitions)
    +    metricsNames.zip(metricsValues).foreach(x => metrics(x._1).add(x._2))
    --- End diff --
    
    For metrics purpose, this change is a bit too large. I'd try not to increase the complexity for now. I added a comment for this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120373405
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    --- End diff --
    
    I should have better code comment on this part too. I'll add it in next commit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    cc @cloud-fan @rxin @hvanhovell This is ready for review now. Please take a look when you have time. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125803128
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -314,21 +339,40 @@ object FileFormatWriter extends Logging {
     
               recordsInFile = 0
               releaseResources()
    +          numOutputRows += recordsInFile
               newOutputWriter(fileCounter)
             }
     
             val internalRow = iter.next()
    +        val startTime = System.nanoTime()
             currentWriter.write(internalRow)
    +        timeOnCurrentFile += (System.nanoTime() - startTime)
    --- End diff --
    
    I'm neutral about the per-row time tracking. Comparing with actual data writing, it should be faster. IMHO, so the performance penalty may be ignored?
    
    In both `SingleDirectoryWriteTask` and `DynamicPartitionWriteTask`, in each iteration, we first pull the row from the iterator and then write the row out (and measure the timing). So the time spent on pulling rows from data pipeline should be already excluded. That's why I'm not worried it may be inaccurate due to data pipelining.
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77788/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120115392
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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}
    +
    +/**
    + * 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
    --- End diff --
    
    better omit the body altogether


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/78777/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120369741
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -446,7 +451,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))
    +        qe.executedPlan.transform {
    +          case f: FileWritingCommandExec =>
    +            val newCmd = f.cmd.withExternalMetrics(externalMetrics.getOrElse(null))
    +            FileWritingCommandExec(newCmd, f.children)
    +        }.execute()
    --- End diff --
    
    `InsertIntoTable` is a general logical node representing `inserting some data into a table`. It is not a `logical.Command`. It shouldn't be a `FileWritingCommand` actually. As it's unresolved, making it to extend `FileWritingCommand` and have metrics don't make sense too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77584 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77584/testReport)** for PR 18159 at commit [`a6efa73`](https://github.com/apache/spark/commit/a6efa7380591498a000ee89eff8460e9e6463f9d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77788 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77788/testReport)** for PR 18159 at commit [`dd637ce`](https://github.com/apache/spark/commit/dd637ce9691a7fe0655cd0b16d772c9890157ea2).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class FileWritingCommandExec(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120120540
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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}
    +
    +/**
    + * 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`,
    + * 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
    +    }
    +
    +    val times = writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0))
    +    val avgWritingTime = if (times.size > 0) {
    +      times.sum / times.size
    +    } else {
    +      0
    +    }
    --- End diff --
    
    Is there no library/Utils function that computes the average value of a list? if not, better add one and use it here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78050 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78050/testReport)** for PR 18159 at commit [`6e8ffc5`](https://github.com/apache/spark/commit/6e8ffc5b847905dcd703b1614858693c94f5fdae).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77677 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77677/testReport)** for PR 18159 at commit [`43c17c0`](https://github.com/apache/spark/commit/43c17c08cf326fe59d583afcb070100771f1239d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77677/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120334376
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    +
    +  /**
    +   * 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 {
    +    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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: Map[String, SQLMetric])
    +      (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
    +    }
    +
    +    // 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 = 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 = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
    +    SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metricsNames.map(metrics(_)))
    +  }
    +
    +  def run(
    +    sparkSession: SparkSession,
    +    children: Seq[SparkPlan],
    +    metrics: Map[String, SQLMetric],
    +    metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row]
    --- End diff --
    
    Why does this take `metrics` as a parameter, when the trait already has a `metrics` member?
    
    A big part of this patch is just about passing down metrics through this interface.. It also makes it quite hard to follow. Is there no way we can avoid this? It would be a significant improvement if you could find a solution.
    
    Basically try to specify the metrics (either the Map you have, or None) together with the corresponding callback function just once, when you first instantiate `FileWritingCommandExec` in `SparkStrategies.scala`. Then you won't need to pass around metrics all over the place and you also won't need the `transform ... withExternalMetrics` hack.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77697/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/78968/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125400321
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -47,10 +56,73 @@ 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
    --- End diff --
    
    Yeah, it's for small partition like the test case.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120349943
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -446,7 +451,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))
    +        qe.executedPlan.transform {
    +          case f: FileWritingCommandExec =>
    +            val newCmd = f.cmd.withExternalMetrics(externalMetrics.getOrElse(null))
    +            FileWritingCommandExec(newCmd, f.children)
    +        }.execute()
    --- End diff --
    
    Can you provide specified suggestion regarding this? I don't think there is a possible way to avoid this.
    
    Those commands are invoking another commands to write the data out. We want to have the metrics updated is the metrics in the original physical node, not the invoked one. To do this, we either pass the callback or the metrics in the original plan into the invoked one. Do you think we have other choice?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77697 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77697/testReport)** for PR 18159 at commit [`e8e952b`](https://github.com/apache/spark/commit/e8e952bf50af8d636cec04d855540840347b6eda).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r119999871
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -17,38 +17,97 @@
     
     package org.apache.spark.sql.execution.command
     
    +import scala.collection.mutable
    +
     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}
     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._
     
     /**
    - * 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 {
    +
    +  /**
    +   * 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.
    --- End diff --
    
    Ok. Let me revert specified metrics.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77765/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120375471
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    +
    +  /**
    +   * 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 {
    +    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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: Map[String, SQLMetric])
    +      (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
    +    }
    +
    +    // 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 = metrics.keys.toSeq.sorted
    +    val metricsValues = Seq(writingTime, numFiles, totalNumBytes, totalNumOutput, numPartitions)
    +    metricsNames.zip(metricsValues).foreach(x => metrics(x._1).add(x._2))
    --- End diff --
    
    I meant user as in caller of this function. This function only works when the input satisfies some requirements and silently fails otherwise. I get it that it's a private method, but it's very error prone.
    
    Imagine I want to extend this by adding another metric called `avgNumFilesPerPart`. If I add it to the end of the `metricsValues` Seq, then all metrics will get messed up (because of the ordering).
    
    At the very least, leave a comment saying that the `metricValues` need to be sorted alphabetically (and rename `writingTime` to `avgWritingTime`). But you should rather consider using a `WriteMetrics` class instead of a Map.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77584 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77584/testReport)** for PR 18159 at commit [`a6efa73`](https://github.com/apache/spark/commit/a6efa7380591498a000ee89eff8460e9e6463f9d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait WriteOutFileCommand extends logical.Command `
      * `trait RunnableCommand extends logical.Command `
      * `trait CommandExec extends SparkPlan `
      * `case class WrittenFileCommandExec(`
      * `case class ExecutedCommandExec(cmd: RunnableCommand) extends CommandExec `
      * `case class ExecutedWriteSummary(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78777 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78777/testReport)** for PR 18159 at commit [`a7e53bb`](https://github.com/apache/spark/commit/a7e53bbc6116acfc6f958c062095e3c74fa4195f).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #79142 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79142/testReport)** for PR 18159 at commit [`04e79d9`](https://github.com/apache/spark/commit/04e79d9255f923b9e5a740d20e110b62280310f6).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77616/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125418557
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -47,10 +56,73 @@ 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
    --- End diff --
    
    Ok. Since it should be rare, I removed this part so the codes can be simpler.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120120649
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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}
    +
    +/**
    + * 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`,
    + * 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
    --- End diff --
    
    I feel this function belongs to the trait, next to the method that creates these metrics.
    Right now you're spelling out this list of metric names in two different places (here + trait)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77787 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77787/testReport)** for PR 18159 at commit [`16b0b00`](https://github.com/apache/spark/commit/16b0b0016cbc9790a4cbe2a3972f155e2c128e5d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class FileWritingCommandExec(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125218388
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -273,12 +271,36 @@ object FileFormatWriter extends Logging {
        * automatically trigger task aborts.
        */
       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.
    +     * The data structures used to measure metrics during writing.
          */
    -    def execute(iterator: Iterator[InternalRow]): Set[String]
    +    protected val writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty
    --- End diff --
    
    OK so we need to send back `totalWriteTime` and `numFilesWithNonZeroWritingTime` right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    `InsertIntoHadoopFsRelationCommand` is case with less trouble. We can do it like that: passing `RunnableCommand.metrics` to `ExecutedCommandExec`.
    
    `CreateDataSourceTableAsSelectCommand` and `CreateHiveTableAsSelectCommand` make thing more complicate, because they invoke other commands to write data. So we need to update the upper metrics instead of the invoked command's metrics.
    
    We even can't update metrics in `CreateDataSourceTableAsSelectCommand` if it writes to non file-format datasource.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79142/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #79019 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79019/testReport)** for PR 18159 at commit [`6ea6bbb`](https://github.com/apache/spark/commit/6ea6bbb819a2ed90f86bde7415a8fada809ce4c9).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78968 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78968/testReport)** for PR 18159 at commit [`1defb3a`](https://github.com/apache/spark/commit/1defb3ad063b975559dfc14e7fc791e837b840e8).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    how about we only deal with `InsertIntoHadoopFsRelationCommand` for now? I think `CreateDataSourceTableAsSelectCommand` need some refactor to decouple the table creation and data insertion.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77765 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77765/testReport)** for PR 18159 at commit [`0af718d`](https://github.com/apache/spark/commit/0af718d15ed9c6bcf4e8de19528affdc492d1257).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/78050/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77714 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77714/testReport)** for PR 18159 at commit [`e8acb49`](https://github.com/apache/spark/commit/e8acb49300a2cd497e5eacd36945ddaf04ebdaa8).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77698 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77698/testReport)** for PR 18159 at commit [`8bb02d9`](https://github.com/apache/spark/commit/8bb02d972c0f5606ef9870ec4d515b721275f653).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77712 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77712/testReport)** for PR 18159 at commit [`32f49ea`](https://github.com/apache/spark/commit/32f49eae328c97e3be1d0492d000c83ea06bb11d).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78050 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78050/testReport)** for PR 18159 at commit [`6e8ffc5`](https://github.com/apache/spark/commit/6e8ffc5b847905dcd703b1614858693c94f5fdae).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78803 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78803/testReport)** for PR 18159 at commit [`8926156`](https://github.com/apache/spark/commit/89261564a6251b4eeebff6f0455536c20499c248).
     * This patch **fails due to an unknown error code, -10**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120377948
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -446,7 +451,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))
    +        qe.executedPlan.transform {
    +          case f: FileWritingCommandExec =>
    +            val newCmd = f.cmd.withExternalMetrics(externalMetrics.getOrElse(null))
    +            FileWritingCommandExec(newCmd, f.children)
    +        }.execute()
    --- End diff --
    
    Yeah, I agree that's not a good way to go..


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77787 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77787/testReport)** for PR 18159 at commit [`16b0b00`](https://github.com/apache/spark/commit/16b0b0016cbc9790a4cbe2a3972f155e2c128e5d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125461897
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -314,21 +339,40 @@ object FileFormatWriter extends Logging {
     
               recordsInFile = 0
               releaseResources()
    +          numOutputRows += recordsInFile
               newOutputWriter(fileCounter)
             }
     
             val internalRow = iter.next()
    +        val startTime = System.nanoTime()
             currentWriter.write(internalRow)
    +        timeOnCurrentFile += (System.nanoTime() - startTime)
    --- End diff --
    
    I'm worried about this per-row time tracking may have performance penalty, and also may be inaccurate because the real data flow is pipelined. cc @rxin who should have more experience in this area.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125389285
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -47,10 +56,73 @@ trait RunnableCommand extends logical.Command {
     }
     
     /**
    + * A special `RunnableCommand` which writes data out and updates metrics.
    + */
    +trait DataWritingCommand extends RunnableCommand {
    --- End diff --
    
    let's move it to a new file


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125158558
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -19,24 +19,65 @@ 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}
     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 {
    +
    +  // 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
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  protected def callbackMetricsUpdater(writeSummaries: Seq[ExecutedWriteSummary]): Unit = {
    --- End diff --
    
    Ok. I created a trait for them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r119995109
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -17,38 +17,97 @@
     
     package org.apache.spark.sql.execution.command
     
    +import scala.collection.mutable
    +
     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}
     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._
     
     /**
    - * 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 {
    --- End diff --
    
    move it to a new file?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    @cloud-fan The screenshot looks like:
    
    <img width="358" alt="screen shot 2017-06-06 at 1 30 15 pm" src="https://cloud.githubusercontent.com/assets/68855/26815029/614f13ba-4abc-11e7-9fbf-2248f0b7211d.png">
    
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125151815
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -19,24 +19,65 @@ 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}
     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 {
    +
    +  // 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
    --- End diff --
    
    usually we don't need `private[sql]` under the `execution` package.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r124739262
  
    --- Diff: core/src/main/scala/org/apache/spark/util/Utils.scala ---
    @@ -2659,6 +2668,13 @@ private[spark] object Utils extends Logging {
         redact(redactionPattern, kvs.toArray)
       }
     
    +  def average[T](ts: Iterable[T])(implicit num: Numeric[T]): Double = {
    --- End diff --
    
    shall we require array type parameter? We are calling `size` twice in this method, which can be very slow if the input is not indexed seq.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r121889646
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -60,14 +56,11 @@ case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) e
        */
       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])
    +    invokeCommand.map(converter(_).asInstanceOf[InternalRow])
       }
     
    +  protected[sql] val invokeCommand: Seq[Row]
    --- End diff --
    
    the name looks weird for a `val`, shall we make it a `def`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120117616
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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}
    +
    +/**
    + * 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`,
    + * 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
    +    }
    +
    +    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")
    --- End diff --
    
    should just be `metrics.keys`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120355478
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    +
    +  /**
    +   * 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 {
    +    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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: Map[String, SQLMetric])
    +      (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
    +    }
    +
    +    // 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.
    --- End diff --
    
    Excluding 0 values because 0 values can make the average writing time as 0. It seems to me that it doesn't really make sense to show 0 writing time. Seems it indicates the writing doesn't cost time.
    
    1ms brings the average down, but you still get a meaningful time value.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77788 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77788/testReport)** for PR 18159 at commit [`dd637ce`](https://github.com/apache/spark/commit/dd637ce9691a7fe0655cd0b16d772c9890157ea2).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r121940285
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,93 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.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 {
    +  def run(
    +    sparkSession: SparkSession,
    +    children: Seq[SparkPlan],
    +    fileCommandExec: 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],
    +    givenMetrics: Option[Map[String, SQLMetric]] = 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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(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
    +    }
    +
    +    // 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
    +    // Note: for simplifying metric values assignment, we put the values as the alphabetically
    +    // sorted of the metric keys.
    +    val metricsNames = metrics.keys.toSeq.sorted
    --- End diff --
    
    `givenMetrics` comes from other `FileWritingCommandExec` or an empty. When it's an empty map, means the wrapped command won't call this callback. But I agree this is loose guarantee. I'll update this in next commit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    hmm anyway to shorten the change? this change is a bit too big for metrics ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77677 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77677/testReport)** for PR 18159 at commit [`43c17c0`](https://github.com/apache/spark/commit/43c17c08cf326fe59d583afcb070100771f1239d).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77639 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77639/testReport)** for PR 18159 at commit [`143764c`](https://github.com/apache/spark/commit/143764cf266b4cec4e1d2c529a3673cb95fcc523).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77712/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77721/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77718 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77718/testReport)** for PR 18159 at commit [`82de2a3`](https://github.com/apache/spark/commit/82de2a375fd8bc6ecb22a8529261c87540b2ccce).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125212464
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -47,10 +56,56 @@ trait RunnableCommand extends logical.Command {
     }
     
     /**
    + * A trait for classes that can update its metrics of data writing operation.
    + */
    +trait MetricUpdater {
    --- End diff --
    
    I'd like to call it `trait InsertionCommand extends RunnableCommand`, as we are updating `avgTime`, `numFiles` etc, which is specific to insertion.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120354430
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    --- End diff --
    
    I've explained the purpose of this in below comments.
    
    Once a command invokes another command to write data. We have to update the metrics in the physical plan of the first command, not the second.
    
    Before we invoke the second command, we have to replace the metrics in the physical plan's metrics with the metrics in the first one (i.e., the external metrics).
    
    It seems to me that you miss the relationship that there is two different commands.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120352370
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    +
    +  /**
    +   * 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 {
    +    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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: Map[String, SQLMetric])
    +      (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
    +    }
    +
    +    // 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 = metrics.keys.toSeq.sorted
    +    val metricsValues = Seq(writingTime, numFiles, totalNumBytes, totalNumOutput, numPartitions)
    +    metricsNames.zip(metricsValues).foreach(x => metrics(x._1).add(x._2))
    --- End diff --
    
    I am not sure if I correctly understand your point.
    
    The external metrics are always coming from the same kind of trait. It's not coming from user, how it's user-provided map?
    
    The order of keys returned by `map.keys` is not determined as I tried. Without sorting, how do we make sure we match the metrics values with correct metrics keys?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r121890420
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,93 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.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 {
    +  def run(
    +    sparkSession: SparkSession,
    +    children: Seq[SparkPlan],
    +    fileCommandExec: 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],
    +    givenMetrics: Option[Map[String, SQLMetric]] = 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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(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
    +    }
    +
    +    // 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
    +    // Note: for simplifying metric values assignment, we put the values as the alphabetically
    +    // sorted of the metric keys.
    +    val metricsNames = metrics.keys.toSeq.sorted
    --- End diff --
    
    how do you guarantee the `metrics` contains avg, numFiles, etc. as it's created by `givenMetrics.getOrElse`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77719 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77719/testReport)** for PR 18159 at commit [`ba1070b`](https://github.com/apache/spark/commit/ba1070b20aa2eb507ca859f58468e6b19709364b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77632 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77632/testReport)** for PR 18159 at commit [`15fb308`](https://github.com/apache/spark/commit/15fb3082fdfa4e149c715193f3421b893f851d96).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125390211
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -47,10 +56,73 @@ 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
    --- End diff --
    
    I guess this should be rare?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125400399
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -47,10 +56,73 @@ trait RunnableCommand extends logical.Command {
     }
     
     /**
    + * A special `RunnableCommand` which writes data out and updates metrics.
    + */
    +trait DataWritingCommand extends RunnableCommand {
    --- End diff --
    
    Ok. Sure.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77616 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77616/testReport)** for PR 18159 at commit [`2ff2fa1`](https://github.com/apache/spark/commit/2ff2fa13bc0b46a1074c2814ad69f23516664968).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    @cloud-fan @rxin I've removed about 200+ lines from this. Please take a look if you have time. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120000935
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -17,38 +17,97 @@
     
     package org.apache.spark.sql.execution.command
     
    +import scala.collection.mutable
    +
     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}
     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._
     
     /**
    - * 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 {
    +
    +  /**
    +   * 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.
    --- End diff --
    
    @cloud-fan I removed the part of specified metrics per file/partition. Now it is about 600 lines, and about 180 lines are tests. Do you think it's okay?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125158551
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -19,24 +19,65 @@ 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}
     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 {
    +
    +  // 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
    --- End diff --
    
    Oh. got it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    @adrian-ionescu Thanks for the review. I've addressed the above comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125151880
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -19,24 +19,65 @@ 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}
     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 {
    +
    +  // 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
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  protected def callbackMetricsUpdater(writeSummaries: Seq[ExecutedWriteSummary]): Unit = {
    --- End diff --
    
    I think it's more reasonable to do this in `InsertIntoHadoopFsRelation`. If you are worried about duplicated code,  we can create a trait for `InsertIntoHadoopFsRelation` and `InsertIntoHive`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r126015755
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -314,21 +339,40 @@ object FileFormatWriter extends Logging {
     
               recordsInFile = 0
               releaseResources()
    +          numOutputRows += recordsInFile
               newOutputWriter(fileCounter)
             }
     
             val internalRow = iter.next()
    +        val startTime = System.nanoTime()
             currentWriter.write(internalRow)
    +        timeOnCurrentFile += (System.nanoTime() - startTime)
    --- End diff --
    
    We should absolutely NOT do per row tracking.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77714 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77714/testReport)** for PR 18159 at commit [`e8acb49`](https://github.com/apache/spark/commit/e8acb49300a2cd497e5eacd36945ddaf04ebdaa8).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77713 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77713/testReport)** for PR 18159 at commit [`3137566`](https://github.com/apache/spark/commit/3137566ee6dd6fc9cc17abd1085229e1d781bb32).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77712 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77712/testReport)** for PR 18159 at commit [`32f49ea`](https://github.com/apache/spark/commit/32f49eae328c97e3be1d0492d000c83ea06bb11d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    thanks, merging to master! we can have follow-up PRs if @rxin has more thoughts about https://github.com/apache/spark/pull/18159#discussion_r125461897


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120115651
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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}
    +
    +/**
    + * 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`,
    + * save the result to prevent multiple executions, and record necessary metrics for UI.
    + */
    +case class WrittenFileCommandExec(
    --- End diff --
    
    for consistency, better call this `<traitName>Exec`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77592 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77592/testReport)** for PR 18159 at commit [`d1a227e`](https://github.com/apache/spark/commit/d1a227e16dd31059e26b0a306eded7378c88327b).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Also, if I understand correctly, for metrics such as "bytes of written files", "number of output rows" you currently show totals. It would be good to include avg/min/max per file and per partition. This is all to make it easier to identify data skew issues.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77710/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125389753
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -314,21 +339,40 @@ object FileFormatWriter extends Logging {
     
               recordsInFile = 0
               releaseResources()
    +          numOutputRows += recordsInFile
               newOutputWriter(fileCounter)
             }
     
             val internalRow = iter.next()
    +        val startTime = System.nanoTime()
             currentWriter.write(internalRow)
    +        timeOnCurrentFile += (System.nanoTime() - startTime)
    --- End diff --
    
    instead of tracking the time here, how about we do it in `newOutputWriter`?
    ```
    var startTime = -1
    def newOutputWriter {
      if (startTime == -1) {
        startTime = System.nanoTime()
      } else {
        val currentTime = System.nanoTime()
        totalWritingTime += currentTime - startTime
        startTime = currentTime
      }
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77708 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77708/testReport)** for PR 18159 at commit [`778007d`](https://github.com/apache/spark/commit/778007d354030648e379040d7bb647553422a031).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78968 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78968/testReport)** for PR 18159 at commit [`1defb3a`](https://github.com/apache/spark/commit/1defb3ad063b975559dfc14e7fc791e837b840e8).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77719/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77632/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120370076
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    +
    +  /**
    +   * 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 {
    +    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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: Map[String, SQLMetric])
    +      (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
    +    }
    +
    +    // 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 = 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 = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
    +    SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metricsNames.map(metrics(_)))
    +  }
    +
    +  def run(
    +    sparkSession: SparkSession,
    +    children: Seq[SparkPlan],
    +    metrics: Map[String, SQLMetric],
    +    metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row]
    --- End diff --
    
    I understand that, but I think it can still be simplified. See below: https://github.com/apache/spark/pull/18159#discussion_r120359550


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120116201
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/WrittenFileCommandExec.scala ---
    @@ -0,0 +1,103 @@
    +/*
    + * 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}
    +
    +/**
    + * A logical command specialized for writing data out. `WriteOutFileCommand`s are
    + * wrapped in `WrittenFileCommandExec` during execution.
    + */
    +trait WriteOutFileCommand extends logical.Command {
    --- End diff --
    
    FileWritingCommand ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77789 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77789/testReport)** for PR 18159 at commit [`9b87b4f`](https://github.com/apache/spark/commit/9b87b4f306ff5f46f0795ff40712e71698cddab7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class FileWritingCommandExec(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78969 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78969/testReport)** for PR 18159 at commit [`32f4abf`](https://github.com/apache/spark/commit/32f4abf64263a16ef9c0cefd55990d30cd93ce38).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125213106
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala ---
    @@ -53,11 +55,22 @@ 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 lazy val metrics: Map[String, SQLMetric] = {
    --- End diff --
    
    can we move this to the parent trait?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    @adrian-ionescu Thanks! That's good points. I'll update this based on your suggestion.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120129166
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -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(_ => ()))
    --- End diff --
    
    This looks like a hack and you did it in 3 different places.
    There has to be a way to get it to work via `.execute()` and revert this..
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120123638
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -58,15 +56,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]
    --- End diff --
    
    By making this abstract, you ended up writing two very similar implementations, which only differ in the arguments to `cmd.run()`
    Consider extracting out only the differing part into an abstract function and then have this be something like
    ```
      val converter = CatalystTypeConverters.createToCatalystConverter(schema) 
      val rows = <abstract_function_call>
      rows.map(converter(_).asInstanceOf[InternalRow])
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r119999816
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -17,38 +17,97 @@
     
     package org.apache.spark.sql.execution.command
     
    +import scala.collection.mutable
    +
     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}
     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._
     
     /**
    - * 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 {
    +
    +  /**
    +   * 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.
    --- End diff --
    
    shall we just add general metrics first? I hope this can make the PR smaller...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    ping @cloud-fan @gatorsmile for review.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r126039846
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -314,21 +339,40 @@ object FileFormatWriter extends Logging {
     
               recordsInFile = 0
               releaseResources()
    +          numOutputRows += recordsInFile
               newOutputWriter(fileCounter)
             }
     
             val internalRow = iter.next()
    +        val startTime = System.nanoTime()
             currentWriter.write(internalRow)
    +        timeOnCurrentFile += (System.nanoTime() - startTime)
    --- End diff --
    
    Ok. I'll get rid of per row tracking in a follow-up PR. Thanks @rxin.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r125217117
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -273,12 +271,36 @@ object FileFormatWriter extends Logging {
        * automatically trigger task aborts.
        */
       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.
    +     * The data structures used to measure metrics during writing.
          */
    -    def execute(iterator: Iterator[InternalRow]): Set[String]
    +    protected val writingTimePerFile: mutable.ArrayBuffer[Long] = mutable.ArrayBuffer.empty
    --- End diff --
    
    Now I exclude the zero values of writing time (actually < 1ms). So I actually don't count totoalWritingTime / numFiles.
    
    totoalWritingTime / numFiles can easily result a zero writing time, for example the total writing time for (1ms, 1ms, 0ms, 0ms) is 2ms, and totoalWritingTime / numFiles = 2ms / 4 = 0ms (the smallest unit is 1ms).
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78803 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78803/testReport)** for PR 18159 at commit [`8926156`](https://github.com/apache/spark/commit/89261564a6251b4eeebff6f0455536c20499c248).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77789/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78777 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78777/testReport)** for PR 18159 at commit [`a7e53bb`](https://github.com/apache/spark/commit/a7e53bbc6116acfc6f958c062095e3c74fa4195f).
     * This patch passes all tests.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79084/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #79084 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79084/testReport)** for PR 18159 at commit [`8380023`](https://github.com/apache/spark/commit/8380023ab96fa6febe19918da9284fe14aea1f53).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77592 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77592/testReport)** for PR 18159 at commit [`d1a227e`](https://github.com/apache/spark/commit/d1a227e16dd31059e26b0a306eded7378c88327b).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77709/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77710 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77710/testReport)** for PR 18159 at commit [`9819f01`](https://github.com/apache/spark/commit/9819f0103a15dd948c049eb7130f577f084b28e4).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77717/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120319308
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    +
    +  /**
    +   * 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 {
    +    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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: Map[String, SQLMetric])
    +      (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
    +    }
    +
    +    // 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.
    --- End diff --
    
    Seems arbitrary to exclude 0 values. 1ms values also bring the average down, yet you're keeping those.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120332795
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    --- End diff --
    
    It's not clear to me why we need this and the comment is not helpful.
    Looks to me like this is currently either null/None, or the map that's defined below, in `def metrics`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77787/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77765 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77765/testReport)** for PR 18159 at commit [`0af718d`](https://github.com/apache/spark/commit/0af718d15ed9c6bcf4e8de19528affdc492d1257).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #18159: [SPARK-20703][SQL] Associate metrics with data wr...

Posted by adrian-ionescu <gi...@git.apache.org>.
Github user adrian-ionescu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120311829
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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 {
    +
    +  // 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
    +  }
    +
    +  /**
    +   * 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 {
    +    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")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing operation.
    +   */
    +  private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: Map[String, SQLMetric])
    +      (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
    +    }
    +
    +    // 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 = metrics.keys.toSeq.sorted
    +    val metricsValues = Seq(writingTime, numFiles, totalNumBytes, totalNumOutput, numPartitions)
    +    metricsNames.zip(metricsValues).foreach(x => metrics(x._1).add(x._2))
    --- End diff --
    
    This doesn't make sense if `_externalMetrics` are provided, as there's no guarantee that the user-provided map is exactly as you expect. It just happens to work when it's an empty map, but it's very fragile, not to mention the fact that it relies on the alphabetical ordering of the keys.. Please rework this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #78780 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78780/testReport)** for PR 18159 at commit [`8926156`](https://github.com/apache/spark/commit/89261564a6251b4eeebff6f0455536c20499c248).
     * This patch **fails due to an unknown error code, -10**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL][WIP] Associate metrics with data writ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77584/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #18159: [SPARK-20703][SQL] Associate metrics with data writes on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18159
  
    **[Test build #77721 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77721/testReport)** for PR 18159 at commit [`775d2c2`](https://github.com/apache/spark/commit/775d2c284666238e25cbe4cdaf95677fde358eb4).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org