I think managing a lifecycle around the existing MR OutputFormat's makes a
lot of sense for the streaming environment. Having them unified in the
Flink Streaming API will make users' lives much better, and keeps the
streaming world open to the large existing ecosystem.
On Tue, Feb 9, 2016 at 6:13
Most of the problems we had with OutputFormats is that many were
implemented in a batchy way:
- They buffer data and write large chunks at some points
- They need the "close()" call before any consistent result is guaranteed
That is mostly the case for FileOutputFormats, but not exclusively (s
I think you have a point, Nick. OutputFormats on its own have the same
fault-tolerance semantics as SinkFunctions. What kind of failure semantics
they guarantee depends on the actual implementation. For instance, the
RMQSource has exactly-once semantics but the RMQSink currently does not. If
you ca
I think this depends on the implementation of the OutputFormat. For
instance, an HBase, Cassandra or ES OF will handle most operations as
idempotent when the scheme is designed appropriately.
You are (rightly) focusing on FileOF's, which also depend on the semantics
of their implementation. MR alw
Hi,
one problem that I see with OutputFormat is that they are not made for a
streaming world. By this, I mean that they don’t handle failure well and don’t
consider fault-torelant streaming, i.e. exactly once semantics. For example,
what would be expected to happen if a job with a FileOutputForm
On Mon, Feb 8, 2016 at 9:51 AM, Maximilian Michels wrote:
> Changing the class hierarchy would break backwards-compatibility of the
> API. However, we could add another method to DataStream to easily use
> OutputFormats in streaming.
>
Indeed, that's why I suggested deprecating one and moving to
Changing the class hierarchy would break backwards-compatibility of the
API. However, we could add another method to DataStream to easily use
OutputFormats in streaming.
How did you write your adapter? I came up with the one below. Admittedly,
it is sort of a hack but works fine. By the way, you c
In my case, I have my application code that is calling addSink, for which
I'm writing a test that needs to use LocalCollectionOutputFormat. Having
two separate class hierarchies is not helpful, hence the adapter. Much of
this code already exists in the implementation of FileSinkFunction, so the
pro
Hi Nick,
SinkFunction just implements user-defined functions on incoming
elements. OutputFormat offers more lifecycle methods. Thus it is a
more powerful interface. The OutputFormat originally comes from the
batch API, whereas the SinkFunction originates from streaming. Those
were more separate co
Heya,
Is there a plan to consolidate these two interfaces? They appear to provide
identical functionality, differing only in lifecycle management. I found
myself writing an adaptor so I can consume an OutputFormat where a
SinkFunction is expected; there's not much to it. This seems like code that
10 matches
Mail list logo