At this point I recommend that new applications are built using structured
streaming. The engine was GA-ed as of Spark 2.2 and I know of several very
large (trillions of records) production jobs that are running in Structured
Streaming.  All of our production pipelines at databricks are written using
structured streaming as well.

Regarding the comparison with RDDs: The situation here is different than
when thinking about batch DataFrames vs. RDDs.  DataFrames are "just" a
higher-level abstraction on RDDs.  Structured streaming is a completely new
implementation that does not use DStreams at all, but instead directly runs
jobs using RDDs.  The advantages over DStreams include:
 - The ability to start and stop individual queries (rather than needing to
start/stop a separate StreamingContext)
 - The ability to upgrade your stream and still start from an existing
checkpoint
 - Support for working with Spark SQL data sources (json, parquet, etc)
 - Higher level APIs (DataFrames and SQL) and lambda functions (Datasets)
 - Support for event time aggregation

At this point, with the addition of mapGroupsWithState and
flatMapGroupsWithState, I think we should be at feature parity with
DStreams (and the state store does incremental checkpoints that are more
efficient than the DStream store).  However if there are applications you
are having a hard time porting over, please let us know!

On Wed, Jan 31, 2018 at 5:42 AM, vijay.bvp <bvpsa...@gmail.com> wrote:

> here is my two cents, experts please correct me if wrong
>
> its important to understand why one over other and for what kind of use
> case. There might be sometime in future where low level API's are
> abstracted
> and become legacy but for now in Spark RDD API is the core and low level
> API, all higher APIs translate to RDD ultimately,  and RDD's are immutable.
>
> https://spark.apache.org/docs/latest/structured-streaming-
> programming-guide.html#unsupported-operations
> these are things that are not supported and this list needs to be validated
> with the use case you have.
>
> From my experience Structured Streaming is still new and DStreams API is a
> matured API.
> some things that are missing or need to explore more.
>
> watermarking/windowing based on no of records in a particular window
>
> assuming you have watermark and windowing on event time of the data,  the
> resultant dataframe is grouped data set, only thing you can do is run
> aggregate functions. you can't simply use that output as another dataframe
> and manipulate. There is a custom aggregator but I feel its limited.
>
> https://spark.apache.org/docs/latest/structured-streaming-
> programming-guide.html#arbitrary-stateful-operations
> There is option to do stateful operations, using GroupState where the
> function gets iterator of events for that window. This is the closest
> access
> to StateStore a developer could get.
> This arbitrary state that programmer could keep across invocations has its
> limitations as such how much state we could keep?, is that state stored in
> driver memory? What happens if the spark job fails is this checkpointed or
> restored?
>
> thanks
> Vijay
>
>
>
> --
> Sent from: http://apache-spark-user-list.1001560.n3.nabble.com/
>
> ---------------------------------------------------------------------
> To unsubscribe e-mail: user-unsubscr...@spark.apache.org
>
>

Reply via email to