Skip to content

Conversation

@gaborgsomogyi
Copy link
Contributor

What changes were proposed in this pull request?

Output metrics were not filled when parquet sink used.

This PR fixes this problem by passing a BasicWriteJobStatsTracker in FileStreamSink.

How was this patch tested?

Additional unit test added.

@SparkQA
Copy link

SparkQA commented Mar 6, 2018

Test build #87992 has finished for PR 20745 at commit 55aa8bc.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • case class CatalogColumnStat(
  • case class LocalRelation(
  • case class StreamingDataSourceV2Relation(

@gaborgsomogyi
Copy link
Contributor Author

Re-created the PR because something got stuck in the previous one.
cc @tdas @zsxwing @vanzin

@gaborgsomogyi
Copy link
Contributor Author

ping @koeninger

Copy link
Member

@zsxwing zsxwing left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The streaming changes look good to me.

cc @cloud-fan to review others.

failAfter(streamingTimeout) {
query.processAllAvailable()
}

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: it's better to add the below statement here to avoid flakiness.

spark.sparkContext.listenerBus.waitUntilEmpty(streamingTimeout.toMillis)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added.

query.processAllAvailable()
}

assert(numTasks === 2)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would just check numTasks > 0 since it depends on the configurations and the number of CPU codes.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed.

@SparkQA
Copy link

SparkQA commented Mar 16, 2018

Test build #88323 has finished for PR 20745 at commit 315cdd6.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

val inputData = MemoryStream[String]
val df = inputData.toDF()

val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should use withTempDir to clean up the temp directory at the end

@cloud-fan
Copy link
Contributor

LGTM, can you also attach a web UI SQL tab screenshot? thanks!

@gaborgsomogyi
Copy link
Contributor Author

screen shot 2018-03-20 at 22 37 37

@gaborgsomogyi
Copy link
Contributor Author

screen shot 2018-03-20 at 22 46 50

@cloud-fan
Copy link
Contributor

hmm, the screenshot you attached is not parquet, is it?

@gaborgsomogyi
Copy link
Contributor Author

I've started history server then executed test("SPARK-23288 writing and checking output metrics") with spark.eventLog.enabled parameter. Now there is only one entry in the application area. Have I made something wrong?

override def onTaskEnd(taskEnd: SparkListenerTaskEnd) {
val outputMetrics = taskEnd.taskMetrics.outputMetrics
recordsWritten += outputMetrics.recordsWritten
bytesWritten += outputMetrics.bytesWritten
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Without registering statsTrackers output metrics are not filled and assert(recordsWritten === 5) and assert(bytesWritten > 0) blows up.

@SparkQA
Copy link

SparkQA commented Mar 21, 2018

Test build #88453 has finished for PR 20745 at commit 214cddc.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gaborgsomogyi
Copy link
Contributor Author

retest this please

@SparkQA
Copy link

SparkQA commented Mar 21, 2018

Test build #88457 has finished for PR 20745 at commit 214cddc.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

thanks, merging to master/2.3!

@asfgit asfgit closed this in 918c7e9 Mar 21, 2018
asfgit pushed a commit that referenced this pull request Mar 21, 2018
## What changes were proposed in this pull request?

Output metrics were not filled when parquet sink used.

This PR fixes this problem by passing a `BasicWriteJobStatsTracker` in `FileStreamSink`.

## How was this patch tested?

Additional unit test added.

Author: Gabor Somogyi <[email protected]>

Closes #20745 from gaborgsomogyi/SPARK-23288.

(cherry picked from commit 918c7e9)
Signed-off-by: Wenchen Fan <[email protected]>
peter-toth pushed a commit to peter-toth/spark that referenced this pull request Oct 6, 2018
## What changes were proposed in this pull request?

Output metrics were not filled when parquet sink used.

This PR fixes this problem by passing a `BasicWriteJobStatsTracker` in `FileStreamSink`.

## How was this patch tested?

Additional unit test added.

Author: Gabor Somogyi <[email protected]>

Closes apache#20745 from gaborgsomogyi/SPARK-23288.

(cherry picked from commit 918c7e9)
Signed-off-by: Wenchen Fan <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants