Github user tdas commented on a diff in the pull request: https://github.com/apache/spark/pull/13342#discussion_r66532056 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala --- @@ -401,6 +381,53 @@ final class DataFrameWriter private[sql](df: DataFrame) { } /** + * :: Experimental :: + * Starts the execution of the streaming query, which will continually send results to the given + * [[ForeachWriter]] as new data arrives. The returned [[ContinuousQuery]] object can be used to + * interact with the stream. + * + * @since 2.0.0 + */ + @Experimental + def foreach(writer: ForeachWriter[T]): ContinuousQuery = { + assertNotBucketed("foreach") + assertStreaming( + "foreach() on streaming Datasets and DataFrames can only be called on continuous queries") + + val queryName = extraOptions.getOrElse("queryName", StreamExecution.nextName) + val sink = new ForeachSink[T](ds.sparkSession.sparkContext.clean(writer))(ds.exprEnc) + df.sparkSession.sessionState.continuousQueryManager.startQuery( + queryName, + getCheckpointLocation(queryName, required = false), + df, + sink, + outputMode, + trigger) + } + + /** + * Returns the checkpointLocation for a query. If `required` is `true` but the checkpoint + * location is not set, [[AnalysisException]] will be thrown. If `required` is `false`, a temp + * folder will be created if the checkpoint location is not set. + */ + private def getCheckpointLocation(queryName: String, required: Boolean): String = { + extraOptions.get("checkpointLocation").map { userSpecified => + new Path(userSpecified).toUri.toString + }.orElse { + df.sparkSession.conf.get(SQLConf.CHECKPOINT_LOCATION).map { location => + new Path(location, queryName).toUri.toString + } + }.getOrElse { + if (required) { + throw new AnalysisException("checkpointLocation must be specified either " + + "through option() or SQLConf") --- End diff -- `option()` --> `option("checkpointLocation", ...)` `SQLConf` --> `sqlContext.conf......` (complete it) Makes it easier for the user
--- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org