@@ -133,9 +133,6 @@ setMethod("schema",
133133# '
134134# ' Print the logical and physical Catalyst plans to the console for debugging.
135135# '
136- # ' @param x a SparkDataFrame.
137- # ' @param extended Logical. If extended is FALSE, explain() only prints the physical plan.
138- # ' @param ... further arguments to be passed to or from other methods.
139136# ' @family SparkDataFrame functions
140137# ' @aliases explain,SparkDataFrame-method
141138# ' @rdname explain
@@ -3515,3 +3512,104 @@ setMethod("getNumPartitions",
35153512 function (x ) {
35163513 callJMethod(callJMethod(x @ sdf , " rdd" ), " getNumPartitions" )
35173514 })
3515+
3516+ # ' isStreaming
3517+ # '
3518+ # ' Returns TRUE if this SparkDataFrame contains one or more sources that continuously return data
3519+ # ' as it arrives.
3520+ # '
3521+ # ' @param x A SparkDataFrame
3522+ # ' @return TRUE if this SparkDataFrame is from a streaming source
3523+ # ' @family SparkDataFrame functions
3524+ # ' @aliases isStreaming,SparkDataFrame-method
3525+ # ' @rdname isStreaming
3526+ # ' @name isStreaming
3527+ # ' @seealso \link{read.stream} \link{write.stream}
3528+ # ' @export
3529+ # ' @examples
3530+ # '\dontrun{
3531+ # ' sparkR.session()
3532+ # ' df <- read.stream("socket", host = "localhost", port = 9999)
3533+ # ' isStreaming(df)
3534+ # ' }
3535+ # ' @note isStreaming since 2.2.0
3536+ # ' @note experimental
3537+ setMethod ("isStreaming ",
3538+ signature(x = " SparkDataFrame" ),
3539+ function (x ) {
3540+ callJMethod(x @ sdf , " isStreaming" )
3541+ })
3542+
3543+ # ' Write the streaming SparkDataFrame to a data source.
3544+ # '
3545+ # ' The data source is specified by the \code{source} and a set of options (...).
3546+ # ' If \code{source} is not specified, the default data source configured by
3547+ # ' spark.sql.sources.default will be used.
3548+ # '
3549+ # ' Additionally, \code{outputMode} specifies how data of a streaming SparkDataFrame is written to a
3550+ # ' output data source. There are three modes:
3551+ # ' \itemize{
3552+ # ' \item append: Only the new rows in the streaming SparkDataFrame will be written out. This
3553+ # ' output mode can be only be used in queries that do not contain any aggregation.
3554+ # ' \item complete: All the rows in the streaming SparkDataFrame will be written out every time
3555+ # ' there are some updates. This output mode can only be used in queries that
3556+ # ' contain aggregations.
3557+ # ' \item update: Only the rows that were updated in the streaming SparkDataFrame will be written
3558+ # ' out every time there are some updates. If the query doesn't contain aggregations,
3559+ # ' it will be equivalent to \code{append} mode.
3560+ # ' }
3561+ # '
3562+ # ' @param df a streaming SparkDataFrame.
3563+ # ' @param source a name for external data source.
3564+ # ' @param outputMode one of 'append', 'complete', 'update'.
3565+ # ' @param ... additional argument(s) passed to the method.
3566+ # '
3567+ # ' @family SparkDataFrame functions
3568+ # ' @seealso \link{read.stream}
3569+ # ' @aliases write.stream,SparkDataFrame-method
3570+ # ' @rdname write.stream
3571+ # ' @name write.stream
3572+ # ' @export
3573+ # ' @examples
3574+ # '\dontrun{
3575+ # ' sparkR.session()
3576+ # ' df <- read.stream("socket", host = "localhost", port = 9999)
3577+ # ' isStreaming(df)
3578+ # ' wordCounts <- count(group_by(df, "value"))
3579+ # '
3580+ # ' # console
3581+ # ' q <- write.stream(wordCounts, "console", outputMode = "complete")
3582+ # ' # text stream
3583+ # ' q <- write.stream(df, "text", path = "/home/user/out", checkpointLocation = "/home/user/cp")
3584+ # ' # memory stream
3585+ # ' q <- write.stream(wordCounts, "memory", queryName = "outs", outputMode = "complete")
3586+ # ' head(sql("SELECT * from outs"))
3587+ # ' queryName(q)
3588+ # '
3589+ # ' stopQuery(q)
3590+ # ' }
3591+ # ' @note write.stream since 2.2.0
3592+ # ' @note experimental
3593+ setMethod ("write.stream ",
3594+ signature(df = " SparkDataFrame" ),
3595+ function (df , source = NULL , outputMode = NULL , ... ) {
3596+ if (! is.null(source ) && ! is.character(source )) {
3597+ stop(" source should be character, NULL or omitted. It is the data source specified " ,
3598+ " in 'spark.sql.sources.default' configuration by default." )
3599+ }
3600+ if (! is.null(outputMode ) && ! is.character(outputMode )) {
3601+ stop(" outputMode should be charactor or omitted." )
3602+ }
3603+ if (is.null(source )) {
3604+ source <- getDefaultSqlSource()
3605+ }
3606+ options <- varargsToStrEnv(... )
3607+ write <- handledCallJMethod(df @ sdf , " writeStream" )
3608+ write <- callJMethod(write , " format" , source )
3609+ if (! is.null(outputMode )) {
3610+ write <- callJMethod(write , " outputMode" , outputMode )
3611+ }
3612+ write <- callJMethod(write , " options" , options )
3613+ ssq <- handledCallJMethod(write , " start" )
3614+ streamingQuery(ssq )
3615+ })
0 commit comments