You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Aljoscha Krettek <al...@apache.org> on 2018/10/31 16:22:12 UTC

[DISCUSS] FLIP-27: Refactor Source Interface

Hi All,

In order to finally get the ball rolling on the new source interface that we have discussed for so long I finally created a FLIP: https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface

I cc'ed Thomas and Jamie because of the ongoing work/discussion about adding per-partition watermark support to the Kinesis source and because this would enable generic implementation of event-time alignment for all sources. Maybe we need another FLIP for the event-time alignment part, especially the part about information sharing between operations (I'm not calling it state sharing because state has a special meaning in Flink).

Please discuss away!

Aljoscha



Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Thanks for the explanation, Piotr,

I agree that the completable future solution would work for single-threaded
readers. From API perspective, returning a completable future means the
reader must have an internal thread to complete that future. I was actually
thinking of some sources that are "thread-less" like the Kafka consumers.

The Kafka consumer itself does not have an internal thread except the
heartbeat thread which does nothing but heartbeat. So a Kafka consumer
relies on the user thread to call poll() to make progress. Inside the
poll() methods, things are asynchronous (with very few exceptions such as
rebalance which cannot be interrupted), receiving responses, sending
FetchRequests, Heartbeat, etc. So technically speaking, the consumer itself
is "thread-less",

In that case, if the consumer provides a isBlocked() method and return a
CompletableFuture, unless the users call poll() again, that
CompletableFuture will never be completed because the consumer itself does
not have any thread to complete that future. Instead, it relies on the user
thread, which is holding the future, to complete that same future.

While it looks counter intuitive at the first glance, such thread-less
readers could be more efficient in some cases. For example, if there are
hundreds of readers in a single task, the thread-less readers can be
managed by a single thread. That thread just need to call poll() on each
readers. On the other hand, for the single-threaded readers, there will be
one thread per reader, hundreds of threads in total. From this perspective,
such thread-less readers can do pretty well in terms of limiting the number
of threads. And users can also choose to use a thread pool to manage these
thread-less readers if they wish. And it is also trivial to wrap such
readers to create a single-threaded reader.

BTW, regarding the isBlock() method, I have a few more questions.
21, Is a method isReady() with boolean as a return value equivalent?
Personally I found it is a little bit confusing in what is supposed to be
returned when the future is completed.
22. if the implementation of isBlocked() is optional, how do the callers
know whether the method is properly implemented or not? Does not
implemented mean it always return a completed future?

Thanks,

Jiangjie (Becket) Qin




On Sat, Nov 3, 2018 at 4:30 AM Piotr Nowojski <pi...@data-artisans.com>
wrote:

> Hey Becket,
>
> Re 2.
>
> With:
>
> If source is purely single threaded and blocking, then it could be
> implemented in the following way:
>
> /*
> * Return a future, which when completed means that source has more data
> and getNext() will not block.
> * If you wish to use benefits of non blocking connectors, please implement
> this method appropriately.
> */
> CompletableFuture<?> isBlocked() {
>         return CompletableFuture.completedFuture(null); // this would be
> the default behaviour, so user wouldn’t need to override this at all
> }
>
> T getNext() {
>         // do some blocking reading operation
>         return result;
> }
>
> Implementing `isBlocked` doesn’t have to be mandatory. It’s more like an
> optional optimisation that some connectors might provide.
>
> Providing non blocking `poll` method doesn’t solve the problem of actually
> limiting the number of active threads. One of the potential benefits of
> `CompletableFuture<?> isBlocked()` is that we could have a fixed size pool
> of worker threads. Worker thread could pick a non blocked task that’s
> waiting to be executed and to this `CompletableFuture<?>` would be needed
> to juggle between blocked/active state. Other potential side benefit could
> be for reporting in UI/metrics which tasks are blocked (kind of like
> current back pressure monitoring).
>
> Maybe such extension could use of some PoC that would (or not) show some
> benefits.
>
> Piotrek
>
> > On 1 Nov 2018, at 19:29, Becket Qin <be...@gmail.com> wrote:
> >
> > Thanks for the FLIP, Aljoscha.
> >
> > The proposal makes sense to me. Separating the split discovery and
> > consumption is very useful as it enables Flink to better manage the
> sources.
> >
> > Looking at the interface, I have a few questions:
> > 1. *SplitEnumerator*.*discoverNewSplits()* seems assuming that the number
> > of splits can only increase, In your example, the source was Kafka, so
> the
> > assumption was true. But I am wondering are there case that the number of
> > splits can decrease?
> > 2. I agree with Piotr that we need to be careful about potentially
> blocking
> > implementations. However, it is not clear to me how does the completable
> > future work if the underlying reader does not have its own thread (e.g. a
> > Kafka consumer). In that case, the future will never be completed unless
> > the caller thread touches the reader again. I am wondering if the
> following
> > interfaces for the reader makes sense:
> >    boolean isDone(); // Whether the source has more records.
> >    T poll(); // non-blocking read. We can add a timeout if needed.
> >    T take(); // blocking read;
> > This seems more intuitive to people who are familiar with existing
> > convention of poll() and take(). And with the non-blocking poll() we
> could
> > have an nio Selector-like API when there are multiple splits.
> >
> > BTW, it would be really helpful if there is some Java doc describing the
> > behavior of the the interfaces in the FLIP.
> >
> > Thanks again for the great proposal.
> >
> > Jiangjie (Becket) Qin
> >
> > On Thu, Nov 1, 2018 at 6:08 PM Piotr Nowojski <pi...@data-artisans.com>
> > wrote:
> >
> >> Hi,
> >>
> >> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
> >> possible improvements. I have one proposal. Instead of having a method:
> >>
> >> boolean advance() throws IOException;
> >>
> >> I would replace it with
> >>
> >> /*
> >> * Return a future, which when completed means that source has more data
> >> and getNext() will not block.
> >> * If you wish to use benefits of non blocking connectors, please
> >> implement this method appropriately.
> >> */
> >> default CompletableFuture<?> isBlocked() {
> >>        return CompletableFuture.completedFuture(null);
> >> }
> >>
> >> And rename `getCurrent()` to `getNext()`.
> >>
> >> Couple of arguments:
> >> 1. I don’t understand the division of work between `advance()` and
> >> `getCurrent()`. What should be done in which, especially for connectors
> >> that handle records in batches (like Kafka) and when should you call
> >> `advance` and when `getCurrent()`.
> >> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in the
> >> future to have asynchronous/non blocking connectors and more efficiently
> >> handle large number of blocked threads, without busy waiting. While at
> the
> >> same time it doesn’t add much complexity, since naive connector
> >> implementations can be always blocking.
> >> 3. This also would allow us to use a fixed size thread pool of task
> >> executors, instead of one thread per task.
> >>
> >> Piotrek
> >>
> >>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
> wrote:
> >>>
> >>> Hi All,
> >>>
> >>> In order to finally get the ball rolling on the new source interface
> >> that we have discussed for so long I finally created a FLIP:
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>
> >>> I cc'ed Thomas and Jamie because of the ongoing work/discussion about
> >> adding per-partition watermark support to the Kinesis source and because
> >> this would enable generic implementation of event-time alignment for all
> >> sources. Maybe we need another FLIP for the event-time alignment part,
> >> especially the part about information sharing between operations (I'm
> not
> >> calling it state sharing because state has a special meaning in Flink).
> >>>
> >>> Please discuss away!
> >>>
> >>> Aljoscha
> >>>
> >>>
> >>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@data-artisans.com>.
Hey Becket,

Re 2.

With:

If source is purely single threaded and blocking, then it could be implemented in the following way:

/* 
* Return a future, which when completed means that source has more data and getNext() will not block.
* If you wish to use benefits of non blocking connectors, please implement this method appropriately.
*/
CompletableFuture<?> isBlocked() {
	return CompletableFuture.completedFuture(null); // this would be the default behaviour, so user wouldn’t need to override this at all 
}

T getNext() {
	// do some blocking reading operation
	return result;
}

Implementing `isBlocked` doesn’t have to be mandatory. It’s more like an optional optimisation that some connectors might provide.

Providing non blocking `poll` method doesn’t solve the problem of actually limiting the number of active threads. One of the potential benefits of `CompletableFuture<?> isBlocked()` is that we could have a fixed size pool of worker threads. Worker thread could pick a non blocked task that’s waiting to be executed and to this `CompletableFuture<?>` would be needed to juggle between blocked/active state. Other potential side benefit could be for reporting in UI/metrics which tasks are blocked (kind of like current back pressure monitoring).

Maybe such extension could use of some PoC that would (or not) show some benefits.

Piotrek 

> On 1 Nov 2018, at 19:29, Becket Qin <be...@gmail.com> wrote:
> 
> Thanks for the FLIP, Aljoscha.
> 
> The proposal makes sense to me. Separating the split discovery and
> consumption is very useful as it enables Flink to better manage the sources.
> 
> Looking at the interface, I have a few questions:
> 1. *SplitEnumerator*.*discoverNewSplits()* seems assuming that the number
> of splits can only increase, In your example, the source was Kafka, so the
> assumption was true. But I am wondering are there case that the number of
> splits can decrease?
> 2. I agree with Piotr that we need to be careful about potentially blocking
> implementations. However, it is not clear to me how does the completable
> future work if the underlying reader does not have its own thread (e.g. a
> Kafka consumer). In that case, the future will never be completed unless
> the caller thread touches the reader again. I am wondering if the following
> interfaces for the reader makes sense:
>    boolean isDone(); // Whether the source has more records.
>    T poll(); // non-blocking read. We can add a timeout if needed.
>    T take(); // blocking read;
> This seems more intuitive to people who are familiar with existing
> convention of poll() and take(). And with the non-blocking poll() we could
> have an nio Selector-like API when there are multiple splits.
> 
> BTW, it would be really helpful if there is some Java doc describing the
> behavior of the the interfaces in the FLIP.
> 
> Thanks again for the great proposal.
> 
> Jiangjie (Becket) Qin
> 
> On Thu, Nov 1, 2018 at 6:08 PM Piotr Nowojski <pi...@data-artisans.com>
> wrote:
> 
>> Hi,
>> 
>> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
>> possible improvements. I have one proposal. Instead of having a method:
>> 
>> boolean advance() throws IOException;
>> 
>> I would replace it with
>> 
>> /*
>> * Return a future, which when completed means that source has more data
>> and getNext() will not block.
>> * If you wish to use benefits of non blocking connectors, please
>> implement this method appropriately.
>> */
>> default CompletableFuture<?> isBlocked() {
>>        return CompletableFuture.completedFuture(null);
>> }
>> 
>> And rename `getCurrent()` to `getNext()`.
>> 
>> Couple of arguments:
>> 1. I don’t understand the division of work between `advance()` and
>> `getCurrent()`. What should be done in which, especially for connectors
>> that handle records in batches (like Kafka) and when should you call
>> `advance` and when `getCurrent()`.
>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in the
>> future to have asynchronous/non blocking connectors and more efficiently
>> handle large number of blocked threads, without busy waiting. While at the
>> same time it doesn’t add much complexity, since naive connector
>> implementations can be always blocking.
>> 3. This also would allow us to use a fixed size thread pool of task
>> executors, instead of one thread per task.
>> 
>> Piotrek
>> 
>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org> wrote:
>>> 
>>> Hi All,
>>> 
>>> In order to finally get the ball rolling on the new source interface
>> that we have discussed for so long I finally created a FLIP:
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>> 
>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion about
>> adding per-partition watermark support to the Kinesis source and because
>> this would enable generic implementation of event-time alignment for all
>> sources. Maybe we need another FLIP for the event-time alignment part,
>> especially the part about information sharing between operations (I'm not
>> calling it state sharing because state has a special meaning in Flink).
>>> 
>>> Please discuss away!
>>> 
>>> Aljoscha
>>> 
>>> 
>> 
>> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Thanks for the FLIP, Aljoscha.

The proposal makes sense to me. Separating the split discovery and
consumption is very useful as it enables Flink to better manage the sources.

Looking at the interface, I have a few questions:
1. *SplitEnumerator*.*discoverNewSplits()* seems assuming that the number
of splits can only increase, In your example, the source was Kafka, so the
assumption was true. But I am wondering are there case that the number of
splits can decrease?
2. I agree with Piotr that we need to be careful about potentially blocking
implementations. However, it is not clear to me how does the completable
future work if the underlying reader does not have its own thread (e.g. a
Kafka consumer). In that case, the future will never be completed unless
the caller thread touches the reader again. I am wondering if the following
interfaces for the reader makes sense:
    boolean isDone(); // Whether the source has more records.
    T poll(); // non-blocking read. We can add a timeout if needed.
    T take(); // blocking read;
This seems more intuitive to people who are familiar with existing
convention of poll() and take(). And with the non-blocking poll() we could
have an nio Selector-like API when there are multiple splits.

BTW, it would be really helpful if there is some Java doc describing the
behavior of the the interfaces in the FLIP.

Thanks again for the great proposal.

Jiangjie (Becket) Qin

On Thu, Nov 1, 2018 at 6:08 PM Piotr Nowojski <pi...@data-artisans.com>
wrote:

> Hi,
>
> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
> possible improvements. I have one proposal. Instead of having a method:
>
> boolean advance() throws IOException;
>
> I would replace it with
>
> /*
>  * Return a future, which when completed means that source has more data
> and getNext() will not block.
>  * If you wish to use benefits of non blocking connectors, please
> implement this method appropriately.
>  */
> default CompletableFuture<?> isBlocked() {
>         return CompletableFuture.completedFuture(null);
> }
>
> And rename `getCurrent()` to `getNext()`.
>
> Couple of arguments:
> 1. I don’t understand the division of work between `advance()` and
> `getCurrent()`. What should be done in which, especially for connectors
> that handle records in batches (like Kafka) and when should you call
> `advance` and when `getCurrent()`.
> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in the
> future to have asynchronous/non blocking connectors and more efficiently
> handle large number of blocked threads, without busy waiting. While at the
> same time it doesn’t add much complexity, since naive connector
> implementations can be always blocking.
> 3. This also would allow us to use a fixed size thread pool of task
> executors, instead of one thread per task.
>
> Piotrek
>
> > On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org> wrote:
> >
> > Hi All,
> >
> > In order to finally get the ball rolling on the new source interface
> that we have discussed for so long I finally created a FLIP:
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >
> > I cc'ed Thomas and Jamie because of the ongoing work/discussion about
> adding per-partition watermark support to the Kinesis source and because
> this would enable generic implementation of event-time alignment for all
> sources. Maybe we need another FLIP for the event-time alignment part,
> especially the part about information sharing between operations (I'm not
> calling it state sharing because state has a special meaning in Flink).
> >
> > Please discuss away!
> >
> > Aljoscha
> >
> >
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Piotrek,

Regarding the split assignment. My hunch is that Flink might not have
enough information to assign the splits to the readers in the best way.
Even if a SplitReader says it COULD take another split, it does not mean it
is the best reader to take the split. For example, it is possible that two
physical splits are in the same host so it can be handled by a SplitReader
which already has a connection to that physical host. But the other readers
can still take care of that split if no one has an established connection.
It seems cumbersome for Flink to get involved in such details.

My understanding is that the splits returned by the source are actually
logical splits instead of physical splits. The contract between the Flink
and Source implementation is that each logical split will be read by
exactly one split reader. If there are multiple physical splits that should
be read by a single SplitReader, it is the Source implementation's
responsibility to put them into the same logical split and so Flink will
let a single split reader handle them.

For some sources, if they have additional properties such as filterable or
projectable, Flink should have additional interfaces like
FilterableSourceReader / ProjectableSourceReader, which is what we did for
the TableSource.

Thanks,

Jiangjie (Becket) Qin

On Thu, Nov 22, 2018 at 10:38 PM Piotr Nowojski <pi...@data-artisans.com>
wrote:

> Hi Becket,
>
> I think the problem is not with the split re-assignment, but with dynamic
> split discovery. We do not always know before the hand the number of splits
> (for example Kafka partition/topic discovery, but this can also happen in
> batch), while the source parallelism is fixed/known before hand.
>
> > 1. What if the SplitReader implementation cannot easily add a split to
> read on the fly?
>
> Always initiating one consumer per split will not be efficient in many
> cases. While if the connector needs to instantiate a new reader per each
> split, connector can handle this internally (addSplit() would close
> previous reader and create new one).
>
> > 2. Does Flink have to be involved in splits assignment?
>
> I think that this might be a good shared logic between different
> connectors.
>
> > @Biao,
> > If I understand correctly, the concern you raised was that a Source may
> > return a lot of splits and thus Flink may have to create a lot of fetcher
> > threads. This is a valid concern, but I cannot think of a solution to
> that.
> > After all, the SplitReaders may be written by third parties. Poor
> > implementations seem difficult to prevent.
>
> I think we can solve this and this is not as uncommon as you might think.
> In batch word, usually/often you have one split per HDFS chunk, each chunk
> being 64-256MB. With peta byte tables you end up with range from millions
> to billions of splits. This becomes a bottleneck if splits can be
> efficiently filtered out/eliminated based on some header (ORC header for
> example). In other words, if you have huge number of splits that are very
> cheap/quick to process.
>
> Piotrek
>
> > On 22 Nov 2018, at 04:54, Becket Qin <be...@gmail.com> wrote:
> >
> > Thanks Piotrek,
> >
> >> void SplitReader#addSplit(Split)
> >> boolean SplitReader#doesWantMoreSplits()
> >
> > I have two questions about this API.
> > 1. What if the SplitReader implementation cannot easily add a split to
> read
> > on the fly?
> > 2. Does Flink have to be involved in splits assignment?
> >
> > I am wondering if it would be simpler to let the enumerator indicate
> > whether a split reassignment is needed. If the answer is yes, Flink can
> > just start from the beginning to get all the splits and create one reader
> > per split. This might be a little more expensive than dynamically adding
> a
> > split to a reader, but given that the splits change should be rare, it is
> > probably acceptable.
> >
> > In the Kafka case, the SplitT may just be a consumer. The enumerator will
> > simply check if the topic has new partitions to be assigned to this
> reader.
> >
> > @Biao,
> > If I understand correctly, the concern you raised was that a Source may
> > return a lot of splits and thus Flink may have to create a lot of fetcher
> > threads. This is a valid concern, but I cannot think of a solution to
> that.
> > After all, the SplitReaders may be written by third parties. Poor
> > implementations seem difficult to prevent.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > On Wed, Nov 21, 2018 at 10:13 PM Piotr Nowojski <piotr@data-artisans.com
> >
> > wrote:
> >
> >> Hi again,
> >>
> >>> However I don't like the thread mode which starts a thread for each
> >> split.
> >>> Starting extra thread in operator is not an ideal way IMO. Especially
> >>> thread count is decided by split count. So I was wondering if there is
> a
> >>> more elegant way. Do we really want these threads in Flink core?
> >>
> >> Biao you have raised an important issue. Indeed it seems like the
> current
> >> proposal is missing something. I would guess that we need a mechanism
> for
> >> adding new splits to an already existing SplitReader and some logic to
> >> determine whether current instance can accept more splits or not. For
> >> example
> >>
> >> void SplitReader#addSplit(Split)
> >> boolean SplitReader#doesWantMoreSplits()
> >>
> >> Flink could randomly/round robin assign new splits to the SplitReaders
> >> that `doWantMoreSplits()`. Batch file readers might implement some
> custom
> >> logic in `doesWantMoreSplits()`, like one SplitReader can have at most N
> >> enqueued splits?
> >>
> >> Also what about Kafka. Isn’t it the case that one KafkaConsumer can read
> >> from multiple splits? So Kafka’s SplitReader should always return true
> from
> >> `doesWantMoreSplits()`?
> >>
> >> What do you think?
> >>
> >> Re: Becket
> >>
> >> I’m +1 for Sync and AsyncSplitReader.
> >>
> >> Piotrek
> >>
> >>> On 21 Nov 2018, at 14:49, Becket Qin <be...@gmail.com> wrote:
> >>>
> >>> Hi Aljoscha,
> >>>
> >>> Good point on the potential optimization in the source. One thing to
> >>> clarify, by "adding a minimumTimestamp()/maximumTimestamp() method pair
> >> to
> >>> the split interface", did you mean "split reader interface"? If so,
> what
> >>> should the readers do if they do not have such additional information?
> I
> >> am
> >>> wondering if it is possible to leave such optimization to the source
> >>> internal implementation.
> >>>
> >>> @all
> >>> After reading all the feedback, Biao and I talked a little bit offline.
> >> We
> >>> would like to share some new thoughts with you and see what do you
> think.
> >>>
> >>> When looking at the Source API, we were trying to answer two questions.
> >>> First of all, how would Flink use this API if someone else implemented
> >> it.
> >>> Secondly, how would the connector contributors implement the interface?
> >> How
> >>> difficult is the implementation.
> >>>
> >>> KafkaConsumer is a typical example of a thread-less reader. The idea
> was
> >> to
> >>> allow different threading model on top of it. It could be a global
> single
> >>> thread handles record fetching and processing in an event loop pattern;
> >> it
> >>> could also be one dedicated fetcher thread for each consumer and a
> >> separate
> >>> thread pool for record processing. The API gives the freedom of picking
> >> up
> >>> threading model to the users. To answer the first question, I would
> love
> >> to
> >>> have such a source reader API so Flink can choose whatever threading
> >> model
> >>> it wants. However, implementing such an interface could be pretty
> >>> challenging and error prone.
> >>>
> >>> On the other hand, having a source reader with a naive blocking socket
> is
> >>> probably simple enough in most cases (actually sometimes this might
> even
> >> be
> >>> the most efficient way). But it does not leave much option to Flink
> other
> >>> than creating one thread per reader.
> >>>
> >>> Given the above thoughts, it might be reasonable to separate the
> >>> SplitReader API into two: SyncReader and AsyncReader. The sync reader
> >> just
> >>> has a simple blocking takeNext() API. And the AsyncReader just has a
> >>> pollNext(Callback) or Future<?> pollNext(). All the other methods are
> >>> shared by both readers and could be put into a package private parent
> >>> interface like BaseSplitReader.
> >>>
> >>> Having these two readers allows both complicated and simple
> >> implementation,
> >>> depending on the SplitReader writers. From Flink's perspective, it will
> >>> choose a more efficient threading model if the SplitReader is an
> >>> AsyncReader. Otherwise, it may have to use the one thread per reader
> >> model
> >>> if the reader is a SyncReader. Users can also choose to implement both
> >>> interface, in that case, it is up to Flink to choose which interface to
> >> use.
> >>>
> >>> Admittedly, this solution does have one more interface, but still seems
> >>> rewarding. Any thoughts?
> >>>
> >>> Thanks,
> >>>
> >>> Jiangjie (Becket) Qin
> >>>
> >>>
> >>> On Sun, Nov 18, 2018 at 11:33 PM Biao Liu <mm...@gmail.com> wrote:
> >>>
> >>>> Hi community,
> >>>>
> >>>> Thank you guys for sharing ideas.
> >>>>
> >>>> The thing I really concern is about the thread mode.
> >>>> Actually in Alibaba, we have implemented our "split reader" based
> source
> >>>> two years ago. That's based on "SourceFunction", it's just an
> extension
> >> not
> >>>> a refactoring. It's almost same with the version Thomas and Jamie
> >> described
> >>>> in Google Doc. It really helps in many scenarios.
> >>>>
> >>>> However I don't like the thread mode which starts a thread for each
> >> split.
> >>>> Starting extra thread in operator is not an ideal way IMO. Especially
> >>>> thread count is decided by split count. So I was wondering if there
> is a
> >>>> more elegant way. Do we really want these threads in Flink core?
> >>>>
> >>>> I agree that blocking interface is more easy to implement. Could we at
> >>>> least separate the split reader with source function into different
> >>>> interfaces? Not all sources would like to read all splits
> concurrently.
> >> In
> >>>> batch scenario, reading splits one by one is more general. And also
> not
> >> all
> >>>> sources are partitioned, right?
> >>>> I prefer there is a new source interface with "pull mode" only, no
> >> split.
> >>>> There is a splittable source extended it. And there is one
> >> implementation
> >>>> that starting threads for each split, reading all splits concurrently.
> >>>>
> >>>>
> >>>> Thomas Weise <th...@apache.org> 于2018年11月18日周日 上午3:18写道:
> >>>>
> >>>>> @Aljoscha to address your question first: In the case of the Kinesis
> >>>>> consumer (with current Kinesis consumer API), there would also be N+1
> >>>>> threads. I have implemented a prototype similar to what is shown in
> >>>> Jamie's
> >>>>> document, where the thread ownership is similar to what you have done
> >> for
> >>>>> Kafka.
> >>>>>
> >>>>> The equivalent of split reader manages its own thread and the "source
> >>>> main
> >>>>> thread" is responsible for emitting the data. The interface between
> >> the N
> >>>>> reader threads and the 1 emitter is a blocking queue per consumer
> >> thread.
> >>>>> The emitter can now control which queue to consume from based on the
> >>>> event
> >>>>> time progress.
> >>>>>
> >>>>> This is akin to a "non-blocking" interface *between emitter and split
> >>>>> reader*. Emitter uses poll to retrieve records from the N queues
> (which
> >>>>> requires non-blocking interaction). The emitter is independent of the
> >>>> split
> >>>>> reader implementation, that part could live in Flink.
> >>>>>
> >>>>> Regarding whether or not to assume that split readers always need a
> >>>> thread
> >>>>> and in addition that these reader threads should be managed by Flink:
> >> It
> >>>>> depends on the API of respective external systems and I would not
> bake
> >>>> that
> >>>>> assumption into Flink. Some client libraries manage their own threads
> >>>> (see
> >>>>> push based API like JMS and as I understand it may also apply to the
> >> new
> >>>>> fan-out Kinesis API:
> >>>>>
> >>>>>
> >>>>
> >>
> https://docs.aws.amazon.com/streams/latest/dev/building-enhanced-consumers-api.html
> >>>>> ).
> >>>>> In such cases it would not make sense to layer another reader thread
> on
> >>>>> top. It may instead be better if Flink provides to the split reader
> the
> >>>>> queue/buffer to push records to.
> >>>>>
> >>>>> The discussion so far has largely ignored the discovery aspect. There
> >> are
> >>>>> some important considerations such as ordering dependency of splits
> and
> >>>>> work rebalancing that may affect the split reader interface. Should
> we
> >>>> fork
> >>>>> this into a separate thread?
> >>>>>
> >>>>> Thanks,
> >>>>> Thomas
> >>>>>
> >>>>>
> >>>>> On Fri, Nov 16, 2018 at 8:09 AM Piotr Nowojski <
> >> piotr@data-artisans.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Hi Jamie,
> >>>>>>
> >>>>>> As it was already covered with my discussion with Becket, there is
> an
> >>>>> easy
> >>>>>> way to provide blocking API on top of non-blocking API. And yes we
> >> both
> >>>>>> agreed that blocking API is easier to implement by users.
> >>>>>>
> >>>>>> I also do not agree with respect to usefulness of non blocking API.
> >>>>>> Actually Kafka connector is the one that could be more efficient
> >> thanks
> >>>>> to
> >>>>>> the removal of the one layer of threading.
> >>>>>>
> >>>>>> Piotrek
> >>>>>>
> >>>>>>> On 16 Nov 2018, at 02:21, Jamie Grier <jg...@lyft.com.INVALID>
> >>>> wrote:
> >>>>>>>
> >>>>>>> Thanks Aljoscha for getting this effort going!
> >>>>>>>
> >>>>>>> There's been plenty of discussion here already and I'll add my big
> +1
> >>>>> to
> >>>>>>> making this interface very simple to implement for a new
> >>>>>>> Source/SplitReader.  Writing a new production quality connector for
> >>>>> Flink
> >>>>>>> is very difficult today and requires a lot of detailed knowledge
> >>>> about
> >>>>>>> Flink, event time progress, watermarking, idle shard detection, etc
> >>>> and
> >>>>>> it
> >>>>>>> would be good to move almost all of this type of code into Flink
> >>>> itself
> >>>>>> and
> >>>>>>> out of source implementations.  I also think this is totally doable
> >>>> and
> >>>>>> I'm
> >>>>>>> really excited to see this happening.
> >>>>>>>
> >>>>>>> I do have a couple of thoughts about the API and the
> implementation..
> >>>>>>>
> >>>>>>> In a perfect world there would be a single thread per Flink source
> >>>>>> sub-task
> >>>>>>> and no additional threads for SplitReaders -- but this assumes a
> >>>> world
> >>>>>>> where you have true async IO APIs for the upstream systems (like
> >>>> Kafka
> >>>>>> and
> >>>>>>> Kinesis, S3, HDFS, etc).  If that world did exist the single thread
> >>>>> could
> >>>>>>> just sit in an efficient select() call waiting for new data to
> arrive
> >>>>> on
> >>>>>>> any Split.  That'd be awesome..
> >>>>>>>
> >>>>>>> But, that world doesn't exist and given that practical
> consideration
> >>>> I
> >>>>>>> would think the next best implementation is going to be, in
> practice,
> >>>>>>> probably a thread per SplitReader that does nothing but call the
> >>>> source
> >>>>>> API
> >>>>>>> and drop whatever it reads into a (blocking) queue -- as Aljoscha
> >>>>>> mentioned
> >>>>>>> (calling it N+1) and as we started to describe here:
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> https://docs.google.com/document/d/13HxfsucoN7aUls1FX202KFAVZ4IMLXEZWyRfZNd6WFM/edit#heading=h.lfi2u4fv9cxa
> >>>>>>>
> >>>>>>> I guess my point is that I think we should strive to move as much
> of
> >>>>>>> something like the diagram referenced in the above doc into Flink
> >>>>> itself
> >>>>>>> and out of sources and simplify the SplitReader API as much as
> >>>> possible
> >>>>>> as
> >>>>>>> well.
> >>>>>>>
> >>>>>>> With the above in mind and with regard to the discussion about
> >>>>> blocking,
> >>>>>>> etc..  I'm not sure I agree with some of the discussion so far with
> >>>>>> regard
> >>>>>>> to this API design.  The calls to the upstream systems
> >>>> (kafka/kinesis)
> >>>>>> are
> >>>>>>> in fact going to be blocking calls.  So a simple API without the
> >>>>>> constraint
> >>>>>>> that the methods must be implemented in a non-blocking way seems
> >>>> better
> >>>>>> to
> >>>>>>> me from the point of view of somebody writing a new source
> >>>>>> implementation.
> >>>>>>> My concern is that if you force the implementer of the SplitReader
> >>>>>>> interface to do so in a non-blocking way you're just going to make
> it
> >>>>>>> harder to write those implementations.  Those calls to read the
> next
> >>>>> bit
> >>>>>> of
> >>>>>>> data are going to be blocking calls with most known important
> sources
> >>>>> --
> >>>>>> at
> >>>>>>> least Kafka/Kinesis/HDFS -- so I think maybe we should just deal
> with
> >>>>>> that
> >>>>>>> head on and work around it a higher level so the SplitReader
> >>>> interface
> >>>>>>> stays super simple to implement.  This means we manage all the
> >>>>> threading
> >>>>>> in
> >>>>>>> Flink core, the API stays pull-based, and the implementer is
> allowed
> >>>> to
> >>>>>>> simply block until they have data to return.
> >>>>>>>
> >>>>>>> I maybe would change my mind about this if truly asynchronous APIs
> to
> >>>>> the
> >>>>>>> upstream source systems were likely to be available in the near
> >>>> future
> >>>>> or
> >>>>>>> are now and I'm just ignorant of it.  But even then the supporting
> >>>> code
> >>>>>> in
> >>>>>>> Flink to drive async and sync sources would be different and in
> fact
> >>>>> they
> >>>>>>> might just have different APIs altogether -- SplitReader vs
> >>>>>>> AsyncSplitReader maybe.
> >>>>>>>
> >>>>>>> In the end I think playing with the implementation, across more
> than
> >>>>> one
> >>>>>>> source, and moving as much common code into Flink itself will
> reveal
> >>>>> the
> >>>>>>> best API of course.
> >>>>>>>
> >>>>>>> One other interesting note is that you need to preserve
> per-partition
> >>>>>>> ordering so you have to take care with the implementation if it
> were
> >>>> to
> >>>>>> be
> >>>>>>> based on a thread pool and futures so as not to reorder the reads.
> >>>>>>>
> >>>>>>> Anyway, I'm thrilled to see this starting to move forward and I'd
> >>>> very
> >>>>>> much
> >>>>>>> like to help with the implementation wherever I can.  We're doing a
> >>>>>>> simplified internal version of some of this at Lyft for just
> Kinesis
> >>>>>>> because we need a solution for event time alignment in the very
> short
> >>>>>> term
> >>>>>>> but we'd like to immediately start helping to do this properly in
> >>>> Flink
> >>>>>>> after that.  One of the end goals for us is event time alignment
> >>>> across
> >>>>>>> heterogeneous sources.  Another is making it possible for
> non-expert
> >>>>>> users
> >>>>>>> to have a high probability of being able to write their own,
> correct,
> >>>>>>> connectors.
> >>>>>>>
> >>>>>>> -Jamie
> >>>>>>>
> >>>>>>> On Thu, Nov 15, 2018 at 3:43 AM Aljoscha Krettek <
> >>>> aljoscha@apache.org>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi,
> >>>>>>>>
> >>>>>>>> I thought I had sent this mail a while ago but I must have
> forgotten
> >>>>> to
> >>>>>>>> send it.
> >>>>>>>>
> >>>>>>>> There is another thing we should consider for splits: the range of
> >>>>>>>> timestamps that it can contain. For example, the splits of a file
> >>>>> source
> >>>>>>>> would know what the minimum and maximum timestamp in the splits
> is,
> >>>>>>>> roughly. For infinite splits, such as Kafka partitions, the
> minimum
> >>>>>> would
> >>>>>>>> be meaningful but the maximum would be +Inf. If the splits expose
> >>>> the
> >>>>>>>> interval of time that they contain the readers, or the component
> >>>> that
> >>>>>>>> manages the readers can make decisions about which splits to
> forward
> >>>>> and
> >>>>>>>> read first. And it can also influence the minimum watermark that a
> >>>>>> reader
> >>>>>>>> forwards: it should never emit a watermark if it knows there are
> >>>>> splits
> >>>>>> to
> >>>>>>>> read that have a lower minimum timestamp. I think it should be as
> >>>> easy
> >>>>>> as
> >>>>>>>> adding a minimumTimestamp()/maximumTimestamp() method pair to the
> >>>>> split
> >>>>>>>> interface.
> >>>>>>>>
> >>>>>>>> Another thing we need to resolve is the actual reader interface. I
> >>>> see
> >>>>>>>> there has been some good discussion but I don't know if we have
> >>>>>> consensus.
> >>>>>>>> We should try and see how specific sources could be implemented
> with
> >>>>> the
> >>>>>>>> new interface. For example, for Kafka I think we need to have N+1
> >>>>>> threads
> >>>>>>>> per task (where N is the number of splits that a task is reading
> >>>>> from).
> >>>>>> On
> >>>>>>>> thread is responsible for reading from the splits. And each split
> >>>> has
> >>>>>> its
> >>>>>>>> own (internal) thread for reading from Kafka and putting messages
> in
> >>>>> an
> >>>>>>>> internal queue to pull from. This is similar to how the current
> >>>> Kafka
> >>>>>>>> source is implemented, which has a separate fetcher thread. The
> >>>> reason
> >>>>>> for
> >>>>>>>> this split is that we always need to try reading from Kafka to
> keep
> >>>>> the
> >>>>>>>> throughput up. In the current implementation the internal queue
> (or
> >>>>>>>> handover) limits the read rate of the reader threads.
> >>>>>>>>
> >>>>>>>> @Thomas, what do you think this would look like for Kinesis?
> >>>>>>>>
> >>>>>>>> Best,
> >>>>>>>> Aljoscha
> >>>>>>>>
> >>>>>>>>> On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com>
> >>>> wrote:
> >>>>>>>>>
> >>>>>>>>> Hi Piotrek,
> >>>>>>>>>
> >>>>>>>>> Thanks a lot for the detailed reply. All makes sense to me.
> >>>>>>>>>
> >>>>>>>>> WRT the confusion between advance() / getCurrent(), do you think
> it
> >>>>>> would
> >>>>>>>>> help if we combine them and have something like:
> >>>>>>>>>
> >>>>>>>>> CompletableFuture<T> getNext();
> >>>>>>>>> long getWatermark();
> >>>>>>>>> long getCurrentTimestamp();
> >>>>>>>>>
> >>>>>>>>> Cheers,
> >>>>>>>>>
> >>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>
> >>>>>>>>> On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <
> >>>>>> piotr@data-artisans.com>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi,
> >>>>>>>>>>
> >>>>>>>>>> Thanks again for the detailed answer :) Sorry for responding
> with
> >>>> a
> >>>>>>>> delay.
> >>>>>>>>>>
> >>>>>>>>>>> Completely agree that in pattern 2, having a callback is
> >>>> necessary
> >>>>>> for
> >>>>>>>>>> that
> >>>>>>>>>>> single thread outside of the connectors. And the connectors
> MUST
> >>>>> have
> >>>>>>>>>>> internal threads.
> >>>>>>>>>>
> >>>>>>>>>> Yes, this thread will have to exists somewhere. In pattern 2 it
> >>>>> exists
> >>>>>>>> in
> >>>>>>>>>> the connector (at least from the perspective of the Flink
> >>>> execution
> >>>>>>>>>> engine). In pattern 1 it exists inside the Flink execution
> engine.
> >>>>>> With
> >>>>>>>>>> completely blocking connectors, like simple reading from files,
> >>>> both
> >>>>>> of
> >>>>>>>>>> those approaches are basically the same. The difference is when
> >>>> user
> >>>>>>>>>> implementing Flink source is already working with a non blocking
> >>>>> code
> >>>>>>>> with
> >>>>>>>>>> some internal threads. In this case, pattern 1 would result in
> >>>>> "double
> >>>>>>>>>> thread wrapping”, while pattern 2 would allow to skip one layer
> of
> >>>>>>>>>> indirection.
> >>>>>>>>>>
> >>>>>>>>>>> If we go that way, we should have something like "void
> >>>>>>>>>>> poll(Callback) / void advance(callback)". I am curious how
> would
> >>>>>>>>>>> CompletableFuture work here, though. If 10 readers returns 10
> >>>>>>>> completable
> >>>>>>>>>>> futures, will there be 10 additional threads (so 20 threads in
> >>>>> total)
> >>>>>>>>>>> blocking waiting on them? Or will there be a single thread busy
> >>>>> loop
> >>>>>>>>>>> checking around?
> >>>>>>>>>>
> >>>>>>>>>> To be honest, I haven’t thought this completely through and I
> >>>>> haven’t
> >>>>>>>>>> tested/POC’ed it. Having said that, I can think of at least
> couple
> >>>>> of
> >>>>>>>>>> solutions. First is something like this:
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> >>>>>>>>>> <
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Line:
> >>>>>>>>>>
> >>>>>>>>>>                             `blocked = split.process();`
> >>>>>>>>>>
> >>>>>>>>>> Is where the execution goes into to the task/sources. This is
> >>>> where
> >>>>>> the
> >>>>>>>>>> returned future is handled:
> >>>>>>>>>>
> >>>>>>>>>>                             blocked.addListener(() -> {
> >>>>>>>>>>                                 blockedSplits.remove(split);
> >>>>>>>>>>                                 // reset the level priority to
> >>>>>>>> prevent
> >>>>>>>>>> previously-blocked splits from starving existing splits
> >>>>>>>>>>                                 split.resetLevelPriority();
> >>>>>>>>>>                                 waitingSplits.offer(split);
> >>>>>>>>>>                             }, executor);
> >>>>>>>>>>
> >>>>>>>>>> Fundamentally callbacks and Futures are more or less
> >>>> interchangeable
> >>>>>> You
> >>>>>>>>>> can always wrap one into another (creating a callback that
> >>>>> completes a
> >>>>>>>>>> future and attach a callback once future completes). In this
> case
> >>>>> the
> >>>>>>>>>> difference for me is mostly:
> >>>>>>>>>> - api with passing callback allows the callback to be fired
> >>>> multiple
> >>>>>>>> times
> >>>>>>>>>> and to fire it even if the connector is not blocked. This is
> what
> >>>> I
> >>>>>>>> meant
> >>>>>>>>>> by saying that api `CompletableFuture<?> isBlocked()` is a bit
> >>>>>> simpler.
> >>>>>>>>>> Connector can only return either “I’m not blocked” or “I’m
> blocked
> >>>>>> and I
> >>>>>>>>>> will tell you only once when I’m not blocked anymore”.
> >>>>>>>>>>
> >>>>>>>>>> But this is not the most important thing for me here. For me
> >>>>> important
> >>>>>>>>>> thing is to try our best to make Flink task’s control and
> >>>> execution
> >>>>>>>> single
> >>>>>>>>>> threaded. For that both callback and future APIs should work the
> >>>>> same.
> >>>>>>>>>>
> >>>>>>>>>>> WRT pattern 1, a single blocking take() API should just work.
> The
> >>>>>> good
> >>>>>>>>>>> thing is that a blocking read API is usually simpler to
> >>>> implement.
> >>>>>>>>>>
> >>>>>>>>>> Yes, they are easier to implement (especially if you are not the
> >>>> one
> >>>>>>>> that
> >>>>>>>>>> have to deal with the additional threading required around them
> ;)
> >>>>> ).
> >>>>>>>> But
> >>>>>>>>>> to answer this issue, if we choose pattern 2, we can always
> >>>> provide
> >>>>> a
> >>>>>>>>>> proxy/wrapper that would using the internal thread implement the
> >>>>>>>>>> non-blocking API while exposing blocking API to the user. It
> would
> >>>>>>>>>> implement pattern 2 for the user exposing to him pattern 1. In
> >>>> other
> >>>>>>>> words
> >>>>>>>>>> implementing pattern 1 in pattern 2 paradigm, while making it
> >>>>> possible
> >>>>>>>> to
> >>>>>>>>>> implement pure pattern 2 connectors.
> >>>>>>>>>>
> >>>>>>>>>>> BTW, one thing I am also trying to avoid is pushing users to
> >>>>> perform
> >>>>>> IO
> >>>>>>>>>> in
> >>>>>>>>>>> a method like "isBlocked()". If the method is expected to fetch
> >>>>>> records
> >>>>>>>>>>> (even if not returning them), naming it something more explicit
> >>>>> would
> >>>>>>>>>> help
> >>>>>>>>>>> avoid confusion.
> >>>>>>>>>>
> >>>>>>>>>> If we choose so, we could rework it into something like:
> >>>>>>>>>>
> >>>>>>>>>> CompletableFuture<?> advance()
> >>>>>>>>>> T getCurrent();
> >>>>>>>>>> Watermark getCurrentWatermark()
> >>>>>>>>>>
> >>>>>>>>>> But as I wrote before, this is more confusing to me for the
> exact
> >>>>>>>> reasons
> >>>>>>>>>> you mentioned :) I would be confused what should be done in
> >>>>>> `adanvce()`
> >>>>>>>> and
> >>>>>>>>>> what in `getCurrent()`. However, again this naming issue is not
> >>>> that
> >>>>>>>>>> important to me and probably is matter of taste/personal
> >>>>> preferences.
> >>>>>>>>>>
> >>>>>>>>>> Piotrek
> >>>>>>>>>>
> >>>>>>>>>>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com>
> >>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>> Hi Piotrek,
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks for the explanation. We are probably talking about the
> >>>> same
> >>>>>>>> thing
> >>>>>>>>>>> but in different ways. To clarify a little bit, I think there
> are
> >>>>> two
> >>>>>>>>>>> patterns to read from a connector.
> >>>>>>>>>>>
> >>>>>>>>>>> Pattern 1: Thread-less connector with a blocking read API.
> >>>> Outside
> >>>>> of
> >>>>>>>> the
> >>>>>>>>>>> connector, there is one IO thread per reader, doing blocking
> >>>> read.
> >>>>> An
> >>>>>>>>>>> additional thread will interact with all the IO threads.
> >>>>>>>>>>> Pattern 2: Connector with internal thread(s) and non-blocking
> >>>> API.
> >>>>>>>>>> Outside
> >>>>>>>>>>> of the connector, there is one thread for ALL readers, doing IO
> >>>>>> relying
> >>>>>>>>>> on
> >>>>>>>>>>> notification callbacks in the reader.
> >>>>>>>>>>>
> >>>>>>>>>>> In both patterns, there must be at least one thread per
> >>>> connector,
> >>>>>>>> either
> >>>>>>>>>>> inside (created by connector writers) or outside (created by
> >>>> Flink)
> >>>>>> of
> >>>>>>>>>> the
> >>>>>>>>>>> connector. Ideally there are NUM_CONNECTORS + 1 threads in
> total,
> >>>>> to
> >>>>>>>> make
> >>>>>>>>>>> sure that 1 thread is fully non-blocking.
> >>>>>>>>>>>
> >>>>>>>>>>>> Btw, I don’t know if you understand my point. Having only
> >>>> `poll()`
> >>>>>> and
> >>>>>>>>>>> `take()` is not enough for single threaded task. If our source
> >>>>>>>> interface
> >>>>>>>>>>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
> >>>>>>>>>>> isBlocked(),`, there is no way to implement single threaded
> task
> >>>>> that
> >>>>>>>>>> both
> >>>>>>>>>>> reads the data from the source connector and can also react to
> >>>>> system
> >>>>>>>>>>> events. Ok, non >blocking `poll()` would allow that, but with
> >>>> busy
> >>>>>>>>>> looping.
> >>>>>>>>>>>
> >>>>>>>>>>> Completely agree that in pattern 2, having a callback is
> >>>> necessary
> >>>>>> for
> >>>>>>>>>> that
> >>>>>>>>>>> single thread outside of the connectors. And the connectors
> MUST
> >>>>> have
> >>>>>>>>>>> internal threads. If we go that way, we should have something
> >>>> like
> >>>>>>>> "void
> >>>>>>>>>>> poll(Callback) / void advance(callback)". I am curious how
> would
> >>>>>>>>>>> CompletableFuture work here, though. If 10 readers returns 10
> >>>>>>>> completable
> >>>>>>>>>>> futures, will there be 10 additional threads (so 20 threads in
> >>>>> total)
> >>>>>>>>>>> blocking waiting on them? Or will there be a single thread busy
> >>>>> loop
> >>>>>>>>>>> checking around?
> >>>>>>>>>>>
> >>>>>>>>>>> WRT pattern 1, a single blocking take() API should just work.
> The
> >>>>>> good
> >>>>>>>>>>> thing is that a blocking read API is usually simpler to
> >>>> implement.
> >>>>> An
> >>>>>>>>>>> additional non-blocking "T poll()" method here is indeed
> optional
> >>>>> and
> >>>>>>>>>> could
> >>>>>>>>>>> be used in cases like Flink does not want the thread to block
> >>>>>> forever.
> >>>>>>>>>> They
> >>>>>>>>>>> can also be combined to have a "T poll(Timeout)", which is
> >>>> exactly
> >>>>>> what
> >>>>>>>>>>> KafkaConsumer did.
> >>>>>>>>>>>
> >>>>>>>>>>> It sounds that you are proposing pattern 2 with something
> similar
> >>>>> to
> >>>>>>>> NIO2
> >>>>>>>>>>> AsynchronousByteChannel[1]. That API would work, except that
> the
> >>>>>>>>>> signature
> >>>>>>>>>>> returning future seems not necessary. If that is the case, a
> >>>> minor
> >>>>>>>> change
> >>>>>>>>>>> on the current FLIP proposal to have "void advance(callback)"
> >>>>> should
> >>>>>>>>>> work.
> >>>>>>>>>>> And this means the connectors MUST have their internal threads.
> >>>>>>>>>>>
> >>>>>>>>>>> BTW, one thing I am also trying to avoid is pushing users to
> >>>>> perform
> >>>>>> IO
> >>>>>>>>>> in
> >>>>>>>>>>> a method like "isBlocked()". If the method is expected to fetch
> >>>>>> records
> >>>>>>>>>>> (even if not returning them), naming it something more explicit
> >>>>> would
> >>>>>>>>>> help
> >>>>>>>>>>> avoid confusion.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>>
> >>>>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>>>
> >>>>>>>>>>> [1]
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
> >>>>>>>>>>>
> >>>>>>>>>>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <
> >>>>>>>> piotr@data-artisans.com>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hi
> >>>>>>>>>>>>
> >>>>>>>>>>>> Good point with select/epoll, however I do not see how they
> >>>>> couldn’t
> >>>>>>>> be
> >>>>>>>>>>>> with Flink if we would like single task in Flink to be
> >>>>>> single-threaded
> >>>>>>>>>> (and
> >>>>>>>>>>>> I believe we should pursue this goal). If your connector
> blocks
> >>>> on
> >>>>>>>>>>>> `select`, then it can not process/handle control messages from
> >>>>>> Flink,
> >>>>>>>>>> like
> >>>>>>>>>>>> checkpoints, releasing resources and potentially output
> flushes.
> >>>>>> This
> >>>>>>>>>> would
> >>>>>>>>>>>> require tight integration between connector and Flink’s main
> >>>> event
> >>>>>>>>>>>> loop/selects/etc.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Looking at it from other perspective. Let’s assume that we
> have
> >>>> a
> >>>>>>>>>>>> connector implemented on top of `select`/`epoll`. In order to
> >>>>>>>> integrate
> >>>>>>>>>> it
> >>>>>>>>>>>> with Flink’s checkpointing/flushes/resource releasing it will
> >>>> have
> >>>>>> to
> >>>>>>>> be
> >>>>>>>>>>>> executed in separate thread one way or another. At least if
> our
> >>>>> API
> >>>>>>>> will
> >>>>>>>>>>>> enforce/encourage non blocking implementations with some kind
> of
> >>>>>>>>>>>> notifications (`isBlocked()` or `notify()` callback), some
> >>>>>> connectors
> >>>>>>>>>> might
> >>>>>>>>>>>> skip one layer of wapping threads.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Btw, I don’t know if you understand my point. Having only
> >>>> `poll()`
> >>>>>> and
> >>>>>>>>>>>> `take()` is not enough for single threaded task. If our source
> >>>>>>>> interface
> >>>>>>>>>>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
> >>>>>>>>>>>> isBlocked(),`, there is no way to implement single threaded
> task
> >>>>>> that
> >>>>>>>>>> both
> >>>>>>>>>>>> reads the data from the source connector and can also react to
> >>>>>> system
> >>>>>>>>>>>> events. Ok, non blocking `poll()` would allow that, but with
> >>>> busy
> >>>>>>>>>> looping.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Piotrek
> >>>>>>>>>>>>
> >>>>>>>>>>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com>
> >>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Hi Piotrek,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> But I don’t see a reason why we should expose both blocking
> >>>>>> `take()`
> >>>>>>>>>> and
> >>>>>>>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone
> >>>> (Flink
> >>>>>>>>>> engine
> >>>>>>>>>>>> or
> >>>>>>>>>>>>> connector) would have to do the same busy
> >>>>>>>>>>>>>> looping anyway and I think it would be better to have a
> >>>> simpler
> >>>>>>>>>>>> connector
> >>>>>>>>>>>>> API (that would solve our problems) and force connectors to
> >>>>> comply
> >>>>>>>> one
> >>>>>>>>>>>> way
> >>>>>>>>>>>>> or another.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> If we let the block happen inside the connector, the blocking
> >>>>> does
> >>>>>>>> not
> >>>>>>>>>>>> have
> >>>>>>>>>>>>> to be a busy loop. For example, to do the block waiting
> >>>>>> efficiently,
> >>>>>>>>>> the
> >>>>>>>>>>>>> connector can use java NIO selector().select which relies on
> OS
> >>>>>>>> syscall
> >>>>>>>>>>>>> like epoll[1] instead of busy looping. But if Flink engine
> >>>> blocks
> >>>>>>>>>> outside
> >>>>>>>>>>>>> the connector, it pretty much has to do the busy loop. So if
> >>>>> there
> >>>>>> is
> >>>>>>>>>>>> only
> >>>>>>>>>>>>> one API to get the element, a blocking getNextElement() makes
> >>>>> more
> >>>>>>>>>> sense.
> >>>>>>>>>>>>> In any case, we should avoid ambiguity. It has to be crystal
> >>>>> clear
> >>>>>>>>>> about
> >>>>>>>>>>>>> whether a method is expected to be blocking or non-blocking.
> >>>>>>>> Otherwise
> >>>>>>>>>> it
> >>>>>>>>>>>>> would be very difficult for Flink engine to do the right
> thing
> >>>>> with
> >>>>>>>> the
> >>>>>>>>>>>>> connectors. At the first glance at getCurrent(), the expected
> >>>>>>>> behavior
> >>>>>>>>>> is
> >>>>>>>>>>>>> not quite clear.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> That said, I do agree that functionality wise, poll() and
> >>>> take()
> >>>>>> kind
> >>>>>>>>>> of
> >>>>>>>>>>>>> overlap. But they are actually not quite different from
> >>>>>>>>>>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the
> >>>> only
> >>>>>>>>>>>>> difference is that poll() also returns the next record if it
> is
> >>>>>>>>>>>> available.
> >>>>>>>>>>>>> But I agree that the isBlocked() + getNextElement() is more
> >>>>>> flexible
> >>>>>>>> as
> >>>>>>>>>>>>> users can just check the record availability, but not fetch
> the
> >>>>>> next
> >>>>>>>>>>>>> element.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> In case of thread-less readers with only non-blocking
> >>>>>> `queue.poll()`
> >>>>>>>>>> (is
> >>>>>>>>>>>>> that really a thing? I can not think about a real
> >>>> implementation
> >>>>>> that
> >>>>>>>>>>>>> enforces such constraints)
> >>>>>>>>>>>>> Right, it is pretty much a syntax sugar to allow user combine
> >>>> the
> >>>>>>>>>>>>> check-and-take into one method. It could be achieved with
> >>>>>>>> isBlocked() +
> >>>>>>>>>>>>> getNextElement().
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
> >>>>>>>>>> piotr@data-artisans.com>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi Becket,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> With my proposal, both of your examples would have to be
> >>>> solved
> >>>>> by
> >>>>>>>> the
> >>>>>>>>>>>>>> connector and solution to both problems would be the same:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Pretend that connector is never blocked (`isBlocked() {
> return
> >>>>>>>>>>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in
> blocking
> >>>>>>>> fashion
> >>>>>>>>>>>> (or
> >>>>>>>>>>>>>> semi blocking with return of control from time to time to
> >>>> allow
> >>>>>> for
> >>>>>>>>>>>>>> checkpointing, network flushing and other resource
> management
> >>>>>> things
> >>>>>>>>>> to
> >>>>>>>>>>>>>> happen in the same main thread). In other words, exactly how
> >>>> you
> >>>>>>>> would
> >>>>>>>>>>>>>> implement `take()` method or how the same source connector
> >>>> would
> >>>>>> be
> >>>>>>>>>>>>>> implemented NOW with current source interface. The
> difference
> >>>>> with
> >>>>>>>>>>>> current
> >>>>>>>>>>>>>> interface would be only that main loop would be outside of
> the
> >>>>>>>>>>>> connector,
> >>>>>>>>>>>>>> and instead of periodically releasing checkpointing lock,
> >>>>>>>> periodically
> >>>>>>>>>>>>>> `return null;` or `return Optional.empty();` from
> >>>>>>>> `getNextElement()`.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> In case of thread-less readers with only non-blocking
> >>>>>> `queue.poll()`
> >>>>>>>>>> (is
> >>>>>>>>>>>>>> that really a thing? I can not think about a real
> >>>> implementation
> >>>>>>>> that
> >>>>>>>>>>>>>> enforces such constraints), we could provide a wrapper that
> >>>>> hides
> >>>>>>>> the
> >>>>>>>>>>>> busy
> >>>>>>>>>>>>>> looping. The same applies how to solve forever blocking
> >>>> readers
> >>>>> -
> >>>>>> we
> >>>>>>>>>>>> could
> >>>>>>>>>>>>>> provider another wrapper running the connector in separate
> >>>>> thread.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> But I don’t see a reason why we should expose both blocking
> >>>>>> `take()`
> >>>>>>>>>> and
> >>>>>>>>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone
> >>>>> (Flink
> >>>>>>>>>>>> engine or
> >>>>>>>>>>>>>> connector) would have to do the same busy looping anyway
> and I
> >>>>>> think
> >>>>>>>>>> it
> >>>>>>>>>>>>>> would be better to have a simpler connector API (that would
> >>>>> solve
> >>>>>>>> our
> >>>>>>>>>>>>>> problems) and force connectors to comply one way or another.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Piotrek
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com>
> >>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi Piotr,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I might have misunderstood you proposal. But let me try to
> >>>>>> explain
> >>>>>>>> my
> >>>>>>>>>>>>>>> concern. I am thinking about the following case:
> >>>>>>>>>>>>>>> 1. a reader has the following two interfaces,
> >>>>>>>>>>>>>>> boolean isBlocked()
> >>>>>>>>>>>>>>> T getNextElement()
> >>>>>>>>>>>>>>> 2. the implementation of getNextElement() is non-blocking.
> >>>>>>>>>>>>>>> 3. The reader is thread-less, i.e. it does not have any
> >>>>> internal
> >>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>> For example, it might just delegate the getNextElement()
> to a
> >>>>>>>>>>>>>> queue.poll(),
> >>>>>>>>>>>>>>> and isBlocked() is just queue.isEmpty().
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> How can Flink efficiently implement a blocking reading
> >>>> behavior
> >>>>>>>> with
> >>>>>>>>>>>> this
> >>>>>>>>>>>>>>> reader? Either a tight loop or a backoff interval is
> needed.
> >>>>>>>> Neither
> >>>>>>>>>> of
> >>>>>>>>>>>>>>> them is ideal.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Now let's say in the reader mentioned above implements a
> >>>>> blocking
> >>>>>>>>>>>>>>> getNextElement() method. Because there is no internal
> thread
> >>>> in
> >>>>>> the
> >>>>>>>>>>>>>> reader,
> >>>>>>>>>>>>>>> after isBlocked() returns false. Flink will still have to
> >>>> loop
> >>>>> on
> >>>>>>>>>>>>>>> isBlocked() to check whether the next record is available.
> If
> >>>>> the
> >>>>>>>>>> next
> >>>>>>>>>>>>>>> record reaches after 10 min, it is a tight loop for 10 min.
> >>>> You
> >>>>>>>> have
> >>>>>>>>>>>>>>> probably noticed that in this case, even isBlocked()
> returns
> >>>> a
> >>>>>>>>>> future,
> >>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>> future() will not be completed if Flink does not call some
> >>>>> method
> >>>>>>>>>> from
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> reader, because the reader has no internal thread to
> complete
> >>>>>> that
> >>>>>>>>>>>> future
> >>>>>>>>>>>>>>> by itself.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Due to the above reasons, a blocking take() API would allow
> >>>>> Flink
> >>>>>>>> to
> >>>>>>>>>>>> have
> >>>>>>>>>>>>>>> an efficient way to read from a reader. There are many ways
> >>>> to
> >>>>>> wake
> >>>>>>>>>> up
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> blocking thread when checkpointing is needed depending on
> the
> >>>>>>>>>>>>>>> implementation. But I think the poll()/take() API would
> also
> >>>>> work
> >>>>>>>> in
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>> case.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
> >>>>>>>>>> piotr@data-artisans.com
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> a)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> BTW, regarding the isBlock() method, I have a few more
> >>>>>> questions.
> >>>>>>>>>> 21,
> >>>>>>>>>>>>>> Is
> >>>>>>>>>>>>>>>> a method isReady() with boolean as a return value
> >>>>>>>>>>>>>>>>> equivalent? Personally I found it is a little bit
> confusing
> >>>>> in
> >>>>>>>> what
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>> supposed to be returned when the future is completed. 22.
> if
> >>>>>>>>>>>>>>>>> the implementation of isBlocked() is optional, how do the
> >>>>>> callers
> >>>>>>>>>>>> know
> >>>>>>>>>>>>>>>> whether the method is properly implemented or not?
> >>>>>>>>>>>>>>>>> Does not implemented mean it always return a completed
> >>>>> future?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an
> >>>>> equivalent
> >>>>>>>> to
> >>>>>>>>>>>>>>>> `boolean hasNext()` which in case of “false” provides some
> >>>>> kind
> >>>>>>>> of a
> >>>>>>>>>>>>>>>> listener/callback that notifies about presence of next
> >>>>> element.
> >>>>>>>>>> There
> >>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>> some minor details, like `CompletableFuture<?>` has a
> >>>> minimal
> >>>>>> two
> >>>>>>>>>>>> state
> >>>>>>>>>>>>>>>> logic:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 1. Future is completed - we have more data
> >>>>>>>>>>>>>>>> 2. Future not yet completed - we don’t have data now, but
> we
> >>>>>>>>>> might/we
> >>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>> have in the future
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> While `boolean hasNext()` and `notify()` callback are a
> bit
> >>>>> more
> >>>>>>>>>>>>>>>> complicated/dispersed and can lead/encourage `notify()`
> >>>> spam.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> b)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method
> >>>>> like
> >>>>>>>>>>>>>> `getNext`
> >>>>>>>>>>>>>>>> the `getNext` would need return a
> >>>>>>>>>>>>>>>>> `ElementWithTimestamp` because some sources want to add
> >>>>>> timestamp
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>> every element. IMO, this is not so memory friendly
> >>>>>>>>>>>>>>>>> so I prefer this design.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Guowei I don’t quite understand this. Could you elaborate
> >>>> why
> >>>>>>>>>> having a
> >>>>>>>>>>>>>>>> separate `advance()` help?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> c)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Regarding advance/poll/take. What’s the value of having
> two
> >>>>>>>> separate
> >>>>>>>>>>>>>>>> methods: poll and take? Which one of them should be called
> >>>> and
> >>>>>>>> which
> >>>>>>>>>>>>>>>> implemented? What’s the benefit of having those methods
> >>>>> compared
> >>>>>>>> to
> >>>>>>>>>>>>>> having
> >>>>>>>>>>>>>>>> a one single method `getNextElement()` (or `pollElement()
> or
> >>>>>>>>>> whatever
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> name it) with following contract:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> CompletableFuture<?> isBlocked();
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> /**
> >>>>>>>>>>>>>>>> Return next element - will be called only if `isBlocked()`
> >>>> is
> >>>>>>>>>>>> completed.
> >>>>>>>>>>>>>>>> Try to implement it in non blocking fashion, but if that’s
> >>>>>>>>>> impossible
> >>>>>>>>>>>> or
> >>>>>>>>>>>>>>>> you just don’t need the effort, you can block in this
> >>>> method.
> >>>>>>>>>>>>>>>> */
> >>>>>>>>>>>>>>>> T getNextElement();
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I mean, if the connector is implemented non-blockingly,
> >>>> Flink
> >>>>>>>> should
> >>>>>>>>>>>> use
> >>>>>>>>>>>>>>>> it that way. If it’s not, then `poll()` will `throw new
> >>>>>>>>>>>>>>>> NotImplementedException()`. Implementing both of them and
> >>>>>>>> providing
> >>>>>>>>>>>>>> both of
> >>>>>>>>>>>>>>>> them to Flink wouldn’t make a sense, thus why not merge
> them
> >>>>>> into
> >>>>>>>> a
> >>>>>>>>>>>>>> single
> >>>>>>>>>>>>>>>> method call that should preferably (but not necessarily
> need
> >>>>> to)
> >>>>>>>> be
> >>>>>>>>>>>>>>>> non-blocking? It’s not like we are implementing general
> >>>>> purpose
> >>>>>>>>>>>> `Queue`,
> >>>>>>>>>>>>>>>> which users might want to call either of `poll` or `take`.
> >>>> We
> >>>>>>>> would
> >>>>>>>>>>>>>> always
> >>>>>>>>>>>>>>>> prefer to call `poll`, but if it’s blocking, then still we
> >>>>> have
> >>>>>> no
> >>>>>>>>>>>>>> choice,
> >>>>>>>>>>>>>>>> but to call it and block on it.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> d)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
> >>>> source
> >>>>>> is
> >>>>>>>>>> very
> >>>>>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
> >>>>>> another
> >>>>>>>>>> way
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly
> if
> >>>>>> every
> >>>>>>>>>>>>>> advance
> >>>>>>>>>>>>>>>>> call return a Future.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I didn’t want to mention this, to not clog my initial
> >>>>> proposal,
> >>>>>>>> but
> >>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>> is a simple solution for the problem:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> public interface SplitReader {
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> (…)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> CompletableFuture<?> NOT_BLOCKED =
> >>>>>>>>>>>>>>>> CompletableFuture.completedFuture(null);
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> /**
> >>>>>>>>>>>>>>>> * Returns a future that will be completed when the page
> >>>> source
> >>>>>>>>>>>>>> becomes
> >>>>>>>>>>>>>>>> * unblocked.  If the page source is not blocked, this
> method
> >>>>>>>>>> should
> >>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>> * {@code NOT_BLOCKED}.
> >>>>>>>>>>>>>>>> */
> >>>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked()
> >>>>>>>>>>>>>>>> {
> >>>>>>>>>>>>>>>>  return NOT_BLOCKED;
> >>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> If we are blocked and we are waiting for the IO, then
> >>>>> creating a
> >>>>>>>> new
> >>>>>>>>>>>>>>>> Future is non-issue. Under full throttle/throughput and
> not
> >>>>>>>> blocked
> >>>>>>>>>>>>>> sources
> >>>>>>>>>>>>>>>> returning a static `NOT_BLOCKED` constant  should also
> solve
> >>>>> the
> >>>>>>>>>>>>>> problem.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> One more remark, non-blocking sources might be a necessity
> >>>> in
> >>>>> a
> >>>>>>>>>> single
> >>>>>>>>>>>>>>>> threaded model without a checkpointing lock. (Currently
> when
> >>>>>>>> sources
> >>>>>>>>>>>> are
> >>>>>>>>>>>>>>>> blocked, they can release checkpointing lock and
> re-acquire
> >>>> it
> >>>>>>>> again
> >>>>>>>>>>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for
> >>>>>>>> checkpoints
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>> happen when source is idling. In that case either
> `notify()`
> >>>>> or
> >>>>>> my
> >>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>> `isBlocked()` would allow to avoid busy-looping.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Piotrek
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <
> becket.qin@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi Thomas,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> The iterator-like API was also the first thing that came
> to
> >>>>> me.
> >>>>>>>> But
> >>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>> seems a little confusing that hasNext() does not mean
> "the
> >>>>>> stream
> >>>>>>>>>> has
> >>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>> ended", but means "the next record is ready", which is
> >>>>>>>> repurposing
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> well
> >>>>>>>>>>>>>>>>> known meaning of hasNext(). If we follow the
> >>>> hasNext()/next()
> >>>>>>>>>>>> pattern,
> >>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>> additional isNextReady() method to indicate whether the
> >>>> next
> >>>>>>>> record
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> ready seems more intuitive to me.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Similarly, in poll()/take() pattern, another method of
> >>>>> isDone()
> >>>>>>>> is
> >>>>>>>>>>>>>> needed
> >>>>>>>>>>>>>>>>> to indicate whether the stream has ended or not.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
> >>>>>>>>>>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
> >>>>>>>>>>>>>> implementation.
> >>>>>>>>>>>>>>>>> When I am implementing a reader, I could have a couple of
> >>>>>>>> choices:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> - A thread-less reader that does not have any internal
> >>>>> thread.
> >>>>>>>>>>>>>>>>> - When poll() is called, the same calling thread will
> >>>>> perform a
> >>>>>>>>>> bunch
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> IO asynchronously.
> >>>>>>>>>>>>>>>>> - When take() is called, the same calling thread will
> >>>>> perform a
> >>>>>>>>>>>>>>>> bunch
> >>>>>>>>>>>>>>>>> of IO and wait until the record is ready.
> >>>>>>>>>>>>>>>>> - A reader with internal threads performing network IO
> and
> >>>>> put
> >>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>> into a buffer.
> >>>>>>>>>>>>>>>>> - When poll() is called, the calling thread simply reads
> >>>> from
> >>>>>>>> the
> >>>>>>>>>>>>>>>>> buffer and return empty result immediately if there is no
> >>>>>>>> record.
> >>>>>>>>>>>>>>>>> - When take() is called, the calling thread reads from
> the
> >>>>>>>> buffer
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> block waiting if the buffer is empty.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On the other hand, with the
> hasNext()/next()/isNextReady()
> >>>>> API,
> >>>>>>>> it
> >>>>>>>>>> is
> >>>>>>>>>>>>>>>> less
> >>>>>>>>>>>>>>>>> intuitive for the reader developers to write the
> >>>> thread-less
> >>>>>>>>>> pattern.
> >>>>>>>>>>>>>>>>> Although technically speaking one can still do the
> >>>>> asynchronous
> >>>>>>>> IO
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>> prepare the record in isNextReady(). But it is inexplicit
> >>>> and
> >>>>>>>> seems
> >>>>>>>>>>>>>>>>> somewhat hacky.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <
> >>>> thw@apache.org>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Couple more points regarding discovery:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The proposal mentions that discovery could be outside
> the
> >>>>>>>>>> execution
> >>>>>>>>>>>>>>>> graph.
> >>>>>>>>>>>>>>>>>> Today, discovered partitions/shards are checkpointed. I
> >>>>>> believe
> >>>>>>>>>> that
> >>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>> also need to be the case in the future, even when
> >>>> discovery
> >>>>>> and
> >>>>>>>>>>>>>> reading
> >>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>> split between different tasks.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> For cases such as resharding of a Kinesis stream, the
> >>>>>>>> relationship
> >>>>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>>>> splits needs to be considered. Splits cannot be randomly
> >>>>>>>>>> distributed
> >>>>>>>>>>>>>>>> over
> >>>>>>>>>>>>>>>>>> readers in certain situations. An example was mentioned
> >>>>> here:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thomas
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <
> >>>> thw@apache.org
> >>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks for getting the ball rolling on this!
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Can the number of splits decrease? Yes, splits can be
> >>>>> closed
> >>>>>>>> and
> >>>>>>>>>> go
> >>>>>>>>>>>>>>>> away.
> >>>>>>>>>>>>>>>>>>> An example would be a shard merge in Kinesis (2
> existing
> >>>>>> shards
> >>>>>>>>>>>> will
> >>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> closed and replaced with a new shard).
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive
> >>>>>> approach
> >>>>>>>>>>>> would
> >>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> the thread-less IO model (pull based, non-blocking,
> >>>> caller
> >>>>>>>>>>>> retrieves
> >>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>> records when available). The current Kinesis API
> requires
> >>>>> the
> >>>>>>>> use
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>> threads. But that can be internal to the split reader
> and
> >>>>>> does
> >>>>>>>>>> not
> >>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> be a source API concern. In fact, that's what we are
> >>>>> working
> >>>>>> on
> >>>>>>>>>>>> right
> >>>>>>>>>>>>>>>> now
> >>>>>>>>>>>>>>>>>>> as improvement to the existing consumer: Each shard
> >>>>> consumer
> >>>>>>>>>> thread
> >>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>> push to a queue, the consumer main thread will poll the
> >>>>>>>> queue(s).
> >>>>>>>>>>>> It
> >>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> The proposed SplitReader interface would fit the
> >>>>> thread-less
> >>>>>> IO
> >>>>>>>>>>>>>> model.
> >>>>>>>>>>>>>>>>>>> Similar to an iterator, we find out if there is a new
> >>>>> element
> >>>>>>>>>>>>>> (hasNext)
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> if so, move to it (next()). Separate calls deliver the
> >>>> meta
> >>>>>>>>>>>>>> information
> >>>>>>>>>>>>>>>>>>> (timestamp, watermark). Perhaps advance call could
> offer
> >>>> a
> >>>>>>>>>> timeout
> >>>>>>>>>>>>>>>>>> option,
> >>>>>>>>>>>>>>>>>>> so that the caller does not end up in a busy wait. On
> the
> >>>>>> other
> >>>>>>>>>>>>>> hand, a
> >>>>>>>>>>>>>>>>>>> caller processing multiple splits may want to cycle
> >>>> through
> >>>>>>>> fast,
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> process elements of other splits as soon as they become
> >>>>>>>>>> available.
> >>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>> nice
> >>>>>>>>>>>>>>>>>>> thing is that this "split merge" logic can now live in
> >>>>> Flink
> >>>>>>>> and
> >>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> optimized and shared between different sources.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>> Thomas
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <
> >>>>>> guowei.mgw@gmail.com
> >>>>>>>>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>>>>>> Thanks Aljoscha for this FLIP.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
> >>>>>> source
> >>>>>>>> is
> >>>>>>>>>>>>>> very
> >>>>>>>>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may
> >>>> be
> >>>>>>>>>> another
> >>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> achieve this. I think it may be not very memory
> friendly
> >>>>> if
> >>>>>>>>>> every
> >>>>>>>>>>>>>>>>>> advance
> >>>>>>>>>>>>>>>>>>>> call return a Future.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> public interface Listener {
> >>>>>>>>>>>>>>>>>>>> public void notify();
> >>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> public interface SplitReader() {
> >>>>>>>>>>>>>>>>>>>> /**
> >>>>>>>>>>>>>>>>>>>> * When there is no element temporarily, this will
> return
> >>>>>>>>>> false.
> >>>>>>>>>>>>>>>>>>>> * When elements is available again splitReader can
> call
> >>>>>>>>>>>>>>>>>>>> listener.notify()
> >>>>>>>>>>>>>>>>>>>> * In addition the frame would check `advance`
> >>>>> periodically .
> >>>>>>>>>>>>>>>>>>>> * Of course advance can always return true and ignore
> >>>> the
> >>>>>>>>>>>>>>>> listener
> >>>>>>>>>>>>>>>>>>>> argument for simplicity.
> >>>>>>>>>>>>>>>>>>>> */
> >>>>>>>>>>>>>>>>>>>> public boolean advance(Listener listener);
> >>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create
> >>>> all
> >>>>>>>> Splits
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>> how
> >>>>>>>>>>>>>>>>>>>> to create a SplitReader from a Split. But there is no
> >>>>>> strategy
> >>>>>>>>>> for
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> user
> >>>>>>>>>>>>>>>>>>>> to choose how to assign the splits to the tasks. I
> think
> >>>>> we
> >>>>>>>>>> could
> >>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> Enum to let user to choose.
> >>>>>>>>>>>>>>>>>>>> /**
> >>>>>>>>>>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
> >>>>>>>>>>>>>>>>>>>> Location,
> >>>>>>>>>>>>>>>>>>>> Workload,
> >>>>>>>>>>>>>>>>>>>> Random,
> >>>>>>>>>>>>>>>>>>>> Average
> >>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>> */
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one
> >>>> method
> >>>>>> like
> >>>>>>>>>>>>>>>> `getNext`
> >>>>>>>>>>>>>>>>>>>> the `getNext` would need return a
> `ElementWithTimestamp`
> >>>>>>>> because
> >>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>> sources want to add timestamp to every element. IMO,
> >>>> this
> >>>>> is
> >>>>>>>> not
> >>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>>>> friendly so I prefer this design.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com>
> 于2018年11月1日周四
> >>>>>>>>>> 下午6:08写道:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking
> quite
> >>>> a
> >>>>>> lot
> >>>>>>>> of
> >>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>> possible improvements. I have one proposal. Instead
> of
> >>>>>>>> having a
> >>>>>>>>>>>>>>>> method:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> boolean advance() throws IOException;
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I would replace it with
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> /*
> >>>>>>>>>>>>>>>>>>>>> * Return a future, which when completed means that
> >>>> source
> >>>>>> has
> >>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>> data
> >>>>>>>>>>>>>>>>>>>>> and getNext() will not block.
> >>>>>>>>>>>>>>>>>>>>> * If you wish to use benefits of non blocking
> >>>> connectors,
> >>>>>>>>>> please
> >>>>>>>>>>>>>>>>>>>>> implement this method appropriately.
> >>>>>>>>>>>>>>>>>>>>> */
> >>>>>>>>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
> >>>>>>>>>>>>>>>>>>>>> return CompletableFuture.completedFuture(null);
> >>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Couple of arguments:
> >>>>>>>>>>>>>>>>>>>>> 1. I don’t understand the division of work between
> >>>>>>>> `advance()`
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> `getCurrent()`. What should be done in which,
> >>>> especially
> >>>>>> for
> >>>>>>>>>>>>>>>> connectors
> >>>>>>>>>>>>>>>>>>>>> that handle records in batches (like Kafka) and when
> >>>>> should
> >>>>>>>> you
> >>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>> `advance` and when `getCurrent()`.
> >>>>>>>>>>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>`
> will
> >>>>>> allow
> >>>>>>>>>> us
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> future to have asynchronous/non blocking connectors
> and
> >>>>>> more
> >>>>>>>>>>>>>>>>>> efficiently
> >>>>>>>>>>>>>>>>>>>>> handle large number of blocked threads, without busy
> >>>>>> waiting.
> >>>>>>>>>>>> While
> >>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> same time it doesn’t add much complexity, since naive
> >>>>>>>> connector
> >>>>>>>>>>>>>>>>>>>>> implementations can be always blocking.
> >>>>>>>>>>>>>>>>>>>>> 3. This also would allow us to use a fixed size
> thread
> >>>>> pool
> >>>>>>>> of
> >>>>>>>>>>>> task
> >>>>>>>>>>>>>>>>>>>>> executors, instead of one thread per task.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Piotrek
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
> >>>>>>>>>> aljoscha@apache.org
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hi All,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> In order to finally get the ball rolling on the new
> >>>>> source
> >>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>>>>> that we have discussed for so long I finally created
> a
> >>>>>> FLIP:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
> >>>>>>>>>> work/discussion
> >>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>> adding per-partition watermark support to the Kinesis
> >>>>>> source
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> because
> >>>>>>>>>>>>>>>>>>>>> this would enable generic implementation of
> event-time
> >>>>>>>>>> alignment
> >>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>> sources. Maybe we need another FLIP for the
> event-time
> >>>>>>>>>> alignment
> >>>>>>>>>>>>>>>> part,
> >>>>>>>>>>>>>>>>>>>>> especially the part about information sharing between
> >>>>>>>>>> operations
> >>>>>>>>>>>>>> (I'm
> >>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>> calling it state sharing because state has a special
> >>>>>> meaning
> >>>>>>>> in
> >>>>>>>>>>>>>>>> Flink).
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Please discuss away!
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Aljoscha
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@data-artisans.com>.
Hi Becket,

I think the problem is not with the split re-assignment, but with dynamic split discovery. We do not always know before the hand the number of splits (for example Kafka partition/topic discovery, but this can also happen in batch), while the source parallelism is fixed/known before hand.

> 1. What if the SplitReader implementation cannot easily add a split to read on the fly?

Always initiating one consumer per split will not be efficient in many cases. While if the connector needs to instantiate a new reader per each split, connector can handle this internally (addSplit() would close previous reader and create new one).

> 2. Does Flink have to be involved in splits assignment?

I think that this might be a good shared logic between different connectors.

> @Biao,
> If I understand correctly, the concern you raised was that a Source may
> return a lot of splits and thus Flink may have to create a lot of fetcher
> threads. This is a valid concern, but I cannot think of a solution to that.
> After all, the SplitReaders may be written by third parties. Poor
> implementations seem difficult to prevent.

I think we can solve this and this is not as uncommon as you might think. In batch word, usually/often you have one split per HDFS chunk, each chunk being 64-256MB. With peta byte tables you end up with range from millions to billions of splits. This becomes a bottleneck if splits can be efficiently filtered out/eliminated based on some header (ORC header for example). In other words, if you have huge number of splits that are very cheap/quick to process.

Piotrek

> On 22 Nov 2018, at 04:54, Becket Qin <be...@gmail.com> wrote:
> 
> Thanks Piotrek,
> 
>> void SplitReader#addSplit(Split)
>> boolean SplitReader#doesWantMoreSplits()
> 
> I have two questions about this API.
> 1. What if the SplitReader implementation cannot easily add a split to read
> on the fly?
> 2. Does Flink have to be involved in splits assignment?
> 
> I am wondering if it would be simpler to let the enumerator indicate
> whether a split reassignment is needed. If the answer is yes, Flink can
> just start from the beginning to get all the splits and create one reader
> per split. This might be a little more expensive than dynamically adding a
> split to a reader, but given that the splits change should be rare, it is
> probably acceptable.
> 
> In the Kafka case, the SplitT may just be a consumer. The enumerator will
> simply check if the topic has new partitions to be assigned to this reader.
> 
> @Biao,
> If I understand correctly, the concern you raised was that a Source may
> return a lot of splits and thus Flink may have to create a lot of fetcher
> threads. This is a valid concern, but I cannot think of a solution to that.
> After all, the SplitReaders may be written by third parties. Poor
> implementations seem difficult to prevent.
> 
> Thanks,
> 
> Jiangjie (Becket) Qin
> 
> On Wed, Nov 21, 2018 at 10:13 PM Piotr Nowojski <pi...@data-artisans.com>
> wrote:
> 
>> Hi again,
>> 
>>> However I don't like the thread mode which starts a thread for each
>> split.
>>> Starting extra thread in operator is not an ideal way IMO. Especially
>>> thread count is decided by split count. So I was wondering if there is a
>>> more elegant way. Do we really want these threads in Flink core?
>> 
>> Biao you have raised an important issue. Indeed it seems like the current
>> proposal is missing something. I would guess that we need a mechanism for
>> adding new splits to an already existing SplitReader and some logic to
>> determine whether current instance can accept more splits or not. For
>> example
>> 
>> void SplitReader#addSplit(Split)
>> boolean SplitReader#doesWantMoreSplits()
>> 
>> Flink could randomly/round robin assign new splits to the SplitReaders
>> that `doWantMoreSplits()`. Batch file readers might implement some custom
>> logic in `doesWantMoreSplits()`, like one SplitReader can have at most N
>> enqueued splits?
>> 
>> Also what about Kafka. Isn’t it the case that one KafkaConsumer can read
>> from multiple splits? So Kafka’s SplitReader should always return true from
>> `doesWantMoreSplits()`?
>> 
>> What do you think?
>> 
>> Re: Becket
>> 
>> I’m +1 for Sync and AsyncSplitReader.
>> 
>> Piotrek
>> 
>>> On 21 Nov 2018, at 14:49, Becket Qin <be...@gmail.com> wrote:
>>> 
>>> Hi Aljoscha,
>>> 
>>> Good point on the potential optimization in the source. One thing to
>>> clarify, by "adding a minimumTimestamp()/maximumTimestamp() method pair
>> to
>>> the split interface", did you mean "split reader interface"? If so, what
>>> should the readers do if they do not have such additional information? I
>> am
>>> wondering if it is possible to leave such optimization to the source
>>> internal implementation.
>>> 
>>> @all
>>> After reading all the feedback, Biao and I talked a little bit offline.
>> We
>>> would like to share some new thoughts with you and see what do you think.
>>> 
>>> When looking at the Source API, we were trying to answer two questions.
>>> First of all, how would Flink use this API if someone else implemented
>> it.
>>> Secondly, how would the connector contributors implement the interface?
>> How
>>> difficult is the implementation.
>>> 
>>> KafkaConsumer is a typical example of a thread-less reader. The idea was
>> to
>>> allow different threading model on top of it. It could be a global single
>>> thread handles record fetching and processing in an event loop pattern;
>> it
>>> could also be one dedicated fetcher thread for each consumer and a
>> separate
>>> thread pool for record processing. The API gives the freedom of picking
>> up
>>> threading model to the users. To answer the first question, I would love
>> to
>>> have such a source reader API so Flink can choose whatever threading
>> model
>>> it wants. However, implementing such an interface could be pretty
>>> challenging and error prone.
>>> 
>>> On the other hand, having a source reader with a naive blocking socket is
>>> probably simple enough in most cases (actually sometimes this might even
>> be
>>> the most efficient way). But it does not leave much option to Flink other
>>> than creating one thread per reader.
>>> 
>>> Given the above thoughts, it might be reasonable to separate the
>>> SplitReader API into two: SyncReader and AsyncReader. The sync reader
>> just
>>> has a simple blocking takeNext() API. And the AsyncReader just has a
>>> pollNext(Callback) or Future<?> pollNext(). All the other methods are
>>> shared by both readers and could be put into a package private parent
>>> interface like BaseSplitReader.
>>> 
>>> Having these two readers allows both complicated and simple
>> implementation,
>>> depending on the SplitReader writers. From Flink's perspective, it will
>>> choose a more efficient threading model if the SplitReader is an
>>> AsyncReader. Otherwise, it may have to use the one thread per reader
>> model
>>> if the reader is a SyncReader. Users can also choose to implement both
>>> interface, in that case, it is up to Flink to choose which interface to
>> use.
>>> 
>>> Admittedly, this solution does have one more interface, but still seems
>>> rewarding. Any thoughts?
>>> 
>>> Thanks,
>>> 
>>> Jiangjie (Becket) Qin
>>> 
>>> 
>>> On Sun, Nov 18, 2018 at 11:33 PM Biao Liu <mm...@gmail.com> wrote:
>>> 
>>>> Hi community,
>>>> 
>>>> Thank you guys for sharing ideas.
>>>> 
>>>> The thing I really concern is about the thread mode.
>>>> Actually in Alibaba, we have implemented our "split reader" based source
>>>> two years ago. That's based on "SourceFunction", it's just an extension
>> not
>>>> a refactoring. It's almost same with the version Thomas and Jamie
>> described
>>>> in Google Doc. It really helps in many scenarios.
>>>> 
>>>> However I don't like the thread mode which starts a thread for each
>> split.
>>>> Starting extra thread in operator is not an ideal way IMO. Especially
>>>> thread count is decided by split count. So I was wondering if there is a
>>>> more elegant way. Do we really want these threads in Flink core?
>>>> 
>>>> I agree that blocking interface is more easy to implement. Could we at
>>>> least separate the split reader with source function into different
>>>> interfaces? Not all sources would like to read all splits concurrently.
>> In
>>>> batch scenario, reading splits one by one is more general. And also not
>> all
>>>> sources are partitioned, right?
>>>> I prefer there is a new source interface with "pull mode" only, no
>> split.
>>>> There is a splittable source extended it. And there is one
>> implementation
>>>> that starting threads for each split, reading all splits concurrently.
>>>> 
>>>> 
>>>> Thomas Weise <th...@apache.org> 于2018年11月18日周日 上午3:18写道:
>>>> 
>>>>> @Aljoscha to address your question first: In the case of the Kinesis
>>>>> consumer (with current Kinesis consumer API), there would also be N+1
>>>>> threads. I have implemented a prototype similar to what is shown in
>>>> Jamie's
>>>>> document, where the thread ownership is similar to what you have done
>> for
>>>>> Kafka.
>>>>> 
>>>>> The equivalent of split reader manages its own thread and the "source
>>>> main
>>>>> thread" is responsible for emitting the data. The interface between
>> the N
>>>>> reader threads and the 1 emitter is a blocking queue per consumer
>> thread.
>>>>> The emitter can now control which queue to consume from based on the
>>>> event
>>>>> time progress.
>>>>> 
>>>>> This is akin to a "non-blocking" interface *between emitter and split
>>>>> reader*. Emitter uses poll to retrieve records from the N queues (which
>>>>> requires non-blocking interaction). The emitter is independent of the
>>>> split
>>>>> reader implementation, that part could live in Flink.
>>>>> 
>>>>> Regarding whether or not to assume that split readers always need a
>>>> thread
>>>>> and in addition that these reader threads should be managed by Flink:
>> It
>>>>> depends on the API of respective external systems and I would not bake
>>>> that
>>>>> assumption into Flink. Some client libraries manage their own threads
>>>> (see
>>>>> push based API like JMS and as I understand it may also apply to the
>> new
>>>>> fan-out Kinesis API:
>>>>> 
>>>>> 
>>>> 
>> https://docs.aws.amazon.com/streams/latest/dev/building-enhanced-consumers-api.html
>>>>> ).
>>>>> In such cases it would not make sense to layer another reader thread on
>>>>> top. It may instead be better if Flink provides to the split reader the
>>>>> queue/buffer to push records to.
>>>>> 
>>>>> The discussion so far has largely ignored the discovery aspect. There
>> are
>>>>> some important considerations such as ordering dependency of splits and
>>>>> work rebalancing that may affect the split reader interface. Should we
>>>> fork
>>>>> this into a separate thread?
>>>>> 
>>>>> Thanks,
>>>>> Thomas
>>>>> 
>>>>> 
>>>>> On Fri, Nov 16, 2018 at 8:09 AM Piotr Nowojski <
>> piotr@data-artisans.com>
>>>>> wrote:
>>>>> 
>>>>>> Hi Jamie,
>>>>>> 
>>>>>> As it was already covered with my discussion with Becket, there is an
>>>>> easy
>>>>>> way to provide blocking API on top of non-blocking API. And yes we
>> both
>>>>>> agreed that blocking API is easier to implement by users.
>>>>>> 
>>>>>> I also do not agree with respect to usefulness of non blocking API.
>>>>>> Actually Kafka connector is the one that could be more efficient
>> thanks
>>>>> to
>>>>>> the removal of the one layer of threading.
>>>>>> 
>>>>>> Piotrek
>>>>>> 
>>>>>>> On 16 Nov 2018, at 02:21, Jamie Grier <jg...@lyft.com.INVALID>
>>>> wrote:
>>>>>>> 
>>>>>>> Thanks Aljoscha for getting this effort going!
>>>>>>> 
>>>>>>> There's been plenty of discussion here already and I'll add my big +1
>>>>> to
>>>>>>> making this interface very simple to implement for a new
>>>>>>> Source/SplitReader.  Writing a new production quality connector for
>>>>> Flink
>>>>>>> is very difficult today and requires a lot of detailed knowledge
>>>> about
>>>>>>> Flink, event time progress, watermarking, idle shard detection, etc
>>>> and
>>>>>> it
>>>>>>> would be good to move almost all of this type of code into Flink
>>>> itself
>>>>>> and
>>>>>>> out of source implementations.  I also think this is totally doable
>>>> and
>>>>>> I'm
>>>>>>> really excited to see this happening.
>>>>>>> 
>>>>>>> I do have a couple of thoughts about the API and the implementation..
>>>>>>> 
>>>>>>> In a perfect world there would be a single thread per Flink source
>>>>>> sub-task
>>>>>>> and no additional threads for SplitReaders -- but this assumes a
>>>> world
>>>>>>> where you have true async IO APIs for the upstream systems (like
>>>> Kafka
>>>>>> and
>>>>>>> Kinesis, S3, HDFS, etc).  If that world did exist the single thread
>>>>> could
>>>>>>> just sit in an efficient select() call waiting for new data to arrive
>>>>> on
>>>>>>> any Split.  That'd be awesome..
>>>>>>> 
>>>>>>> But, that world doesn't exist and given that practical consideration
>>>> I
>>>>>>> would think the next best implementation is going to be, in practice,
>>>>>>> probably a thread per SplitReader that does nothing but call the
>>>> source
>>>>>> API
>>>>>>> and drop whatever it reads into a (blocking) queue -- as Aljoscha
>>>>>> mentioned
>>>>>>> (calling it N+1) and as we started to describe here:
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> https://docs.google.com/document/d/13HxfsucoN7aUls1FX202KFAVZ4IMLXEZWyRfZNd6WFM/edit#heading=h.lfi2u4fv9cxa
>>>>>>> 
>>>>>>> I guess my point is that I think we should strive to move as much of
>>>>>>> something like the diagram referenced in the above doc into Flink
>>>>> itself
>>>>>>> and out of sources and simplify the SplitReader API as much as
>>>> possible
>>>>>> as
>>>>>>> well.
>>>>>>> 
>>>>>>> With the above in mind and with regard to the discussion about
>>>>> blocking,
>>>>>>> etc..  I'm not sure I agree with some of the discussion so far with
>>>>>> regard
>>>>>>> to this API design.  The calls to the upstream systems
>>>> (kafka/kinesis)
>>>>>> are
>>>>>>> in fact going to be blocking calls.  So a simple API without the
>>>>>> constraint
>>>>>>> that the methods must be implemented in a non-blocking way seems
>>>> better
>>>>>> to
>>>>>>> me from the point of view of somebody writing a new source
>>>>>> implementation.
>>>>>>> My concern is that if you force the implementer of the SplitReader
>>>>>>> interface to do so in a non-blocking way you're just going to make it
>>>>>>> harder to write those implementations.  Those calls to read the next
>>>>> bit
>>>>>> of
>>>>>>> data are going to be blocking calls with most known important sources
>>>>> --
>>>>>> at
>>>>>>> least Kafka/Kinesis/HDFS -- so I think maybe we should just deal with
>>>>>> that
>>>>>>> head on and work around it a higher level so the SplitReader
>>>> interface
>>>>>>> stays super simple to implement.  This means we manage all the
>>>>> threading
>>>>>> in
>>>>>>> Flink core, the API stays pull-based, and the implementer is allowed
>>>> to
>>>>>>> simply block until they have data to return.
>>>>>>> 
>>>>>>> I maybe would change my mind about this if truly asynchronous APIs to
>>>>> the
>>>>>>> upstream source systems were likely to be available in the near
>>>> future
>>>>> or
>>>>>>> are now and I'm just ignorant of it.  But even then the supporting
>>>> code
>>>>>> in
>>>>>>> Flink to drive async and sync sources would be different and in fact
>>>>> they
>>>>>>> might just have different APIs altogether -- SplitReader vs
>>>>>>> AsyncSplitReader maybe.
>>>>>>> 
>>>>>>> In the end I think playing with the implementation, across more than
>>>>> one
>>>>>>> source, and moving as much common code into Flink itself will reveal
>>>>> the
>>>>>>> best API of course.
>>>>>>> 
>>>>>>> One other interesting note is that you need to preserve per-partition
>>>>>>> ordering so you have to take care with the implementation if it were
>>>> to
>>>>>> be
>>>>>>> based on a thread pool and futures so as not to reorder the reads.
>>>>>>> 
>>>>>>> Anyway, I'm thrilled to see this starting to move forward and I'd
>>>> very
>>>>>> much
>>>>>>> like to help with the implementation wherever I can.  We're doing a
>>>>>>> simplified internal version of some of this at Lyft for just Kinesis
>>>>>>> because we need a solution for event time alignment in the very short
>>>>>> term
>>>>>>> but we'd like to immediately start helping to do this properly in
>>>> Flink
>>>>>>> after that.  One of the end goals for us is event time alignment
>>>> across
>>>>>>> heterogeneous sources.  Another is making it possible for non-expert
>>>>>> users
>>>>>>> to have a high probability of being able to write their own, correct,
>>>>>>> connectors.
>>>>>>> 
>>>>>>> -Jamie
>>>>>>> 
>>>>>>> On Thu, Nov 15, 2018 at 3:43 AM Aljoscha Krettek <
>>>> aljoscha@apache.org>
>>>>>>> wrote:
>>>>>>> 
>>>>>>>> Hi,
>>>>>>>> 
>>>>>>>> I thought I had sent this mail a while ago but I must have forgotten
>>>>> to
>>>>>>>> send it.
>>>>>>>> 
>>>>>>>> There is another thing we should consider for splits: the range of
>>>>>>>> timestamps that it can contain. For example, the splits of a file
>>>>> source
>>>>>>>> would know what the minimum and maximum timestamp in the splits is,
>>>>>>>> roughly. For infinite splits, such as Kafka partitions, the minimum
>>>>>> would
>>>>>>>> be meaningful but the maximum would be +Inf. If the splits expose
>>>> the
>>>>>>>> interval of time that they contain the readers, or the component
>>>> that
>>>>>>>> manages the readers can make decisions about which splits to forward
>>>>> and
>>>>>>>> read first. And it can also influence the minimum watermark that a
>>>>>> reader
>>>>>>>> forwards: it should never emit a watermark if it knows there are
>>>>> splits
>>>>>> to
>>>>>>>> read that have a lower minimum timestamp. I think it should be as
>>>> easy
>>>>>> as
>>>>>>>> adding a minimumTimestamp()/maximumTimestamp() method pair to the
>>>>> split
>>>>>>>> interface.
>>>>>>>> 
>>>>>>>> Another thing we need to resolve is the actual reader interface. I
>>>> see
>>>>>>>> there has been some good discussion but I don't know if we have
>>>>>> consensus.
>>>>>>>> We should try and see how specific sources could be implemented with
>>>>> the
>>>>>>>> new interface. For example, for Kafka I think we need to have N+1
>>>>>> threads
>>>>>>>> per task (where N is the number of splits that a task is reading
>>>>> from).
>>>>>> On
>>>>>>>> thread is responsible for reading from the splits. And each split
>>>> has
>>>>>> its
>>>>>>>> own (internal) thread for reading from Kafka and putting messages in
>>>>> an
>>>>>>>> internal queue to pull from. This is similar to how the current
>>>> Kafka
>>>>>>>> source is implemented, which has a separate fetcher thread. The
>>>> reason
>>>>>> for
>>>>>>>> this split is that we always need to try reading from Kafka to keep
>>>>> the
>>>>>>>> throughput up. In the current implementation the internal queue (or
>>>>>>>> handover) limits the read rate of the reader threads.
>>>>>>>> 
>>>>>>>> @Thomas, what do you think this would look like for Kinesis?
>>>>>>>> 
>>>>>>>> Best,
>>>>>>>> Aljoscha
>>>>>>>> 
>>>>>>>>> On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com>
>>>> wrote:
>>>>>>>>> 
>>>>>>>>> Hi Piotrek,
>>>>>>>>> 
>>>>>>>>> Thanks a lot for the detailed reply. All makes sense to me.
>>>>>>>>> 
>>>>>>>>> WRT the confusion between advance() / getCurrent(), do you think it
>>>>>> would
>>>>>>>>> help if we combine them and have something like:
>>>>>>>>> 
>>>>>>>>> CompletableFuture<T> getNext();
>>>>>>>>> long getWatermark();
>>>>>>>>> long getCurrentTimestamp();
>>>>>>>>> 
>>>>>>>>> Cheers,
>>>>>>>>> 
>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>> 
>>>>>>>>> On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <
>>>>>> piotr@data-artisans.com>
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Hi,
>>>>>>>>>> 
>>>>>>>>>> Thanks again for the detailed answer :) Sorry for responding with
>>>> a
>>>>>>>> delay.
>>>>>>>>>> 
>>>>>>>>>>> Completely agree that in pattern 2, having a callback is
>>>> necessary
>>>>>> for
>>>>>>>>>> that
>>>>>>>>>>> single thread outside of the connectors. And the connectors MUST
>>>>> have
>>>>>>>>>>> internal threads.
>>>>>>>>>> 
>>>>>>>>>> Yes, this thread will have to exists somewhere. In pattern 2 it
>>>>> exists
>>>>>>>> in
>>>>>>>>>> the connector (at least from the perspective of the Flink
>>>> execution
>>>>>>>>>> engine). In pattern 1 it exists inside the Flink execution engine.
>>>>>> With
>>>>>>>>>> completely blocking connectors, like simple reading from files,
>>>> both
>>>>>> of
>>>>>>>>>> those approaches are basically the same. The difference is when
>>>> user
>>>>>>>>>> implementing Flink source is already working with a non blocking
>>>>> code
>>>>>>>> with
>>>>>>>>>> some internal threads. In this case, pattern 1 would result in
>>>>> "double
>>>>>>>>>> thread wrapping”, while pattern 2 would allow to skip one layer of
>>>>>>>>>> indirection.
>>>>>>>>>> 
>>>>>>>>>>> If we go that way, we should have something like "void
>>>>>>>>>>> poll(Callback) / void advance(callback)". I am curious how would
>>>>>>>>>>> CompletableFuture work here, though. If 10 readers returns 10
>>>>>>>> completable
>>>>>>>>>>> futures, will there be 10 additional threads (so 20 threads in
>>>>> total)
>>>>>>>>>>> blocking waiting on them? Or will there be a single thread busy
>>>>> loop
>>>>>>>>>>> checking around?
>>>>>>>>>> 
>>>>>>>>>> To be honest, I haven’t thought this completely through and I
>>>>> haven’t
>>>>>>>>>> tested/POC’ed it. Having said that, I can think of at least couple
>>>>> of
>>>>>>>>>> solutions. First is something like this:
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>>>>>>>>>> <
>>>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> Line:
>>>>>>>>>> 
>>>>>>>>>>                             `blocked = split.process();`
>>>>>>>>>> 
>>>>>>>>>> Is where the execution goes into to the task/sources. This is
>>>> where
>>>>>> the
>>>>>>>>>> returned future is handled:
>>>>>>>>>> 
>>>>>>>>>>                             blocked.addListener(() -> {
>>>>>>>>>>                                 blockedSplits.remove(split);
>>>>>>>>>>                                 // reset the level priority to
>>>>>>>> prevent
>>>>>>>>>> previously-blocked splits from starving existing splits
>>>>>>>>>>                                 split.resetLevelPriority();
>>>>>>>>>>                                 waitingSplits.offer(split);
>>>>>>>>>>                             }, executor);
>>>>>>>>>> 
>>>>>>>>>> Fundamentally callbacks and Futures are more or less
>>>> interchangeable
>>>>>> You
>>>>>>>>>> can always wrap one into another (creating a callback that
>>>>> completes a
>>>>>>>>>> future and attach a callback once future completes). In this case
>>>>> the
>>>>>>>>>> difference for me is mostly:
>>>>>>>>>> - api with passing callback allows the callback to be fired
>>>> multiple
>>>>>>>> times
>>>>>>>>>> and to fire it even if the connector is not blocked. This is what
>>>> I
>>>>>>>> meant
>>>>>>>>>> by saying that api `CompletableFuture<?> isBlocked()` is a bit
>>>>>> simpler.
>>>>>>>>>> Connector can only return either “I’m not blocked” or “I’m blocked
>>>>>> and I
>>>>>>>>>> will tell you only once when I’m not blocked anymore”.
>>>>>>>>>> 
>>>>>>>>>> But this is not the most important thing for me here. For me
>>>>> important
>>>>>>>>>> thing is to try our best to make Flink task’s control and
>>>> execution
>>>>>>>> single
>>>>>>>>>> threaded. For that both callback and future APIs should work the
>>>>> same.
>>>>>>>>>> 
>>>>>>>>>>> WRT pattern 1, a single blocking take() API should just work. The
>>>>>> good
>>>>>>>>>>> thing is that a blocking read API is usually simpler to
>>>> implement.
>>>>>>>>>> 
>>>>>>>>>> Yes, they are easier to implement (especially if you are not the
>>>> one
>>>>>>>> that
>>>>>>>>>> have to deal with the additional threading required around them ;)
>>>>> ).
>>>>>>>> But
>>>>>>>>>> to answer this issue, if we choose pattern 2, we can always
>>>> provide
>>>>> a
>>>>>>>>>> proxy/wrapper that would using the internal thread implement the
>>>>>>>>>> non-blocking API while exposing blocking API to the user. It would
>>>>>>>>>> implement pattern 2 for the user exposing to him pattern 1. In
>>>> other
>>>>>>>> words
>>>>>>>>>> implementing pattern 1 in pattern 2 paradigm, while making it
>>>>> possible
>>>>>>>> to
>>>>>>>>>> implement pure pattern 2 connectors.
>>>>>>>>>> 
>>>>>>>>>>> BTW, one thing I am also trying to avoid is pushing users to
>>>>> perform
>>>>>> IO
>>>>>>>>>> in
>>>>>>>>>>> a method like "isBlocked()". If the method is expected to fetch
>>>>>> records
>>>>>>>>>>> (even if not returning them), naming it something more explicit
>>>>> would
>>>>>>>>>> help
>>>>>>>>>>> avoid confusion.
>>>>>>>>>> 
>>>>>>>>>> If we choose so, we could rework it into something like:
>>>>>>>>>> 
>>>>>>>>>> CompletableFuture<?> advance()
>>>>>>>>>> T getCurrent();
>>>>>>>>>> Watermark getCurrentWatermark()
>>>>>>>>>> 
>>>>>>>>>> But as I wrote before, this is more confusing to me for the exact
>>>>>>>> reasons
>>>>>>>>>> you mentioned :) I would be confused what should be done in
>>>>>> `adanvce()`
>>>>>>>> and
>>>>>>>>>> what in `getCurrent()`. However, again this naming issue is not
>>>> that
>>>>>>>>>> important to me and probably is matter of taste/personal
>>>>> preferences.
>>>>>>>>>> 
>>>>>>>>>> Piotrek
>>>>>>>>>> 
>>>>>>>>>>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com>
>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>> Hi Piotrek,
>>>>>>>>>>> 
>>>>>>>>>>> Thanks for the explanation. We are probably talking about the
>>>> same
>>>>>>>> thing
>>>>>>>>>>> but in different ways. To clarify a little bit, I think there are
>>>>> two
>>>>>>>>>>> patterns to read from a connector.
>>>>>>>>>>> 
>>>>>>>>>>> Pattern 1: Thread-less connector with a blocking read API.
>>>> Outside
>>>>> of
>>>>>>>> the
>>>>>>>>>>> connector, there is one IO thread per reader, doing blocking
>>>> read.
>>>>> An
>>>>>>>>>>> additional thread will interact with all the IO threads.
>>>>>>>>>>> Pattern 2: Connector with internal thread(s) and non-blocking
>>>> API.
>>>>>>>>>> Outside
>>>>>>>>>>> of the connector, there is one thread for ALL readers, doing IO
>>>>>> relying
>>>>>>>>>> on
>>>>>>>>>>> notification callbacks in the reader.
>>>>>>>>>>> 
>>>>>>>>>>> In both patterns, there must be at least one thread per
>>>> connector,
>>>>>>>> either
>>>>>>>>>>> inside (created by connector writers) or outside (created by
>>>> Flink)
>>>>>> of
>>>>>>>>>> the
>>>>>>>>>>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total,
>>>>> to
>>>>>>>> make
>>>>>>>>>>> sure that 1 thread is fully non-blocking.
>>>>>>>>>>> 
>>>>>>>>>>>> Btw, I don’t know if you understand my point. Having only
>>>> `poll()`
>>>>>> and
>>>>>>>>>>> `take()` is not enough for single threaded task. If our source
>>>>>>>> interface
>>>>>>>>>>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
>>>>>>>>>>> isBlocked(),`, there is no way to implement single threaded task
>>>>> that
>>>>>>>>>> both
>>>>>>>>>>> reads the data from the source connector and can also react to
>>>>> system
>>>>>>>>>>> events. Ok, non >blocking `poll()` would allow that, but with
>>>> busy
>>>>>>>>>> looping.
>>>>>>>>>>> 
>>>>>>>>>>> Completely agree that in pattern 2, having a callback is
>>>> necessary
>>>>>> for
>>>>>>>>>> that
>>>>>>>>>>> single thread outside of the connectors. And the connectors MUST
>>>>> have
>>>>>>>>>>> internal threads. If we go that way, we should have something
>>>> like
>>>>>>>> "void
>>>>>>>>>>> poll(Callback) / void advance(callback)". I am curious how would
>>>>>>>>>>> CompletableFuture work here, though. If 10 readers returns 10
>>>>>>>> completable
>>>>>>>>>>> futures, will there be 10 additional threads (so 20 threads in
>>>>> total)
>>>>>>>>>>> blocking waiting on them? Or will there be a single thread busy
>>>>> loop
>>>>>>>>>>> checking around?
>>>>>>>>>>> 
>>>>>>>>>>> WRT pattern 1, a single blocking take() API should just work. The
>>>>>> good
>>>>>>>>>>> thing is that a blocking read API is usually simpler to
>>>> implement.
>>>>> An
>>>>>>>>>>> additional non-blocking "T poll()" method here is indeed optional
>>>>> and
>>>>>>>>>> could
>>>>>>>>>>> be used in cases like Flink does not want the thread to block
>>>>>> forever.
>>>>>>>>>> They
>>>>>>>>>>> can also be combined to have a "T poll(Timeout)", which is
>>>> exactly
>>>>>> what
>>>>>>>>>>> KafkaConsumer did.
>>>>>>>>>>> 
>>>>>>>>>>> It sounds that you are proposing pattern 2 with something similar
>>>>> to
>>>>>>>> NIO2
>>>>>>>>>>> AsynchronousByteChannel[1]. That API would work, except that the
>>>>>>>>>> signature
>>>>>>>>>>> returning future seems not necessary. If that is the case, a
>>>> minor
>>>>>>>> change
>>>>>>>>>>> on the current FLIP proposal to have "void advance(callback)"
>>>>> should
>>>>>>>>>> work.
>>>>>>>>>>> And this means the connectors MUST have their internal threads.
>>>>>>>>>>> 
>>>>>>>>>>> BTW, one thing I am also trying to avoid is pushing users to
>>>>> perform
>>>>>> IO
>>>>>>>>>> in
>>>>>>>>>>> a method like "isBlocked()". If the method is expected to fetch
>>>>>> records
>>>>>>>>>>> (even if not returning them), naming it something more explicit
>>>>> would
>>>>>>>>>> help
>>>>>>>>>>> avoid confusion.
>>>>>>>>>>> 
>>>>>>>>>>> Thanks,
>>>>>>>>>>> 
>>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>> 
>>>>>>>>>>> [1]
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
>>>>>>>>>>> 
>>>>>>>>>>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <
>>>>>>>> piotr@data-artisans.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Hi
>>>>>>>>>>>> 
>>>>>>>>>>>> Good point with select/epoll, however I do not see how they
>>>>> couldn’t
>>>>>>>> be
>>>>>>>>>>>> with Flink if we would like single task in Flink to be
>>>>>> single-threaded
>>>>>>>>>> (and
>>>>>>>>>>>> I believe we should pursue this goal). If your connector blocks
>>>> on
>>>>>>>>>>>> `select`, then it can not process/handle control messages from
>>>>>> Flink,
>>>>>>>>>> like
>>>>>>>>>>>> checkpoints, releasing resources and potentially output flushes.
>>>>>> This
>>>>>>>>>> would
>>>>>>>>>>>> require tight integration between connector and Flink’s main
>>>> event
>>>>>>>>>>>> loop/selects/etc.
>>>>>>>>>>>> 
>>>>>>>>>>>> Looking at it from other perspective. Let’s assume that we have
>>>> a
>>>>>>>>>>>> connector implemented on top of `select`/`epoll`. In order to
>>>>>>>> integrate
>>>>>>>>>> it
>>>>>>>>>>>> with Flink’s checkpointing/flushes/resource releasing it will
>>>> have
>>>>>> to
>>>>>>>> be
>>>>>>>>>>>> executed in separate thread one way or another. At least if our
>>>>> API
>>>>>>>> will
>>>>>>>>>>>> enforce/encourage non blocking implementations with some kind of
>>>>>>>>>>>> notifications (`isBlocked()` or `notify()` callback), some
>>>>>> connectors
>>>>>>>>>> might
>>>>>>>>>>>> skip one layer of wapping threads.
>>>>>>>>>>>> 
>>>>>>>>>>>> Btw, I don’t know if you understand my point. Having only
>>>> `poll()`
>>>>>> and
>>>>>>>>>>>> `take()` is not enough for single threaded task. If our source
>>>>>>>> interface
>>>>>>>>>>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
>>>>>>>>>>>> isBlocked(),`, there is no way to implement single threaded task
>>>>>> that
>>>>>>>>>> both
>>>>>>>>>>>> reads the data from the source connector and can also react to
>>>>>> system
>>>>>>>>>>>> events. Ok, non blocking `poll()` would allow that, but with
>>>> busy
>>>>>>>>>> looping.
>>>>>>>>>>>> 
>>>>>>>>>>>> Piotrek
>>>>>>>>>>>> 
>>>>>>>>>>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com>
>>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Hi Piotrek,
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> But I don’t see a reason why we should expose both blocking
>>>>>> `take()`
>>>>>>>>>> and
>>>>>>>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone
>>>> (Flink
>>>>>>>>>> engine
>>>>>>>>>>>> or
>>>>>>>>>>>>> connector) would have to do the same busy
>>>>>>>>>>>>>> looping anyway and I think it would be better to have a
>>>> simpler
>>>>>>>>>>>> connector
>>>>>>>>>>>>> API (that would solve our problems) and force connectors to
>>>>> comply
>>>>>>>> one
>>>>>>>>>>>> way
>>>>>>>>>>>>> or another.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> If we let the block happen inside the connector, the blocking
>>>>> does
>>>>>>>> not
>>>>>>>>>>>> have
>>>>>>>>>>>>> to be a busy loop. For example, to do the block waiting
>>>>>> efficiently,
>>>>>>>>>> the
>>>>>>>>>>>>> connector can use java NIO selector().select which relies on OS
>>>>>>>> syscall
>>>>>>>>>>>>> like epoll[1] instead of busy looping. But if Flink engine
>>>> blocks
>>>>>>>>>> outside
>>>>>>>>>>>>> the connector, it pretty much has to do the busy loop. So if
>>>>> there
>>>>>> is
>>>>>>>>>>>> only
>>>>>>>>>>>>> one API to get the element, a blocking getNextElement() makes
>>>>> more
>>>>>>>>>> sense.
>>>>>>>>>>>>> In any case, we should avoid ambiguity. It has to be crystal
>>>>> clear
>>>>>>>>>> about
>>>>>>>>>>>>> whether a method is expected to be blocking or non-blocking.
>>>>>>>> Otherwise
>>>>>>>>>> it
>>>>>>>>>>>>> would be very difficult for Flink engine to do the right thing
>>>>> with
>>>>>>>> the
>>>>>>>>>>>>> connectors. At the first glance at getCurrent(), the expected
>>>>>>>> behavior
>>>>>>>>>> is
>>>>>>>>>>>>> not quite clear.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> That said, I do agree that functionality wise, poll() and
>>>> take()
>>>>>> kind
>>>>>>>>>> of
>>>>>>>>>>>>> overlap. But they are actually not quite different from
>>>>>>>>>>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the
>>>> only
>>>>>>>>>>>>> difference is that poll() also returns the next record if it is
>>>>>>>>>>>> available.
>>>>>>>>>>>>> But I agree that the isBlocked() + getNextElement() is more
>>>>>> flexible
>>>>>>>> as
>>>>>>>>>>>>> users can just check the record availability, but not fetch the
>>>>>> next
>>>>>>>>>>>>> element.
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> In case of thread-less readers with only non-blocking
>>>>>> `queue.poll()`
>>>>>>>>>> (is
>>>>>>>>>>>>> that really a thing? I can not think about a real
>>>> implementation
>>>>>> that
>>>>>>>>>>>>> enforces such constraints)
>>>>>>>>>>>>> Right, it is pretty much a syntax sugar to allow user combine
>>>> the
>>>>>>>>>>>>> check-and-take into one method. It could be achieved with
>>>>>>>> isBlocked() +
>>>>>>>>>>>>> getNextElement().
>>>>>>>>>>>>> 
>>>>>>>>>>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
>>>>>>>>>> piotr@data-artisans.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Hi Becket,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> With my proposal, both of your examples would have to be
>>>> solved
>>>>> by
>>>>>>>> the
>>>>>>>>>>>>>> connector and solution to both problems would be the same:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Pretend that connector is never blocked (`isBlocked() { return
>>>>>>>>>>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking
>>>>>>>> fashion
>>>>>>>>>>>> (or
>>>>>>>>>>>>>> semi blocking with return of control from time to time to
>>>> allow
>>>>>> for
>>>>>>>>>>>>>> checkpointing, network flushing and other resource management
>>>>>> things
>>>>>>>>>> to
>>>>>>>>>>>>>> happen in the same main thread). In other words, exactly how
>>>> you
>>>>>>>> would
>>>>>>>>>>>>>> implement `take()` method or how the same source connector
>>>> would
>>>>>> be
>>>>>>>>>>>>>> implemented NOW with current source interface. The difference
>>>>> with
>>>>>>>>>>>> current
>>>>>>>>>>>>>> interface would be only that main loop would be outside of the
>>>>>>>>>>>> connector,
>>>>>>>>>>>>>> and instead of periodically releasing checkpointing lock,
>>>>>>>> periodically
>>>>>>>>>>>>>> `return null;` or `return Optional.empty();` from
>>>>>>>> `getNextElement()`.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> In case of thread-less readers with only non-blocking
>>>>>> `queue.poll()`
>>>>>>>>>> (is
>>>>>>>>>>>>>> that really a thing? I can not think about a real
>>>> implementation
>>>>>>>> that
>>>>>>>>>>>>>> enforces such constraints), we could provide a wrapper that
>>>>> hides
>>>>>>>> the
>>>>>>>>>>>> busy
>>>>>>>>>>>>>> looping. The same applies how to solve forever blocking
>>>> readers
>>>>> -
>>>>>> we
>>>>>>>>>>>> could
>>>>>>>>>>>>>> provider another wrapper running the connector in separate
>>>>> thread.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> But I don’t see a reason why we should expose both blocking
>>>>>> `take()`
>>>>>>>>>> and
>>>>>>>>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone
>>>>> (Flink
>>>>>>>>>>>> engine or
>>>>>>>>>>>>>> connector) would have to do the same busy looping anyway and I
>>>>>> think
>>>>>>>>>> it
>>>>>>>>>>>>>> would be better to have a simpler connector API (that would
>>>>> solve
>>>>>>>> our
>>>>>>>>>>>>>> problems) and force connectors to comply one way or another.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Piotrek
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com>
>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Hi Piotr,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> I might have misunderstood you proposal. But let me try to
>>>>>> explain
>>>>>>>> my
>>>>>>>>>>>>>>> concern. I am thinking about the following case:
>>>>>>>>>>>>>>> 1. a reader has the following two interfaces,
>>>>>>>>>>>>>>> boolean isBlocked()
>>>>>>>>>>>>>>> T getNextElement()
>>>>>>>>>>>>>>> 2. the implementation of getNextElement() is non-blocking.
>>>>>>>>>>>>>>> 3. The reader is thread-less, i.e. it does not have any
>>>>> internal
>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>> For example, it might just delegate the getNextElement() to a
>>>>>>>>>>>>>> queue.poll(),
>>>>>>>>>>>>>>> and isBlocked() is just queue.isEmpty().
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> How can Flink efficiently implement a blocking reading
>>>> behavior
>>>>>>>> with
>>>>>>>>>>>> this
>>>>>>>>>>>>>>> reader? Either a tight loop or a backoff interval is needed.
>>>>>>>> Neither
>>>>>>>>>> of
>>>>>>>>>>>>>>> them is ideal.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Now let's say in the reader mentioned above implements a
>>>>> blocking
>>>>>>>>>>>>>>> getNextElement() method. Because there is no internal thread
>>>> in
>>>>>> the
>>>>>>>>>>>>>> reader,
>>>>>>>>>>>>>>> after isBlocked() returns false. Flink will still have to
>>>> loop
>>>>> on
>>>>>>>>>>>>>>> isBlocked() to check whether the next record is available. If
>>>>> the
>>>>>>>>>> next
>>>>>>>>>>>>>>> record reaches after 10 min, it is a tight loop for 10 min.
>>>> You
>>>>>>>> have
>>>>>>>>>>>>>>> probably noticed that in this case, even isBlocked() returns
>>>> a
>>>>>>>>>> future,
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>> future() will not be completed if Flink does not call some
>>>>> method
>>>>>>>>>> from
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> reader, because the reader has no internal thread to complete
>>>>>> that
>>>>>>>>>>>> future
>>>>>>>>>>>>>>> by itself.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Due to the above reasons, a blocking take() API would allow
>>>>> Flink
>>>>>>>> to
>>>>>>>>>>>> have
>>>>>>>>>>>>>>> an efficient way to read from a reader. There are many ways
>>>> to
>>>>>> wake
>>>>>>>>>> up
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> blocking thread when checkpointing is needed depending on the
>>>>>>>>>>>>>>> implementation. But I think the poll()/take() API would also
>>>>> work
>>>>>>>> in
>>>>>>>>>>>> that
>>>>>>>>>>>>>>> case.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
>>>>>>>>>> piotr@data-artisans.com
>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> a)
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> BTW, regarding the isBlock() method, I have a few more
>>>>>> questions.
>>>>>>>>>> 21,
>>>>>>>>>>>>>> Is
>>>>>>>>>>>>>>>> a method isReady() with boolean as a return value
>>>>>>>>>>>>>>>>> equivalent? Personally I found it is a little bit confusing
>>>>> in
>>>>>>>> what
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> supposed to be returned when the future is completed. 22. if
>>>>>>>>>>>>>>>>> the implementation of isBlocked() is optional, how do the
>>>>>> callers
>>>>>>>>>>>> know
>>>>>>>>>>>>>>>> whether the method is properly implemented or not?
>>>>>>>>>>>>>>>>> Does not implemented mean it always return a completed
>>>>> future?
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an
>>>>> equivalent
>>>>>>>> to
>>>>>>>>>>>>>>>> `boolean hasNext()` which in case of “false” provides some
>>>>> kind
>>>>>>>> of a
>>>>>>>>>>>>>>>> listener/callback that notifies about presence of next
>>>>> element.
>>>>>>>>>> There
>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>> some minor details, like `CompletableFuture<?>` has a
>>>> minimal
>>>>>> two
>>>>>>>>>>>> state
>>>>>>>>>>>>>>>> logic:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1. Future is completed - we have more data
>>>>>>>>>>>>>>>> 2. Future not yet completed - we don’t have data now, but we
>>>>>>>>>> might/we
>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>> have in the future
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit
>>>>> more
>>>>>>>>>>>>>>>> complicated/dispersed and can lead/encourage `notify()`
>>>> spam.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> b)
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method
>>>>> like
>>>>>>>>>>>>>> `getNext`
>>>>>>>>>>>>>>>> the `getNext` would need return a
>>>>>>>>>>>>>>>>> `ElementWithTimestamp` because some sources want to add
>>>>>> timestamp
>>>>>>>>>> to
>>>>>>>>>>>>>>>> every element. IMO, this is not so memory friendly
>>>>>>>>>>>>>>>>> so I prefer this design.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Guowei I don’t quite understand this. Could you elaborate
>>>> why
>>>>>>>>>> having a
>>>>>>>>>>>>>>>> separate `advance()` help?
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> c)
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Regarding advance/poll/take. What’s the value of having two
>>>>>>>> separate
>>>>>>>>>>>>>>>> methods: poll and take? Which one of them should be called
>>>> and
>>>>>>>> which
>>>>>>>>>>>>>>>> implemented? What’s the benefit of having those methods
>>>>> compared
>>>>>>>> to
>>>>>>>>>>>>>> having
>>>>>>>>>>>>>>>> a one single method `getNextElement()` (or `pollElement() or
>>>>>>>>>> whatever
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> name it) with following contract:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> CompletableFuture<?> isBlocked();
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>>>> Return next element - will be called only if `isBlocked()`
>>>> is
>>>>>>>>>>>> completed.
>>>>>>>>>>>>>>>> Try to implement it in non blocking fashion, but if that’s
>>>>>>>>>> impossible
>>>>>>>>>>>> or
>>>>>>>>>>>>>>>> you just don’t need the effort, you can block in this
>>>> method.
>>>>>>>>>>>>>>>> */
>>>>>>>>>>>>>>>> T getNextElement();
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> I mean, if the connector is implemented non-blockingly,
>>>> Flink
>>>>>>>> should
>>>>>>>>>>>> use
>>>>>>>>>>>>>>>> it that way. If it’s not, then `poll()` will `throw new
>>>>>>>>>>>>>>>> NotImplementedException()`. Implementing both of them and
>>>>>>>> providing
>>>>>>>>>>>>>> both of
>>>>>>>>>>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them
>>>>>> into
>>>>>>>> a
>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>> method call that should preferably (but not necessarily need
>>>>> to)
>>>>>>>> be
>>>>>>>>>>>>>>>> non-blocking? It’s not like we are implementing general
>>>>> purpose
>>>>>>>>>>>> `Queue`,
>>>>>>>>>>>>>>>> which users might want to call either of `poll` or `take`.
>>>> We
>>>>>>>> would
>>>>>>>>>>>>>> always
>>>>>>>>>>>>>>>> prefer to call `poll`, but if it’s blocking, then still we
>>>>> have
>>>>>> no
>>>>>>>>>>>>>> choice,
>>>>>>>>>>>>>>>> but to call it and block on it.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> d)
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
>>>> source
>>>>>> is
>>>>>>>>>> very
>>>>>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
>>>>>> another
>>>>>>>>>> way
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly if
>>>>>> every
>>>>>>>>>>>>>> advance
>>>>>>>>>>>>>>>>> call return a Future.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> I didn’t want to mention this, to not clog my initial
>>>>> proposal,
>>>>>>>> but
>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>> is a simple solution for the problem:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> public interface SplitReader {
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> (…)
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> CompletableFuture<?> NOT_BLOCKED =
>>>>>>>>>>>>>>>> CompletableFuture.completedFuture(null);
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>>>> * Returns a future that will be completed when the page
>>>> source
>>>>>>>>>>>>>> becomes
>>>>>>>>>>>>>>>> * unblocked.  If the page source is not blocked, this method
>>>>>>>>>> should
>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>> * {@code NOT_BLOCKED}.
>>>>>>>>>>>>>>>> */
>>>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked()
>>>>>>>>>>>>>>>> {
>>>>>>>>>>>>>>>>  return NOT_BLOCKED;
>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> If we are blocked and we are waiting for the IO, then
>>>>> creating a
>>>>>>>> new
>>>>>>>>>>>>>>>> Future is non-issue. Under full throttle/throughput and not
>>>>>>>> blocked
>>>>>>>>>>>>>> sources
>>>>>>>>>>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve
>>>>> the
>>>>>>>>>>>>>> problem.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> One more remark, non-blocking sources might be a necessity
>>>> in
>>>>> a
>>>>>>>>>> single
>>>>>>>>>>>>>>>> threaded model without a checkpointing lock. (Currently when
>>>>>>>> sources
>>>>>>>>>>>> are
>>>>>>>>>>>>>>>> blocked, they can release checkpointing lock and re-acquire
>>>> it
>>>>>>>> again
>>>>>>>>>>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for
>>>>>>>> checkpoints
>>>>>>>>>> to
>>>>>>>>>>>>>>>> happen when source is idling. In that case either `notify()`
>>>>> or
>>>>>> my
>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>> `isBlocked()` would allow to avoid busy-looping.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Piotrek
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Hi Thomas,
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> The iterator-like API was also the first thing that came to
>>>>> me.
>>>>>>>> But
>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>> seems a little confusing that hasNext() does not mean "the
>>>>>> stream
>>>>>>>>>> has
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>> ended", but means "the next record is ready", which is
>>>>>>>> repurposing
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> well
>>>>>>>>>>>>>>>>> known meaning of hasNext(). If we follow the
>>>> hasNext()/next()
>>>>>>>>>>>> pattern,
>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>> additional isNextReady() method to indicate whether the
>>>> next
>>>>>>>> record
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> ready seems more intuitive to me.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Similarly, in poll()/take() pattern, another method of
>>>>> isDone()
>>>>>>>> is
>>>>>>>>>>>>>> needed
>>>>>>>>>>>>>>>>> to indicate whether the stream has ended or not.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
>>>>>>>>>>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
>>>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>> When I am implementing a reader, I could have a couple of
>>>>>>>> choices:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> - A thread-less reader that does not have any internal
>>>>> thread.
>>>>>>>>>>>>>>>>> - When poll() is called, the same calling thread will
>>>>> perform a
>>>>>>>>>> bunch
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> IO asynchronously.
>>>>>>>>>>>>>>>>> - When take() is called, the same calling thread will
>>>>> perform a
>>>>>>>>>>>>>>>> bunch
>>>>>>>>>>>>>>>>> of IO and wait until the record is ready.
>>>>>>>>>>>>>>>>> - A reader with internal threads performing network IO and
>>>>> put
>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>> into a buffer.
>>>>>>>>>>>>>>>>> - When poll() is called, the calling thread simply reads
>>>> from
>>>>>>>> the
>>>>>>>>>>>>>>>>> buffer and return empty result immediately if there is no
>>>>>>>> record.
>>>>>>>>>>>>>>>>> - When take() is called, the calling thread reads from the
>>>>>>>> buffer
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> block waiting if the buffer is empty.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady()
>>>>> API,
>>>>>>>> it
>>>>>>>>>> is
>>>>>>>>>>>>>>>> less
>>>>>>>>>>>>>>>>> intuitive for the reader developers to write the
>>>> thread-less
>>>>>>>>>> pattern.
>>>>>>>>>>>>>>>>> Although technically speaking one can still do the
>>>>> asynchronous
>>>>>>>> IO
>>>>>>>>>> to
>>>>>>>>>>>>>>>>> prepare the record in isNextReady(). But it is inexplicit
>>>> and
>>>>>>>> seems
>>>>>>>>>>>>>>>>> somewhat hacky.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <
>>>> thw@apache.org>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Couple more points regarding discovery:
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> The proposal mentions that discovery could be outside the
>>>>>>>>>> execution
>>>>>>>>>>>>>>>> graph.
>>>>>>>>>>>>>>>>>> Today, discovered partitions/shards are checkpointed. I
>>>>>> believe
>>>>>>>>>> that
>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>> also need to be the case in the future, even when
>>>> discovery
>>>>>> and
>>>>>>>>>>>>>> reading
>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>> split between different tasks.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> For cases such as resharding of a Kinesis stream, the
>>>>>>>> relationship
>>>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>>>> splits needs to be considered. Splits cannot be randomly
>>>>>>>>>> distributed
>>>>>>>>>>>>>>>> over
>>>>>>>>>>>>>>>>>> readers in certain situations. An example was mentioned
>>>>> here:
>>>>>>>>>>>>>>>>>> 
>>>>>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Thomas
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <
>>>> thw@apache.org
>>>>>> 
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Thanks for getting the ball rolling on this!
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Can the number of splits decrease? Yes, splits can be
>>>>> closed
>>>>>>>> and
>>>>>>>>>> go
>>>>>>>>>>>>>>>> away.
>>>>>>>>>>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing
>>>>>> shards
>>>>>>>>>>>> will
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> closed and replaced with a new shard).
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive
>>>>>> approach
>>>>>>>>>>>> would
>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> the thread-less IO model (pull based, non-blocking,
>>>> caller
>>>>>>>>>>>> retrieves
>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>> records when available). The current Kinesis API requires
>>>>> the
>>>>>>>> use
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> threads. But that can be internal to the split reader and
>>>>>> does
>>>>>>>>>> not
>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> be a source API concern. In fact, that's what we are
>>>>> working
>>>>>> on
>>>>>>>>>>>> right
>>>>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>>>> as improvement to the existing consumer: Each shard
>>>>> consumer
>>>>>>>>>> thread
>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>> push to a queue, the consumer main thread will poll the
>>>>>>>> queue(s).
>>>>>>>>>>>> It
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> The proposed SplitReader interface would fit the
>>>>> thread-less
>>>>>> IO
>>>>>>>>>>>>>> model.
>>>>>>>>>>>>>>>>>>> Similar to an iterator, we find out if there is a new
>>>>> element
>>>>>>>>>>>>>> (hasNext)
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> if so, move to it (next()). Separate calls deliver the
>>>> meta
>>>>>>>>>>>>>> information
>>>>>>>>>>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer
>>>> a
>>>>>>>>>> timeout
>>>>>>>>>>>>>>>>>> option,
>>>>>>>>>>>>>>>>>>> so that the caller does not end up in a busy wait. On the
>>>>>> other
>>>>>>>>>>>>>> hand, a
>>>>>>>>>>>>>>>>>>> caller processing multiple splits may want to cycle
>>>> through
>>>>>>>> fast,
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> process elements of other splits as soon as they become
>>>>>>>>>> available.
>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>> nice
>>>>>>>>>>>>>>>>>>> thing is that this "split merge" logic can now live in
>>>>> Flink
>>>>>>>> and
>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> optimized and shared between different sources.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>> Thomas
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <
>>>>>> guowei.mgw@gmail.com
>>>>>>>>> 
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>> Thanks Aljoscha for this FLIP.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
>>>>>> source
>>>>>>>> is
>>>>>>>>>>>>>> very
>>>>>>>>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may
>>>> be
>>>>>>>>>> another
>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly
>>>>> if
>>>>>>>>>> every
>>>>>>>>>>>>>>>>>> advance
>>>>>>>>>>>>>>>>>>>> call return a Future.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> public interface Listener {
>>>>>>>>>>>>>>>>>>>> public void notify();
>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> public interface SplitReader() {
>>>>>>>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>>>>>>>> * When there is no element temporarily, this will return
>>>>>>>>>> false.
>>>>>>>>>>>>>>>>>>>> * When elements is available again splitReader can call
>>>>>>>>>>>>>>>>>>>> listener.notify()
>>>>>>>>>>>>>>>>>>>> * In addition the frame would check `advance`
>>>>> periodically .
>>>>>>>>>>>>>>>>>>>> * Of course advance can always return true and ignore
>>>> the
>>>>>>>>>>>>>>>> listener
>>>>>>>>>>>>>>>>>>>> argument for simplicity.
>>>>>>>>>>>>>>>>>>>> */
>>>>>>>>>>>>>>>>>>>> public boolean advance(Listener listener);
>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create
>>>> all
>>>>>>>> Splits
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>>>> to create a SplitReader from a Split. But there is no
>>>>>> strategy
>>>>>>>>>> for
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>> to choose how to assign the splits to the tasks. I think
>>>>> we
>>>>>>>>>> could
>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> Enum to let user to choose.
>>>>>>>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
>>>>>>>>>>>>>>>>>>>> Location,
>>>>>>>>>>>>>>>>>>>> Workload,
>>>>>>>>>>>>>>>>>>>> Random,
>>>>>>>>>>>>>>>>>>>> Average
>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>> */
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one
>>>> method
>>>>>> like
>>>>>>>>>>>>>>>> `getNext`
>>>>>>>>>>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp`
>>>>>>>> because
>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>> sources want to add timestamp to every element. IMO,
>>>> this
>>>>> is
>>>>>>>> not
>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>>>> friendly so I prefer this design.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
>>>>>>>>>> 下午6:08写道:
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite
>>>> a
>>>>>> lot
>>>>>>>> of
>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>> possible improvements. I have one proposal. Instead of
>>>>>>>> having a
>>>>>>>>>>>>>>>> method:
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> boolean advance() throws IOException;
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> I would replace it with
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> /*
>>>>>>>>>>>>>>>>>>>>> * Return a future, which when completed means that
>>>> source
>>>>>> has
>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>> data
>>>>>>>>>>>>>>>>>>>>> and getNext() will not block.
>>>>>>>>>>>>>>>>>>>>> * If you wish to use benefits of non blocking
>>>> connectors,
>>>>>>>>>> please
>>>>>>>>>>>>>>>>>>>>> implement this method appropriately.
>>>>>>>>>>>>>>>>>>>>> */
>>>>>>>>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
>>>>>>>>>>>>>>>>>>>>> return CompletableFuture.completedFuture(null);
>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Couple of arguments:
>>>>>>>>>>>>>>>>>>>>> 1. I don’t understand the division of work between
>>>>>>>> `advance()`
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> `getCurrent()`. What should be done in which,
>>>> especially
>>>>>> for
>>>>>>>>>>>>>>>> connectors
>>>>>>>>>>>>>>>>>>>>> that handle records in batches (like Kafka) and when
>>>>> should
>>>>>>>> you
>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>> `advance` and when `getCurrent()`.
>>>>>>>>>>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will
>>>>>> allow
>>>>>>>>>> us
>>>>>>>>>>>> in
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> future to have asynchronous/non blocking connectors and
>>>>>> more
>>>>>>>>>>>>>>>>>> efficiently
>>>>>>>>>>>>>>>>>>>>> handle large number of blocked threads, without busy
>>>>>> waiting.
>>>>>>>>>>>> While
>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> same time it doesn’t add much complexity, since naive
>>>>>>>> connector
>>>>>>>>>>>>>>>>>>>>> implementations can be always blocking.
>>>>>>>>>>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread
>>>>> pool
>>>>>>>> of
>>>>>>>>>>>> task
>>>>>>>>>>>>>>>>>>>>> executors, instead of one thread per task.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Piotrek
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
>>>>>>>>>> aljoscha@apache.org
>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Hi All,
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> In order to finally get the ball rolling on the new
>>>>> source
>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>> that we have discussed for so long I finally created a
>>>>>> FLIP:
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
>>>>>>>>>> work/discussion
>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>> adding per-partition watermark support to the Kinesis
>>>>>> source
>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>>>>> this would enable generic implementation of event-time
>>>>>>>>>> alignment
>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
>>>>>>>>>> alignment
>>>>>>>>>>>>>>>> part,
>>>>>>>>>>>>>>>>>>>>> especially the part about information sharing between
>>>>>>>>>> operations
>>>>>>>>>>>>>> (I'm
>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> calling it state sharing because state has a special
>>>>>> meaning
>>>>>>>> in
>>>>>>>>>>>>>>>> Flink).
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Please discuss away!
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Aljoscha
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> 
>> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Thanks Piotrek,

> void SplitReader#addSplit(Split)
> boolean SplitReader#doesWantMoreSplits()

I have two questions about this API.
1. What if the SplitReader implementation cannot easily add a split to read
on the fly?
2. Does Flink have to be involved in splits assignment?

I am wondering if it would be simpler to let the enumerator indicate
whether a split reassignment is needed. If the answer is yes, Flink can
just start from the beginning to get all the splits and create one reader
per split. This might be a little more expensive than dynamically adding a
split to a reader, but given that the splits change should be rare, it is
probably acceptable.

In the Kafka case, the SplitT may just be a consumer. The enumerator will
simply check if the topic has new partitions to be assigned to this reader.

@Biao,
If I understand correctly, the concern you raised was that a Source may
return a lot of splits and thus Flink may have to create a lot of fetcher
threads. This is a valid concern, but I cannot think of a solution to that.
After all, the SplitReaders may be written by third parties. Poor
implementations seem difficult to prevent.

Thanks,

Jiangjie (Becket) Qin

On Wed, Nov 21, 2018 at 10:13 PM Piotr Nowojski <pi...@data-artisans.com>
wrote:

> Hi again,
>
> > However I don't like the thread mode which starts a thread for each
> split.
> > Starting extra thread in operator is not an ideal way IMO. Especially
> > thread count is decided by split count. So I was wondering if there is a
> > more elegant way. Do we really want these threads in Flink core?
>
> Biao you have raised an important issue. Indeed it seems like the current
> proposal is missing something. I would guess that we need a mechanism for
> adding new splits to an already existing SplitReader and some logic to
> determine whether current instance can accept more splits or not. For
> example
>
> void SplitReader#addSplit(Split)
> boolean SplitReader#doesWantMoreSplits()
>
> Flink could randomly/round robin assign new splits to the SplitReaders
> that `doWantMoreSplits()`. Batch file readers might implement some custom
> logic in `doesWantMoreSplits()`, like one SplitReader can have at most N
> enqueued splits?
>
> Also what about Kafka. Isn’t it the case that one KafkaConsumer can read
> from multiple splits? So Kafka’s SplitReader should always return true from
> `doesWantMoreSplits()`?
>
> What do you think?
>
> Re: Becket
>
> I’m +1 for Sync and AsyncSplitReader.
>
> Piotrek
>
> > On 21 Nov 2018, at 14:49, Becket Qin <be...@gmail.com> wrote:
> >
> > Hi Aljoscha,
> >
> > Good point on the potential optimization in the source. One thing to
> > clarify, by "adding a minimumTimestamp()/maximumTimestamp() method pair
> to
> > the split interface", did you mean "split reader interface"? If so, what
> > should the readers do if they do not have such additional information? I
> am
> > wondering if it is possible to leave such optimization to the source
> > internal implementation.
> >
> > @all
> > After reading all the feedback, Biao and I talked a little bit offline.
> We
> > would like to share some new thoughts with you and see what do you think.
> >
> > When looking at the Source API, we were trying to answer two questions.
> > First of all, how would Flink use this API if someone else implemented
> it.
> > Secondly, how would the connector contributors implement the interface?
> How
> > difficult is the implementation.
> >
> > KafkaConsumer is a typical example of a thread-less reader. The idea was
> to
> > allow different threading model on top of it. It could be a global single
> > thread handles record fetching and processing in an event loop pattern;
> it
> > could also be one dedicated fetcher thread for each consumer and a
> separate
> > thread pool for record processing. The API gives the freedom of picking
> up
> > threading model to the users. To answer the first question, I would love
> to
> > have such a source reader API so Flink can choose whatever threading
> model
> > it wants. However, implementing such an interface could be pretty
> > challenging and error prone.
> >
> > On the other hand, having a source reader with a naive blocking socket is
> > probably simple enough in most cases (actually sometimes this might even
> be
> > the most efficient way). But it does not leave much option to Flink other
> > than creating one thread per reader.
> >
> > Given the above thoughts, it might be reasonable to separate the
> > SplitReader API into two: SyncReader and AsyncReader. The sync reader
> just
> > has a simple blocking takeNext() API. And the AsyncReader just has a
> > pollNext(Callback) or Future<?> pollNext(). All the other methods are
> > shared by both readers and could be put into a package private parent
> > interface like BaseSplitReader.
> >
> > Having these two readers allows both complicated and simple
> implementation,
> > depending on the SplitReader writers. From Flink's perspective, it will
> > choose a more efficient threading model if the SplitReader is an
> > AsyncReader. Otherwise, it may have to use the one thread per reader
> model
> > if the reader is a SyncReader. Users can also choose to implement both
> > interface, in that case, it is up to Flink to choose which interface to
> use.
> >
> > Admittedly, this solution does have one more interface, but still seems
> > rewarding. Any thoughts?
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> >
> > On Sun, Nov 18, 2018 at 11:33 PM Biao Liu <mm...@gmail.com> wrote:
> >
> >> Hi community,
> >>
> >> Thank you guys for sharing ideas.
> >>
> >> The thing I really concern is about the thread mode.
> >> Actually in Alibaba, we have implemented our "split reader" based source
> >> two years ago. That's based on "SourceFunction", it's just an extension
> not
> >> a refactoring. It's almost same with the version Thomas and Jamie
> described
> >> in Google Doc. It really helps in many scenarios.
> >>
> >> However I don't like the thread mode which starts a thread for each
> split.
> >> Starting extra thread in operator is not an ideal way IMO. Especially
> >> thread count is decided by split count. So I was wondering if there is a
> >> more elegant way. Do we really want these threads in Flink core?
> >>
> >> I agree that blocking interface is more easy to implement. Could we at
> >> least separate the split reader with source function into different
> >> interfaces? Not all sources would like to read all splits concurrently.
> In
> >> batch scenario, reading splits one by one is more general. And also not
> all
> >> sources are partitioned, right?
> >> I prefer there is a new source interface with "pull mode" only, no
> split.
> >> There is a splittable source extended it. And there is one
> implementation
> >> that starting threads for each split, reading all splits concurrently.
> >>
> >>
> >> Thomas Weise <th...@apache.org> 于2018年11月18日周日 上午3:18写道:
> >>
> >>> @Aljoscha to address your question first: In the case of the Kinesis
> >>> consumer (with current Kinesis consumer API), there would also be N+1
> >>> threads. I have implemented a prototype similar to what is shown in
> >> Jamie's
> >>> document, where the thread ownership is similar to what you have done
> for
> >>> Kafka.
> >>>
> >>> The equivalent of split reader manages its own thread and the "source
> >> main
> >>> thread" is responsible for emitting the data. The interface between
> the N
> >>> reader threads and the 1 emitter is a blocking queue per consumer
> thread.
> >>> The emitter can now control which queue to consume from based on the
> >> event
> >>> time progress.
> >>>
> >>> This is akin to a "non-blocking" interface *between emitter and split
> >>> reader*. Emitter uses poll to retrieve records from the N queues (which
> >>> requires non-blocking interaction). The emitter is independent of the
> >> split
> >>> reader implementation, that part could live in Flink.
> >>>
> >>> Regarding whether or not to assume that split readers always need a
> >> thread
> >>> and in addition that these reader threads should be managed by Flink:
> It
> >>> depends on the API of respective external systems and I would not bake
> >> that
> >>> assumption into Flink. Some client libraries manage their own threads
> >> (see
> >>> push based API like JMS and as I understand it may also apply to the
> new
> >>> fan-out Kinesis API:
> >>>
> >>>
> >>
> https://docs.aws.amazon.com/streams/latest/dev/building-enhanced-consumers-api.html
> >>> ).
> >>> In such cases it would not make sense to layer another reader thread on
> >>> top. It may instead be better if Flink provides to the split reader the
> >>> queue/buffer to push records to.
> >>>
> >>> The discussion so far has largely ignored the discovery aspect. There
> are
> >>> some important considerations such as ordering dependency of splits and
> >>> work rebalancing that may affect the split reader interface. Should we
> >> fork
> >>> this into a separate thread?
> >>>
> >>> Thanks,
> >>> Thomas
> >>>
> >>>
> >>> On Fri, Nov 16, 2018 at 8:09 AM Piotr Nowojski <
> piotr@data-artisans.com>
> >>> wrote:
> >>>
> >>>> Hi Jamie,
> >>>>
> >>>> As it was already covered with my discussion with Becket, there is an
> >>> easy
> >>>> way to provide blocking API on top of non-blocking API. And yes we
> both
> >>>> agreed that blocking API is easier to implement by users.
> >>>>
> >>>> I also do not agree with respect to usefulness of non blocking API.
> >>>> Actually Kafka connector is the one that could be more efficient
> thanks
> >>> to
> >>>> the removal of the one layer of threading.
> >>>>
> >>>> Piotrek
> >>>>
> >>>>> On 16 Nov 2018, at 02:21, Jamie Grier <jg...@lyft.com.INVALID>
> >> wrote:
> >>>>>
> >>>>> Thanks Aljoscha for getting this effort going!
> >>>>>
> >>>>> There's been plenty of discussion here already and I'll add my big +1
> >>> to
> >>>>> making this interface very simple to implement for a new
> >>>>> Source/SplitReader.  Writing a new production quality connector for
> >>> Flink
> >>>>> is very difficult today and requires a lot of detailed knowledge
> >> about
> >>>>> Flink, event time progress, watermarking, idle shard detection, etc
> >> and
> >>>> it
> >>>>> would be good to move almost all of this type of code into Flink
> >> itself
> >>>> and
> >>>>> out of source implementations.  I also think this is totally doable
> >> and
> >>>> I'm
> >>>>> really excited to see this happening.
> >>>>>
> >>>>> I do have a couple of thoughts about the API and the implementation..
> >>>>>
> >>>>> In a perfect world there would be a single thread per Flink source
> >>>> sub-task
> >>>>> and no additional threads for SplitReaders -- but this assumes a
> >> world
> >>>>> where you have true async IO APIs for the upstream systems (like
> >> Kafka
> >>>> and
> >>>>> Kinesis, S3, HDFS, etc).  If that world did exist the single thread
> >>> could
> >>>>> just sit in an efficient select() call waiting for new data to arrive
> >>> on
> >>>>> any Split.  That'd be awesome..
> >>>>>
> >>>>> But, that world doesn't exist and given that practical consideration
> >> I
> >>>>> would think the next best implementation is going to be, in practice,
> >>>>> probably a thread per SplitReader that does nothing but call the
> >> source
> >>>> API
> >>>>> and drop whatever it reads into a (blocking) queue -- as Aljoscha
> >>>> mentioned
> >>>>> (calling it N+1) and as we started to describe here:
> >>>>>
> >>>>
> >>>
> >>
> https://docs.google.com/document/d/13HxfsucoN7aUls1FX202KFAVZ4IMLXEZWyRfZNd6WFM/edit#heading=h.lfi2u4fv9cxa
> >>>>>
> >>>>> I guess my point is that I think we should strive to move as much of
> >>>>> something like the diagram referenced in the above doc into Flink
> >>> itself
> >>>>> and out of sources and simplify the SplitReader API as much as
> >> possible
> >>>> as
> >>>>> well.
> >>>>>
> >>>>> With the above in mind and with regard to the discussion about
> >>> blocking,
> >>>>> etc..  I'm not sure I agree with some of the discussion so far with
> >>>> regard
> >>>>> to this API design.  The calls to the upstream systems
> >> (kafka/kinesis)
> >>>> are
> >>>>> in fact going to be blocking calls.  So a simple API without the
> >>>> constraint
> >>>>> that the methods must be implemented in a non-blocking way seems
> >> better
> >>>> to
> >>>>> me from the point of view of somebody writing a new source
> >>>> implementation.
> >>>>> My concern is that if you force the implementer of the SplitReader
> >>>>> interface to do so in a non-blocking way you're just going to make it
> >>>>> harder to write those implementations.  Those calls to read the next
> >>> bit
> >>>> of
> >>>>> data are going to be blocking calls with most known important sources
> >>> --
> >>>> at
> >>>>> least Kafka/Kinesis/HDFS -- so I think maybe we should just deal with
> >>>> that
> >>>>> head on and work around it a higher level so the SplitReader
> >> interface
> >>>>> stays super simple to implement.  This means we manage all the
> >>> threading
> >>>> in
> >>>>> Flink core, the API stays pull-based, and the implementer is allowed
> >> to
> >>>>> simply block until they have data to return.
> >>>>>
> >>>>> I maybe would change my mind about this if truly asynchronous APIs to
> >>> the
> >>>>> upstream source systems were likely to be available in the near
> >> future
> >>> or
> >>>>> are now and I'm just ignorant of it.  But even then the supporting
> >> code
> >>>> in
> >>>>> Flink to drive async and sync sources would be different and in fact
> >>> they
> >>>>> might just have different APIs altogether -- SplitReader vs
> >>>>> AsyncSplitReader maybe.
> >>>>>
> >>>>> In the end I think playing with the implementation, across more than
> >>> one
> >>>>> source, and moving as much common code into Flink itself will reveal
> >>> the
> >>>>> best API of course.
> >>>>>
> >>>>> One other interesting note is that you need to preserve per-partition
> >>>>> ordering so you have to take care with the implementation if it were
> >> to
> >>>> be
> >>>>> based on a thread pool and futures so as not to reorder the reads.
> >>>>>
> >>>>> Anyway, I'm thrilled to see this starting to move forward and I'd
> >> very
> >>>> much
> >>>>> like to help with the implementation wherever I can.  We're doing a
> >>>>> simplified internal version of some of this at Lyft for just Kinesis
> >>>>> because we need a solution for event time alignment in the very short
> >>>> term
> >>>>> but we'd like to immediately start helping to do this properly in
> >> Flink
> >>>>> after that.  One of the end goals for us is event time alignment
> >> across
> >>>>> heterogeneous sources.  Another is making it possible for non-expert
> >>>> users
> >>>>> to have a high probability of being able to write their own, correct,
> >>>>> connectors.
> >>>>>
> >>>>> -Jamie
> >>>>>
> >>>>> On Thu, Nov 15, 2018 at 3:43 AM Aljoscha Krettek <
> >> aljoscha@apache.org>
> >>>>> wrote:
> >>>>>
> >>>>>> Hi,
> >>>>>>
> >>>>>> I thought I had sent this mail a while ago but I must have forgotten
> >>> to
> >>>>>> send it.
> >>>>>>
> >>>>>> There is another thing we should consider for splits: the range of
> >>>>>> timestamps that it can contain. For example, the splits of a file
> >>> source
> >>>>>> would know what the minimum and maximum timestamp in the splits is,
> >>>>>> roughly. For infinite splits, such as Kafka partitions, the minimum
> >>>> would
> >>>>>> be meaningful but the maximum would be +Inf. If the splits expose
> >> the
> >>>>>> interval of time that they contain the readers, or the component
> >> that
> >>>>>> manages the readers can make decisions about which splits to forward
> >>> and
> >>>>>> read first. And it can also influence the minimum watermark that a
> >>>> reader
> >>>>>> forwards: it should never emit a watermark if it knows there are
> >>> splits
> >>>> to
> >>>>>> read that have a lower minimum timestamp. I think it should be as
> >> easy
> >>>> as
> >>>>>> adding a minimumTimestamp()/maximumTimestamp() method pair to the
> >>> split
> >>>>>> interface.
> >>>>>>
> >>>>>> Another thing we need to resolve is the actual reader interface. I
> >> see
> >>>>>> there has been some good discussion but I don't know if we have
> >>>> consensus.
> >>>>>> We should try and see how specific sources could be implemented with
> >>> the
> >>>>>> new interface. For example, for Kafka I think we need to have N+1
> >>>> threads
> >>>>>> per task (where N is the number of splits that a task is reading
> >>> from).
> >>>> On
> >>>>>> thread is responsible for reading from the splits. And each split
> >> has
> >>>> its
> >>>>>> own (internal) thread for reading from Kafka and putting messages in
> >>> an
> >>>>>> internal queue to pull from. This is similar to how the current
> >> Kafka
> >>>>>> source is implemented, which has a separate fetcher thread. The
> >> reason
> >>>> for
> >>>>>> this split is that we always need to try reading from Kafka to keep
> >>> the
> >>>>>> throughput up. In the current implementation the internal queue (or
> >>>>>> handover) limits the read rate of the reader threads.
> >>>>>>
> >>>>>> @Thomas, what do you think this would look like for Kinesis?
> >>>>>>
> >>>>>> Best,
> >>>>>> Aljoscha
> >>>>>>
> >>>>>>> On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com>
> >> wrote:
> >>>>>>>
> >>>>>>> Hi Piotrek,
> >>>>>>>
> >>>>>>> Thanks a lot for the detailed reply. All makes sense to me.
> >>>>>>>
> >>>>>>> WRT the confusion between advance() / getCurrent(), do you think it
> >>>> would
> >>>>>>> help if we combine them and have something like:
> >>>>>>>
> >>>>>>> CompletableFuture<T> getNext();
> >>>>>>> long getWatermark();
> >>>>>>> long getCurrentTimestamp();
> >>>>>>>
> >>>>>>> Cheers,
> >>>>>>>
> >>>>>>> Jiangjie (Becket) Qin
> >>>>>>>
> >>>>>>> On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <
> >>>> piotr@data-artisans.com>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi,
> >>>>>>>>
> >>>>>>>> Thanks again for the detailed answer :) Sorry for responding with
> >> a
> >>>>>> delay.
> >>>>>>>>
> >>>>>>>>> Completely agree that in pattern 2, having a callback is
> >> necessary
> >>>> for
> >>>>>>>> that
> >>>>>>>>> single thread outside of the connectors. And the connectors MUST
> >>> have
> >>>>>>>>> internal threads.
> >>>>>>>>
> >>>>>>>> Yes, this thread will have to exists somewhere. In pattern 2 it
> >>> exists
> >>>>>> in
> >>>>>>>> the connector (at least from the perspective of the Flink
> >> execution
> >>>>>>>> engine). In pattern 1 it exists inside the Flink execution engine.
> >>>> With
> >>>>>>>> completely blocking connectors, like simple reading from files,
> >> both
> >>>> of
> >>>>>>>> those approaches are basically the same. The difference is when
> >> user
> >>>>>>>> implementing Flink source is already working with a non blocking
> >>> code
> >>>>>> with
> >>>>>>>> some internal threads. In this case, pattern 1 would result in
> >>> "double
> >>>>>>>> thread wrapping”, while pattern 2 would allow to skip one layer of
> >>>>>>>> indirection.
> >>>>>>>>
> >>>>>>>>> If we go that way, we should have something like "void
> >>>>>>>>> poll(Callback) / void advance(callback)". I am curious how would
> >>>>>>>>> CompletableFuture work here, though. If 10 readers returns 10
> >>>>>> completable
> >>>>>>>>> futures, will there be 10 additional threads (so 20 threads in
> >>> total)
> >>>>>>>>> blocking waiting on them? Or will there be a single thread busy
> >>> loop
> >>>>>>>>> checking around?
> >>>>>>>>
> >>>>>>>> To be honest, I haven’t thought this completely through and I
> >>> haven’t
> >>>>>>>> tested/POC’ed it. Having said that, I can think of at least couple
> >>> of
> >>>>>>>> solutions. First is something like this:
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> >>>>>>>> <
> >>>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>> Line:
> >>>>>>>>
> >>>>>>>>                              `blocked = split.process();`
> >>>>>>>>
> >>>>>>>> Is where the execution goes into to the task/sources. This is
> >> where
> >>>> the
> >>>>>>>> returned future is handled:
> >>>>>>>>
> >>>>>>>>                              blocked.addListener(() -> {
> >>>>>>>>                                  blockedSplits.remove(split);
> >>>>>>>>                                  // reset the level priority to
> >>>>>> prevent
> >>>>>>>> previously-blocked splits from starving existing splits
> >>>>>>>>                                  split.resetLevelPriority();
> >>>>>>>>                                  waitingSplits.offer(split);
> >>>>>>>>                              }, executor);
> >>>>>>>>
> >>>>>>>> Fundamentally callbacks and Futures are more or less
> >> interchangeable
> >>>> You
> >>>>>>>> can always wrap one into another (creating a callback that
> >>> completes a
> >>>>>>>> future and attach a callback once future completes). In this case
> >>> the
> >>>>>>>> difference for me is mostly:
> >>>>>>>> - api with passing callback allows the callback to be fired
> >> multiple
> >>>>>> times
> >>>>>>>> and to fire it even if the connector is not blocked. This is what
> >> I
> >>>>>> meant
> >>>>>>>> by saying that api `CompletableFuture<?> isBlocked()` is a bit
> >>>> simpler.
> >>>>>>>> Connector can only return either “I’m not blocked” or “I’m blocked
> >>>> and I
> >>>>>>>> will tell you only once when I’m not blocked anymore”.
> >>>>>>>>
> >>>>>>>> But this is not the most important thing for me here. For me
> >>> important
> >>>>>>>> thing is to try our best to make Flink task’s control and
> >> execution
> >>>>>> single
> >>>>>>>> threaded. For that both callback and future APIs should work the
> >>> same.
> >>>>>>>>
> >>>>>>>>> WRT pattern 1, a single blocking take() API should just work. The
> >>>> good
> >>>>>>>>> thing is that a blocking read API is usually simpler to
> >> implement.
> >>>>>>>>
> >>>>>>>> Yes, they are easier to implement (especially if you are not the
> >> one
> >>>>>> that
> >>>>>>>> have to deal with the additional threading required around them ;)
> >>> ).
> >>>>>> But
> >>>>>>>> to answer this issue, if we choose pattern 2, we can always
> >> provide
> >>> a
> >>>>>>>> proxy/wrapper that would using the internal thread implement the
> >>>>>>>> non-blocking API while exposing blocking API to the user. It would
> >>>>>>>> implement pattern 2 for the user exposing to him pattern 1. In
> >> other
> >>>>>> words
> >>>>>>>> implementing pattern 1 in pattern 2 paradigm, while making it
> >>> possible
> >>>>>> to
> >>>>>>>> implement pure pattern 2 connectors.
> >>>>>>>>
> >>>>>>>>> BTW, one thing I am also trying to avoid is pushing users to
> >>> perform
> >>>> IO
> >>>>>>>> in
> >>>>>>>>> a method like "isBlocked()". If the method is expected to fetch
> >>>> records
> >>>>>>>>> (even if not returning them), naming it something more explicit
> >>> would
> >>>>>>>> help
> >>>>>>>>> avoid confusion.
> >>>>>>>>
> >>>>>>>> If we choose so, we could rework it into something like:
> >>>>>>>>
> >>>>>>>> CompletableFuture<?> advance()
> >>>>>>>> T getCurrent();
> >>>>>>>> Watermark getCurrentWatermark()
> >>>>>>>>
> >>>>>>>> But as I wrote before, this is more confusing to me for the exact
> >>>>>> reasons
> >>>>>>>> you mentioned :) I would be confused what should be done in
> >>>> `adanvce()`
> >>>>>> and
> >>>>>>>> what in `getCurrent()`. However, again this naming issue is not
> >> that
> >>>>>>>> important to me and probably is matter of taste/personal
> >>> preferences.
> >>>>>>>>
> >>>>>>>> Piotrek
> >>>>>>>>
> >>>>>>>>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com>
> >> wrote:
> >>>>>>>>>
> >>>>>>>>> Hi Piotrek,
> >>>>>>>>>
> >>>>>>>>> Thanks for the explanation. We are probably talking about the
> >> same
> >>>>>> thing
> >>>>>>>>> but in different ways. To clarify a little bit, I think there are
> >>> two
> >>>>>>>>> patterns to read from a connector.
> >>>>>>>>>
> >>>>>>>>> Pattern 1: Thread-less connector with a blocking read API.
> >> Outside
> >>> of
> >>>>>> the
> >>>>>>>>> connector, there is one IO thread per reader, doing blocking
> >> read.
> >>> An
> >>>>>>>>> additional thread will interact with all the IO threads.
> >>>>>>>>> Pattern 2: Connector with internal thread(s) and non-blocking
> >> API.
> >>>>>>>> Outside
> >>>>>>>>> of the connector, there is one thread for ALL readers, doing IO
> >>>> relying
> >>>>>>>> on
> >>>>>>>>> notification callbacks in the reader.
> >>>>>>>>>
> >>>>>>>>> In both patterns, there must be at least one thread per
> >> connector,
> >>>>>> either
> >>>>>>>>> inside (created by connector writers) or outside (created by
> >> Flink)
> >>>> of
> >>>>>>>> the
> >>>>>>>>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total,
> >>> to
> >>>>>> make
> >>>>>>>>> sure that 1 thread is fully non-blocking.
> >>>>>>>>>
> >>>>>>>>>> Btw, I don’t know if you understand my point. Having only
> >> `poll()`
> >>>> and
> >>>>>>>>> `take()` is not enough for single threaded task. If our source
> >>>>>> interface
> >>>>>>>>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
> >>>>>>>>> isBlocked(),`, there is no way to implement single threaded task
> >>> that
> >>>>>>>> both
> >>>>>>>>> reads the data from the source connector and can also react to
> >>> system
> >>>>>>>>> events. Ok, non >blocking `poll()` would allow that, but with
> >> busy
> >>>>>>>> looping.
> >>>>>>>>>
> >>>>>>>>> Completely agree that in pattern 2, having a callback is
> >> necessary
> >>>> for
> >>>>>>>> that
> >>>>>>>>> single thread outside of the connectors. And the connectors MUST
> >>> have
> >>>>>>>>> internal threads. If we go that way, we should have something
> >> like
> >>>>>> "void
> >>>>>>>>> poll(Callback) / void advance(callback)". I am curious how would
> >>>>>>>>> CompletableFuture work here, though. If 10 readers returns 10
> >>>>>> completable
> >>>>>>>>> futures, will there be 10 additional threads (so 20 threads in
> >>> total)
> >>>>>>>>> blocking waiting on them? Or will there be a single thread busy
> >>> loop
> >>>>>>>>> checking around?
> >>>>>>>>>
> >>>>>>>>> WRT pattern 1, a single blocking take() API should just work. The
> >>>> good
> >>>>>>>>> thing is that a blocking read API is usually simpler to
> >> implement.
> >>> An
> >>>>>>>>> additional non-blocking "T poll()" method here is indeed optional
> >>> and
> >>>>>>>> could
> >>>>>>>>> be used in cases like Flink does not want the thread to block
> >>>> forever.
> >>>>>>>> They
> >>>>>>>>> can also be combined to have a "T poll(Timeout)", which is
> >> exactly
> >>>> what
> >>>>>>>>> KafkaConsumer did.
> >>>>>>>>>
> >>>>>>>>> It sounds that you are proposing pattern 2 with something similar
> >>> to
> >>>>>> NIO2
> >>>>>>>>> AsynchronousByteChannel[1]. That API would work, except that the
> >>>>>>>> signature
> >>>>>>>>> returning future seems not necessary. If that is the case, a
> >> minor
> >>>>>> change
> >>>>>>>>> on the current FLIP proposal to have "void advance(callback)"
> >>> should
> >>>>>>>> work.
> >>>>>>>>> And this means the connectors MUST have their internal threads.
> >>>>>>>>>
> >>>>>>>>> BTW, one thing I am also trying to avoid is pushing users to
> >>> perform
> >>>> IO
> >>>>>>>> in
> >>>>>>>>> a method like "isBlocked()". If the method is expected to fetch
> >>>> records
> >>>>>>>>> (even if not returning them), naming it something more explicit
> >>> would
> >>>>>>>> help
> >>>>>>>>> avoid confusion.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>>
> >>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>
> >>>>>>>>> [1]
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
> >>>>>>>>>
> >>>>>>>>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <
> >>>>>> piotr@data-artisans.com>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi
> >>>>>>>>>>
> >>>>>>>>>> Good point with select/epoll, however I do not see how they
> >>> couldn’t
> >>>>>> be
> >>>>>>>>>> with Flink if we would like single task in Flink to be
> >>>> single-threaded
> >>>>>>>> (and
> >>>>>>>>>> I believe we should pursue this goal). If your connector blocks
> >> on
> >>>>>>>>>> `select`, then it can not process/handle control messages from
> >>>> Flink,
> >>>>>>>> like
> >>>>>>>>>> checkpoints, releasing resources and potentially output flushes.
> >>>> This
> >>>>>>>> would
> >>>>>>>>>> require tight integration between connector and Flink’s main
> >> event
> >>>>>>>>>> loop/selects/etc.
> >>>>>>>>>>
> >>>>>>>>>> Looking at it from other perspective. Let’s assume that we have
> >> a
> >>>>>>>>>> connector implemented on top of `select`/`epoll`. In order to
> >>>>>> integrate
> >>>>>>>> it
> >>>>>>>>>> with Flink’s checkpointing/flushes/resource releasing it will
> >> have
> >>>> to
> >>>>>> be
> >>>>>>>>>> executed in separate thread one way or another. At least if our
> >>> API
> >>>>>> will
> >>>>>>>>>> enforce/encourage non blocking implementations with some kind of
> >>>>>>>>>> notifications (`isBlocked()` or `notify()` callback), some
> >>>> connectors
> >>>>>>>> might
> >>>>>>>>>> skip one layer of wapping threads.
> >>>>>>>>>>
> >>>>>>>>>> Btw, I don’t know if you understand my point. Having only
> >> `poll()`
> >>>> and
> >>>>>>>>>> `take()` is not enough for single threaded task. If our source
> >>>>>> interface
> >>>>>>>>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
> >>>>>>>>>> isBlocked(),`, there is no way to implement single threaded task
> >>>> that
> >>>>>>>> both
> >>>>>>>>>> reads the data from the source connector and can also react to
> >>>> system
> >>>>>>>>>> events. Ok, non blocking `poll()` would allow that, but with
> >> busy
> >>>>>>>> looping.
> >>>>>>>>>>
> >>>>>>>>>> Piotrek
> >>>>>>>>>>
> >>>>>>>>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com>
> >>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>> Hi Piotrek,
> >>>>>>>>>>>
> >>>>>>>>>>>> But I don’t see a reason why we should expose both blocking
> >>>> `take()`
> >>>>>>>> and
> >>>>>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone
> >> (Flink
> >>>>>>>> engine
> >>>>>>>>>> or
> >>>>>>>>>>> connector) would have to do the same busy
> >>>>>>>>>>>> looping anyway and I think it would be better to have a
> >> simpler
> >>>>>>>>>> connector
> >>>>>>>>>>> API (that would solve our problems) and force connectors to
> >>> comply
> >>>>>> one
> >>>>>>>>>> way
> >>>>>>>>>>> or another.
> >>>>>>>>>>>
> >>>>>>>>>>> If we let the block happen inside the connector, the blocking
> >>> does
> >>>>>> not
> >>>>>>>>>> have
> >>>>>>>>>>> to be a busy loop. For example, to do the block waiting
> >>>> efficiently,
> >>>>>>>> the
> >>>>>>>>>>> connector can use java NIO selector().select which relies on OS
> >>>>>> syscall
> >>>>>>>>>>> like epoll[1] instead of busy looping. But if Flink engine
> >> blocks
> >>>>>>>> outside
> >>>>>>>>>>> the connector, it pretty much has to do the busy loop. So if
> >>> there
> >>>> is
> >>>>>>>>>> only
> >>>>>>>>>>> one API to get the element, a blocking getNextElement() makes
> >>> more
> >>>>>>>> sense.
> >>>>>>>>>>> In any case, we should avoid ambiguity. It has to be crystal
> >>> clear
> >>>>>>>> about
> >>>>>>>>>>> whether a method is expected to be blocking or non-blocking.
> >>>>>> Otherwise
> >>>>>>>> it
> >>>>>>>>>>> would be very difficult for Flink engine to do the right thing
> >>> with
> >>>>>> the
> >>>>>>>>>>> connectors. At the first glance at getCurrent(), the expected
> >>>>>> behavior
> >>>>>>>> is
> >>>>>>>>>>> not quite clear.
> >>>>>>>>>>>
> >>>>>>>>>>> That said, I do agree that functionality wise, poll() and
> >> take()
> >>>> kind
> >>>>>>>> of
> >>>>>>>>>>> overlap. But they are actually not quite different from
> >>>>>>>>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the
> >> only
> >>>>>>>>>>> difference is that poll() also returns the next record if it is
> >>>>>>>>>> available.
> >>>>>>>>>>> But I agree that the isBlocked() + getNextElement() is more
> >>>> flexible
> >>>>>> as
> >>>>>>>>>>> users can just check the record availability, but not fetch the
> >>>> next
> >>>>>>>>>>> element.
> >>>>>>>>>>>
> >>>>>>>>>>>> In case of thread-less readers with only non-blocking
> >>>> `queue.poll()`
> >>>>>>>> (is
> >>>>>>>>>>> that really a thing? I can not think about a real
> >> implementation
> >>>> that
> >>>>>>>>>>> enforces such constraints)
> >>>>>>>>>>> Right, it is pretty much a syntax sugar to allow user combine
> >> the
> >>>>>>>>>>> check-and-take into one method. It could be achieved with
> >>>>>> isBlocked() +
> >>>>>>>>>>> getNextElement().
> >>>>>>>>>>>
> >>>>>>>>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>>
> >>>>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>>>
> >>>>>>>>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
> >>>>>>>> piotr@data-artisans.com>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hi Becket,
> >>>>>>>>>>>>
> >>>>>>>>>>>> With my proposal, both of your examples would have to be
> >> solved
> >>> by
> >>>>>> the
> >>>>>>>>>>>> connector and solution to both problems would be the same:
> >>>>>>>>>>>>
> >>>>>>>>>>>> Pretend that connector is never blocked (`isBlocked() { return
> >>>>>>>>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking
> >>>>>> fashion
> >>>>>>>>>> (or
> >>>>>>>>>>>> semi blocking with return of control from time to time to
> >> allow
> >>>> for
> >>>>>>>>>>>> checkpointing, network flushing and other resource management
> >>>> things
> >>>>>>>> to
> >>>>>>>>>>>> happen in the same main thread). In other words, exactly how
> >> you
> >>>>>> would
> >>>>>>>>>>>> implement `take()` method or how the same source connector
> >> would
> >>>> be
> >>>>>>>>>>>> implemented NOW with current source interface. The difference
> >>> with
> >>>>>>>>>> current
> >>>>>>>>>>>> interface would be only that main loop would be outside of the
> >>>>>>>>>> connector,
> >>>>>>>>>>>> and instead of periodically releasing checkpointing lock,
> >>>>>> periodically
> >>>>>>>>>>>> `return null;` or `return Optional.empty();` from
> >>>>>> `getNextElement()`.
> >>>>>>>>>>>>
> >>>>>>>>>>>> In case of thread-less readers with only non-blocking
> >>>> `queue.poll()`
> >>>>>>>> (is
> >>>>>>>>>>>> that really a thing? I can not think about a real
> >> implementation
> >>>>>> that
> >>>>>>>>>>>> enforces such constraints), we could provide a wrapper that
> >>> hides
> >>>>>> the
> >>>>>>>>>> busy
> >>>>>>>>>>>> looping. The same applies how to solve forever blocking
> >> readers
> >>> -
> >>>> we
> >>>>>>>>>> could
> >>>>>>>>>>>> provider another wrapper running the connector in separate
> >>> thread.
> >>>>>>>>>>>>
> >>>>>>>>>>>> But I don’t see a reason why we should expose both blocking
> >>>> `take()`
> >>>>>>>> and
> >>>>>>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone
> >>> (Flink
> >>>>>>>>>> engine or
> >>>>>>>>>>>> connector) would have to do the same busy looping anyway and I
> >>>> think
> >>>>>>>> it
> >>>>>>>>>>>> would be better to have a simpler connector API (that would
> >>> solve
> >>>>>> our
> >>>>>>>>>>>> problems) and force connectors to comply one way or another.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Piotrek
> >>>>>>>>>>>>
> >>>>>>>>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com>
> >>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Hi Piotr,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I might have misunderstood you proposal. But let me try to
> >>>> explain
> >>>>>> my
> >>>>>>>>>>>>> concern. I am thinking about the following case:
> >>>>>>>>>>>>> 1. a reader has the following two interfaces,
> >>>>>>>>>>>>> boolean isBlocked()
> >>>>>>>>>>>>> T getNextElement()
> >>>>>>>>>>>>> 2. the implementation of getNextElement() is non-blocking.
> >>>>>>>>>>>>> 3. The reader is thread-less, i.e. it does not have any
> >>> internal
> >>>>>>>>>> thread.
> >>>>>>>>>>>>> For example, it might just delegate the getNextElement() to a
> >>>>>>>>>>>> queue.poll(),
> >>>>>>>>>>>>> and isBlocked() is just queue.isEmpty().
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> How can Flink efficiently implement a blocking reading
> >> behavior
> >>>>>> with
> >>>>>>>>>> this
> >>>>>>>>>>>>> reader? Either a tight loop or a backoff interval is needed.
> >>>>>> Neither
> >>>>>>>> of
> >>>>>>>>>>>>> them is ideal.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Now let's say in the reader mentioned above implements a
> >>> blocking
> >>>>>>>>>>>>> getNextElement() method. Because there is no internal thread
> >> in
> >>>> the
> >>>>>>>>>>>> reader,
> >>>>>>>>>>>>> after isBlocked() returns false. Flink will still have to
> >> loop
> >>> on
> >>>>>>>>>>>>> isBlocked() to check whether the next record is available. If
> >>> the
> >>>>>>>> next
> >>>>>>>>>>>>> record reaches after 10 min, it is a tight loop for 10 min.
> >> You
> >>>>>> have
> >>>>>>>>>>>>> probably noticed that in this case, even isBlocked() returns
> >> a
> >>>>>>>> future,
> >>>>>>>>>>>> that
> >>>>>>>>>>>>> future() will not be completed if Flink does not call some
> >>> method
> >>>>>>>> from
> >>>>>>>>>>>> the
> >>>>>>>>>>>>> reader, because the reader has no internal thread to complete
> >>>> that
> >>>>>>>>>> future
> >>>>>>>>>>>>> by itself.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Due to the above reasons, a blocking take() API would allow
> >>> Flink
> >>>>>> to
> >>>>>>>>>> have
> >>>>>>>>>>>>> an efficient way to read from a reader. There are many ways
> >> to
> >>>> wake
> >>>>>>>> up
> >>>>>>>>>>>> the
> >>>>>>>>>>>>> blocking thread when checkpointing is needed depending on the
> >>>>>>>>>>>>> implementation. But I think the poll()/take() API would also
> >>> work
> >>>>>> in
> >>>>>>>>>> that
> >>>>>>>>>>>>> case.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
> >>>>>>>> piotr@data-artisans.com
> >>>>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> a)
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> BTW, regarding the isBlock() method, I have a few more
> >>>> questions.
> >>>>>>>> 21,
> >>>>>>>>>>>> Is
> >>>>>>>>>>>>>> a method isReady() with boolean as a return value
> >>>>>>>>>>>>>>> equivalent? Personally I found it is a little bit confusing
> >>> in
> >>>>>> what
> >>>>>>>>>> is
> >>>>>>>>>>>>>> supposed to be returned when the future is completed. 22. if
> >>>>>>>>>>>>>>> the implementation of isBlocked() is optional, how do the
> >>>> callers
> >>>>>>>>>> know
> >>>>>>>>>>>>>> whether the method is properly implemented or not?
> >>>>>>>>>>>>>>> Does not implemented mean it always return a completed
> >>> future?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an
> >>> equivalent
> >>>>>> to
> >>>>>>>>>>>>>> `boolean hasNext()` which in case of “false” provides some
> >>> kind
> >>>>>> of a
> >>>>>>>>>>>>>> listener/callback that notifies about presence of next
> >>> element.
> >>>>>>>> There
> >>>>>>>>>>>> are
> >>>>>>>>>>>>>> some minor details, like `CompletableFuture<?>` has a
> >> minimal
> >>>> two
> >>>>>>>>>> state
> >>>>>>>>>>>>>> logic:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 1. Future is completed - we have more data
> >>>>>>>>>>>>>> 2. Future not yet completed - we don’t have data now, but we
> >>>>>>>> might/we
> >>>>>>>>>>>> will
> >>>>>>>>>>>>>> have in the future
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit
> >>> more
> >>>>>>>>>>>>>> complicated/dispersed and can lead/encourage `notify()`
> >> spam.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> b)
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method
> >>> like
> >>>>>>>>>>>> `getNext`
> >>>>>>>>>>>>>> the `getNext` would need return a
> >>>>>>>>>>>>>>> `ElementWithTimestamp` because some sources want to add
> >>>> timestamp
> >>>>>>>> to
> >>>>>>>>>>>>>> every element. IMO, this is not so memory friendly
> >>>>>>>>>>>>>>> so I prefer this design.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Guowei I don’t quite understand this. Could you elaborate
> >> why
> >>>>>>>> having a
> >>>>>>>>>>>>>> separate `advance()` help?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> c)
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regarding advance/poll/take. What’s the value of having two
> >>>>>> separate
> >>>>>>>>>>>>>> methods: poll and take? Which one of them should be called
> >> and
> >>>>>> which
> >>>>>>>>>>>>>> implemented? What’s the benefit of having those methods
> >>> compared
> >>>>>> to
> >>>>>>>>>>>> having
> >>>>>>>>>>>>>> a one single method `getNextElement()` (or `pollElement() or
> >>>>>>>> whatever
> >>>>>>>>>> we
> >>>>>>>>>>>>>> name it) with following contract:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> CompletableFuture<?> isBlocked();
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> /**
> >>>>>>>>>>>>>> Return next element - will be called only if `isBlocked()`
> >> is
> >>>>>>>>>> completed.
> >>>>>>>>>>>>>> Try to implement it in non blocking fashion, but if that’s
> >>>>>>>> impossible
> >>>>>>>>>> or
> >>>>>>>>>>>>>> you just don’t need the effort, you can block in this
> >> method.
> >>>>>>>>>>>>>> */
> >>>>>>>>>>>>>> T getNextElement();
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I mean, if the connector is implemented non-blockingly,
> >> Flink
> >>>>>> should
> >>>>>>>>>> use
> >>>>>>>>>>>>>> it that way. If it’s not, then `poll()` will `throw new
> >>>>>>>>>>>>>> NotImplementedException()`. Implementing both of them and
> >>>>>> providing
> >>>>>>>>>>>> both of
> >>>>>>>>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them
> >>>> into
> >>>>>> a
> >>>>>>>>>>>> single
> >>>>>>>>>>>>>> method call that should preferably (but not necessarily need
> >>> to)
> >>>>>> be
> >>>>>>>>>>>>>> non-blocking? It’s not like we are implementing general
> >>> purpose
> >>>>>>>>>> `Queue`,
> >>>>>>>>>>>>>> which users might want to call either of `poll` or `take`.
> >> We
> >>>>>> would
> >>>>>>>>>>>> always
> >>>>>>>>>>>>>> prefer to call `poll`, but if it’s blocking, then still we
> >>> have
> >>>> no
> >>>>>>>>>>>> choice,
> >>>>>>>>>>>>>> but to call it and block on it.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> d)
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
> >> source
> >>>> is
> >>>>>>>> very
> >>>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
> >>>> another
> >>>>>>>> way
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly if
> >>>> every
> >>>>>>>>>>>> advance
> >>>>>>>>>>>>>>> call return a Future.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I didn’t want to mention this, to not clog my initial
> >>> proposal,
> >>>>>> but
> >>>>>>>>>>>> there
> >>>>>>>>>>>>>> is a simple solution for the problem:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> public interface SplitReader {
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> (…)
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> CompletableFuture<?> NOT_BLOCKED =
> >>>>>>>>>>>>>> CompletableFuture.completedFuture(null);
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> /**
> >>>>>>>>>>>>>> * Returns a future that will be completed when the page
> >> source
> >>>>>>>>>>>> becomes
> >>>>>>>>>>>>>> * unblocked.  If the page source is not blocked, this method
> >>>>>>>> should
> >>>>>>>>>>>>>> return
> >>>>>>>>>>>>>> * {@code NOT_BLOCKED}.
> >>>>>>>>>>>>>> */
> >>>>>>>>>>>>>> default CompletableFuture<?> isBlocked()
> >>>>>>>>>>>>>> {
> >>>>>>>>>>>>>>   return NOT_BLOCKED;
> >>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> If we are blocked and we are waiting for the IO, then
> >>> creating a
> >>>>>> new
> >>>>>>>>>>>>>> Future is non-issue. Under full throttle/throughput and not
> >>>>>> blocked
> >>>>>>>>>>>> sources
> >>>>>>>>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve
> >>> the
> >>>>>>>>>>>> problem.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> One more remark, non-blocking sources might be a necessity
> >> in
> >>> a
> >>>>>>>> single
> >>>>>>>>>>>>>> threaded model without a checkpointing lock. (Currently when
> >>>>>> sources
> >>>>>>>>>> are
> >>>>>>>>>>>>>> blocked, they can release checkpointing lock and re-acquire
> >> it
> >>>>>> again
> >>>>>>>>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for
> >>>>>> checkpoints
> >>>>>>>> to
> >>>>>>>>>>>>>> happen when source is idling. In that case either `notify()`
> >>> or
> >>>> my
> >>>>>>>>>>>> proposed
> >>>>>>>>>>>>>> `isBlocked()` would allow to avoid busy-looping.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Piotrek
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com>
> >>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi Thomas,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The iterator-like API was also the first thing that came to
> >>> me.
> >>>>>> But
> >>>>>>>>>> it
> >>>>>>>>>>>>>>> seems a little confusing that hasNext() does not mean "the
> >>>> stream
> >>>>>>>> has
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>>> ended", but means "the next record is ready", which is
> >>>>>> repurposing
> >>>>>>>>>> the
> >>>>>>>>>>>>>> well
> >>>>>>>>>>>>>>> known meaning of hasNext(). If we follow the
> >> hasNext()/next()
> >>>>>>>>>> pattern,
> >>>>>>>>>>>> an
> >>>>>>>>>>>>>>> additional isNextReady() method to indicate whether the
> >> next
> >>>>>> record
> >>>>>>>>>> is
> >>>>>>>>>>>>>>> ready seems more intuitive to me.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Similarly, in poll()/take() pattern, another method of
> >>> isDone()
> >>>>>> is
> >>>>>>>>>>>> needed
> >>>>>>>>>>>>>>> to indicate whether the stream has ended or not.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
> >>>>>>>>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
> >>>>>>>>>>>> implementation.
> >>>>>>>>>>>>>>> When I am implementing a reader, I could have a couple of
> >>>>>> choices:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> - A thread-less reader that does not have any internal
> >>> thread.
> >>>>>>>>>>>>>>> - When poll() is called, the same calling thread will
> >>> perform a
> >>>>>>>> bunch
> >>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>> IO asynchronously.
> >>>>>>>>>>>>>>> - When take() is called, the same calling thread will
> >>> perform a
> >>>>>>>>>>>>>> bunch
> >>>>>>>>>>>>>>> of IO and wait until the record is ready.
> >>>>>>>>>>>>>>> - A reader with internal threads performing network IO and
> >>> put
> >>>>>>>>>> records
> >>>>>>>>>>>>>>> into a buffer.
> >>>>>>>>>>>>>>> - When poll() is called, the calling thread simply reads
> >> from
> >>>>>> the
> >>>>>>>>>>>>>>> buffer and return empty result immediately if there is no
> >>>>>> record.
> >>>>>>>>>>>>>>> - When take() is called, the calling thread reads from the
> >>>>>> buffer
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>> block waiting if the buffer is empty.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady()
> >>> API,
> >>>>>> it
> >>>>>>>> is
> >>>>>>>>>>>>>> less
> >>>>>>>>>>>>>>> intuitive for the reader developers to write the
> >> thread-less
> >>>>>>>> pattern.
> >>>>>>>>>>>>>>> Although technically speaking one can still do the
> >>> asynchronous
> >>>>>> IO
> >>>>>>>> to
> >>>>>>>>>>>>>>> prepare the record in isNextReady(). But it is inexplicit
> >> and
> >>>>>> seems
> >>>>>>>>>>>>>>> somewhat hacky.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <
> >> thw@apache.org>
> >>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Couple more points regarding discovery:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> The proposal mentions that discovery could be outside the
> >>>>>>>> execution
> >>>>>>>>>>>>>> graph.
> >>>>>>>>>>>>>>>> Today, discovered partitions/shards are checkpointed. I
> >>>> believe
> >>>>>>>> that
> >>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>> also need to be the case in the future, even when
> >> discovery
> >>>> and
> >>>>>>>>>>>> reading
> >>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>> split between different tasks.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> For cases such as resharding of a Kinesis stream, the
> >>>>>> relationship
> >>>>>>>>>>>>>> between
> >>>>>>>>>>>>>>>> splits needs to be considered. Splits cannot be randomly
> >>>>>>>> distributed
> >>>>>>>>>>>>>> over
> >>>>>>>>>>>>>>>> readers in certain situations. An example was mentioned
> >>> here:
> >>>>>>>>>>>>>>>>
> >>>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thomas
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <
> >> thw@apache.org
> >>>>
> >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks for getting the ball rolling on this!
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Can the number of splits decrease? Yes, splits can be
> >>> closed
> >>>>>> and
> >>>>>>>> go
> >>>>>>>>>>>>>> away.
> >>>>>>>>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing
> >>>> shards
> >>>>>>>>>> will
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>> closed and replaced with a new shard).
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive
> >>>> approach
> >>>>>>>>>> would
> >>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>> the thread-less IO model (pull based, non-blocking,
> >> caller
> >>>>>>>>>> retrieves
> >>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>> records when available). The current Kinesis API requires
> >>> the
> >>>>>> use
> >>>>>>>>>> of
> >>>>>>>>>>>>>>>>> threads. But that can be internal to the split reader and
> >>>> does
> >>>>>>>> not
> >>>>>>>>>>>> need
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> be a source API concern. In fact, that's what we are
> >>> working
> >>>> on
> >>>>>>>>>> right
> >>>>>>>>>>>>>> now
> >>>>>>>>>>>>>>>>> as improvement to the existing consumer: Each shard
> >>> consumer
> >>>>>>>> thread
> >>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>> push to a queue, the consumer main thread will poll the
> >>>>>> queue(s).
> >>>>>>>>>> It
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> The proposed SplitReader interface would fit the
> >>> thread-less
> >>>> IO
> >>>>>>>>>>>> model.
> >>>>>>>>>>>>>>>>> Similar to an iterator, we find out if there is a new
> >>> element
> >>>>>>>>>>>> (hasNext)
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> if so, move to it (next()). Separate calls deliver the
> >> meta
> >>>>>>>>>>>> information
> >>>>>>>>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer
> >> a
> >>>>>>>> timeout
> >>>>>>>>>>>>>>>> option,
> >>>>>>>>>>>>>>>>> so that the caller does not end up in a busy wait. On the
> >>>> other
> >>>>>>>>>>>> hand, a
> >>>>>>>>>>>>>>>>> caller processing multiple splits may want to cycle
> >> through
> >>>>>> fast,
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>> process elements of other splits as soon as they become
> >>>>>>>> available.
> >>>>>>>>>>>> The
> >>>>>>>>>>>>>>>> nice
> >>>>>>>>>>>>>>>>> thing is that this "split merge" logic can now live in
> >>> Flink
> >>>>>> and
> >>>>>>>> be
> >>>>>>>>>>>>>>>>> optimized and shared between different sources.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>> Thomas
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <
> >>>> guowei.mgw@gmail.com
> >>>>>>>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>>>> Thanks Aljoscha for this FLIP.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
> >>>> source
> >>>>>> is
> >>>>>>>>>>>> very
> >>>>>>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may
> >> be
> >>>>>>>> another
> >>>>>>>>>>>> way
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly
> >>> if
> >>>>>>>> every
> >>>>>>>>>>>>>>>> advance
> >>>>>>>>>>>>>>>>>> call return a Future.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> public interface Listener {
> >>>>>>>>>>>>>>>>>> public void notify();
> >>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> public interface SplitReader() {
> >>>>>>>>>>>>>>>>>> /**
> >>>>>>>>>>>>>>>>>> * When there is no element temporarily, this will return
> >>>>>>>> false.
> >>>>>>>>>>>>>>>>>> * When elements is available again splitReader can call
> >>>>>>>>>>>>>>>>>> listener.notify()
> >>>>>>>>>>>>>>>>>> * In addition the frame would check `advance`
> >>> periodically .
> >>>>>>>>>>>>>>>>>> * Of course advance can always return true and ignore
> >> the
> >>>>>>>>>>>>>> listener
> >>>>>>>>>>>>>>>>>> argument for simplicity.
> >>>>>>>>>>>>>>>>>> */
> >>>>>>>>>>>>>>>>>> public boolean advance(Listener listener);
> >>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create
> >> all
> >>>>>> Splits
> >>>>>>>>>> and
> >>>>>>>>>>>>>> how
> >>>>>>>>>>>>>>>>>> to create a SplitReader from a Split. But there is no
> >>>> strategy
> >>>>>>>> for
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> user
> >>>>>>>>>>>>>>>>>> to choose how to assign the splits to the tasks. I think
> >>> we
> >>>>>>>> could
> >>>>>>>>>>>> add
> >>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>> Enum to let user to choose.
> >>>>>>>>>>>>>>>>>> /**
> >>>>>>>>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
> >>>>>>>>>>>>>>>>>> Location,
> >>>>>>>>>>>>>>>>>> Workload,
> >>>>>>>>>>>>>>>>>> Random,
> >>>>>>>>>>>>>>>>>> Average
> >>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>> */
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one
> >> method
> >>>> like
> >>>>>>>>>>>>>> `getNext`
> >>>>>>>>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp`
> >>>>>> because
> >>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>> sources want to add timestamp to every element. IMO,
> >> this
> >>> is
> >>>>>> not
> >>>>>>>>>> so
> >>>>>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>> friendly so I prefer this design.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
> >>>>>>>> 下午6:08写道:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite
> >> a
> >>>> lot
> >>>>>> of
> >>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>> possible improvements. I have one proposal. Instead of
> >>>>>> having a
> >>>>>>>>>>>>>> method:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> boolean advance() throws IOException;
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I would replace it with
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> /*
> >>>>>>>>>>>>>>>>>>> * Return a future, which when completed means that
> >> source
> >>>> has
> >>>>>>>>>> more
> >>>>>>>>>>>>>>>> data
> >>>>>>>>>>>>>>>>>>> and getNext() will not block.
> >>>>>>>>>>>>>>>>>>> * If you wish to use benefits of non blocking
> >> connectors,
> >>>>>>>> please
> >>>>>>>>>>>>>>>>>>> implement this method appropriately.
> >>>>>>>>>>>>>>>>>>> */
> >>>>>>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
> >>>>>>>>>>>>>>>>>>>  return CompletableFuture.completedFuture(null);
> >>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Couple of arguments:
> >>>>>>>>>>>>>>>>>>> 1. I don’t understand the division of work between
> >>>>>> `advance()`
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> `getCurrent()`. What should be done in which,
> >> especially
> >>>> for
> >>>>>>>>>>>>>> connectors
> >>>>>>>>>>>>>>>>>>> that handle records in batches (like Kafka) and when
> >>> should
> >>>>>> you
> >>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>> `advance` and when `getCurrent()`.
> >>>>>>>>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will
> >>>> allow
> >>>>>>>> us
> >>>>>>>>>> in
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> future to have asynchronous/non blocking connectors and
> >>>> more
> >>>>>>>>>>>>>>>> efficiently
> >>>>>>>>>>>>>>>>>>> handle large number of blocked threads, without busy
> >>>> waiting.
> >>>>>>>>>> While
> >>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> same time it doesn’t add much complexity, since naive
> >>>>>> connector
> >>>>>>>>>>>>>>>>>>> implementations can be always blocking.
> >>>>>>>>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread
> >>> pool
> >>>>>> of
> >>>>>>>>>> task
> >>>>>>>>>>>>>>>>>>> executors, instead of one thread per task.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Piotrek
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
> >>>>>>>> aljoscha@apache.org
> >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi All,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> In order to finally get the ball rolling on the new
> >>> source
> >>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>>> that we have discussed for so long I finally created a
> >>>> FLIP:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
> >>>>>>>> work/discussion
> >>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>> adding per-partition watermark support to the Kinesis
> >>>> source
> >>>>>>>> and
> >>>>>>>>>>>>>>>> because
> >>>>>>>>>>>>>>>>>>> this would enable generic implementation of event-time
> >>>>>>>> alignment
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>>>> all
> >>>>>>>>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
> >>>>>>>> alignment
> >>>>>>>>>>>>>> part,
> >>>>>>>>>>>>>>>>>>> especially the part about information sharing between
> >>>>>>>> operations
> >>>>>>>>>>>> (I'm
> >>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>> calling it state sharing because state has a special
> >>>> meaning
> >>>>>> in
> >>>>>>>>>>>>>> Flink).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Please discuss away!
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Aljoscha
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>>
> >>>>
> >>>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@data-artisans.com>.
Hi again,

> However I don't like the thread mode which starts a thread for each split.
> Starting extra thread in operator is not an ideal way IMO. Especially
> thread count is decided by split count. So I was wondering if there is a
> more elegant way. Do we really want these threads in Flink core?

Biao you have raised an important issue. Indeed it seems like the current proposal is missing something. I would guess that we need a mechanism for adding new splits to an already existing SplitReader and some logic to determine whether current instance can accept more splits or not. For example

void SplitReader#addSplit(Split)
boolean SplitReader#doesWantMoreSplits()

Flink could randomly/round robin assign new splits to the SplitReaders that `doWantMoreSplits()`. Batch file readers might implement some custom logic in `doesWantMoreSplits()`, like one SplitReader can have at most N enqueued splits?

Also what about Kafka. Isn’t it the case that one KafkaConsumer can read from multiple splits? So Kafka’s SplitReader should always return true from `doesWantMoreSplits()`?

What do you think?

Re: Becket

I’m +1 for Sync and AsyncSplitReader.

Piotrek

> On 21 Nov 2018, at 14:49, Becket Qin <be...@gmail.com> wrote:
> 
> Hi Aljoscha,
> 
> Good point on the potential optimization in the source. One thing to
> clarify, by "adding a minimumTimestamp()/maximumTimestamp() method pair to
> the split interface", did you mean "split reader interface"? If so, what
> should the readers do if they do not have such additional information? I am
> wondering if it is possible to leave such optimization to the source
> internal implementation.
> 
> @all
> After reading all the feedback, Biao and I talked a little bit offline. We
> would like to share some new thoughts with you and see what do you think.
> 
> When looking at the Source API, we were trying to answer two questions.
> First of all, how would Flink use this API if someone else implemented it.
> Secondly, how would the connector contributors implement the interface? How
> difficult is the implementation.
> 
> KafkaConsumer is a typical example of a thread-less reader. The idea was to
> allow different threading model on top of it. It could be a global single
> thread handles record fetching and processing in an event loop pattern; it
> could also be one dedicated fetcher thread for each consumer and a separate
> thread pool for record processing. The API gives the freedom of picking up
> threading model to the users. To answer the first question, I would love to
> have such a source reader API so Flink can choose whatever threading model
> it wants. However, implementing such an interface could be pretty
> challenging and error prone.
> 
> On the other hand, having a source reader with a naive blocking socket is
> probably simple enough in most cases (actually sometimes this might even be
> the most efficient way). But it does not leave much option to Flink other
> than creating one thread per reader.
> 
> Given the above thoughts, it might be reasonable to separate the
> SplitReader API into two: SyncReader and AsyncReader. The sync reader just
> has a simple blocking takeNext() API. And the AsyncReader just has a
> pollNext(Callback) or Future<?> pollNext(). All the other methods are
> shared by both readers and could be put into a package private parent
> interface like BaseSplitReader.
> 
> Having these two readers allows both complicated and simple implementation,
> depending on the SplitReader writers. From Flink's perspective, it will
> choose a more efficient threading model if the SplitReader is an
> AsyncReader. Otherwise, it may have to use the one thread per reader model
> if the reader is a SyncReader. Users can also choose to implement both
> interface, in that case, it is up to Flink to choose which interface to use.
> 
> Admittedly, this solution does have one more interface, but still seems
> rewarding. Any thoughts?
> 
> Thanks,
> 
> Jiangjie (Becket) Qin
> 
> 
> On Sun, Nov 18, 2018 at 11:33 PM Biao Liu <mm...@gmail.com> wrote:
> 
>> Hi community,
>> 
>> Thank you guys for sharing ideas.
>> 
>> The thing I really concern is about the thread mode.
>> Actually in Alibaba, we have implemented our "split reader" based source
>> two years ago. That's based on "SourceFunction", it's just an extension not
>> a refactoring. It's almost same with the version Thomas and Jamie described
>> in Google Doc. It really helps in many scenarios.
>> 
>> However I don't like the thread mode which starts a thread for each split.
>> Starting extra thread in operator is not an ideal way IMO. Especially
>> thread count is decided by split count. So I was wondering if there is a
>> more elegant way. Do we really want these threads in Flink core?
>> 
>> I agree that blocking interface is more easy to implement. Could we at
>> least separate the split reader with source function into different
>> interfaces? Not all sources would like to read all splits concurrently. In
>> batch scenario, reading splits one by one is more general. And also not all
>> sources are partitioned, right?
>> I prefer there is a new source interface with "pull mode" only, no split.
>> There is a splittable source extended it. And there is one implementation
>> that starting threads for each split, reading all splits concurrently.
>> 
>> 
>> Thomas Weise <th...@apache.org> 于2018年11月18日周日 上午3:18写道:
>> 
>>> @Aljoscha to address your question first: In the case of the Kinesis
>>> consumer (with current Kinesis consumer API), there would also be N+1
>>> threads. I have implemented a prototype similar to what is shown in
>> Jamie's
>>> document, where the thread ownership is similar to what you have done for
>>> Kafka.
>>> 
>>> The equivalent of split reader manages its own thread and the "source
>> main
>>> thread" is responsible for emitting the data. The interface between the N
>>> reader threads and the 1 emitter is a blocking queue per consumer thread.
>>> The emitter can now control which queue to consume from based on the
>> event
>>> time progress.
>>> 
>>> This is akin to a "non-blocking" interface *between emitter and split
>>> reader*. Emitter uses poll to retrieve records from the N queues (which
>>> requires non-blocking interaction). The emitter is independent of the
>> split
>>> reader implementation, that part could live in Flink.
>>> 
>>> Regarding whether or not to assume that split readers always need a
>> thread
>>> and in addition that these reader threads should be managed by Flink: It
>>> depends on the API of respective external systems and I would not bake
>> that
>>> assumption into Flink. Some client libraries manage their own threads
>> (see
>>> push based API like JMS and as I understand it may also apply to the new
>>> fan-out Kinesis API:
>>> 
>>> 
>> https://docs.aws.amazon.com/streams/latest/dev/building-enhanced-consumers-api.html
>>> ).
>>> In such cases it would not make sense to layer another reader thread on
>>> top. It may instead be better if Flink provides to the split reader the
>>> queue/buffer to push records to.
>>> 
>>> The discussion so far has largely ignored the discovery aspect. There are
>>> some important considerations such as ordering dependency of splits and
>>> work rebalancing that may affect the split reader interface. Should we
>> fork
>>> this into a separate thread?
>>> 
>>> Thanks,
>>> Thomas
>>> 
>>> 
>>> On Fri, Nov 16, 2018 at 8:09 AM Piotr Nowojski <pi...@data-artisans.com>
>>> wrote:
>>> 
>>>> Hi Jamie,
>>>> 
>>>> As it was already covered with my discussion with Becket, there is an
>>> easy
>>>> way to provide blocking API on top of non-blocking API. And yes we both
>>>> agreed that blocking API is easier to implement by users.
>>>> 
>>>> I also do not agree with respect to usefulness of non blocking API.
>>>> Actually Kafka connector is the one that could be more efficient thanks
>>> to
>>>> the removal of the one layer of threading.
>>>> 
>>>> Piotrek
>>>> 
>>>>> On 16 Nov 2018, at 02:21, Jamie Grier <jg...@lyft.com.INVALID>
>> wrote:
>>>>> 
>>>>> Thanks Aljoscha for getting this effort going!
>>>>> 
>>>>> There's been plenty of discussion here already and I'll add my big +1
>>> to
>>>>> making this interface very simple to implement for a new
>>>>> Source/SplitReader.  Writing a new production quality connector for
>>> Flink
>>>>> is very difficult today and requires a lot of detailed knowledge
>> about
>>>>> Flink, event time progress, watermarking, idle shard detection, etc
>> and
>>>> it
>>>>> would be good to move almost all of this type of code into Flink
>> itself
>>>> and
>>>>> out of source implementations.  I also think this is totally doable
>> and
>>>> I'm
>>>>> really excited to see this happening.
>>>>> 
>>>>> I do have a couple of thoughts about the API and the implementation..
>>>>> 
>>>>> In a perfect world there would be a single thread per Flink source
>>>> sub-task
>>>>> and no additional threads for SplitReaders -- but this assumes a
>> world
>>>>> where you have true async IO APIs for the upstream systems (like
>> Kafka
>>>> and
>>>>> Kinesis, S3, HDFS, etc).  If that world did exist the single thread
>>> could
>>>>> just sit in an efficient select() call waiting for new data to arrive
>>> on
>>>>> any Split.  That'd be awesome..
>>>>> 
>>>>> But, that world doesn't exist and given that practical consideration
>> I
>>>>> would think the next best implementation is going to be, in practice,
>>>>> probably a thread per SplitReader that does nothing but call the
>> source
>>>> API
>>>>> and drop whatever it reads into a (blocking) queue -- as Aljoscha
>>>> mentioned
>>>>> (calling it N+1) and as we started to describe here:
>>>>> 
>>>> 
>>> 
>> https://docs.google.com/document/d/13HxfsucoN7aUls1FX202KFAVZ4IMLXEZWyRfZNd6WFM/edit#heading=h.lfi2u4fv9cxa
>>>>> 
>>>>> I guess my point is that I think we should strive to move as much of
>>>>> something like the diagram referenced in the above doc into Flink
>>> itself
>>>>> and out of sources and simplify the SplitReader API as much as
>> possible
>>>> as
>>>>> well.
>>>>> 
>>>>> With the above in mind and with regard to the discussion about
>>> blocking,
>>>>> etc..  I'm not sure I agree with some of the discussion so far with
>>>> regard
>>>>> to this API design.  The calls to the upstream systems
>> (kafka/kinesis)
>>>> are
>>>>> in fact going to be blocking calls.  So a simple API without the
>>>> constraint
>>>>> that the methods must be implemented in a non-blocking way seems
>> better
>>>> to
>>>>> me from the point of view of somebody writing a new source
>>>> implementation.
>>>>> My concern is that if you force the implementer of the SplitReader
>>>>> interface to do so in a non-blocking way you're just going to make it
>>>>> harder to write those implementations.  Those calls to read the next
>>> bit
>>>> of
>>>>> data are going to be blocking calls with most known important sources
>>> --
>>>> at
>>>>> least Kafka/Kinesis/HDFS -- so I think maybe we should just deal with
>>>> that
>>>>> head on and work around it a higher level so the SplitReader
>> interface
>>>>> stays super simple to implement.  This means we manage all the
>>> threading
>>>> in
>>>>> Flink core, the API stays pull-based, and the implementer is allowed
>> to
>>>>> simply block until they have data to return.
>>>>> 
>>>>> I maybe would change my mind about this if truly asynchronous APIs to
>>> the
>>>>> upstream source systems were likely to be available in the near
>> future
>>> or
>>>>> are now and I'm just ignorant of it.  But even then the supporting
>> code
>>>> in
>>>>> Flink to drive async and sync sources would be different and in fact
>>> they
>>>>> might just have different APIs altogether -- SplitReader vs
>>>>> AsyncSplitReader maybe.
>>>>> 
>>>>> In the end I think playing with the implementation, across more than
>>> one
>>>>> source, and moving as much common code into Flink itself will reveal
>>> the
>>>>> best API of course.
>>>>> 
>>>>> One other interesting note is that you need to preserve per-partition
>>>>> ordering so you have to take care with the implementation if it were
>> to
>>>> be
>>>>> based on a thread pool and futures so as not to reorder the reads.
>>>>> 
>>>>> Anyway, I'm thrilled to see this starting to move forward and I'd
>> very
>>>> much
>>>>> like to help with the implementation wherever I can.  We're doing a
>>>>> simplified internal version of some of this at Lyft for just Kinesis
>>>>> because we need a solution for event time alignment in the very short
>>>> term
>>>>> but we'd like to immediately start helping to do this properly in
>> Flink
>>>>> after that.  One of the end goals for us is event time alignment
>> across
>>>>> heterogeneous sources.  Another is making it possible for non-expert
>>>> users
>>>>> to have a high probability of being able to write their own, correct,
>>>>> connectors.
>>>>> 
>>>>> -Jamie
>>>>> 
>>>>> On Thu, Nov 15, 2018 at 3:43 AM Aljoscha Krettek <
>> aljoscha@apache.org>
>>>>> wrote:
>>>>> 
>>>>>> Hi,
>>>>>> 
>>>>>> I thought I had sent this mail a while ago but I must have forgotten
>>> to
>>>>>> send it.
>>>>>> 
>>>>>> There is another thing we should consider for splits: the range of
>>>>>> timestamps that it can contain. For example, the splits of a file
>>> source
>>>>>> would know what the minimum and maximum timestamp in the splits is,
>>>>>> roughly. For infinite splits, such as Kafka partitions, the minimum
>>>> would
>>>>>> be meaningful but the maximum would be +Inf. If the splits expose
>> the
>>>>>> interval of time that they contain the readers, or the component
>> that
>>>>>> manages the readers can make decisions about which splits to forward
>>> and
>>>>>> read first. And it can also influence the minimum watermark that a
>>>> reader
>>>>>> forwards: it should never emit a watermark if it knows there are
>>> splits
>>>> to
>>>>>> read that have a lower minimum timestamp. I think it should be as
>> easy
>>>> as
>>>>>> adding a minimumTimestamp()/maximumTimestamp() method pair to the
>>> split
>>>>>> interface.
>>>>>> 
>>>>>> Another thing we need to resolve is the actual reader interface. I
>> see
>>>>>> there has been some good discussion but I don't know if we have
>>>> consensus.
>>>>>> We should try and see how specific sources could be implemented with
>>> the
>>>>>> new interface. For example, for Kafka I think we need to have N+1
>>>> threads
>>>>>> per task (where N is the number of splits that a task is reading
>>> from).
>>>> On
>>>>>> thread is responsible for reading from the splits. And each split
>> has
>>>> its
>>>>>> own (internal) thread for reading from Kafka and putting messages in
>>> an
>>>>>> internal queue to pull from. This is similar to how the current
>> Kafka
>>>>>> source is implemented, which has a separate fetcher thread. The
>> reason
>>>> for
>>>>>> this split is that we always need to try reading from Kafka to keep
>>> the
>>>>>> throughput up. In the current implementation the internal queue (or
>>>>>> handover) limits the read rate of the reader threads.
>>>>>> 
>>>>>> @Thomas, what do you think this would look like for Kinesis?
>>>>>> 
>>>>>> Best,
>>>>>> Aljoscha
>>>>>> 
>>>>>>> On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com>
>> wrote:
>>>>>>> 
>>>>>>> Hi Piotrek,
>>>>>>> 
>>>>>>> Thanks a lot for the detailed reply. All makes sense to me.
>>>>>>> 
>>>>>>> WRT the confusion between advance() / getCurrent(), do you think it
>>>> would
>>>>>>> help if we combine them and have something like:
>>>>>>> 
>>>>>>> CompletableFuture<T> getNext();
>>>>>>> long getWatermark();
>>>>>>> long getCurrentTimestamp();
>>>>>>> 
>>>>>>> Cheers,
>>>>>>> 
>>>>>>> Jiangjie (Becket) Qin
>>>>>>> 
>>>>>>> On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <
>>>> piotr@data-artisans.com>
>>>>>>> wrote:
>>>>>>> 
>>>>>>>> Hi,
>>>>>>>> 
>>>>>>>> Thanks again for the detailed answer :) Sorry for responding with
>> a
>>>>>> delay.
>>>>>>>> 
>>>>>>>>> Completely agree that in pattern 2, having a callback is
>> necessary
>>>> for
>>>>>>>> that
>>>>>>>>> single thread outside of the connectors. And the connectors MUST
>>> have
>>>>>>>>> internal threads.
>>>>>>>> 
>>>>>>>> Yes, this thread will have to exists somewhere. In pattern 2 it
>>> exists
>>>>>> in
>>>>>>>> the connector (at least from the perspective of the Flink
>> execution
>>>>>>>> engine). In pattern 1 it exists inside the Flink execution engine.
>>>> With
>>>>>>>> completely blocking connectors, like simple reading from files,
>> both
>>>> of
>>>>>>>> those approaches are basically the same. The difference is when
>> user
>>>>>>>> implementing Flink source is already working with a non blocking
>>> code
>>>>>> with
>>>>>>>> some internal threads. In this case, pattern 1 would result in
>>> "double
>>>>>>>> thread wrapping”, while pattern 2 would allow to skip one layer of
>>>>>>>> indirection.
>>>>>>>> 
>>>>>>>>> If we go that way, we should have something like "void
>>>>>>>>> poll(Callback) / void advance(callback)". I am curious how would
>>>>>>>>> CompletableFuture work here, though. If 10 readers returns 10
>>>>>> completable
>>>>>>>>> futures, will there be 10 additional threads (so 20 threads in
>>> total)
>>>>>>>>> blocking waiting on them? Or will there be a single thread busy
>>> loop
>>>>>>>>> checking around?
>>>>>>>> 
>>>>>>>> To be honest, I haven’t thought this completely through and I
>>> haven’t
>>>>>>>> tested/POC’ed it. Having said that, I can think of at least couple
>>> of
>>>>>>>> solutions. First is something like this:
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>> 
>>> 
>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>>>>>>>> <
>>>>>>>> 
>>>>>> 
>>>> 
>>> 
>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> Line:
>>>>>>>> 
>>>>>>>>                              `blocked = split.process();`
>>>>>>>> 
>>>>>>>> Is where the execution goes into to the task/sources. This is
>> where
>>>> the
>>>>>>>> returned future is handled:
>>>>>>>> 
>>>>>>>>                              blocked.addListener(() -> {
>>>>>>>>                                  blockedSplits.remove(split);
>>>>>>>>                                  // reset the level priority to
>>>>>> prevent
>>>>>>>> previously-blocked splits from starving existing splits
>>>>>>>>                                  split.resetLevelPriority();
>>>>>>>>                                  waitingSplits.offer(split);
>>>>>>>>                              }, executor);
>>>>>>>> 
>>>>>>>> Fundamentally callbacks and Futures are more or less
>> interchangeable
>>>> You
>>>>>>>> can always wrap one into another (creating a callback that
>>> completes a
>>>>>>>> future and attach a callback once future completes). In this case
>>> the
>>>>>>>> difference for me is mostly:
>>>>>>>> - api with passing callback allows the callback to be fired
>> multiple
>>>>>> times
>>>>>>>> and to fire it even if the connector is not blocked. This is what
>> I
>>>>>> meant
>>>>>>>> by saying that api `CompletableFuture<?> isBlocked()` is a bit
>>>> simpler.
>>>>>>>> Connector can only return either “I’m not blocked” or “I’m blocked
>>>> and I
>>>>>>>> will tell you only once when I’m not blocked anymore”.
>>>>>>>> 
>>>>>>>> But this is not the most important thing for me here. For me
>>> important
>>>>>>>> thing is to try our best to make Flink task’s control and
>> execution
>>>>>> single
>>>>>>>> threaded. For that both callback and future APIs should work the
>>> same.
>>>>>>>> 
>>>>>>>>> WRT pattern 1, a single blocking take() API should just work. The
>>>> good
>>>>>>>>> thing is that a blocking read API is usually simpler to
>> implement.
>>>>>>>> 
>>>>>>>> Yes, they are easier to implement (especially if you are not the
>> one
>>>>>> that
>>>>>>>> have to deal with the additional threading required around them ;)
>>> ).
>>>>>> But
>>>>>>>> to answer this issue, if we choose pattern 2, we can always
>> provide
>>> a
>>>>>>>> proxy/wrapper that would using the internal thread implement the
>>>>>>>> non-blocking API while exposing blocking API to the user. It would
>>>>>>>> implement pattern 2 for the user exposing to him pattern 1. In
>> other
>>>>>> words
>>>>>>>> implementing pattern 1 in pattern 2 paradigm, while making it
>>> possible
>>>>>> to
>>>>>>>> implement pure pattern 2 connectors.
>>>>>>>> 
>>>>>>>>> BTW, one thing I am also trying to avoid is pushing users to
>>> perform
>>>> IO
>>>>>>>> in
>>>>>>>>> a method like "isBlocked()". If the method is expected to fetch
>>>> records
>>>>>>>>> (even if not returning them), naming it something more explicit
>>> would
>>>>>>>> help
>>>>>>>>> avoid confusion.
>>>>>>>> 
>>>>>>>> If we choose so, we could rework it into something like:
>>>>>>>> 
>>>>>>>> CompletableFuture<?> advance()
>>>>>>>> T getCurrent();
>>>>>>>> Watermark getCurrentWatermark()
>>>>>>>> 
>>>>>>>> But as I wrote before, this is more confusing to me for the exact
>>>>>> reasons
>>>>>>>> you mentioned :) I would be confused what should be done in
>>>> `adanvce()`
>>>>>> and
>>>>>>>> what in `getCurrent()`. However, again this naming issue is not
>> that
>>>>>>>> important to me and probably is matter of taste/personal
>>> preferences.
>>>>>>>> 
>>>>>>>> Piotrek
>>>>>>>> 
>>>>>>>>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com>
>> wrote:
>>>>>>>>> 
>>>>>>>>> Hi Piotrek,
>>>>>>>>> 
>>>>>>>>> Thanks for the explanation. We are probably talking about the
>> same
>>>>>> thing
>>>>>>>>> but in different ways. To clarify a little bit, I think there are
>>> two
>>>>>>>>> patterns to read from a connector.
>>>>>>>>> 
>>>>>>>>> Pattern 1: Thread-less connector with a blocking read API.
>> Outside
>>> of
>>>>>> the
>>>>>>>>> connector, there is one IO thread per reader, doing blocking
>> read.
>>> An
>>>>>>>>> additional thread will interact with all the IO threads.
>>>>>>>>> Pattern 2: Connector with internal thread(s) and non-blocking
>> API.
>>>>>>>> Outside
>>>>>>>>> of the connector, there is one thread for ALL readers, doing IO
>>>> relying
>>>>>>>> on
>>>>>>>>> notification callbacks in the reader.
>>>>>>>>> 
>>>>>>>>> In both patterns, there must be at least one thread per
>> connector,
>>>>>> either
>>>>>>>>> inside (created by connector writers) or outside (created by
>> Flink)
>>>> of
>>>>>>>> the
>>>>>>>>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total,
>>> to
>>>>>> make
>>>>>>>>> sure that 1 thread is fully non-blocking.
>>>>>>>>> 
>>>>>>>>>> Btw, I don’t know if you understand my point. Having only
>> `poll()`
>>>> and
>>>>>>>>> `take()` is not enough for single threaded task. If our source
>>>>>> interface
>>>>>>>>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
>>>>>>>>> isBlocked(),`, there is no way to implement single threaded task
>>> that
>>>>>>>> both
>>>>>>>>> reads the data from the source connector and can also react to
>>> system
>>>>>>>>> events. Ok, non >blocking `poll()` would allow that, but with
>> busy
>>>>>>>> looping.
>>>>>>>>> 
>>>>>>>>> Completely agree that in pattern 2, having a callback is
>> necessary
>>>> for
>>>>>>>> that
>>>>>>>>> single thread outside of the connectors. And the connectors MUST
>>> have
>>>>>>>>> internal threads. If we go that way, we should have something
>> like
>>>>>> "void
>>>>>>>>> poll(Callback) / void advance(callback)". I am curious how would
>>>>>>>>> CompletableFuture work here, though. If 10 readers returns 10
>>>>>> completable
>>>>>>>>> futures, will there be 10 additional threads (so 20 threads in
>>> total)
>>>>>>>>> blocking waiting on them? Or will there be a single thread busy
>>> loop
>>>>>>>>> checking around?
>>>>>>>>> 
>>>>>>>>> WRT pattern 1, a single blocking take() API should just work. The
>>>> good
>>>>>>>>> thing is that a blocking read API is usually simpler to
>> implement.
>>> An
>>>>>>>>> additional non-blocking "T poll()" method here is indeed optional
>>> and
>>>>>>>> could
>>>>>>>>> be used in cases like Flink does not want the thread to block
>>>> forever.
>>>>>>>> They
>>>>>>>>> can also be combined to have a "T poll(Timeout)", which is
>> exactly
>>>> what
>>>>>>>>> KafkaConsumer did.
>>>>>>>>> 
>>>>>>>>> It sounds that you are proposing pattern 2 with something similar
>>> to
>>>>>> NIO2
>>>>>>>>> AsynchronousByteChannel[1]. That API would work, except that the
>>>>>>>> signature
>>>>>>>>> returning future seems not necessary. If that is the case, a
>> minor
>>>>>> change
>>>>>>>>> on the current FLIP proposal to have "void advance(callback)"
>>> should
>>>>>>>> work.
>>>>>>>>> And this means the connectors MUST have their internal threads.
>>>>>>>>> 
>>>>>>>>> BTW, one thing I am also trying to avoid is pushing users to
>>> perform
>>>> IO
>>>>>>>> in
>>>>>>>>> a method like "isBlocked()". If the method is expected to fetch
>>>> records
>>>>>>>>> (even if not returning them), naming it something more explicit
>>> would
>>>>>>>> help
>>>>>>>>> avoid confusion.
>>>>>>>>> 
>>>>>>>>> Thanks,
>>>>>>>>> 
>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>> 
>>>>>>>>> [1]
>>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>> 
>>> 
>> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
>>>>>>>>> 
>>>>>>>>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <
>>>>>> piotr@data-artisans.com>
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Hi
>>>>>>>>>> 
>>>>>>>>>> Good point with select/epoll, however I do not see how they
>>> couldn’t
>>>>>> be
>>>>>>>>>> with Flink if we would like single task in Flink to be
>>>> single-threaded
>>>>>>>> (and
>>>>>>>>>> I believe we should pursue this goal). If your connector blocks
>> on
>>>>>>>>>> `select`, then it can not process/handle control messages from
>>>> Flink,
>>>>>>>> like
>>>>>>>>>> checkpoints, releasing resources and potentially output flushes.
>>>> This
>>>>>>>> would
>>>>>>>>>> require tight integration between connector and Flink’s main
>> event
>>>>>>>>>> loop/selects/etc.
>>>>>>>>>> 
>>>>>>>>>> Looking at it from other perspective. Let’s assume that we have
>> a
>>>>>>>>>> connector implemented on top of `select`/`epoll`. In order to
>>>>>> integrate
>>>>>>>> it
>>>>>>>>>> with Flink’s checkpointing/flushes/resource releasing it will
>> have
>>>> to
>>>>>> be
>>>>>>>>>> executed in separate thread one way or another. At least if our
>>> API
>>>>>> will
>>>>>>>>>> enforce/encourage non blocking implementations with some kind of
>>>>>>>>>> notifications (`isBlocked()` or `notify()` callback), some
>>>> connectors
>>>>>>>> might
>>>>>>>>>> skip one layer of wapping threads.
>>>>>>>>>> 
>>>>>>>>>> Btw, I don’t know if you understand my point. Having only
>> `poll()`
>>>> and
>>>>>>>>>> `take()` is not enough for single threaded task. If our source
>>>>>> interface
>>>>>>>>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
>>>>>>>>>> isBlocked(),`, there is no way to implement single threaded task
>>>> that
>>>>>>>> both
>>>>>>>>>> reads the data from the source connector and can also react to
>>>> system
>>>>>>>>>> events. Ok, non blocking `poll()` would allow that, but with
>> busy
>>>>>>>> looping.
>>>>>>>>>> 
>>>>>>>>>> Piotrek
>>>>>>>>>> 
>>>>>>>>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com>
>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>> Hi Piotrek,
>>>>>>>>>>> 
>>>>>>>>>>>> But I don’t see a reason why we should expose both blocking
>>>> `take()`
>>>>>>>> and
>>>>>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone
>> (Flink
>>>>>>>> engine
>>>>>>>>>> or
>>>>>>>>>>> connector) would have to do the same busy
>>>>>>>>>>>> looping anyway and I think it would be better to have a
>> simpler
>>>>>>>>>> connector
>>>>>>>>>>> API (that would solve our problems) and force connectors to
>>> comply
>>>>>> one
>>>>>>>>>> way
>>>>>>>>>>> or another.
>>>>>>>>>>> 
>>>>>>>>>>> If we let the block happen inside the connector, the blocking
>>> does
>>>>>> not
>>>>>>>>>> have
>>>>>>>>>>> to be a busy loop. For example, to do the block waiting
>>>> efficiently,
>>>>>>>> the
>>>>>>>>>>> connector can use java NIO selector().select which relies on OS
>>>>>> syscall
>>>>>>>>>>> like epoll[1] instead of busy looping. But if Flink engine
>> blocks
>>>>>>>> outside
>>>>>>>>>>> the connector, it pretty much has to do the busy loop. So if
>>> there
>>>> is
>>>>>>>>>> only
>>>>>>>>>>> one API to get the element, a blocking getNextElement() makes
>>> more
>>>>>>>> sense.
>>>>>>>>>>> In any case, we should avoid ambiguity. It has to be crystal
>>> clear
>>>>>>>> about
>>>>>>>>>>> whether a method is expected to be blocking or non-blocking.
>>>>>> Otherwise
>>>>>>>> it
>>>>>>>>>>> would be very difficult for Flink engine to do the right thing
>>> with
>>>>>> the
>>>>>>>>>>> connectors. At the first glance at getCurrent(), the expected
>>>>>> behavior
>>>>>>>> is
>>>>>>>>>>> not quite clear.
>>>>>>>>>>> 
>>>>>>>>>>> That said, I do agree that functionality wise, poll() and
>> take()
>>>> kind
>>>>>>>> of
>>>>>>>>>>> overlap. But they are actually not quite different from
>>>>>>>>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the
>> only
>>>>>>>>>>> difference is that poll() also returns the next record if it is
>>>>>>>>>> available.
>>>>>>>>>>> But I agree that the isBlocked() + getNextElement() is more
>>>> flexible
>>>>>> as
>>>>>>>>>>> users can just check the record availability, but not fetch the
>>>> next
>>>>>>>>>>> element.
>>>>>>>>>>> 
>>>>>>>>>>>> In case of thread-less readers with only non-blocking
>>>> `queue.poll()`
>>>>>>>> (is
>>>>>>>>>>> that really a thing? I can not think about a real
>> implementation
>>>> that
>>>>>>>>>>> enforces such constraints)
>>>>>>>>>>> Right, it is pretty much a syntax sugar to allow user combine
>> the
>>>>>>>>>>> check-and-take into one method. It could be achieved with
>>>>>> isBlocked() +
>>>>>>>>>>> getNextElement().
>>>>>>>>>>> 
>>>>>>>>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
>>>>>>>>>>> 
>>>>>>>>>>> Thanks,
>>>>>>>>>>> 
>>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>> 
>>>>>>>>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
>>>>>>>> piotr@data-artisans.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Hi Becket,
>>>>>>>>>>>> 
>>>>>>>>>>>> With my proposal, both of your examples would have to be
>> solved
>>> by
>>>>>> the
>>>>>>>>>>>> connector and solution to both problems would be the same:
>>>>>>>>>>>> 
>>>>>>>>>>>> Pretend that connector is never blocked (`isBlocked() { return
>>>>>>>>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking
>>>>>> fashion
>>>>>>>>>> (or
>>>>>>>>>>>> semi blocking with return of control from time to time to
>> allow
>>>> for
>>>>>>>>>>>> checkpointing, network flushing and other resource management
>>>> things
>>>>>>>> to
>>>>>>>>>>>> happen in the same main thread). In other words, exactly how
>> you
>>>>>> would
>>>>>>>>>>>> implement `take()` method or how the same source connector
>> would
>>>> be
>>>>>>>>>>>> implemented NOW with current source interface. The difference
>>> with
>>>>>>>>>> current
>>>>>>>>>>>> interface would be only that main loop would be outside of the
>>>>>>>>>> connector,
>>>>>>>>>>>> and instead of periodically releasing checkpointing lock,
>>>>>> periodically
>>>>>>>>>>>> `return null;` or `return Optional.empty();` from
>>>>>> `getNextElement()`.
>>>>>>>>>>>> 
>>>>>>>>>>>> In case of thread-less readers with only non-blocking
>>>> `queue.poll()`
>>>>>>>> (is
>>>>>>>>>>>> that really a thing? I can not think about a real
>> implementation
>>>>>> that
>>>>>>>>>>>> enforces such constraints), we could provide a wrapper that
>>> hides
>>>>>> the
>>>>>>>>>> busy
>>>>>>>>>>>> looping. The same applies how to solve forever blocking
>> readers
>>> -
>>>> we
>>>>>>>>>> could
>>>>>>>>>>>> provider another wrapper running the connector in separate
>>> thread.
>>>>>>>>>>>> 
>>>>>>>>>>>> But I don’t see a reason why we should expose both blocking
>>>> `take()`
>>>>>>>> and
>>>>>>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone
>>> (Flink
>>>>>>>>>> engine or
>>>>>>>>>>>> connector) would have to do the same busy looping anyway and I
>>>> think
>>>>>>>> it
>>>>>>>>>>>> would be better to have a simpler connector API (that would
>>> solve
>>>>>> our
>>>>>>>>>>>> problems) and force connectors to comply one way or another.
>>>>>>>>>>>> 
>>>>>>>>>>>> Piotrek
>>>>>>>>>>>> 
>>>>>>>>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com>
>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Hi Piotr,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I might have misunderstood you proposal. But let me try to
>>>> explain
>>>>>> my
>>>>>>>>>>>>> concern. I am thinking about the following case:
>>>>>>>>>>>>> 1. a reader has the following two interfaces,
>>>>>>>>>>>>> boolean isBlocked()
>>>>>>>>>>>>> T getNextElement()
>>>>>>>>>>>>> 2. the implementation of getNextElement() is non-blocking.
>>>>>>>>>>>>> 3. The reader is thread-less, i.e. it does not have any
>>> internal
>>>>>>>>>> thread.
>>>>>>>>>>>>> For example, it might just delegate the getNextElement() to a
>>>>>>>>>>>> queue.poll(),
>>>>>>>>>>>>> and isBlocked() is just queue.isEmpty().
>>>>>>>>>>>>> 
>>>>>>>>>>>>> How can Flink efficiently implement a blocking reading
>> behavior
>>>>>> with
>>>>>>>>>> this
>>>>>>>>>>>>> reader? Either a tight loop or a backoff interval is needed.
>>>>>> Neither
>>>>>>>> of
>>>>>>>>>>>>> them is ideal.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Now let's say in the reader mentioned above implements a
>>> blocking
>>>>>>>>>>>>> getNextElement() method. Because there is no internal thread
>> in
>>>> the
>>>>>>>>>>>> reader,
>>>>>>>>>>>>> after isBlocked() returns false. Flink will still have to
>> loop
>>> on
>>>>>>>>>>>>> isBlocked() to check whether the next record is available. If
>>> the
>>>>>>>> next
>>>>>>>>>>>>> record reaches after 10 min, it is a tight loop for 10 min.
>> You
>>>>>> have
>>>>>>>>>>>>> probably noticed that in this case, even isBlocked() returns
>> a
>>>>>>>> future,
>>>>>>>>>>>> that
>>>>>>>>>>>>> future() will not be completed if Flink does not call some
>>> method
>>>>>>>> from
>>>>>>>>>>>> the
>>>>>>>>>>>>> reader, because the reader has no internal thread to complete
>>>> that
>>>>>>>>>> future
>>>>>>>>>>>>> by itself.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Due to the above reasons, a blocking take() API would allow
>>> Flink
>>>>>> to
>>>>>>>>>> have
>>>>>>>>>>>>> an efficient way to read from a reader. There are many ways
>> to
>>>> wake
>>>>>>>> up
>>>>>>>>>>>> the
>>>>>>>>>>>>> blocking thread when checkpointing is needed depending on the
>>>>>>>>>>>>> implementation. But I think the poll()/take() API would also
>>> work
>>>>>> in
>>>>>>>>>> that
>>>>>>>>>>>>> case.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
>>>>>>>> piotr@data-artisans.com
>>>>>>>>>>> 
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> a)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> BTW, regarding the isBlock() method, I have a few more
>>>> questions.
>>>>>>>> 21,
>>>>>>>>>>>> Is
>>>>>>>>>>>>>> a method isReady() with boolean as a return value
>>>>>>>>>>>>>>> equivalent? Personally I found it is a little bit confusing
>>> in
>>>>>> what
>>>>>>>>>> is
>>>>>>>>>>>>>> supposed to be returned when the future is completed. 22. if
>>>>>>>>>>>>>>> the implementation of isBlocked() is optional, how do the
>>>> callers
>>>>>>>>>> know
>>>>>>>>>>>>>> whether the method is properly implemented or not?
>>>>>>>>>>>>>>> Does not implemented mean it always return a completed
>>> future?
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an
>>> equivalent
>>>>>> to
>>>>>>>>>>>>>> `boolean hasNext()` which in case of “false” provides some
>>> kind
>>>>>> of a
>>>>>>>>>>>>>> listener/callback that notifies about presence of next
>>> element.
>>>>>>>> There
>>>>>>>>>>>> are
>>>>>>>>>>>>>> some minor details, like `CompletableFuture<?>` has a
>> minimal
>>>> two
>>>>>>>>>> state
>>>>>>>>>>>>>> logic:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 1. Future is completed - we have more data
>>>>>>>>>>>>>> 2. Future not yet completed - we don’t have data now, but we
>>>>>>>> might/we
>>>>>>>>>>>> will
>>>>>>>>>>>>>> have in the future
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit
>>> more
>>>>>>>>>>>>>> complicated/dispersed and can lead/encourage `notify()`
>> spam.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> b)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method
>>> like
>>>>>>>>>>>> `getNext`
>>>>>>>>>>>>>> the `getNext` would need return a
>>>>>>>>>>>>>>> `ElementWithTimestamp` because some sources want to add
>>>> timestamp
>>>>>>>> to
>>>>>>>>>>>>>> every element. IMO, this is not so memory friendly
>>>>>>>>>>>>>>> so I prefer this design.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Guowei I don’t quite understand this. Could you elaborate
>> why
>>>>>>>> having a
>>>>>>>>>>>>>> separate `advance()` help?
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> c)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Regarding advance/poll/take. What’s the value of having two
>>>>>> separate
>>>>>>>>>>>>>> methods: poll and take? Which one of them should be called
>> and
>>>>>> which
>>>>>>>>>>>>>> implemented? What’s the benefit of having those methods
>>> compared
>>>>>> to
>>>>>>>>>>>> having
>>>>>>>>>>>>>> a one single method `getNextElement()` (or `pollElement() or
>>>>>>>> whatever
>>>>>>>>>> we
>>>>>>>>>>>>>> name it) with following contract:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> CompletableFuture<?> isBlocked();
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>> Return next element - will be called only if `isBlocked()`
>> is
>>>>>>>>>> completed.
>>>>>>>>>>>>>> Try to implement it in non blocking fashion, but if that’s
>>>>>>>> impossible
>>>>>>>>>> or
>>>>>>>>>>>>>> you just don’t need the effort, you can block in this
>> method.
>>>>>>>>>>>>>> */
>>>>>>>>>>>>>> T getNextElement();
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> I mean, if the connector is implemented non-blockingly,
>> Flink
>>>>>> should
>>>>>>>>>> use
>>>>>>>>>>>>>> it that way. If it’s not, then `poll()` will `throw new
>>>>>>>>>>>>>> NotImplementedException()`. Implementing both of them and
>>>>>> providing
>>>>>>>>>>>> both of
>>>>>>>>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them
>>>> into
>>>>>> a
>>>>>>>>>>>> single
>>>>>>>>>>>>>> method call that should preferably (but not necessarily need
>>> to)
>>>>>> be
>>>>>>>>>>>>>> non-blocking? It’s not like we are implementing general
>>> purpose
>>>>>>>>>> `Queue`,
>>>>>>>>>>>>>> which users might want to call either of `poll` or `take`.
>> We
>>>>>> would
>>>>>>>>>>>> always
>>>>>>>>>>>>>> prefer to call `poll`, but if it’s blocking, then still we
>>> have
>>>> no
>>>>>>>>>>>> choice,
>>>>>>>>>>>>>> but to call it and block on it.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> d)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
>> source
>>>> is
>>>>>>>> very
>>>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
>>>> another
>>>>>>>> way
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly if
>>>> every
>>>>>>>>>>>> advance
>>>>>>>>>>>>>>> call return a Future.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> I didn’t want to mention this, to not clog my initial
>>> proposal,
>>>>>> but
>>>>>>>>>>>> there
>>>>>>>>>>>>>> is a simple solution for the problem:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> public interface SplitReader {
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> (…)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> CompletableFuture<?> NOT_BLOCKED =
>>>>>>>>>>>>>> CompletableFuture.completedFuture(null);
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>> * Returns a future that will be completed when the page
>> source
>>>>>>>>>>>> becomes
>>>>>>>>>>>>>> * unblocked.  If the page source is not blocked, this method
>>>>>>>> should
>>>>>>>>>>>>>> return
>>>>>>>>>>>>>> * {@code NOT_BLOCKED}.
>>>>>>>>>>>>>> */
>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked()
>>>>>>>>>>>>>> {
>>>>>>>>>>>>>>   return NOT_BLOCKED;
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> If we are blocked and we are waiting for the IO, then
>>> creating a
>>>>>> new
>>>>>>>>>>>>>> Future is non-issue. Under full throttle/throughput and not
>>>>>> blocked
>>>>>>>>>>>> sources
>>>>>>>>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve
>>> the
>>>>>>>>>>>> problem.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> One more remark, non-blocking sources might be a necessity
>> in
>>> a
>>>>>>>> single
>>>>>>>>>>>>>> threaded model without a checkpointing lock. (Currently when
>>>>>> sources
>>>>>>>>>> are
>>>>>>>>>>>>>> blocked, they can release checkpointing lock and re-acquire
>> it
>>>>>> again
>>>>>>>>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for
>>>>>> checkpoints
>>>>>>>> to
>>>>>>>>>>>>>> happen when source is idling. In that case either `notify()`
>>> or
>>>> my
>>>>>>>>>>>> proposed
>>>>>>>>>>>>>> `isBlocked()` would allow to avoid busy-looping.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Piotrek
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com>
>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Hi Thomas,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> The iterator-like API was also the first thing that came to
>>> me.
>>>>>> But
>>>>>>>>>> it
>>>>>>>>>>>>>>> seems a little confusing that hasNext() does not mean "the
>>>> stream
>>>>>>>> has
>>>>>>>>>>>> not
>>>>>>>>>>>>>>> ended", but means "the next record is ready", which is
>>>>>> repurposing
>>>>>>>>>> the
>>>>>>>>>>>>>> well
>>>>>>>>>>>>>>> known meaning of hasNext(). If we follow the
>> hasNext()/next()
>>>>>>>>>> pattern,
>>>>>>>>>>>> an
>>>>>>>>>>>>>>> additional isNextReady() method to indicate whether the
>> next
>>>>>> record
>>>>>>>>>> is
>>>>>>>>>>>>>>> ready seems more intuitive to me.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Similarly, in poll()/take() pattern, another method of
>>> isDone()
>>>>>> is
>>>>>>>>>>>> needed
>>>>>>>>>>>>>>> to indicate whether the stream has ended or not.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
>>>>>>>>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>> When I am implementing a reader, I could have a couple of
>>>>>> choices:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> - A thread-less reader that does not have any internal
>>> thread.
>>>>>>>>>>>>>>> - When poll() is called, the same calling thread will
>>> perform a
>>>>>>>> bunch
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>> IO asynchronously.
>>>>>>>>>>>>>>> - When take() is called, the same calling thread will
>>> perform a
>>>>>>>>>>>>>> bunch
>>>>>>>>>>>>>>> of IO and wait until the record is ready.
>>>>>>>>>>>>>>> - A reader with internal threads performing network IO and
>>> put
>>>>>>>>>> records
>>>>>>>>>>>>>>> into a buffer.
>>>>>>>>>>>>>>> - When poll() is called, the calling thread simply reads
>> from
>>>>>> the
>>>>>>>>>>>>>>> buffer and return empty result immediately if there is no
>>>>>> record.
>>>>>>>>>>>>>>> - When take() is called, the calling thread reads from the
>>>>>> buffer
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> block waiting if the buffer is empty.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady()
>>> API,
>>>>>> it
>>>>>>>> is
>>>>>>>>>>>>>> less
>>>>>>>>>>>>>>> intuitive for the reader developers to write the
>> thread-less
>>>>>>>> pattern.
>>>>>>>>>>>>>>> Although technically speaking one can still do the
>>> asynchronous
>>>>>> IO
>>>>>>>> to
>>>>>>>>>>>>>>> prepare the record in isNextReady(). But it is inexplicit
>> and
>>>>>> seems
>>>>>>>>>>>>>>> somewhat hacky.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <
>> thw@apache.org>
>>>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Couple more points regarding discovery:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> The proposal mentions that discovery could be outside the
>>>>>>>> execution
>>>>>>>>>>>>>> graph.
>>>>>>>>>>>>>>>> Today, discovered partitions/shards are checkpointed. I
>>>> believe
>>>>>>>> that
>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>> also need to be the case in the future, even when
>> discovery
>>>> and
>>>>>>>>>>>> reading
>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>> split between different tasks.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> For cases such as resharding of a Kinesis stream, the
>>>>>> relationship
>>>>>>>>>>>>>> between
>>>>>>>>>>>>>>>> splits needs to be considered. Splits cannot be randomly
>>>>>>>> distributed
>>>>>>>>>>>>>> over
>>>>>>>>>>>>>>>> readers in certain situations. An example was mentioned
>>> here:
>>>>>>>>>>>>>>>> 
>>>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Thomas
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <
>> thw@apache.org
>>>> 
>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Thanks for getting the ball rolling on this!
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Can the number of splits decrease? Yes, splits can be
>>> closed
>>>>>> and
>>>>>>>> go
>>>>>>>>>>>>>> away.
>>>>>>>>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing
>>>> shards
>>>>>>>>>> will
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>> closed and replaced with a new shard).
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive
>>>> approach
>>>>>>>>>> would
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>> the thread-less IO model (pull based, non-blocking,
>> caller
>>>>>>>>>> retrieves
>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>> records when available). The current Kinesis API requires
>>> the
>>>>>> use
>>>>>>>>>> of
>>>>>>>>>>>>>>>>> threads. But that can be internal to the split reader and
>>>> does
>>>>>>>> not
>>>>>>>>>>>> need
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> be a source API concern. In fact, that's what we are
>>> working
>>>> on
>>>>>>>>>> right
>>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>> as improvement to the existing consumer: Each shard
>>> consumer
>>>>>>>> thread
>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>> push to a queue, the consumer main thread will poll the
>>>>>> queue(s).
>>>>>>>>>> It
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> The proposed SplitReader interface would fit the
>>> thread-less
>>>> IO
>>>>>>>>>>>> model.
>>>>>>>>>>>>>>>>> Similar to an iterator, we find out if there is a new
>>> element
>>>>>>>>>>>> (hasNext)
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> if so, move to it (next()). Separate calls deliver the
>> meta
>>>>>>>>>>>> information
>>>>>>>>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer
>> a
>>>>>>>> timeout
>>>>>>>>>>>>>>>> option,
>>>>>>>>>>>>>>>>> so that the caller does not end up in a busy wait. On the
>>>> other
>>>>>>>>>>>> hand, a
>>>>>>>>>>>>>>>>> caller processing multiple splits may want to cycle
>> through
>>>>>> fast,
>>>>>>>>>> to
>>>>>>>>>>>>>>>>> process elements of other splits as soon as they become
>>>>>>>> available.
>>>>>>>>>>>> The
>>>>>>>>>>>>>>>> nice
>>>>>>>>>>>>>>>>> thing is that this "split merge" logic can now live in
>>> Flink
>>>>>> and
>>>>>>>> be
>>>>>>>>>>>>>>>>> optimized and shared between different sources.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> Thomas
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <
>>>> guowei.mgw@gmail.com
>>>>>>> 
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>> Thanks Aljoscha for this FLIP.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
>>>> source
>>>>>> is
>>>>>>>>>>>> very
>>>>>>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may
>> be
>>>>>>>> another
>>>>>>>>>>>> way
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly
>>> if
>>>>>>>> every
>>>>>>>>>>>>>>>> advance
>>>>>>>>>>>>>>>>>> call return a Future.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> public interface Listener {
>>>>>>>>>>>>>>>>>> public void notify();
>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> public interface SplitReader() {
>>>>>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>>>>>> * When there is no element temporarily, this will return
>>>>>>>> false.
>>>>>>>>>>>>>>>>>> * When elements is available again splitReader can call
>>>>>>>>>>>>>>>>>> listener.notify()
>>>>>>>>>>>>>>>>>> * In addition the frame would check `advance`
>>> periodically .
>>>>>>>>>>>>>>>>>> * Of course advance can always return true and ignore
>> the
>>>>>>>>>>>>>> listener
>>>>>>>>>>>>>>>>>> argument for simplicity.
>>>>>>>>>>>>>>>>>> */
>>>>>>>>>>>>>>>>>> public boolean advance(Listener listener);
>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create
>> all
>>>>>> Splits
>>>>>>>>>> and
>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>> to create a SplitReader from a Split. But there is no
>>>> strategy
>>>>>>>> for
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>> to choose how to assign the splits to the tasks. I think
>>> we
>>>>>>>> could
>>>>>>>>>>>> add
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> Enum to let user to choose.
>>>>>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
>>>>>>>>>>>>>>>>>> Location,
>>>>>>>>>>>>>>>>>> Workload,
>>>>>>>>>>>>>>>>>> Random,
>>>>>>>>>>>>>>>>>> Average
>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>> */
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one
>> method
>>>> like
>>>>>>>>>>>>>> `getNext`
>>>>>>>>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp`
>>>>>> because
>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>> sources want to add timestamp to every element. IMO,
>> this
>>> is
>>>>>> not
>>>>>>>>>> so
>>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>> friendly so I prefer this design.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
>>>>>>>> 下午6:08写道:
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite
>> a
>>>> lot
>>>>>> of
>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>> possible improvements. I have one proposal. Instead of
>>>>>> having a
>>>>>>>>>>>>>> method:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> boolean advance() throws IOException;
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> I would replace it with
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> /*
>>>>>>>>>>>>>>>>>>> * Return a future, which when completed means that
>> source
>>>> has
>>>>>>>>>> more
>>>>>>>>>>>>>>>> data
>>>>>>>>>>>>>>>>>>> and getNext() will not block.
>>>>>>>>>>>>>>>>>>> * If you wish to use benefits of non blocking
>> connectors,
>>>>>>>> please
>>>>>>>>>>>>>>>>>>> implement this method appropriately.
>>>>>>>>>>>>>>>>>>> */
>>>>>>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
>>>>>>>>>>>>>>>>>>>  return CompletableFuture.completedFuture(null);
>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Couple of arguments:
>>>>>>>>>>>>>>>>>>> 1. I don’t understand the division of work between
>>>>>> `advance()`
>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> `getCurrent()`. What should be done in which,
>> especially
>>>> for
>>>>>>>>>>>>>> connectors
>>>>>>>>>>>>>>>>>>> that handle records in batches (like Kafka) and when
>>> should
>>>>>> you
>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>> `advance` and when `getCurrent()`.
>>>>>>>>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will
>>>> allow
>>>>>>>> us
>>>>>>>>>> in
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> future to have asynchronous/non blocking connectors and
>>>> more
>>>>>>>>>>>>>>>> efficiently
>>>>>>>>>>>>>>>>>>> handle large number of blocked threads, without busy
>>>> waiting.
>>>>>>>>>> While
>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> same time it doesn’t add much complexity, since naive
>>>>>> connector
>>>>>>>>>>>>>>>>>>> implementations can be always blocking.
>>>>>>>>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread
>>> pool
>>>>>> of
>>>>>>>>>> task
>>>>>>>>>>>>>>>>>>> executors, instead of one thread per task.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Piotrek
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
>>>>>>>> aljoscha@apache.org
>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Hi All,
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> In order to finally get the ball rolling on the new
>>> source
>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>> that we have discussed for so long I finally created a
>>>> FLIP:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>> 
>>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
>>>>>>>> work/discussion
>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>> adding per-partition watermark support to the Kinesis
>>>> source
>>>>>>>> and
>>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>>> this would enable generic implementation of event-time
>>>>>>>> alignment
>>>>>>>>>>>> for
>>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
>>>>>>>> alignment
>>>>>>>>>>>>>> part,
>>>>>>>>>>>>>>>>>>> especially the part about information sharing between
>>>>>>>> operations
>>>>>>>>>>>> (I'm
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>> calling it state sharing because state has a special
>>>> meaning
>>>>>> in
>>>>>>>>>>>>>> Flink).
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Please discuss away!
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Aljoscha
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>>>> 
>>>> 
>>>> 
>>>> 
>>> 
>> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Aljoscha,

Good point on the potential optimization in the source. One thing to
clarify, by "adding a minimumTimestamp()/maximumTimestamp() method pair to
the split interface", did you mean "split reader interface"? If so, what
should the readers do if they do not have such additional information? I am
wondering if it is possible to leave such optimization to the source
internal implementation.

@all
After reading all the feedback, Biao and I talked a little bit offline. We
would like to share some new thoughts with you and see what do you think.

When looking at the Source API, we were trying to answer two questions.
First of all, how would Flink use this API if someone else implemented it.
Secondly, how would the connector contributors implement the interface? How
difficult is the implementation.

KafkaConsumer is a typical example of a thread-less reader. The idea was to
allow different threading model on top of it. It could be a global single
thread handles record fetching and processing in an event loop pattern; it
could also be one dedicated fetcher thread for each consumer and a separate
thread pool for record processing. The API gives the freedom of picking up
threading model to the users. To answer the first question, I would love to
have such a source reader API so Flink can choose whatever threading model
it wants. However, implementing such an interface could be pretty
challenging and error prone.

On the other hand, having a source reader with a naive blocking socket is
probably simple enough in most cases (actually sometimes this might even be
the most efficient way). But it does not leave much option to Flink other
than creating one thread per reader.

Given the above thoughts, it might be reasonable to separate the
SplitReader API into two: SyncReader and AsyncReader. The sync reader just
has a simple blocking takeNext() API. And the AsyncReader just has a
pollNext(Callback) or Future<?> pollNext(). All the other methods are
shared by both readers and could be put into a package private parent
interface like BaseSplitReader.

Having these two readers allows both complicated and simple implementation,
depending on the SplitReader writers. From Flink's perspective, it will
choose a more efficient threading model if the SplitReader is an
AsyncReader. Otherwise, it may have to use the one thread per reader model
if the reader is a SyncReader. Users can also choose to implement both
interface, in that case, it is up to Flink to choose which interface to use.

Admittedly, this solution does have one more interface, but still seems
rewarding. Any thoughts?

Thanks,

Jiangjie (Becket) Qin


On Sun, Nov 18, 2018 at 11:33 PM Biao Liu <mm...@gmail.com> wrote:

> Hi community,
>
> Thank you guys for sharing ideas.
>
> The thing I really concern is about the thread mode.
> Actually in Alibaba, we have implemented our "split reader" based source
> two years ago. That's based on "SourceFunction", it's just an extension not
> a refactoring. It's almost same with the version Thomas and Jamie described
> in Google Doc. It really helps in many scenarios.
>
> However I don't like the thread mode which starts a thread for each split.
> Starting extra thread in operator is not an ideal way IMO. Especially
> thread count is decided by split count. So I was wondering if there is a
> more elegant way. Do we really want these threads in Flink core?
>
> I agree that blocking interface is more easy to implement. Could we at
> least separate the split reader with source function into different
> interfaces? Not all sources would like to read all splits concurrently. In
> batch scenario, reading splits one by one is more general. And also not all
> sources are partitioned, right?
> I prefer there is a new source interface with "pull mode" only, no split.
> There is a splittable source extended it. And there is one implementation
> that starting threads for each split, reading all splits concurrently.
>
>
> Thomas Weise <th...@apache.org> 于2018年11月18日周日 上午3:18写道:
>
> > @Aljoscha to address your question first: In the case of the Kinesis
> > consumer (with current Kinesis consumer API), there would also be N+1
> > threads. I have implemented a prototype similar to what is shown in
> Jamie's
> > document, where the thread ownership is similar to what you have done for
> > Kafka.
> >
> > The equivalent of split reader manages its own thread and the "source
> main
> > thread" is responsible for emitting the data. The interface between the N
> > reader threads and the 1 emitter is a blocking queue per consumer thread.
> > The emitter can now control which queue to consume from based on the
> event
> > time progress.
> >
> > This is akin to a "non-blocking" interface *between emitter and split
> > reader*. Emitter uses poll to retrieve records from the N queues (which
> > requires non-blocking interaction). The emitter is independent of the
> split
> > reader implementation, that part could live in Flink.
> >
> > Regarding whether or not to assume that split readers always need a
> thread
> > and in addition that these reader threads should be managed by Flink: It
> > depends on the API of respective external systems and I would not bake
> that
> > assumption into Flink. Some client libraries manage their own threads
> (see
> > push based API like JMS and as I understand it may also apply to the new
> > fan-out Kinesis API:
> >
> >
> https://docs.aws.amazon.com/streams/latest/dev/building-enhanced-consumers-api.html
> > ).
> > In such cases it would not make sense to layer another reader thread on
> > top. It may instead be better if Flink provides to the split reader the
> > queue/buffer to push records to.
> >
> > The discussion so far has largely ignored the discovery aspect. There are
> > some important considerations such as ordering dependency of splits and
> > work rebalancing that may affect the split reader interface. Should we
> fork
> > this into a separate thread?
> >
> > Thanks,
> > Thomas
> >
> >
> > On Fri, Nov 16, 2018 at 8:09 AM Piotr Nowojski <pi...@data-artisans.com>
> > wrote:
> >
> > > Hi Jamie,
> > >
> > > As it was already covered with my discussion with Becket, there is an
> > easy
> > > way to provide blocking API on top of non-blocking API. And yes we both
> > > agreed that blocking API is easier to implement by users.
> > >
> > > I also do not agree with respect to usefulness of non blocking API.
> > > Actually Kafka connector is the one that could be more efficient thanks
> > to
> > > the removal of the one layer of threading.
> > >
> > > Piotrek
> > >
> > > > On 16 Nov 2018, at 02:21, Jamie Grier <jg...@lyft.com.INVALID>
> wrote:
> > > >
> > > > Thanks Aljoscha for getting this effort going!
> > > >
> > > > There's been plenty of discussion here already and I'll add my big +1
> > to
> > > > making this interface very simple to implement for a new
> > > > Source/SplitReader.  Writing a new production quality connector for
> > Flink
> > > > is very difficult today and requires a lot of detailed knowledge
> about
> > > > Flink, event time progress, watermarking, idle shard detection, etc
> and
> > > it
> > > > would be good to move almost all of this type of code into Flink
> itself
> > > and
> > > > out of source implementations.  I also think this is totally doable
> and
> > > I'm
> > > > really excited to see this happening.
> > > >
> > > > I do have a couple of thoughts about the API and the implementation..
> > > >
> > > > In a perfect world there would be a single thread per Flink source
> > > sub-task
> > > > and no additional threads for SplitReaders -- but this assumes a
> world
> > > > where you have true async IO APIs for the upstream systems (like
> Kafka
> > > and
> > > > Kinesis, S3, HDFS, etc).  If that world did exist the single thread
> > could
> > > > just sit in an efficient select() call waiting for new data to arrive
> > on
> > > > any Split.  That'd be awesome..
> > > >
> > > > But, that world doesn't exist and given that practical consideration
> I
> > > > would think the next best implementation is going to be, in practice,
> > > > probably a thread per SplitReader that does nothing but call the
> source
> > > API
> > > > and drop whatever it reads into a (blocking) queue -- as Aljoscha
> > > mentioned
> > > > (calling it N+1) and as we started to describe here:
> > > >
> > >
> >
> https://docs.google.com/document/d/13HxfsucoN7aUls1FX202KFAVZ4IMLXEZWyRfZNd6WFM/edit#heading=h.lfi2u4fv9cxa
> > > >
> > > > I guess my point is that I think we should strive to move as much of
> > > > something like the diagram referenced in the above doc into Flink
> > itself
> > > > and out of sources and simplify the SplitReader API as much as
> possible
> > > as
> > > > well.
> > > >
> > > > With the above in mind and with regard to the discussion about
> > blocking,
> > > > etc..  I'm not sure I agree with some of the discussion so far with
> > > regard
> > > > to this API design.  The calls to the upstream systems
> (kafka/kinesis)
> > > are
> > > > in fact going to be blocking calls.  So a simple API without the
> > > constraint
> > > > that the methods must be implemented in a non-blocking way seems
> better
> > > to
> > > > me from the point of view of somebody writing a new source
> > > implementation.
> > > > My concern is that if you force the implementer of the SplitReader
> > > > interface to do so in a non-blocking way you're just going to make it
> > > > harder to write those implementations.  Those calls to read the next
> > bit
> > > of
> > > > data are going to be blocking calls with most known important sources
> > --
> > > at
> > > > least Kafka/Kinesis/HDFS -- so I think maybe we should just deal with
> > > that
> > > > head on and work around it a higher level so the SplitReader
> interface
> > > > stays super simple to implement.  This means we manage all the
> > threading
> > > in
> > > > Flink core, the API stays pull-based, and the implementer is allowed
> to
> > > > simply block until they have data to return.
> > > >
> > > > I maybe would change my mind about this if truly asynchronous APIs to
> > the
> > > > upstream source systems were likely to be available in the near
> future
> > or
> > > > are now and I'm just ignorant of it.  But even then the supporting
> code
> > > in
> > > > Flink to drive async and sync sources would be different and in fact
> > they
> > > > might just have different APIs altogether -- SplitReader vs
> > > > AsyncSplitReader maybe.
> > > >
> > > > In the end I think playing with the implementation, across more than
> > one
> > > > source, and moving as much common code into Flink itself will reveal
> > the
> > > > best API of course.
> > > >
> > > > One other interesting note is that you need to preserve per-partition
> > > > ordering so you have to take care with the implementation if it were
> to
> > > be
> > > > based on a thread pool and futures so as not to reorder the reads.
> > > >
> > > > Anyway, I'm thrilled to see this starting to move forward and I'd
> very
> > > much
> > > > like to help with the implementation wherever I can.  We're doing a
> > > > simplified internal version of some of this at Lyft for just Kinesis
> > > > because we need a solution for event time alignment in the very short
> > > term
> > > > but we'd like to immediately start helping to do this properly in
> Flink
> > > > after that.  One of the end goals for us is event time alignment
> across
> > > > heterogeneous sources.  Another is making it possible for non-expert
> > > users
> > > > to have a high probability of being able to write their own, correct,
> > > > connectors.
> > > >
> > > > -Jamie
> > > >
> > > > On Thu, Nov 15, 2018 at 3:43 AM Aljoscha Krettek <
> aljoscha@apache.org>
> > > > wrote:
> > > >
> > > >> Hi,
> > > >>
> > > >> I thought I had sent this mail a while ago but I must have forgotten
> > to
> > > >> send it.
> > > >>
> > > >> There is another thing we should consider for splits: the range of
> > > >> timestamps that it can contain. For example, the splits of a file
> > source
> > > >> would know what the minimum and maximum timestamp in the splits is,
> > > >> roughly. For infinite splits, such as Kafka partitions, the minimum
> > > would
> > > >> be meaningful but the maximum would be +Inf. If the splits expose
> the
> > > >> interval of time that they contain the readers, or the component
> that
> > > >> manages the readers can make decisions about which splits to forward
> > and
> > > >> read first. And it can also influence the minimum watermark that a
> > > reader
> > > >> forwards: it should never emit a watermark if it knows there are
> > splits
> > > to
> > > >> read that have a lower minimum timestamp. I think it should be as
> easy
> > > as
> > > >> adding a minimumTimestamp()/maximumTimestamp() method pair to the
> > split
> > > >> interface.
> > > >>
> > > >> Another thing we need to resolve is the actual reader interface. I
> see
> > > >> there has been some good discussion but I don't know if we have
> > > consensus.
> > > >> We should try and see how specific sources could be implemented with
> > the
> > > >> new interface. For example, for Kafka I think we need to have N+1
> > > threads
> > > >> per task (where N is the number of splits that a task is reading
> > from).
> > > On
> > > >> thread is responsible for reading from the splits. And each split
> has
> > > its
> > > >> own (internal) thread for reading from Kafka and putting messages in
> > an
> > > >> internal queue to pull from. This is similar to how the current
> Kafka
> > > >> source is implemented, which has a separate fetcher thread. The
> reason
> > > for
> > > >> this split is that we always need to try reading from Kafka to keep
> > the
> > > >> throughput up. In the current implementation the internal queue (or
> > > >> handover) limits the read rate of the reader threads.
> > > >>
> > > >> @Thomas, what do you think this would look like for Kinesis?
> > > >>
> > > >> Best,
> > > >> Aljoscha
> > > >>
> > > >>> On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com>
> wrote:
> > > >>>
> > > >>> Hi Piotrek,
> > > >>>
> > > >>> Thanks a lot for the detailed reply. All makes sense to me.
> > > >>>
> > > >>> WRT the confusion between advance() / getCurrent(), do you think it
> > > would
> > > >>> help if we combine them and have something like:
> > > >>>
> > > >>> CompletableFuture<T> getNext();
> > > >>> long getWatermark();
> > > >>> long getCurrentTimestamp();
> > > >>>
> > > >>> Cheers,
> > > >>>
> > > >>> Jiangjie (Becket) Qin
> > > >>>
> > > >>> On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <
> > > piotr@data-artisans.com>
> > > >>> wrote:
> > > >>>
> > > >>>> Hi,
> > > >>>>
> > > >>>> Thanks again for the detailed answer :) Sorry for responding with
> a
> > > >> delay.
> > > >>>>
> > > >>>>> Completely agree that in pattern 2, having a callback is
> necessary
> > > for
> > > >>>> that
> > > >>>>> single thread outside of the connectors. And the connectors MUST
> > have
> > > >>>>> internal threads.
> > > >>>>
> > > >>>> Yes, this thread will have to exists somewhere. In pattern 2 it
> > exists
> > > >> in
> > > >>>> the connector (at least from the perspective of the Flink
> execution
> > > >>>> engine). In pattern 1 it exists inside the Flink execution engine.
> > > With
> > > >>>> completely blocking connectors, like simple reading from files,
> both
> > > of
> > > >>>> those approaches are basically the same. The difference is when
> user
> > > >>>> implementing Flink source is already working with a non blocking
> > code
> > > >> with
> > > >>>> some internal threads. In this case, pattern 1 would result in
> > "double
> > > >>>> thread wrapping”, while pattern 2 would allow to skip one layer of
> > > >>>> indirection.
> > > >>>>
> > > >>>>> If we go that way, we should have something like "void
> > > >>>>> poll(Callback) / void advance(callback)". I am curious how would
> > > >>>>> CompletableFuture work here, though. If 10 readers returns 10
> > > >> completable
> > > >>>>> futures, will there be 10 additional threads (so 20 threads in
> > total)
> > > >>>>> blocking waiting on them? Or will there be a single thread busy
> > loop
> > > >>>>> checking around?
> > > >>>>
> > > >>>> To be honest, I haven’t thought this completely through and I
> > haven’t
> > > >>>> tested/POC’ed it. Having said that, I can think of at least couple
> > of
> > > >>>> solutions. First is something like this:
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>
> > >
> >
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> > > >>>> <
> > > >>>>
> > > >>
> > >
> >
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> > > >>>>>
> > > >>>>
> > > >>>> Line:
> > > >>>>
> > > >>>>                               `blocked = split.process();`
> > > >>>>
> > > >>>> Is where the execution goes into to the task/sources. This is
> where
> > > the
> > > >>>> returned future is handled:
> > > >>>>
> > > >>>>                               blocked.addListener(() -> {
> > > >>>>                                   blockedSplits.remove(split);
> > > >>>>                                   // reset the level priority to
> > > >> prevent
> > > >>>> previously-blocked splits from starving existing splits
> > > >>>>                                   split.resetLevelPriority();
> > > >>>>                                   waitingSplits.offer(split);
> > > >>>>                               }, executor);
> > > >>>>
> > > >>>> Fundamentally callbacks and Futures are more or less
> interchangeable
> > > You
> > > >>>> can always wrap one into another (creating a callback that
> > completes a
> > > >>>> future and attach a callback once future completes). In this case
> > the
> > > >>>> difference for me is mostly:
> > > >>>> - api with passing callback allows the callback to be fired
> multiple
> > > >> times
> > > >>>> and to fire it even if the connector is not blocked. This is what
> I
> > > >> meant
> > > >>>> by saying that api `CompletableFuture<?> isBlocked()` is a bit
> > > simpler.
> > > >>>> Connector can only return either “I’m not blocked” or “I’m blocked
> > > and I
> > > >>>> will tell you only once when I’m not blocked anymore”.
> > > >>>>
> > > >>>> But this is not the most important thing for me here. For me
> > important
> > > >>>> thing is to try our best to make Flink task’s control and
> execution
> > > >> single
> > > >>>> threaded. For that both callback and future APIs should work the
> > same.
> > > >>>>
> > > >>>>> WRT pattern 1, a single blocking take() API should just work. The
> > > good
> > > >>>>> thing is that a blocking read API is usually simpler to
> implement.
> > > >>>>
> > > >>>> Yes, they are easier to implement (especially if you are not the
> one
> > > >> that
> > > >>>> have to deal with the additional threading required around them ;)
> > ).
> > > >> But
> > > >>>> to answer this issue, if we choose pattern 2, we can always
> provide
> > a
> > > >>>> proxy/wrapper that would using the internal thread implement the
> > > >>>> non-blocking API while exposing blocking API to the user. It would
> > > >>>> implement pattern 2 for the user exposing to him pattern 1. In
> other
> > > >> words
> > > >>>> implementing pattern 1 in pattern 2 paradigm, while making it
> > possible
> > > >> to
> > > >>>> implement pure pattern 2 connectors.
> > > >>>>
> > > >>>>> BTW, one thing I am also trying to avoid is pushing users to
> > perform
> > > IO
> > > >>>> in
> > > >>>>> a method like "isBlocked()". If the method is expected to fetch
> > > records
> > > >>>>> (even if not returning them), naming it something more explicit
> > would
> > > >>>> help
> > > >>>>> avoid confusion.
> > > >>>>
> > > >>>> If we choose so, we could rework it into something like:
> > > >>>>
> > > >>>> CompletableFuture<?> advance()
> > > >>>> T getCurrent();
> > > >>>> Watermark getCurrentWatermark()
> > > >>>>
> > > >>>> But as I wrote before, this is more confusing to me for the exact
> > > >> reasons
> > > >>>> you mentioned :) I would be confused what should be done in
> > > `adanvce()`
> > > >> and
> > > >>>> what in `getCurrent()`. However, again this naming issue is not
> that
> > > >>>> important to me and probably is matter of taste/personal
> > preferences.
> > > >>>>
> > > >>>> Piotrek
> > > >>>>
> > > >>>>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com>
> wrote:
> > > >>>>>
> > > >>>>> Hi Piotrek,
> > > >>>>>
> > > >>>>> Thanks for the explanation. We are probably talking about the
> same
> > > >> thing
> > > >>>>> but in different ways. To clarify a little bit, I think there are
> > two
> > > >>>>> patterns to read from a connector.
> > > >>>>>
> > > >>>>> Pattern 1: Thread-less connector with a blocking read API.
> Outside
> > of
> > > >> the
> > > >>>>> connector, there is one IO thread per reader, doing blocking
> read.
> > An
> > > >>>>> additional thread will interact with all the IO threads.
> > > >>>>> Pattern 2: Connector with internal thread(s) and non-blocking
> API.
> > > >>>> Outside
> > > >>>>> of the connector, there is one thread for ALL readers, doing IO
> > > relying
> > > >>>> on
> > > >>>>> notification callbacks in the reader.
> > > >>>>>
> > > >>>>> In both patterns, there must be at least one thread per
> connector,
> > > >> either
> > > >>>>> inside (created by connector writers) or outside (created by
> Flink)
> > > of
> > > >>>> the
> > > >>>>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total,
> > to
> > > >> make
> > > >>>>> sure that 1 thread is fully non-blocking.
> > > >>>>>
> > > >>>>>> Btw, I don’t know if you understand my point. Having only
> `poll()`
> > > and
> > > >>>>> `take()` is not enough for single threaded task. If our source
> > > >> interface
> > > >>>>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
> > > >>>>> isBlocked(),`, there is no way to implement single threaded task
> > that
> > > >>>> both
> > > >>>>> reads the data from the source connector and can also react to
> > system
> > > >>>>> events. Ok, non >blocking `poll()` would allow that, but with
> busy
> > > >>>> looping.
> > > >>>>>
> > > >>>>> Completely agree that in pattern 2, having a callback is
> necessary
> > > for
> > > >>>> that
> > > >>>>> single thread outside of the connectors. And the connectors MUST
> > have
> > > >>>>> internal threads. If we go that way, we should have something
> like
> > > >> "void
> > > >>>>> poll(Callback) / void advance(callback)". I am curious how would
> > > >>>>> CompletableFuture work here, though. If 10 readers returns 10
> > > >> completable
> > > >>>>> futures, will there be 10 additional threads (so 20 threads in
> > total)
> > > >>>>> blocking waiting on them? Or will there be a single thread busy
> > loop
> > > >>>>> checking around?
> > > >>>>>
> > > >>>>> WRT pattern 1, a single blocking take() API should just work. The
> > > good
> > > >>>>> thing is that a blocking read API is usually simpler to
> implement.
> > An
> > > >>>>> additional non-blocking "T poll()" method here is indeed optional
> > and
> > > >>>> could
> > > >>>>> be used in cases like Flink does not want the thread to block
> > > forever.
> > > >>>> They
> > > >>>>> can also be combined to have a "T poll(Timeout)", which is
> exactly
> > > what
> > > >>>>> KafkaConsumer did.
> > > >>>>>
> > > >>>>> It sounds that you are proposing pattern 2 with something similar
> > to
> > > >> NIO2
> > > >>>>> AsynchronousByteChannel[1]. That API would work, except that the
> > > >>>> signature
> > > >>>>> returning future seems not necessary. If that is the case, a
> minor
> > > >> change
> > > >>>>> on the current FLIP proposal to have "void advance(callback)"
> > should
> > > >>>> work.
> > > >>>>> And this means the connectors MUST have their internal threads.
> > > >>>>>
> > > >>>>> BTW, one thing I am also trying to avoid is pushing users to
> > perform
> > > IO
> > > >>>> in
> > > >>>>> a method like "isBlocked()". If the method is expected to fetch
> > > records
> > > >>>>> (even if not returning them), naming it something more explicit
> > would
> > > >>>> help
> > > >>>>> avoid confusion.
> > > >>>>>
> > > >>>>> Thanks,
> > > >>>>>
> > > >>>>> Jiangjie (Becket) Qin
> > > >>>>>
> > > >>>>> [1]
> > > >>>>>
> > > >>>>
> > > >>
> > >
> >
> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
> > > >>>>>
> > > >>>>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <
> > > >> piotr@data-artisans.com>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>>> Hi
> > > >>>>>>
> > > >>>>>> Good point with select/epoll, however I do not see how they
> > couldn’t
> > > >> be
> > > >>>>>> with Flink if we would like single task in Flink to be
> > > single-threaded
> > > >>>> (and
> > > >>>>>> I believe we should pursue this goal). If your connector blocks
> on
> > > >>>>>> `select`, then it can not process/handle control messages from
> > > Flink,
> > > >>>> like
> > > >>>>>> checkpoints, releasing resources and potentially output flushes.
> > > This
> > > >>>> would
> > > >>>>>> require tight integration between connector and Flink’s main
> event
> > > >>>>>> loop/selects/etc.
> > > >>>>>>
> > > >>>>>> Looking at it from other perspective. Let’s assume that we have
> a
> > > >>>>>> connector implemented on top of `select`/`epoll`. In order to
> > > >> integrate
> > > >>>> it
> > > >>>>>> with Flink’s checkpointing/flushes/resource releasing it will
> have
> > > to
> > > >> be
> > > >>>>>> executed in separate thread one way or another. At least if our
> > API
> > > >> will
> > > >>>>>> enforce/encourage non blocking implementations with some kind of
> > > >>>>>> notifications (`isBlocked()` or `notify()` callback), some
> > > connectors
> > > >>>> might
> > > >>>>>> skip one layer of wapping threads.
> > > >>>>>>
> > > >>>>>> Btw, I don’t know if you understand my point. Having only
> `poll()`
> > > and
> > > >>>>>> `take()` is not enough for single threaded task. If our source
> > > >> interface
> > > >>>>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
> > > >>>>>> isBlocked(),`, there is no way to implement single threaded task
> > > that
> > > >>>> both
> > > >>>>>> reads the data from the source connector and can also react to
> > > system
> > > >>>>>> events. Ok, non blocking `poll()` would allow that, but with
> busy
> > > >>>> looping.
> > > >>>>>>
> > > >>>>>> Piotrek
> > > >>>>>>
> > > >>>>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com>
> > wrote:
> > > >>>>>>>
> > > >>>>>>> Hi Piotrek,
> > > >>>>>>>
> > > >>>>>>>> But I don’t see a reason why we should expose both blocking
> > > `take()`
> > > >>>> and
> > > >>>>>>> non-blocking `poll()` methods to the Flink engine. Someone
> (Flink
> > > >>>> engine
> > > >>>>>> or
> > > >>>>>>> connector) would have to do the same busy
> > > >>>>>>>> looping anyway and I think it would be better to have a
> simpler
> > > >>>>>> connector
> > > >>>>>>> API (that would solve our problems) and force connectors to
> > comply
> > > >> one
> > > >>>>>> way
> > > >>>>>>> or another.
> > > >>>>>>>
> > > >>>>>>> If we let the block happen inside the connector, the blocking
> > does
> > > >> not
> > > >>>>>> have
> > > >>>>>>> to be a busy loop. For example, to do the block waiting
> > > efficiently,
> > > >>>> the
> > > >>>>>>> connector can use java NIO selector().select which relies on OS
> > > >> syscall
> > > >>>>>>> like epoll[1] instead of busy looping. But if Flink engine
> blocks
> > > >>>> outside
> > > >>>>>>> the connector, it pretty much has to do the busy loop. So if
> > there
> > > is
> > > >>>>>> only
> > > >>>>>>> one API to get the element, a blocking getNextElement() makes
> > more
> > > >>>> sense.
> > > >>>>>>> In any case, we should avoid ambiguity. It has to be crystal
> > clear
> > > >>>> about
> > > >>>>>>> whether a method is expected to be blocking or non-blocking.
> > > >> Otherwise
> > > >>>> it
> > > >>>>>>> would be very difficult for Flink engine to do the right thing
> > with
> > > >> the
> > > >>>>>>> connectors. At the first glance at getCurrent(), the expected
> > > >> behavior
> > > >>>> is
> > > >>>>>>> not quite clear.
> > > >>>>>>>
> > > >>>>>>> That said, I do agree that functionality wise, poll() and
> take()
> > > kind
> > > >>>> of
> > > >>>>>>> overlap. But they are actually not quite different from
> > > >>>>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the
> only
> > > >>>>>>> difference is that poll() also returns the next record if it is
> > > >>>>>> available.
> > > >>>>>>> But I agree that the isBlocked() + getNextElement() is more
> > > flexible
> > > >> as
> > > >>>>>>> users can just check the record availability, but not fetch the
> > > next
> > > >>>>>>> element.
> > > >>>>>>>
> > > >>>>>>>> In case of thread-less readers with only non-blocking
> > > `queue.poll()`
> > > >>>> (is
> > > >>>>>>> that really a thing? I can not think about a real
> implementation
> > > that
> > > >>>>>>> enforces such constraints)
> > > >>>>>>> Right, it is pretty much a syntax sugar to allow user combine
> the
> > > >>>>>>> check-and-take into one method. It could be achieved with
> > > >> isBlocked() +
> > > >>>>>>> getNextElement().
> > > >>>>>>>
> > > >>>>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
> > > >>>>>>>
> > > >>>>>>> Thanks,
> > > >>>>>>>
> > > >>>>>>> Jiangjie (Becket) Qin
> > > >>>>>>>
> > > >>>>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
> > > >>>> piotr@data-artisans.com>
> > > >>>>>>> wrote:
> > > >>>>>>>
> > > >>>>>>>> Hi Becket,
> > > >>>>>>>>
> > > >>>>>>>> With my proposal, both of your examples would have to be
> solved
> > by
> > > >> the
> > > >>>>>>>> connector and solution to both problems would be the same:
> > > >>>>>>>>
> > > >>>>>>>> Pretend that connector is never blocked (`isBlocked() { return
> > > >>>>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking
> > > >> fashion
> > > >>>>>> (or
> > > >>>>>>>> semi blocking with return of control from time to time to
> allow
> > > for
> > > >>>>>>>> checkpointing, network flushing and other resource management
> > > things
> > > >>>> to
> > > >>>>>>>> happen in the same main thread). In other words, exactly how
> you
> > > >> would
> > > >>>>>>>> implement `take()` method or how the same source connector
> would
> > > be
> > > >>>>>>>> implemented NOW with current source interface. The difference
> > with
> > > >>>>>> current
> > > >>>>>>>> interface would be only that main loop would be outside of the
> > > >>>>>> connector,
> > > >>>>>>>> and instead of periodically releasing checkpointing lock,
> > > >> periodically
> > > >>>>>>>> `return null;` or `return Optional.empty();` from
> > > >> `getNextElement()`.
> > > >>>>>>>>
> > > >>>>>>>> In case of thread-less readers with only non-blocking
> > > `queue.poll()`
> > > >>>> (is
> > > >>>>>>>> that really a thing? I can not think about a real
> implementation
> > > >> that
> > > >>>>>>>> enforces such constraints), we could provide a wrapper that
> > hides
> > > >> the
> > > >>>>>> busy
> > > >>>>>>>> looping. The same applies how to solve forever blocking
> readers
> > -
> > > we
> > > >>>>>> could
> > > >>>>>>>> provider another wrapper running the connector in separate
> > thread.
> > > >>>>>>>>
> > > >>>>>>>> But I don’t see a reason why we should expose both blocking
> > > `take()`
> > > >>>> and
> > > >>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone
> > (Flink
> > > >>>>>> engine or
> > > >>>>>>>> connector) would have to do the same busy looping anyway and I
> > > think
> > > >>>> it
> > > >>>>>>>> would be better to have a simpler connector API (that would
> > solve
> > > >> our
> > > >>>>>>>> problems) and force connectors to comply one way or another.
> > > >>>>>>>>
> > > >>>>>>>> Piotrek
> > > >>>>>>>>
> > > >>>>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com>
> > > wrote:
> > > >>>>>>>>>
> > > >>>>>>>>> Hi Piotr,
> > > >>>>>>>>>
> > > >>>>>>>>> I might have misunderstood you proposal. But let me try to
> > > explain
> > > >> my
> > > >>>>>>>>> concern. I am thinking about the following case:
> > > >>>>>>>>> 1. a reader has the following two interfaces,
> > > >>>>>>>>> boolean isBlocked()
> > > >>>>>>>>> T getNextElement()
> > > >>>>>>>>> 2. the implementation of getNextElement() is non-blocking.
> > > >>>>>>>>> 3. The reader is thread-less, i.e. it does not have any
> > internal
> > > >>>>>> thread.
> > > >>>>>>>>> For example, it might just delegate the getNextElement() to a
> > > >>>>>>>> queue.poll(),
> > > >>>>>>>>> and isBlocked() is just queue.isEmpty().
> > > >>>>>>>>>
> > > >>>>>>>>> How can Flink efficiently implement a blocking reading
> behavior
> > > >> with
> > > >>>>>> this
> > > >>>>>>>>> reader? Either a tight loop or a backoff interval is needed.
> > > >> Neither
> > > >>>> of
> > > >>>>>>>>> them is ideal.
> > > >>>>>>>>>
> > > >>>>>>>>> Now let's say in the reader mentioned above implements a
> > blocking
> > > >>>>>>>>> getNextElement() method. Because there is no internal thread
> in
> > > the
> > > >>>>>>>> reader,
> > > >>>>>>>>> after isBlocked() returns false. Flink will still have to
> loop
> > on
> > > >>>>>>>>> isBlocked() to check whether the next record is available. If
> > the
> > > >>>> next
> > > >>>>>>>>> record reaches after 10 min, it is a tight loop for 10 min.
> You
> > > >> have
> > > >>>>>>>>> probably noticed that in this case, even isBlocked() returns
> a
> > > >>>> future,
> > > >>>>>>>> that
> > > >>>>>>>>> future() will not be completed if Flink does not call some
> > method
> > > >>>> from
> > > >>>>>>>> the
> > > >>>>>>>>> reader, because the reader has no internal thread to complete
> > > that
> > > >>>>>> future
> > > >>>>>>>>> by itself.
> > > >>>>>>>>>
> > > >>>>>>>>> Due to the above reasons, a blocking take() API would allow
> > Flink
> > > >> to
> > > >>>>>> have
> > > >>>>>>>>> an efficient way to read from a reader. There are many ways
> to
> > > wake
> > > >>>> up
> > > >>>>>>>> the
> > > >>>>>>>>> blocking thread when checkpointing is needed depending on the
> > > >>>>>>>>> implementation. But I think the poll()/take() API would also
> > work
> > > >> in
> > > >>>>>> that
> > > >>>>>>>>> case.
> > > >>>>>>>>>
> > > >>>>>>>>> Thanks,
> > > >>>>>>>>>
> > > >>>>>>>>> Jiangjie (Becket) Qin
> > > >>>>>>>>>
> > > >>>>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
> > > >>>> piotr@data-artisans.com
> > > >>>>>>>
> > > >>>>>>>>> wrote:
> > > >>>>>>>>>
> > > >>>>>>>>>> Hi,
> > > >>>>>>>>>>
> > > >>>>>>>>>> a)
> > > >>>>>>>>>>
> > > >>>>>>>>>>> BTW, regarding the isBlock() method, I have a few more
> > > questions.
> > > >>>> 21,
> > > >>>>>>>> Is
> > > >>>>>>>>>> a method isReady() with boolean as a return value
> > > >>>>>>>>>>> equivalent? Personally I found it is a little bit confusing
> > in
> > > >> what
> > > >>>>>> is
> > > >>>>>>>>>> supposed to be returned when the future is completed. 22. if
> > > >>>>>>>>>>> the implementation of isBlocked() is optional, how do the
> > > callers
> > > >>>>>> know
> > > >>>>>>>>>> whether the method is properly implemented or not?
> > > >>>>>>>>>>> Does not implemented mean it always return a completed
> > future?
> > > >>>>>>>>>>
> > > >>>>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an
> > equivalent
> > > >> to
> > > >>>>>>>>>> `boolean hasNext()` which in case of “false” provides some
> > kind
> > > >> of a
> > > >>>>>>>>>> listener/callback that notifies about presence of next
> > element.
> > > >>>> There
> > > >>>>>>>> are
> > > >>>>>>>>>> some minor details, like `CompletableFuture<?>` has a
> minimal
> > > two
> > > >>>>>> state
> > > >>>>>>>>>> logic:
> > > >>>>>>>>>>
> > > >>>>>>>>>> 1. Future is completed - we have more data
> > > >>>>>>>>>> 2. Future not yet completed - we don’t have data now, but we
> > > >>>> might/we
> > > >>>>>>>> will
> > > >>>>>>>>>> have in the future
> > > >>>>>>>>>>
> > > >>>>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit
> > more
> > > >>>>>>>>>> complicated/dispersed and can lead/encourage `notify()`
> spam.
> > > >>>>>>>>>>
> > > >>>>>>>>>> b)
> > > >>>>>>>>>>
> > > >>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method
> > like
> > > >>>>>>>> `getNext`
> > > >>>>>>>>>> the `getNext` would need return a
> > > >>>>>>>>>>> `ElementWithTimestamp` because some sources want to add
> > > timestamp
> > > >>>> to
> > > >>>>>>>>>> every element. IMO, this is not so memory friendly
> > > >>>>>>>>>>> so I prefer this design.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Guowei I don’t quite understand this. Could you elaborate
> why
> > > >>>> having a
> > > >>>>>>>>>> separate `advance()` help?
> > > >>>>>>>>>>
> > > >>>>>>>>>> c)
> > > >>>>>>>>>>
> > > >>>>>>>>>> Regarding advance/poll/take. What’s the value of having two
> > > >> separate
> > > >>>>>>>>>> methods: poll and take? Which one of them should be called
> and
> > > >> which
> > > >>>>>>>>>> implemented? What’s the benefit of having those methods
> > compared
> > > >> to
> > > >>>>>>>> having
> > > >>>>>>>>>> a one single method `getNextElement()` (or `pollElement() or
> > > >>>> whatever
> > > >>>>>> we
> > > >>>>>>>>>> name it) with following contract:
> > > >>>>>>>>>>
> > > >>>>>>>>>> CompletableFuture<?> isBlocked();
> > > >>>>>>>>>>
> > > >>>>>>>>>> /**
> > > >>>>>>>>>> Return next element - will be called only if `isBlocked()`
> is
> > > >>>>>> completed.
> > > >>>>>>>>>> Try to implement it in non blocking fashion, but if that’s
> > > >>>> impossible
> > > >>>>>> or
> > > >>>>>>>>>> you just don’t need the effort, you can block in this
> method.
> > > >>>>>>>>>> */
> > > >>>>>>>>>> T getNextElement();
> > > >>>>>>>>>>
> > > >>>>>>>>>> I mean, if the connector is implemented non-blockingly,
> Flink
> > > >> should
> > > >>>>>> use
> > > >>>>>>>>>> it that way. If it’s not, then `poll()` will `throw new
> > > >>>>>>>>>> NotImplementedException()`. Implementing both of them and
> > > >> providing
> > > >>>>>>>> both of
> > > >>>>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them
> > > into
> > > >> a
> > > >>>>>>>> single
> > > >>>>>>>>>> method call that should preferably (but not necessarily need
> > to)
> > > >> be
> > > >>>>>>>>>> non-blocking? It’s not like we are implementing general
> > purpose
> > > >>>>>> `Queue`,
> > > >>>>>>>>>> which users might want to call either of `poll` or `take`.
> We
> > > >> would
> > > >>>>>>>> always
> > > >>>>>>>>>> prefer to call `poll`, but if it’s blocking, then still we
> > have
> > > no
> > > >>>>>>>> choice,
> > > >>>>>>>>>> but to call it and block on it.
> > > >>>>>>>>>>
> > > >>>>>>>>>> d)
> > > >>>>>>>>>>
> > > >>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
> source
> > > is
> > > >>>> very
> > > >>>>>>>>>>> important. But in addition to `Future/poll`, there may be
> > > another
> > > >>>> way
> > > >>>>>>>> to
> > > >>>>>>>>>>> achieve this. I think it may be not very memory friendly if
> > > every
> > > >>>>>>>> advance
> > > >>>>>>>>>>> call return a Future.
> > > >>>>>>>>>>
> > > >>>>>>>>>> I didn’t want to mention this, to not clog my initial
> > proposal,
> > > >> but
> > > >>>>>>>> there
> > > >>>>>>>>>> is a simple solution for the problem:
> > > >>>>>>>>>>
> > > >>>>>>>>>> public interface SplitReader {
> > > >>>>>>>>>>
> > > >>>>>>>>>> (…)
> > > >>>>>>>>>>
> > > >>>>>>>>>> CompletableFuture<?> NOT_BLOCKED =
> > > >>>>>>>>>> CompletableFuture.completedFuture(null);
> > > >>>>>>>>>>
> > > >>>>>>>>>> /**
> > > >>>>>>>>>> * Returns a future that will be completed when the page
> source
> > > >>>>>>>> becomes
> > > >>>>>>>>>> * unblocked.  If the page source is not blocked, this method
> > > >>>> should
> > > >>>>>>>>>> return
> > > >>>>>>>>>> * {@code NOT_BLOCKED}.
> > > >>>>>>>>>> */
> > > >>>>>>>>>> default CompletableFuture<?> isBlocked()
> > > >>>>>>>>>> {
> > > >>>>>>>>>>    return NOT_BLOCKED;
> > > >>>>>>>>>> }
> > > >>>>>>>>>>
> > > >>>>>>>>>> If we are blocked and we are waiting for the IO, then
> > creating a
> > > >> new
> > > >>>>>>>>>> Future is non-issue. Under full throttle/throughput and not
> > > >> blocked
> > > >>>>>>>> sources
> > > >>>>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve
> > the
> > > >>>>>>>> problem.
> > > >>>>>>>>>>
> > > >>>>>>>>>> One more remark, non-blocking sources might be a necessity
> in
> > a
> > > >>>> single
> > > >>>>>>>>>> threaded model without a checkpointing lock. (Currently when
> > > >> sources
> > > >>>>>> are
> > > >>>>>>>>>> blocked, they can release checkpointing lock and re-acquire
> it
> > > >> again
> > > >>>>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for
> > > >> checkpoints
> > > >>>> to
> > > >>>>>>>>>> happen when source is idling. In that case either `notify()`
> > or
> > > my
> > > >>>>>>>> proposed
> > > >>>>>>>>>> `isBlocked()` would allow to avoid busy-looping.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Piotrek
> > > >>>>>>>>>>
> > > >>>>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com>
> > > >> wrote:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Hi Thomas,
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> The iterator-like API was also the first thing that came to
> > me.
> > > >> But
> > > >>>>>> it
> > > >>>>>>>>>>> seems a little confusing that hasNext() does not mean "the
> > > stream
> > > >>>> has
> > > >>>>>>>> not
> > > >>>>>>>>>>> ended", but means "the next record is ready", which is
> > > >> repurposing
> > > >>>>>> the
> > > >>>>>>>>>> well
> > > >>>>>>>>>>> known meaning of hasNext(). If we follow the
> hasNext()/next()
> > > >>>>>> pattern,
> > > >>>>>>>> an
> > > >>>>>>>>>>> additional isNextReady() method to indicate whether the
> next
> > > >> record
> > > >>>>>> is
> > > >>>>>>>>>>> ready seems more intuitive to me.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Similarly, in poll()/take() pattern, another method of
> > isDone()
> > > >> is
> > > >>>>>>>> needed
> > > >>>>>>>>>>> to indicate whether the stream has ended or not.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
> > > >>>>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
> > > >>>>>>>> implementation.
> > > >>>>>>>>>>> When I am implementing a reader, I could have a couple of
> > > >> choices:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> - A thread-less reader that does not have any internal
> > thread.
> > > >>>>>>>>>>> - When poll() is called, the same calling thread will
> > perform a
> > > >>>> bunch
> > > >>>>>>>>>> of
> > > >>>>>>>>>>> IO asynchronously.
> > > >>>>>>>>>>> - When take() is called, the same calling thread will
> > perform a
> > > >>>>>>>>>> bunch
> > > >>>>>>>>>>> of IO and wait until the record is ready.
> > > >>>>>>>>>>> - A reader with internal threads performing network IO and
> > put
> > > >>>>>> records
> > > >>>>>>>>>>> into a buffer.
> > > >>>>>>>>>>> - When poll() is called, the calling thread simply reads
> from
> > > >> the
> > > >>>>>>>>>>> buffer and return empty result immediately if there is no
> > > >> record.
> > > >>>>>>>>>>> - When take() is called, the calling thread reads from the
> > > >> buffer
> > > >>>>>>>>>> and
> > > >>>>>>>>>>> block waiting if the buffer is empty.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady()
> > API,
> > > >> it
> > > >>>> is
> > > >>>>>>>>>> less
> > > >>>>>>>>>>> intuitive for the reader developers to write the
> thread-less
> > > >>>> pattern.
> > > >>>>>>>>>>> Although technically speaking one can still do the
> > asynchronous
> > > >> IO
> > > >>>> to
> > > >>>>>>>>>>> prepare the record in isNextReady(). But it is inexplicit
> and
> > > >> seems
> > > >>>>>>>>>>> somewhat hacky.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Jiangjie (Becket) Qin
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <
> thw@apache.org>
> > > >>>> wrote:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>> Couple more points regarding discovery:
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> The proposal mentions that discovery could be outside the
> > > >>>> execution
> > > >>>>>>>>>> graph.
> > > >>>>>>>>>>>> Today, discovered partitions/shards are checkpointed. I
> > > believe
> > > >>>> that
> > > >>>>>>>>>> will
> > > >>>>>>>>>>>> also need to be the case in the future, even when
> discovery
> > > and
> > > >>>>>>>> reading
> > > >>>>>>>>>> are
> > > >>>>>>>>>>>> split between different tasks.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> For cases such as resharding of a Kinesis stream, the
> > > >> relationship
> > > >>>>>>>>>> between
> > > >>>>>>>>>>>> splits needs to be considered. Splits cannot be randomly
> > > >>>> distributed
> > > >>>>>>>>>> over
> > > >>>>>>>>>>>> readers in certain situations. An example was mentioned
> > here:
> > > >>>>>>>>>>>>
> > > >> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Thomas
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <
> thw@apache.org
> > >
> > > >>>> wrote:
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>> Thanks for getting the ball rolling on this!
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Can the number of splits decrease? Yes, splits can be
> > closed
> > > >> and
> > > >>>> go
> > > >>>>>>>>>> away.
> > > >>>>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing
> > > shards
> > > >>>>>> will
> > > >>>>>>>> be
> > > >>>>>>>>>>>>> closed and replaced with a new shard).
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive
> > > approach
> > > >>>>>> would
> > > >>>>>>>>>> be
> > > >>>>>>>>>>>>> the thread-less IO model (pull based, non-blocking,
> caller
> > > >>>>>> retrieves
> > > >>>>>>>>>> new
> > > >>>>>>>>>>>>> records when available). The current Kinesis API requires
> > the
> > > >> use
> > > >>>>>> of
> > > >>>>>>>>>>>>> threads. But that can be internal to the split reader and
> > > does
> > > >>>> not
> > > >>>>>>>> need
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>> be a source API concern. In fact, that's what we are
> > working
> > > on
> > > >>>>>> right
> > > >>>>>>>>>> now
> > > >>>>>>>>>>>>> as improvement to the existing consumer: Each shard
> > consumer
> > > >>>> thread
> > > >>>>>>>>>> will
> > > >>>>>>>>>>>>> push to a queue, the consumer main thread will poll the
> > > >> queue(s).
> > > >>>>>> It
> > > >>>>>>>> is
> > > >>>>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> The proposed SplitReader interface would fit the
> > thread-less
> > > IO
> > > >>>>>>>> model.
> > > >>>>>>>>>>>>> Similar to an iterator, we find out if there is a new
> > element
> > > >>>>>>>> (hasNext)
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>> if so, move to it (next()). Separate calls deliver the
> meta
> > > >>>>>>>> information
> > > >>>>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer
> a
> > > >>>> timeout
> > > >>>>>>>>>>>> option,
> > > >>>>>>>>>>>>> so that the caller does not end up in a busy wait. On the
> > > other
> > > >>>>>>>> hand, a
> > > >>>>>>>>>>>>> caller processing multiple splits may want to cycle
> through
> > > >> fast,
> > > >>>>>> to
> > > >>>>>>>>>>>>> process elements of other splits as soon as they become
> > > >>>> available.
> > > >>>>>>>> The
> > > >>>>>>>>>>>> nice
> > > >>>>>>>>>>>>> thing is that this "split merge" logic can now live in
> > Flink
> > > >> and
> > > >>>> be
> > > >>>>>>>>>>>>> optimized and shared between different sources.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>> Thomas
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <
> > > guowei.mgw@gmail.com
> > > >>>
> > > >>>>>>>> wrote:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Hi,
> > > >>>>>>>>>>>>>> Thanks Aljoscha for this FLIP.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
> > > source
> > > >> is
> > > >>>>>>>> very
> > > >>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may
> be
> > > >>>> another
> > > >>>>>>>> way
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly
> > if
> > > >>>> every
> > > >>>>>>>>>>>> advance
> > > >>>>>>>>>>>>>> call return a Future.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> public interface Listener {
> > > >>>>>>>>>>>>>> public void notify();
> > > >>>>>>>>>>>>>> }
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> public interface SplitReader() {
> > > >>>>>>>>>>>>>> /**
> > > >>>>>>>>>>>>>> * When there is no element temporarily, this will return
> > > >>>> false.
> > > >>>>>>>>>>>>>> * When elements is available again splitReader can call
> > > >>>>>>>>>>>>>> listener.notify()
> > > >>>>>>>>>>>>>> * In addition the frame would check `advance`
> > periodically .
> > > >>>>>>>>>>>>>> * Of course advance can always return true and ignore
> the
> > > >>>>>>>>>> listener
> > > >>>>>>>>>>>>>> argument for simplicity.
> > > >>>>>>>>>>>>>> */
> > > >>>>>>>>>>>>>> public boolean advance(Listener listener);
> > > >>>>>>>>>>>>>> }
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create
> all
> > > >> Splits
> > > >>>>>> and
> > > >>>>>>>>>> how
> > > >>>>>>>>>>>>>> to create a SplitReader from a Split. But there is no
> > > strategy
> > > >>>> for
> > > >>>>>>>> the
> > > >>>>>>>>>>>> user
> > > >>>>>>>>>>>>>> to choose how to assign the splits to the tasks. I think
> > we
> > > >>>> could
> > > >>>>>>>> add
> > > >>>>>>>>>> a
> > > >>>>>>>>>>>>>> Enum to let user to choose.
> > > >>>>>>>>>>>>>> /**
> > > >>>>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
> > > >>>>>>>>>>>>>> Location,
> > > >>>>>>>>>>>>>> Workload,
> > > >>>>>>>>>>>>>> Random,
> > > >>>>>>>>>>>>>> Average
> > > >>>>>>>>>>>>>> }
> > > >>>>>>>>>>>>>> */
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one
> method
> > > like
> > > >>>>>>>>>> `getNext`
> > > >>>>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp`
> > > >> because
> > > >>>>>>>> some
> > > >>>>>>>>>>>>>> sources want to add timestamp to every element. IMO,
> this
> > is
> > > >> not
> > > >>>>>> so
> > > >>>>>>>>>>>> memory
> > > >>>>>>>>>>>>>> friendly so I prefer this design.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Thanks
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
> > > >>>> 下午6:08写道:
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Hi,
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite
> a
> > > lot
> > > >> of
> > > >>>>>>>> other
> > > >>>>>>>>>>>>>>> possible improvements. I have one proposal. Instead of
> > > >> having a
> > > >>>>>>>>>> method:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> boolean advance() throws IOException;
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> I would replace it with
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> /*
> > > >>>>>>>>>>>>>>> * Return a future, which when completed means that
> source
> > > has
> > > >>>>>> more
> > > >>>>>>>>>>>> data
> > > >>>>>>>>>>>>>>> and getNext() will not block.
> > > >>>>>>>>>>>>>>> * If you wish to use benefits of non blocking
> connectors,
> > > >>>> please
> > > >>>>>>>>>>>>>>> implement this method appropriately.
> > > >>>>>>>>>>>>>>> */
> > > >>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
> > > >>>>>>>>>>>>>>>   return CompletableFuture.completedFuture(null);
> > > >>>>>>>>>>>>>>> }
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Couple of arguments:
> > > >>>>>>>>>>>>>>> 1. I don’t understand the division of work between
> > > >> `advance()`
> > > >>>>>> and
> > > >>>>>>>>>>>>>>> `getCurrent()`. What should be done in which,
> especially
> > > for
> > > >>>>>>>>>> connectors
> > > >>>>>>>>>>>>>>> that handle records in batches (like Kafka) and when
> > should
> > > >> you
> > > >>>>>>>> call
> > > >>>>>>>>>>>>>>> `advance` and when `getCurrent()`.
> > > >>>>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will
> > > allow
> > > >>>> us
> > > >>>>>> in
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>> future to have asynchronous/non blocking connectors and
> > > more
> > > >>>>>>>>>>>> efficiently
> > > >>>>>>>>>>>>>>> handle large number of blocked threads, without busy
> > > waiting.
> > > >>>>>> While
> > > >>>>>>>>>> at
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>> same time it doesn’t add much complexity, since naive
> > > >> connector
> > > >>>>>>>>>>>>>>> implementations can be always blocking.
> > > >>>>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread
> > pool
> > > >> of
> > > >>>>>> task
> > > >>>>>>>>>>>>>>> executors, instead of one thread per task.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Piotrek
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
> > > >>>> aljoscha@apache.org
> > > >>>>>>>
> > > >>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Hi All,
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> In order to finally get the ball rolling on the new
> > source
> > > >>>>>>>> interface
> > > >>>>>>>>>>>>>>> that we have discussed for so long I finally created a
> > > FLIP:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>
> > > >>>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
> > > >>>> work/discussion
> > > >>>>>>>>>> about
> > > >>>>>>>>>>>>>>> adding per-partition watermark support to the Kinesis
> > > source
> > > >>>> and
> > > >>>>>>>>>>>> because
> > > >>>>>>>>>>>>>>> this would enable generic implementation of event-time
> > > >>>> alignment
> > > >>>>>>>> for
> > > >>>>>>>>>>>> all
> > > >>>>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
> > > >>>> alignment
> > > >>>>>>>>>> part,
> > > >>>>>>>>>>>>>>> especially the part about information sharing between
> > > >>>> operations
> > > >>>>>>>> (I'm
> > > >>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>> calling it state sharing because state has a special
> > > meaning
> > > >> in
> > > >>>>>>>>>> Flink).
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Please discuss away!
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Aljoscha
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>
> > > >>>>
> > > >>
> > > >>
> > >
> > >
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Biao Liu <mm...@gmail.com>.
Hi community,

Thank you guys for sharing ideas.

The thing I really concern is about the thread mode.
Actually in Alibaba, we have implemented our "split reader" based source
two years ago. That's based on "SourceFunction", it's just an extension not
a refactoring. It's almost same with the version Thomas and Jamie described
in Google Doc. It really helps in many scenarios.

However I don't like the thread mode which starts a thread for each split.
Starting extra thread in operator is not an ideal way IMO. Especially
thread count is decided by split count. So I was wondering if there is a
more elegant way. Do we really want these threads in Flink core?

I agree that blocking interface is more easy to implement. Could we at
least separate the split reader with source function into different
interfaces? Not all sources would like to read all splits concurrently. In
batch scenario, reading splits one by one is more general. And also not all
sources are partitioned, right?
I prefer there is a new source interface with "pull mode" only, no split.
There is a splittable source extended it. And there is one implementation
that starting threads for each split, reading all splits concurrently.


Thomas Weise <th...@apache.org> 于2018年11月18日周日 上午3:18写道:

> @Aljoscha to address your question first: In the case of the Kinesis
> consumer (with current Kinesis consumer API), there would also be N+1
> threads. I have implemented a prototype similar to what is shown in Jamie's
> document, where the thread ownership is similar to what you have done for
> Kafka.
>
> The equivalent of split reader manages its own thread and the "source main
> thread" is responsible for emitting the data. The interface between the N
> reader threads and the 1 emitter is a blocking queue per consumer thread.
> The emitter can now control which queue to consume from based on the event
> time progress.
>
> This is akin to a "non-blocking" interface *between emitter and split
> reader*. Emitter uses poll to retrieve records from the N queues (which
> requires non-blocking interaction). The emitter is independent of the split
> reader implementation, that part could live in Flink.
>
> Regarding whether or not to assume that split readers always need a thread
> and in addition that these reader threads should be managed by Flink: It
> depends on the API of respective external systems and I would not bake that
> assumption into Flink. Some client libraries manage their own threads (see
> push based API like JMS and as I understand it may also apply to the new
> fan-out Kinesis API:
>
> https://docs.aws.amazon.com/streams/latest/dev/building-enhanced-consumers-api.html
> ).
> In such cases it would not make sense to layer another reader thread on
> top. It may instead be better if Flink provides to the split reader the
> queue/buffer to push records to.
>
> The discussion so far has largely ignored the discovery aspect. There are
> some important considerations such as ordering dependency of splits and
> work rebalancing that may affect the split reader interface. Should we fork
> this into a separate thread?
>
> Thanks,
> Thomas
>
>
> On Fri, Nov 16, 2018 at 8:09 AM Piotr Nowojski <pi...@data-artisans.com>
> wrote:
>
> > Hi Jamie,
> >
> > As it was already covered with my discussion with Becket, there is an
> easy
> > way to provide blocking API on top of non-blocking API. And yes we both
> > agreed that blocking API is easier to implement by users.
> >
> > I also do not agree with respect to usefulness of non blocking API.
> > Actually Kafka connector is the one that could be more efficient thanks
> to
> > the removal of the one layer of threading.
> >
> > Piotrek
> >
> > > On 16 Nov 2018, at 02:21, Jamie Grier <jg...@lyft.com.INVALID> wrote:
> > >
> > > Thanks Aljoscha for getting this effort going!
> > >
> > > There's been plenty of discussion here already and I'll add my big +1
> to
> > > making this interface very simple to implement for a new
> > > Source/SplitReader.  Writing a new production quality connector for
> Flink
> > > is very difficult today and requires a lot of detailed knowledge about
> > > Flink, event time progress, watermarking, idle shard detection, etc and
> > it
> > > would be good to move almost all of this type of code into Flink itself
> > and
> > > out of source implementations.  I also think this is totally doable and
> > I'm
> > > really excited to see this happening.
> > >
> > > I do have a couple of thoughts about the API and the implementation..
> > >
> > > In a perfect world there would be a single thread per Flink source
> > sub-task
> > > and no additional threads for SplitReaders -- but this assumes a world
> > > where you have true async IO APIs for the upstream systems (like Kafka
> > and
> > > Kinesis, S3, HDFS, etc).  If that world did exist the single thread
> could
> > > just sit in an efficient select() call waiting for new data to arrive
> on
> > > any Split.  That'd be awesome..
> > >
> > > But, that world doesn't exist and given that practical consideration I
> > > would think the next best implementation is going to be, in practice,
> > > probably a thread per SplitReader that does nothing but call the source
> > API
> > > and drop whatever it reads into a (blocking) queue -- as Aljoscha
> > mentioned
> > > (calling it N+1) and as we started to describe here:
> > >
> >
> https://docs.google.com/document/d/13HxfsucoN7aUls1FX202KFAVZ4IMLXEZWyRfZNd6WFM/edit#heading=h.lfi2u4fv9cxa
> > >
> > > I guess my point is that I think we should strive to move as much of
> > > something like the diagram referenced in the above doc into Flink
> itself
> > > and out of sources and simplify the SplitReader API as much as possible
> > as
> > > well.
> > >
> > > With the above in mind and with regard to the discussion about
> blocking,
> > > etc..  I'm not sure I agree with some of the discussion so far with
> > regard
> > > to this API design.  The calls to the upstream systems (kafka/kinesis)
> > are
> > > in fact going to be blocking calls.  So a simple API without the
> > constraint
> > > that the methods must be implemented in a non-blocking way seems better
> > to
> > > me from the point of view of somebody writing a new source
> > implementation.
> > > My concern is that if you force the implementer of the SplitReader
> > > interface to do so in a non-blocking way you're just going to make it
> > > harder to write those implementations.  Those calls to read the next
> bit
> > of
> > > data are going to be blocking calls with most known important sources
> --
> > at
> > > least Kafka/Kinesis/HDFS -- so I think maybe we should just deal with
> > that
> > > head on and work around it a higher level so the SplitReader interface
> > > stays super simple to implement.  This means we manage all the
> threading
> > in
> > > Flink core, the API stays pull-based, and the implementer is allowed to
> > > simply block until they have data to return.
> > >
> > > I maybe would change my mind about this if truly asynchronous APIs to
> the
> > > upstream source systems were likely to be available in the near future
> or
> > > are now and I'm just ignorant of it.  But even then the supporting code
> > in
> > > Flink to drive async and sync sources would be different and in fact
> they
> > > might just have different APIs altogether -- SplitReader vs
> > > AsyncSplitReader maybe.
> > >
> > > In the end I think playing with the implementation, across more than
> one
> > > source, and moving as much common code into Flink itself will reveal
> the
> > > best API of course.
> > >
> > > One other interesting note is that you need to preserve per-partition
> > > ordering so you have to take care with the implementation if it were to
> > be
> > > based on a thread pool and futures so as not to reorder the reads.
> > >
> > > Anyway, I'm thrilled to see this starting to move forward and I'd very
> > much
> > > like to help with the implementation wherever I can.  We're doing a
> > > simplified internal version of some of this at Lyft for just Kinesis
> > > because we need a solution for event time alignment in the very short
> > term
> > > but we'd like to immediately start helping to do this properly in Flink
> > > after that.  One of the end goals for us is event time alignment across
> > > heterogeneous sources.  Another is making it possible for non-expert
> > users
> > > to have a high probability of being able to write their own, correct,
> > > connectors.
> > >
> > > -Jamie
> > >
> > > On Thu, Nov 15, 2018 at 3:43 AM Aljoscha Krettek <al...@apache.org>
> > > wrote:
> > >
> > >> Hi,
> > >>
> > >> I thought I had sent this mail a while ago but I must have forgotten
> to
> > >> send it.
> > >>
> > >> There is another thing we should consider for splits: the range of
> > >> timestamps that it can contain. For example, the splits of a file
> source
> > >> would know what the minimum and maximum timestamp in the splits is,
> > >> roughly. For infinite splits, such as Kafka partitions, the minimum
> > would
> > >> be meaningful but the maximum would be +Inf. If the splits expose the
> > >> interval of time that they contain the readers, or the component that
> > >> manages the readers can make decisions about which splits to forward
> and
> > >> read first. And it can also influence the minimum watermark that a
> > reader
> > >> forwards: it should never emit a watermark if it knows there are
> splits
> > to
> > >> read that have a lower minimum timestamp. I think it should be as easy
> > as
> > >> adding a minimumTimestamp()/maximumTimestamp() method pair to the
> split
> > >> interface.
> > >>
> > >> Another thing we need to resolve is the actual reader interface. I see
> > >> there has been some good discussion but I don't know if we have
> > consensus.
> > >> We should try and see how specific sources could be implemented with
> the
> > >> new interface. For example, for Kafka I think we need to have N+1
> > threads
> > >> per task (where N is the number of splits that a task is reading
> from).
> > On
> > >> thread is responsible for reading from the splits. And each split has
> > its
> > >> own (internal) thread for reading from Kafka and putting messages in
> an
> > >> internal queue to pull from. This is similar to how the current Kafka
> > >> source is implemented, which has a separate fetcher thread. The reason
> > for
> > >> this split is that we always need to try reading from Kafka to keep
> the
> > >> throughput up. In the current implementation the internal queue (or
> > >> handover) limits the read rate of the reader threads.
> > >>
> > >> @Thomas, what do you think this would look like for Kinesis?
> > >>
> > >> Best,
> > >> Aljoscha
> > >>
> > >>> On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com> wrote:
> > >>>
> > >>> Hi Piotrek,
> > >>>
> > >>> Thanks a lot for the detailed reply. All makes sense to me.
> > >>>
> > >>> WRT the confusion between advance() / getCurrent(), do you think it
> > would
> > >>> help if we combine them and have something like:
> > >>>
> > >>> CompletableFuture<T> getNext();
> > >>> long getWatermark();
> > >>> long getCurrentTimestamp();
> > >>>
> > >>> Cheers,
> > >>>
> > >>> Jiangjie (Becket) Qin
> > >>>
> > >>> On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <
> > piotr@data-artisans.com>
> > >>> wrote:
> > >>>
> > >>>> Hi,
> > >>>>
> > >>>> Thanks again for the detailed answer :) Sorry for responding with a
> > >> delay.
> > >>>>
> > >>>>> Completely agree that in pattern 2, having a callback is necessary
> > for
> > >>>> that
> > >>>>> single thread outside of the connectors. And the connectors MUST
> have
> > >>>>> internal threads.
> > >>>>
> > >>>> Yes, this thread will have to exists somewhere. In pattern 2 it
> exists
> > >> in
> > >>>> the connector (at least from the perspective of the Flink execution
> > >>>> engine). In pattern 1 it exists inside the Flink execution engine.
> > With
> > >>>> completely blocking connectors, like simple reading from files, both
> > of
> > >>>> those approaches are basically the same. The difference is when user
> > >>>> implementing Flink source is already working with a non blocking
> code
> > >> with
> > >>>> some internal threads. In this case, pattern 1 would result in
> "double
> > >>>> thread wrapping”, while pattern 2 would allow to skip one layer of
> > >>>> indirection.
> > >>>>
> > >>>>> If we go that way, we should have something like "void
> > >>>>> poll(Callback) / void advance(callback)". I am curious how would
> > >>>>> CompletableFuture work here, though. If 10 readers returns 10
> > >> completable
> > >>>>> futures, will there be 10 additional threads (so 20 threads in
> total)
> > >>>>> blocking waiting on them? Or will there be a single thread busy
> loop
> > >>>>> checking around?
> > >>>>
> > >>>> To be honest, I haven’t thought this completely through and I
> haven’t
> > >>>> tested/POC’ed it. Having said that, I can think of at least couple
> of
> > >>>> solutions. First is something like this:
> > >>>>
> > >>>>
> > >>>>
> > >>
> >
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> > >>>> <
> > >>>>
> > >>
> >
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> > >>>>>
> > >>>>
> > >>>> Line:
> > >>>>
> > >>>>                               `blocked = split.process();`
> > >>>>
> > >>>> Is where the execution goes into to the task/sources. This is where
> > the
> > >>>> returned future is handled:
> > >>>>
> > >>>>                               blocked.addListener(() -> {
> > >>>>                                   blockedSplits.remove(split);
> > >>>>                                   // reset the level priority to
> > >> prevent
> > >>>> previously-blocked splits from starving existing splits
> > >>>>                                   split.resetLevelPriority();
> > >>>>                                   waitingSplits.offer(split);
> > >>>>                               }, executor);
> > >>>>
> > >>>> Fundamentally callbacks and Futures are more or less interchangeable
> > You
> > >>>> can always wrap one into another (creating a callback that
> completes a
> > >>>> future and attach a callback once future completes). In this case
> the
> > >>>> difference for me is mostly:
> > >>>> - api with passing callback allows the callback to be fired multiple
> > >> times
> > >>>> and to fire it even if the connector is not blocked. This is what I
> > >> meant
> > >>>> by saying that api `CompletableFuture<?> isBlocked()` is a bit
> > simpler.
> > >>>> Connector can only return either “I’m not blocked” or “I’m blocked
> > and I
> > >>>> will tell you only once when I’m not blocked anymore”.
> > >>>>
> > >>>> But this is not the most important thing for me here. For me
> important
> > >>>> thing is to try our best to make Flink task’s control and execution
> > >> single
> > >>>> threaded. For that both callback and future APIs should work the
> same.
> > >>>>
> > >>>>> WRT pattern 1, a single blocking take() API should just work. The
> > good
> > >>>>> thing is that a blocking read API is usually simpler to implement.
> > >>>>
> > >>>> Yes, they are easier to implement (especially if you are not the one
> > >> that
> > >>>> have to deal with the additional threading required around them ;)
> ).
> > >> But
> > >>>> to answer this issue, if we choose pattern 2, we can always provide
> a
> > >>>> proxy/wrapper that would using the internal thread implement the
> > >>>> non-blocking API while exposing blocking API to the user. It would
> > >>>> implement pattern 2 for the user exposing to him pattern 1. In other
> > >> words
> > >>>> implementing pattern 1 in pattern 2 paradigm, while making it
> possible
> > >> to
> > >>>> implement pure pattern 2 connectors.
> > >>>>
> > >>>>> BTW, one thing I am also trying to avoid is pushing users to
> perform
> > IO
> > >>>> in
> > >>>>> a method like "isBlocked()". If the method is expected to fetch
> > records
> > >>>>> (even if not returning them), naming it something more explicit
> would
> > >>>> help
> > >>>>> avoid confusion.
> > >>>>
> > >>>> If we choose so, we could rework it into something like:
> > >>>>
> > >>>> CompletableFuture<?> advance()
> > >>>> T getCurrent();
> > >>>> Watermark getCurrentWatermark()
> > >>>>
> > >>>> But as I wrote before, this is more confusing to me for the exact
> > >> reasons
> > >>>> you mentioned :) I would be confused what should be done in
> > `adanvce()`
> > >> and
> > >>>> what in `getCurrent()`. However, again this naming issue is not that
> > >>>> important to me and probably is matter of taste/personal
> preferences.
> > >>>>
> > >>>> Piotrek
> > >>>>
> > >>>>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com> wrote:
> > >>>>>
> > >>>>> Hi Piotrek,
> > >>>>>
> > >>>>> Thanks for the explanation. We are probably talking about the same
> > >> thing
> > >>>>> but in different ways. To clarify a little bit, I think there are
> two
> > >>>>> patterns to read from a connector.
> > >>>>>
> > >>>>> Pattern 1: Thread-less connector with a blocking read API. Outside
> of
> > >> the
> > >>>>> connector, there is one IO thread per reader, doing blocking read.
> An
> > >>>>> additional thread will interact with all the IO threads.
> > >>>>> Pattern 2: Connector with internal thread(s) and non-blocking API.
> > >>>> Outside
> > >>>>> of the connector, there is one thread for ALL readers, doing IO
> > relying
> > >>>> on
> > >>>>> notification callbacks in the reader.
> > >>>>>
> > >>>>> In both patterns, there must be at least one thread per connector,
> > >> either
> > >>>>> inside (created by connector writers) or outside (created by Flink)
> > of
> > >>>> the
> > >>>>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total,
> to
> > >> make
> > >>>>> sure that 1 thread is fully non-blocking.
> > >>>>>
> > >>>>>> Btw, I don’t know if you understand my point. Having only `poll()`
> > and
> > >>>>> `take()` is not enough for single threaded task. If our source
> > >> interface
> > >>>>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
> > >>>>> isBlocked(),`, there is no way to implement single threaded task
> that
> > >>>> both
> > >>>>> reads the data from the source connector and can also react to
> system
> > >>>>> events. Ok, non >blocking `poll()` would allow that, but with busy
> > >>>> looping.
> > >>>>>
> > >>>>> Completely agree that in pattern 2, having a callback is necessary
> > for
> > >>>> that
> > >>>>> single thread outside of the connectors. And the connectors MUST
> have
> > >>>>> internal threads. If we go that way, we should have something like
> > >> "void
> > >>>>> poll(Callback) / void advance(callback)". I am curious how would
> > >>>>> CompletableFuture work here, though. If 10 readers returns 10
> > >> completable
> > >>>>> futures, will there be 10 additional threads (so 20 threads in
> total)
> > >>>>> blocking waiting on them? Or will there be a single thread busy
> loop
> > >>>>> checking around?
> > >>>>>
> > >>>>> WRT pattern 1, a single blocking take() API should just work. The
> > good
> > >>>>> thing is that a blocking read API is usually simpler to implement.
> An
> > >>>>> additional non-blocking "T poll()" method here is indeed optional
> and
> > >>>> could
> > >>>>> be used in cases like Flink does not want the thread to block
> > forever.
> > >>>> They
> > >>>>> can also be combined to have a "T poll(Timeout)", which is exactly
> > what
> > >>>>> KafkaConsumer did.
> > >>>>>
> > >>>>> It sounds that you are proposing pattern 2 with something similar
> to
> > >> NIO2
> > >>>>> AsynchronousByteChannel[1]. That API would work, except that the
> > >>>> signature
> > >>>>> returning future seems not necessary. If that is the case, a minor
> > >> change
> > >>>>> on the current FLIP proposal to have "void advance(callback)"
> should
> > >>>> work.
> > >>>>> And this means the connectors MUST have their internal threads.
> > >>>>>
> > >>>>> BTW, one thing I am also trying to avoid is pushing users to
> perform
> > IO
> > >>>> in
> > >>>>> a method like "isBlocked()". If the method is expected to fetch
> > records
> > >>>>> (even if not returning them), naming it something more explicit
> would
> > >>>> help
> > >>>>> avoid confusion.
> > >>>>>
> > >>>>> Thanks,
> > >>>>>
> > >>>>> Jiangjie (Becket) Qin
> > >>>>>
> > >>>>> [1]
> > >>>>>
> > >>>>
> > >>
> >
> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
> > >>>>>
> > >>>>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <
> > >> piotr@data-artisans.com>
> > >>>>> wrote:
> > >>>>>
> > >>>>>> Hi
> > >>>>>>
> > >>>>>> Good point with select/epoll, however I do not see how they
> couldn’t
> > >> be
> > >>>>>> with Flink if we would like single task in Flink to be
> > single-threaded
> > >>>> (and
> > >>>>>> I believe we should pursue this goal). If your connector blocks on
> > >>>>>> `select`, then it can not process/handle control messages from
> > Flink,
> > >>>> like
> > >>>>>> checkpoints, releasing resources and potentially output flushes.
> > This
> > >>>> would
> > >>>>>> require tight integration between connector and Flink’s main event
> > >>>>>> loop/selects/etc.
> > >>>>>>
> > >>>>>> Looking at it from other perspective. Let’s assume that we have a
> > >>>>>> connector implemented on top of `select`/`epoll`. In order to
> > >> integrate
> > >>>> it
> > >>>>>> with Flink’s checkpointing/flushes/resource releasing it will have
> > to
> > >> be
> > >>>>>> executed in separate thread one way or another. At least if our
> API
> > >> will
> > >>>>>> enforce/encourage non blocking implementations with some kind of
> > >>>>>> notifications (`isBlocked()` or `notify()` callback), some
> > connectors
> > >>>> might
> > >>>>>> skip one layer of wapping threads.
> > >>>>>>
> > >>>>>> Btw, I don’t know if you understand my point. Having only `poll()`
> > and
> > >>>>>> `take()` is not enough for single threaded task. If our source
> > >> interface
> > >>>>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
> > >>>>>> isBlocked(),`, there is no way to implement single threaded task
> > that
> > >>>> both
> > >>>>>> reads the data from the source connector and can also react to
> > system
> > >>>>>> events. Ok, non blocking `poll()` would allow that, but with busy
> > >>>> looping.
> > >>>>>>
> > >>>>>> Piotrek
> > >>>>>>
> > >>>>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com>
> wrote:
> > >>>>>>>
> > >>>>>>> Hi Piotrek,
> > >>>>>>>
> > >>>>>>>> But I don’t see a reason why we should expose both blocking
> > `take()`
> > >>>> and
> > >>>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
> > >>>> engine
> > >>>>>> or
> > >>>>>>> connector) would have to do the same busy
> > >>>>>>>> looping anyway and I think it would be better to have a simpler
> > >>>>>> connector
> > >>>>>>> API (that would solve our problems) and force connectors to
> comply
> > >> one
> > >>>>>> way
> > >>>>>>> or another.
> > >>>>>>>
> > >>>>>>> If we let the block happen inside the connector, the blocking
> does
> > >> not
> > >>>>>> have
> > >>>>>>> to be a busy loop. For example, to do the block waiting
> > efficiently,
> > >>>> the
> > >>>>>>> connector can use java NIO selector().select which relies on OS
> > >> syscall
> > >>>>>>> like epoll[1] instead of busy looping. But if Flink engine blocks
> > >>>> outside
> > >>>>>>> the connector, it pretty much has to do the busy loop. So if
> there
> > is
> > >>>>>> only
> > >>>>>>> one API to get the element, a blocking getNextElement() makes
> more
> > >>>> sense.
> > >>>>>>> In any case, we should avoid ambiguity. It has to be crystal
> clear
> > >>>> about
> > >>>>>>> whether a method is expected to be blocking or non-blocking.
> > >> Otherwise
> > >>>> it
> > >>>>>>> would be very difficult for Flink engine to do the right thing
> with
> > >> the
> > >>>>>>> connectors. At the first glance at getCurrent(), the expected
> > >> behavior
> > >>>> is
> > >>>>>>> not quite clear.
> > >>>>>>>
> > >>>>>>> That said, I do agree that functionality wise, poll() and take()
> > kind
> > >>>> of
> > >>>>>>> overlap. But they are actually not quite different from
> > >>>>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the only
> > >>>>>>> difference is that poll() also returns the next record if it is
> > >>>>>> available.
> > >>>>>>> But I agree that the isBlocked() + getNextElement() is more
> > flexible
> > >> as
> > >>>>>>> users can just check the record availability, but not fetch the
> > next
> > >>>>>>> element.
> > >>>>>>>
> > >>>>>>>> In case of thread-less readers with only non-blocking
> > `queue.poll()`
> > >>>> (is
> > >>>>>>> that really a thing? I can not think about a real implementation
> > that
> > >>>>>>> enforces such constraints)
> > >>>>>>> Right, it is pretty much a syntax sugar to allow user combine the
> > >>>>>>> check-and-take into one method. It could be achieved with
> > >> isBlocked() +
> > >>>>>>> getNextElement().
> > >>>>>>>
> > >>>>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
> > >>>>>>>
> > >>>>>>> Thanks,
> > >>>>>>>
> > >>>>>>> Jiangjie (Becket) Qin
> > >>>>>>>
> > >>>>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
> > >>>> piotr@data-artisans.com>
> > >>>>>>> wrote:
> > >>>>>>>
> > >>>>>>>> Hi Becket,
> > >>>>>>>>
> > >>>>>>>> With my proposal, both of your examples would have to be solved
> by
> > >> the
> > >>>>>>>> connector and solution to both problems would be the same:
> > >>>>>>>>
> > >>>>>>>> Pretend that connector is never blocked (`isBlocked() { return
> > >>>>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking
> > >> fashion
> > >>>>>> (or
> > >>>>>>>> semi blocking with return of control from time to time to allow
> > for
> > >>>>>>>> checkpointing, network flushing and other resource management
> > things
> > >>>> to
> > >>>>>>>> happen in the same main thread). In other words, exactly how you
> > >> would
> > >>>>>>>> implement `take()` method or how the same source connector would
> > be
> > >>>>>>>> implemented NOW with current source interface. The difference
> with
> > >>>>>> current
> > >>>>>>>> interface would be only that main loop would be outside of the
> > >>>>>> connector,
> > >>>>>>>> and instead of periodically releasing checkpointing lock,
> > >> periodically
> > >>>>>>>> `return null;` or `return Optional.empty();` from
> > >> `getNextElement()`.
> > >>>>>>>>
> > >>>>>>>> In case of thread-less readers with only non-blocking
> > `queue.poll()`
> > >>>> (is
> > >>>>>>>> that really a thing? I can not think about a real implementation
> > >> that
> > >>>>>>>> enforces such constraints), we could provide a wrapper that
> hides
> > >> the
> > >>>>>> busy
> > >>>>>>>> looping. The same applies how to solve forever blocking readers
> -
> > we
> > >>>>>> could
> > >>>>>>>> provider another wrapper running the connector in separate
> thread.
> > >>>>>>>>
> > >>>>>>>> But I don’t see a reason why we should expose both blocking
> > `take()`
> > >>>> and
> > >>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone
> (Flink
> > >>>>>> engine or
> > >>>>>>>> connector) would have to do the same busy looping anyway and I
> > think
> > >>>> it
> > >>>>>>>> would be better to have a simpler connector API (that would
> solve
> > >> our
> > >>>>>>>> problems) and force connectors to comply one way or another.
> > >>>>>>>>
> > >>>>>>>> Piotrek
> > >>>>>>>>
> > >>>>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com>
> > wrote:
> > >>>>>>>>>
> > >>>>>>>>> Hi Piotr,
> > >>>>>>>>>
> > >>>>>>>>> I might have misunderstood you proposal. But let me try to
> > explain
> > >> my
> > >>>>>>>>> concern. I am thinking about the following case:
> > >>>>>>>>> 1. a reader has the following two interfaces,
> > >>>>>>>>> boolean isBlocked()
> > >>>>>>>>> T getNextElement()
> > >>>>>>>>> 2. the implementation of getNextElement() is non-blocking.
> > >>>>>>>>> 3. The reader is thread-less, i.e. it does not have any
> internal
> > >>>>>> thread.
> > >>>>>>>>> For example, it might just delegate the getNextElement() to a
> > >>>>>>>> queue.poll(),
> > >>>>>>>>> and isBlocked() is just queue.isEmpty().
> > >>>>>>>>>
> > >>>>>>>>> How can Flink efficiently implement a blocking reading behavior
> > >> with
> > >>>>>> this
> > >>>>>>>>> reader? Either a tight loop or a backoff interval is needed.
> > >> Neither
> > >>>> of
> > >>>>>>>>> them is ideal.
> > >>>>>>>>>
> > >>>>>>>>> Now let's say in the reader mentioned above implements a
> blocking
> > >>>>>>>>> getNextElement() method. Because there is no internal thread in
> > the
> > >>>>>>>> reader,
> > >>>>>>>>> after isBlocked() returns false. Flink will still have to loop
> on
> > >>>>>>>>> isBlocked() to check whether the next record is available. If
> the
> > >>>> next
> > >>>>>>>>> record reaches after 10 min, it is a tight loop for 10 min. You
> > >> have
> > >>>>>>>>> probably noticed that in this case, even isBlocked() returns a
> > >>>> future,
> > >>>>>>>> that
> > >>>>>>>>> future() will not be completed if Flink does not call some
> method
> > >>>> from
> > >>>>>>>> the
> > >>>>>>>>> reader, because the reader has no internal thread to complete
> > that
> > >>>>>> future
> > >>>>>>>>> by itself.
> > >>>>>>>>>
> > >>>>>>>>> Due to the above reasons, a blocking take() API would allow
> Flink
> > >> to
> > >>>>>> have
> > >>>>>>>>> an efficient way to read from a reader. There are many ways to
> > wake
> > >>>> up
> > >>>>>>>> the
> > >>>>>>>>> blocking thread when checkpointing is needed depending on the
> > >>>>>>>>> implementation. But I think the poll()/take() API would also
> work
> > >> in
> > >>>>>> that
> > >>>>>>>>> case.
> > >>>>>>>>>
> > >>>>>>>>> Thanks,
> > >>>>>>>>>
> > >>>>>>>>> Jiangjie (Becket) Qin
> > >>>>>>>>>
> > >>>>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
> > >>>> piotr@data-artisans.com
> > >>>>>>>
> > >>>>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> Hi,
> > >>>>>>>>>>
> > >>>>>>>>>> a)
> > >>>>>>>>>>
> > >>>>>>>>>>> BTW, regarding the isBlock() method, I have a few more
> > questions.
> > >>>> 21,
> > >>>>>>>> Is
> > >>>>>>>>>> a method isReady() with boolean as a return value
> > >>>>>>>>>>> equivalent? Personally I found it is a little bit confusing
> in
> > >> what
> > >>>>>> is
> > >>>>>>>>>> supposed to be returned when the future is completed. 22. if
> > >>>>>>>>>>> the implementation of isBlocked() is optional, how do the
> > callers
> > >>>>>> know
> > >>>>>>>>>> whether the method is properly implemented or not?
> > >>>>>>>>>>> Does not implemented mean it always return a completed
> future?
> > >>>>>>>>>>
> > >>>>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an
> equivalent
> > >> to
> > >>>>>>>>>> `boolean hasNext()` which in case of “false” provides some
> kind
> > >> of a
> > >>>>>>>>>> listener/callback that notifies about presence of next
> element.
> > >>>> There
> > >>>>>>>> are
> > >>>>>>>>>> some minor details, like `CompletableFuture<?>` has a minimal
> > two
> > >>>>>> state
> > >>>>>>>>>> logic:
> > >>>>>>>>>>
> > >>>>>>>>>> 1. Future is completed - we have more data
> > >>>>>>>>>> 2. Future not yet completed - we don’t have data now, but we
> > >>>> might/we
> > >>>>>>>> will
> > >>>>>>>>>> have in the future
> > >>>>>>>>>>
> > >>>>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit
> more
> > >>>>>>>>>> complicated/dispersed and can lead/encourage `notify()` spam.
> > >>>>>>>>>>
> > >>>>>>>>>> b)
> > >>>>>>>>>>
> > >>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method
> like
> > >>>>>>>> `getNext`
> > >>>>>>>>>> the `getNext` would need return a
> > >>>>>>>>>>> `ElementWithTimestamp` because some sources want to add
> > timestamp
> > >>>> to
> > >>>>>>>>>> every element. IMO, this is not so memory friendly
> > >>>>>>>>>>> so I prefer this design.
> > >>>>>>>>>>
> > >>>>>>>>>> Guowei I don’t quite understand this. Could you elaborate why
> > >>>> having a
> > >>>>>>>>>> separate `advance()` help?
> > >>>>>>>>>>
> > >>>>>>>>>> c)
> > >>>>>>>>>>
> > >>>>>>>>>> Regarding advance/poll/take. What’s the value of having two
> > >> separate
> > >>>>>>>>>> methods: poll and take? Which one of them should be called and
> > >> which
> > >>>>>>>>>> implemented? What’s the benefit of having those methods
> compared
> > >> to
> > >>>>>>>> having
> > >>>>>>>>>> a one single method `getNextElement()` (or `pollElement() or
> > >>>> whatever
> > >>>>>> we
> > >>>>>>>>>> name it) with following contract:
> > >>>>>>>>>>
> > >>>>>>>>>> CompletableFuture<?> isBlocked();
> > >>>>>>>>>>
> > >>>>>>>>>> /**
> > >>>>>>>>>> Return next element - will be called only if `isBlocked()` is
> > >>>>>> completed.
> > >>>>>>>>>> Try to implement it in non blocking fashion, but if that’s
> > >>>> impossible
> > >>>>>> or
> > >>>>>>>>>> you just don’t need the effort, you can block in this method.
> > >>>>>>>>>> */
> > >>>>>>>>>> T getNextElement();
> > >>>>>>>>>>
> > >>>>>>>>>> I mean, if the connector is implemented non-blockingly, Flink
> > >> should
> > >>>>>> use
> > >>>>>>>>>> it that way. If it’s not, then `poll()` will `throw new
> > >>>>>>>>>> NotImplementedException()`. Implementing both of them and
> > >> providing
> > >>>>>>>> both of
> > >>>>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them
> > into
> > >> a
> > >>>>>>>> single
> > >>>>>>>>>> method call that should preferably (but not necessarily need
> to)
> > >> be
> > >>>>>>>>>> non-blocking? It’s not like we are implementing general
> purpose
> > >>>>>> `Queue`,
> > >>>>>>>>>> which users might want to call either of `poll` or `take`. We
> > >> would
> > >>>>>>>> always
> > >>>>>>>>>> prefer to call `poll`, but if it’s blocking, then still we
> have
> > no
> > >>>>>>>> choice,
> > >>>>>>>>>> but to call it and block on it.
> > >>>>>>>>>>
> > >>>>>>>>>> d)
> > >>>>>>>>>>
> > >>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source
> > is
> > >>>> very
> > >>>>>>>>>>> important. But in addition to `Future/poll`, there may be
> > another
> > >>>> way
> > >>>>>>>> to
> > >>>>>>>>>>> achieve this. I think it may be not very memory friendly if
> > every
> > >>>>>>>> advance
> > >>>>>>>>>>> call return a Future.
> > >>>>>>>>>>
> > >>>>>>>>>> I didn’t want to mention this, to not clog my initial
> proposal,
> > >> but
> > >>>>>>>> there
> > >>>>>>>>>> is a simple solution for the problem:
> > >>>>>>>>>>
> > >>>>>>>>>> public interface SplitReader {
> > >>>>>>>>>>
> > >>>>>>>>>> (…)
> > >>>>>>>>>>
> > >>>>>>>>>> CompletableFuture<?> NOT_BLOCKED =
> > >>>>>>>>>> CompletableFuture.completedFuture(null);
> > >>>>>>>>>>
> > >>>>>>>>>> /**
> > >>>>>>>>>> * Returns a future that will be completed when the page source
> > >>>>>>>> becomes
> > >>>>>>>>>> * unblocked.  If the page source is not blocked, this method
> > >>>> should
> > >>>>>>>>>> return
> > >>>>>>>>>> * {@code NOT_BLOCKED}.
> > >>>>>>>>>> */
> > >>>>>>>>>> default CompletableFuture<?> isBlocked()
> > >>>>>>>>>> {
> > >>>>>>>>>>    return NOT_BLOCKED;
> > >>>>>>>>>> }
> > >>>>>>>>>>
> > >>>>>>>>>> If we are blocked and we are waiting for the IO, then
> creating a
> > >> new
> > >>>>>>>>>> Future is non-issue. Under full throttle/throughput and not
> > >> blocked
> > >>>>>>>> sources
> > >>>>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve
> the
> > >>>>>>>> problem.
> > >>>>>>>>>>
> > >>>>>>>>>> One more remark, non-blocking sources might be a necessity in
> a
> > >>>> single
> > >>>>>>>>>> threaded model without a checkpointing lock. (Currently when
> > >> sources
> > >>>>>> are
> > >>>>>>>>>> blocked, they can release checkpointing lock and re-acquire it
> > >> again
> > >>>>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for
> > >> checkpoints
> > >>>> to
> > >>>>>>>>>> happen when source is idling. In that case either `notify()`
> or
> > my
> > >>>>>>>> proposed
> > >>>>>>>>>> `isBlocked()` would allow to avoid busy-looping.
> > >>>>>>>>>>
> > >>>>>>>>>> Piotrek
> > >>>>>>>>>>
> > >>>>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com>
> > >> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>> Hi Thomas,
> > >>>>>>>>>>>
> > >>>>>>>>>>> The iterator-like API was also the first thing that came to
> me.
> > >> But
> > >>>>>> it
> > >>>>>>>>>>> seems a little confusing that hasNext() does not mean "the
> > stream
> > >>>> has
> > >>>>>>>> not
> > >>>>>>>>>>> ended", but means "the next record is ready", which is
> > >> repurposing
> > >>>>>> the
> > >>>>>>>>>> well
> > >>>>>>>>>>> known meaning of hasNext(). If we follow the hasNext()/next()
> > >>>>>> pattern,
> > >>>>>>>> an
> > >>>>>>>>>>> additional isNextReady() method to indicate whether the next
> > >> record
> > >>>>>> is
> > >>>>>>>>>>> ready seems more intuitive to me.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Similarly, in poll()/take() pattern, another method of
> isDone()
> > >> is
> > >>>>>>>> needed
> > >>>>>>>>>>> to indicate whether the stream has ended or not.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
> > >>>>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
> > >>>>>>>> implementation.
> > >>>>>>>>>>> When I am implementing a reader, I could have a couple of
> > >> choices:
> > >>>>>>>>>>>
> > >>>>>>>>>>> - A thread-less reader that does not have any internal
> thread.
> > >>>>>>>>>>> - When poll() is called, the same calling thread will
> perform a
> > >>>> bunch
> > >>>>>>>>>> of
> > >>>>>>>>>>> IO asynchronously.
> > >>>>>>>>>>> - When take() is called, the same calling thread will
> perform a
> > >>>>>>>>>> bunch
> > >>>>>>>>>>> of IO and wait until the record is ready.
> > >>>>>>>>>>> - A reader with internal threads performing network IO and
> put
> > >>>>>> records
> > >>>>>>>>>>> into a buffer.
> > >>>>>>>>>>> - When poll() is called, the calling thread simply reads from
> > >> the
> > >>>>>>>>>>> buffer and return empty result immediately if there is no
> > >> record.
> > >>>>>>>>>>> - When take() is called, the calling thread reads from the
> > >> buffer
> > >>>>>>>>>> and
> > >>>>>>>>>>> block waiting if the buffer is empty.
> > >>>>>>>>>>>
> > >>>>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady()
> API,
> > >> it
> > >>>> is
> > >>>>>>>>>> less
> > >>>>>>>>>>> intuitive for the reader developers to write the thread-less
> > >>>> pattern.
> > >>>>>>>>>>> Although technically speaking one can still do the
> asynchronous
> > >> IO
> > >>>> to
> > >>>>>>>>>>> prepare the record in isNextReady(). But it is inexplicit and
> > >> seems
> > >>>>>>>>>>> somewhat hacky.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thanks,
> > >>>>>>>>>>>
> > >>>>>>>>>>> Jiangjie (Becket) Qin
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org>
> > >>>> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>>> Couple more points regarding discovery:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> The proposal mentions that discovery could be outside the
> > >>>> execution
> > >>>>>>>>>> graph.
> > >>>>>>>>>>>> Today, discovered partitions/shards are checkpointed. I
> > believe
> > >>>> that
> > >>>>>>>>>> will
> > >>>>>>>>>>>> also need to be the case in the future, even when discovery
> > and
> > >>>>>>>> reading
> > >>>>>>>>>> are
> > >>>>>>>>>>>> split between different tasks.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> For cases such as resharding of a Kinesis stream, the
> > >> relationship
> > >>>>>>>>>> between
> > >>>>>>>>>>>> splits needs to be considered. Splits cannot be randomly
> > >>>> distributed
> > >>>>>>>>>> over
> > >>>>>>>>>>>> readers in certain situations. An example was mentioned
> here:
> > >>>>>>>>>>>>
> > >> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Thomas
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <thw@apache.org
> >
> > >>>> wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>> Thanks for getting the ball rolling on this!
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Can the number of splits decrease? Yes, splits can be
> closed
> > >> and
> > >>>> go
> > >>>>>>>>>> away.
> > >>>>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing
> > shards
> > >>>>>> will
> > >>>>>>>> be
> > >>>>>>>>>>>>> closed and replaced with a new shard).
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive
> > approach
> > >>>>>> would
> > >>>>>>>>>> be
> > >>>>>>>>>>>>> the thread-less IO model (pull based, non-blocking, caller
> > >>>>>> retrieves
> > >>>>>>>>>> new
> > >>>>>>>>>>>>> records when available). The current Kinesis API requires
> the
> > >> use
> > >>>>>> of
> > >>>>>>>>>>>>> threads. But that can be internal to the split reader and
> > does
> > >>>> not
> > >>>>>>>> need
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>> be a source API concern. In fact, that's what we are
> working
> > on
> > >>>>>> right
> > >>>>>>>>>> now
> > >>>>>>>>>>>>> as improvement to the existing consumer: Each shard
> consumer
> > >>>> thread
> > >>>>>>>>>> will
> > >>>>>>>>>>>>> push to a queue, the consumer main thread will poll the
> > >> queue(s).
> > >>>>>> It
> > >>>>>>>> is
> > >>>>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> The proposed SplitReader interface would fit the
> thread-less
> > IO
> > >>>>>>>> model.
> > >>>>>>>>>>>>> Similar to an iterator, we find out if there is a new
> element
> > >>>>>>>> (hasNext)
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>> if so, move to it (next()). Separate calls deliver the meta
> > >>>>>>>> information
> > >>>>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer a
> > >>>> timeout
> > >>>>>>>>>>>> option,
> > >>>>>>>>>>>>> so that the caller does not end up in a busy wait. On the
> > other
> > >>>>>>>> hand, a
> > >>>>>>>>>>>>> caller processing multiple splits may want to cycle through
> > >> fast,
> > >>>>>> to
> > >>>>>>>>>>>>> process elements of other splits as soon as they become
> > >>>> available.
> > >>>>>>>> The
> > >>>>>>>>>>>> nice
> > >>>>>>>>>>>>> thing is that this "split merge" logic can now live in
> Flink
> > >> and
> > >>>> be
> > >>>>>>>>>>>>> optimized and shared between different sources.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>> Thomas
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <
> > guowei.mgw@gmail.com
> > >>>
> > >>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>> Thanks Aljoscha for this FLIP.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
> > source
> > >> is
> > >>>>>>>> very
> > >>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
> > >>>> another
> > >>>>>>>> way
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly
> if
> > >>>> every
> > >>>>>>>>>>>> advance
> > >>>>>>>>>>>>>> call return a Future.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> public interface Listener {
> > >>>>>>>>>>>>>> public void notify();
> > >>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> public interface SplitReader() {
> > >>>>>>>>>>>>>> /**
> > >>>>>>>>>>>>>> * When there is no element temporarily, this will return
> > >>>> false.
> > >>>>>>>>>>>>>> * When elements is available again splitReader can call
> > >>>>>>>>>>>>>> listener.notify()
> > >>>>>>>>>>>>>> * In addition the frame would check `advance`
> periodically .
> > >>>>>>>>>>>>>> * Of course advance can always return true and ignore the
> > >>>>>>>>>> listener
> > >>>>>>>>>>>>>> argument for simplicity.
> > >>>>>>>>>>>>>> */
> > >>>>>>>>>>>>>> public boolean advance(Listener listener);
> > >>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create all
> > >> Splits
> > >>>>>> and
> > >>>>>>>>>> how
> > >>>>>>>>>>>>>> to create a SplitReader from a Split. But there is no
> > strategy
> > >>>> for
> > >>>>>>>> the
> > >>>>>>>>>>>> user
> > >>>>>>>>>>>>>> to choose how to assign the splits to the tasks. I think
> we
> > >>>> could
> > >>>>>>>> add
> > >>>>>>>>>> a
> > >>>>>>>>>>>>>> Enum to let user to choose.
> > >>>>>>>>>>>>>> /**
> > >>>>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
> > >>>>>>>>>>>>>> Location,
> > >>>>>>>>>>>>>> Workload,
> > >>>>>>>>>>>>>> Random,
> > >>>>>>>>>>>>>> Average
> > >>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>> */
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method
> > like
> > >>>>>>>>>> `getNext`
> > >>>>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp`
> > >> because
> > >>>>>>>> some
> > >>>>>>>>>>>>>> sources want to add timestamp to every element. IMO, this
> is
> > >> not
> > >>>>>> so
> > >>>>>>>>>>>> memory
> > >>>>>>>>>>>>>> friendly so I prefer this design.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Thanks
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
> > >>>> 下午6:08写道:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a
> > lot
> > >> of
> > >>>>>>>> other
> > >>>>>>>>>>>>>>> possible improvements. I have one proposal. Instead of
> > >> having a
> > >>>>>>>>>> method:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> boolean advance() throws IOException;
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> I would replace it with
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> /*
> > >>>>>>>>>>>>>>> * Return a future, which when completed means that source
> > has
> > >>>>>> more
> > >>>>>>>>>>>> data
> > >>>>>>>>>>>>>>> and getNext() will not block.
> > >>>>>>>>>>>>>>> * If you wish to use benefits of non blocking connectors,
> > >>>> please
> > >>>>>>>>>>>>>>> implement this method appropriately.
> > >>>>>>>>>>>>>>> */
> > >>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
> > >>>>>>>>>>>>>>>   return CompletableFuture.completedFuture(null);
> > >>>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Couple of arguments:
> > >>>>>>>>>>>>>>> 1. I don’t understand the division of work between
> > >> `advance()`
> > >>>>>> and
> > >>>>>>>>>>>>>>> `getCurrent()`. What should be done in which, especially
> > for
> > >>>>>>>>>> connectors
> > >>>>>>>>>>>>>>> that handle records in batches (like Kafka) and when
> should
> > >> you
> > >>>>>>>> call
> > >>>>>>>>>>>>>>> `advance` and when `getCurrent()`.
> > >>>>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will
> > allow
> > >>>> us
> > >>>>>> in
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>> future to have asynchronous/non blocking connectors and
> > more
> > >>>>>>>>>>>> efficiently
> > >>>>>>>>>>>>>>> handle large number of blocked threads, without busy
> > waiting.
> > >>>>>> While
> > >>>>>>>>>> at
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>> same time it doesn’t add much complexity, since naive
> > >> connector
> > >>>>>>>>>>>>>>> implementations can be always blocking.
> > >>>>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread
> pool
> > >> of
> > >>>>>> task
> > >>>>>>>>>>>>>>> executors, instead of one thread per task.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Piotrek
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
> > >>>> aljoscha@apache.org
> > >>>>>>>
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Hi All,
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> In order to finally get the ball rolling on the new
> source
> > >>>>>>>> interface
> > >>>>>>>>>>>>>>> that we have discussed for so long I finally created a
> > FLIP:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>
> > >>>>>>
> > >>>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
> > >>>> work/discussion
> > >>>>>>>>>> about
> > >>>>>>>>>>>>>>> adding per-partition watermark support to the Kinesis
> > source
> > >>>> and
> > >>>>>>>>>>>> because
> > >>>>>>>>>>>>>>> this would enable generic implementation of event-time
> > >>>> alignment
> > >>>>>>>> for
> > >>>>>>>>>>>> all
> > >>>>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
> > >>>> alignment
> > >>>>>>>>>> part,
> > >>>>>>>>>>>>>>> especially the part about information sharing between
> > >>>> operations
> > >>>>>>>> (I'm
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>>>> calling it state sharing because state has a special
> > meaning
> > >> in
> > >>>>>>>>>> Flink).
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Please discuss away!
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Aljoscha
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>
> > >>>>
> > >>
> > >>
> >
> >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Thomas Weise <th...@apache.org>.
@Aljoscha to address your question first: In the case of the Kinesis
consumer (with current Kinesis consumer API), there would also be N+1
threads. I have implemented a prototype similar to what is shown in Jamie's
document, where the thread ownership is similar to what you have done for
Kafka.

The equivalent of split reader manages its own thread and the "source main
thread" is responsible for emitting the data. The interface between the N
reader threads and the 1 emitter is a blocking queue per consumer thread.
The emitter can now control which queue to consume from based on the event
time progress.

This is akin to a "non-blocking" interface *between emitter and split
reader*. Emitter uses poll to retrieve records from the N queues (which
requires non-blocking interaction). The emitter is independent of the split
reader implementation, that part could live in Flink.

Regarding whether or not to assume that split readers always need a thread
and in addition that these reader threads should be managed by Flink: It
depends on the API of respective external systems and I would not bake that
assumption into Flink. Some client libraries manage their own threads (see
push based API like JMS and as I understand it may also apply to the new
fan-out Kinesis API:
https://docs.aws.amazon.com/streams/latest/dev/building-enhanced-consumers-api.html).
In such cases it would not make sense to layer another reader thread on
top. It may instead be better if Flink provides to the split reader the
queue/buffer to push records to.

The discussion so far has largely ignored the discovery aspect. There are
some important considerations such as ordering dependency of splits and
work rebalancing that may affect the split reader interface. Should we fork
this into a separate thread?

Thanks,
Thomas


On Fri, Nov 16, 2018 at 8:09 AM Piotr Nowojski <pi...@data-artisans.com>
wrote:

> Hi Jamie,
>
> As it was already covered with my discussion with Becket, there is an easy
> way to provide blocking API on top of non-blocking API. And yes we both
> agreed that blocking API is easier to implement by users.
>
> I also do not agree with respect to usefulness of non blocking API.
> Actually Kafka connector is the one that could be more efficient thanks to
> the removal of the one layer of threading.
>
> Piotrek
>
> > On 16 Nov 2018, at 02:21, Jamie Grier <jg...@lyft.com.INVALID> wrote:
> >
> > Thanks Aljoscha for getting this effort going!
> >
> > There's been plenty of discussion here already and I'll add my big +1 to
> > making this interface very simple to implement for a new
> > Source/SplitReader.  Writing a new production quality connector for Flink
> > is very difficult today and requires a lot of detailed knowledge about
> > Flink, event time progress, watermarking, idle shard detection, etc and
> it
> > would be good to move almost all of this type of code into Flink itself
> and
> > out of source implementations.  I also think this is totally doable and
> I'm
> > really excited to see this happening.
> >
> > I do have a couple of thoughts about the API and the implementation..
> >
> > In a perfect world there would be a single thread per Flink source
> sub-task
> > and no additional threads for SplitReaders -- but this assumes a world
> > where you have true async IO APIs for the upstream systems (like Kafka
> and
> > Kinesis, S3, HDFS, etc).  If that world did exist the single thread could
> > just sit in an efficient select() call waiting for new data to arrive on
> > any Split.  That'd be awesome..
> >
> > But, that world doesn't exist and given that practical consideration I
> > would think the next best implementation is going to be, in practice,
> > probably a thread per SplitReader that does nothing but call the source
> API
> > and drop whatever it reads into a (blocking) queue -- as Aljoscha
> mentioned
> > (calling it N+1) and as we started to describe here:
> >
> https://docs.google.com/document/d/13HxfsucoN7aUls1FX202KFAVZ4IMLXEZWyRfZNd6WFM/edit#heading=h.lfi2u4fv9cxa
> >
> > I guess my point is that I think we should strive to move as much of
> > something like the diagram referenced in the above doc into Flink itself
> > and out of sources and simplify the SplitReader API as much as possible
> as
> > well.
> >
> > With the above in mind and with regard to the discussion about blocking,
> > etc..  I'm not sure I agree with some of the discussion so far with
> regard
> > to this API design.  The calls to the upstream systems (kafka/kinesis)
> are
> > in fact going to be blocking calls.  So a simple API without the
> constraint
> > that the methods must be implemented in a non-blocking way seems better
> to
> > me from the point of view of somebody writing a new source
> implementation.
> > My concern is that if you force the implementer of the SplitReader
> > interface to do so in a non-blocking way you're just going to make it
> > harder to write those implementations.  Those calls to read the next bit
> of
> > data are going to be blocking calls with most known important sources --
> at
> > least Kafka/Kinesis/HDFS -- so I think maybe we should just deal with
> that
> > head on and work around it a higher level so the SplitReader interface
> > stays super simple to implement.  This means we manage all the threading
> in
> > Flink core, the API stays pull-based, and the implementer is allowed to
> > simply block until they have data to return.
> >
> > I maybe would change my mind about this if truly asynchronous APIs to the
> > upstream source systems were likely to be available in the near future or
> > are now and I'm just ignorant of it.  But even then the supporting code
> in
> > Flink to drive async and sync sources would be different and in fact they
> > might just have different APIs altogether -- SplitReader vs
> > AsyncSplitReader maybe.
> >
> > In the end I think playing with the implementation, across more than one
> > source, and moving as much common code into Flink itself will reveal the
> > best API of course.
> >
> > One other interesting note is that you need to preserve per-partition
> > ordering so you have to take care with the implementation if it were to
> be
> > based on a thread pool and futures so as not to reorder the reads.
> >
> > Anyway, I'm thrilled to see this starting to move forward and I'd very
> much
> > like to help with the implementation wherever I can.  We're doing a
> > simplified internal version of some of this at Lyft for just Kinesis
> > because we need a solution for event time alignment in the very short
> term
> > but we'd like to immediately start helping to do this properly in Flink
> > after that.  One of the end goals for us is event time alignment across
> > heterogeneous sources.  Another is making it possible for non-expert
> users
> > to have a high probability of being able to write their own, correct,
> > connectors.
> >
> > -Jamie
> >
> > On Thu, Nov 15, 2018 at 3:43 AM Aljoscha Krettek <al...@apache.org>
> > wrote:
> >
> >> Hi,
> >>
> >> I thought I had sent this mail a while ago but I must have forgotten to
> >> send it.
> >>
> >> There is another thing we should consider for splits: the range of
> >> timestamps that it can contain. For example, the splits of a file source
> >> would know what the minimum and maximum timestamp in the splits is,
> >> roughly. For infinite splits, such as Kafka partitions, the minimum
> would
> >> be meaningful but the maximum would be +Inf. If the splits expose the
> >> interval of time that they contain the readers, or the component that
> >> manages the readers can make decisions about which splits to forward and
> >> read first. And it can also influence the minimum watermark that a
> reader
> >> forwards: it should never emit a watermark if it knows there are splits
> to
> >> read that have a lower minimum timestamp. I think it should be as easy
> as
> >> adding a minimumTimestamp()/maximumTimestamp() method pair to the split
> >> interface.
> >>
> >> Another thing we need to resolve is the actual reader interface. I see
> >> there has been some good discussion but I don't know if we have
> consensus.
> >> We should try and see how specific sources could be implemented with the
> >> new interface. For example, for Kafka I think we need to have N+1
> threads
> >> per task (where N is the number of splits that a task is reading from).
> On
> >> thread is responsible for reading from the splits. And each split has
> its
> >> own (internal) thread for reading from Kafka and putting messages in an
> >> internal queue to pull from. This is similar to how the current Kafka
> >> source is implemented, which has a separate fetcher thread. The reason
> for
> >> this split is that we always need to try reading from Kafka to keep the
> >> throughput up. In the current implementation the internal queue (or
> >> handover) limits the read rate of the reader threads.
> >>
> >> @Thomas, what do you think this would look like for Kinesis?
> >>
> >> Best,
> >> Aljoscha
> >>
> >>> On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com> wrote:
> >>>
> >>> Hi Piotrek,
> >>>
> >>> Thanks a lot for the detailed reply. All makes sense to me.
> >>>
> >>> WRT the confusion between advance() / getCurrent(), do you think it
> would
> >>> help if we combine them and have something like:
> >>>
> >>> CompletableFuture<T> getNext();
> >>> long getWatermark();
> >>> long getCurrentTimestamp();
> >>>
> >>> Cheers,
> >>>
> >>> Jiangjie (Becket) Qin
> >>>
> >>> On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <
> piotr@data-artisans.com>
> >>> wrote:
> >>>
> >>>> Hi,
> >>>>
> >>>> Thanks again for the detailed answer :) Sorry for responding with a
> >> delay.
> >>>>
> >>>>> Completely agree that in pattern 2, having a callback is necessary
> for
> >>>> that
> >>>>> single thread outside of the connectors. And the connectors MUST have
> >>>>> internal threads.
> >>>>
> >>>> Yes, this thread will have to exists somewhere. In pattern 2 it exists
> >> in
> >>>> the connector (at least from the perspective of the Flink execution
> >>>> engine). In pattern 1 it exists inside the Flink execution engine.
> With
> >>>> completely blocking connectors, like simple reading from files, both
> of
> >>>> those approaches are basically the same. The difference is when user
> >>>> implementing Flink source is already working with a non blocking code
> >> with
> >>>> some internal threads. In this case, pattern 1 would result in "double
> >>>> thread wrapping”, while pattern 2 would allow to skip one layer of
> >>>> indirection.
> >>>>
> >>>>> If we go that way, we should have something like "void
> >>>>> poll(Callback) / void advance(callback)". I am curious how would
> >>>>> CompletableFuture work here, though. If 10 readers returns 10
> >> completable
> >>>>> futures, will there be 10 additional threads (so 20 threads in total)
> >>>>> blocking waiting on them? Or will there be a single thread busy loop
> >>>>> checking around?
> >>>>
> >>>> To be honest, I haven’t thought this completely through and I haven’t
> >>>> tested/POC’ed it. Having said that, I can think of at least couple of
> >>>> solutions. First is something like this:
> >>>>
> >>>>
> >>>>
> >>
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> >>>> <
> >>>>
> >>
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> >>>>>
> >>>>
> >>>> Line:
> >>>>
> >>>>                               `blocked = split.process();`
> >>>>
> >>>> Is where the execution goes into to the task/sources. This is where
> the
> >>>> returned future is handled:
> >>>>
> >>>>                               blocked.addListener(() -> {
> >>>>                                   blockedSplits.remove(split);
> >>>>                                   // reset the level priority to
> >> prevent
> >>>> previously-blocked splits from starving existing splits
> >>>>                                   split.resetLevelPriority();
> >>>>                                   waitingSplits.offer(split);
> >>>>                               }, executor);
> >>>>
> >>>> Fundamentally callbacks and Futures are more or less interchangeable
> You
> >>>> can always wrap one into another (creating a callback that completes a
> >>>> future and attach a callback once future completes). In this case the
> >>>> difference for me is mostly:
> >>>> - api with passing callback allows the callback to be fired multiple
> >> times
> >>>> and to fire it even if the connector is not blocked. This is what I
> >> meant
> >>>> by saying that api `CompletableFuture<?> isBlocked()` is a bit
> simpler.
> >>>> Connector can only return either “I’m not blocked” or “I’m blocked
> and I
> >>>> will tell you only once when I’m not blocked anymore”.
> >>>>
> >>>> But this is not the most important thing for me here. For me important
> >>>> thing is to try our best to make Flink task’s control and execution
> >> single
> >>>> threaded. For that both callback and future APIs should work the same.
> >>>>
> >>>>> WRT pattern 1, a single blocking take() API should just work. The
> good
> >>>>> thing is that a blocking read API is usually simpler to implement.
> >>>>
> >>>> Yes, they are easier to implement (especially if you are not the one
> >> that
> >>>> have to deal with the additional threading required around them ;) ).
> >> But
> >>>> to answer this issue, if we choose pattern 2, we can always provide a
> >>>> proxy/wrapper that would using the internal thread implement the
> >>>> non-blocking API while exposing blocking API to the user. It would
> >>>> implement pattern 2 for the user exposing to him pattern 1. In other
> >> words
> >>>> implementing pattern 1 in pattern 2 paradigm, while making it possible
> >> to
> >>>> implement pure pattern 2 connectors.
> >>>>
> >>>>> BTW, one thing I am also trying to avoid is pushing users to perform
> IO
> >>>> in
> >>>>> a method like "isBlocked()". If the method is expected to fetch
> records
> >>>>> (even if not returning them), naming it something more explicit would
> >>>> help
> >>>>> avoid confusion.
> >>>>
> >>>> If we choose so, we could rework it into something like:
> >>>>
> >>>> CompletableFuture<?> advance()
> >>>> T getCurrent();
> >>>> Watermark getCurrentWatermark()
> >>>>
> >>>> But as I wrote before, this is more confusing to me for the exact
> >> reasons
> >>>> you mentioned :) I would be confused what should be done in
> `adanvce()`
> >> and
> >>>> what in `getCurrent()`. However, again this naming issue is not that
> >>>> important to me and probably is matter of taste/personal preferences.
> >>>>
> >>>> Piotrek
> >>>>
> >>>>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com> wrote:
> >>>>>
> >>>>> Hi Piotrek,
> >>>>>
> >>>>> Thanks for the explanation. We are probably talking about the same
> >> thing
> >>>>> but in different ways. To clarify a little bit, I think there are two
> >>>>> patterns to read from a connector.
> >>>>>
> >>>>> Pattern 1: Thread-less connector with a blocking read API. Outside of
> >> the
> >>>>> connector, there is one IO thread per reader, doing blocking read. An
> >>>>> additional thread will interact with all the IO threads.
> >>>>> Pattern 2: Connector with internal thread(s) and non-blocking API.
> >>>> Outside
> >>>>> of the connector, there is one thread for ALL readers, doing IO
> relying
> >>>> on
> >>>>> notification callbacks in the reader.
> >>>>>
> >>>>> In both patterns, there must be at least one thread per connector,
> >> either
> >>>>> inside (created by connector writers) or outside (created by Flink)
> of
> >>>> the
> >>>>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total, to
> >> make
> >>>>> sure that 1 thread is fully non-blocking.
> >>>>>
> >>>>>> Btw, I don’t know if you understand my point. Having only `poll()`
> and
> >>>>> `take()` is not enough for single threaded task. If our source
> >> interface
> >>>>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
> >>>>> isBlocked(),`, there is no way to implement single threaded task that
> >>>> both
> >>>>> reads the data from the source connector and can also react to system
> >>>>> events. Ok, non >blocking `poll()` would allow that, but with busy
> >>>> looping.
> >>>>>
> >>>>> Completely agree that in pattern 2, having a callback is necessary
> for
> >>>> that
> >>>>> single thread outside of the connectors. And the connectors MUST have
> >>>>> internal threads. If we go that way, we should have something like
> >> "void
> >>>>> poll(Callback) / void advance(callback)". I am curious how would
> >>>>> CompletableFuture work here, though. If 10 readers returns 10
> >> completable
> >>>>> futures, will there be 10 additional threads (so 20 threads in total)
> >>>>> blocking waiting on them? Or will there be a single thread busy loop
> >>>>> checking around?
> >>>>>
> >>>>> WRT pattern 1, a single blocking take() API should just work. The
> good
> >>>>> thing is that a blocking read API is usually simpler to implement. An
> >>>>> additional non-blocking "T poll()" method here is indeed optional and
> >>>> could
> >>>>> be used in cases like Flink does not want the thread to block
> forever.
> >>>> They
> >>>>> can also be combined to have a "T poll(Timeout)", which is exactly
> what
> >>>>> KafkaConsumer did.
> >>>>>
> >>>>> It sounds that you are proposing pattern 2 with something similar to
> >> NIO2
> >>>>> AsynchronousByteChannel[1]. That API would work, except that the
> >>>> signature
> >>>>> returning future seems not necessary. If that is the case, a minor
> >> change
> >>>>> on the current FLIP proposal to have "void advance(callback)" should
> >>>> work.
> >>>>> And this means the connectors MUST have their internal threads.
> >>>>>
> >>>>> BTW, one thing I am also trying to avoid is pushing users to perform
> IO
> >>>> in
> >>>>> a method like "isBlocked()". If the method is expected to fetch
> records
> >>>>> (even if not returning them), naming it something more explicit would
> >>>> help
> >>>>> avoid confusion.
> >>>>>
> >>>>> Thanks,
> >>>>>
> >>>>> Jiangjie (Becket) Qin
> >>>>>
> >>>>> [1]
> >>>>>
> >>>>
> >>
> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
> >>>>>
> >>>>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <
> >> piotr@data-artisans.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Hi
> >>>>>>
> >>>>>> Good point with select/epoll, however I do not see how they couldn’t
> >> be
> >>>>>> with Flink if we would like single task in Flink to be
> single-threaded
> >>>> (and
> >>>>>> I believe we should pursue this goal). If your connector blocks on
> >>>>>> `select`, then it can not process/handle control messages from
> Flink,
> >>>> like
> >>>>>> checkpoints, releasing resources and potentially output flushes.
> This
> >>>> would
> >>>>>> require tight integration between connector and Flink’s main event
> >>>>>> loop/selects/etc.
> >>>>>>
> >>>>>> Looking at it from other perspective. Let’s assume that we have a
> >>>>>> connector implemented on top of `select`/`epoll`. In order to
> >> integrate
> >>>> it
> >>>>>> with Flink’s checkpointing/flushes/resource releasing it will have
> to
> >> be
> >>>>>> executed in separate thread one way or another. At least if our API
> >> will
> >>>>>> enforce/encourage non blocking implementations with some kind of
> >>>>>> notifications (`isBlocked()` or `notify()` callback), some
> connectors
> >>>> might
> >>>>>> skip one layer of wapping threads.
> >>>>>>
> >>>>>> Btw, I don’t know if you understand my point. Having only `poll()`
> and
> >>>>>> `take()` is not enough for single threaded task. If our source
> >> interface
> >>>>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
> >>>>>> isBlocked(),`, there is no way to implement single threaded task
> that
> >>>> both
> >>>>>> reads the data from the source connector and can also react to
> system
> >>>>>> events. Ok, non blocking `poll()` would allow that, but with busy
> >>>> looping.
> >>>>>>
> >>>>>> Piotrek
> >>>>>>
> >>>>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com> wrote:
> >>>>>>>
> >>>>>>> Hi Piotrek,
> >>>>>>>
> >>>>>>>> But I don’t see a reason why we should expose both blocking
> `take()`
> >>>> and
> >>>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
> >>>> engine
> >>>>>> or
> >>>>>>> connector) would have to do the same busy
> >>>>>>>> looping anyway and I think it would be better to have a simpler
> >>>>>> connector
> >>>>>>> API (that would solve our problems) and force connectors to comply
> >> one
> >>>>>> way
> >>>>>>> or another.
> >>>>>>>
> >>>>>>> If we let the block happen inside the connector, the blocking does
> >> not
> >>>>>> have
> >>>>>>> to be a busy loop. For example, to do the block waiting
> efficiently,
> >>>> the
> >>>>>>> connector can use java NIO selector().select which relies on OS
> >> syscall
> >>>>>>> like epoll[1] instead of busy looping. But if Flink engine blocks
> >>>> outside
> >>>>>>> the connector, it pretty much has to do the busy loop. So if there
> is
> >>>>>> only
> >>>>>>> one API to get the element, a blocking getNextElement() makes more
> >>>> sense.
> >>>>>>> In any case, we should avoid ambiguity. It has to be crystal clear
> >>>> about
> >>>>>>> whether a method is expected to be blocking or non-blocking.
> >> Otherwise
> >>>> it
> >>>>>>> would be very difficult for Flink engine to do the right thing with
> >> the
> >>>>>>> connectors. At the first glance at getCurrent(), the expected
> >> behavior
> >>>> is
> >>>>>>> not quite clear.
> >>>>>>>
> >>>>>>> That said, I do agree that functionality wise, poll() and take()
> kind
> >>>> of
> >>>>>>> overlap. But they are actually not quite different from
> >>>>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the only
> >>>>>>> difference is that poll() also returns the next record if it is
> >>>>>> available.
> >>>>>>> But I agree that the isBlocked() + getNextElement() is more
> flexible
> >> as
> >>>>>>> users can just check the record availability, but not fetch the
> next
> >>>>>>> element.
> >>>>>>>
> >>>>>>>> In case of thread-less readers with only non-blocking
> `queue.poll()`
> >>>> (is
> >>>>>>> that really a thing? I can not think about a real implementation
> that
> >>>>>>> enforces such constraints)
> >>>>>>> Right, it is pretty much a syntax sugar to allow user combine the
> >>>>>>> check-and-take into one method. It could be achieved with
> >> isBlocked() +
> >>>>>>> getNextElement().
> >>>>>>>
> >>>>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>>
> >>>>>>> Jiangjie (Becket) Qin
> >>>>>>>
> >>>>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
> >>>> piotr@data-artisans.com>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi Becket,
> >>>>>>>>
> >>>>>>>> With my proposal, both of your examples would have to be solved by
> >> the
> >>>>>>>> connector and solution to both problems would be the same:
> >>>>>>>>
> >>>>>>>> Pretend that connector is never blocked (`isBlocked() { return
> >>>>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking
> >> fashion
> >>>>>> (or
> >>>>>>>> semi blocking with return of control from time to time to allow
> for
> >>>>>>>> checkpointing, network flushing and other resource management
> things
> >>>> to
> >>>>>>>> happen in the same main thread). In other words, exactly how you
> >> would
> >>>>>>>> implement `take()` method or how the same source connector would
> be
> >>>>>>>> implemented NOW with current source interface. The difference with
> >>>>>> current
> >>>>>>>> interface would be only that main loop would be outside of the
> >>>>>> connector,
> >>>>>>>> and instead of periodically releasing checkpointing lock,
> >> periodically
> >>>>>>>> `return null;` or `return Optional.empty();` from
> >> `getNextElement()`.
> >>>>>>>>
> >>>>>>>> In case of thread-less readers with only non-blocking
> `queue.poll()`
> >>>> (is
> >>>>>>>> that really a thing? I can not think about a real implementation
> >> that
> >>>>>>>> enforces such constraints), we could provide a wrapper that hides
> >> the
> >>>>>> busy
> >>>>>>>> looping. The same applies how to solve forever blocking readers -
> we
> >>>>>> could
> >>>>>>>> provider another wrapper running the connector in separate thread.
> >>>>>>>>
> >>>>>>>> But I don’t see a reason why we should expose both blocking
> `take()`
> >>>> and
> >>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
> >>>>>> engine or
> >>>>>>>> connector) would have to do the same busy looping anyway and I
> think
> >>>> it
> >>>>>>>> would be better to have a simpler connector API (that would solve
> >> our
> >>>>>>>> problems) and force connectors to comply one way or another.
> >>>>>>>>
> >>>>>>>> Piotrek
> >>>>>>>>
> >>>>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com>
> wrote:
> >>>>>>>>>
> >>>>>>>>> Hi Piotr,
> >>>>>>>>>
> >>>>>>>>> I might have misunderstood you proposal. But let me try to
> explain
> >> my
> >>>>>>>>> concern. I am thinking about the following case:
> >>>>>>>>> 1. a reader has the following two interfaces,
> >>>>>>>>> boolean isBlocked()
> >>>>>>>>> T getNextElement()
> >>>>>>>>> 2. the implementation of getNextElement() is non-blocking.
> >>>>>>>>> 3. The reader is thread-less, i.e. it does not have any internal
> >>>>>> thread.
> >>>>>>>>> For example, it might just delegate the getNextElement() to a
> >>>>>>>> queue.poll(),
> >>>>>>>>> and isBlocked() is just queue.isEmpty().
> >>>>>>>>>
> >>>>>>>>> How can Flink efficiently implement a blocking reading behavior
> >> with
> >>>>>> this
> >>>>>>>>> reader? Either a tight loop or a backoff interval is needed.
> >> Neither
> >>>> of
> >>>>>>>>> them is ideal.
> >>>>>>>>>
> >>>>>>>>> Now let's say in the reader mentioned above implements a blocking
> >>>>>>>>> getNextElement() method. Because there is no internal thread in
> the
> >>>>>>>> reader,
> >>>>>>>>> after isBlocked() returns false. Flink will still have to loop on
> >>>>>>>>> isBlocked() to check whether the next record is available. If the
> >>>> next
> >>>>>>>>> record reaches after 10 min, it is a tight loop for 10 min. You
> >> have
> >>>>>>>>> probably noticed that in this case, even isBlocked() returns a
> >>>> future,
> >>>>>>>> that
> >>>>>>>>> future() will not be completed if Flink does not call some method
> >>>> from
> >>>>>>>> the
> >>>>>>>>> reader, because the reader has no internal thread to complete
> that
> >>>>>> future
> >>>>>>>>> by itself.
> >>>>>>>>>
> >>>>>>>>> Due to the above reasons, a blocking take() API would allow Flink
> >> to
> >>>>>> have
> >>>>>>>>> an efficient way to read from a reader. There are many ways to
> wake
> >>>> up
> >>>>>>>> the
> >>>>>>>>> blocking thread when checkpointing is needed depending on the
> >>>>>>>>> implementation. But I think the poll()/take() API would also work
> >> in
> >>>>>> that
> >>>>>>>>> case.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>>
> >>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>
> >>>>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
> >>>> piotr@data-artisans.com
> >>>>>>>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi,
> >>>>>>>>>>
> >>>>>>>>>> a)
> >>>>>>>>>>
> >>>>>>>>>>> BTW, regarding the isBlock() method, I have a few more
> questions.
> >>>> 21,
> >>>>>>>> Is
> >>>>>>>>>> a method isReady() with boolean as a return value
> >>>>>>>>>>> equivalent? Personally I found it is a little bit confusing in
> >> what
> >>>>>> is
> >>>>>>>>>> supposed to be returned when the future is completed. 22. if
> >>>>>>>>>>> the implementation of isBlocked() is optional, how do the
> callers
> >>>>>> know
> >>>>>>>>>> whether the method is properly implemented or not?
> >>>>>>>>>>> Does not implemented mean it always return a completed future?
> >>>>>>>>>>
> >>>>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an equivalent
> >> to
> >>>>>>>>>> `boolean hasNext()` which in case of “false” provides some kind
> >> of a
> >>>>>>>>>> listener/callback that notifies about presence of next element.
> >>>> There
> >>>>>>>> are
> >>>>>>>>>> some minor details, like `CompletableFuture<?>` has a minimal
> two
> >>>>>> state
> >>>>>>>>>> logic:
> >>>>>>>>>>
> >>>>>>>>>> 1. Future is completed - we have more data
> >>>>>>>>>> 2. Future not yet completed - we don’t have data now, but we
> >>>> might/we
> >>>>>>>> will
> >>>>>>>>>> have in the future
> >>>>>>>>>>
> >>>>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit more
> >>>>>>>>>> complicated/dispersed and can lead/encourage `notify()` spam.
> >>>>>>>>>>
> >>>>>>>>>> b)
> >>>>>>>>>>
> >>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> >>>>>>>> `getNext`
> >>>>>>>>>> the `getNext` would need return a
> >>>>>>>>>>> `ElementWithTimestamp` because some sources want to add
> timestamp
> >>>> to
> >>>>>>>>>> every element. IMO, this is not so memory friendly
> >>>>>>>>>>> so I prefer this design.
> >>>>>>>>>>
> >>>>>>>>>> Guowei I don’t quite understand this. Could you elaborate why
> >>>> having a
> >>>>>>>>>> separate `advance()` help?
> >>>>>>>>>>
> >>>>>>>>>> c)
> >>>>>>>>>>
> >>>>>>>>>> Regarding advance/poll/take. What’s the value of having two
> >> separate
> >>>>>>>>>> methods: poll and take? Which one of them should be called and
> >> which
> >>>>>>>>>> implemented? What’s the benefit of having those methods compared
> >> to
> >>>>>>>> having
> >>>>>>>>>> a one single method `getNextElement()` (or `pollElement() or
> >>>> whatever
> >>>>>> we
> >>>>>>>>>> name it) with following contract:
> >>>>>>>>>>
> >>>>>>>>>> CompletableFuture<?> isBlocked();
> >>>>>>>>>>
> >>>>>>>>>> /**
> >>>>>>>>>> Return next element - will be called only if `isBlocked()` is
> >>>>>> completed.
> >>>>>>>>>> Try to implement it in non blocking fashion, but if that’s
> >>>> impossible
> >>>>>> or
> >>>>>>>>>> you just don’t need the effort, you can block in this method.
> >>>>>>>>>> */
> >>>>>>>>>> T getNextElement();
> >>>>>>>>>>
> >>>>>>>>>> I mean, if the connector is implemented non-blockingly, Flink
> >> should
> >>>>>> use
> >>>>>>>>>> it that way. If it’s not, then `poll()` will `throw new
> >>>>>>>>>> NotImplementedException()`. Implementing both of them and
> >> providing
> >>>>>>>> both of
> >>>>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them
> into
> >> a
> >>>>>>>> single
> >>>>>>>>>> method call that should preferably (but not necessarily need to)
> >> be
> >>>>>>>>>> non-blocking? It’s not like we are implementing general purpose
> >>>>>> `Queue`,
> >>>>>>>>>> which users might want to call either of `poll` or `take`. We
> >> would
> >>>>>>>> always
> >>>>>>>>>> prefer to call `poll`, but if it’s blocking, then still we have
> no
> >>>>>>>> choice,
> >>>>>>>>>> but to call it and block on it.
> >>>>>>>>>>
> >>>>>>>>>> d)
> >>>>>>>>>>
> >>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source
> is
> >>>> very
> >>>>>>>>>>> important. But in addition to `Future/poll`, there may be
> another
> >>>> way
> >>>>>>>> to
> >>>>>>>>>>> achieve this. I think it may be not very memory friendly if
> every
> >>>>>>>> advance
> >>>>>>>>>>> call return a Future.
> >>>>>>>>>>
> >>>>>>>>>> I didn’t want to mention this, to not clog my initial proposal,
> >> but
> >>>>>>>> there
> >>>>>>>>>> is a simple solution for the problem:
> >>>>>>>>>>
> >>>>>>>>>> public interface SplitReader {
> >>>>>>>>>>
> >>>>>>>>>> (…)
> >>>>>>>>>>
> >>>>>>>>>> CompletableFuture<?> NOT_BLOCKED =
> >>>>>>>>>> CompletableFuture.completedFuture(null);
> >>>>>>>>>>
> >>>>>>>>>> /**
> >>>>>>>>>> * Returns a future that will be completed when the page source
> >>>>>>>> becomes
> >>>>>>>>>> * unblocked.  If the page source is not blocked, this method
> >>>> should
> >>>>>>>>>> return
> >>>>>>>>>> * {@code NOT_BLOCKED}.
> >>>>>>>>>> */
> >>>>>>>>>> default CompletableFuture<?> isBlocked()
> >>>>>>>>>> {
> >>>>>>>>>>    return NOT_BLOCKED;
> >>>>>>>>>> }
> >>>>>>>>>>
> >>>>>>>>>> If we are blocked and we are waiting for the IO, then creating a
> >> new
> >>>>>>>>>> Future is non-issue. Under full throttle/throughput and not
> >> blocked
> >>>>>>>> sources
> >>>>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve the
> >>>>>>>> problem.
> >>>>>>>>>>
> >>>>>>>>>> One more remark, non-blocking sources might be a necessity in a
> >>>> single
> >>>>>>>>>> threaded model without a checkpointing lock. (Currently when
> >> sources
> >>>>>> are
> >>>>>>>>>> blocked, they can release checkpointing lock and re-acquire it
> >> again
> >>>>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for
> >> checkpoints
> >>>> to
> >>>>>>>>>> happen when source is idling. In that case either `notify()` or
> my
> >>>>>>>> proposed
> >>>>>>>>>> `isBlocked()` would allow to avoid busy-looping.
> >>>>>>>>>>
> >>>>>>>>>> Piotrek
> >>>>>>>>>>
> >>>>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com>
> >> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>> Hi Thomas,
> >>>>>>>>>>>
> >>>>>>>>>>> The iterator-like API was also the first thing that came to me.
> >> But
> >>>>>> it
> >>>>>>>>>>> seems a little confusing that hasNext() does not mean "the
> stream
> >>>> has
> >>>>>>>> not
> >>>>>>>>>>> ended", but means "the next record is ready", which is
> >> repurposing
> >>>>>> the
> >>>>>>>>>> well
> >>>>>>>>>>> known meaning of hasNext(). If we follow the hasNext()/next()
> >>>>>> pattern,
> >>>>>>>> an
> >>>>>>>>>>> additional isNextReady() method to indicate whether the next
> >> record
> >>>>>> is
> >>>>>>>>>>> ready seems more intuitive to me.
> >>>>>>>>>>>
> >>>>>>>>>>> Similarly, in poll()/take() pattern, another method of isDone()
> >> is
> >>>>>>>> needed
> >>>>>>>>>>> to indicate whether the stream has ended or not.
> >>>>>>>>>>>
> >>>>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
> >>>>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
> >>>>>>>> implementation.
> >>>>>>>>>>> When I am implementing a reader, I could have a couple of
> >> choices:
> >>>>>>>>>>>
> >>>>>>>>>>> - A thread-less reader that does not have any internal thread.
> >>>>>>>>>>> - When poll() is called, the same calling thread will perform a
> >>>> bunch
> >>>>>>>>>> of
> >>>>>>>>>>> IO asynchronously.
> >>>>>>>>>>> - When take() is called, the same calling thread will perform a
> >>>>>>>>>> bunch
> >>>>>>>>>>> of IO and wait until the record is ready.
> >>>>>>>>>>> - A reader with internal threads performing network IO and put
> >>>>>> records
> >>>>>>>>>>> into a buffer.
> >>>>>>>>>>> - When poll() is called, the calling thread simply reads from
> >> the
> >>>>>>>>>>> buffer and return empty result immediately if there is no
> >> record.
> >>>>>>>>>>> - When take() is called, the calling thread reads from the
> >> buffer
> >>>>>>>>>> and
> >>>>>>>>>>> block waiting if the buffer is empty.
> >>>>>>>>>>>
> >>>>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady() API,
> >> it
> >>>> is
> >>>>>>>>>> less
> >>>>>>>>>>> intuitive for the reader developers to write the thread-less
> >>>> pattern.
> >>>>>>>>>>> Although technically speaking one can still do the asynchronous
> >> IO
> >>>> to
> >>>>>>>>>>> prepare the record in isNextReady(). But it is inexplicit and
> >> seems
> >>>>>>>>>>> somewhat hacky.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>>
> >>>>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org>
> >>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Couple more points regarding discovery:
> >>>>>>>>>>>>
> >>>>>>>>>>>> The proposal mentions that discovery could be outside the
> >>>> execution
> >>>>>>>>>> graph.
> >>>>>>>>>>>> Today, discovered partitions/shards are checkpointed. I
> believe
> >>>> that
> >>>>>>>>>> will
> >>>>>>>>>>>> also need to be the case in the future, even when discovery
> and
> >>>>>>>> reading
> >>>>>>>>>> are
> >>>>>>>>>>>> split between different tasks.
> >>>>>>>>>>>>
> >>>>>>>>>>>> For cases such as resharding of a Kinesis stream, the
> >> relationship
> >>>>>>>>>> between
> >>>>>>>>>>>> splits needs to be considered. Splits cannot be randomly
> >>>> distributed
> >>>>>>>>>> over
> >>>>>>>>>>>> readers in certain situations. An example was mentioned here:
> >>>>>>>>>>>>
> >> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thomas
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org>
> >>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks for getting the ball rolling on this!
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Can the number of splits decrease? Yes, splits can be closed
> >> and
> >>>> go
> >>>>>>>>>> away.
> >>>>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing
> shards
> >>>>>> will
> >>>>>>>> be
> >>>>>>>>>>>>> closed and replaced with a new shard).
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive
> approach
> >>>>>> would
> >>>>>>>>>> be
> >>>>>>>>>>>>> the thread-less IO model (pull based, non-blocking, caller
> >>>>>> retrieves
> >>>>>>>>>> new
> >>>>>>>>>>>>> records when available). The current Kinesis API requires the
> >> use
> >>>>>> of
> >>>>>>>>>>>>> threads. But that can be internal to the split reader and
> does
> >>>> not
> >>>>>>>> need
> >>>>>>>>>>>> to
> >>>>>>>>>>>>> be a source API concern. In fact, that's what we are working
> on
> >>>>>> right
> >>>>>>>>>> now
> >>>>>>>>>>>>> as improvement to the existing consumer: Each shard consumer
> >>>> thread
> >>>>>>>>>> will
> >>>>>>>>>>>>> push to a queue, the consumer main thread will poll the
> >> queue(s).
> >>>>>> It
> >>>>>>>> is
> >>>>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> The proposed SplitReader interface would fit the thread-less
> IO
> >>>>>>>> model.
> >>>>>>>>>>>>> Similar to an iterator, we find out if there is a new element
> >>>>>>>> (hasNext)
> >>>>>>>>>>>> and
> >>>>>>>>>>>>> if so, move to it (next()). Separate calls deliver the meta
> >>>>>>>> information
> >>>>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer a
> >>>> timeout
> >>>>>>>>>>>> option,
> >>>>>>>>>>>>> so that the caller does not end up in a busy wait. On the
> other
> >>>>>>>> hand, a
> >>>>>>>>>>>>> caller processing multiple splits may want to cycle through
> >> fast,
> >>>>>> to
> >>>>>>>>>>>>> process elements of other splits as soon as they become
> >>>> available.
> >>>>>>>> The
> >>>>>>>>>>>> nice
> >>>>>>>>>>>>> thing is that this "split merge" logic can now live in Flink
> >> and
> >>>> be
> >>>>>>>>>>>>> optimized and shared between different sources.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>> Thomas
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <
> guowei.mgw@gmail.com
> >>>
> >>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>> Thanks Aljoscha for this FLIP.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking
> source
> >> is
> >>>>>>>> very
> >>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
> >>>> another
> >>>>>>>> way
> >>>>>>>>>> to
> >>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly if
> >>>> every
> >>>>>>>>>>>> advance
> >>>>>>>>>>>>>> call return a Future.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> public interface Listener {
> >>>>>>>>>>>>>> public void notify();
> >>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> public interface SplitReader() {
> >>>>>>>>>>>>>> /**
> >>>>>>>>>>>>>> * When there is no element temporarily, this will return
> >>>> false.
> >>>>>>>>>>>>>> * When elements is available again splitReader can call
> >>>>>>>>>>>>>> listener.notify()
> >>>>>>>>>>>>>> * In addition the frame would check `advance` periodically .
> >>>>>>>>>>>>>> * Of course advance can always return true and ignore the
> >>>>>>>>>> listener
> >>>>>>>>>>>>>> argument for simplicity.
> >>>>>>>>>>>>>> */
> >>>>>>>>>>>>>> public boolean advance(Listener listener);
> >>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create all
> >> Splits
> >>>>>> and
> >>>>>>>>>> how
> >>>>>>>>>>>>>> to create a SplitReader from a Split. But there is no
> strategy
> >>>> for
> >>>>>>>> the
> >>>>>>>>>>>> user
> >>>>>>>>>>>>>> to choose how to assign the splits to the tasks. I think we
> >>>> could
> >>>>>>>> add
> >>>>>>>>>> a
> >>>>>>>>>>>>>> Enum to let user to choose.
> >>>>>>>>>>>>>> /**
> >>>>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
> >>>>>>>>>>>>>> Location,
> >>>>>>>>>>>>>> Workload,
> >>>>>>>>>>>>>> Random,
> >>>>>>>>>>>>>> Average
> >>>>>>>>>>>>>> }
> >>>>>>>>>>>>>> */
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method
> like
> >>>>>>>>>> `getNext`
> >>>>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp`
> >> because
> >>>>>>>> some
> >>>>>>>>>>>>>> sources want to add timestamp to every element. IMO, this is
> >> not
> >>>>>> so
> >>>>>>>>>>>> memory
> >>>>>>>>>>>>>> friendly so I prefer this design.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
> >>>> 下午6:08写道:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a
> lot
> >> of
> >>>>>>>> other
> >>>>>>>>>>>>>>> possible improvements. I have one proposal. Instead of
> >> having a
> >>>>>>>>>> method:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> boolean advance() throws IOException;
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I would replace it with
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> /*
> >>>>>>>>>>>>>>> * Return a future, which when completed means that source
> has
> >>>>>> more
> >>>>>>>>>>>> data
> >>>>>>>>>>>>>>> and getNext() will not block.
> >>>>>>>>>>>>>>> * If you wish to use benefits of non blocking connectors,
> >>>> please
> >>>>>>>>>>>>>>> implement this method appropriately.
> >>>>>>>>>>>>>>> */
> >>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
> >>>>>>>>>>>>>>>   return CompletableFuture.completedFuture(null);
> >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Couple of arguments:
> >>>>>>>>>>>>>>> 1. I don’t understand the division of work between
> >> `advance()`
> >>>>>> and
> >>>>>>>>>>>>>>> `getCurrent()`. What should be done in which, especially
> for
> >>>>>>>>>> connectors
> >>>>>>>>>>>>>>> that handle records in batches (like Kafka) and when should
> >> you
> >>>>>>>> call
> >>>>>>>>>>>>>>> `advance` and when `getCurrent()`.
> >>>>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will
> allow
> >>>> us
> >>>>>> in
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> future to have asynchronous/non blocking connectors and
> more
> >>>>>>>>>>>> efficiently
> >>>>>>>>>>>>>>> handle large number of blocked threads, without busy
> waiting.
> >>>>>> While
> >>>>>>>>>> at
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>> same time it doesn’t add much complexity, since naive
> >> connector
> >>>>>>>>>>>>>>> implementations can be always blocking.
> >>>>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread pool
> >> of
> >>>>>> task
> >>>>>>>>>>>>>>> executors, instead of one thread per task.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Piotrek
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
> >>>> aljoscha@apache.org
> >>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi All,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> In order to finally get the ball rolling on the new source
> >>>>>>>> interface
> >>>>>>>>>>>>>>> that we have discussed for so long I finally created a
> FLIP:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
> >>>> work/discussion
> >>>>>>>>>> about
> >>>>>>>>>>>>>>> adding per-partition watermark support to the Kinesis
> source
> >>>> and
> >>>>>>>>>>>> because
> >>>>>>>>>>>>>>> this would enable generic implementation of event-time
> >>>> alignment
> >>>>>>>> for
> >>>>>>>>>>>> all
> >>>>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
> >>>> alignment
> >>>>>>>>>> part,
> >>>>>>>>>>>>>>> especially the part about information sharing between
> >>>> operations
> >>>>>>>> (I'm
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>>> calling it state sharing because state has a special
> meaning
> >> in
> >>>>>>>>>> Flink).
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Please discuss away!
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Aljoscha
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>>
> >>
> >>
>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@data-artisans.com>.
Hi Jamie,

As it was already covered with my discussion with Becket, there is an easy way to provide blocking API on top of non-blocking API. And yes we both agreed that blocking API is easier to implement by users.

I also do not agree with respect to usefulness of non blocking API. Actually Kafka connector is the one that could be more efficient thanks to the removal of the one layer of threading.

Piotrek

> On 16 Nov 2018, at 02:21, Jamie Grier <jg...@lyft.com.INVALID> wrote:
> 
> Thanks Aljoscha for getting this effort going!
> 
> There's been plenty of discussion here already and I'll add my big +1 to
> making this interface very simple to implement for a new
> Source/SplitReader.  Writing a new production quality connector for Flink
> is very difficult today and requires a lot of detailed knowledge about
> Flink, event time progress, watermarking, idle shard detection, etc and it
> would be good to move almost all of this type of code into Flink itself and
> out of source implementations.  I also think this is totally doable and I'm
> really excited to see this happening.
> 
> I do have a couple of thoughts about the API and the implementation..
> 
> In a perfect world there would be a single thread per Flink source sub-task
> and no additional threads for SplitReaders -- but this assumes a world
> where you have true async IO APIs for the upstream systems (like Kafka and
> Kinesis, S3, HDFS, etc).  If that world did exist the single thread could
> just sit in an efficient select() call waiting for new data to arrive on
> any Split.  That'd be awesome..
> 
> But, that world doesn't exist and given that practical consideration I
> would think the next best implementation is going to be, in practice,
> probably a thread per SplitReader that does nothing but call the source API
> and drop whatever it reads into a (blocking) queue -- as Aljoscha mentioned
> (calling it N+1) and as we started to describe here:
> https://docs.google.com/document/d/13HxfsucoN7aUls1FX202KFAVZ4IMLXEZWyRfZNd6WFM/edit#heading=h.lfi2u4fv9cxa
> 
> I guess my point is that I think we should strive to move as much of
> something like the diagram referenced in the above doc into Flink itself
> and out of sources and simplify the SplitReader API as much as possible as
> well.
> 
> With the above in mind and with regard to the discussion about blocking,
> etc..  I'm not sure I agree with some of the discussion so far with regard
> to this API design.  The calls to the upstream systems (kafka/kinesis) are
> in fact going to be blocking calls.  So a simple API without the constraint
> that the methods must be implemented in a non-blocking way seems better to
> me from the point of view of somebody writing a new source implementation.
> My concern is that if you force the implementer of the SplitReader
> interface to do so in a non-blocking way you're just going to make it
> harder to write those implementations.  Those calls to read the next bit of
> data are going to be blocking calls with most known important sources -- at
> least Kafka/Kinesis/HDFS -- so I think maybe we should just deal with that
> head on and work around it a higher level so the SplitReader interface
> stays super simple to implement.  This means we manage all the threading in
> Flink core, the API stays pull-based, and the implementer is allowed to
> simply block until they have data to return.
> 
> I maybe would change my mind about this if truly asynchronous APIs to the
> upstream source systems were likely to be available in the near future or
> are now and I'm just ignorant of it.  But even then the supporting code in
> Flink to drive async and sync sources would be different and in fact they
> might just have different APIs altogether -- SplitReader vs
> AsyncSplitReader maybe.
> 
> In the end I think playing with the implementation, across more than one
> source, and moving as much common code into Flink itself will reveal the
> best API of course.
> 
> One other interesting note is that you need to preserve per-partition
> ordering so you have to take care with the implementation if it were to be
> based on a thread pool and futures so as not to reorder the reads.
> 
> Anyway, I'm thrilled to see this starting to move forward and I'd very much
> like to help with the implementation wherever I can.  We're doing a
> simplified internal version of some of this at Lyft for just Kinesis
> because we need a solution for event time alignment in the very short term
> but we'd like to immediately start helping to do this properly in Flink
> after that.  One of the end goals for us is event time alignment across
> heterogeneous sources.  Another is making it possible for non-expert users
> to have a high probability of being able to write their own, correct,
> connectors.
> 
> -Jamie
> 
> On Thu, Nov 15, 2018 at 3:43 AM Aljoscha Krettek <al...@apache.org>
> wrote:
> 
>> Hi,
>> 
>> I thought I had sent this mail a while ago but I must have forgotten to
>> send it.
>> 
>> There is another thing we should consider for splits: the range of
>> timestamps that it can contain. For example, the splits of a file source
>> would know what the minimum and maximum timestamp in the splits is,
>> roughly. For infinite splits, such as Kafka partitions, the minimum would
>> be meaningful but the maximum would be +Inf. If the splits expose the
>> interval of time that they contain the readers, or the component that
>> manages the readers can make decisions about which splits to forward and
>> read first. And it can also influence the minimum watermark that a reader
>> forwards: it should never emit a watermark if it knows there are splits to
>> read that have a lower minimum timestamp. I think it should be as easy as
>> adding a minimumTimestamp()/maximumTimestamp() method pair to the split
>> interface.
>> 
>> Another thing we need to resolve is the actual reader interface. I see
>> there has been some good discussion but I don't know if we have consensus.
>> We should try and see how specific sources could be implemented with the
>> new interface. For example, for Kafka I think we need to have N+1 threads
>> per task (where N is the number of splits that a task is reading from). On
>> thread is responsible for reading from the splits. And each split has its
>> own (internal) thread for reading from Kafka and putting messages in an
>> internal queue to pull from. This is similar to how the current Kafka
>> source is implemented, which has a separate fetcher thread. The reason for
>> this split is that we always need to try reading from Kafka to keep the
>> throughput up. In the current implementation the internal queue (or
>> handover) limits the read rate of the reader threads.
>> 
>> @Thomas, what do you think this would look like for Kinesis?
>> 
>> Best,
>> Aljoscha
>> 
>>> On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com> wrote:
>>> 
>>> Hi Piotrek,
>>> 
>>> Thanks a lot for the detailed reply. All makes sense to me.
>>> 
>>> WRT the confusion between advance() / getCurrent(), do you think it would
>>> help if we combine them and have something like:
>>> 
>>> CompletableFuture<T> getNext();
>>> long getWatermark();
>>> long getCurrentTimestamp();
>>> 
>>> Cheers,
>>> 
>>> Jiangjie (Becket) Qin
>>> 
>>> On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <pi...@data-artisans.com>
>>> wrote:
>>> 
>>>> Hi,
>>>> 
>>>> Thanks again for the detailed answer :) Sorry for responding with a
>> delay.
>>>> 
>>>>> Completely agree that in pattern 2, having a callback is necessary for
>>>> that
>>>>> single thread outside of the connectors. And the connectors MUST have
>>>>> internal threads.
>>>> 
>>>> Yes, this thread will have to exists somewhere. In pattern 2 it exists
>> in
>>>> the connector (at least from the perspective of the Flink execution
>>>> engine). In pattern 1 it exists inside the Flink execution engine. With
>>>> completely blocking connectors, like simple reading from files, both of
>>>> those approaches are basically the same. The difference is when user
>>>> implementing Flink source is already working with a non blocking code
>> with
>>>> some internal threads. In this case, pattern 1 would result in "double
>>>> thread wrapping”, while pattern 2 would allow to skip one layer of
>>>> indirection.
>>>> 
>>>>> If we go that way, we should have something like "void
>>>>> poll(Callback) / void advance(callback)". I am curious how would
>>>>> CompletableFuture work here, though. If 10 readers returns 10
>> completable
>>>>> futures, will there be 10 additional threads (so 20 threads in total)
>>>>> blocking waiting on them? Or will there be a single thread busy loop
>>>>> checking around?
>>>> 
>>>> To be honest, I haven’t thought this completely through and I haven’t
>>>> tested/POC’ed it. Having said that, I can think of at least couple of
>>>> solutions. First is something like this:
>>>> 
>>>> 
>>>> 
>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>>>> <
>>>> 
>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>>>>> 
>>>> 
>>>> Line:
>>>> 
>>>>                               `blocked = split.process();`
>>>> 
>>>> Is where the execution goes into to the task/sources. This is where the
>>>> returned future is handled:
>>>> 
>>>>                               blocked.addListener(() -> {
>>>>                                   blockedSplits.remove(split);
>>>>                                   // reset the level priority to
>> prevent
>>>> previously-blocked splits from starving existing splits
>>>>                                   split.resetLevelPriority();
>>>>                                   waitingSplits.offer(split);
>>>>                               }, executor);
>>>> 
>>>> Fundamentally callbacks and Futures are more or less interchangeable You
>>>> can always wrap one into another (creating a callback that completes a
>>>> future and attach a callback once future completes). In this case the
>>>> difference for me is mostly:
>>>> - api with passing callback allows the callback to be fired multiple
>> times
>>>> and to fire it even if the connector is not blocked. This is what I
>> meant
>>>> by saying that api `CompletableFuture<?> isBlocked()` is a bit simpler.
>>>> Connector can only return either “I’m not blocked” or “I’m blocked and I
>>>> will tell you only once when I’m not blocked anymore”.
>>>> 
>>>> But this is not the most important thing for me here. For me important
>>>> thing is to try our best to make Flink task’s control and execution
>> single
>>>> threaded. For that both callback and future APIs should work the same.
>>>> 
>>>>> WRT pattern 1, a single blocking take() API should just work. The good
>>>>> thing is that a blocking read API is usually simpler to implement.
>>>> 
>>>> Yes, they are easier to implement (especially if you are not the one
>> that
>>>> have to deal with the additional threading required around them ;) ).
>> But
>>>> to answer this issue, if we choose pattern 2, we can always provide a
>>>> proxy/wrapper that would using the internal thread implement the
>>>> non-blocking API while exposing blocking API to the user. It would
>>>> implement pattern 2 for the user exposing to him pattern 1. In other
>> words
>>>> implementing pattern 1 in pattern 2 paradigm, while making it possible
>> to
>>>> implement pure pattern 2 connectors.
>>>> 
>>>>> BTW, one thing I am also trying to avoid is pushing users to perform IO
>>>> in
>>>>> a method like "isBlocked()". If the method is expected to fetch records
>>>>> (even if not returning them), naming it something more explicit would
>>>> help
>>>>> avoid confusion.
>>>> 
>>>> If we choose so, we could rework it into something like:
>>>> 
>>>> CompletableFuture<?> advance()
>>>> T getCurrent();
>>>> Watermark getCurrentWatermark()
>>>> 
>>>> But as I wrote before, this is more confusing to me for the exact
>> reasons
>>>> you mentioned :) I would be confused what should be done in `adanvce()`
>> and
>>>> what in `getCurrent()`. However, again this naming issue is not that
>>>> important to me and probably is matter of taste/personal preferences.
>>>> 
>>>> Piotrek
>>>> 
>>>>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com> wrote:
>>>>> 
>>>>> Hi Piotrek,
>>>>> 
>>>>> Thanks for the explanation. We are probably talking about the same
>> thing
>>>>> but in different ways. To clarify a little bit, I think there are two
>>>>> patterns to read from a connector.
>>>>> 
>>>>> Pattern 1: Thread-less connector with a blocking read API. Outside of
>> the
>>>>> connector, there is one IO thread per reader, doing blocking read. An
>>>>> additional thread will interact with all the IO threads.
>>>>> Pattern 2: Connector with internal thread(s) and non-blocking API.
>>>> Outside
>>>>> of the connector, there is one thread for ALL readers, doing IO relying
>>>> on
>>>>> notification callbacks in the reader.
>>>>> 
>>>>> In both patterns, there must be at least one thread per connector,
>> either
>>>>> inside (created by connector writers) or outside (created by Flink) of
>>>> the
>>>>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total, to
>> make
>>>>> sure that 1 thread is fully non-blocking.
>>>>> 
>>>>>> Btw, I don’t know if you understand my point. Having only `poll()` and
>>>>> `take()` is not enough for single threaded task. If our source
>> interface
>>>>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
>>>>> isBlocked(),`, there is no way to implement single threaded task that
>>>> both
>>>>> reads the data from the source connector and can also react to system
>>>>> events. Ok, non >blocking `poll()` would allow that, but with busy
>>>> looping.
>>>>> 
>>>>> Completely agree that in pattern 2, having a callback is necessary for
>>>> that
>>>>> single thread outside of the connectors. And the connectors MUST have
>>>>> internal threads. If we go that way, we should have something like
>> "void
>>>>> poll(Callback) / void advance(callback)". I am curious how would
>>>>> CompletableFuture work here, though. If 10 readers returns 10
>> completable
>>>>> futures, will there be 10 additional threads (so 20 threads in total)
>>>>> blocking waiting on them? Or will there be a single thread busy loop
>>>>> checking around?
>>>>> 
>>>>> WRT pattern 1, a single blocking take() API should just work. The good
>>>>> thing is that a blocking read API is usually simpler to implement. An
>>>>> additional non-blocking "T poll()" method here is indeed optional and
>>>> could
>>>>> be used in cases like Flink does not want the thread to block forever.
>>>> They
>>>>> can also be combined to have a "T poll(Timeout)", which is exactly what
>>>>> KafkaConsumer did.
>>>>> 
>>>>> It sounds that you are proposing pattern 2 with something similar to
>> NIO2
>>>>> AsynchronousByteChannel[1]. That API would work, except that the
>>>> signature
>>>>> returning future seems not necessary. If that is the case, a minor
>> change
>>>>> on the current FLIP proposal to have "void advance(callback)" should
>>>> work.
>>>>> And this means the connectors MUST have their internal threads.
>>>>> 
>>>>> BTW, one thing I am also trying to avoid is pushing users to perform IO
>>>> in
>>>>> a method like "isBlocked()". If the method is expected to fetch records
>>>>> (even if not returning them), naming it something more explicit would
>>>> help
>>>>> avoid confusion.
>>>>> 
>>>>> Thanks,
>>>>> 
>>>>> Jiangjie (Becket) Qin
>>>>> 
>>>>> [1]
>>>>> 
>>>> 
>> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
>>>>> 
>>>>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <
>> piotr@data-artisans.com>
>>>>> wrote:
>>>>> 
>>>>>> Hi
>>>>>> 
>>>>>> Good point with select/epoll, however I do not see how they couldn’t
>> be
>>>>>> with Flink if we would like single task in Flink to be single-threaded
>>>> (and
>>>>>> I believe we should pursue this goal). If your connector blocks on
>>>>>> `select`, then it can not process/handle control messages from Flink,
>>>> like
>>>>>> checkpoints, releasing resources and potentially output flushes. This
>>>> would
>>>>>> require tight integration between connector and Flink’s main event
>>>>>> loop/selects/etc.
>>>>>> 
>>>>>> Looking at it from other perspective. Let’s assume that we have a
>>>>>> connector implemented on top of `select`/`epoll`. In order to
>> integrate
>>>> it
>>>>>> with Flink’s checkpointing/flushes/resource releasing it will have to
>> be
>>>>>> executed in separate thread one way or another. At least if our API
>> will
>>>>>> enforce/encourage non blocking implementations with some kind of
>>>>>> notifications (`isBlocked()` or `notify()` callback), some connectors
>>>> might
>>>>>> skip one layer of wapping threads.
>>>>>> 
>>>>>> Btw, I don’t know if you understand my point. Having only `poll()` and
>>>>>> `take()` is not enough for single threaded task. If our source
>> interface
>>>>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
>>>>>> isBlocked(),`, there is no way to implement single threaded task that
>>>> both
>>>>>> reads the data from the source connector and can also react to system
>>>>>> events. Ok, non blocking `poll()` would allow that, but with busy
>>>> looping.
>>>>>> 
>>>>>> Piotrek
>>>>>> 
>>>>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com> wrote:
>>>>>>> 
>>>>>>> Hi Piotrek,
>>>>>>> 
>>>>>>>> But I don’t see a reason why we should expose both blocking `take()`
>>>> and
>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
>>>> engine
>>>>>> or
>>>>>>> connector) would have to do the same busy
>>>>>>>> looping anyway and I think it would be better to have a simpler
>>>>>> connector
>>>>>>> API (that would solve our problems) and force connectors to comply
>> one
>>>>>> way
>>>>>>> or another.
>>>>>>> 
>>>>>>> If we let the block happen inside the connector, the blocking does
>> not
>>>>>> have
>>>>>>> to be a busy loop. For example, to do the block waiting efficiently,
>>>> the
>>>>>>> connector can use java NIO selector().select which relies on OS
>> syscall
>>>>>>> like epoll[1] instead of busy looping. But if Flink engine blocks
>>>> outside
>>>>>>> the connector, it pretty much has to do the busy loop. So if there is
>>>>>> only
>>>>>>> one API to get the element, a blocking getNextElement() makes more
>>>> sense.
>>>>>>> In any case, we should avoid ambiguity. It has to be crystal clear
>>>> about
>>>>>>> whether a method is expected to be blocking or non-blocking.
>> Otherwise
>>>> it
>>>>>>> would be very difficult for Flink engine to do the right thing with
>> the
>>>>>>> connectors. At the first glance at getCurrent(), the expected
>> behavior
>>>> is
>>>>>>> not quite clear.
>>>>>>> 
>>>>>>> That said, I do agree that functionality wise, poll() and take() kind
>>>> of
>>>>>>> overlap. But they are actually not quite different from
>>>>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the only
>>>>>>> difference is that poll() also returns the next record if it is
>>>>>> available.
>>>>>>> But I agree that the isBlocked() + getNextElement() is more flexible
>> as
>>>>>>> users can just check the record availability, but not fetch the next
>>>>>>> element.
>>>>>>> 
>>>>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
>>>> (is
>>>>>>> that really a thing? I can not think about a real implementation that
>>>>>>> enforces such constraints)
>>>>>>> Right, it is pretty much a syntax sugar to allow user combine the
>>>>>>> check-and-take into one method. It could be achieved with
>> isBlocked() +
>>>>>>> getNextElement().
>>>>>>> 
>>>>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
>>>>>>> 
>>>>>>> Thanks,
>>>>>>> 
>>>>>>> Jiangjie (Becket) Qin
>>>>>>> 
>>>>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
>>>> piotr@data-artisans.com>
>>>>>>> wrote:
>>>>>>> 
>>>>>>>> Hi Becket,
>>>>>>>> 
>>>>>>>> With my proposal, both of your examples would have to be solved by
>> the
>>>>>>>> connector and solution to both problems would be the same:
>>>>>>>> 
>>>>>>>> Pretend that connector is never blocked (`isBlocked() { return
>>>>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking
>> fashion
>>>>>> (or
>>>>>>>> semi blocking with return of control from time to time to allow for
>>>>>>>> checkpointing, network flushing and other resource management things
>>>> to
>>>>>>>> happen in the same main thread). In other words, exactly how you
>> would
>>>>>>>> implement `take()` method or how the same source connector would be
>>>>>>>> implemented NOW with current source interface. The difference with
>>>>>> current
>>>>>>>> interface would be only that main loop would be outside of the
>>>>>> connector,
>>>>>>>> and instead of periodically releasing checkpointing lock,
>> periodically
>>>>>>>> `return null;` or `return Optional.empty();` from
>> `getNextElement()`.
>>>>>>>> 
>>>>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
>>>> (is
>>>>>>>> that really a thing? I can not think about a real implementation
>> that
>>>>>>>> enforces such constraints), we could provide a wrapper that hides
>> the
>>>>>> busy
>>>>>>>> looping. The same applies how to solve forever blocking readers - we
>>>>>> could
>>>>>>>> provider another wrapper running the connector in separate thread.
>>>>>>>> 
>>>>>>>> But I don’t see a reason why we should expose both blocking `take()`
>>>> and
>>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
>>>>>> engine or
>>>>>>>> connector) would have to do the same busy looping anyway and I think
>>>> it
>>>>>>>> would be better to have a simpler connector API (that would solve
>> our
>>>>>>>> problems) and force connectors to comply one way or another.
>>>>>>>> 
>>>>>>>> Piotrek
>>>>>>>> 
>>>>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
>>>>>>>>> 
>>>>>>>>> Hi Piotr,
>>>>>>>>> 
>>>>>>>>> I might have misunderstood you proposal. But let me try to explain
>> my
>>>>>>>>> concern. I am thinking about the following case:
>>>>>>>>> 1. a reader has the following two interfaces,
>>>>>>>>> boolean isBlocked()
>>>>>>>>> T getNextElement()
>>>>>>>>> 2. the implementation of getNextElement() is non-blocking.
>>>>>>>>> 3. The reader is thread-less, i.e. it does not have any internal
>>>>>> thread.
>>>>>>>>> For example, it might just delegate the getNextElement() to a
>>>>>>>> queue.poll(),
>>>>>>>>> and isBlocked() is just queue.isEmpty().
>>>>>>>>> 
>>>>>>>>> How can Flink efficiently implement a blocking reading behavior
>> with
>>>>>> this
>>>>>>>>> reader? Either a tight loop or a backoff interval is needed.
>> Neither
>>>> of
>>>>>>>>> them is ideal.
>>>>>>>>> 
>>>>>>>>> Now let's say in the reader mentioned above implements a blocking
>>>>>>>>> getNextElement() method. Because there is no internal thread in the
>>>>>>>> reader,
>>>>>>>>> after isBlocked() returns false. Flink will still have to loop on
>>>>>>>>> isBlocked() to check whether the next record is available. If the
>>>> next
>>>>>>>>> record reaches after 10 min, it is a tight loop for 10 min. You
>> have
>>>>>>>>> probably noticed that in this case, even isBlocked() returns a
>>>> future,
>>>>>>>> that
>>>>>>>>> future() will not be completed if Flink does not call some method
>>>> from
>>>>>>>> the
>>>>>>>>> reader, because the reader has no internal thread to complete that
>>>>>> future
>>>>>>>>> by itself.
>>>>>>>>> 
>>>>>>>>> Due to the above reasons, a blocking take() API would allow Flink
>> to
>>>>>> have
>>>>>>>>> an efficient way to read from a reader. There are many ways to wake
>>>> up
>>>>>>>> the
>>>>>>>>> blocking thread when checkpointing is needed depending on the
>>>>>>>>> implementation. But I think the poll()/take() API would also work
>> in
>>>>>> that
>>>>>>>>> case.
>>>>>>>>> 
>>>>>>>>> Thanks,
>>>>>>>>> 
>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>> 
>>>>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
>>>> piotr@data-artisans.com
>>>>>>> 
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Hi,
>>>>>>>>>> 
>>>>>>>>>> a)
>>>>>>>>>> 
>>>>>>>>>>> BTW, regarding the isBlock() method, I have a few more questions.
>>>> 21,
>>>>>>>> Is
>>>>>>>>>> a method isReady() with boolean as a return value
>>>>>>>>>>> equivalent? Personally I found it is a little bit confusing in
>> what
>>>>>> is
>>>>>>>>>> supposed to be returned when the future is completed. 22. if
>>>>>>>>>>> the implementation of isBlocked() is optional, how do the callers
>>>>>> know
>>>>>>>>>> whether the method is properly implemented or not?
>>>>>>>>>>> Does not implemented mean it always return a completed future?
>>>>>>>>>> 
>>>>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an equivalent
>> to
>>>>>>>>>> `boolean hasNext()` which in case of “false” provides some kind
>> of a
>>>>>>>>>> listener/callback that notifies about presence of next element.
>>>> There
>>>>>>>> are
>>>>>>>>>> some minor details, like `CompletableFuture<?>` has a minimal two
>>>>>> state
>>>>>>>>>> logic:
>>>>>>>>>> 
>>>>>>>>>> 1. Future is completed - we have more data
>>>>>>>>>> 2. Future not yet completed - we don’t have data now, but we
>>>> might/we
>>>>>>>> will
>>>>>>>>>> have in the future
>>>>>>>>>> 
>>>>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit more
>>>>>>>>>> complicated/dispersed and can lead/encourage `notify()` spam.
>>>>>>>>>> 
>>>>>>>>>> b)
>>>>>>>>>> 
>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>>>>>>>> `getNext`
>>>>>>>>>> the `getNext` would need return a
>>>>>>>>>>> `ElementWithTimestamp` because some sources want to add timestamp
>>>> to
>>>>>>>>>> every element. IMO, this is not so memory friendly
>>>>>>>>>>> so I prefer this design.
>>>>>>>>>> 
>>>>>>>>>> Guowei I don’t quite understand this. Could you elaborate why
>>>> having a
>>>>>>>>>> separate `advance()` help?
>>>>>>>>>> 
>>>>>>>>>> c)
>>>>>>>>>> 
>>>>>>>>>> Regarding advance/poll/take. What’s the value of having two
>> separate
>>>>>>>>>> methods: poll and take? Which one of them should be called and
>> which
>>>>>>>>>> implemented? What’s the benefit of having those methods compared
>> to
>>>>>>>> having
>>>>>>>>>> a one single method `getNextElement()` (or `pollElement() or
>>>> whatever
>>>>>> we
>>>>>>>>>> name it) with following contract:
>>>>>>>>>> 
>>>>>>>>>> CompletableFuture<?> isBlocked();
>>>>>>>>>> 
>>>>>>>>>> /**
>>>>>>>>>> Return next element - will be called only if `isBlocked()` is
>>>>>> completed.
>>>>>>>>>> Try to implement it in non blocking fashion, but if that’s
>>>> impossible
>>>>>> or
>>>>>>>>>> you just don’t need the effort, you can block in this method.
>>>>>>>>>> */
>>>>>>>>>> T getNextElement();
>>>>>>>>>> 
>>>>>>>>>> I mean, if the connector is implemented non-blockingly, Flink
>> should
>>>>>> use
>>>>>>>>>> it that way. If it’s not, then `poll()` will `throw new
>>>>>>>>>> NotImplementedException()`. Implementing both of them and
>> providing
>>>>>>>> both of
>>>>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them into
>> a
>>>>>>>> single
>>>>>>>>>> method call that should preferably (but not necessarily need to)
>> be
>>>>>>>>>> non-blocking? It’s not like we are implementing general purpose
>>>>>> `Queue`,
>>>>>>>>>> which users might want to call either of `poll` or `take`. We
>> would
>>>>>>>> always
>>>>>>>>>> prefer to call `poll`, but if it’s blocking, then still we have no
>>>>>>>> choice,
>>>>>>>>>> but to call it and block on it.
>>>>>>>>>> 
>>>>>>>>>> d)
>>>>>>>>>> 
>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
>>>> very
>>>>>>>>>>> important. But in addition to `Future/poll`, there may be another
>>>> way
>>>>>>>> to
>>>>>>>>>>> achieve this. I think it may be not very memory friendly if every
>>>>>>>> advance
>>>>>>>>>>> call return a Future.
>>>>>>>>>> 
>>>>>>>>>> I didn’t want to mention this, to not clog my initial proposal,
>> but
>>>>>>>> there
>>>>>>>>>> is a simple solution for the problem:
>>>>>>>>>> 
>>>>>>>>>> public interface SplitReader {
>>>>>>>>>> 
>>>>>>>>>> (…)
>>>>>>>>>> 
>>>>>>>>>> CompletableFuture<?> NOT_BLOCKED =
>>>>>>>>>> CompletableFuture.completedFuture(null);
>>>>>>>>>> 
>>>>>>>>>> /**
>>>>>>>>>> * Returns a future that will be completed when the page source
>>>>>>>> becomes
>>>>>>>>>> * unblocked.  If the page source is not blocked, this method
>>>> should
>>>>>>>>>> return
>>>>>>>>>> * {@code NOT_BLOCKED}.
>>>>>>>>>> */
>>>>>>>>>> default CompletableFuture<?> isBlocked()
>>>>>>>>>> {
>>>>>>>>>>    return NOT_BLOCKED;
>>>>>>>>>> }
>>>>>>>>>> 
>>>>>>>>>> If we are blocked and we are waiting for the IO, then creating a
>> new
>>>>>>>>>> Future is non-issue. Under full throttle/throughput and not
>> blocked
>>>>>>>> sources
>>>>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve the
>>>>>>>> problem.
>>>>>>>>>> 
>>>>>>>>>> One more remark, non-blocking sources might be a necessity in a
>>>> single
>>>>>>>>>> threaded model without a checkpointing lock. (Currently when
>> sources
>>>>>> are
>>>>>>>>>> blocked, they can release checkpointing lock and re-acquire it
>> again
>>>>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for
>> checkpoints
>>>> to
>>>>>>>>>> happen when source is idling. In that case either `notify()` or my
>>>>>>>> proposed
>>>>>>>>>> `isBlocked()` would allow to avoid busy-looping.
>>>>>>>>>> 
>>>>>>>>>> Piotrek
>>>>>>>>>> 
>>>>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com>
>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>> Hi Thomas,
>>>>>>>>>>> 
>>>>>>>>>>> The iterator-like API was also the first thing that came to me.
>> But
>>>>>> it
>>>>>>>>>>> seems a little confusing that hasNext() does not mean "the stream
>>>> has
>>>>>>>> not
>>>>>>>>>>> ended", but means "the next record is ready", which is
>> repurposing
>>>>>> the
>>>>>>>>>> well
>>>>>>>>>>> known meaning of hasNext(). If we follow the hasNext()/next()
>>>>>> pattern,
>>>>>>>> an
>>>>>>>>>>> additional isNextReady() method to indicate whether the next
>> record
>>>>>> is
>>>>>>>>>>> ready seems more intuitive to me.
>>>>>>>>>>> 
>>>>>>>>>>> Similarly, in poll()/take() pattern, another method of isDone()
>> is
>>>>>>>> needed
>>>>>>>>>>> to indicate whether the stream has ended or not.
>>>>>>>>>>> 
>>>>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
>>>>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
>>>>>>>> implementation.
>>>>>>>>>>> When I am implementing a reader, I could have a couple of
>> choices:
>>>>>>>>>>> 
>>>>>>>>>>> - A thread-less reader that does not have any internal thread.
>>>>>>>>>>> - When poll() is called, the same calling thread will perform a
>>>> bunch
>>>>>>>>>> of
>>>>>>>>>>> IO asynchronously.
>>>>>>>>>>> - When take() is called, the same calling thread will perform a
>>>>>>>>>> bunch
>>>>>>>>>>> of IO and wait until the record is ready.
>>>>>>>>>>> - A reader with internal threads performing network IO and put
>>>>>> records
>>>>>>>>>>> into a buffer.
>>>>>>>>>>> - When poll() is called, the calling thread simply reads from
>> the
>>>>>>>>>>> buffer and return empty result immediately if there is no
>> record.
>>>>>>>>>>> - When take() is called, the calling thread reads from the
>> buffer
>>>>>>>>>> and
>>>>>>>>>>> block waiting if the buffer is empty.
>>>>>>>>>>> 
>>>>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady() API,
>> it
>>>> is
>>>>>>>>>> less
>>>>>>>>>>> intuitive for the reader developers to write the thread-less
>>>> pattern.
>>>>>>>>>>> Although technically speaking one can still do the asynchronous
>> IO
>>>> to
>>>>>>>>>>> prepare the record in isNextReady(). But it is inexplicit and
>> seems
>>>>>>>>>>> somewhat hacky.
>>>>>>>>>>> 
>>>>>>>>>>> Thanks,
>>>>>>>>>>> 
>>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org>
>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Couple more points regarding discovery:
>>>>>>>>>>>> 
>>>>>>>>>>>> The proposal mentions that discovery could be outside the
>>>> execution
>>>>>>>>>> graph.
>>>>>>>>>>>> Today, discovered partitions/shards are checkpointed. I believe
>>>> that
>>>>>>>>>> will
>>>>>>>>>>>> also need to be the case in the future, even when discovery and
>>>>>>>> reading
>>>>>>>>>> are
>>>>>>>>>>>> split between different tasks.
>>>>>>>>>>>> 
>>>>>>>>>>>> For cases such as resharding of a Kinesis stream, the
>> relationship
>>>>>>>>>> between
>>>>>>>>>>>> splits needs to be considered. Splits cannot be randomly
>>>> distributed
>>>>>>>>>> over
>>>>>>>>>>>> readers in certain situations. An example was mentioned here:
>>>>>>>>>>>> 
>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>>>>>>>>>>>> 
>>>>>>>>>>>> Thomas
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org>
>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks for getting the ball rolling on this!
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Can the number of splits decrease? Yes, splits can be closed
>> and
>>>> go
>>>>>>>>>> away.
>>>>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing shards
>>>>>> will
>>>>>>>> be
>>>>>>>>>>>>> closed and replaced with a new shard).
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive approach
>>>>>> would
>>>>>>>>>> be
>>>>>>>>>>>>> the thread-less IO model (pull based, non-blocking, caller
>>>>>> retrieves
>>>>>>>>>> new
>>>>>>>>>>>>> records when available). The current Kinesis API requires the
>> use
>>>>>> of
>>>>>>>>>>>>> threads. But that can be internal to the split reader and does
>>>> not
>>>>>>>> need
>>>>>>>>>>>> to
>>>>>>>>>>>>> be a source API concern. In fact, that's what we are working on
>>>>>> right
>>>>>>>>>> now
>>>>>>>>>>>>> as improvement to the existing consumer: Each shard consumer
>>>> thread
>>>>>>>>>> will
>>>>>>>>>>>>> push to a queue, the consumer main thread will poll the
>> queue(s).
>>>>>> It
>>>>>>>> is
>>>>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> The proposed SplitReader interface would fit the thread-less IO
>>>>>>>> model.
>>>>>>>>>>>>> Similar to an iterator, we find out if there is a new element
>>>>>>>> (hasNext)
>>>>>>>>>>>> and
>>>>>>>>>>>>> if so, move to it (next()). Separate calls deliver the meta
>>>>>>>> information
>>>>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer a
>>>> timeout
>>>>>>>>>>>> option,
>>>>>>>>>>>>> so that the caller does not end up in a busy wait. On the other
>>>>>>>> hand, a
>>>>>>>>>>>>> caller processing multiple splits may want to cycle through
>> fast,
>>>>>> to
>>>>>>>>>>>>> process elements of other splits as soon as they become
>>>> available.
>>>>>>>> The
>>>>>>>>>>>> nice
>>>>>>>>>>>>> thing is that this "split merge" logic can now live in Flink
>> and
>>>> be
>>>>>>>>>>>>> optimized and shared between different sources.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> Thomas
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <guowei.mgw@gmail.com
>>> 
>>>>>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>> Thanks Aljoscha for this FLIP.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source
>> is
>>>>>>>> very
>>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
>>>> another
>>>>>>>> way
>>>>>>>>>> to
>>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly if
>>>> every
>>>>>>>>>>>> advance
>>>>>>>>>>>>>> call return a Future.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> public interface Listener {
>>>>>>>>>>>>>> public void notify();
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> public interface SplitReader() {
>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>> * When there is no element temporarily, this will return
>>>> false.
>>>>>>>>>>>>>> * When elements is available again splitReader can call
>>>>>>>>>>>>>> listener.notify()
>>>>>>>>>>>>>> * In addition the frame would check `advance` periodically .
>>>>>>>>>>>>>> * Of course advance can always return true and ignore the
>>>>>>>>>> listener
>>>>>>>>>>>>>> argument for simplicity.
>>>>>>>>>>>>>> */
>>>>>>>>>>>>>> public boolean advance(Listener listener);
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create all
>> Splits
>>>>>> and
>>>>>>>>>> how
>>>>>>>>>>>>>> to create a SplitReader from a Split. But there is no strategy
>>>> for
>>>>>>>> the
>>>>>>>>>>>> user
>>>>>>>>>>>>>> to choose how to assign the splits to the tasks. I think we
>>>> could
>>>>>>>> add
>>>>>>>>>> a
>>>>>>>>>>>>>> Enum to let user to choose.
>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
>>>>>>>>>>>>>> Location,
>>>>>>>>>>>>>> Workload,
>>>>>>>>>>>>>> Random,
>>>>>>>>>>>>>> Average
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>> */
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>>>>>>>>>> `getNext`
>>>>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp`
>> because
>>>>>>>> some
>>>>>>>>>>>>>> sources want to add timestamp to every element. IMO, this is
>> not
>>>>>> so
>>>>>>>>>>>> memory
>>>>>>>>>>>>>> friendly so I prefer this design.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
>>>> 下午6:08写道:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot
>> of
>>>>>>>> other
>>>>>>>>>>>>>>> possible improvements. I have one proposal. Instead of
>> having a
>>>>>>>>>> method:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> boolean advance() throws IOException;
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> I would replace it with
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> /*
>>>>>>>>>>>>>>> * Return a future, which when completed means that source has
>>>>>> more
>>>>>>>>>>>> data
>>>>>>>>>>>>>>> and getNext() will not block.
>>>>>>>>>>>>>>> * If you wish to use benefits of non blocking connectors,
>>>> please
>>>>>>>>>>>>>>> implement this method appropriately.
>>>>>>>>>>>>>>> */
>>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
>>>>>>>>>>>>>>>   return CompletableFuture.completedFuture(null);
>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Couple of arguments:
>>>>>>>>>>>>>>> 1. I don’t understand the division of work between
>> `advance()`
>>>>>> and
>>>>>>>>>>>>>>> `getCurrent()`. What should be done in which, especially for
>>>>>>>>>> connectors
>>>>>>>>>>>>>>> that handle records in batches (like Kafka) and when should
>> you
>>>>>>>> call
>>>>>>>>>>>>>>> `advance` and when `getCurrent()`.
>>>>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow
>>>> us
>>>>>> in
>>>>>>>>>> the
>>>>>>>>>>>>>>> future to have asynchronous/non blocking connectors and more
>>>>>>>>>>>> efficiently
>>>>>>>>>>>>>>> handle large number of blocked threads, without busy waiting.
>>>>>> While
>>>>>>>>>> at
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> same time it doesn’t add much complexity, since naive
>> connector
>>>>>>>>>>>>>>> implementations can be always blocking.
>>>>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread pool
>> of
>>>>>> task
>>>>>>>>>>>>>>> executors, instead of one thread per task.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Piotrek
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
>>>> aljoscha@apache.org
>>>>>>> 
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Hi All,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> In order to finally get the ball rolling on the new source
>>>>>>>> interface
>>>>>>>>>>>>>>> that we have discussed for so long I finally created a FLIP:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
>>>> work/discussion
>>>>>>>>>> about
>>>>>>>>>>>>>>> adding per-partition watermark support to the Kinesis source
>>>> and
>>>>>>>>>>>> because
>>>>>>>>>>>>>>> this would enable generic implementation of event-time
>>>> alignment
>>>>>>>> for
>>>>>>>>>>>> all
>>>>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
>>>> alignment
>>>>>>>>>> part,
>>>>>>>>>>>>>>> especially the part about information sharing between
>>>> operations
>>>>>>>> (I'm
>>>>>>>>>>>> not
>>>>>>>>>>>>>>> calling it state sharing because state has a special meaning
>> in
>>>>>>>>>> Flink).
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Please discuss away!
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Aljoscha
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>>>> 
>>>> 
>>>> 
>> 
>> 



Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Jamie Grier <jg...@lyft.com.INVALID>.
Thanks Aljoscha for getting this effort going!

There's been plenty of discussion here already and I'll add my big +1 to
making this interface very simple to implement for a new
Source/SplitReader.  Writing a new production quality connector for Flink
is very difficult today and requires a lot of detailed knowledge about
Flink, event time progress, watermarking, idle shard detection, etc and it
would be good to move almost all of this type of code into Flink itself and
out of source implementations.  I also think this is totally doable and I'm
really excited to see this happening.

I do have a couple of thoughts about the API and the implementation..

In a perfect world there would be a single thread per Flink source sub-task
and no additional threads for SplitReaders -- but this assumes a world
where you have true async IO APIs for the upstream systems (like Kafka and
Kinesis, S3, HDFS, etc).  If that world did exist the single thread could
just sit in an efficient select() call waiting for new data to arrive on
any Split.  That'd be awesome..

But, that world doesn't exist and given that practical consideration I
would think the next best implementation is going to be, in practice,
probably a thread per SplitReader that does nothing but call the source API
and drop whatever it reads into a (blocking) queue -- as Aljoscha mentioned
(calling it N+1) and as we started to describe here:
https://docs.google.com/document/d/13HxfsucoN7aUls1FX202KFAVZ4IMLXEZWyRfZNd6WFM/edit#heading=h.lfi2u4fv9cxa

I guess my point is that I think we should strive to move as much of
something like the diagram referenced in the above doc into Flink itself
and out of sources and simplify the SplitReader API as much as possible as
well.

With the above in mind and with regard to the discussion about blocking,
etc..  I'm not sure I agree with some of the discussion so far with regard
to this API design.  The calls to the upstream systems (kafka/kinesis) are
in fact going to be blocking calls.  So a simple API without the constraint
that the methods must be implemented in a non-blocking way seems better to
me from the point of view of somebody writing a new source implementation.
My concern is that if you force the implementer of the SplitReader
interface to do so in a non-blocking way you're just going to make it
harder to write those implementations.  Those calls to read the next bit of
data are going to be blocking calls with most known important sources -- at
least Kafka/Kinesis/HDFS -- so I think maybe we should just deal with that
head on and work around it a higher level so the SplitReader interface
stays super simple to implement.  This means we manage all the threading in
Flink core, the API stays pull-based, and the implementer is allowed to
simply block until they have data to return.

I maybe would change my mind about this if truly asynchronous APIs to the
upstream source systems were likely to be available in the near future or
are now and I'm just ignorant of it.  But even then the supporting code in
Flink to drive async and sync sources would be different and in fact they
might just have different APIs altogether -- SplitReader vs
AsyncSplitReader maybe.

In the end I think playing with the implementation, across more than one
source, and moving as much common code into Flink itself will reveal the
best API of course.

One other interesting note is that you need to preserve per-partition
ordering so you have to take care with the implementation if it were to be
based on a thread pool and futures so as not to reorder the reads.

Anyway, I'm thrilled to see this starting to move forward and I'd very much
like to help with the implementation wherever I can.  We're doing a
simplified internal version of some of this at Lyft for just Kinesis
because we need a solution for event time alignment in the very short term
but we'd like to immediately start helping to do this properly in Flink
after that.  One of the end goals for us is event time alignment across
heterogeneous sources.  Another is making it possible for non-expert users
to have a high probability of being able to write their own, correct,
connectors.

-Jamie

On Thu, Nov 15, 2018 at 3:43 AM Aljoscha Krettek <al...@apache.org>
wrote:

> Hi,
>
> I thought I had sent this mail a while ago but I must have forgotten to
> send it.
>
> There is another thing we should consider for splits: the range of
> timestamps that it can contain. For example, the splits of a file source
> would know what the minimum and maximum timestamp in the splits is,
> roughly. For infinite splits, such as Kafka partitions, the minimum would
> be meaningful but the maximum would be +Inf. If the splits expose the
> interval of time that they contain the readers, or the component that
> manages the readers can make decisions about which splits to forward and
> read first. And it can also influence the minimum watermark that a reader
> forwards: it should never emit a watermark if it knows there are splits to
> read that have a lower minimum timestamp. I think it should be as easy as
> adding a minimumTimestamp()/maximumTimestamp() method pair to the split
> interface.
>
> Another thing we need to resolve is the actual reader interface. I see
> there has been some good discussion but I don't know if we have consensus.
> We should try and see how specific sources could be implemented with the
> new interface. For example, for Kafka I think we need to have N+1 threads
> per task (where N is the number of splits that a task is reading from). On
> thread is responsible for reading from the splits. And each split has its
> own (internal) thread for reading from Kafka and putting messages in an
> internal queue to pull from. This is similar to how the current Kafka
> source is implemented, which has a separate fetcher thread. The reason for
> this split is that we always need to try reading from Kafka to keep the
> throughput up. In the current implementation the internal queue (or
> handover) limits the read rate of the reader threads.
>
> @Thomas, what do you think this would look like for Kinesis?
>
> Best,
> Aljoscha
>
> > On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com> wrote:
> >
> > Hi Piotrek,
> >
> > Thanks a lot for the detailed reply. All makes sense to me.
> >
> > WRT the confusion between advance() / getCurrent(), do you think it would
> > help if we combine them and have something like:
> >
> > CompletableFuture<T> getNext();
> > long getWatermark();
> > long getCurrentTimestamp();
> >
> > Cheers,
> >
> > Jiangjie (Becket) Qin
> >
> > On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <pi...@data-artisans.com>
> > wrote:
> >
> >> Hi,
> >>
> >> Thanks again for the detailed answer :) Sorry for responding with a
> delay.
> >>
> >>> Completely agree that in pattern 2, having a callback is necessary for
> >> that
> >>> single thread outside of the connectors. And the connectors MUST have
> >>> internal threads.
> >>
> >> Yes, this thread will have to exists somewhere. In pattern 2 it exists
> in
> >> the connector (at least from the perspective of the Flink execution
> >> engine). In pattern 1 it exists inside the Flink execution engine. With
> >> completely blocking connectors, like simple reading from files, both of
> >> those approaches are basically the same. The difference is when user
> >> implementing Flink source is already working with a non blocking code
> with
> >> some internal threads. In this case, pattern 1 would result in "double
> >> thread wrapping”, while pattern 2 would allow to skip one layer of
> >> indirection.
> >>
> >>> If we go that way, we should have something like "void
> >>> poll(Callback) / void advance(callback)". I am curious how would
> >>> CompletableFuture work here, though. If 10 readers returns 10
> completable
> >>> futures, will there be 10 additional threads (so 20 threads in total)
> >>> blocking waiting on them? Or will there be a single thread busy loop
> >>> checking around?
> >>
> >> To be honest, I haven’t thought this completely through and I haven’t
> >> tested/POC’ed it. Having said that, I can think of at least couple of
> >> solutions. First is something like this:
> >>
> >>
> >>
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> >> <
> >>
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> >>>
> >>
> >> Line:
> >>
> >>                                `blocked = split.process();`
> >>
> >> Is where the execution goes into to the task/sources. This is where the
> >> returned future is handled:
> >>
> >>                                blocked.addListener(() -> {
> >>                                    blockedSplits.remove(split);
> >>                                    // reset the level priority to
> prevent
> >> previously-blocked splits from starving existing splits
> >>                                    split.resetLevelPriority();
> >>                                    waitingSplits.offer(split);
> >>                                }, executor);
> >>
> >> Fundamentally callbacks and Futures are more or less interchangeable You
> >> can always wrap one into another (creating a callback that completes a
> >> future and attach a callback once future completes). In this case the
> >> difference for me is mostly:
> >> - api with passing callback allows the callback to be fired multiple
> times
> >> and to fire it even if the connector is not blocked. This is what I
> meant
> >> by saying that api `CompletableFuture<?> isBlocked()` is a bit simpler.
> >> Connector can only return either “I’m not blocked” or “I’m blocked and I
> >> will tell you only once when I’m not blocked anymore”.
> >>
> >> But this is not the most important thing for me here. For me important
> >> thing is to try our best to make Flink task’s control and execution
> single
> >> threaded. For that both callback and future APIs should work the same.
> >>
> >>> WRT pattern 1, a single blocking take() API should just work. The good
> >>> thing is that a blocking read API is usually simpler to implement.
> >>
> >> Yes, they are easier to implement (especially if you are not the one
> that
> >> have to deal with the additional threading required around them ;) ).
> But
> >> to answer this issue, if we choose pattern 2, we can always provide a
> >> proxy/wrapper that would using the internal thread implement the
> >> non-blocking API while exposing blocking API to the user. It would
> >> implement pattern 2 for the user exposing to him pattern 1. In other
> words
> >> implementing pattern 1 in pattern 2 paradigm, while making it possible
> to
> >> implement pure pattern 2 connectors.
> >>
> >>> BTW, one thing I am also trying to avoid is pushing users to perform IO
> >> in
> >>> a method like "isBlocked()". If the method is expected to fetch records
> >>> (even if not returning them), naming it something more explicit would
> >> help
> >>> avoid confusion.
> >>
> >> If we choose so, we could rework it into something like:
> >>
> >> CompletableFuture<?> advance()
> >> T getCurrent();
> >> Watermark getCurrentWatermark()
> >>
> >> But as I wrote before, this is more confusing to me for the exact
> reasons
> >> you mentioned :) I would be confused what should be done in `adanvce()`
> and
> >> what in `getCurrent()`. However, again this naming issue is not that
> >> important to me and probably is matter of taste/personal preferences.
> >>
> >> Piotrek
> >>
> >>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com> wrote:
> >>>
> >>> Hi Piotrek,
> >>>
> >>> Thanks for the explanation. We are probably talking about the same
> thing
> >>> but in different ways. To clarify a little bit, I think there are two
> >>> patterns to read from a connector.
> >>>
> >>> Pattern 1: Thread-less connector with a blocking read API. Outside of
> the
> >>> connector, there is one IO thread per reader, doing blocking read. An
> >>> additional thread will interact with all the IO threads.
> >>> Pattern 2: Connector with internal thread(s) and non-blocking API.
> >> Outside
> >>> of the connector, there is one thread for ALL readers, doing IO relying
> >> on
> >>> notification callbacks in the reader.
> >>>
> >>> In both patterns, there must be at least one thread per connector,
> either
> >>> inside (created by connector writers) or outside (created by Flink) of
> >> the
> >>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total, to
> make
> >>> sure that 1 thread is fully non-blocking.
> >>>
> >>>> Btw, I don’t know if you understand my point. Having only `poll()` and
> >>> `take()` is not enough for single threaded task. If our source
> interface
> >>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
> >>> isBlocked(),`, there is no way to implement single threaded task that
> >> both
> >>> reads the data from the source connector and can also react to system
> >>> events. Ok, non >blocking `poll()` would allow that, but with busy
> >> looping.
> >>>
> >>> Completely agree that in pattern 2, having a callback is necessary for
> >> that
> >>> single thread outside of the connectors. And the connectors MUST have
> >>> internal threads. If we go that way, we should have something like
> "void
> >>> poll(Callback) / void advance(callback)". I am curious how would
> >>> CompletableFuture work here, though. If 10 readers returns 10
> completable
> >>> futures, will there be 10 additional threads (so 20 threads in total)
> >>> blocking waiting on them? Or will there be a single thread busy loop
> >>> checking around?
> >>>
> >>> WRT pattern 1, a single blocking take() API should just work. The good
> >>> thing is that a blocking read API is usually simpler to implement. An
> >>> additional non-blocking "T poll()" method here is indeed optional and
> >> could
> >>> be used in cases like Flink does not want the thread to block forever.
> >> They
> >>> can also be combined to have a "T poll(Timeout)", which is exactly what
> >>> KafkaConsumer did.
> >>>
> >>> It sounds that you are proposing pattern 2 with something similar to
> NIO2
> >>> AsynchronousByteChannel[1]. That API would work, except that the
> >> signature
> >>> returning future seems not necessary. If that is the case, a minor
> change
> >>> on the current FLIP proposal to have "void advance(callback)" should
> >> work.
> >>> And this means the connectors MUST have their internal threads.
> >>>
> >>> BTW, one thing I am also trying to avoid is pushing users to perform IO
> >> in
> >>> a method like "isBlocked()". If the method is expected to fetch records
> >>> (even if not returning them), naming it something more explicit would
> >> help
> >>> avoid confusion.
> >>>
> >>> Thanks,
> >>>
> >>> Jiangjie (Becket) Qin
> >>>
> >>> [1]
> >>>
> >>
> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
> >>>
> >>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <
> piotr@data-artisans.com>
> >>> wrote:
> >>>
> >>>> Hi
> >>>>
> >>>> Good point with select/epoll, however I do not see how they couldn’t
> be
> >>>> with Flink if we would like single task in Flink to be single-threaded
> >> (and
> >>>> I believe we should pursue this goal). If your connector blocks on
> >>>> `select`, then it can not process/handle control messages from Flink,
> >> like
> >>>> checkpoints, releasing resources and potentially output flushes. This
> >> would
> >>>> require tight integration between connector and Flink’s main event
> >>>> loop/selects/etc.
> >>>>
> >>>> Looking at it from other perspective. Let’s assume that we have a
> >>>> connector implemented on top of `select`/`epoll`. In order to
> integrate
> >> it
> >>>> with Flink’s checkpointing/flushes/resource releasing it will have to
> be
> >>>> executed in separate thread one way or another. At least if our API
> will
> >>>> enforce/encourage non blocking implementations with some kind of
> >>>> notifications (`isBlocked()` or `notify()` callback), some connectors
> >> might
> >>>> skip one layer of wapping threads.
> >>>>
> >>>> Btw, I don’t know if you understand my point. Having only `poll()` and
> >>>> `take()` is not enough for single threaded task. If our source
> interface
> >>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
> >>>> isBlocked(),`, there is no way to implement single threaded task that
> >> both
> >>>> reads the data from the source connector and can also react to system
> >>>> events. Ok, non blocking `poll()` would allow that, but with busy
> >> looping.
> >>>>
> >>>> Piotrek
> >>>>
> >>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com> wrote:
> >>>>>
> >>>>> Hi Piotrek,
> >>>>>
> >>>>>> But I don’t see a reason why we should expose both blocking `take()`
> >> and
> >>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
> >> engine
> >>>> or
> >>>>> connector) would have to do the same busy
> >>>>>> looping anyway and I think it would be better to have a simpler
> >>>> connector
> >>>>> API (that would solve our problems) and force connectors to comply
> one
> >>>> way
> >>>>> or another.
> >>>>>
> >>>>> If we let the block happen inside the connector, the blocking does
> not
> >>>> have
> >>>>> to be a busy loop. For example, to do the block waiting efficiently,
> >> the
> >>>>> connector can use java NIO selector().select which relies on OS
> syscall
> >>>>> like epoll[1] instead of busy looping. But if Flink engine blocks
> >> outside
> >>>>> the connector, it pretty much has to do the busy loop. So if there is
> >>>> only
> >>>>> one API to get the element, a blocking getNextElement() makes more
> >> sense.
> >>>>> In any case, we should avoid ambiguity. It has to be crystal clear
> >> about
> >>>>> whether a method is expected to be blocking or non-blocking.
> Otherwise
> >> it
> >>>>> would be very difficult for Flink engine to do the right thing with
> the
> >>>>> connectors. At the first glance at getCurrent(), the expected
> behavior
> >> is
> >>>>> not quite clear.
> >>>>>
> >>>>> That said, I do agree that functionality wise, poll() and take() kind
> >> of
> >>>>> overlap. But they are actually not quite different from
> >>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the only
> >>>>> difference is that poll() also returns the next record if it is
> >>>> available.
> >>>>> But I agree that the isBlocked() + getNextElement() is more flexible
> as
> >>>>> users can just check the record availability, but not fetch the next
> >>>>> element.
> >>>>>
> >>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
> >> (is
> >>>>> that really a thing? I can not think about a real implementation that
> >>>>> enforces such constraints)
> >>>>> Right, it is pretty much a syntax sugar to allow user combine the
> >>>>> check-and-take into one method. It could be achieved with
> isBlocked() +
> >>>>> getNextElement().
> >>>>>
> >>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
> >>>>>
> >>>>> Thanks,
> >>>>>
> >>>>> Jiangjie (Becket) Qin
> >>>>>
> >>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
> >> piotr@data-artisans.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Hi Becket,
> >>>>>>
> >>>>>> With my proposal, both of your examples would have to be solved by
> the
> >>>>>> connector and solution to both problems would be the same:
> >>>>>>
> >>>>>> Pretend that connector is never blocked (`isBlocked() { return
> >>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking
> fashion
> >>>> (or
> >>>>>> semi blocking with return of control from time to time to allow for
> >>>>>> checkpointing, network flushing and other resource management things
> >> to
> >>>>>> happen in the same main thread). In other words, exactly how you
> would
> >>>>>> implement `take()` method or how the same source connector would be
> >>>>>> implemented NOW with current source interface. The difference with
> >>>> current
> >>>>>> interface would be only that main loop would be outside of the
> >>>> connector,
> >>>>>> and instead of periodically releasing checkpointing lock,
> periodically
> >>>>>> `return null;` or `return Optional.empty();` from
> `getNextElement()`.
> >>>>>>
> >>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
> >> (is
> >>>>>> that really a thing? I can not think about a real implementation
> that
> >>>>>> enforces such constraints), we could provide a wrapper that hides
> the
> >>>> busy
> >>>>>> looping. The same applies how to solve forever blocking readers - we
> >>>> could
> >>>>>> provider another wrapper running the connector in separate thread.
> >>>>>>
> >>>>>> But I don’t see a reason why we should expose both blocking `take()`
> >> and
> >>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
> >>>> engine or
> >>>>>> connector) would have to do the same busy looping anyway and I think
> >> it
> >>>>>> would be better to have a simpler connector API (that would solve
> our
> >>>>>> problems) and force connectors to comply one way or another.
> >>>>>>
> >>>>>> Piotrek
> >>>>>>
> >>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
> >>>>>>>
> >>>>>>> Hi Piotr,
> >>>>>>>
> >>>>>>> I might have misunderstood you proposal. But let me try to explain
> my
> >>>>>>> concern. I am thinking about the following case:
> >>>>>>> 1. a reader has the following two interfaces,
> >>>>>>> boolean isBlocked()
> >>>>>>> T getNextElement()
> >>>>>>> 2. the implementation of getNextElement() is non-blocking.
> >>>>>>> 3. The reader is thread-less, i.e. it does not have any internal
> >>>> thread.
> >>>>>>> For example, it might just delegate the getNextElement() to a
> >>>>>> queue.poll(),
> >>>>>>> and isBlocked() is just queue.isEmpty().
> >>>>>>>
> >>>>>>> How can Flink efficiently implement a blocking reading behavior
> with
> >>>> this
> >>>>>>> reader? Either a tight loop or a backoff interval is needed.
> Neither
> >> of
> >>>>>>> them is ideal.
> >>>>>>>
> >>>>>>> Now let's say in the reader mentioned above implements a blocking
> >>>>>>> getNextElement() method. Because there is no internal thread in the
> >>>>>> reader,
> >>>>>>> after isBlocked() returns false. Flink will still have to loop on
> >>>>>>> isBlocked() to check whether the next record is available. If the
> >> next
> >>>>>>> record reaches after 10 min, it is a tight loop for 10 min. You
> have
> >>>>>>> probably noticed that in this case, even isBlocked() returns a
> >> future,
> >>>>>> that
> >>>>>>> future() will not be completed if Flink does not call some method
> >> from
> >>>>>> the
> >>>>>>> reader, because the reader has no internal thread to complete that
> >>>> future
> >>>>>>> by itself.
> >>>>>>>
> >>>>>>> Due to the above reasons, a blocking take() API would allow Flink
> to
> >>>> have
> >>>>>>> an efficient way to read from a reader. There are many ways to wake
> >> up
> >>>>>> the
> >>>>>>> blocking thread when checkpointing is needed depending on the
> >>>>>>> implementation. But I think the poll()/take() API would also work
> in
> >>>> that
> >>>>>>> case.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>>
> >>>>>>> Jiangjie (Becket) Qin
> >>>>>>>
> >>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
> >> piotr@data-artisans.com
> >>>>>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi,
> >>>>>>>>
> >>>>>>>> a)
> >>>>>>>>
> >>>>>>>>> BTW, regarding the isBlock() method, I have a few more questions.
> >> 21,
> >>>>>> Is
> >>>>>>>> a method isReady() with boolean as a return value
> >>>>>>>>> equivalent? Personally I found it is a little bit confusing in
> what
> >>>> is
> >>>>>>>> supposed to be returned when the future is completed. 22. if
> >>>>>>>>> the implementation of isBlocked() is optional, how do the callers
> >>>> know
> >>>>>>>> whether the method is properly implemented or not?
> >>>>>>>>> Does not implemented mean it always return a completed future?
> >>>>>>>>
> >>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an equivalent
> to
> >>>>>>>> `boolean hasNext()` which in case of “false” provides some kind
> of a
> >>>>>>>> listener/callback that notifies about presence of next element.
> >> There
> >>>>>> are
> >>>>>>>> some minor details, like `CompletableFuture<?>` has a minimal two
> >>>> state
> >>>>>>>> logic:
> >>>>>>>>
> >>>>>>>> 1. Future is completed - we have more data
> >>>>>>>> 2. Future not yet completed - we don’t have data now, but we
> >> might/we
> >>>>>> will
> >>>>>>>> have in the future
> >>>>>>>>
> >>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit more
> >>>>>>>> complicated/dispersed and can lead/encourage `notify()` spam.
> >>>>>>>>
> >>>>>>>> b)
> >>>>>>>>
> >>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> >>>>>> `getNext`
> >>>>>>>> the `getNext` would need return a
> >>>>>>>>> `ElementWithTimestamp` because some sources want to add timestamp
> >> to
> >>>>>>>> every element. IMO, this is not so memory friendly
> >>>>>>>>> so I prefer this design.
> >>>>>>>>
> >>>>>>>> Guowei I don’t quite understand this. Could you elaborate why
> >> having a
> >>>>>>>> separate `advance()` help?
> >>>>>>>>
> >>>>>>>> c)
> >>>>>>>>
> >>>>>>>> Regarding advance/poll/take. What’s the value of having two
> separate
> >>>>>>>> methods: poll and take? Which one of them should be called and
> which
> >>>>>>>> implemented? What’s the benefit of having those methods compared
> to
> >>>>>> having
> >>>>>>>> a one single method `getNextElement()` (or `pollElement() or
> >> whatever
> >>>> we
> >>>>>>>> name it) with following contract:
> >>>>>>>>
> >>>>>>>> CompletableFuture<?> isBlocked();
> >>>>>>>>
> >>>>>>>> /**
> >>>>>>>> Return next element - will be called only if `isBlocked()` is
> >>>> completed.
> >>>>>>>> Try to implement it in non blocking fashion, but if that’s
> >> impossible
> >>>> or
> >>>>>>>> you just don’t need the effort, you can block in this method.
> >>>>>>>> */
> >>>>>>>> T getNextElement();
> >>>>>>>>
> >>>>>>>> I mean, if the connector is implemented non-blockingly, Flink
> should
> >>>> use
> >>>>>>>> it that way. If it’s not, then `poll()` will `throw new
> >>>>>>>> NotImplementedException()`. Implementing both of them and
> providing
> >>>>>> both of
> >>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them into
> a
> >>>>>> single
> >>>>>>>> method call that should preferably (but not necessarily need to)
> be
> >>>>>>>> non-blocking? It’s not like we are implementing general purpose
> >>>> `Queue`,
> >>>>>>>> which users might want to call either of `poll` or `take`. We
> would
> >>>>>> always
> >>>>>>>> prefer to call `poll`, but if it’s blocking, then still we have no
> >>>>>> choice,
> >>>>>>>> but to call it and block on it.
> >>>>>>>>
> >>>>>>>> d)
> >>>>>>>>
> >>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
> >> very
> >>>>>>>>> important. But in addition to `Future/poll`, there may be another
> >> way
> >>>>>> to
> >>>>>>>>> achieve this. I think it may be not very memory friendly if every
> >>>>>> advance
> >>>>>>>>> call return a Future.
> >>>>>>>>
> >>>>>>>> I didn’t want to mention this, to not clog my initial proposal,
> but
> >>>>>> there
> >>>>>>>> is a simple solution for the problem:
> >>>>>>>>
> >>>>>>>> public interface SplitReader {
> >>>>>>>>
> >>>>>>>> (…)
> >>>>>>>>
> >>>>>>>> CompletableFuture<?> NOT_BLOCKED =
> >>>>>>>> CompletableFuture.completedFuture(null);
> >>>>>>>>
> >>>>>>>> /**
> >>>>>>>>  * Returns a future that will be completed when the page source
> >>>>>> becomes
> >>>>>>>>  * unblocked.  If the page source is not blocked, this method
> >> should
> >>>>>>>> return
> >>>>>>>>  * {@code NOT_BLOCKED}.
> >>>>>>>>  */
> >>>>>>>> default CompletableFuture<?> isBlocked()
> >>>>>>>> {
> >>>>>>>>     return NOT_BLOCKED;
> >>>>>>>> }
> >>>>>>>>
> >>>>>>>> If we are blocked and we are waiting for the IO, then creating a
> new
> >>>>>>>> Future is non-issue. Under full throttle/throughput and not
> blocked
> >>>>>> sources
> >>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve the
> >>>>>> problem.
> >>>>>>>>
> >>>>>>>> One more remark, non-blocking sources might be a necessity in a
> >> single
> >>>>>>>> threaded model without a checkpointing lock. (Currently when
> sources
> >>>> are
> >>>>>>>> blocked, they can release checkpointing lock and re-acquire it
> again
> >>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for
> checkpoints
> >> to
> >>>>>>>> happen when source is idling. In that case either `notify()` or my
> >>>>>> proposed
> >>>>>>>> `isBlocked()` would allow to avoid busy-looping.
> >>>>>>>>
> >>>>>>>> Piotrek
> >>>>>>>>
> >>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com>
> wrote:
> >>>>>>>>>
> >>>>>>>>> Hi Thomas,
> >>>>>>>>>
> >>>>>>>>> The iterator-like API was also the first thing that came to me.
> But
> >>>> it
> >>>>>>>>> seems a little confusing that hasNext() does not mean "the stream
> >> has
> >>>>>> not
> >>>>>>>>> ended", but means "the next record is ready", which is
> repurposing
> >>>> the
> >>>>>>>> well
> >>>>>>>>> known meaning of hasNext(). If we follow the hasNext()/next()
> >>>> pattern,
> >>>>>> an
> >>>>>>>>> additional isNextReady() method to indicate whether the next
> record
> >>>> is
> >>>>>>>>> ready seems more intuitive to me.
> >>>>>>>>>
> >>>>>>>>> Similarly, in poll()/take() pattern, another method of isDone()
> is
> >>>>>> needed
> >>>>>>>>> to indicate whether the stream has ended or not.
> >>>>>>>>>
> >>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
> >>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
> >>>>>> implementation.
> >>>>>>>>> When I am implementing a reader, I could have a couple of
> choices:
> >>>>>>>>>
> >>>>>>>>> - A thread-less reader that does not have any internal thread.
> >>>>>>>>> - When poll() is called, the same calling thread will perform a
> >> bunch
> >>>>>>>> of
> >>>>>>>>>  IO asynchronously.
> >>>>>>>>>  - When take() is called, the same calling thread will perform a
> >>>>>>>> bunch
> >>>>>>>>>  of IO and wait until the record is ready.
> >>>>>>>>> - A reader with internal threads performing network IO and put
> >>>> records
> >>>>>>>>> into a buffer.
> >>>>>>>>>  - When poll() is called, the calling thread simply reads from
> the
> >>>>>>>>>  buffer and return empty result immediately if there is no
> record.
> >>>>>>>>>  - When take() is called, the calling thread reads from the
> buffer
> >>>>>>>> and
> >>>>>>>>>  block waiting if the buffer is empty.
> >>>>>>>>>
> >>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady() API,
> it
> >> is
> >>>>>>>> less
> >>>>>>>>> intuitive for the reader developers to write the thread-less
> >> pattern.
> >>>>>>>>> Although technically speaking one can still do the asynchronous
> IO
> >> to
> >>>>>>>>> prepare the record in isNextReady(). But it is inexplicit and
> seems
> >>>>>>>>> somewhat hacky.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>>
> >>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org>
> >> wrote:
> >>>>>>>>>
> >>>>>>>>>> Couple more points regarding discovery:
> >>>>>>>>>>
> >>>>>>>>>> The proposal mentions that discovery could be outside the
> >> execution
> >>>>>>>> graph.
> >>>>>>>>>> Today, discovered partitions/shards are checkpointed. I believe
> >> that
> >>>>>>>> will
> >>>>>>>>>> also need to be the case in the future, even when discovery and
> >>>>>> reading
> >>>>>>>> are
> >>>>>>>>>> split between different tasks.
> >>>>>>>>>>
> >>>>>>>>>> For cases such as resharding of a Kinesis stream, the
> relationship
> >>>>>>>> between
> >>>>>>>>>> splits needs to be considered. Splits cannot be randomly
> >> distributed
> >>>>>>>> over
> >>>>>>>>>> readers in certain situations. An example was mentioned here:
> >>>>>>>>>>
> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >>>>>>>>>>
> >>>>>>>>>> Thomas
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org>
> >> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Thanks for getting the ball rolling on this!
> >>>>>>>>>>>
> >>>>>>>>>>> Can the number of splits decrease? Yes, splits can be closed
> and
> >> go
> >>>>>>>> away.
> >>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing shards
> >>>> will
> >>>>>> be
> >>>>>>>>>>> closed and replaced with a new shard).
> >>>>>>>>>>>
> >>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive approach
> >>>> would
> >>>>>>>> be
> >>>>>>>>>>> the thread-less IO model (pull based, non-blocking, caller
> >>>> retrieves
> >>>>>>>> new
> >>>>>>>>>>> records when available). The current Kinesis API requires the
> use
> >>>> of
> >>>>>>>>>>> threads. But that can be internal to the split reader and does
> >> not
> >>>>>> need
> >>>>>>>>>> to
> >>>>>>>>>>> be a source API concern. In fact, that's what we are working on
> >>>> right
> >>>>>>>> now
> >>>>>>>>>>> as improvement to the existing consumer: Each shard consumer
> >> thread
> >>>>>>>> will
> >>>>>>>>>>> push to a queue, the consumer main thread will poll the
> queue(s).
> >>>> It
> >>>>>> is
> >>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
> >>>>>>>>>>>
> >>>>>>>>>>> The proposed SplitReader interface would fit the thread-less IO
> >>>>>> model.
> >>>>>>>>>>> Similar to an iterator, we find out if there is a new element
> >>>>>> (hasNext)
> >>>>>>>>>> and
> >>>>>>>>>>> if so, move to it (next()). Separate calls deliver the meta
> >>>>>> information
> >>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer a
> >> timeout
> >>>>>>>>>> option,
> >>>>>>>>>>> so that the caller does not end up in a busy wait. On the other
> >>>>>> hand, a
> >>>>>>>>>>> caller processing multiple splits may want to cycle through
> fast,
> >>>> to
> >>>>>>>>>>> process elements of other splits as soon as they become
> >> available.
> >>>>>> The
> >>>>>>>>>> nice
> >>>>>>>>>>> thing is that this "split merge" logic can now live in Flink
> and
> >> be
> >>>>>>>>>>> optimized and shared between different sources.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>> Thomas
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <guowei.mgw@gmail.com
> >
> >>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hi,
> >>>>>>>>>>>> Thanks Aljoscha for this FLIP.
> >>>>>>>>>>>>
> >>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source
> is
> >>>>>> very
> >>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
> >> another
> >>>>>> way
> >>>>>>>> to
> >>>>>>>>>>>> achieve this. I think it may be not very memory friendly if
> >> every
> >>>>>>>>>> advance
> >>>>>>>>>>>> call return a Future.
> >>>>>>>>>>>>
> >>>>>>>>>>>> public interface Listener {
> >>>>>>>>>>>> public void notify();
> >>>>>>>>>>>> }
> >>>>>>>>>>>>
> >>>>>>>>>>>> public interface SplitReader() {
> >>>>>>>>>>>> /**
> >>>>>>>>>>>>  * When there is no element temporarily, this will return
> >> false.
> >>>>>>>>>>>>  * When elements is available again splitReader can call
> >>>>>>>>>>>> listener.notify()
> >>>>>>>>>>>>  * In addition the frame would check `advance` periodically .
> >>>>>>>>>>>>  * Of course advance can always return true and ignore the
> >>>>>>>> listener
> >>>>>>>>>>>> argument for simplicity.
> >>>>>>>>>>>>  */
> >>>>>>>>>>>> public boolean advance(Listener listener);
> >>>>>>>>>>>> }
> >>>>>>>>>>>>
> >>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create all
> Splits
> >>>> and
> >>>>>>>> how
> >>>>>>>>>>>> to create a SplitReader from a Split. But there is no strategy
> >> for
> >>>>>> the
> >>>>>>>>>> user
> >>>>>>>>>>>> to choose how to assign the splits to the tasks. I think we
> >> could
> >>>>>> add
> >>>>>>>> a
> >>>>>>>>>>>> Enum to let user to choose.
> >>>>>>>>>>>> /**
> >>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
> >>>>>>>>>>>> Location,
> >>>>>>>>>>>> Workload,
> >>>>>>>>>>>> Random,
> >>>>>>>>>>>> Average
> >>>>>>>>>>>> }
> >>>>>>>>>>>> */
> >>>>>>>>>>>>
> >>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> >>>>>>>> `getNext`
> >>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp`
> because
> >>>>>> some
> >>>>>>>>>>>> sources want to add timestamp to every element. IMO, this is
> not
> >>>> so
> >>>>>>>>>> memory
> >>>>>>>>>>>> friendly so I prefer this design.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks
> >>>>>>>>>>>>
> >>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
> >> 下午6:08写道:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot
> of
> >>>>>> other
> >>>>>>>>>>>>> possible improvements. I have one proposal. Instead of
> having a
> >>>>>>>> method:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> boolean advance() throws IOException;
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I would replace it with
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> /*
> >>>>>>>>>>>>> * Return a future, which when completed means that source has
> >>>> more
> >>>>>>>>>> data
> >>>>>>>>>>>>> and getNext() will not block.
> >>>>>>>>>>>>> * If you wish to use benefits of non blocking connectors,
> >> please
> >>>>>>>>>>>>> implement this method appropriately.
> >>>>>>>>>>>>> */
> >>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
> >>>>>>>>>>>>>    return CompletableFuture.completedFuture(null);
> >>>>>>>>>>>>> }
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Couple of arguments:
> >>>>>>>>>>>>> 1. I don’t understand the division of work between
> `advance()`
> >>>> and
> >>>>>>>>>>>>> `getCurrent()`. What should be done in which, especially for
> >>>>>>>> connectors
> >>>>>>>>>>>>> that handle records in batches (like Kafka) and when should
> you
> >>>>>> call
> >>>>>>>>>>>>> `advance` and when `getCurrent()`.
> >>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow
> >> us
> >>>> in
> >>>>>>>> the
> >>>>>>>>>>>>> future to have asynchronous/non blocking connectors and more
> >>>>>>>>>> efficiently
> >>>>>>>>>>>>> handle large number of blocked threads, without busy waiting.
> >>>> While
> >>>>>>>> at
> >>>>>>>>>> the
> >>>>>>>>>>>>> same time it doesn’t add much complexity, since naive
> connector
> >>>>>>>>>>>>> implementations can be always blocking.
> >>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread pool
> of
> >>>> task
> >>>>>>>>>>>>> executors, instead of one thread per task.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Piotrek
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
> >> aljoscha@apache.org
> >>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi All,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> In order to finally get the ball rolling on the new source
> >>>>>> interface
> >>>>>>>>>>>>> that we have discussed for so long I finally created a FLIP:
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
> >> work/discussion
> >>>>>>>> about
> >>>>>>>>>>>>> adding per-partition watermark support to the Kinesis source
> >> and
> >>>>>>>>>> because
> >>>>>>>>>>>>> this would enable generic implementation of event-time
> >> alignment
> >>>>>> for
> >>>>>>>>>> all
> >>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
> >> alignment
> >>>>>>>> part,
> >>>>>>>>>>>>> especially the part about information sharing between
> >> operations
> >>>>>> (I'm
> >>>>>>>>>> not
> >>>>>>>>>>>>> calling it state sharing because state has a special meaning
> in
> >>>>>>>> Flink).
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Please discuss away!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Aljoscha
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>>
> >>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@data-artisans.com>.
Hi,

One more thing. I think the Kafka client would be a good example of a connector that could use of this `isBlocked()`/callbacks single threaded API from the “Pattern 2”

If we have N threads per N splits, there would be no need for the (N+1)th thread. It could be implemented as a non blocking queue, that notifies the callback/completes the blocked future whenever the queue becomes non empty. The same thread that handles checkpoints, network flushes, resource management could handle reading from this queue.

Piotrek

> On 15 Nov 2018, at 17:13, Piotr Nowojski <pi...@data-artisans.com> wrote:
> 
> Hi
> 
> Re: Becket
> 
>> WRT the confusion between advance() / getCurrent(), do you think it would
>> help if we combine them and have something like:
>> 
>> CompletableFuture<T> getNext();
>> long getWatermark();
>> long getCurrentTimestamp();
> 
> I think that technically this would work the same as `CompletableFuture<?> isBlocked()`, `CompletableFuture<?> advance()` or callbac/`notify()` options. I see two differences:
> 1. in this case once connector unblocks itself and completes the future, Flink’s engine would be responsible for holding the record somewhere, while during this time Flink’s engine can be busy doing other things. Maybe that’s not a big issue, but will slightly complicate the execution engine.
> 2. This might cause some performance overhead, since every record will have to go through the future. As I wrote somewhere before, both `advance()` and `isBlocked()` during full throughput could return static/const NOT_BLOCKED instance, which should/could behave better.
> 
> Nevertheless maybe the choice between those options is secondary one and could be done somewhere else/later or during comparison of some POCs?
> 
> Re: Aljoscha
> 
>> I think it should be as easy as adding a minimumTimestamp()/maximumTimestamp() method pair to the split interface.
> 
> I think that `minimumTimestamp()/maximumTimestamp()` extension seems reasonable if we want Flink to be aware of that. Since watermark handling/emitting would be a custom logic anyway, maybe `minimum` and `maximum` timestamps of a split could be handled as a private fields of the specific connector implementation? I mean, the current proposal with `getCurrentTimestamp()` method indicates that this logic will be hidden from the Flink’s engine anyway, so there might be no need to expose them via API?
> 
>> I see there has been some good discussion but I don't know if we have consensus.
> 
> I think we are converging to a point that having some kind of additional notification that the connector is not blocked anymore would be more flexible for us.
> 
> From the perspective of the execution engine, I would be in favour of testing out our ideas and maybe benchmarking them to make sure that we are not omitting something.
> 
> Piotrek
> 
>> On 15 Nov 2018, at 12:43, Aljoscha Krettek <al...@apache.org> wrote:
>> 
>> Hi,
>> 
>> I thought I had sent this mail a while ago but I must have forgotten to send it.
>> 
>> There is another thing we should consider for splits: the range of timestamps that it can contain. For example, the splits of a file source would know what the minimum and maximum timestamp in the splits is, roughly. For infinite splits, such as Kafka partitions, the minimum would be meaningful but the maximum would be +Inf. If the splits expose the interval of time that they contain the readers, or the component that manages the readers can make decisions about which splits to forward and read first. And it can also influence the minimum watermark that a reader forwards: it should never emit a watermark if it knows there are splits to read that have a lower minimum timestamp. I think it should be as easy as adding a minimumTimestamp()/maximumTimestamp() method pair to the split interface.
>> 
>> Another thing we need to resolve is the actual reader interface. I see there has been some good discussion but I don't know if we have consensus. We should try and see how specific sources could be implemented with the new interface. For example, for Kafka I think we need to have N+1 threads per task (where N is the number of splits that a task is reading from). On thread is responsible for reading from the splits. And each split has its own (internal) thread for reading from Kafka and putting messages in an internal queue to pull from. This is similar to how the current Kafka source is implemented, which has a separate fetcher thread. The reason for this split is that we always need to try reading from Kafka to keep the throughput up. In the current implementation the internal queue (or handover) limits the read rate of the reader threads.
>> 
>> @Thomas, what do you think this would look like for Kinesis?
>> 
>> Best,
>> Aljoscha
>> 
>>> On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com> wrote:
>>> 
>>> Hi Piotrek,
>>> 
>>> Thanks a lot for the detailed reply. All makes sense to me.
>>> 
>>> WRT the confusion between advance() / getCurrent(), do you think it would
>>> help if we combine them and have something like:
>>> 
>>> CompletableFuture<T> getNext();
>>> long getWatermark();
>>> long getCurrentTimestamp();
>>> 
>>> Cheers,
>>> 
>>> Jiangjie (Becket) Qin
>>> 
>>> On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <pi...@data-artisans.com>
>>> wrote:
>>> 
>>>> Hi,
>>>> 
>>>> Thanks again for the detailed answer :) Sorry for responding with a delay.
>>>> 
>>>>> Completely agree that in pattern 2, having a callback is necessary for
>>>> that
>>>>> single thread outside of the connectors. And the connectors MUST have
>>>>> internal threads.
>>>> 
>>>> Yes, this thread will have to exists somewhere. In pattern 2 it exists in
>>>> the connector (at least from the perspective of the Flink execution
>>>> engine). In pattern 1 it exists inside the Flink execution engine. With
>>>> completely blocking connectors, like simple reading from files, both of
>>>> those approaches are basically the same. The difference is when user
>>>> implementing Flink source is already working with a non blocking code with
>>>> some internal threads. In this case, pattern 1 would result in "double
>>>> thread wrapping”, while pattern 2 would allow to skip one layer of
>>>> indirection.
>>>> 
>>>>> If we go that way, we should have something like "void
>>>>> poll(Callback) / void advance(callback)". I am curious how would
>>>>> CompletableFuture work here, though. If 10 readers returns 10 completable
>>>>> futures, will there be 10 additional threads (so 20 threads in total)
>>>>> blocking waiting on them? Or will there be a single thread busy loop
>>>>> checking around?
>>>> 
>>>> To be honest, I haven’t thought this completely through and I haven’t
>>>> tested/POC’ed it. Having said that, I can think of at least couple of
>>>> solutions. First is something like this:
>>>> 
>>>> 
>>>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>>>> <
>>>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>>>>> 
>>>> 
>>>> Line:
>>>> 
>>>>                              `blocked = split.process();`
>>>> 
>>>> Is where the execution goes into to the task/sources. This is where the
>>>> returned future is handled:
>>>> 
>>>>                              blocked.addListener(() -> {
>>>>                                  blockedSplits.remove(split);
>>>>                                  // reset the level priority to prevent
>>>> previously-blocked splits from starving existing splits
>>>>                                  split.resetLevelPriority();
>>>>                                  waitingSplits.offer(split);
>>>>                              }, executor);
>>>> 
>>>> Fundamentally callbacks and Futures are more or less interchangeable You
>>>> can always wrap one into another (creating a callback that completes a
>>>> future and attach a callback once future completes). In this case the
>>>> difference for me is mostly:
>>>> - api with passing callback allows the callback to be fired multiple times
>>>> and to fire it even if the connector is not blocked. This is what I meant
>>>> by saying that api `CompletableFuture<?> isBlocked()` is a bit simpler.
>>>> Connector can only return either “I’m not blocked” or “I’m blocked and I
>>>> will tell you only once when I’m not blocked anymore”.
>>>> 
>>>> But this is not the most important thing for me here. For me important
>>>> thing is to try our best to make Flink task’s control and execution single
>>>> threaded. For that both callback and future APIs should work the same.
>>>> 
>>>>> WRT pattern 1, a single blocking take() API should just work. The good
>>>>> thing is that a blocking read API is usually simpler to implement.
>>>> 
>>>> Yes, they are easier to implement (especially if you are not the one that
>>>> have to deal with the additional threading required around them ;) ). But
>>>> to answer this issue, if we choose pattern 2, we can always provide a
>>>> proxy/wrapper that would using the internal thread implement the
>>>> non-blocking API while exposing blocking API to the user. It would
>>>> implement pattern 2 for the user exposing to him pattern 1. In other words
>>>> implementing pattern 1 in pattern 2 paradigm, while making it possible to
>>>> implement pure pattern 2 connectors.
>>>> 
>>>>> BTW, one thing I am also trying to avoid is pushing users to perform IO
>>>> in
>>>>> a method like "isBlocked()". If the method is expected to fetch records
>>>>> (even if not returning them), naming it something more explicit would
>>>> help
>>>>> avoid confusion.
>>>> 
>>>> If we choose so, we could rework it into something like:
>>>> 
>>>> CompletableFuture<?> advance()
>>>> T getCurrent();
>>>> Watermark getCurrentWatermark()
>>>> 
>>>> But as I wrote before, this is more confusing to me for the exact reasons
>>>> you mentioned :) I would be confused what should be done in `adanvce()` and
>>>> what in `getCurrent()`. However, again this naming issue is not that
>>>> important to me and probably is matter of taste/personal preferences.
>>>> 
>>>> Piotrek
>>>> 
>>>>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com> wrote:
>>>>> 
>>>>> Hi Piotrek,
>>>>> 
>>>>> Thanks for the explanation. We are probably talking about the same thing
>>>>> but in different ways. To clarify a little bit, I think there are two
>>>>> patterns to read from a connector.
>>>>> 
>>>>> Pattern 1: Thread-less connector with a blocking read API. Outside of the
>>>>> connector, there is one IO thread per reader, doing blocking read. An
>>>>> additional thread will interact with all the IO threads.
>>>>> Pattern 2: Connector with internal thread(s) and non-blocking API.
>>>> Outside
>>>>> of the connector, there is one thread for ALL readers, doing IO relying
>>>> on
>>>>> notification callbacks in the reader.
>>>>> 
>>>>> In both patterns, there must be at least one thread per connector, either
>>>>> inside (created by connector writers) or outside (created by Flink) of
>>>> the
>>>>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total, to make
>>>>> sure that 1 thread is fully non-blocking.
>>>>> 
>>>>>> Btw, I don’t know if you understand my point. Having only `poll()` and
>>>>> `take()` is not enough for single threaded task. If our source interface
>>>>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
>>>>> isBlocked(),`, there is no way to implement single threaded task that
>>>> both
>>>>> reads the data from the source connector and can also react to system
>>>>> events. Ok, non >blocking `poll()` would allow that, but with busy
>>>> looping.
>>>>> 
>>>>> Completely agree that in pattern 2, having a callback is necessary for
>>>> that
>>>>> single thread outside of the connectors. And the connectors MUST have
>>>>> internal threads. If we go that way, we should have something like "void
>>>>> poll(Callback) / void advance(callback)". I am curious how would
>>>>> CompletableFuture work here, though. If 10 readers returns 10 completable
>>>>> futures, will there be 10 additional threads (so 20 threads in total)
>>>>> blocking waiting on them? Or will there be a single thread busy loop
>>>>> checking around?
>>>>> 
>>>>> WRT pattern 1, a single blocking take() API should just work. The good
>>>>> thing is that a blocking read API is usually simpler to implement. An
>>>>> additional non-blocking "T poll()" method here is indeed optional and
>>>> could
>>>>> be used in cases like Flink does not want the thread to block forever.
>>>> They
>>>>> can also be combined to have a "T poll(Timeout)", which is exactly what
>>>>> KafkaConsumer did.
>>>>> 
>>>>> It sounds that you are proposing pattern 2 with something similar to NIO2
>>>>> AsynchronousByteChannel[1]. That API would work, except that the
>>>> signature
>>>>> returning future seems not necessary. If that is the case, a minor change
>>>>> on the current FLIP proposal to have "void advance(callback)" should
>>>> work.
>>>>> And this means the connectors MUST have their internal threads.
>>>>> 
>>>>> BTW, one thing I am also trying to avoid is pushing users to perform IO
>>>> in
>>>>> a method like "isBlocked()". If the method is expected to fetch records
>>>>> (even if not returning them), naming it something more explicit would
>>>> help
>>>>> avoid confusion.
>>>>> 
>>>>> Thanks,
>>>>> 
>>>>> Jiangjie (Becket) Qin
>>>>> 
>>>>> [1]
>>>>> 
>>>> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
>>>>> 
>>>>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <pi...@data-artisans.com>
>>>>> wrote:
>>>>> 
>>>>>> Hi
>>>>>> 
>>>>>> Good point with select/epoll, however I do not see how they couldn’t be
>>>>>> with Flink if we would like single task in Flink to be single-threaded
>>>> (and
>>>>>> I believe we should pursue this goal). If your connector blocks on
>>>>>> `select`, then it can not process/handle control messages from Flink,
>>>> like
>>>>>> checkpoints, releasing resources and potentially output flushes. This
>>>> would
>>>>>> require tight integration between connector and Flink’s main event
>>>>>> loop/selects/etc.
>>>>>> 
>>>>>> Looking at it from other perspective. Let’s assume that we have a
>>>>>> connector implemented on top of `select`/`epoll`. In order to integrate
>>>> it
>>>>>> with Flink’s checkpointing/flushes/resource releasing it will have to be
>>>>>> executed in separate thread one way or another. At least if our API will
>>>>>> enforce/encourage non blocking implementations with some kind of
>>>>>> notifications (`isBlocked()` or `notify()` callback), some connectors
>>>> might
>>>>>> skip one layer of wapping threads.
>>>>>> 
>>>>>> Btw, I don’t know if you understand my point. Having only `poll()` and
>>>>>> `take()` is not enough for single threaded task. If our source interface
>>>>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
>>>>>> isBlocked(),`, there is no way to implement single threaded task that
>>>> both
>>>>>> reads the data from the source connector and can also react to system
>>>>>> events. Ok, non blocking `poll()` would allow that, but with busy
>>>> looping.
>>>>>> 
>>>>>> Piotrek
>>>>>> 
>>>>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com> wrote:
>>>>>>> 
>>>>>>> Hi Piotrek,
>>>>>>> 
>>>>>>>> But I don’t see a reason why we should expose both blocking `take()`
>>>> and
>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
>>>> engine
>>>>>> or
>>>>>>> connector) would have to do the same busy
>>>>>>>> looping anyway and I think it would be better to have a simpler
>>>>>> connector
>>>>>>> API (that would solve our problems) and force connectors to comply one
>>>>>> way
>>>>>>> or another.
>>>>>>> 
>>>>>>> If we let the block happen inside the connector, the blocking does not
>>>>>> have
>>>>>>> to be a busy loop. For example, to do the block waiting efficiently,
>>>> the
>>>>>>> connector can use java NIO selector().select which relies on OS syscall
>>>>>>> like epoll[1] instead of busy looping. But if Flink engine blocks
>>>> outside
>>>>>>> the connector, it pretty much has to do the busy loop. So if there is
>>>>>> only
>>>>>>> one API to get the element, a blocking getNextElement() makes more
>>>> sense.
>>>>>>> In any case, we should avoid ambiguity. It has to be crystal clear
>>>> about
>>>>>>> whether a method is expected to be blocking or non-blocking. Otherwise
>>>> it
>>>>>>> would be very difficult for Flink engine to do the right thing with the
>>>>>>> connectors. At the first glance at getCurrent(), the expected behavior
>>>> is
>>>>>>> not quite clear.
>>>>>>> 
>>>>>>> That said, I do agree that functionality wise, poll() and take() kind
>>>> of
>>>>>>> overlap. But they are actually not quite different from
>>>>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the only
>>>>>>> difference is that poll() also returns the next record if it is
>>>>>> available.
>>>>>>> But I agree that the isBlocked() + getNextElement() is more flexible as
>>>>>>> users can just check the record availability, but not fetch the next
>>>>>>> element.
>>>>>>> 
>>>>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
>>>> (is
>>>>>>> that really a thing? I can not think about a real implementation that
>>>>>>> enforces such constraints)
>>>>>>> Right, it is pretty much a syntax sugar to allow user combine the
>>>>>>> check-and-take into one method. It could be achieved with isBlocked() +
>>>>>>> getNextElement().
>>>>>>> 
>>>>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
>>>>>>> 
>>>>>>> Thanks,
>>>>>>> 
>>>>>>> Jiangjie (Becket) Qin
>>>>>>> 
>>>>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
>>>> piotr@data-artisans.com>
>>>>>>> wrote:
>>>>>>> 
>>>>>>>> Hi Becket,
>>>>>>>> 
>>>>>>>> With my proposal, both of your examples would have to be solved by the
>>>>>>>> connector and solution to both problems would be the same:
>>>>>>>> 
>>>>>>>> Pretend that connector is never blocked (`isBlocked() { return
>>>>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking fashion
>>>>>> (or
>>>>>>>> semi blocking with return of control from time to time to allow for
>>>>>>>> checkpointing, network flushing and other resource management things
>>>> to
>>>>>>>> happen in the same main thread). In other words, exactly how you would
>>>>>>>> implement `take()` method or how the same source connector would be
>>>>>>>> implemented NOW with current source interface. The difference with
>>>>>> current
>>>>>>>> interface would be only that main loop would be outside of the
>>>>>> connector,
>>>>>>>> and instead of periodically releasing checkpointing lock, periodically
>>>>>>>> `return null;` or `return Optional.empty();` from `getNextElement()`.
>>>>>>>> 
>>>>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
>>>> (is
>>>>>>>> that really a thing? I can not think about a real implementation that
>>>>>>>> enforces such constraints), we could provide a wrapper that hides the
>>>>>> busy
>>>>>>>> looping. The same applies how to solve forever blocking readers - we
>>>>>> could
>>>>>>>> provider another wrapper running the connector in separate thread.
>>>>>>>> 
>>>>>>>> But I don’t see a reason why we should expose both blocking `take()`
>>>> and
>>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
>>>>>> engine or
>>>>>>>> connector) would have to do the same busy looping anyway and I think
>>>> it
>>>>>>>> would be better to have a simpler connector API (that would solve our
>>>>>>>> problems) and force connectors to comply one way or another.
>>>>>>>> 
>>>>>>>> Piotrek
>>>>>>>> 
>>>>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
>>>>>>>>> 
>>>>>>>>> Hi Piotr,
>>>>>>>>> 
>>>>>>>>> I might have misunderstood you proposal. But let me try to explain my
>>>>>>>>> concern. I am thinking about the following case:
>>>>>>>>> 1. a reader has the following two interfaces,
>>>>>>>>> boolean isBlocked()
>>>>>>>>> T getNextElement()
>>>>>>>>> 2. the implementation of getNextElement() is non-blocking.
>>>>>>>>> 3. The reader is thread-less, i.e. it does not have any internal
>>>>>> thread.
>>>>>>>>> For example, it might just delegate the getNextElement() to a
>>>>>>>> queue.poll(),
>>>>>>>>> and isBlocked() is just queue.isEmpty().
>>>>>>>>> 
>>>>>>>>> How can Flink efficiently implement a blocking reading behavior with
>>>>>> this
>>>>>>>>> reader? Either a tight loop or a backoff interval is needed. Neither
>>>> of
>>>>>>>>> them is ideal.
>>>>>>>>> 
>>>>>>>>> Now let's say in the reader mentioned above implements a blocking
>>>>>>>>> getNextElement() method. Because there is no internal thread in the
>>>>>>>> reader,
>>>>>>>>> after isBlocked() returns false. Flink will still have to loop on
>>>>>>>>> isBlocked() to check whether the next record is available. If the
>>>> next
>>>>>>>>> record reaches after 10 min, it is a tight loop for 10 min. You have
>>>>>>>>> probably noticed that in this case, even isBlocked() returns a
>>>> future,
>>>>>>>> that
>>>>>>>>> future() will not be completed if Flink does not call some method
>>>> from
>>>>>>>> the
>>>>>>>>> reader, because the reader has no internal thread to complete that
>>>>>> future
>>>>>>>>> by itself.
>>>>>>>>> 
>>>>>>>>> Due to the above reasons, a blocking take() API would allow Flink to
>>>>>> have
>>>>>>>>> an efficient way to read from a reader. There are many ways to wake
>>>> up
>>>>>>>> the
>>>>>>>>> blocking thread when checkpointing is needed depending on the
>>>>>>>>> implementation. But I think the poll()/take() API would also work in
>>>>>> that
>>>>>>>>> case.
>>>>>>>>> 
>>>>>>>>> Thanks,
>>>>>>>>> 
>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>> 
>>>>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
>>>> piotr@data-artisans.com
>>>>>>> 
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Hi,
>>>>>>>>>> 
>>>>>>>>>> a)
>>>>>>>>>> 
>>>>>>>>>>> BTW, regarding the isBlock() method, I have a few more questions.
>>>> 21,
>>>>>>>> Is
>>>>>>>>>> a method isReady() with boolean as a return value
>>>>>>>>>>> equivalent? Personally I found it is a little bit confusing in what
>>>>>> is
>>>>>>>>>> supposed to be returned when the future is completed. 22. if
>>>>>>>>>>> the implementation of isBlocked() is optional, how do the callers
>>>>>> know
>>>>>>>>>> whether the method is properly implemented or not?
>>>>>>>>>>> Does not implemented mean it always return a completed future?
>>>>>>>>>> 
>>>>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an equivalent to
>>>>>>>>>> `boolean hasNext()` which in case of “false” provides some kind of a
>>>>>>>>>> listener/callback that notifies about presence of next element.
>>>> There
>>>>>>>> are
>>>>>>>>>> some minor details, like `CompletableFuture<?>` has a minimal two
>>>>>> state
>>>>>>>>>> logic:
>>>>>>>>>> 
>>>>>>>>>> 1. Future is completed - we have more data
>>>>>>>>>> 2. Future not yet completed - we don’t have data now, but we
>>>> might/we
>>>>>>>> will
>>>>>>>>>> have in the future
>>>>>>>>>> 
>>>>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit more
>>>>>>>>>> complicated/dispersed and can lead/encourage `notify()` spam.
>>>>>>>>>> 
>>>>>>>>>> b)
>>>>>>>>>> 
>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>>>>>>>> `getNext`
>>>>>>>>>> the `getNext` would need return a
>>>>>>>>>>> `ElementWithTimestamp` because some sources want to add timestamp
>>>> to
>>>>>>>>>> every element. IMO, this is not so memory friendly
>>>>>>>>>>> so I prefer this design.
>>>>>>>>>> 
>>>>>>>>>> Guowei I don’t quite understand this. Could you elaborate why
>>>> having a
>>>>>>>>>> separate `advance()` help?
>>>>>>>>>> 
>>>>>>>>>> c)
>>>>>>>>>> 
>>>>>>>>>> Regarding advance/poll/take. What’s the value of having two separate
>>>>>>>>>> methods: poll and take? Which one of them should be called and which
>>>>>>>>>> implemented? What’s the benefit of having those methods compared to
>>>>>>>> having
>>>>>>>>>> a one single method `getNextElement()` (or `pollElement() or
>>>> whatever
>>>>>> we
>>>>>>>>>> name it) with following contract:
>>>>>>>>>> 
>>>>>>>>>> CompletableFuture<?> isBlocked();
>>>>>>>>>> 
>>>>>>>>>> /**
>>>>>>>>>> Return next element - will be called only if `isBlocked()` is
>>>>>> completed.
>>>>>>>>>> Try to implement it in non blocking fashion, but if that’s
>>>> impossible
>>>>>> or
>>>>>>>>>> you just don’t need the effort, you can block in this method.
>>>>>>>>>> */
>>>>>>>>>> T getNextElement();
>>>>>>>>>> 
>>>>>>>>>> I mean, if the connector is implemented non-blockingly, Flink should
>>>>>> use
>>>>>>>>>> it that way. If it’s not, then `poll()` will `throw new
>>>>>>>>>> NotImplementedException()`. Implementing both of them and providing
>>>>>>>> both of
>>>>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them into a
>>>>>>>> single
>>>>>>>>>> method call that should preferably (but not necessarily need to) be
>>>>>>>>>> non-blocking? It’s not like we are implementing general purpose
>>>>>> `Queue`,
>>>>>>>>>> which users might want to call either of `poll` or `take`. We would
>>>>>>>> always
>>>>>>>>>> prefer to call `poll`, but if it’s blocking, then still we have no
>>>>>>>> choice,
>>>>>>>>>> but to call it and block on it.
>>>>>>>>>> 
>>>>>>>>>> d)
>>>>>>>>>> 
>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
>>>> very
>>>>>>>>>>> important. But in addition to `Future/poll`, there may be another
>>>> way
>>>>>>>> to
>>>>>>>>>>> achieve this. I think it may be not very memory friendly if every
>>>>>>>> advance
>>>>>>>>>>> call return a Future.
>>>>>>>>>> 
>>>>>>>>>> I didn’t want to mention this, to not clog my initial proposal, but
>>>>>>>> there
>>>>>>>>>> is a simple solution for the problem:
>>>>>>>>>> 
>>>>>>>>>> public interface SplitReader {
>>>>>>>>>> 
>>>>>>>>>> (…)
>>>>>>>>>> 
>>>>>>>>>> CompletableFuture<?> NOT_BLOCKED =
>>>>>>>>>> CompletableFuture.completedFuture(null);
>>>>>>>>>> 
>>>>>>>>>> /**
>>>>>>>>>> * Returns a future that will be completed when the page source
>>>>>>>> becomes
>>>>>>>>>> * unblocked.  If the page source is not blocked, this method
>>>> should
>>>>>>>>>> return
>>>>>>>>>> * {@code NOT_BLOCKED}.
>>>>>>>>>> */
>>>>>>>>>> default CompletableFuture<?> isBlocked()
>>>>>>>>>> {
>>>>>>>>>>   return NOT_BLOCKED;
>>>>>>>>>> }
>>>>>>>>>> 
>>>>>>>>>> If we are blocked and we are waiting for the IO, then creating a new
>>>>>>>>>> Future is non-issue. Under full throttle/throughput and not blocked
>>>>>>>> sources
>>>>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve the
>>>>>>>> problem.
>>>>>>>>>> 
>>>>>>>>>> One more remark, non-blocking sources might be a necessity in a
>>>> single
>>>>>>>>>> threaded model without a checkpointing lock. (Currently when sources
>>>>>> are
>>>>>>>>>> blocked, they can release checkpointing lock and re-acquire it again
>>>>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for checkpoints
>>>> to
>>>>>>>>>> happen when source is idling. In that case either `notify()` or my
>>>>>>>> proposed
>>>>>>>>>> `isBlocked()` would allow to avoid busy-looping.
>>>>>>>>>> 
>>>>>>>>>> Piotrek
>>>>>>>>>> 
>>>>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
>>>>>>>>>>> 
>>>>>>>>>>> Hi Thomas,
>>>>>>>>>>> 
>>>>>>>>>>> The iterator-like API was also the first thing that came to me. But
>>>>>> it
>>>>>>>>>>> seems a little confusing that hasNext() does not mean "the stream
>>>> has
>>>>>>>> not
>>>>>>>>>>> ended", but means "the next record is ready", which is repurposing
>>>>>> the
>>>>>>>>>> well
>>>>>>>>>>> known meaning of hasNext(). If we follow the hasNext()/next()
>>>>>> pattern,
>>>>>>>> an
>>>>>>>>>>> additional isNextReady() method to indicate whether the next record
>>>>>> is
>>>>>>>>>>> ready seems more intuitive to me.
>>>>>>>>>>> 
>>>>>>>>>>> Similarly, in poll()/take() pattern, another method of isDone() is
>>>>>>>> needed
>>>>>>>>>>> to indicate whether the stream has ended or not.
>>>>>>>>>>> 
>>>>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
>>>>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
>>>>>>>> implementation.
>>>>>>>>>>> When I am implementing a reader, I could have a couple of choices:
>>>>>>>>>>> 
>>>>>>>>>>> - A thread-less reader that does not have any internal thread.
>>>>>>>>>>> - When poll() is called, the same calling thread will perform a
>>>> bunch
>>>>>>>>>> of
>>>>>>>>>>> IO asynchronously.
>>>>>>>>>>> - When take() is called, the same calling thread will perform a
>>>>>>>>>> bunch
>>>>>>>>>>> of IO and wait until the record is ready.
>>>>>>>>>>> - A reader with internal threads performing network IO and put
>>>>>> records
>>>>>>>>>>> into a buffer.
>>>>>>>>>>> - When poll() is called, the calling thread simply reads from the
>>>>>>>>>>> buffer and return empty result immediately if there is no record.
>>>>>>>>>>> - When take() is called, the calling thread reads from the buffer
>>>>>>>>>> and
>>>>>>>>>>> block waiting if the buffer is empty.
>>>>>>>>>>> 
>>>>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady() API, it
>>>> is
>>>>>>>>>> less
>>>>>>>>>>> intuitive for the reader developers to write the thread-less
>>>> pattern.
>>>>>>>>>>> Although technically speaking one can still do the asynchronous IO
>>>> to
>>>>>>>>>>> prepare the record in isNextReady(). But it is inexplicit and seems
>>>>>>>>>>> somewhat hacky.
>>>>>>>>>>> 
>>>>>>>>>>> Thanks,
>>>>>>>>>>> 
>>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org>
>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Couple more points regarding discovery:
>>>>>>>>>>>> 
>>>>>>>>>>>> The proposal mentions that discovery could be outside the
>>>> execution
>>>>>>>>>> graph.
>>>>>>>>>>>> Today, discovered partitions/shards are checkpointed. I believe
>>>> that
>>>>>>>>>> will
>>>>>>>>>>>> also need to be the case in the future, even when discovery and
>>>>>>>> reading
>>>>>>>>>> are
>>>>>>>>>>>> split between different tasks.
>>>>>>>>>>>> 
>>>>>>>>>>>> For cases such as resharding of a Kinesis stream, the relationship
>>>>>>>>>> between
>>>>>>>>>>>> splits needs to be considered. Splits cannot be randomly
>>>> distributed
>>>>>>>>>> over
>>>>>>>>>>>> readers in certain situations. An example was mentioned here:
>>>>>>>>>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>>>>>>>>>>>> 
>>>>>>>>>>>> Thomas
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org>
>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks for getting the ball rolling on this!
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Can the number of splits decrease? Yes, splits can be closed and
>>>> go
>>>>>>>>>> away.
>>>>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing shards
>>>>>> will
>>>>>>>> be
>>>>>>>>>>>>> closed and replaced with a new shard).
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive approach
>>>>>> would
>>>>>>>>>> be
>>>>>>>>>>>>> the thread-less IO model (pull based, non-blocking, caller
>>>>>> retrieves
>>>>>>>>>> new
>>>>>>>>>>>>> records when available). The current Kinesis API requires the use
>>>>>> of
>>>>>>>>>>>>> threads. But that can be internal to the split reader and does
>>>> not
>>>>>>>> need
>>>>>>>>>>>> to
>>>>>>>>>>>>> be a source API concern. In fact, that's what we are working on
>>>>>> right
>>>>>>>>>> now
>>>>>>>>>>>>> as improvement to the existing consumer: Each shard consumer
>>>> thread
>>>>>>>>>> will
>>>>>>>>>>>>> push to a queue, the consumer main thread will poll the queue(s).
>>>>>> It
>>>>>>>> is
>>>>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> The proposed SplitReader interface would fit the thread-less IO
>>>>>>>> model.
>>>>>>>>>>>>> Similar to an iterator, we find out if there is a new element
>>>>>>>> (hasNext)
>>>>>>>>>>>> and
>>>>>>>>>>>>> if so, move to it (next()). Separate calls deliver the meta
>>>>>>>> information
>>>>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer a
>>>> timeout
>>>>>>>>>>>> option,
>>>>>>>>>>>>> so that the caller does not end up in a busy wait. On the other
>>>>>>>> hand, a
>>>>>>>>>>>>> caller processing multiple splits may want to cycle through fast,
>>>>>> to
>>>>>>>>>>>>> process elements of other splits as soon as they become
>>>> available.
>>>>>>>> The
>>>>>>>>>>>> nice
>>>>>>>>>>>>> thing is that this "split merge" logic can now live in Flink and
>>>> be
>>>>>>>>>>>>> optimized and shared between different sources.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> Thomas
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>> Thanks Aljoscha for this FLIP.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
>>>>>>>> very
>>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
>>>> another
>>>>>>>> way
>>>>>>>>>> to
>>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly if
>>>> every
>>>>>>>>>>>> advance
>>>>>>>>>>>>>> call return a Future.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> public interface Listener {
>>>>>>>>>>>>>> public void notify();
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> public interface SplitReader() {
>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>> * When there is no element temporarily, this will return
>>>> false.
>>>>>>>>>>>>>> * When elements is available again splitReader can call
>>>>>>>>>>>>>> listener.notify()
>>>>>>>>>>>>>> * In addition the frame would check `advance` periodically .
>>>>>>>>>>>>>> * Of course advance can always return true and ignore the
>>>>>>>>>> listener
>>>>>>>>>>>>>> argument for simplicity.
>>>>>>>>>>>>>> */
>>>>>>>>>>>>>> public boolean advance(Listener listener);
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create all Splits
>>>>>> and
>>>>>>>>>> how
>>>>>>>>>>>>>> to create a SplitReader from a Split. But there is no strategy
>>>> for
>>>>>>>> the
>>>>>>>>>>>> user
>>>>>>>>>>>>>> to choose how to assign the splits to the tasks. I think we
>>>> could
>>>>>>>> add
>>>>>>>>>> a
>>>>>>>>>>>>>> Enum to let user to choose.
>>>>>>>>>>>>>> /**
>>>>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
>>>>>>>>>>>>>> Location,
>>>>>>>>>>>>>> Workload,
>>>>>>>>>>>>>> Random,
>>>>>>>>>>>>>> Average
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>> */
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>>>>>>>>>> `getNext`
>>>>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp` because
>>>>>>>> some
>>>>>>>>>>>>>> sources want to add timestamp to every element. IMO, this is not
>>>>>> so
>>>>>>>>>>>> memory
>>>>>>>>>>>>>> friendly so I prefer this design.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
>>>> 下午6:08写道:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
>>>>>>>> other
>>>>>>>>>>>>>>> possible improvements. I have one proposal. Instead of having a
>>>>>>>>>> method:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> boolean advance() throws IOException;
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> I would replace it with
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> /*
>>>>>>>>>>>>>>> * Return a future, which when completed means that source has
>>>>>> more
>>>>>>>>>>>> data
>>>>>>>>>>>>>>> and getNext() will not block.
>>>>>>>>>>>>>>> * If you wish to use benefits of non blocking connectors,
>>>> please
>>>>>>>>>>>>>>> implement this method appropriately.
>>>>>>>>>>>>>>> */
>>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
>>>>>>>>>>>>>>>  return CompletableFuture.completedFuture(null);
>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Couple of arguments:
>>>>>>>>>>>>>>> 1. I don’t understand the division of work between `advance()`
>>>>>> and
>>>>>>>>>>>>>>> `getCurrent()`. What should be done in which, especially for
>>>>>>>>>> connectors
>>>>>>>>>>>>>>> that handle records in batches (like Kafka) and when should you
>>>>>>>> call
>>>>>>>>>>>>>>> `advance` and when `getCurrent()`.
>>>>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow
>>>> us
>>>>>> in
>>>>>>>>>> the
>>>>>>>>>>>>>>> future to have asynchronous/non blocking connectors and more
>>>>>>>>>>>> efficiently
>>>>>>>>>>>>>>> handle large number of blocked threads, without busy waiting.
>>>>>> While
>>>>>>>>>> at
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> same time it doesn’t add much complexity, since naive connector
>>>>>>>>>>>>>>> implementations can be always blocking.
>>>>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread pool of
>>>>>> task
>>>>>>>>>>>>>>> executors, instead of one thread per task.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Piotrek
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
>>>> aljoscha@apache.org
>>>>>>> 
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Hi All,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> In order to finally get the ball rolling on the new source
>>>>>>>> interface
>>>>>>>>>>>>>>> that we have discussed for so long I finally created a FLIP:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
>>>> work/discussion
>>>>>>>>>> about
>>>>>>>>>>>>>>> adding per-partition watermark support to the Kinesis source
>>>> and
>>>>>>>>>>>> because
>>>>>>>>>>>>>>> this would enable generic implementation of event-time
>>>> alignment
>>>>>>>> for
>>>>>>>>>>>> all
>>>>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
>>>> alignment
>>>>>>>>>> part,
>>>>>>>>>>>>>>> especially the part about information sharing between
>>>> operations
>>>>>>>> (I'm
>>>>>>>>>>>> not
>>>>>>>>>>>>>>> calling it state sharing because state has a special meaning in
>>>>>>>>>> Flink).
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Please discuss away!
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Aljoscha
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>>>> 
>>>> 
>>>> 
>> 
> 



Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@data-artisans.com>.
Hi

Re: Becket

> WRT the confusion between advance() / getCurrent(), do you think it would
> help if we combine them and have something like:
> 
> CompletableFuture<T> getNext();
> long getWatermark();
> long getCurrentTimestamp();

I think that technically this would work the same as `CompletableFuture<?> isBlocked()`, `CompletableFuture<?> advance()` or callbac/`notify()` options. I see two differences:
1. in this case once connector unblocks itself and completes the future, Flink’s engine would be responsible for holding the record somewhere, while during this time Flink’s engine can be busy doing other things. Maybe that’s not a big issue, but will slightly complicate the execution engine.
2. This might cause some performance overhead, since every record will have to go through the future. As I wrote somewhere before, both `advance()` and `isBlocked()` during full throughput could return static/const NOT_BLOCKED instance, which should/could behave better.

Nevertheless maybe the choice between those options is secondary one and could be done somewhere else/later or during comparison of some POCs?

Re: Aljoscha

>  I think it should be as easy as adding a minimumTimestamp()/maximumTimestamp() method pair to the split interface.

I think that `minimumTimestamp()/maximumTimestamp()` extension seems reasonable if we want Flink to be aware of that. Since watermark handling/emitting would be a custom logic anyway, maybe `minimum` and `maximum` timestamps of a split could be handled as a private fields of the specific connector implementation? I mean, the current proposal with `getCurrentTimestamp()` method indicates that this logic will be hidden from the Flink’s engine anyway, so there might be no need to expose them via API?

>  I see there has been some good discussion but I don't know if we have consensus.

I think we are converging to a point that having some kind of additional notification that the connector is not blocked anymore would be more flexible for us.

From the perspective of the execution engine, I would be in favour of testing out our ideas and maybe benchmarking them to make sure that we are not omitting something.

Piotrek

> On 15 Nov 2018, at 12:43, Aljoscha Krettek <al...@apache.org> wrote:
> 
> Hi,
> 
> I thought I had sent this mail a while ago but I must have forgotten to send it.
> 
> There is another thing we should consider for splits: the range of timestamps that it can contain. For example, the splits of a file source would know what the minimum and maximum timestamp in the splits is, roughly. For infinite splits, such as Kafka partitions, the minimum would be meaningful but the maximum would be +Inf. If the splits expose the interval of time that they contain the readers, or the component that manages the readers can make decisions about which splits to forward and read first. And it can also influence the minimum watermark that a reader forwards: it should never emit a watermark if it knows there are splits to read that have a lower minimum timestamp. I think it should be as easy as adding a minimumTimestamp()/maximumTimestamp() method pair to the split interface.
> 
> Another thing we need to resolve is the actual reader interface. I see there has been some good discussion but I don't know if we have consensus. We should try and see how specific sources could be implemented with the new interface. For example, for Kafka I think we need to have N+1 threads per task (where N is the number of splits that a task is reading from). On thread is responsible for reading from the splits. And each split has its own (internal) thread for reading from Kafka and putting messages in an internal queue to pull from. This is similar to how the current Kafka source is implemented, which has a separate fetcher thread. The reason for this split is that we always need to try reading from Kafka to keep the throughput up. In the current implementation the internal queue (or handover) limits the read rate of the reader threads.
> 
> @Thomas, what do you think this would look like for Kinesis?
> 
> Best,
> Aljoscha
> 
>> On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com> wrote:
>> 
>> Hi Piotrek,
>> 
>> Thanks a lot for the detailed reply. All makes sense to me.
>> 
>> WRT the confusion between advance() / getCurrent(), do you think it would
>> help if we combine them and have something like:
>> 
>> CompletableFuture<T> getNext();
>> long getWatermark();
>> long getCurrentTimestamp();
>> 
>> Cheers,
>> 
>> Jiangjie (Becket) Qin
>> 
>> On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <pi...@data-artisans.com>
>> wrote:
>> 
>>> Hi,
>>> 
>>> Thanks again for the detailed answer :) Sorry for responding with a delay.
>>> 
>>>> Completely agree that in pattern 2, having a callback is necessary for
>>> that
>>>> single thread outside of the connectors. And the connectors MUST have
>>>> internal threads.
>>> 
>>> Yes, this thread will have to exists somewhere. In pattern 2 it exists in
>>> the connector (at least from the perspective of the Flink execution
>>> engine). In pattern 1 it exists inside the Flink execution engine. With
>>> completely blocking connectors, like simple reading from files, both of
>>> those approaches are basically the same. The difference is when user
>>> implementing Flink source is already working with a non blocking code with
>>> some internal threads. In this case, pattern 1 would result in "double
>>> thread wrapping”, while pattern 2 would allow to skip one layer of
>>> indirection.
>>> 
>>>> If we go that way, we should have something like "void
>>>> poll(Callback) / void advance(callback)". I am curious how would
>>>> CompletableFuture work here, though. If 10 readers returns 10 completable
>>>> futures, will there be 10 additional threads (so 20 threads in total)
>>>> blocking waiting on them? Or will there be a single thread busy loop
>>>> checking around?
>>> 
>>> To be honest, I haven’t thought this completely through and I haven’t
>>> tested/POC’ed it. Having said that, I can think of at least couple of
>>> solutions. First is something like this:
>>> 
>>> 
>>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>>> <
>>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>>>> 
>>> 
>>> Line:
>>> 
>>>                               `blocked = split.process();`
>>> 
>>> Is where the execution goes into to the task/sources. This is where the
>>> returned future is handled:
>>> 
>>>                               blocked.addListener(() -> {
>>>                                   blockedSplits.remove(split);
>>>                                   // reset the level priority to prevent
>>> previously-blocked splits from starving existing splits
>>>                                   split.resetLevelPriority();
>>>                                   waitingSplits.offer(split);
>>>                               }, executor);
>>> 
>>> Fundamentally callbacks and Futures are more or less interchangeable You
>>> can always wrap one into another (creating a callback that completes a
>>> future and attach a callback once future completes). In this case the
>>> difference for me is mostly:
>>> - api with passing callback allows the callback to be fired multiple times
>>> and to fire it even if the connector is not blocked. This is what I meant
>>> by saying that api `CompletableFuture<?> isBlocked()` is a bit simpler.
>>> Connector can only return either “I’m not blocked” or “I’m blocked and I
>>> will tell you only once when I’m not blocked anymore”.
>>> 
>>> But this is not the most important thing for me here. For me important
>>> thing is to try our best to make Flink task’s control and execution single
>>> threaded. For that both callback and future APIs should work the same.
>>> 
>>>> WRT pattern 1, a single blocking take() API should just work. The good
>>>> thing is that a blocking read API is usually simpler to implement.
>>> 
>>> Yes, they are easier to implement (especially if you are not the one that
>>> have to deal with the additional threading required around them ;) ). But
>>> to answer this issue, if we choose pattern 2, we can always provide a
>>> proxy/wrapper that would using the internal thread implement the
>>> non-blocking API while exposing blocking API to the user. It would
>>> implement pattern 2 for the user exposing to him pattern 1. In other words
>>> implementing pattern 1 in pattern 2 paradigm, while making it possible to
>>> implement pure pattern 2 connectors.
>>> 
>>>> BTW, one thing I am also trying to avoid is pushing users to perform IO
>>> in
>>>> a method like "isBlocked()". If the method is expected to fetch records
>>>> (even if not returning them), naming it something more explicit would
>>> help
>>>> avoid confusion.
>>> 
>>> If we choose so, we could rework it into something like:
>>> 
>>> CompletableFuture<?> advance()
>>> T getCurrent();
>>> Watermark getCurrentWatermark()
>>> 
>>> But as I wrote before, this is more confusing to me for the exact reasons
>>> you mentioned :) I would be confused what should be done in `adanvce()` and
>>> what in `getCurrent()`. However, again this naming issue is not that
>>> important to me and probably is matter of taste/personal preferences.
>>> 
>>> Piotrek
>>> 
>>>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com> wrote:
>>>> 
>>>> Hi Piotrek,
>>>> 
>>>> Thanks for the explanation. We are probably talking about the same thing
>>>> but in different ways. To clarify a little bit, I think there are two
>>>> patterns to read from a connector.
>>>> 
>>>> Pattern 1: Thread-less connector with a blocking read API. Outside of the
>>>> connector, there is one IO thread per reader, doing blocking read. An
>>>> additional thread will interact with all the IO threads.
>>>> Pattern 2: Connector with internal thread(s) and non-blocking API.
>>> Outside
>>>> of the connector, there is one thread for ALL readers, doing IO relying
>>> on
>>>> notification callbacks in the reader.
>>>> 
>>>> In both patterns, there must be at least one thread per connector, either
>>>> inside (created by connector writers) or outside (created by Flink) of
>>> the
>>>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total, to make
>>>> sure that 1 thread is fully non-blocking.
>>>> 
>>>>> Btw, I don’t know if you understand my point. Having only `poll()` and
>>>> `take()` is not enough for single threaded task. If our source interface
>>>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
>>>> isBlocked(),`, there is no way to implement single threaded task that
>>> both
>>>> reads the data from the source connector and can also react to system
>>>> events. Ok, non >blocking `poll()` would allow that, but with busy
>>> looping.
>>>> 
>>>> Completely agree that in pattern 2, having a callback is necessary for
>>> that
>>>> single thread outside of the connectors. And the connectors MUST have
>>>> internal threads. If we go that way, we should have something like "void
>>>> poll(Callback) / void advance(callback)". I am curious how would
>>>> CompletableFuture work here, though. If 10 readers returns 10 completable
>>>> futures, will there be 10 additional threads (so 20 threads in total)
>>>> blocking waiting on them? Or will there be a single thread busy loop
>>>> checking around?
>>>> 
>>>> WRT pattern 1, a single blocking take() API should just work. The good
>>>> thing is that a blocking read API is usually simpler to implement. An
>>>> additional non-blocking "T poll()" method here is indeed optional and
>>> could
>>>> be used in cases like Flink does not want the thread to block forever.
>>> They
>>>> can also be combined to have a "T poll(Timeout)", which is exactly what
>>>> KafkaConsumer did.
>>>> 
>>>> It sounds that you are proposing pattern 2 with something similar to NIO2
>>>> AsynchronousByteChannel[1]. That API would work, except that the
>>> signature
>>>> returning future seems not necessary. If that is the case, a minor change
>>>> on the current FLIP proposal to have "void advance(callback)" should
>>> work.
>>>> And this means the connectors MUST have their internal threads.
>>>> 
>>>> BTW, one thing I am also trying to avoid is pushing users to perform IO
>>> in
>>>> a method like "isBlocked()". If the method is expected to fetch records
>>>> (even if not returning them), naming it something more explicit would
>>> help
>>>> avoid confusion.
>>>> 
>>>> Thanks,
>>>> 
>>>> Jiangjie (Becket) Qin
>>>> 
>>>> [1]
>>>> 
>>> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
>>>> 
>>>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <pi...@data-artisans.com>
>>>> wrote:
>>>> 
>>>>> Hi
>>>>> 
>>>>> Good point with select/epoll, however I do not see how they couldn’t be
>>>>> with Flink if we would like single task in Flink to be single-threaded
>>> (and
>>>>> I believe we should pursue this goal). If your connector blocks on
>>>>> `select`, then it can not process/handle control messages from Flink,
>>> like
>>>>> checkpoints, releasing resources and potentially output flushes. This
>>> would
>>>>> require tight integration between connector and Flink’s main event
>>>>> loop/selects/etc.
>>>>> 
>>>>> Looking at it from other perspective. Let’s assume that we have a
>>>>> connector implemented on top of `select`/`epoll`. In order to integrate
>>> it
>>>>> with Flink’s checkpointing/flushes/resource releasing it will have to be
>>>>> executed in separate thread one way or another. At least if our API will
>>>>> enforce/encourage non blocking implementations with some kind of
>>>>> notifications (`isBlocked()` or `notify()` callback), some connectors
>>> might
>>>>> skip one layer of wapping threads.
>>>>> 
>>>>> Btw, I don’t know if you understand my point. Having only `poll()` and
>>>>> `take()` is not enough for single threaded task. If our source interface
>>>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
>>>>> isBlocked(),`, there is no way to implement single threaded task that
>>> both
>>>>> reads the data from the source connector and can also react to system
>>>>> events. Ok, non blocking `poll()` would allow that, but with busy
>>> looping.
>>>>> 
>>>>> Piotrek
>>>>> 
>>>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com> wrote:
>>>>>> 
>>>>>> Hi Piotrek,
>>>>>> 
>>>>>>> But I don’t see a reason why we should expose both blocking `take()`
>>> and
>>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
>>> engine
>>>>> or
>>>>>> connector) would have to do the same busy
>>>>>>> looping anyway and I think it would be better to have a simpler
>>>>> connector
>>>>>> API (that would solve our problems) and force connectors to comply one
>>>>> way
>>>>>> or another.
>>>>>> 
>>>>>> If we let the block happen inside the connector, the blocking does not
>>>>> have
>>>>>> to be a busy loop. For example, to do the block waiting efficiently,
>>> the
>>>>>> connector can use java NIO selector().select which relies on OS syscall
>>>>>> like epoll[1] instead of busy looping. But if Flink engine blocks
>>> outside
>>>>>> the connector, it pretty much has to do the busy loop. So if there is
>>>>> only
>>>>>> one API to get the element, a blocking getNextElement() makes more
>>> sense.
>>>>>> In any case, we should avoid ambiguity. It has to be crystal clear
>>> about
>>>>>> whether a method is expected to be blocking or non-blocking. Otherwise
>>> it
>>>>>> would be very difficult for Flink engine to do the right thing with the
>>>>>> connectors. At the first glance at getCurrent(), the expected behavior
>>> is
>>>>>> not quite clear.
>>>>>> 
>>>>>> That said, I do agree that functionality wise, poll() and take() kind
>>> of
>>>>>> overlap. But they are actually not quite different from
>>>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the only
>>>>>> difference is that poll() also returns the next record if it is
>>>>> available.
>>>>>> But I agree that the isBlocked() + getNextElement() is more flexible as
>>>>>> users can just check the record availability, but not fetch the next
>>>>>> element.
>>>>>> 
>>>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
>>> (is
>>>>>> that really a thing? I can not think about a real implementation that
>>>>>> enforces such constraints)
>>>>>> Right, it is pretty much a syntax sugar to allow user combine the
>>>>>> check-and-take into one method. It could be achieved with isBlocked() +
>>>>>> getNextElement().
>>>>>> 
>>>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
>>>>>> 
>>>>>> Thanks,
>>>>>> 
>>>>>> Jiangjie (Becket) Qin
>>>>>> 
>>>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
>>> piotr@data-artisans.com>
>>>>>> wrote:
>>>>>> 
>>>>>>> Hi Becket,
>>>>>>> 
>>>>>>> With my proposal, both of your examples would have to be solved by the
>>>>>>> connector and solution to both problems would be the same:
>>>>>>> 
>>>>>>> Pretend that connector is never blocked (`isBlocked() { return
>>>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking fashion
>>>>> (or
>>>>>>> semi blocking with return of control from time to time to allow for
>>>>>>> checkpointing, network flushing and other resource management things
>>> to
>>>>>>> happen in the same main thread). In other words, exactly how you would
>>>>>>> implement `take()` method or how the same source connector would be
>>>>>>> implemented NOW with current source interface. The difference with
>>>>> current
>>>>>>> interface would be only that main loop would be outside of the
>>>>> connector,
>>>>>>> and instead of periodically releasing checkpointing lock, periodically
>>>>>>> `return null;` or `return Optional.empty();` from `getNextElement()`.
>>>>>>> 
>>>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
>>> (is
>>>>>>> that really a thing? I can not think about a real implementation that
>>>>>>> enforces such constraints), we could provide a wrapper that hides the
>>>>> busy
>>>>>>> looping. The same applies how to solve forever blocking readers - we
>>>>> could
>>>>>>> provider another wrapper running the connector in separate thread.
>>>>>>> 
>>>>>>> But I don’t see a reason why we should expose both blocking `take()`
>>> and
>>>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
>>>>> engine or
>>>>>>> connector) would have to do the same busy looping anyway and I think
>>> it
>>>>>>> would be better to have a simpler connector API (that would solve our
>>>>>>> problems) and force connectors to comply one way or another.
>>>>>>> 
>>>>>>> Piotrek
>>>>>>> 
>>>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
>>>>>>>> 
>>>>>>>> Hi Piotr,
>>>>>>>> 
>>>>>>>> I might have misunderstood you proposal. But let me try to explain my
>>>>>>>> concern. I am thinking about the following case:
>>>>>>>> 1. a reader has the following two interfaces,
>>>>>>>> boolean isBlocked()
>>>>>>>> T getNextElement()
>>>>>>>> 2. the implementation of getNextElement() is non-blocking.
>>>>>>>> 3. The reader is thread-less, i.e. it does not have any internal
>>>>> thread.
>>>>>>>> For example, it might just delegate the getNextElement() to a
>>>>>>> queue.poll(),
>>>>>>>> and isBlocked() is just queue.isEmpty().
>>>>>>>> 
>>>>>>>> How can Flink efficiently implement a blocking reading behavior with
>>>>> this
>>>>>>>> reader? Either a tight loop or a backoff interval is needed. Neither
>>> of
>>>>>>>> them is ideal.
>>>>>>>> 
>>>>>>>> Now let's say in the reader mentioned above implements a blocking
>>>>>>>> getNextElement() method. Because there is no internal thread in the
>>>>>>> reader,
>>>>>>>> after isBlocked() returns false. Flink will still have to loop on
>>>>>>>> isBlocked() to check whether the next record is available. If the
>>> next
>>>>>>>> record reaches after 10 min, it is a tight loop for 10 min. You have
>>>>>>>> probably noticed that in this case, even isBlocked() returns a
>>> future,
>>>>>>> that
>>>>>>>> future() will not be completed if Flink does not call some method
>>> from
>>>>>>> the
>>>>>>>> reader, because the reader has no internal thread to complete that
>>>>> future
>>>>>>>> by itself.
>>>>>>>> 
>>>>>>>> Due to the above reasons, a blocking take() API would allow Flink to
>>>>> have
>>>>>>>> an efficient way to read from a reader. There are many ways to wake
>>> up
>>>>>>> the
>>>>>>>> blocking thread when checkpointing is needed depending on the
>>>>>>>> implementation. But I think the poll()/take() API would also work in
>>>>> that
>>>>>>>> case.
>>>>>>>> 
>>>>>>>> Thanks,
>>>>>>>> 
>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>> 
>>>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
>>> piotr@data-artisans.com
>>>>>> 
>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>>> Hi,
>>>>>>>>> 
>>>>>>>>> a)
>>>>>>>>> 
>>>>>>>>>> BTW, regarding the isBlock() method, I have a few more questions.
>>> 21,
>>>>>>> Is
>>>>>>>>> a method isReady() with boolean as a return value
>>>>>>>>>> equivalent? Personally I found it is a little bit confusing in what
>>>>> is
>>>>>>>>> supposed to be returned when the future is completed. 22. if
>>>>>>>>>> the implementation of isBlocked() is optional, how do the callers
>>>>> know
>>>>>>>>> whether the method is properly implemented or not?
>>>>>>>>>> Does not implemented mean it always return a completed future?
>>>>>>>>> 
>>>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an equivalent to
>>>>>>>>> `boolean hasNext()` which in case of “false” provides some kind of a
>>>>>>>>> listener/callback that notifies about presence of next element.
>>> There
>>>>>>> are
>>>>>>>>> some minor details, like `CompletableFuture<?>` has a minimal two
>>>>> state
>>>>>>>>> logic:
>>>>>>>>> 
>>>>>>>>> 1. Future is completed - we have more data
>>>>>>>>> 2. Future not yet completed - we don’t have data now, but we
>>> might/we
>>>>>>> will
>>>>>>>>> have in the future
>>>>>>>>> 
>>>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit more
>>>>>>>>> complicated/dispersed and can lead/encourage `notify()` spam.
>>>>>>>>> 
>>>>>>>>> b)
>>>>>>>>> 
>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>>>>>>> `getNext`
>>>>>>>>> the `getNext` would need return a
>>>>>>>>>> `ElementWithTimestamp` because some sources want to add timestamp
>>> to
>>>>>>>>> every element. IMO, this is not so memory friendly
>>>>>>>>>> so I prefer this design.
>>>>>>>>> 
>>>>>>>>> Guowei I don’t quite understand this. Could you elaborate why
>>> having a
>>>>>>>>> separate `advance()` help?
>>>>>>>>> 
>>>>>>>>> c)
>>>>>>>>> 
>>>>>>>>> Regarding advance/poll/take. What’s the value of having two separate
>>>>>>>>> methods: poll and take? Which one of them should be called and which
>>>>>>>>> implemented? What’s the benefit of having those methods compared to
>>>>>>> having
>>>>>>>>> a one single method `getNextElement()` (or `pollElement() or
>>> whatever
>>>>> we
>>>>>>>>> name it) with following contract:
>>>>>>>>> 
>>>>>>>>> CompletableFuture<?> isBlocked();
>>>>>>>>> 
>>>>>>>>> /**
>>>>>>>>> Return next element - will be called only if `isBlocked()` is
>>>>> completed.
>>>>>>>>> Try to implement it in non blocking fashion, but if that’s
>>> impossible
>>>>> or
>>>>>>>>> you just don’t need the effort, you can block in this method.
>>>>>>>>> */
>>>>>>>>> T getNextElement();
>>>>>>>>> 
>>>>>>>>> I mean, if the connector is implemented non-blockingly, Flink should
>>>>> use
>>>>>>>>> it that way. If it’s not, then `poll()` will `throw new
>>>>>>>>> NotImplementedException()`. Implementing both of them and providing
>>>>>>> both of
>>>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them into a
>>>>>>> single
>>>>>>>>> method call that should preferably (but not necessarily need to) be
>>>>>>>>> non-blocking? It’s not like we are implementing general purpose
>>>>> `Queue`,
>>>>>>>>> which users might want to call either of `poll` or `take`. We would
>>>>>>> always
>>>>>>>>> prefer to call `poll`, but if it’s blocking, then still we have no
>>>>>>> choice,
>>>>>>>>> but to call it and block on it.
>>>>>>>>> 
>>>>>>>>> d)
>>>>>>>>> 
>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
>>> very
>>>>>>>>>> important. But in addition to `Future/poll`, there may be another
>>> way
>>>>>>> to
>>>>>>>>>> achieve this. I think it may be not very memory friendly if every
>>>>>>> advance
>>>>>>>>>> call return a Future.
>>>>>>>>> 
>>>>>>>>> I didn’t want to mention this, to not clog my initial proposal, but
>>>>>>> there
>>>>>>>>> is a simple solution for the problem:
>>>>>>>>> 
>>>>>>>>> public interface SplitReader {
>>>>>>>>> 
>>>>>>>>> (…)
>>>>>>>>> 
>>>>>>>>> CompletableFuture<?> NOT_BLOCKED =
>>>>>>>>> CompletableFuture.completedFuture(null);
>>>>>>>>> 
>>>>>>>>> /**
>>>>>>>>> * Returns a future that will be completed when the page source
>>>>>>> becomes
>>>>>>>>> * unblocked.  If the page source is not blocked, this method
>>> should
>>>>>>>>> return
>>>>>>>>> * {@code NOT_BLOCKED}.
>>>>>>>>> */
>>>>>>>>> default CompletableFuture<?> isBlocked()
>>>>>>>>> {
>>>>>>>>>    return NOT_BLOCKED;
>>>>>>>>> }
>>>>>>>>> 
>>>>>>>>> If we are blocked and we are waiting for the IO, then creating a new
>>>>>>>>> Future is non-issue. Under full throttle/throughput and not blocked
>>>>>>> sources
>>>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve the
>>>>>>> problem.
>>>>>>>>> 
>>>>>>>>> One more remark, non-blocking sources might be a necessity in a
>>> single
>>>>>>>>> threaded model without a checkpointing lock. (Currently when sources
>>>>> are
>>>>>>>>> blocked, they can release checkpointing lock and re-acquire it again
>>>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for checkpoints
>>> to
>>>>>>>>> happen when source is idling. In that case either `notify()` or my
>>>>>>> proposed
>>>>>>>>> `isBlocked()` would allow to avoid busy-looping.
>>>>>>>>> 
>>>>>>>>> Piotrek
>>>>>>>>> 
>>>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
>>>>>>>>>> 
>>>>>>>>>> Hi Thomas,
>>>>>>>>>> 
>>>>>>>>>> The iterator-like API was also the first thing that came to me. But
>>>>> it
>>>>>>>>>> seems a little confusing that hasNext() does not mean "the stream
>>> has
>>>>>>> not
>>>>>>>>>> ended", but means "the next record is ready", which is repurposing
>>>>> the
>>>>>>>>> well
>>>>>>>>>> known meaning of hasNext(). If we follow the hasNext()/next()
>>>>> pattern,
>>>>>>> an
>>>>>>>>>> additional isNextReady() method to indicate whether the next record
>>>>> is
>>>>>>>>>> ready seems more intuitive to me.
>>>>>>>>>> 
>>>>>>>>>> Similarly, in poll()/take() pattern, another method of isDone() is
>>>>>>> needed
>>>>>>>>>> to indicate whether the stream has ended or not.
>>>>>>>>>> 
>>>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
>>>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
>>>>>>> implementation.
>>>>>>>>>> When I am implementing a reader, I could have a couple of choices:
>>>>>>>>>> 
>>>>>>>>>> - A thread-less reader that does not have any internal thread.
>>>>>>>>>> - When poll() is called, the same calling thread will perform a
>>> bunch
>>>>>>>>> of
>>>>>>>>>> IO asynchronously.
>>>>>>>>>> - When take() is called, the same calling thread will perform a
>>>>>>>>> bunch
>>>>>>>>>> of IO and wait until the record is ready.
>>>>>>>>>> - A reader with internal threads performing network IO and put
>>>>> records
>>>>>>>>>> into a buffer.
>>>>>>>>>> - When poll() is called, the calling thread simply reads from the
>>>>>>>>>> buffer and return empty result immediately if there is no record.
>>>>>>>>>> - When take() is called, the calling thread reads from the buffer
>>>>>>>>> and
>>>>>>>>>> block waiting if the buffer is empty.
>>>>>>>>>> 
>>>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady() API, it
>>> is
>>>>>>>>> less
>>>>>>>>>> intuitive for the reader developers to write the thread-less
>>> pattern.
>>>>>>>>>> Although technically speaking one can still do the asynchronous IO
>>> to
>>>>>>>>>> prepare the record in isNextReady(). But it is inexplicit and seems
>>>>>>>>>> somewhat hacky.
>>>>>>>>>> 
>>>>>>>>>> Thanks,
>>>>>>>>>> 
>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org>
>>> wrote:
>>>>>>>>>> 
>>>>>>>>>>> Couple more points regarding discovery:
>>>>>>>>>>> 
>>>>>>>>>>> The proposal mentions that discovery could be outside the
>>> execution
>>>>>>>>> graph.
>>>>>>>>>>> Today, discovered partitions/shards are checkpointed. I believe
>>> that
>>>>>>>>> will
>>>>>>>>>>> also need to be the case in the future, even when discovery and
>>>>>>> reading
>>>>>>>>> are
>>>>>>>>>>> split between different tasks.
>>>>>>>>>>> 
>>>>>>>>>>> For cases such as resharding of a Kinesis stream, the relationship
>>>>>>>>> between
>>>>>>>>>>> splits needs to be considered. Splits cannot be randomly
>>> distributed
>>>>>>>>> over
>>>>>>>>>>> readers in certain situations. An example was mentioned here:
>>>>>>>>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>>>>>>>>>>> 
>>>>>>>>>>> Thomas
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org>
>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Thanks for getting the ball rolling on this!
>>>>>>>>>>>> 
>>>>>>>>>>>> Can the number of splits decrease? Yes, splits can be closed and
>>> go
>>>>>>>>> away.
>>>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing shards
>>>>> will
>>>>>>> be
>>>>>>>>>>>> closed and replaced with a new shard).
>>>>>>>>>>>> 
>>>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive approach
>>>>> would
>>>>>>>>> be
>>>>>>>>>>>> the thread-less IO model (pull based, non-blocking, caller
>>>>> retrieves
>>>>>>>>> new
>>>>>>>>>>>> records when available). The current Kinesis API requires the use
>>>>> of
>>>>>>>>>>>> threads. But that can be internal to the split reader and does
>>> not
>>>>>>> need
>>>>>>>>>>> to
>>>>>>>>>>>> be a source API concern. In fact, that's what we are working on
>>>>> right
>>>>>>>>> now
>>>>>>>>>>>> as improvement to the existing consumer: Each shard consumer
>>> thread
>>>>>>>>> will
>>>>>>>>>>>> push to a queue, the consumer main thread will poll the queue(s).
>>>>> It
>>>>>>> is
>>>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
>>>>>>>>>>>> 
>>>>>>>>>>>> The proposed SplitReader interface would fit the thread-less IO
>>>>>>> model.
>>>>>>>>>>>> Similar to an iterator, we find out if there is a new element
>>>>>>> (hasNext)
>>>>>>>>>>> and
>>>>>>>>>>>> if so, move to it (next()). Separate calls deliver the meta
>>>>>>> information
>>>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer a
>>> timeout
>>>>>>>>>>> option,
>>>>>>>>>>>> so that the caller does not end up in a busy wait. On the other
>>>>>>> hand, a
>>>>>>>>>>>> caller processing multiple splits may want to cycle through fast,
>>>>> to
>>>>>>>>>>>> process elements of other splits as soon as they become
>>> available.
>>>>>>> The
>>>>>>>>>>> nice
>>>>>>>>>>>> thing is that this "split merge" logic can now live in Flink and
>>> be
>>>>>>>>>>>> optimized and shared between different sources.
>>>>>>>>>>>> 
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Thomas
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>> Thanks Aljoscha for this FLIP.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
>>>>>>> very
>>>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
>>> another
>>>>>>> way
>>>>>>>>> to
>>>>>>>>>>>>> achieve this. I think it may be not very memory friendly if
>>> every
>>>>>>>>>>> advance
>>>>>>>>>>>>> call return a Future.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> public interface Listener {
>>>>>>>>>>>>> public void notify();
>>>>>>>>>>>>> }
>>>>>>>>>>>>> 
>>>>>>>>>>>>> public interface SplitReader() {
>>>>>>>>>>>>> /**
>>>>>>>>>>>>> * When there is no element temporarily, this will return
>>> false.
>>>>>>>>>>>>> * When elements is available again splitReader can call
>>>>>>>>>>>>> listener.notify()
>>>>>>>>>>>>> * In addition the frame would check `advance` periodically .
>>>>>>>>>>>>> * Of course advance can always return true and ignore the
>>>>>>>>> listener
>>>>>>>>>>>>> argument for simplicity.
>>>>>>>>>>>>> */
>>>>>>>>>>>>> public boolean advance(Listener listener);
>>>>>>>>>>>>> }
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create all Splits
>>>>> and
>>>>>>>>> how
>>>>>>>>>>>>> to create a SplitReader from a Split. But there is no strategy
>>> for
>>>>>>> the
>>>>>>>>>>> user
>>>>>>>>>>>>> to choose how to assign the splits to the tasks. I think we
>>> could
>>>>>>> add
>>>>>>>>> a
>>>>>>>>>>>>> Enum to let user to choose.
>>>>>>>>>>>>> /**
>>>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
>>>>>>>>>>>>> Location,
>>>>>>>>>>>>> Workload,
>>>>>>>>>>>>> Random,
>>>>>>>>>>>>> Average
>>>>>>>>>>>>> }
>>>>>>>>>>>>> */
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>>>>>>>>> `getNext`
>>>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp` because
>>>>>>> some
>>>>>>>>>>>>> sources want to add timestamp to every element. IMO, this is not
>>>>> so
>>>>>>>>>>> memory
>>>>>>>>>>>>> friendly so I prefer this design.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
>>> 下午6:08写道:
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
>>>>>>> other
>>>>>>>>>>>>>> possible improvements. I have one proposal. Instead of having a
>>>>>>>>> method:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> boolean advance() throws IOException;
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> I would replace it with
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> /*
>>>>>>>>>>>>>> * Return a future, which when completed means that source has
>>>>> more
>>>>>>>>>>> data
>>>>>>>>>>>>>> and getNext() will not block.
>>>>>>>>>>>>>> * If you wish to use benefits of non blocking connectors,
>>> please
>>>>>>>>>>>>>> implement this method appropriately.
>>>>>>>>>>>>>> */
>>>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
>>>>>>>>>>>>>>   return CompletableFuture.completedFuture(null);
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Couple of arguments:
>>>>>>>>>>>>>> 1. I don’t understand the division of work between `advance()`
>>>>> and
>>>>>>>>>>>>>> `getCurrent()`. What should be done in which, especially for
>>>>>>>>> connectors
>>>>>>>>>>>>>> that handle records in batches (like Kafka) and when should you
>>>>>>> call
>>>>>>>>>>>>>> `advance` and when `getCurrent()`.
>>>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow
>>> us
>>>>> in
>>>>>>>>> the
>>>>>>>>>>>>>> future to have asynchronous/non blocking connectors and more
>>>>>>>>>>> efficiently
>>>>>>>>>>>>>> handle large number of blocked threads, without busy waiting.
>>>>> While
>>>>>>>>> at
>>>>>>>>>>> the
>>>>>>>>>>>>>> same time it doesn’t add much complexity, since naive connector
>>>>>>>>>>>>>> implementations can be always blocking.
>>>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread pool of
>>>>> task
>>>>>>>>>>>>>> executors, instead of one thread per task.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Piotrek
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
>>> aljoscha@apache.org
>>>>>> 
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Hi All,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> In order to finally get the ball rolling on the new source
>>>>>>> interface
>>>>>>>>>>>>>> that we have discussed for so long I finally created a FLIP:
>>>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>> 
>>>>>>> 
>>>>> 
>>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
>>> work/discussion
>>>>>>>>> about
>>>>>>>>>>>>>> adding per-partition watermark support to the Kinesis source
>>> and
>>>>>>>>>>> because
>>>>>>>>>>>>>> this would enable generic implementation of event-time
>>> alignment
>>>>>>> for
>>>>>>>>>>> all
>>>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
>>> alignment
>>>>>>>>> part,
>>>>>>>>>>>>>> especially the part about information sharing between
>>> operations
>>>>>>> (I'm
>>>>>>>>>>> not
>>>>>>>>>>>>>> calling it state sharing because state has a special meaning in
>>>>>>>>> Flink).
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Please discuss away!
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Aljoscha
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>> 
>>>>> 
>>> 
>>> 
> 



Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Steven Wu <st...@gmail.com>.
>  And each split has its own (internal) thread for reading from Kafka and
putting messages in an internal queue to pull from. This is similar to how
the current Kafka source is implemented, which has a separate fetcher
thread.

Aljoscha, in kafka case, one split may contain multiple kafka partitions,
right?

On Thu, Nov 15, 2018 at 3:43 AM Aljoscha Krettek <al...@apache.org>
wrote:

> Hi,
>
> I thought I had sent this mail a while ago but I must have forgotten to
> send it.
>
> There is another thing we should consider for splits: the range of
> timestamps that it can contain. For example, the splits of a file source
> would know what the minimum and maximum timestamp in the splits is,
> roughly. For infinite splits, such as Kafka partitions, the minimum would
> be meaningful but the maximum would be +Inf. If the splits expose the
> interval of time that they contain the readers, or the component that
> manages the readers can make decisions about which splits to forward and
> read first. And it can also influence the minimum watermark that a reader
> forwards: it should never emit a watermark if it knows there are splits to
> read that have a lower minimum timestamp. I think it should be as easy as
> adding a minimumTimestamp()/maximumTimestamp() method pair to the split
> interface.
>
> Another thing we need to resolve is the actual reader interface. I see
> there has been some good discussion but I don't know if we have consensus.
> We should try and see how specific sources could be implemented with the
> new interface. For example, for Kafka I think we need to have N+1 threads
> per task (where N is the number of splits that a task is reading from). On
> thread is responsible for reading from the splits. And each split has its
> own (internal) thread for reading from Kafka and putting messages in an
> internal queue to pull from. This is similar to how the current Kafka
> source is implemented, which has a separate fetcher thread. The reason for
> this split is that we always need to try reading from Kafka to keep the
> throughput up. In the current implementation the internal queue (or
> handover) limits the read rate of the reader threads.
>
> @Thomas, what do you think this would look like for Kinesis?
>
> Best,
> Aljoscha
>
> > On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com> wrote:
> >
> > Hi Piotrek,
> >
> > Thanks a lot for the detailed reply. All makes sense to me.
> >
> > WRT the confusion between advance() / getCurrent(), do you think it would
> > help if we combine them and have something like:
> >
> > CompletableFuture<T> getNext();
> > long getWatermark();
> > long getCurrentTimestamp();
> >
> > Cheers,
> >
> > Jiangjie (Becket) Qin
> >
> > On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <pi...@data-artisans.com>
> > wrote:
> >
> >> Hi,
> >>
> >> Thanks again for the detailed answer :) Sorry for responding with a
> delay.
> >>
> >>> Completely agree that in pattern 2, having a callback is necessary for
> >> that
> >>> single thread outside of the connectors. And the connectors MUST have
> >>> internal threads.
> >>
> >> Yes, this thread will have to exists somewhere. In pattern 2 it exists
> in
> >> the connector (at least from the perspective of the Flink execution
> >> engine). In pattern 1 it exists inside the Flink execution engine. With
> >> completely blocking connectors, like simple reading from files, both of
> >> those approaches are basically the same. The difference is when user
> >> implementing Flink source is already working with a non blocking code
> with
> >> some internal threads. In this case, pattern 1 would result in "double
> >> thread wrapping”, while pattern 2 would allow to skip one layer of
> >> indirection.
> >>
> >>> If we go that way, we should have something like "void
> >>> poll(Callback) / void advance(callback)". I am curious how would
> >>> CompletableFuture work here, though. If 10 readers returns 10
> completable
> >>> futures, will there be 10 additional threads (so 20 threads in total)
> >>> blocking waiting on them? Or will there be a single thread busy loop
> >>> checking around?
> >>
> >> To be honest, I haven’t thought this completely through and I haven’t
> >> tested/POC’ed it. Having said that, I can think of at least couple of
> >> solutions. First is something like this:
> >>
> >>
> >>
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> >> <
> >>
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> >>>
> >>
> >> Line:
> >>
> >>                                `blocked = split.process();`
> >>
> >> Is where the execution goes into to the task/sources. This is where the
> >> returned future is handled:
> >>
> >>                                blocked.addListener(() -> {
> >>                                    blockedSplits.remove(split);
> >>                                    // reset the level priority to
> prevent
> >> previously-blocked splits from starving existing splits
> >>                                    split.resetLevelPriority();
> >>                                    waitingSplits.offer(split);
> >>                                }, executor);
> >>
> >> Fundamentally callbacks and Futures are more or less interchangeable You
> >> can always wrap one into another (creating a callback that completes a
> >> future and attach a callback once future completes). In this case the
> >> difference for me is mostly:
> >> - api with passing callback allows the callback to be fired multiple
> times
> >> and to fire it even if the connector is not blocked. This is what I
> meant
> >> by saying that api `CompletableFuture<?> isBlocked()` is a bit simpler.
> >> Connector can only return either “I’m not blocked” or “I’m blocked and I
> >> will tell you only once when I’m not blocked anymore”.
> >>
> >> But this is not the most important thing for me here. For me important
> >> thing is to try our best to make Flink task’s control and execution
> single
> >> threaded. For that both callback and future APIs should work the same.
> >>
> >>> WRT pattern 1, a single blocking take() API should just work. The good
> >>> thing is that a blocking read API is usually simpler to implement.
> >>
> >> Yes, they are easier to implement (especially if you are not the one
> that
> >> have to deal with the additional threading required around them ;) ).
> But
> >> to answer this issue, if we choose pattern 2, we can always provide a
> >> proxy/wrapper that would using the internal thread implement the
> >> non-blocking API while exposing blocking API to the user. It would
> >> implement pattern 2 for the user exposing to him pattern 1. In other
> words
> >> implementing pattern 1 in pattern 2 paradigm, while making it possible
> to
> >> implement pure pattern 2 connectors.
> >>
> >>> BTW, one thing I am also trying to avoid is pushing users to perform IO
> >> in
> >>> a method like "isBlocked()". If the method is expected to fetch records
> >>> (even if not returning them), naming it something more explicit would
> >> help
> >>> avoid confusion.
> >>
> >> If we choose so, we could rework it into something like:
> >>
> >> CompletableFuture<?> advance()
> >> T getCurrent();
> >> Watermark getCurrentWatermark()
> >>
> >> But as I wrote before, this is more confusing to me for the exact
> reasons
> >> you mentioned :) I would be confused what should be done in `adanvce()`
> and
> >> what in `getCurrent()`. However, again this naming issue is not that
> >> important to me and probably is matter of taste/personal preferences.
> >>
> >> Piotrek
> >>
> >>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com> wrote:
> >>>
> >>> Hi Piotrek,
> >>>
> >>> Thanks for the explanation. We are probably talking about the same
> thing
> >>> but in different ways. To clarify a little bit, I think there are two
> >>> patterns to read from a connector.
> >>>
> >>> Pattern 1: Thread-less connector with a blocking read API. Outside of
> the
> >>> connector, there is one IO thread per reader, doing blocking read. An
> >>> additional thread will interact with all the IO threads.
> >>> Pattern 2: Connector with internal thread(s) and non-blocking API.
> >> Outside
> >>> of the connector, there is one thread for ALL readers, doing IO relying
> >> on
> >>> notification callbacks in the reader.
> >>>
> >>> In both patterns, there must be at least one thread per connector,
> either
> >>> inside (created by connector writers) or outside (created by Flink) of
> >> the
> >>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total, to
> make
> >>> sure that 1 thread is fully non-blocking.
> >>>
> >>>> Btw, I don’t know if you understand my point. Having only `poll()` and
> >>> `take()` is not enough for single threaded task. If our source
> interface
> >>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
> >>> isBlocked(),`, there is no way to implement single threaded task that
> >> both
> >>> reads the data from the source connector and can also react to system
> >>> events. Ok, non >blocking `poll()` would allow that, but with busy
> >> looping.
> >>>
> >>> Completely agree that in pattern 2, having a callback is necessary for
> >> that
> >>> single thread outside of the connectors. And the connectors MUST have
> >>> internal threads. If we go that way, we should have something like
> "void
> >>> poll(Callback) / void advance(callback)". I am curious how would
> >>> CompletableFuture work here, though. If 10 readers returns 10
> completable
> >>> futures, will there be 10 additional threads (so 20 threads in total)
> >>> blocking waiting on them? Or will there be a single thread busy loop
> >>> checking around?
> >>>
> >>> WRT pattern 1, a single blocking take() API should just work. The good
> >>> thing is that a blocking read API is usually simpler to implement. An
> >>> additional non-blocking "T poll()" method here is indeed optional and
> >> could
> >>> be used in cases like Flink does not want the thread to block forever.
> >> They
> >>> can also be combined to have a "T poll(Timeout)", which is exactly what
> >>> KafkaConsumer did.
> >>>
> >>> It sounds that you are proposing pattern 2 with something similar to
> NIO2
> >>> AsynchronousByteChannel[1]. That API would work, except that the
> >> signature
> >>> returning future seems not necessary. If that is the case, a minor
> change
> >>> on the current FLIP proposal to have "void advance(callback)" should
> >> work.
> >>> And this means the connectors MUST have their internal threads.
> >>>
> >>> BTW, one thing I am also trying to avoid is pushing users to perform IO
> >> in
> >>> a method like "isBlocked()". If the method is expected to fetch records
> >>> (even if not returning them), naming it something more explicit would
> >> help
> >>> avoid confusion.
> >>>
> >>> Thanks,
> >>>
> >>> Jiangjie (Becket) Qin
> >>>
> >>> [1]
> >>>
> >>
> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
> >>>
> >>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <
> piotr@data-artisans.com>
> >>> wrote:
> >>>
> >>>> Hi
> >>>>
> >>>> Good point with select/epoll, however I do not see how they couldn’t
> be
> >>>> with Flink if we would like single task in Flink to be single-threaded
> >> (and
> >>>> I believe we should pursue this goal). If your connector blocks on
> >>>> `select`, then it can not process/handle control messages from Flink,
> >> like
> >>>> checkpoints, releasing resources and potentially output flushes. This
> >> would
> >>>> require tight integration between connector and Flink’s main event
> >>>> loop/selects/etc.
> >>>>
> >>>> Looking at it from other perspective. Let’s assume that we have a
> >>>> connector implemented on top of `select`/`epoll`. In order to
> integrate
> >> it
> >>>> with Flink’s checkpointing/flushes/resource releasing it will have to
> be
> >>>> executed in separate thread one way or another. At least if our API
> will
> >>>> enforce/encourage non blocking implementations with some kind of
> >>>> notifications (`isBlocked()` or `notify()` callback), some connectors
> >> might
> >>>> skip one layer of wapping threads.
> >>>>
> >>>> Btw, I don’t know if you understand my point. Having only `poll()` and
> >>>> `take()` is not enough for single threaded task. If our source
> interface
> >>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
> >>>> isBlocked(),`, there is no way to implement single threaded task that
> >> both
> >>>> reads the data from the source connector and can also react to system
> >>>> events. Ok, non blocking `poll()` would allow that, but with busy
> >> looping.
> >>>>
> >>>> Piotrek
> >>>>
> >>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com> wrote:
> >>>>>
> >>>>> Hi Piotrek,
> >>>>>
> >>>>>> But I don’t see a reason why we should expose both blocking `take()`
> >> and
> >>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
> >> engine
> >>>> or
> >>>>> connector) would have to do the same busy
> >>>>>> looping anyway and I think it would be better to have a simpler
> >>>> connector
> >>>>> API (that would solve our problems) and force connectors to comply
> one
> >>>> way
> >>>>> or another.
> >>>>>
> >>>>> If we let the block happen inside the connector, the blocking does
> not
> >>>> have
> >>>>> to be a busy loop. For example, to do the block waiting efficiently,
> >> the
> >>>>> connector can use java NIO selector().select which relies on OS
> syscall
> >>>>> like epoll[1] instead of busy looping. But if Flink engine blocks
> >> outside
> >>>>> the connector, it pretty much has to do the busy loop. So if there is
> >>>> only
> >>>>> one API to get the element, a blocking getNextElement() makes more
> >> sense.
> >>>>> In any case, we should avoid ambiguity. It has to be crystal clear
> >> about
> >>>>> whether a method is expected to be blocking or non-blocking.
> Otherwise
> >> it
> >>>>> would be very difficult for Flink engine to do the right thing with
> the
> >>>>> connectors. At the first glance at getCurrent(), the expected
> behavior
> >> is
> >>>>> not quite clear.
> >>>>>
> >>>>> That said, I do agree that functionality wise, poll() and take() kind
> >> of
> >>>>> overlap. But they are actually not quite different from
> >>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the only
> >>>>> difference is that poll() also returns the next record if it is
> >>>> available.
> >>>>> But I agree that the isBlocked() + getNextElement() is more flexible
> as
> >>>>> users can just check the record availability, but not fetch the next
> >>>>> element.
> >>>>>
> >>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
> >> (is
> >>>>> that really a thing? I can not think about a real implementation that
> >>>>> enforces such constraints)
> >>>>> Right, it is pretty much a syntax sugar to allow user combine the
> >>>>> check-and-take into one method. It could be achieved with
> isBlocked() +
> >>>>> getNextElement().
> >>>>>
> >>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
> >>>>>
> >>>>> Thanks,
> >>>>>
> >>>>> Jiangjie (Becket) Qin
> >>>>>
> >>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
> >> piotr@data-artisans.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Hi Becket,
> >>>>>>
> >>>>>> With my proposal, both of your examples would have to be solved by
> the
> >>>>>> connector and solution to both problems would be the same:
> >>>>>>
> >>>>>> Pretend that connector is never blocked (`isBlocked() { return
> >>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking
> fashion
> >>>> (or
> >>>>>> semi blocking with return of control from time to time to allow for
> >>>>>> checkpointing, network flushing and other resource management things
> >> to
> >>>>>> happen in the same main thread). In other words, exactly how you
> would
> >>>>>> implement `take()` method or how the same source connector would be
> >>>>>> implemented NOW with current source interface. The difference with
> >>>> current
> >>>>>> interface would be only that main loop would be outside of the
> >>>> connector,
> >>>>>> and instead of periodically releasing checkpointing lock,
> periodically
> >>>>>> `return null;` or `return Optional.empty();` from
> `getNextElement()`.
> >>>>>>
> >>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
> >> (is
> >>>>>> that really a thing? I can not think about a real implementation
> that
> >>>>>> enforces such constraints), we could provide a wrapper that hides
> the
> >>>> busy
> >>>>>> looping. The same applies how to solve forever blocking readers - we
> >>>> could
> >>>>>> provider another wrapper running the connector in separate thread.
> >>>>>>
> >>>>>> But I don’t see a reason why we should expose both blocking `take()`
> >> and
> >>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
> >>>> engine or
> >>>>>> connector) would have to do the same busy looping anyway and I think
> >> it
> >>>>>> would be better to have a simpler connector API (that would solve
> our
> >>>>>> problems) and force connectors to comply one way or another.
> >>>>>>
> >>>>>> Piotrek
> >>>>>>
> >>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
> >>>>>>>
> >>>>>>> Hi Piotr,
> >>>>>>>
> >>>>>>> I might have misunderstood you proposal. But let me try to explain
> my
> >>>>>>> concern. I am thinking about the following case:
> >>>>>>> 1. a reader has the following two interfaces,
> >>>>>>> boolean isBlocked()
> >>>>>>> T getNextElement()
> >>>>>>> 2. the implementation of getNextElement() is non-blocking.
> >>>>>>> 3. The reader is thread-less, i.e. it does not have any internal
> >>>> thread.
> >>>>>>> For example, it might just delegate the getNextElement() to a
> >>>>>> queue.poll(),
> >>>>>>> and isBlocked() is just queue.isEmpty().
> >>>>>>>
> >>>>>>> How can Flink efficiently implement a blocking reading behavior
> with
> >>>> this
> >>>>>>> reader? Either a tight loop or a backoff interval is needed.
> Neither
> >> of
> >>>>>>> them is ideal.
> >>>>>>>
> >>>>>>> Now let's say in the reader mentioned above implements a blocking
> >>>>>>> getNextElement() method. Because there is no internal thread in the
> >>>>>> reader,
> >>>>>>> after isBlocked() returns false. Flink will still have to loop on
> >>>>>>> isBlocked() to check whether the next record is available. If the
> >> next
> >>>>>>> record reaches after 10 min, it is a tight loop for 10 min. You
> have
> >>>>>>> probably noticed that in this case, even isBlocked() returns a
> >> future,
> >>>>>> that
> >>>>>>> future() will not be completed if Flink does not call some method
> >> from
> >>>>>> the
> >>>>>>> reader, because the reader has no internal thread to complete that
> >>>> future
> >>>>>>> by itself.
> >>>>>>>
> >>>>>>> Due to the above reasons, a blocking take() API would allow Flink
> to
> >>>> have
> >>>>>>> an efficient way to read from a reader. There are many ways to wake
> >> up
> >>>>>> the
> >>>>>>> blocking thread when checkpointing is needed depending on the
> >>>>>>> implementation. But I think the poll()/take() API would also work
> in
> >>>> that
> >>>>>>> case.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>>
> >>>>>>> Jiangjie (Becket) Qin
> >>>>>>>
> >>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
> >> piotr@data-artisans.com
> >>>>>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi,
> >>>>>>>>
> >>>>>>>> a)
> >>>>>>>>
> >>>>>>>>> BTW, regarding the isBlock() method, I have a few more questions.
> >> 21,
> >>>>>> Is
> >>>>>>>> a method isReady() with boolean as a return value
> >>>>>>>>> equivalent? Personally I found it is a little bit confusing in
> what
> >>>> is
> >>>>>>>> supposed to be returned when the future is completed. 22. if
> >>>>>>>>> the implementation of isBlocked() is optional, how do the callers
> >>>> know
> >>>>>>>> whether the method is properly implemented or not?
> >>>>>>>>> Does not implemented mean it always return a completed future?
> >>>>>>>>
> >>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an equivalent
> to
> >>>>>>>> `boolean hasNext()` which in case of “false” provides some kind
> of a
> >>>>>>>> listener/callback that notifies about presence of next element.
> >> There
> >>>>>> are
> >>>>>>>> some minor details, like `CompletableFuture<?>` has a minimal two
> >>>> state
> >>>>>>>> logic:
> >>>>>>>>
> >>>>>>>> 1. Future is completed - we have more data
> >>>>>>>> 2. Future not yet completed - we don’t have data now, but we
> >> might/we
> >>>>>> will
> >>>>>>>> have in the future
> >>>>>>>>
> >>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit more
> >>>>>>>> complicated/dispersed and can lead/encourage `notify()` spam.
> >>>>>>>>
> >>>>>>>> b)
> >>>>>>>>
> >>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> >>>>>> `getNext`
> >>>>>>>> the `getNext` would need return a
> >>>>>>>>> `ElementWithTimestamp` because some sources want to add timestamp
> >> to
> >>>>>>>> every element. IMO, this is not so memory friendly
> >>>>>>>>> so I prefer this design.
> >>>>>>>>
> >>>>>>>> Guowei I don’t quite understand this. Could you elaborate why
> >> having a
> >>>>>>>> separate `advance()` help?
> >>>>>>>>
> >>>>>>>> c)
> >>>>>>>>
> >>>>>>>> Regarding advance/poll/take. What’s the value of having two
> separate
> >>>>>>>> methods: poll and take? Which one of them should be called and
> which
> >>>>>>>> implemented? What’s the benefit of having those methods compared
> to
> >>>>>> having
> >>>>>>>> a one single method `getNextElement()` (or `pollElement() or
> >> whatever
> >>>> we
> >>>>>>>> name it) with following contract:
> >>>>>>>>
> >>>>>>>> CompletableFuture<?> isBlocked();
> >>>>>>>>
> >>>>>>>> /**
> >>>>>>>> Return next element - will be called only if `isBlocked()` is
> >>>> completed.
> >>>>>>>> Try to implement it in non blocking fashion, but if that’s
> >> impossible
> >>>> or
> >>>>>>>> you just don’t need the effort, you can block in this method.
> >>>>>>>> */
> >>>>>>>> T getNextElement();
> >>>>>>>>
> >>>>>>>> I mean, if the connector is implemented non-blockingly, Flink
> should
> >>>> use
> >>>>>>>> it that way. If it’s not, then `poll()` will `throw new
> >>>>>>>> NotImplementedException()`. Implementing both of them and
> providing
> >>>>>> both of
> >>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them into
> a
> >>>>>> single
> >>>>>>>> method call that should preferably (but not necessarily need to)
> be
> >>>>>>>> non-blocking? It’s not like we are implementing general purpose
> >>>> `Queue`,
> >>>>>>>> which users might want to call either of `poll` or `take`. We
> would
> >>>>>> always
> >>>>>>>> prefer to call `poll`, but if it’s blocking, then still we have no
> >>>>>> choice,
> >>>>>>>> but to call it and block on it.
> >>>>>>>>
> >>>>>>>> d)
> >>>>>>>>
> >>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
> >> very
> >>>>>>>>> important. But in addition to `Future/poll`, there may be another
> >> way
> >>>>>> to
> >>>>>>>>> achieve this. I think it may be not very memory friendly if every
> >>>>>> advance
> >>>>>>>>> call return a Future.
> >>>>>>>>
> >>>>>>>> I didn’t want to mention this, to not clog my initial proposal,
> but
> >>>>>> there
> >>>>>>>> is a simple solution for the problem:
> >>>>>>>>
> >>>>>>>> public interface SplitReader {
> >>>>>>>>
> >>>>>>>> (…)
> >>>>>>>>
> >>>>>>>> CompletableFuture<?> NOT_BLOCKED =
> >>>>>>>> CompletableFuture.completedFuture(null);
> >>>>>>>>
> >>>>>>>> /**
> >>>>>>>>  * Returns a future that will be completed when the page source
> >>>>>> becomes
> >>>>>>>>  * unblocked.  If the page source is not blocked, this method
> >> should
> >>>>>>>> return
> >>>>>>>>  * {@code NOT_BLOCKED}.
> >>>>>>>>  */
> >>>>>>>> default CompletableFuture<?> isBlocked()
> >>>>>>>> {
> >>>>>>>>     return NOT_BLOCKED;
> >>>>>>>> }
> >>>>>>>>
> >>>>>>>> If we are blocked and we are waiting for the IO, then creating a
> new
> >>>>>>>> Future is non-issue. Under full throttle/throughput and not
> blocked
> >>>>>> sources
> >>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve the
> >>>>>> problem.
> >>>>>>>>
> >>>>>>>> One more remark, non-blocking sources might be a necessity in a
> >> single
> >>>>>>>> threaded model without a checkpointing lock. (Currently when
> sources
> >>>> are
> >>>>>>>> blocked, they can release checkpointing lock and re-acquire it
> again
> >>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for
> checkpoints
> >> to
> >>>>>>>> happen when source is idling. In that case either `notify()` or my
> >>>>>> proposed
> >>>>>>>> `isBlocked()` would allow to avoid busy-looping.
> >>>>>>>>
> >>>>>>>> Piotrek
> >>>>>>>>
> >>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com>
> wrote:
> >>>>>>>>>
> >>>>>>>>> Hi Thomas,
> >>>>>>>>>
> >>>>>>>>> The iterator-like API was also the first thing that came to me.
> But
> >>>> it
> >>>>>>>>> seems a little confusing that hasNext() does not mean "the stream
> >> has
> >>>>>> not
> >>>>>>>>> ended", but means "the next record is ready", which is
> repurposing
> >>>> the
> >>>>>>>> well
> >>>>>>>>> known meaning of hasNext(). If we follow the hasNext()/next()
> >>>> pattern,
> >>>>>> an
> >>>>>>>>> additional isNextReady() method to indicate whether the next
> record
> >>>> is
> >>>>>>>>> ready seems more intuitive to me.
> >>>>>>>>>
> >>>>>>>>> Similarly, in poll()/take() pattern, another method of isDone()
> is
> >>>>>> needed
> >>>>>>>>> to indicate whether the stream has ended or not.
> >>>>>>>>>
> >>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
> >>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
> >>>>>> implementation.
> >>>>>>>>> When I am implementing a reader, I could have a couple of
> choices:
> >>>>>>>>>
> >>>>>>>>> - A thread-less reader that does not have any internal thread.
> >>>>>>>>> - When poll() is called, the same calling thread will perform a
> >> bunch
> >>>>>>>> of
> >>>>>>>>>  IO asynchronously.
> >>>>>>>>>  - When take() is called, the same calling thread will perform a
> >>>>>>>> bunch
> >>>>>>>>>  of IO and wait until the record is ready.
> >>>>>>>>> - A reader with internal threads performing network IO and put
> >>>> records
> >>>>>>>>> into a buffer.
> >>>>>>>>>  - When poll() is called, the calling thread simply reads from
> the
> >>>>>>>>>  buffer and return empty result immediately if there is no
> record.
> >>>>>>>>>  - When take() is called, the calling thread reads from the
> buffer
> >>>>>>>> and
> >>>>>>>>>  block waiting if the buffer is empty.
> >>>>>>>>>
> >>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady() API,
> it
> >> is
> >>>>>>>> less
> >>>>>>>>> intuitive for the reader developers to write the thread-less
> >> pattern.
> >>>>>>>>> Although technically speaking one can still do the asynchronous
> IO
> >> to
> >>>>>>>>> prepare the record in isNextReady(). But it is inexplicit and
> seems
> >>>>>>>>> somewhat hacky.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>>
> >>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org>
> >> wrote:
> >>>>>>>>>
> >>>>>>>>>> Couple more points regarding discovery:
> >>>>>>>>>>
> >>>>>>>>>> The proposal mentions that discovery could be outside the
> >> execution
> >>>>>>>> graph.
> >>>>>>>>>> Today, discovered partitions/shards are checkpointed. I believe
> >> that
> >>>>>>>> will
> >>>>>>>>>> also need to be the case in the future, even when discovery and
> >>>>>> reading
> >>>>>>>> are
> >>>>>>>>>> split between different tasks.
> >>>>>>>>>>
> >>>>>>>>>> For cases such as resharding of a Kinesis stream, the
> relationship
> >>>>>>>> between
> >>>>>>>>>> splits needs to be considered. Splits cannot be randomly
> >> distributed
> >>>>>>>> over
> >>>>>>>>>> readers in certain situations. An example was mentioned here:
> >>>>>>>>>>
> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >>>>>>>>>>
> >>>>>>>>>> Thomas
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org>
> >> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Thanks for getting the ball rolling on this!
> >>>>>>>>>>>
> >>>>>>>>>>> Can the number of splits decrease? Yes, splits can be closed
> and
> >> go
> >>>>>>>> away.
> >>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing shards
> >>>> will
> >>>>>> be
> >>>>>>>>>>> closed and replaced with a new shard).
> >>>>>>>>>>>
> >>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive approach
> >>>> would
> >>>>>>>> be
> >>>>>>>>>>> the thread-less IO model (pull based, non-blocking, caller
> >>>> retrieves
> >>>>>>>> new
> >>>>>>>>>>> records when available). The current Kinesis API requires the
> use
> >>>> of
> >>>>>>>>>>> threads. But that can be internal to the split reader and does
> >> not
> >>>>>> need
> >>>>>>>>>> to
> >>>>>>>>>>> be a source API concern. In fact, that's what we are working on
> >>>> right
> >>>>>>>> now
> >>>>>>>>>>> as improvement to the existing consumer: Each shard consumer
> >> thread
> >>>>>>>> will
> >>>>>>>>>>> push to a queue, the consumer main thread will poll the
> queue(s).
> >>>> It
> >>>>>> is
> >>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
> >>>>>>>>>>>
> >>>>>>>>>>> The proposed SplitReader interface would fit the thread-less IO
> >>>>>> model.
> >>>>>>>>>>> Similar to an iterator, we find out if there is a new element
> >>>>>> (hasNext)
> >>>>>>>>>> and
> >>>>>>>>>>> if so, move to it (next()). Separate calls deliver the meta
> >>>>>> information
> >>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer a
> >> timeout
> >>>>>>>>>> option,
> >>>>>>>>>>> so that the caller does not end up in a busy wait. On the other
> >>>>>> hand, a
> >>>>>>>>>>> caller processing multiple splits may want to cycle through
> fast,
> >>>> to
> >>>>>>>>>>> process elements of other splits as soon as they become
> >> available.
> >>>>>> The
> >>>>>>>>>> nice
> >>>>>>>>>>> thing is that this "split merge" logic can now live in Flink
> and
> >> be
> >>>>>>>>>>> optimized and shared between different sources.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>> Thomas
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <guowei.mgw@gmail.com
> >
> >>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hi,
> >>>>>>>>>>>> Thanks Aljoscha for this FLIP.
> >>>>>>>>>>>>
> >>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source
> is
> >>>>>> very
> >>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
> >> another
> >>>>>> way
> >>>>>>>> to
> >>>>>>>>>>>> achieve this. I think it may be not very memory friendly if
> >> every
> >>>>>>>>>> advance
> >>>>>>>>>>>> call return a Future.
> >>>>>>>>>>>>
> >>>>>>>>>>>> public interface Listener {
> >>>>>>>>>>>> public void notify();
> >>>>>>>>>>>> }
> >>>>>>>>>>>>
> >>>>>>>>>>>> public interface SplitReader() {
> >>>>>>>>>>>> /**
> >>>>>>>>>>>>  * When there is no element temporarily, this will return
> >> false.
> >>>>>>>>>>>>  * When elements is available again splitReader can call
> >>>>>>>>>>>> listener.notify()
> >>>>>>>>>>>>  * In addition the frame would check `advance` periodically .
> >>>>>>>>>>>>  * Of course advance can always return true and ignore the
> >>>>>>>> listener
> >>>>>>>>>>>> argument for simplicity.
> >>>>>>>>>>>>  */
> >>>>>>>>>>>> public boolean advance(Listener listener);
> >>>>>>>>>>>> }
> >>>>>>>>>>>>
> >>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create all
> Splits
> >>>> and
> >>>>>>>> how
> >>>>>>>>>>>> to create a SplitReader from a Split. But there is no strategy
> >> for
> >>>>>> the
> >>>>>>>>>> user
> >>>>>>>>>>>> to choose how to assign the splits to the tasks. I think we
> >> could
> >>>>>> add
> >>>>>>>> a
> >>>>>>>>>>>> Enum to let user to choose.
> >>>>>>>>>>>> /**
> >>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
> >>>>>>>>>>>> Location,
> >>>>>>>>>>>> Workload,
> >>>>>>>>>>>> Random,
> >>>>>>>>>>>> Average
> >>>>>>>>>>>> }
> >>>>>>>>>>>> */
> >>>>>>>>>>>>
> >>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> >>>>>>>> `getNext`
> >>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp`
> because
> >>>>>> some
> >>>>>>>>>>>> sources want to add timestamp to every element. IMO, this is
> not
> >>>> so
> >>>>>>>>>> memory
> >>>>>>>>>>>> friendly so I prefer this design.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks
> >>>>>>>>>>>>
> >>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
> >> 下午6:08写道:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot
> of
> >>>>>> other
> >>>>>>>>>>>>> possible improvements. I have one proposal. Instead of
> having a
> >>>>>>>> method:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> boolean advance() throws IOException;
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I would replace it with
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> /*
> >>>>>>>>>>>>> * Return a future, which when completed means that source has
> >>>> more
> >>>>>>>>>> data
> >>>>>>>>>>>>> and getNext() will not block.
> >>>>>>>>>>>>> * If you wish to use benefits of non blocking connectors,
> >> please
> >>>>>>>>>>>>> implement this method appropriately.
> >>>>>>>>>>>>> */
> >>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
> >>>>>>>>>>>>>    return CompletableFuture.completedFuture(null);
> >>>>>>>>>>>>> }
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Couple of arguments:
> >>>>>>>>>>>>> 1. I don’t understand the division of work between
> `advance()`
> >>>> and
> >>>>>>>>>>>>> `getCurrent()`. What should be done in which, especially for
> >>>>>>>> connectors
> >>>>>>>>>>>>> that handle records in batches (like Kafka) and when should
> you
> >>>>>> call
> >>>>>>>>>>>>> `advance` and when `getCurrent()`.
> >>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow
> >> us
> >>>> in
> >>>>>>>> the
> >>>>>>>>>>>>> future to have asynchronous/non blocking connectors and more
> >>>>>>>>>> efficiently
> >>>>>>>>>>>>> handle large number of blocked threads, without busy waiting.
> >>>> While
> >>>>>>>> at
> >>>>>>>>>> the
> >>>>>>>>>>>>> same time it doesn’t add much complexity, since naive
> connector
> >>>>>>>>>>>>> implementations can be always blocking.
> >>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread pool
> of
> >>>> task
> >>>>>>>>>>>>> executors, instead of one thread per task.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Piotrek
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
> >> aljoscha@apache.org
> >>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi All,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> In order to finally get the ball rolling on the new source
> >>>>>> interface
> >>>>>>>>>>>>> that we have discussed for so long I finally created a FLIP:
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
> >> work/discussion
> >>>>>>>> about
> >>>>>>>>>>>>> adding per-partition watermark support to the Kinesis source
> >> and
> >>>>>>>>>> because
> >>>>>>>>>>>>> this would enable generic implementation of event-time
> >> alignment
> >>>>>> for
> >>>>>>>>>> all
> >>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
> >> alignment
> >>>>>>>> part,
> >>>>>>>>>>>>> especially the part about information sharing between
> >> operations
> >>>>>> (I'm
> >>>>>>>>>> not
> >>>>>>>>>>>>> calling it state sharing because state has a special meaning
> in
> >>>>>>>> Flink).
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Please discuss away!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Aljoscha
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>>
> >>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Aljoscha Krettek <al...@apache.org>.
Hi,

I thought I had sent this mail a while ago but I must have forgotten to send it.

There is another thing we should consider for splits: the range of timestamps that it can contain. For example, the splits of a file source would know what the minimum and maximum timestamp in the splits is, roughly. For infinite splits, such as Kafka partitions, the minimum would be meaningful but the maximum would be +Inf. If the splits expose the interval of time that they contain the readers, or the component that manages the readers can make decisions about which splits to forward and read first. And it can also influence the minimum watermark that a reader forwards: it should never emit a watermark if it knows there are splits to read that have a lower minimum timestamp. I think it should be as easy as adding a minimumTimestamp()/maximumTimestamp() method pair to the split interface.

Another thing we need to resolve is the actual reader interface. I see there has been some good discussion but I don't know if we have consensus. We should try and see how specific sources could be implemented with the new interface. For example, for Kafka I think we need to have N+1 threads per task (where N is the number of splits that a task is reading from). On thread is responsible for reading from the splits. And each split has its own (internal) thread for reading from Kafka and putting messages in an internal queue to pull from. This is similar to how the current Kafka source is implemented, which has a separate fetcher thread. The reason for this split is that we always need to try reading from Kafka to keep the throughput up. In the current implementation the internal queue (or handover) limits the read rate of the reader threads.

@Thomas, what do you think this would look like for Kinesis?

Best,
Aljoscha

> On 15. Nov 2018, at 03:56, Becket Qin <be...@gmail.com> wrote:
> 
> Hi Piotrek,
> 
> Thanks a lot for the detailed reply. All makes sense to me.
> 
> WRT the confusion between advance() / getCurrent(), do you think it would
> help if we combine them and have something like:
> 
> CompletableFuture<T> getNext();
> long getWatermark();
> long getCurrentTimestamp();
> 
> Cheers,
> 
> Jiangjie (Becket) Qin
> 
> On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <pi...@data-artisans.com>
> wrote:
> 
>> Hi,
>> 
>> Thanks again for the detailed answer :) Sorry for responding with a delay.
>> 
>>> Completely agree that in pattern 2, having a callback is necessary for
>> that
>>> single thread outside of the connectors. And the connectors MUST have
>>> internal threads.
>> 
>> Yes, this thread will have to exists somewhere. In pattern 2 it exists in
>> the connector (at least from the perspective of the Flink execution
>> engine). In pattern 1 it exists inside the Flink execution engine. With
>> completely blocking connectors, like simple reading from files, both of
>> those approaches are basically the same. The difference is when user
>> implementing Flink source is already working with a non blocking code with
>> some internal threads. In this case, pattern 1 would result in "double
>> thread wrapping”, while pattern 2 would allow to skip one layer of
>> indirection.
>> 
>>> If we go that way, we should have something like "void
>>> poll(Callback) / void advance(callback)". I am curious how would
>>> CompletableFuture work here, though. If 10 readers returns 10 completable
>>> futures, will there be 10 additional threads (so 20 threads in total)
>>> blocking waiting on them? Or will there be a single thread busy loop
>>> checking around?
>> 
>> To be honest, I haven’t thought this completely through and I haven’t
>> tested/POC’ed it. Having said that, I can think of at least couple of
>> solutions. First is something like this:
>> 
>> 
>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>> <
>> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
>>> 
>> 
>> Line:
>> 
>>                                `blocked = split.process();`
>> 
>> Is where the execution goes into to the task/sources. This is where the
>> returned future is handled:
>> 
>>                                blocked.addListener(() -> {
>>                                    blockedSplits.remove(split);
>>                                    // reset the level priority to prevent
>> previously-blocked splits from starving existing splits
>>                                    split.resetLevelPriority();
>>                                    waitingSplits.offer(split);
>>                                }, executor);
>> 
>> Fundamentally callbacks and Futures are more or less interchangeable You
>> can always wrap one into another (creating a callback that completes a
>> future and attach a callback once future completes). In this case the
>> difference for me is mostly:
>> - api with passing callback allows the callback to be fired multiple times
>> and to fire it even if the connector is not blocked. This is what I meant
>> by saying that api `CompletableFuture<?> isBlocked()` is a bit simpler.
>> Connector can only return either “I’m not blocked” or “I’m blocked and I
>> will tell you only once when I’m not blocked anymore”.
>> 
>> But this is not the most important thing for me here. For me important
>> thing is to try our best to make Flink task’s control and execution single
>> threaded. For that both callback and future APIs should work the same.
>> 
>>> WRT pattern 1, a single blocking take() API should just work. The good
>>> thing is that a blocking read API is usually simpler to implement.
>> 
>> Yes, they are easier to implement (especially if you are not the one that
>> have to deal with the additional threading required around them ;) ). But
>> to answer this issue, if we choose pattern 2, we can always provide a
>> proxy/wrapper that would using the internal thread implement the
>> non-blocking API while exposing blocking API to the user. It would
>> implement pattern 2 for the user exposing to him pattern 1. In other words
>> implementing pattern 1 in pattern 2 paradigm, while making it possible to
>> implement pure pattern 2 connectors.
>> 
>>> BTW, one thing I am also trying to avoid is pushing users to perform IO
>> in
>>> a method like "isBlocked()". If the method is expected to fetch records
>>> (even if not returning them), naming it something more explicit would
>> help
>>> avoid confusion.
>> 
>> If we choose so, we could rework it into something like:
>> 
>> CompletableFuture<?> advance()
>> T getCurrent();
>> Watermark getCurrentWatermark()
>> 
>> But as I wrote before, this is more confusing to me for the exact reasons
>> you mentioned :) I would be confused what should be done in `adanvce()` and
>> what in `getCurrent()`. However, again this naming issue is not that
>> important to me and probably is matter of taste/personal preferences.
>> 
>> Piotrek
>> 
>>> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com> wrote:
>>> 
>>> Hi Piotrek,
>>> 
>>> Thanks for the explanation. We are probably talking about the same thing
>>> but in different ways. To clarify a little bit, I think there are two
>>> patterns to read from a connector.
>>> 
>>> Pattern 1: Thread-less connector with a blocking read API. Outside of the
>>> connector, there is one IO thread per reader, doing blocking read. An
>>> additional thread will interact with all the IO threads.
>>> Pattern 2: Connector with internal thread(s) and non-blocking API.
>> Outside
>>> of the connector, there is one thread for ALL readers, doing IO relying
>> on
>>> notification callbacks in the reader.
>>> 
>>> In both patterns, there must be at least one thread per connector, either
>>> inside (created by connector writers) or outside (created by Flink) of
>> the
>>> connector. Ideally there are NUM_CONNECTORS + 1 threads in total, to make
>>> sure that 1 thread is fully non-blocking.
>>> 
>>>> Btw, I don’t know if you understand my point. Having only `poll()` and
>>> `take()` is not enough for single threaded task. If our source interface
>>> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
>>> isBlocked(),`, there is no way to implement single threaded task that
>> both
>>> reads the data from the source connector and can also react to system
>>> events. Ok, non >blocking `poll()` would allow that, but with busy
>> looping.
>>> 
>>> Completely agree that in pattern 2, having a callback is necessary for
>> that
>>> single thread outside of the connectors. And the connectors MUST have
>>> internal threads. If we go that way, we should have something like "void
>>> poll(Callback) / void advance(callback)". I am curious how would
>>> CompletableFuture work here, though. If 10 readers returns 10 completable
>>> futures, will there be 10 additional threads (so 20 threads in total)
>>> blocking waiting on them? Or will there be a single thread busy loop
>>> checking around?
>>> 
>>> WRT pattern 1, a single blocking take() API should just work. The good
>>> thing is that a blocking read API is usually simpler to implement. An
>>> additional non-blocking "T poll()" method here is indeed optional and
>> could
>>> be used in cases like Flink does not want the thread to block forever.
>> They
>>> can also be combined to have a "T poll(Timeout)", which is exactly what
>>> KafkaConsumer did.
>>> 
>>> It sounds that you are proposing pattern 2 with something similar to NIO2
>>> AsynchronousByteChannel[1]. That API would work, except that the
>> signature
>>> returning future seems not necessary. If that is the case, a minor change
>>> on the current FLIP proposal to have "void advance(callback)" should
>> work.
>>> And this means the connectors MUST have their internal threads.
>>> 
>>> BTW, one thing I am also trying to avoid is pushing users to perform IO
>> in
>>> a method like "isBlocked()". If the method is expected to fetch records
>>> (even if not returning them), naming it something more explicit would
>> help
>>> avoid confusion.
>>> 
>>> Thanks,
>>> 
>>> Jiangjie (Becket) Qin
>>> 
>>> [1]
>>> 
>> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
>>> 
>>> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <pi...@data-artisans.com>
>>> wrote:
>>> 
>>>> Hi
>>>> 
>>>> Good point with select/epoll, however I do not see how they couldn’t be
>>>> with Flink if we would like single task in Flink to be single-threaded
>> (and
>>>> I believe we should pursue this goal). If your connector blocks on
>>>> `select`, then it can not process/handle control messages from Flink,
>> like
>>>> checkpoints, releasing resources and potentially output flushes. This
>> would
>>>> require tight integration between connector and Flink’s main event
>>>> loop/selects/etc.
>>>> 
>>>> Looking at it from other perspective. Let’s assume that we have a
>>>> connector implemented on top of `select`/`epoll`. In order to integrate
>> it
>>>> with Flink’s checkpointing/flushes/resource releasing it will have to be
>>>> executed in separate thread one way or another. At least if our API will
>>>> enforce/encourage non blocking implementations with some kind of
>>>> notifications (`isBlocked()` or `notify()` callback), some connectors
>> might
>>>> skip one layer of wapping threads.
>>>> 
>>>> Btw, I don’t know if you understand my point. Having only `poll()` and
>>>> `take()` is not enough for single threaded task. If our source interface
>>>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
>>>> isBlocked(),`, there is no way to implement single threaded task that
>> both
>>>> reads the data from the source connector and can also react to system
>>>> events. Ok, non blocking `poll()` would allow that, but with busy
>> looping.
>>>> 
>>>> Piotrek
>>>> 
>>>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com> wrote:
>>>>> 
>>>>> Hi Piotrek,
>>>>> 
>>>>>> But I don’t see a reason why we should expose both blocking `take()`
>> and
>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
>> engine
>>>> or
>>>>> connector) would have to do the same busy
>>>>>> looping anyway and I think it would be better to have a simpler
>>>> connector
>>>>> API (that would solve our problems) and force connectors to comply one
>>>> way
>>>>> or another.
>>>>> 
>>>>> If we let the block happen inside the connector, the blocking does not
>>>> have
>>>>> to be a busy loop. For example, to do the block waiting efficiently,
>> the
>>>>> connector can use java NIO selector().select which relies on OS syscall
>>>>> like epoll[1] instead of busy looping. But if Flink engine blocks
>> outside
>>>>> the connector, it pretty much has to do the busy loop. So if there is
>>>> only
>>>>> one API to get the element, a blocking getNextElement() makes more
>> sense.
>>>>> In any case, we should avoid ambiguity. It has to be crystal clear
>> about
>>>>> whether a method is expected to be blocking or non-blocking. Otherwise
>> it
>>>>> would be very difficult for Flink engine to do the right thing with the
>>>>> connectors. At the first glance at getCurrent(), the expected behavior
>> is
>>>>> not quite clear.
>>>>> 
>>>>> That said, I do agree that functionality wise, poll() and take() kind
>> of
>>>>> overlap. But they are actually not quite different from
>>>>> isBlocked()/getNextElement(). Compared with isBlocked(), the only
>>>>> difference is that poll() also returns the next record if it is
>>>> available.
>>>>> But I agree that the isBlocked() + getNextElement() is more flexible as
>>>>> users can just check the record availability, but not fetch the next
>>>>> element.
>>>>> 
>>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
>> (is
>>>>> that really a thing? I can not think about a real implementation that
>>>>> enforces such constraints)
>>>>> Right, it is pretty much a syntax sugar to allow user combine the
>>>>> check-and-take into one method. It could be achieved with isBlocked() +
>>>>> getNextElement().
>>>>> 
>>>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
>>>>> 
>>>>> Thanks,
>>>>> 
>>>>> Jiangjie (Becket) Qin
>>>>> 
>>>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
>> piotr@data-artisans.com>
>>>>> wrote:
>>>>> 
>>>>>> Hi Becket,
>>>>>> 
>>>>>> With my proposal, both of your examples would have to be solved by the
>>>>>> connector and solution to both problems would be the same:
>>>>>> 
>>>>>> Pretend that connector is never blocked (`isBlocked() { return
>>>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking fashion
>>>> (or
>>>>>> semi blocking with return of control from time to time to allow for
>>>>>> checkpointing, network flushing and other resource management things
>> to
>>>>>> happen in the same main thread). In other words, exactly how you would
>>>>>> implement `take()` method or how the same source connector would be
>>>>>> implemented NOW with current source interface. The difference with
>>>> current
>>>>>> interface would be only that main loop would be outside of the
>>>> connector,
>>>>>> and instead of periodically releasing checkpointing lock, periodically
>>>>>> `return null;` or `return Optional.empty();` from `getNextElement()`.
>>>>>> 
>>>>>> In case of thread-less readers with only non-blocking `queue.poll()`
>> (is
>>>>>> that really a thing? I can not think about a real implementation that
>>>>>> enforces such constraints), we could provide a wrapper that hides the
>>>> busy
>>>>>> looping. The same applies how to solve forever blocking readers - we
>>>> could
>>>>>> provider another wrapper running the connector in separate thread.
>>>>>> 
>>>>>> But I don’t see a reason why we should expose both blocking `take()`
>> and
>>>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
>>>> engine or
>>>>>> connector) would have to do the same busy looping anyway and I think
>> it
>>>>>> would be better to have a simpler connector API (that would solve our
>>>>>> problems) and force connectors to comply one way or another.
>>>>>> 
>>>>>> Piotrek
>>>>>> 
>>>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
>>>>>>> 
>>>>>>> Hi Piotr,
>>>>>>> 
>>>>>>> I might have misunderstood you proposal. But let me try to explain my
>>>>>>> concern. I am thinking about the following case:
>>>>>>> 1. a reader has the following two interfaces,
>>>>>>> boolean isBlocked()
>>>>>>> T getNextElement()
>>>>>>> 2. the implementation of getNextElement() is non-blocking.
>>>>>>> 3. The reader is thread-less, i.e. it does not have any internal
>>>> thread.
>>>>>>> For example, it might just delegate the getNextElement() to a
>>>>>> queue.poll(),
>>>>>>> and isBlocked() is just queue.isEmpty().
>>>>>>> 
>>>>>>> How can Flink efficiently implement a blocking reading behavior with
>>>> this
>>>>>>> reader? Either a tight loop or a backoff interval is needed. Neither
>> of
>>>>>>> them is ideal.
>>>>>>> 
>>>>>>> Now let's say in the reader mentioned above implements a blocking
>>>>>>> getNextElement() method. Because there is no internal thread in the
>>>>>> reader,
>>>>>>> after isBlocked() returns false. Flink will still have to loop on
>>>>>>> isBlocked() to check whether the next record is available. If the
>> next
>>>>>>> record reaches after 10 min, it is a tight loop for 10 min. You have
>>>>>>> probably noticed that in this case, even isBlocked() returns a
>> future,
>>>>>> that
>>>>>>> future() will not be completed if Flink does not call some method
>> from
>>>>>> the
>>>>>>> reader, because the reader has no internal thread to complete that
>>>> future
>>>>>>> by itself.
>>>>>>> 
>>>>>>> Due to the above reasons, a blocking take() API would allow Flink to
>>>> have
>>>>>>> an efficient way to read from a reader. There are many ways to wake
>> up
>>>>>> the
>>>>>>> blocking thread when checkpointing is needed depending on the
>>>>>>> implementation. But I think the poll()/take() API would also work in
>>>> that
>>>>>>> case.
>>>>>>> 
>>>>>>> Thanks,
>>>>>>> 
>>>>>>> Jiangjie (Becket) Qin
>>>>>>> 
>>>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
>> piotr@data-artisans.com
>>>>> 
>>>>>>> wrote:
>>>>>>> 
>>>>>>>> Hi,
>>>>>>>> 
>>>>>>>> a)
>>>>>>>> 
>>>>>>>>> BTW, regarding the isBlock() method, I have a few more questions.
>> 21,
>>>>>> Is
>>>>>>>> a method isReady() with boolean as a return value
>>>>>>>>> equivalent? Personally I found it is a little bit confusing in what
>>>> is
>>>>>>>> supposed to be returned when the future is completed. 22. if
>>>>>>>>> the implementation of isBlocked() is optional, how do the callers
>>>> know
>>>>>>>> whether the method is properly implemented or not?
>>>>>>>>> Does not implemented mean it always return a completed future?
>>>>>>>> 
>>>>>>>> `CompletableFuture<?> isBlocked()` is more or less an equivalent to
>>>>>>>> `boolean hasNext()` which in case of “false” provides some kind of a
>>>>>>>> listener/callback that notifies about presence of next element.
>> There
>>>>>> are
>>>>>>>> some minor details, like `CompletableFuture<?>` has a minimal two
>>>> state
>>>>>>>> logic:
>>>>>>>> 
>>>>>>>> 1. Future is completed - we have more data
>>>>>>>> 2. Future not yet completed - we don’t have data now, but we
>> might/we
>>>>>> will
>>>>>>>> have in the future
>>>>>>>> 
>>>>>>>> While `boolean hasNext()` and `notify()` callback are a bit more
>>>>>>>> complicated/dispersed and can lead/encourage `notify()` spam.
>>>>>>>> 
>>>>>>>> b)
>>>>>>>> 
>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>>>>>> `getNext`
>>>>>>>> the `getNext` would need return a
>>>>>>>>> `ElementWithTimestamp` because some sources want to add timestamp
>> to
>>>>>>>> every element. IMO, this is not so memory friendly
>>>>>>>>> so I prefer this design.
>>>>>>>> 
>>>>>>>> Guowei I don’t quite understand this. Could you elaborate why
>> having a
>>>>>>>> separate `advance()` help?
>>>>>>>> 
>>>>>>>> c)
>>>>>>>> 
>>>>>>>> Regarding advance/poll/take. What’s the value of having two separate
>>>>>>>> methods: poll and take? Which one of them should be called and which
>>>>>>>> implemented? What’s the benefit of having those methods compared to
>>>>>> having
>>>>>>>> a one single method `getNextElement()` (or `pollElement() or
>> whatever
>>>> we
>>>>>>>> name it) with following contract:
>>>>>>>> 
>>>>>>>> CompletableFuture<?> isBlocked();
>>>>>>>> 
>>>>>>>> /**
>>>>>>>> Return next element - will be called only if `isBlocked()` is
>>>> completed.
>>>>>>>> Try to implement it in non blocking fashion, but if that’s
>> impossible
>>>> or
>>>>>>>> you just don’t need the effort, you can block in this method.
>>>>>>>> */
>>>>>>>> T getNextElement();
>>>>>>>> 
>>>>>>>> I mean, if the connector is implemented non-blockingly, Flink should
>>>> use
>>>>>>>> it that way. If it’s not, then `poll()` will `throw new
>>>>>>>> NotImplementedException()`. Implementing both of them and providing
>>>>>> both of
>>>>>>>> them to Flink wouldn’t make a sense, thus why not merge them into a
>>>>>> single
>>>>>>>> method call that should preferably (but not necessarily need to) be
>>>>>>>> non-blocking? It’s not like we are implementing general purpose
>>>> `Queue`,
>>>>>>>> which users might want to call either of `poll` or `take`. We would
>>>>>> always
>>>>>>>> prefer to call `poll`, but if it’s blocking, then still we have no
>>>>>> choice,
>>>>>>>> but to call it and block on it.
>>>>>>>> 
>>>>>>>> d)
>>>>>>>> 
>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
>> very
>>>>>>>>> important. But in addition to `Future/poll`, there may be another
>> way
>>>>>> to
>>>>>>>>> achieve this. I think it may be not very memory friendly if every
>>>>>> advance
>>>>>>>>> call return a Future.
>>>>>>>> 
>>>>>>>> I didn’t want to mention this, to not clog my initial proposal, but
>>>>>> there
>>>>>>>> is a simple solution for the problem:
>>>>>>>> 
>>>>>>>> public interface SplitReader {
>>>>>>>> 
>>>>>>>> (…)
>>>>>>>> 
>>>>>>>> CompletableFuture<?> NOT_BLOCKED =
>>>>>>>> CompletableFuture.completedFuture(null);
>>>>>>>> 
>>>>>>>> /**
>>>>>>>>  * Returns a future that will be completed when the page source
>>>>>> becomes
>>>>>>>>  * unblocked.  If the page source is not blocked, this method
>> should
>>>>>>>> return
>>>>>>>>  * {@code NOT_BLOCKED}.
>>>>>>>>  */
>>>>>>>> default CompletableFuture<?> isBlocked()
>>>>>>>> {
>>>>>>>>     return NOT_BLOCKED;
>>>>>>>> }
>>>>>>>> 
>>>>>>>> If we are blocked and we are waiting for the IO, then creating a new
>>>>>>>> Future is non-issue. Under full throttle/throughput and not blocked
>>>>>> sources
>>>>>>>> returning a static `NOT_BLOCKED` constant  should also solve the
>>>>>> problem.
>>>>>>>> 
>>>>>>>> One more remark, non-blocking sources might be a necessity in a
>> single
>>>>>>>> threaded model without a checkpointing lock. (Currently when sources
>>>> are
>>>>>>>> blocked, they can release checkpointing lock and re-acquire it again
>>>>>>>> later). Non-blocking `poll`/`getNext()` would allow for checkpoints
>> to
>>>>>>>> happen when source is idling. In that case either `notify()` or my
>>>>>> proposed
>>>>>>>> `isBlocked()` would allow to avoid busy-looping.
>>>>>>>> 
>>>>>>>> Piotrek
>>>>>>>> 
>>>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
>>>>>>>>> 
>>>>>>>>> Hi Thomas,
>>>>>>>>> 
>>>>>>>>> The iterator-like API was also the first thing that came to me. But
>>>> it
>>>>>>>>> seems a little confusing that hasNext() does not mean "the stream
>> has
>>>>>> not
>>>>>>>>> ended", but means "the next record is ready", which is repurposing
>>>> the
>>>>>>>> well
>>>>>>>>> known meaning of hasNext(). If we follow the hasNext()/next()
>>>> pattern,
>>>>>> an
>>>>>>>>> additional isNextReady() method to indicate whether the next record
>>>> is
>>>>>>>>> ready seems more intuitive to me.
>>>>>>>>> 
>>>>>>>>> Similarly, in poll()/take() pattern, another method of isDone() is
>>>>>> needed
>>>>>>>>> to indicate whether the stream has ended or not.
>>>>>>>>> 
>>>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
>>>>>>>>> isDone()/poll()/take() seems more flexible for the reader
>>>>>> implementation.
>>>>>>>>> When I am implementing a reader, I could have a couple of choices:
>>>>>>>>> 
>>>>>>>>> - A thread-less reader that does not have any internal thread.
>>>>>>>>> - When poll() is called, the same calling thread will perform a
>> bunch
>>>>>>>> of
>>>>>>>>>  IO asynchronously.
>>>>>>>>>  - When take() is called, the same calling thread will perform a
>>>>>>>> bunch
>>>>>>>>>  of IO and wait until the record is ready.
>>>>>>>>> - A reader with internal threads performing network IO and put
>>>> records
>>>>>>>>> into a buffer.
>>>>>>>>>  - When poll() is called, the calling thread simply reads from the
>>>>>>>>>  buffer and return empty result immediately if there is no record.
>>>>>>>>>  - When take() is called, the calling thread reads from the buffer
>>>>>>>> and
>>>>>>>>>  block waiting if the buffer is empty.
>>>>>>>>> 
>>>>>>>>> On the other hand, with the hasNext()/next()/isNextReady() API, it
>> is
>>>>>>>> less
>>>>>>>>> intuitive for the reader developers to write the thread-less
>> pattern.
>>>>>>>>> Although technically speaking one can still do the asynchronous IO
>> to
>>>>>>>>> prepare the record in isNextReady(). But it is inexplicit and seems
>>>>>>>>> somewhat hacky.
>>>>>>>>> 
>>>>>>>>> Thanks,
>>>>>>>>> 
>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org>
>> wrote:
>>>>>>>>> 
>>>>>>>>>> Couple more points regarding discovery:
>>>>>>>>>> 
>>>>>>>>>> The proposal mentions that discovery could be outside the
>> execution
>>>>>>>> graph.
>>>>>>>>>> Today, discovered partitions/shards are checkpointed. I believe
>> that
>>>>>>>> will
>>>>>>>>>> also need to be the case in the future, even when discovery and
>>>>>> reading
>>>>>>>> are
>>>>>>>>>> split between different tasks.
>>>>>>>>>> 
>>>>>>>>>> For cases such as resharding of a Kinesis stream, the relationship
>>>>>>>> between
>>>>>>>>>> splits needs to be considered. Splits cannot be randomly
>> distributed
>>>>>>>> over
>>>>>>>>>> readers in certain situations. An example was mentioned here:
>>>>>>>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>>>>>>>>>> 
>>>>>>>>>> Thomas
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org>
>> wrote:
>>>>>>>>>> 
>>>>>>>>>>> Thanks for getting the ball rolling on this!
>>>>>>>>>>> 
>>>>>>>>>>> Can the number of splits decrease? Yes, splits can be closed and
>> go
>>>>>>>> away.
>>>>>>>>>>> An example would be a shard merge in Kinesis (2 existing shards
>>>> will
>>>>>> be
>>>>>>>>>>> closed and replaced with a new shard).
>>>>>>>>>>> 
>>>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive approach
>>>> would
>>>>>>>> be
>>>>>>>>>>> the thread-less IO model (pull based, non-blocking, caller
>>>> retrieves
>>>>>>>> new
>>>>>>>>>>> records when available). The current Kinesis API requires the use
>>>> of
>>>>>>>>>>> threads. But that can be internal to the split reader and does
>> not
>>>>>> need
>>>>>>>>>> to
>>>>>>>>>>> be a source API concern. In fact, that's what we are working on
>>>> right
>>>>>>>> now
>>>>>>>>>>> as improvement to the existing consumer: Each shard consumer
>> thread
>>>>>>>> will
>>>>>>>>>>> push to a queue, the consumer main thread will poll the queue(s).
>>>> It
>>>>>> is
>>>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
>>>>>>>>>>> 
>>>>>>>>>>> The proposed SplitReader interface would fit the thread-less IO
>>>>>> model.
>>>>>>>>>>> Similar to an iterator, we find out if there is a new element
>>>>>> (hasNext)
>>>>>>>>>> and
>>>>>>>>>>> if so, move to it (next()). Separate calls deliver the meta
>>>>>> information
>>>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer a
>> timeout
>>>>>>>>>> option,
>>>>>>>>>>> so that the caller does not end up in a busy wait. On the other
>>>>>> hand, a
>>>>>>>>>>> caller processing multiple splits may want to cycle through fast,
>>>> to
>>>>>>>>>>> process elements of other splits as soon as they become
>> available.
>>>>>> The
>>>>>>>>>> nice
>>>>>>>>>>> thing is that this "split merge" logic can now live in Flink and
>> be
>>>>>>>>>>> optimized and shared between different sources.
>>>>>>>>>>> 
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Thomas
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Hi,
>>>>>>>>>>>> Thanks Aljoscha for this FLIP.
>>>>>>>>>>>> 
>>>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
>>>>>> very
>>>>>>>>>>>> important. But in addition to `Future/poll`, there may be
>> another
>>>>>> way
>>>>>>>> to
>>>>>>>>>>>> achieve this. I think it may be not very memory friendly if
>> every
>>>>>>>>>> advance
>>>>>>>>>>>> call return a Future.
>>>>>>>>>>>> 
>>>>>>>>>>>> public interface Listener {
>>>>>>>>>>>> public void notify();
>>>>>>>>>>>> }
>>>>>>>>>>>> 
>>>>>>>>>>>> public interface SplitReader() {
>>>>>>>>>>>> /**
>>>>>>>>>>>>  * When there is no element temporarily, this will return
>> false.
>>>>>>>>>>>>  * When elements is available again splitReader can call
>>>>>>>>>>>> listener.notify()
>>>>>>>>>>>>  * In addition the frame would check `advance` periodically .
>>>>>>>>>>>>  * Of course advance can always return true and ignore the
>>>>>>>> listener
>>>>>>>>>>>> argument for simplicity.
>>>>>>>>>>>>  */
>>>>>>>>>>>> public boolean advance(Listener listener);
>>>>>>>>>>>> }
>>>>>>>>>>>> 
>>>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create all Splits
>>>> and
>>>>>>>> how
>>>>>>>>>>>> to create a SplitReader from a Split. But there is no strategy
>> for
>>>>>> the
>>>>>>>>>> user
>>>>>>>>>>>> to choose how to assign the splits to the tasks. I think we
>> could
>>>>>> add
>>>>>>>> a
>>>>>>>>>>>> Enum to let user to choose.
>>>>>>>>>>>> /**
>>>>>>>>>>>> public Enum SplitsAssignmentPolicy {
>>>>>>>>>>>> Location,
>>>>>>>>>>>> Workload,
>>>>>>>>>>>> Random,
>>>>>>>>>>>> Average
>>>>>>>>>>>> }
>>>>>>>>>>>> */
>>>>>>>>>>>> 
>>>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>>>>>>>> `getNext`
>>>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp` because
>>>>>> some
>>>>>>>>>>>> sources want to add timestamp to every element. IMO, this is not
>>>> so
>>>>>>>>>> memory
>>>>>>>>>>>> friendly so I prefer this design.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> Thanks
>>>>>>>>>>>> 
>>>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
>> 下午6:08写道:
>>>>>>>>>>>> 
>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
>>>>>> other
>>>>>>>>>>>>> possible improvements. I have one proposal. Instead of having a
>>>>>>>> method:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> boolean advance() throws IOException;
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I would replace it with
>>>>>>>>>>>>> 
>>>>>>>>>>>>> /*
>>>>>>>>>>>>> * Return a future, which when completed means that source has
>>>> more
>>>>>>>>>> data
>>>>>>>>>>>>> and getNext() will not block.
>>>>>>>>>>>>> * If you wish to use benefits of non blocking connectors,
>> please
>>>>>>>>>>>>> implement this method appropriately.
>>>>>>>>>>>>> */
>>>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
>>>>>>>>>>>>>    return CompletableFuture.completedFuture(null);
>>>>>>>>>>>>> }
>>>>>>>>>>>>> 
>>>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Couple of arguments:
>>>>>>>>>>>>> 1. I don’t understand the division of work between `advance()`
>>>> and
>>>>>>>>>>>>> `getCurrent()`. What should be done in which, especially for
>>>>>>>> connectors
>>>>>>>>>>>>> that handle records in batches (like Kafka) and when should you
>>>>>> call
>>>>>>>>>>>>> `advance` and when `getCurrent()`.
>>>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow
>> us
>>>> in
>>>>>>>> the
>>>>>>>>>>>>> future to have asynchronous/non blocking connectors and more
>>>>>>>>>> efficiently
>>>>>>>>>>>>> handle large number of blocked threads, without busy waiting.
>>>> While
>>>>>>>> at
>>>>>>>>>> the
>>>>>>>>>>>>> same time it doesn’t add much complexity, since naive connector
>>>>>>>>>>>>> implementations can be always blocking.
>>>>>>>>>>>>> 3. This also would allow us to use a fixed size thread pool of
>>>> task
>>>>>>>>>>>>> executors, instead of one thread per task.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Piotrek
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
>> aljoscha@apache.org
>>>>> 
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Hi All,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> In order to finally get the ball rolling on the new source
>>>>>> interface
>>>>>>>>>>>>> that we have discussed for so long I finally created a FLIP:
>>>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
>> work/discussion
>>>>>>>> about
>>>>>>>>>>>>> adding per-partition watermark support to the Kinesis source
>> and
>>>>>>>>>> because
>>>>>>>>>>>>> this would enable generic implementation of event-time
>> alignment
>>>>>> for
>>>>>>>>>> all
>>>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
>> alignment
>>>>>>>> part,
>>>>>>>>>>>>> especially the part about information sharing between
>> operations
>>>>>> (I'm
>>>>>>>>>> not
>>>>>>>>>>>>> calling it state sharing because state has a special meaning in
>>>>>>>> Flink).
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Please discuss away!
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Aljoscha
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>>>> 
>>>> 
>>>> 
>> 
>> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Piotrek,

Thanks a lot for the detailed reply. All makes sense to me.

WRT the confusion between advance() / getCurrent(), do you think it would
help if we combine them and have something like:

CompletableFuture<T> getNext();
long getWatermark();
long getCurrentTimestamp();

Cheers,

Jiangjie (Becket) Qin

On Tue, Nov 13, 2018 at 9:56 PM Piotr Nowojski <pi...@data-artisans.com>
wrote:

> Hi,
>
> Thanks again for the detailed answer :) Sorry for responding with a delay.
>
> > Completely agree that in pattern 2, having a callback is necessary for
> that
> > single thread outside of the connectors. And the connectors MUST have
> > internal threads.
>
> Yes, this thread will have to exists somewhere. In pattern 2 it exists in
> the connector (at least from the perspective of the Flink execution
> engine). In pattern 1 it exists inside the Flink execution engine. With
> completely blocking connectors, like simple reading from files, both of
> those approaches are basically the same. The difference is when user
> implementing Flink source is already working with a non blocking code with
> some internal threads. In this case, pattern 1 would result in "double
> thread wrapping”, while pattern 2 would allow to skip one layer of
> indirection.
>
> > If we go that way, we should have something like "void
> > poll(Callback) / void advance(callback)". I am curious how would
> > CompletableFuture work here, though. If 10 readers returns 10 completable
> > futures, will there be 10 additional threads (so 20 threads in total)
> > blocking waiting on them? Or will there be a single thread busy loop
> > checking around?
>
> To be honest, I haven’t thought this completely through and I haven’t
> tested/POC’ed it. Having said that, I can think of at least couple of
> solutions. First is something like this:
>
>
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> <
> https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507
> >
>
> Line:
>
>                                 `blocked = split.process();`
>
> Is where the execution goes into to the task/sources. This is where the
> returned future is handled:
>
>                                 blocked.addListener(() -> {
>                                     blockedSplits.remove(split);
>                                     // reset the level priority to prevent
> previously-blocked splits from starving existing splits
>                                     split.resetLevelPriority();
>                                     waitingSplits.offer(split);
>                                 }, executor);
>
> Fundamentally callbacks and Futures are more or less interchangeable You
> can always wrap one into another (creating a callback that completes a
> future and attach a callback once future completes). In this case the
> difference for me is mostly:
> - api with passing callback allows the callback to be fired multiple times
> and to fire it even if the connector is not blocked. This is what I meant
> by saying that api `CompletableFuture<?> isBlocked()` is a bit simpler.
> Connector can only return either “I’m not blocked” or “I’m blocked and I
> will tell you only once when I’m not blocked anymore”.
>
> But this is not the most important thing for me here. For me important
> thing is to try our best to make Flink task’s control and execution single
> threaded. For that both callback and future APIs should work the same.
>
> > WRT pattern 1, a single blocking take() API should just work. The good
> > thing is that a blocking read API is usually simpler to implement.
>
> Yes, they are easier to implement (especially if you are not the one that
> have to deal with the additional threading required around them ;) ). But
> to answer this issue, if we choose pattern 2, we can always provide a
> proxy/wrapper that would using the internal thread implement the
> non-blocking API while exposing blocking API to the user. It would
> implement pattern 2 for the user exposing to him pattern 1. In other words
> implementing pattern 1 in pattern 2 paradigm, while making it possible to
> implement pure pattern 2 connectors.
>
> > BTW, one thing I am also trying to avoid is pushing users to perform IO
> in
> > a method like "isBlocked()". If the method is expected to fetch records
> > (even if not returning them), naming it something more explicit would
> help
> > avoid confusion.
>
> If we choose so, we could rework it into something like:
>
> CompletableFuture<?> advance()
> T getCurrent();
> Watermark getCurrentWatermark()
>
> But as I wrote before, this is more confusing to me for the exact reasons
> you mentioned :) I would be confused what should be done in `adanvce()` and
> what in `getCurrent()`. However, again this naming issue is not that
> important to me and probably is matter of taste/personal preferences.
>
> Piotrek
>
> > On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com> wrote:
> >
> > Hi Piotrek,
> >
> > Thanks for the explanation. We are probably talking about the same thing
> > but in different ways. To clarify a little bit, I think there are two
> > patterns to read from a connector.
> >
> > Pattern 1: Thread-less connector with a blocking read API. Outside of the
> > connector, there is one IO thread per reader, doing blocking read. An
> > additional thread will interact with all the IO threads.
> > Pattern 2: Connector with internal thread(s) and non-blocking API.
> Outside
> > of the connector, there is one thread for ALL readers, doing IO relying
> on
> > notification callbacks in the reader.
> >
> > In both patterns, there must be at least one thread per connector, either
> > inside (created by connector writers) or outside (created by Flink) of
> the
> > connector. Ideally there are NUM_CONNECTORS + 1 threads in total, to make
> > sure that 1 thread is fully non-blocking.
> >
> >> Btw, I don’t know if you understand my point. Having only `poll()` and
> > `take()` is not enough for single threaded task. If our source interface
> > doesn’t provide `notify()` callback nor >`CompletableFuture<?>
> > isBlocked(),`, there is no way to implement single threaded task that
> both
> > reads the data from the source connector and can also react to system
> > events. Ok, non >blocking `poll()` would allow that, but with busy
> looping.
> >
> > Completely agree that in pattern 2, having a callback is necessary for
> that
> > single thread outside of the connectors. And the connectors MUST have
> > internal threads. If we go that way, we should have something like "void
> > poll(Callback) / void advance(callback)". I am curious how would
> > CompletableFuture work here, though. If 10 readers returns 10 completable
> > futures, will there be 10 additional threads (so 20 threads in total)
> > blocking waiting on them? Or will there be a single thread busy loop
> > checking around?
> >
> > WRT pattern 1, a single blocking take() API should just work. The good
> > thing is that a blocking read API is usually simpler to implement. An
> > additional non-blocking "T poll()" method here is indeed optional and
> could
> > be used in cases like Flink does not want the thread to block forever.
> They
> > can also be combined to have a "T poll(Timeout)", which is exactly what
> > KafkaConsumer did.
> >
> > It sounds that you are proposing pattern 2 with something similar to NIO2
> > AsynchronousByteChannel[1]. That API would work, except that the
> signature
> > returning future seems not necessary. If that is the case, a minor change
> > on the current FLIP proposal to have "void advance(callback)" should
> work.
> > And this means the connectors MUST have their internal threads.
> >
> > BTW, one thing I am also trying to avoid is pushing users to perform IO
> in
> > a method like "isBlocked()". If the method is expected to fetch records
> > (even if not returning them), naming it something more explicit would
> help
> > avoid confusion.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > [1]
> >
> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
> >
> > On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <pi...@data-artisans.com>
> > wrote:
> >
> >> Hi
> >>
> >> Good point with select/epoll, however I do not see how they couldn’t be
> >> with Flink if we would like single task in Flink to be single-threaded
> (and
> >> I believe we should pursue this goal). If your connector blocks on
> >> `select`, then it can not process/handle control messages from Flink,
> like
> >> checkpoints, releasing resources and potentially output flushes. This
> would
> >> require tight integration between connector and Flink’s main event
> >> loop/selects/etc.
> >>
> >> Looking at it from other perspective. Let’s assume that we have a
> >> connector implemented on top of `select`/`epoll`. In order to integrate
> it
> >> with Flink’s checkpointing/flushes/resource releasing it will have to be
> >> executed in separate thread one way or another. At least if our API will
> >> enforce/encourage non blocking implementations with some kind of
> >> notifications (`isBlocked()` or `notify()` callback), some connectors
> might
> >> skip one layer of wapping threads.
> >>
> >> Btw, I don’t know if you understand my point. Having only `poll()` and
> >> `take()` is not enough for single threaded task. If our source interface
> >> doesn’t provide `notify()` callback nor `CompletableFuture<?>
> >> isBlocked(),`, there is no way to implement single threaded task that
> both
> >> reads the data from the source connector and can also react to system
> >> events. Ok, non blocking `poll()` would allow that, but with busy
> looping.
> >>
> >> Piotrek
> >>
> >>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com> wrote:
> >>>
> >>> Hi Piotrek,
> >>>
> >>>> But I don’t see a reason why we should expose both blocking `take()`
> and
> >>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
> engine
> >> or
> >>> connector) would have to do the same busy
> >>>> looping anyway and I think it would be better to have a simpler
> >> connector
> >>> API (that would solve our problems) and force connectors to comply one
> >> way
> >>> or another.
> >>>
> >>> If we let the block happen inside the connector, the blocking does not
> >> have
> >>> to be a busy loop. For example, to do the block waiting efficiently,
> the
> >>> connector can use java NIO selector().select which relies on OS syscall
> >>> like epoll[1] instead of busy looping. But if Flink engine blocks
> outside
> >>> the connector, it pretty much has to do the busy loop. So if there is
> >> only
> >>> one API to get the element, a blocking getNextElement() makes more
> sense.
> >>> In any case, we should avoid ambiguity. It has to be crystal clear
> about
> >>> whether a method is expected to be blocking or non-blocking. Otherwise
> it
> >>> would be very difficult for Flink engine to do the right thing with the
> >>> connectors. At the first glance at getCurrent(), the expected behavior
> is
> >>> not quite clear.
> >>>
> >>> That said, I do agree that functionality wise, poll() and take() kind
> of
> >>> overlap. But they are actually not quite different from
> >>> isBlocked()/getNextElement(). Compared with isBlocked(), the only
> >>> difference is that poll() also returns the next record if it is
> >> available.
> >>> But I agree that the isBlocked() + getNextElement() is more flexible as
> >>> users can just check the record availability, but not fetch the next
> >>> element.
> >>>
> >>>> In case of thread-less readers with only non-blocking `queue.poll()`
> (is
> >>> that really a thing? I can not think about a real implementation that
> >>> enforces such constraints)
> >>> Right, it is pretty much a syntax sugar to allow user combine the
> >>> check-and-take into one method. It could be achieved with isBlocked() +
> >>> getNextElement().
> >>>
> >>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
> >>>
> >>> Thanks,
> >>>
> >>> Jiangjie (Becket) Qin
> >>>
> >>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <
> piotr@data-artisans.com>
> >>> wrote:
> >>>
> >>>> Hi Becket,
> >>>>
> >>>> With my proposal, both of your examples would have to be solved by the
> >>>> connector and solution to both problems would be the same:
> >>>>
> >>>> Pretend that connector is never blocked (`isBlocked() { return
> >>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking fashion
> >> (or
> >>>> semi blocking with return of control from time to time to allow for
> >>>> checkpointing, network flushing and other resource management things
> to
> >>>> happen in the same main thread). In other words, exactly how you would
> >>>> implement `take()` method or how the same source connector would be
> >>>> implemented NOW with current source interface. The difference with
> >> current
> >>>> interface would be only that main loop would be outside of the
> >> connector,
> >>>> and instead of periodically releasing checkpointing lock, periodically
> >>>> `return null;` or `return Optional.empty();` from `getNextElement()`.
> >>>>
> >>>> In case of thread-less readers with only non-blocking `queue.poll()`
> (is
> >>>> that really a thing? I can not think about a real implementation that
> >>>> enforces such constraints), we could provide a wrapper that hides the
> >> busy
> >>>> looping. The same applies how to solve forever blocking readers - we
> >> could
> >>>> provider another wrapper running the connector in separate thread.
> >>>>
> >>>> But I don’t see a reason why we should expose both blocking `take()`
> and
> >>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
> >> engine or
> >>>> connector) would have to do the same busy looping anyway and I think
> it
> >>>> would be better to have a simpler connector API (that would solve our
> >>>> problems) and force connectors to comply one way or another.
> >>>>
> >>>> Piotrek
> >>>>
> >>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
> >>>>>
> >>>>> Hi Piotr,
> >>>>>
> >>>>> I might have misunderstood you proposal. But let me try to explain my
> >>>>> concern. I am thinking about the following case:
> >>>>> 1. a reader has the following two interfaces,
> >>>>>  boolean isBlocked()
> >>>>>  T getNextElement()
> >>>>> 2. the implementation of getNextElement() is non-blocking.
> >>>>> 3. The reader is thread-less, i.e. it does not have any internal
> >> thread.
> >>>>> For example, it might just delegate the getNextElement() to a
> >>>> queue.poll(),
> >>>>> and isBlocked() is just queue.isEmpty().
> >>>>>
> >>>>> How can Flink efficiently implement a blocking reading behavior with
> >> this
> >>>>> reader? Either a tight loop or a backoff interval is needed. Neither
> of
> >>>>> them is ideal.
> >>>>>
> >>>>> Now let's say in the reader mentioned above implements a blocking
> >>>>> getNextElement() method. Because there is no internal thread in the
> >>>> reader,
> >>>>> after isBlocked() returns false. Flink will still have to loop on
> >>>>> isBlocked() to check whether the next record is available. If the
> next
> >>>>> record reaches after 10 min, it is a tight loop for 10 min. You have
> >>>>> probably noticed that in this case, even isBlocked() returns a
> future,
> >>>> that
> >>>>> future() will not be completed if Flink does not call some method
> from
> >>>> the
> >>>>> reader, because the reader has no internal thread to complete that
> >> future
> >>>>> by itself.
> >>>>>
> >>>>> Due to the above reasons, a blocking take() API would allow Flink to
> >> have
> >>>>> an efficient way to read from a reader. There are many ways to wake
> up
> >>>> the
> >>>>> blocking thread when checkpointing is needed depending on the
> >>>>> implementation. But I think the poll()/take() API would also work in
> >> that
> >>>>> case.
> >>>>>
> >>>>> Thanks,
> >>>>>
> >>>>> Jiangjie (Becket) Qin
> >>>>>
> >>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <
> piotr@data-artisans.com
> >>>
> >>>>> wrote:
> >>>>>
> >>>>>> Hi,
> >>>>>>
> >>>>>> a)
> >>>>>>
> >>>>>>> BTW, regarding the isBlock() method, I have a few more questions.
> 21,
> >>>> Is
> >>>>>> a method isReady() with boolean as a return value
> >>>>>>> equivalent? Personally I found it is a little bit confusing in what
> >> is
> >>>>>> supposed to be returned when the future is completed. 22. if
> >>>>>>> the implementation of isBlocked() is optional, how do the callers
> >> know
> >>>>>> whether the method is properly implemented or not?
> >>>>>>> Does not implemented mean it always return a completed future?
> >>>>>>
> >>>>>> `CompletableFuture<?> isBlocked()` is more or less an equivalent to
> >>>>>> `boolean hasNext()` which in case of “false” provides some kind of a
> >>>>>> listener/callback that notifies about presence of next element.
> There
> >>>> are
> >>>>>> some minor details, like `CompletableFuture<?>` has a minimal two
> >> state
> >>>>>> logic:
> >>>>>>
> >>>>>> 1. Future is completed - we have more data
> >>>>>> 2. Future not yet completed - we don’t have data now, but we
> might/we
> >>>> will
> >>>>>> have in the future
> >>>>>>
> >>>>>> While `boolean hasNext()` and `notify()` callback are a bit more
> >>>>>> complicated/dispersed and can lead/encourage `notify()` spam.
> >>>>>>
> >>>>>> b)
> >>>>>>
> >>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> >>>> `getNext`
> >>>>>> the `getNext` would need return a
> >>>>>>> `ElementWithTimestamp` because some sources want to add timestamp
> to
> >>>>>> every element. IMO, this is not so memory friendly
> >>>>>>> so I prefer this design.
> >>>>>>
> >>>>>> Guowei I don’t quite understand this. Could you elaborate why
> having a
> >>>>>> separate `advance()` help?
> >>>>>>
> >>>>>> c)
> >>>>>>
> >>>>>> Regarding advance/poll/take. What’s the value of having two separate
> >>>>>> methods: poll and take? Which one of them should be called and which
> >>>>>> implemented? What’s the benefit of having those methods compared to
> >>>> having
> >>>>>> a one single method `getNextElement()` (or `pollElement() or
> whatever
> >> we
> >>>>>> name it) with following contract:
> >>>>>>
> >>>>>> CompletableFuture<?> isBlocked();
> >>>>>>
> >>>>>> /**
> >>>>>> Return next element - will be called only if `isBlocked()` is
> >> completed.
> >>>>>> Try to implement it in non blocking fashion, but if that’s
> impossible
> >> or
> >>>>>> you just don’t need the effort, you can block in this method.
> >>>>>> */
> >>>>>> T getNextElement();
> >>>>>>
> >>>>>> I mean, if the connector is implemented non-blockingly, Flink should
> >> use
> >>>>>> it that way. If it’s not, then `poll()` will `throw new
> >>>>>> NotImplementedException()`. Implementing both of them and providing
> >>>> both of
> >>>>>> them to Flink wouldn’t make a sense, thus why not merge them into a
> >>>> single
> >>>>>> method call that should preferably (but not necessarily need to) be
> >>>>>> non-blocking? It’s not like we are implementing general purpose
> >> `Queue`,
> >>>>>> which users might want to call either of `poll` or `take`. We would
> >>>> always
> >>>>>> prefer to call `poll`, but if it’s blocking, then still we have no
> >>>> choice,
> >>>>>> but to call it and block on it.
> >>>>>>
> >>>>>> d)
> >>>>>>
> >>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
> very
> >>>>>>> important. But in addition to `Future/poll`, there may be another
> way
> >>>> to
> >>>>>>> achieve this. I think it may be not very memory friendly if every
> >>>> advance
> >>>>>>> call return a Future.
> >>>>>>
> >>>>>> I didn’t want to mention this, to not clog my initial proposal, but
> >>>> there
> >>>>>> is a simple solution for the problem:
> >>>>>>
> >>>>>> public interface SplitReader {
> >>>>>>
> >>>>>>  (…)
> >>>>>>
> >>>>>>  CompletableFuture<?> NOT_BLOCKED =
> >>>>>> CompletableFuture.completedFuture(null);
> >>>>>>
> >>>>>>  /**
> >>>>>>   * Returns a future that will be completed when the page source
> >>>> becomes
> >>>>>>   * unblocked.  If the page source is not blocked, this method
> should
> >>>>>> return
> >>>>>>   * {@code NOT_BLOCKED}.
> >>>>>>   */
> >>>>>>  default CompletableFuture<?> isBlocked()
> >>>>>>  {
> >>>>>>      return NOT_BLOCKED;
> >>>>>>  }
> >>>>>>
> >>>>>> If we are blocked and we are waiting for the IO, then creating a new
> >>>>>> Future is non-issue. Under full throttle/throughput and not blocked
> >>>> sources
> >>>>>> returning a static `NOT_BLOCKED` constant  should also solve the
> >>>> problem.
> >>>>>>
> >>>>>> One more remark, non-blocking sources might be a necessity in a
> single
> >>>>>> threaded model without a checkpointing lock. (Currently when sources
> >> are
> >>>>>> blocked, they can release checkpointing lock and re-acquire it again
> >>>>>> later). Non-blocking `poll`/`getNext()` would allow for checkpoints
> to
> >>>>>> happen when source is idling. In that case either `notify()` or my
> >>>> proposed
> >>>>>> `isBlocked()` would allow to avoid busy-looping.
> >>>>>>
> >>>>>> Piotrek
> >>>>>>
> >>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
> >>>>>>>
> >>>>>>> Hi Thomas,
> >>>>>>>
> >>>>>>> The iterator-like API was also the first thing that came to me. But
> >> it
> >>>>>>> seems a little confusing that hasNext() does not mean "the stream
> has
> >>>> not
> >>>>>>> ended", but means "the next record is ready", which is repurposing
> >> the
> >>>>>> well
> >>>>>>> known meaning of hasNext(). If we follow the hasNext()/next()
> >> pattern,
> >>>> an
> >>>>>>> additional isNextReady() method to indicate whether the next record
> >> is
> >>>>>>> ready seems more intuitive to me.
> >>>>>>>
> >>>>>>> Similarly, in poll()/take() pattern, another method of isDone() is
> >>>> needed
> >>>>>>> to indicate whether the stream has ended or not.
> >>>>>>>
> >>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
> >>>>>>> isDone()/poll()/take() seems more flexible for the reader
> >>>> implementation.
> >>>>>>> When I am implementing a reader, I could have a couple of choices:
> >>>>>>>
> >>>>>>> - A thread-less reader that does not have any internal thread.
> >>>>>>> - When poll() is called, the same calling thread will perform a
> bunch
> >>>>>> of
> >>>>>>>   IO asynchronously.
> >>>>>>>   - When take() is called, the same calling thread will perform a
> >>>>>> bunch
> >>>>>>>   of IO and wait until the record is ready.
> >>>>>>> - A reader with internal threads performing network IO and put
> >> records
> >>>>>>> into a buffer.
> >>>>>>>   - When poll() is called, the calling thread simply reads from the
> >>>>>>>   buffer and return empty result immediately if there is no record.
> >>>>>>>   - When take() is called, the calling thread reads from the buffer
> >>>>>> and
> >>>>>>>   block waiting if the buffer is empty.
> >>>>>>>
> >>>>>>> On the other hand, with the hasNext()/next()/isNextReady() API, it
> is
> >>>>>> less
> >>>>>>> intuitive for the reader developers to write the thread-less
> pattern.
> >>>>>>> Although technically speaking one can still do the asynchronous IO
> to
> >>>>>>> prepare the record in isNextReady(). But it is inexplicit and seems
> >>>>>>> somewhat hacky.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>>
> >>>>>>> Jiangjie (Becket) Qin
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org>
> wrote:
> >>>>>>>
> >>>>>>>> Couple more points regarding discovery:
> >>>>>>>>
> >>>>>>>> The proposal mentions that discovery could be outside the
> execution
> >>>>>> graph.
> >>>>>>>> Today, discovered partitions/shards are checkpointed. I believe
> that
> >>>>>> will
> >>>>>>>> also need to be the case in the future, even when discovery and
> >>>> reading
> >>>>>> are
> >>>>>>>> split between different tasks.
> >>>>>>>>
> >>>>>>>> For cases such as resharding of a Kinesis stream, the relationship
> >>>>>> between
> >>>>>>>> splits needs to be considered. Splits cannot be randomly
> distributed
> >>>>>> over
> >>>>>>>> readers in certain situations. An example was mentioned here:
> >>>>>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >>>>>>>>
> >>>>>>>> Thomas
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org>
> wrote:
> >>>>>>>>
> >>>>>>>>> Thanks for getting the ball rolling on this!
> >>>>>>>>>
> >>>>>>>>> Can the number of splits decrease? Yes, splits can be closed and
> go
> >>>>>> away.
> >>>>>>>>> An example would be a shard merge in Kinesis (2 existing shards
> >> will
> >>>> be
> >>>>>>>>> closed and replaced with a new shard).
> >>>>>>>>>
> >>>>>>>>> Regarding advance/poll/take: IMO the least restrictive approach
> >> would
> >>>>>> be
> >>>>>>>>> the thread-less IO model (pull based, non-blocking, caller
> >> retrieves
> >>>>>> new
> >>>>>>>>> records when available). The current Kinesis API requires the use
> >> of
> >>>>>>>>> threads. But that can be internal to the split reader and does
> not
> >>>> need
> >>>>>>>> to
> >>>>>>>>> be a source API concern. In fact, that's what we are working on
> >> right
> >>>>>> now
> >>>>>>>>> as improvement to the existing consumer: Each shard consumer
> thread
> >>>>>> will
> >>>>>>>>> push to a queue, the consumer main thread will poll the queue(s).
> >> It
> >>>> is
> >>>>>>>>> essentially a mapping from threaded IO to non-blocking.
> >>>>>>>>>
> >>>>>>>>> The proposed SplitReader interface would fit the thread-less IO
> >>>> model.
> >>>>>>>>> Similar to an iterator, we find out if there is a new element
> >>>> (hasNext)
> >>>>>>>> and
> >>>>>>>>> if so, move to it (next()). Separate calls deliver the meta
> >>>> information
> >>>>>>>>> (timestamp, watermark). Perhaps advance call could offer a
> timeout
> >>>>>>>> option,
> >>>>>>>>> so that the caller does not end up in a busy wait. On the other
> >>>> hand, a
> >>>>>>>>> caller processing multiple splits may want to cycle through fast,
> >> to
> >>>>>>>>> process elements of other splits as soon as they become
> available.
> >>>> The
> >>>>>>>> nice
> >>>>>>>>> thing is that this "split merge" logic can now live in Flink and
> be
> >>>>>>>>> optimized and shared between different sources.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>> Thomas
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
> >>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi,
> >>>>>>>>>> Thanks Aljoscha for this FLIP.
> >>>>>>>>>>
> >>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
> >>>> very
> >>>>>>>>>> important. But in addition to `Future/poll`, there may be
> another
> >>>> way
> >>>>>> to
> >>>>>>>>>> achieve this. I think it may be not very memory friendly if
> every
> >>>>>>>> advance
> >>>>>>>>>> call return a Future.
> >>>>>>>>>>
> >>>>>>>>>> public interface Listener {
> >>>>>>>>>>  public void notify();
> >>>>>>>>>> }
> >>>>>>>>>>
> >>>>>>>>>> public interface SplitReader() {
> >>>>>>>>>>  /**
> >>>>>>>>>>   * When there is no element temporarily, this will return
> false.
> >>>>>>>>>>   * When elements is available again splitReader can call
> >>>>>>>>>> listener.notify()
> >>>>>>>>>>   * In addition the frame would check `advance` periodically .
> >>>>>>>>>>   * Of course advance can always return true and ignore the
> >>>>>> listener
> >>>>>>>>>> argument for simplicity.
> >>>>>>>>>>   */
> >>>>>>>>>>  public boolean advance(Listener listener);
> >>>>>>>>>> }
> >>>>>>>>>>
> >>>>>>>>>> 2.  The FLIP tells us very clearly that how to create all Splits
> >> and
> >>>>>> how
> >>>>>>>>>> to create a SplitReader from a Split. But there is no strategy
> for
> >>>> the
> >>>>>>>> user
> >>>>>>>>>> to choose how to assign the splits to the tasks. I think we
> could
> >>>> add
> >>>>>> a
> >>>>>>>>>> Enum to let user to choose.
> >>>>>>>>>> /**
> >>>>>>>>>> public Enum SplitsAssignmentPolicy {
> >>>>>>>>>> Location,
> >>>>>>>>>> Workload,
> >>>>>>>>>> Random,
> >>>>>>>>>> Average
> >>>>>>>>>> }
> >>>>>>>>>> */
> >>>>>>>>>>
> >>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> >>>>>> `getNext`
> >>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp` because
> >>>> some
> >>>>>>>>>> sources want to add timestamp to every element. IMO, this is not
> >> so
> >>>>>>>> memory
> >>>>>>>>>> friendly so I prefer this design.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Thanks
> >>>>>>>>>>
> >>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
> 下午6:08写道:
> >>>>>>>>>>
> >>>>>>>>>>> Hi,
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
> >>>> other
> >>>>>>>>>>> possible improvements. I have one proposal. Instead of having a
> >>>>>> method:
> >>>>>>>>>>>
> >>>>>>>>>>> boolean advance() throws IOException;
> >>>>>>>>>>>
> >>>>>>>>>>> I would replace it with
> >>>>>>>>>>>
> >>>>>>>>>>> /*
> >>>>>>>>>>> * Return a future, which when completed means that source has
> >> more
> >>>>>>>> data
> >>>>>>>>>>> and getNext() will not block.
> >>>>>>>>>>> * If you wish to use benefits of non blocking connectors,
> please
> >>>>>>>>>>> implement this method appropriately.
> >>>>>>>>>>> */
> >>>>>>>>>>> default CompletableFuture<?> isBlocked() {
> >>>>>>>>>>>     return CompletableFuture.completedFuture(null);
> >>>>>>>>>>> }
> >>>>>>>>>>>
> >>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
> >>>>>>>>>>>
> >>>>>>>>>>> Couple of arguments:
> >>>>>>>>>>> 1. I don’t understand the division of work between `advance()`
> >> and
> >>>>>>>>>>> `getCurrent()`. What should be done in which, especially for
> >>>>>> connectors
> >>>>>>>>>>> that handle records in batches (like Kafka) and when should you
> >>>> call
> >>>>>>>>>>> `advance` and when `getCurrent()`.
> >>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow
> us
> >> in
> >>>>>> the
> >>>>>>>>>>> future to have asynchronous/non blocking connectors and more
> >>>>>>>> efficiently
> >>>>>>>>>>> handle large number of blocked threads, without busy waiting.
> >> While
> >>>>>> at
> >>>>>>>> the
> >>>>>>>>>>> same time it doesn’t add much complexity, since naive connector
> >>>>>>>>>>> implementations can be always blocking.
> >>>>>>>>>>> 3. This also would allow us to use a fixed size thread pool of
> >> task
> >>>>>>>>>>> executors, instead of one thread per task.
> >>>>>>>>>>>
> >>>>>>>>>>> Piotrek
> >>>>>>>>>>>
> >>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
> aljoscha@apache.org
> >>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>> Hi All,
> >>>>>>>>>>>>
> >>>>>>>>>>>> In order to finally get the ball rolling on the new source
> >>>> interface
> >>>>>>>>>>> that we have discussed for so long I finally created a FLIP:
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>>>>>>>
> >>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing
> work/discussion
> >>>>>> about
> >>>>>>>>>>> adding per-partition watermark support to the Kinesis source
> and
> >>>>>>>> because
> >>>>>>>>>>> this would enable generic implementation of event-time
> alignment
> >>>> for
> >>>>>>>> all
> >>>>>>>>>>> sources. Maybe we need another FLIP for the event-time
> alignment
> >>>>>> part,
> >>>>>>>>>>> especially the part about information sharing between
> operations
> >>>> (I'm
> >>>>>>>> not
> >>>>>>>>>>> calling it state sharing because state has a special meaning in
> >>>>>> Flink).
> >>>>>>>>>>>>
> >>>>>>>>>>>> Please discuss away!
> >>>>>>>>>>>>
> >>>>>>>>>>>> Aljoscha
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>>
> >>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@data-artisans.com>.
Hi,

Thanks again for the detailed answer :) Sorry for responding with a delay.

> Completely agree that in pattern 2, having a callback is necessary for that
> single thread outside of the connectors. And the connectors MUST have
> internal threads.

Yes, this thread will have to exists somewhere. In pattern 2 it exists in the connector (at least from the perspective of the Flink execution engine). In pattern 1 it exists inside the Flink execution engine. With completely blocking connectors, like simple reading from files, both of those approaches are basically the same. The difference is when user implementing Flink source is already working with a non blocking code with some internal threads. In this case, pattern 1 would result in "double thread wrapping”, while pattern 2 would allow to skip one layer of indirection.

> If we go that way, we should have something like "void
> poll(Callback) / void advance(callback)". I am curious how would
> CompletableFuture work here, though. If 10 readers returns 10 completable
> futures, will there be 10 additional threads (so 20 threads in total)
> blocking waiting on them? Or will there be a single thread busy loop
> checking around?

To be honest, I haven’t thought this completely through and I haven’t tested/POC’ed it. Having said that, I can think of at least couple of solutions. First is something like this:

https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507 <https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/execution/executor/TaskExecutor.java#L481-L507>

Line:

                                `blocked = split.process();`

Is where the execution goes into to the task/sources. This is where the returned future is handled:

                                blocked.addListener(() -> {
                                    blockedSplits.remove(split);
                                    // reset the level priority to prevent previously-blocked splits from starving existing splits
                                    split.resetLevelPriority();
                                    waitingSplits.offer(split);
                                }, executor);

Fundamentally callbacks and Futures are more or less interchangeable You can always wrap one into another (creating a callback that completes a future and attach a callback once future completes). In this case the difference for me is mostly:
- api with passing callback allows the callback to be fired multiple times and to fire it even if the connector is not blocked. This is what I meant by saying that api `CompletableFuture<?> isBlocked()` is a bit simpler. Connector can only return either “I’m not blocked” or “I’m blocked and I will tell you only once when I’m not blocked anymore”.

But this is not the most important thing for me here. For me important thing is to try our best to make Flink task’s control and execution single threaded. For that both callback and future APIs should work the same.

> WRT pattern 1, a single blocking take() API should just work. The good
> thing is that a blocking read API is usually simpler to implement. 

Yes, they are easier to implement (especially if you are not the one that have to deal with the additional threading required around them ;) ). But to answer this issue, if we choose pattern 2, we can always provide a proxy/wrapper that would using the internal thread implement the non-blocking API while exposing blocking API to the user. It would implement pattern 2 for the user exposing to him pattern 1. In other words implementing pattern 1 in pattern 2 paradigm, while making it possible to implement pure pattern 2 connectors.

> BTW, one thing I am also trying to avoid is pushing users to perform IO in
> a method like "isBlocked()". If the method is expected to fetch records
> (even if not returning them), naming it something more explicit would help
> avoid confusion.

If we choose so, we could rework it into something like:

CompletableFuture<?> advance()
T getCurrent();
Watermark getCurrentWatermark()

But as I wrote before, this is more confusing to me for the exact reasons you mentioned :) I would be confused what should be done in `adanvce()` and what in `getCurrent()`. However, again this naming issue is not that important to me and probably is matter of taste/personal preferences.

Piotrek 

> On 9 Nov 2018, at 18:37, Becket Qin <be...@gmail.com> wrote:
> 
> Hi Piotrek,
> 
> Thanks for the explanation. We are probably talking about the same thing
> but in different ways. To clarify a little bit, I think there are two
> patterns to read from a connector.
> 
> Pattern 1: Thread-less connector with a blocking read API. Outside of the
> connector, there is one IO thread per reader, doing blocking read. An
> additional thread will interact with all the IO threads.
> Pattern 2: Connector with internal thread(s) and non-blocking API. Outside
> of the connector, there is one thread for ALL readers, doing IO relying on
> notification callbacks in the reader.
> 
> In both patterns, there must be at least one thread per connector, either
> inside (created by connector writers) or outside (created by Flink) of the
> connector. Ideally there are NUM_CONNECTORS + 1 threads in total, to make
> sure that 1 thread is fully non-blocking.
> 
>> Btw, I don’t know if you understand my point. Having only `poll()` and
> `take()` is not enough for single threaded task. If our source interface
> doesn’t provide `notify()` callback nor >`CompletableFuture<?>
> isBlocked(),`, there is no way to implement single threaded task that both
> reads the data from the source connector and can also react to system
> events. Ok, non >blocking `poll()` would allow that, but with busy looping.
> 
> Completely agree that in pattern 2, having a callback is necessary for that
> single thread outside of the connectors. And the connectors MUST have
> internal threads. If we go that way, we should have something like "void
> poll(Callback) / void advance(callback)". I am curious how would
> CompletableFuture work here, though. If 10 readers returns 10 completable
> futures, will there be 10 additional threads (so 20 threads in total)
> blocking waiting on them? Or will there be a single thread busy loop
> checking around?
> 
> WRT pattern 1, a single blocking take() API should just work. The good
> thing is that a blocking read API is usually simpler to implement. An
> additional non-blocking "T poll()" method here is indeed optional and could
> be used in cases like Flink does not want the thread to block forever. They
> can also be combined to have a "T poll(Timeout)", which is exactly what
> KafkaConsumer did.
> 
> It sounds that you are proposing pattern 2 with something similar to NIO2
> AsynchronousByteChannel[1]. That API would work, except that the signature
> returning future seems not necessary. If that is the case, a minor change
> on the current FLIP proposal to have "void advance(callback)" should work.
> And this means the connectors MUST have their internal threads.
> 
> BTW, one thing I am also trying to avoid is pushing users to perform IO in
> a method like "isBlocked()". If the method is expected to fetch records
> (even if not returning them), naming it something more explicit would help
> avoid confusion.
> 
> Thanks,
> 
> Jiangjie (Becket) Qin
> 
> [1]
> https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html
> 
> On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <pi...@data-artisans.com>
> wrote:
> 
>> Hi
>> 
>> Good point with select/epoll, however I do not see how they couldn’t be
>> with Flink if we would like single task in Flink to be single-threaded (and
>> I believe we should pursue this goal). If your connector blocks on
>> `select`, then it can not process/handle control messages from Flink, like
>> checkpoints, releasing resources and potentially output flushes. This would
>> require tight integration between connector and Flink’s main event
>> loop/selects/etc.
>> 
>> Looking at it from other perspective. Let’s assume that we have a
>> connector implemented on top of `select`/`epoll`. In order to integrate it
>> with Flink’s checkpointing/flushes/resource releasing it will have to be
>> executed in separate thread one way or another. At least if our API will
>> enforce/encourage non blocking implementations with some kind of
>> notifications (`isBlocked()` or `notify()` callback), some connectors might
>> skip one layer of wapping threads.
>> 
>> Btw, I don’t know if you understand my point. Having only `poll()` and
>> `take()` is not enough for single threaded task. If our source interface
>> doesn’t provide `notify()` callback nor `CompletableFuture<?>
>> isBlocked(),`, there is no way to implement single threaded task that both
>> reads the data from the source connector and can also react to system
>> events. Ok, non blocking `poll()` would allow that, but with busy looping.
>> 
>> Piotrek
>> 
>>> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com> wrote:
>>> 
>>> Hi Piotrek,
>>> 
>>>> But I don’t see a reason why we should expose both blocking `take()` and
>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink engine
>> or
>>> connector) would have to do the same busy
>>>> looping anyway and I think it would be better to have a simpler
>> connector
>>> API (that would solve our problems) and force connectors to comply one
>> way
>>> or another.
>>> 
>>> If we let the block happen inside the connector, the blocking does not
>> have
>>> to be a busy loop. For example, to do the block waiting efficiently, the
>>> connector can use java NIO selector().select which relies on OS syscall
>>> like epoll[1] instead of busy looping. But if Flink engine blocks outside
>>> the connector, it pretty much has to do the busy loop. So if there is
>> only
>>> one API to get the element, a blocking getNextElement() makes more sense.
>>> In any case, we should avoid ambiguity. It has to be crystal clear about
>>> whether a method is expected to be blocking or non-blocking. Otherwise it
>>> would be very difficult for Flink engine to do the right thing with the
>>> connectors. At the first glance at getCurrent(), the expected behavior is
>>> not quite clear.
>>> 
>>> That said, I do agree that functionality wise, poll() and take() kind of
>>> overlap. But they are actually not quite different from
>>> isBlocked()/getNextElement(). Compared with isBlocked(), the only
>>> difference is that poll() also returns the next record if it is
>> available.
>>> But I agree that the isBlocked() + getNextElement() is more flexible as
>>> users can just check the record availability, but not fetch the next
>>> element.
>>> 
>>>> In case of thread-less readers with only non-blocking `queue.poll()` (is
>>> that really a thing? I can not think about a real implementation that
>>> enforces such constraints)
>>> Right, it is pretty much a syntax sugar to allow user combine the
>>> check-and-take into one method. It could be achieved with isBlocked() +
>>> getNextElement().
>>> 
>>> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
>>> 
>>> Thanks,
>>> 
>>> Jiangjie (Becket) Qin
>>> 
>>> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <pi...@data-artisans.com>
>>> wrote:
>>> 
>>>> Hi Becket,
>>>> 
>>>> With my proposal, both of your examples would have to be solved by the
>>>> connector and solution to both problems would be the same:
>>>> 
>>>> Pretend that connector is never blocked (`isBlocked() { return
>>>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking fashion
>> (or
>>>> semi blocking with return of control from time to time to allow for
>>>> checkpointing, network flushing and other resource management things to
>>>> happen in the same main thread). In other words, exactly how you would
>>>> implement `take()` method or how the same source connector would be
>>>> implemented NOW with current source interface. The difference with
>> current
>>>> interface would be only that main loop would be outside of the
>> connector,
>>>> and instead of periodically releasing checkpointing lock, periodically
>>>> `return null;` or `return Optional.empty();` from `getNextElement()`.
>>>> 
>>>> In case of thread-less readers with only non-blocking `queue.poll()` (is
>>>> that really a thing? I can not think about a real implementation that
>>>> enforces such constraints), we could provide a wrapper that hides the
>> busy
>>>> looping. The same applies how to solve forever blocking readers - we
>> could
>>>> provider another wrapper running the connector in separate thread.
>>>> 
>>>> But I don’t see a reason why we should expose both blocking `take()` and
>>>> non-blocking `poll()` methods to the Flink engine. Someone (Flink
>> engine or
>>>> connector) would have to do the same busy looping anyway and I think it
>>>> would be better to have a simpler connector API (that would solve our
>>>> problems) and force connectors to comply one way or another.
>>>> 
>>>> Piotrek
>>>> 
>>>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
>>>>> 
>>>>> Hi Piotr,
>>>>> 
>>>>> I might have misunderstood you proposal. But let me try to explain my
>>>>> concern. I am thinking about the following case:
>>>>> 1. a reader has the following two interfaces,
>>>>>  boolean isBlocked()
>>>>>  T getNextElement()
>>>>> 2. the implementation of getNextElement() is non-blocking.
>>>>> 3. The reader is thread-less, i.e. it does not have any internal
>> thread.
>>>>> For example, it might just delegate the getNextElement() to a
>>>> queue.poll(),
>>>>> and isBlocked() is just queue.isEmpty().
>>>>> 
>>>>> How can Flink efficiently implement a blocking reading behavior with
>> this
>>>>> reader? Either a tight loop or a backoff interval is needed. Neither of
>>>>> them is ideal.
>>>>> 
>>>>> Now let's say in the reader mentioned above implements a blocking
>>>>> getNextElement() method. Because there is no internal thread in the
>>>> reader,
>>>>> after isBlocked() returns false. Flink will still have to loop on
>>>>> isBlocked() to check whether the next record is available. If the next
>>>>> record reaches after 10 min, it is a tight loop for 10 min. You have
>>>>> probably noticed that in this case, even isBlocked() returns a future,
>>>> that
>>>>> future() will not be completed if Flink does not call some method from
>>>> the
>>>>> reader, because the reader has no internal thread to complete that
>> future
>>>>> by itself.
>>>>> 
>>>>> Due to the above reasons, a blocking take() API would allow Flink to
>> have
>>>>> an efficient way to read from a reader. There are many ways to wake up
>>>> the
>>>>> blocking thread when checkpointing is needed depending on the
>>>>> implementation. But I think the poll()/take() API would also work in
>> that
>>>>> case.
>>>>> 
>>>>> Thanks,
>>>>> 
>>>>> Jiangjie (Becket) Qin
>>>>> 
>>>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <piotr@data-artisans.com
>>> 
>>>>> wrote:
>>>>> 
>>>>>> Hi,
>>>>>> 
>>>>>> a)
>>>>>> 
>>>>>>> BTW, regarding the isBlock() method, I have a few more questions. 21,
>>>> Is
>>>>>> a method isReady() with boolean as a return value
>>>>>>> equivalent? Personally I found it is a little bit confusing in what
>> is
>>>>>> supposed to be returned when the future is completed. 22. if
>>>>>>> the implementation of isBlocked() is optional, how do the callers
>> know
>>>>>> whether the method is properly implemented or not?
>>>>>>> Does not implemented mean it always return a completed future?
>>>>>> 
>>>>>> `CompletableFuture<?> isBlocked()` is more or less an equivalent to
>>>>>> `boolean hasNext()` which in case of “false” provides some kind of a
>>>>>> listener/callback that notifies about presence of next element. There
>>>> are
>>>>>> some minor details, like `CompletableFuture<?>` has a minimal two
>> state
>>>>>> logic:
>>>>>> 
>>>>>> 1. Future is completed - we have more data
>>>>>> 2. Future not yet completed - we don’t have data now, but we might/we
>>>> will
>>>>>> have in the future
>>>>>> 
>>>>>> While `boolean hasNext()` and `notify()` callback are a bit more
>>>>>> complicated/dispersed and can lead/encourage `notify()` spam.
>>>>>> 
>>>>>> b)
>>>>>> 
>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>>>> `getNext`
>>>>>> the `getNext` would need return a
>>>>>>> `ElementWithTimestamp` because some sources want to add timestamp to
>>>>>> every element. IMO, this is not so memory friendly
>>>>>>> so I prefer this design.
>>>>>> 
>>>>>> Guowei I don’t quite understand this. Could you elaborate why having a
>>>>>> separate `advance()` help?
>>>>>> 
>>>>>> c)
>>>>>> 
>>>>>> Regarding advance/poll/take. What’s the value of having two separate
>>>>>> methods: poll and take? Which one of them should be called and which
>>>>>> implemented? What’s the benefit of having those methods compared to
>>>> having
>>>>>> a one single method `getNextElement()` (or `pollElement() or whatever
>> we
>>>>>> name it) with following contract:
>>>>>> 
>>>>>> CompletableFuture<?> isBlocked();
>>>>>> 
>>>>>> /**
>>>>>> Return next element - will be called only if `isBlocked()` is
>> completed.
>>>>>> Try to implement it in non blocking fashion, but if that’s impossible
>> or
>>>>>> you just don’t need the effort, you can block in this method.
>>>>>> */
>>>>>> T getNextElement();
>>>>>> 
>>>>>> I mean, if the connector is implemented non-blockingly, Flink should
>> use
>>>>>> it that way. If it’s not, then `poll()` will `throw new
>>>>>> NotImplementedException()`. Implementing both of them and providing
>>>> both of
>>>>>> them to Flink wouldn’t make a sense, thus why not merge them into a
>>>> single
>>>>>> method call that should preferably (but not necessarily need to) be
>>>>>> non-blocking? It’s not like we are implementing general purpose
>> `Queue`,
>>>>>> which users might want to call either of `poll` or `take`. We would
>>>> always
>>>>>> prefer to call `poll`, but if it’s blocking, then still we have no
>>>> choice,
>>>>>> but to call it and block on it.
>>>>>> 
>>>>>> d)
>>>>>> 
>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is very
>>>>>>> important. But in addition to `Future/poll`, there may be another way
>>>> to
>>>>>>> achieve this. I think it may be not very memory friendly if every
>>>> advance
>>>>>>> call return a Future.
>>>>>> 
>>>>>> I didn’t want to mention this, to not clog my initial proposal, but
>>>> there
>>>>>> is a simple solution for the problem:
>>>>>> 
>>>>>> public interface SplitReader {
>>>>>> 
>>>>>>  (…)
>>>>>> 
>>>>>>  CompletableFuture<?> NOT_BLOCKED =
>>>>>> CompletableFuture.completedFuture(null);
>>>>>> 
>>>>>>  /**
>>>>>>   * Returns a future that will be completed when the page source
>>>> becomes
>>>>>>   * unblocked.  If the page source is not blocked, this method should
>>>>>> return
>>>>>>   * {@code NOT_BLOCKED}.
>>>>>>   */
>>>>>>  default CompletableFuture<?> isBlocked()
>>>>>>  {
>>>>>>      return NOT_BLOCKED;
>>>>>>  }
>>>>>> 
>>>>>> If we are blocked and we are waiting for the IO, then creating a new
>>>>>> Future is non-issue. Under full throttle/throughput and not blocked
>>>> sources
>>>>>> returning a static `NOT_BLOCKED` constant  should also solve the
>>>> problem.
>>>>>> 
>>>>>> One more remark, non-blocking sources might be a necessity in a single
>>>>>> threaded model without a checkpointing lock. (Currently when sources
>> are
>>>>>> blocked, they can release checkpointing lock and re-acquire it again
>>>>>> later). Non-blocking `poll`/`getNext()` would allow for checkpoints to
>>>>>> happen when source is idling. In that case either `notify()` or my
>>>> proposed
>>>>>> `isBlocked()` would allow to avoid busy-looping.
>>>>>> 
>>>>>> Piotrek
>>>>>> 
>>>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
>>>>>>> 
>>>>>>> Hi Thomas,
>>>>>>> 
>>>>>>> The iterator-like API was also the first thing that came to me. But
>> it
>>>>>>> seems a little confusing that hasNext() does not mean "the stream has
>>>> not
>>>>>>> ended", but means "the next record is ready", which is repurposing
>> the
>>>>>> well
>>>>>>> known meaning of hasNext(). If we follow the hasNext()/next()
>> pattern,
>>>> an
>>>>>>> additional isNextReady() method to indicate whether the next record
>> is
>>>>>>> ready seems more intuitive to me.
>>>>>>> 
>>>>>>> Similarly, in poll()/take() pattern, another method of isDone() is
>>>> needed
>>>>>>> to indicate whether the stream has ended or not.
>>>>>>> 
>>>>>>> Compared with hasNext()/next()/isNextReady() pattern,
>>>>>>> isDone()/poll()/take() seems more flexible for the reader
>>>> implementation.
>>>>>>> When I am implementing a reader, I could have a couple of choices:
>>>>>>> 
>>>>>>> - A thread-less reader that does not have any internal thread.
>>>>>>> - When poll() is called, the same calling thread will perform a bunch
>>>>>> of
>>>>>>>   IO asynchronously.
>>>>>>>   - When take() is called, the same calling thread will perform a
>>>>>> bunch
>>>>>>>   of IO and wait until the record is ready.
>>>>>>> - A reader with internal threads performing network IO and put
>> records
>>>>>>> into a buffer.
>>>>>>>   - When poll() is called, the calling thread simply reads from the
>>>>>>>   buffer and return empty result immediately if there is no record.
>>>>>>>   - When take() is called, the calling thread reads from the buffer
>>>>>> and
>>>>>>>   block waiting if the buffer is empty.
>>>>>>> 
>>>>>>> On the other hand, with the hasNext()/next()/isNextReady() API, it is
>>>>>> less
>>>>>>> intuitive for the reader developers to write the thread-less pattern.
>>>>>>> Although technically speaking one can still do the asynchronous IO to
>>>>>>> prepare the record in isNextReady(). But it is inexplicit and seems
>>>>>>> somewhat hacky.
>>>>>>> 
>>>>>>> Thanks,
>>>>>>> 
>>>>>>> Jiangjie (Becket) Qin
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
>>>>>>> 
>>>>>>>> Couple more points regarding discovery:
>>>>>>>> 
>>>>>>>> The proposal mentions that discovery could be outside the execution
>>>>>> graph.
>>>>>>>> Today, discovered partitions/shards are checkpointed. I believe that
>>>>>> will
>>>>>>>> also need to be the case in the future, even when discovery and
>>>> reading
>>>>>> are
>>>>>>>> split between different tasks.
>>>>>>>> 
>>>>>>>> For cases such as resharding of a Kinesis stream, the relationship
>>>>>> between
>>>>>>>> splits needs to be considered. Splits cannot be randomly distributed
>>>>>> over
>>>>>>>> readers in certain situations. An example was mentioned here:
>>>>>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>>>>>>>> 
>>>>>>>> Thomas
>>>>>>>> 
>>>>>>>> 
>>>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
>>>>>>>> 
>>>>>>>>> Thanks for getting the ball rolling on this!
>>>>>>>>> 
>>>>>>>>> Can the number of splits decrease? Yes, splits can be closed and go
>>>>>> away.
>>>>>>>>> An example would be a shard merge in Kinesis (2 existing shards
>> will
>>>> be
>>>>>>>>> closed and replaced with a new shard).
>>>>>>>>> 
>>>>>>>>> Regarding advance/poll/take: IMO the least restrictive approach
>> would
>>>>>> be
>>>>>>>>> the thread-less IO model (pull based, non-blocking, caller
>> retrieves
>>>>>> new
>>>>>>>>> records when available). The current Kinesis API requires the use
>> of
>>>>>>>>> threads. But that can be internal to the split reader and does not
>>>> need
>>>>>>>> to
>>>>>>>>> be a source API concern. In fact, that's what we are working on
>> right
>>>>>> now
>>>>>>>>> as improvement to the existing consumer: Each shard consumer thread
>>>>>> will
>>>>>>>>> push to a queue, the consumer main thread will poll the queue(s).
>> It
>>>> is
>>>>>>>>> essentially a mapping from threaded IO to non-blocking.
>>>>>>>>> 
>>>>>>>>> The proposed SplitReader interface would fit the thread-less IO
>>>> model.
>>>>>>>>> Similar to an iterator, we find out if there is a new element
>>>> (hasNext)
>>>>>>>> and
>>>>>>>>> if so, move to it (next()). Separate calls deliver the meta
>>>> information
>>>>>>>>> (timestamp, watermark). Perhaps advance call could offer a timeout
>>>>>>>> option,
>>>>>>>>> so that the caller does not end up in a busy wait. On the other
>>>> hand, a
>>>>>>>>> caller processing multiple splits may want to cycle through fast,
>> to
>>>>>>>>> process elements of other splits as soon as they become available.
>>>> The
>>>>>>>> nice
>>>>>>>>> thing is that this "split merge" logic can now live in Flink and be
>>>>>>>>> optimized and shared between different sources.
>>>>>>>>> 
>>>>>>>>> Thanks,
>>>>>>>>> Thomas
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Hi,
>>>>>>>>>> Thanks Aljoscha for this FLIP.
>>>>>>>>>> 
>>>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
>>>> very
>>>>>>>>>> important. But in addition to `Future/poll`, there may be another
>>>> way
>>>>>> to
>>>>>>>>>> achieve this. I think it may be not very memory friendly if every
>>>>>>>> advance
>>>>>>>>>> call return a Future.
>>>>>>>>>> 
>>>>>>>>>> public interface Listener {
>>>>>>>>>>  public void notify();
>>>>>>>>>> }
>>>>>>>>>> 
>>>>>>>>>> public interface SplitReader() {
>>>>>>>>>>  /**
>>>>>>>>>>   * When there is no element temporarily, this will return false.
>>>>>>>>>>   * When elements is available again splitReader can call
>>>>>>>>>> listener.notify()
>>>>>>>>>>   * In addition the frame would check `advance` periodically .
>>>>>>>>>>   * Of course advance can always return true and ignore the
>>>>>> listener
>>>>>>>>>> argument for simplicity.
>>>>>>>>>>   */
>>>>>>>>>>  public boolean advance(Listener listener);
>>>>>>>>>> }
>>>>>>>>>> 
>>>>>>>>>> 2.  The FLIP tells us very clearly that how to create all Splits
>> and
>>>>>> how
>>>>>>>>>> to create a SplitReader from a Split. But there is no strategy for
>>>> the
>>>>>>>> user
>>>>>>>>>> to choose how to assign the splits to the tasks. I think we could
>>>> add
>>>>>> a
>>>>>>>>>> Enum to let user to choose.
>>>>>>>>>> /**
>>>>>>>>>> public Enum SplitsAssignmentPolicy {
>>>>>>>>>> Location,
>>>>>>>>>> Workload,
>>>>>>>>>> Random,
>>>>>>>>>> Average
>>>>>>>>>> }
>>>>>>>>>> */
>>>>>>>>>> 
>>>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>>>>>> `getNext`
>>>>>>>>>> the `getNext` would need return a `ElementWithTimestamp` because
>>>> some
>>>>>>>>>> sources want to add timestamp to every element. IMO, this is not
>> so
>>>>>>>> memory
>>>>>>>>>> friendly so I prefer this design.
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> Thanks
>>>>>>>>>> 
>>>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
>>>>>>>>>> 
>>>>>>>>>>> Hi,
>>>>>>>>>>> 
>>>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
>>>> other
>>>>>>>>>>> possible improvements. I have one proposal. Instead of having a
>>>>>> method:
>>>>>>>>>>> 
>>>>>>>>>>> boolean advance() throws IOException;
>>>>>>>>>>> 
>>>>>>>>>>> I would replace it with
>>>>>>>>>>> 
>>>>>>>>>>> /*
>>>>>>>>>>> * Return a future, which when completed means that source has
>> more
>>>>>>>> data
>>>>>>>>>>> and getNext() will not block.
>>>>>>>>>>> * If you wish to use benefits of non blocking connectors, please
>>>>>>>>>>> implement this method appropriately.
>>>>>>>>>>> */
>>>>>>>>>>> default CompletableFuture<?> isBlocked() {
>>>>>>>>>>>     return CompletableFuture.completedFuture(null);
>>>>>>>>>>> }
>>>>>>>>>>> 
>>>>>>>>>>> And rename `getCurrent()` to `getNext()`.
>>>>>>>>>>> 
>>>>>>>>>>> Couple of arguments:
>>>>>>>>>>> 1. I don’t understand the division of work between `advance()`
>> and
>>>>>>>>>>> `getCurrent()`. What should be done in which, especially for
>>>>>> connectors
>>>>>>>>>>> that handle records in batches (like Kafka) and when should you
>>>> call
>>>>>>>>>>> `advance` and when `getCurrent()`.
>>>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us
>> in
>>>>>> the
>>>>>>>>>>> future to have asynchronous/non blocking connectors and more
>>>>>>>> efficiently
>>>>>>>>>>> handle large number of blocked threads, without busy waiting.
>> While
>>>>>> at
>>>>>>>> the
>>>>>>>>>>> same time it doesn’t add much complexity, since naive connector
>>>>>>>>>>> implementations can be always blocking.
>>>>>>>>>>> 3. This also would allow us to use a fixed size thread pool of
>> task
>>>>>>>>>>> executors, instead of one thread per task.
>>>>>>>>>>> 
>>>>>>>>>>> Piotrek
>>>>>>>>>>> 
>>>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <aljoscha@apache.org
>>> 
>>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>> Hi All,
>>>>>>>>>>>> 
>>>>>>>>>>>> In order to finally get the ball rolling on the new source
>>>> interface
>>>>>>>>>>> that we have discussed for so long I finally created a FLIP:
>>>>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>>>>>>> 
>>>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
>>>>>> about
>>>>>>>>>>> adding per-partition watermark support to the Kinesis source and
>>>>>>>> because
>>>>>>>>>>> this would enable generic implementation of event-time alignment
>>>> for
>>>>>>>> all
>>>>>>>>>>> sources. Maybe we need another FLIP for the event-time alignment
>>>>>> part,
>>>>>>>>>>> especially the part about information sharing between operations
>>>> (I'm
>>>>>>>> not
>>>>>>>>>>> calling it state sharing because state has a special meaning in
>>>>>> Flink).
>>>>>>>>>>>> 
>>>>>>>>>>>> Please discuss away!
>>>>>>>>>>>> 
>>>>>>>>>>>> Aljoscha
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>> 
>>>>>> 
>>>>>> 
>>>> 
>>>> 
>> 
>> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Piotrek,

Thanks for the explanation. We are probably talking about the same thing
but in different ways. To clarify a little bit, I think there are two
patterns to read from a connector.

Pattern 1: Thread-less connector with a blocking read API. Outside of the
connector, there is one IO thread per reader, doing blocking read. An
additional thread will interact with all the IO threads.
Pattern 2: Connector with internal thread(s) and non-blocking API. Outside
of the connector, there is one thread for ALL readers, doing IO relying on
notification callbacks in the reader.

In both patterns, there must be at least one thread per connector, either
inside (created by connector writers) or outside (created by Flink) of the
connector. Ideally there are NUM_CONNECTORS + 1 threads in total, to make
sure that 1 thread is fully non-blocking.

>Btw, I don’t know if you understand my point. Having only `poll()` and
`take()` is not enough for single threaded task. If our source interface
doesn’t provide `notify()` callback nor >`CompletableFuture<?>
isBlocked(),`, there is no way to implement single threaded task that both
reads the data from the source connector and can also react to system
events. Ok, non >blocking `poll()` would allow that, but with busy looping.

Completely agree that in pattern 2, having a callback is necessary for that
single thread outside of the connectors. And the connectors MUST have
internal threads. If we go that way, we should have something like "void
poll(Callback) / void advance(callback)". I am curious how would
CompletableFuture work here, though. If 10 readers returns 10 completable
futures, will there be 10 additional threads (so 20 threads in total)
blocking waiting on them? Or will there be a single thread busy loop
checking around?

WRT pattern 1, a single blocking take() API should just work. The good
thing is that a blocking read API is usually simpler to implement. An
additional non-blocking "T poll()" method here is indeed optional and could
be used in cases like Flink does not want the thread to block forever. They
can also be combined to have a "T poll(Timeout)", which is exactly what
KafkaConsumer did.

It sounds that you are proposing pattern 2 with something similar to NIO2
AsynchronousByteChannel[1]. That API would work, except that the signature
returning future seems not necessary. If that is the case, a minor change
on the current FLIP proposal to have "void advance(callback)" should work.
And this means the connectors MUST have their internal threads.

BTW, one thing I am also trying to avoid is pushing users to perform IO in
a method like "isBlocked()". If the method is expected to fetch records
(even if not returning them), naming it something more explicit would help
avoid confusion.

Thanks,

Jiangjie (Becket) Qin

[1]
https://docs.oracle.com/javase/8/docs/api/java/nio/channels/AsynchronousByteChannel.html

On Fri, Nov 9, 2018 at 11:20 PM Piotr Nowojski <pi...@data-artisans.com>
wrote:

> Hi
>
> Good point with select/epoll, however I do not see how they couldn’t be
> with Flink if we would like single task in Flink to be single-threaded (and
> I believe we should pursue this goal). If your connector blocks on
> `select`, then it can not process/handle control messages from Flink, like
> checkpoints, releasing resources and potentially output flushes. This would
> require tight integration between connector and Flink’s main event
> loop/selects/etc.
>
> Looking at it from other perspective. Let’s assume that we have a
> connector implemented on top of `select`/`epoll`. In order to integrate it
> with Flink’s checkpointing/flushes/resource releasing it will have to be
> executed in separate thread one way or another. At least if our API will
> enforce/encourage non blocking implementations with some kind of
> notifications (`isBlocked()` or `notify()` callback), some connectors might
> skip one layer of wapping threads.
>
> Btw, I don’t know if you understand my point. Having only `poll()` and
> `take()` is not enough for single threaded task. If our source interface
> doesn’t provide `notify()` callback nor `CompletableFuture<?>
> isBlocked(),`, there is no way to implement single threaded task that both
> reads the data from the source connector and can also react to system
> events. Ok, non blocking `poll()` would allow that, but with busy looping.
>
> Piotrek
>
> > On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com> wrote:
> >
> > Hi Piotrek,
> >
> >> But I don’t see a reason why we should expose both blocking `take()` and
> > non-blocking `poll()` methods to the Flink engine. Someone (Flink engine
> or
> > connector) would have to do the same busy
> >> looping anyway and I think it would be better to have a simpler
> connector
> > API (that would solve our problems) and force connectors to comply one
> way
> > or another.
> >
> > If we let the block happen inside the connector, the blocking does not
> have
> > to be a busy loop. For example, to do the block waiting efficiently, the
> > connector can use java NIO selector().select which relies on OS syscall
> > like epoll[1] instead of busy looping. But if Flink engine blocks outside
> > the connector, it pretty much has to do the busy loop. So if there is
> only
> > one API to get the element, a blocking getNextElement() makes more sense.
> > In any case, we should avoid ambiguity. It has to be crystal clear about
> > whether a method is expected to be blocking or non-blocking. Otherwise it
> > would be very difficult for Flink engine to do the right thing with the
> > connectors. At the first glance at getCurrent(), the expected behavior is
> > not quite clear.
> >
> > That said, I do agree that functionality wise, poll() and take() kind of
> > overlap. But they are actually not quite different from
> > isBlocked()/getNextElement(). Compared with isBlocked(), the only
> > difference is that poll() also returns the next record if it is
> available.
> > But I agree that the isBlocked() + getNextElement() is more flexible as
> > users can just check the record availability, but not fetch the next
> > element.
> >
> >> In case of thread-less readers with only non-blocking `queue.poll()` (is
> > that really a thing? I can not think about a real implementation that
> > enforces such constraints)
> > Right, it is pretty much a syntax sugar to allow user combine the
> > check-and-take into one method. It could be achieved with isBlocked() +
> > getNextElement().
> >
> > [1] http://man7.org/linux/man-pages/man7/epoll.7.html
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <pi...@data-artisans.com>
> > wrote:
> >
> >> Hi Becket,
> >>
> >> With my proposal, both of your examples would have to be solved by the
> >> connector and solution to both problems would be the same:
> >>
> >> Pretend that connector is never blocked (`isBlocked() { return
> >> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking fashion
> (or
> >> semi blocking with return of control from time to time to allow for
> >> checkpointing, network flushing and other resource management things to
> >> happen in the same main thread). In other words, exactly how you would
> >> implement `take()` method or how the same source connector would be
> >> implemented NOW with current source interface. The difference with
> current
> >> interface would be only that main loop would be outside of the
> connector,
> >> and instead of periodically releasing checkpointing lock, periodically
> >> `return null;` or `return Optional.empty();` from `getNextElement()`.
> >>
> >> In case of thread-less readers with only non-blocking `queue.poll()` (is
> >> that really a thing? I can not think about a real implementation that
> >> enforces such constraints), we could provide a wrapper that hides the
> busy
> >> looping. The same applies how to solve forever blocking readers - we
> could
> >> provider another wrapper running the connector in separate thread.
> >>
> >> But I don’t see a reason why we should expose both blocking `take()` and
> >> non-blocking `poll()` methods to the Flink engine. Someone (Flink
> engine or
> >> connector) would have to do the same busy looping anyway and I think it
> >> would be better to have a simpler connector API (that would solve our
> >> problems) and force connectors to comply one way or another.
> >>
> >> Piotrek
> >>
> >>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
> >>>
> >>> Hi Piotr,
> >>>
> >>> I might have misunderstood you proposal. But let me try to explain my
> >>> concern. I am thinking about the following case:
> >>> 1. a reader has the following two interfaces,
> >>>   boolean isBlocked()
> >>>   T getNextElement()
> >>> 2. the implementation of getNextElement() is non-blocking.
> >>> 3. The reader is thread-less, i.e. it does not have any internal
> thread.
> >>> For example, it might just delegate the getNextElement() to a
> >> queue.poll(),
> >>> and isBlocked() is just queue.isEmpty().
> >>>
> >>> How can Flink efficiently implement a blocking reading behavior with
> this
> >>> reader? Either a tight loop or a backoff interval is needed. Neither of
> >>> them is ideal.
> >>>
> >>> Now let's say in the reader mentioned above implements a blocking
> >>> getNextElement() method. Because there is no internal thread in the
> >> reader,
> >>> after isBlocked() returns false. Flink will still have to loop on
> >>> isBlocked() to check whether the next record is available. If the next
> >>> record reaches after 10 min, it is a tight loop for 10 min. You have
> >>> probably noticed that in this case, even isBlocked() returns a future,
> >> that
> >>> future() will not be completed if Flink does not call some method from
> >> the
> >>> reader, because the reader has no internal thread to complete that
> future
> >>> by itself.
> >>>
> >>> Due to the above reasons, a blocking take() API would allow Flink to
> have
> >>> an efficient way to read from a reader. There are many ways to wake up
> >> the
> >>> blocking thread when checkpointing is needed depending on the
> >>> implementation. But I think the poll()/take() API would also work in
> that
> >>> case.
> >>>
> >>> Thanks,
> >>>
> >>> Jiangjie (Becket) Qin
> >>>
> >>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <piotr@data-artisans.com
> >
> >>> wrote:
> >>>
> >>>> Hi,
> >>>>
> >>>> a)
> >>>>
> >>>>> BTW, regarding the isBlock() method, I have a few more questions. 21,
> >> Is
> >>>> a method isReady() with boolean as a return value
> >>>>> equivalent? Personally I found it is a little bit confusing in what
> is
> >>>> supposed to be returned when the future is completed. 22. if
> >>>>> the implementation of isBlocked() is optional, how do the callers
> know
> >>>> whether the method is properly implemented or not?
> >>>>> Does not implemented mean it always return a completed future?
> >>>>
> >>>> `CompletableFuture<?> isBlocked()` is more or less an equivalent to
> >>>> `boolean hasNext()` which in case of “false” provides some kind of a
> >>>> listener/callback that notifies about presence of next element. There
> >> are
> >>>> some minor details, like `CompletableFuture<?>` has a minimal two
> state
> >>>> logic:
> >>>>
> >>>> 1. Future is completed - we have more data
> >>>> 2. Future not yet completed - we don’t have data now, but we might/we
> >> will
> >>>> have in the future
> >>>>
> >>>> While `boolean hasNext()` and `notify()` callback are a bit more
> >>>> complicated/dispersed and can lead/encourage `notify()` spam.
> >>>>
> >>>> b)
> >>>>
> >>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> >> `getNext`
> >>>> the `getNext` would need return a
> >>>>> `ElementWithTimestamp` because some sources want to add timestamp to
> >>>> every element. IMO, this is not so memory friendly
> >>>>> so I prefer this design.
> >>>>
> >>>> Guowei I don’t quite understand this. Could you elaborate why having a
> >>>> separate `advance()` help?
> >>>>
> >>>> c)
> >>>>
> >>>> Regarding advance/poll/take. What’s the value of having two separate
> >>>> methods: poll and take? Which one of them should be called and which
> >>>> implemented? What’s the benefit of having those methods compared to
> >> having
> >>>> a one single method `getNextElement()` (or `pollElement() or whatever
> we
> >>>> name it) with following contract:
> >>>>
> >>>> CompletableFuture<?> isBlocked();
> >>>>
> >>>> /**
> >>>> Return next element - will be called only if `isBlocked()` is
> completed.
> >>>> Try to implement it in non blocking fashion, but if that’s impossible
> or
> >>>> you just don’t need the effort, you can block in this method.
> >>>> */
> >>>> T getNextElement();
> >>>>
> >>>> I mean, if the connector is implemented non-blockingly, Flink should
> use
> >>>> it that way. If it’s not, then `poll()` will `throw new
> >>>> NotImplementedException()`. Implementing both of them and providing
> >> both of
> >>>> them to Flink wouldn’t make a sense, thus why not merge them into a
> >> single
> >>>> method call that should preferably (but not necessarily need to) be
> >>>> non-blocking? It’s not like we are implementing general purpose
> `Queue`,
> >>>> which users might want to call either of `poll` or `take`. We would
> >> always
> >>>> prefer to call `poll`, but if it’s blocking, then still we have no
> >> choice,
> >>>> but to call it and block on it.
> >>>>
> >>>> d)
> >>>>
> >>>>> 1. I agree with Piotr and Becket that the non-blocking source is very
> >>>>> important. But in addition to `Future/poll`, there may be another way
> >> to
> >>>>> achieve this. I think it may be not very memory friendly if every
> >> advance
> >>>>> call return a Future.
> >>>>
> >>>> I didn’t want to mention this, to not clog my initial proposal, but
> >> there
> >>>> is a simple solution for the problem:
> >>>>
> >>>> public interface SplitReader {
> >>>>
> >>>>   (…)
> >>>>
> >>>>   CompletableFuture<?> NOT_BLOCKED =
> >>>> CompletableFuture.completedFuture(null);
> >>>>
> >>>>   /**
> >>>>    * Returns a future that will be completed when the page source
> >> becomes
> >>>>    * unblocked.  If the page source is not blocked, this method should
> >>>> return
> >>>>    * {@code NOT_BLOCKED}.
> >>>>    */
> >>>>   default CompletableFuture<?> isBlocked()
> >>>>   {
> >>>>       return NOT_BLOCKED;
> >>>>   }
> >>>>
> >>>> If we are blocked and we are waiting for the IO, then creating a new
> >>>> Future is non-issue. Under full throttle/throughput and not blocked
> >> sources
> >>>> returning a static `NOT_BLOCKED` constant  should also solve the
> >> problem.
> >>>>
> >>>> One more remark, non-blocking sources might be a necessity in a single
> >>>> threaded model without a checkpointing lock. (Currently when sources
> are
> >>>> blocked, they can release checkpointing lock and re-acquire it again
> >>>> later). Non-blocking `poll`/`getNext()` would allow for checkpoints to
> >>>> happen when source is idling. In that case either `notify()` or my
> >> proposed
> >>>> `isBlocked()` would allow to avoid busy-looping.
> >>>>
> >>>> Piotrek
> >>>>
> >>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
> >>>>>
> >>>>> Hi Thomas,
> >>>>>
> >>>>> The iterator-like API was also the first thing that came to me. But
> it
> >>>>> seems a little confusing that hasNext() does not mean "the stream has
> >> not
> >>>>> ended", but means "the next record is ready", which is repurposing
> the
> >>>> well
> >>>>> known meaning of hasNext(). If we follow the hasNext()/next()
> pattern,
> >> an
> >>>>> additional isNextReady() method to indicate whether the next record
> is
> >>>>> ready seems more intuitive to me.
> >>>>>
> >>>>> Similarly, in poll()/take() pattern, another method of isDone() is
> >> needed
> >>>>> to indicate whether the stream has ended or not.
> >>>>>
> >>>>> Compared with hasNext()/next()/isNextReady() pattern,
> >>>>> isDone()/poll()/take() seems more flexible for the reader
> >> implementation.
> >>>>> When I am implementing a reader, I could have a couple of choices:
> >>>>>
> >>>>> - A thread-less reader that does not have any internal thread.
> >>>>> - When poll() is called, the same calling thread will perform a bunch
> >>>> of
> >>>>>    IO asynchronously.
> >>>>>    - When take() is called, the same calling thread will perform a
> >>>> bunch
> >>>>>    of IO and wait until the record is ready.
> >>>>> - A reader with internal threads performing network IO and put
> records
> >>>>> into a buffer.
> >>>>>    - When poll() is called, the calling thread simply reads from the
> >>>>>    buffer and return empty result immediately if there is no record.
> >>>>>    - When take() is called, the calling thread reads from the buffer
> >>>> and
> >>>>>    block waiting if the buffer is empty.
> >>>>>
> >>>>> On the other hand, with the hasNext()/next()/isNextReady() API, it is
> >>>> less
> >>>>> intuitive for the reader developers to write the thread-less pattern.
> >>>>> Although technically speaking one can still do the asynchronous IO to
> >>>>> prepare the record in isNextReady(). But it is inexplicit and seems
> >>>>> somewhat hacky.
> >>>>>
> >>>>> Thanks,
> >>>>>
> >>>>> Jiangjie (Becket) Qin
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
> >>>>>
> >>>>>> Couple more points regarding discovery:
> >>>>>>
> >>>>>> The proposal mentions that discovery could be outside the execution
> >>>> graph.
> >>>>>> Today, discovered partitions/shards are checkpointed. I believe that
> >>>> will
> >>>>>> also need to be the case in the future, even when discovery and
> >> reading
> >>>> are
> >>>>>> split between different tasks.
> >>>>>>
> >>>>>> For cases such as resharding of a Kinesis stream, the relationship
> >>>> between
> >>>>>> splits needs to be considered. Splits cannot be randomly distributed
> >>>> over
> >>>>>> readers in certain situations. An example was mentioned here:
> >>>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >>>>>>
> >>>>>> Thomas
> >>>>>>
> >>>>>>
> >>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
> >>>>>>
> >>>>>>> Thanks for getting the ball rolling on this!
> >>>>>>>
> >>>>>>> Can the number of splits decrease? Yes, splits can be closed and go
> >>>> away.
> >>>>>>> An example would be a shard merge in Kinesis (2 existing shards
> will
> >> be
> >>>>>>> closed and replaced with a new shard).
> >>>>>>>
> >>>>>>> Regarding advance/poll/take: IMO the least restrictive approach
> would
> >>>> be
> >>>>>>> the thread-less IO model (pull based, non-blocking, caller
> retrieves
> >>>> new
> >>>>>>> records when available). The current Kinesis API requires the use
> of
> >>>>>>> threads. But that can be internal to the split reader and does not
> >> need
> >>>>>> to
> >>>>>>> be a source API concern. In fact, that's what we are working on
> right
> >>>> now
> >>>>>>> as improvement to the existing consumer: Each shard consumer thread
> >>>> will
> >>>>>>> push to a queue, the consumer main thread will poll the queue(s).
> It
> >> is
> >>>>>>> essentially a mapping from threaded IO to non-blocking.
> >>>>>>>
> >>>>>>> The proposed SplitReader interface would fit the thread-less IO
> >> model.
> >>>>>>> Similar to an iterator, we find out if there is a new element
> >> (hasNext)
> >>>>>> and
> >>>>>>> if so, move to it (next()). Separate calls deliver the meta
> >> information
> >>>>>>> (timestamp, watermark). Perhaps advance call could offer a timeout
> >>>>>> option,
> >>>>>>> so that the caller does not end up in a busy wait. On the other
> >> hand, a
> >>>>>>> caller processing multiple splits may want to cycle through fast,
> to
> >>>>>>> process elements of other splits as soon as they become available.
> >> The
> >>>>>> nice
> >>>>>>> thing is that this "split merge" logic can now live in Flink and be
> >>>>>>> optimized and shared between different sources.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>> Thomas
> >>>>>>>
> >>>>>>>
> >>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
> >> wrote:
> >>>>>>>
> >>>>>>>> Hi,
> >>>>>>>> Thanks Aljoscha for this FLIP.
> >>>>>>>>
> >>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
> >> very
> >>>>>>>> important. But in addition to `Future/poll`, there may be another
> >> way
> >>>> to
> >>>>>>>> achieve this. I think it may be not very memory friendly if every
> >>>>>> advance
> >>>>>>>> call return a Future.
> >>>>>>>>
> >>>>>>>> public interface Listener {
> >>>>>>>>   public void notify();
> >>>>>>>> }
> >>>>>>>>
> >>>>>>>> public interface SplitReader() {
> >>>>>>>>   /**
> >>>>>>>>    * When there is no element temporarily, this will return false.
> >>>>>>>>    * When elements is available again splitReader can call
> >>>>>>>> listener.notify()
> >>>>>>>>    * In addition the frame would check `advance` periodically .
> >>>>>>>>    * Of course advance can always return true and ignore the
> >>>> listener
> >>>>>>>> argument for simplicity.
> >>>>>>>>    */
> >>>>>>>>   public boolean advance(Listener listener);
> >>>>>>>> }
> >>>>>>>>
> >>>>>>>> 2.  The FLIP tells us very clearly that how to create all Splits
> and
> >>>> how
> >>>>>>>> to create a SplitReader from a Split. But there is no strategy for
> >> the
> >>>>>> user
> >>>>>>>> to choose how to assign the splits to the tasks. I think we could
> >> add
> >>>> a
> >>>>>>>> Enum to let user to choose.
> >>>>>>>> /**
> >>>>>>>> public Enum SplitsAssignmentPolicy {
> >>>>>>>>  Location,
> >>>>>>>>  Workload,
> >>>>>>>>  Random,
> >>>>>>>>  Average
> >>>>>>>> }
> >>>>>>>> */
> >>>>>>>>
> >>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> >>>> `getNext`
> >>>>>>>> the `getNext` would need return a `ElementWithTimestamp` because
> >> some
> >>>>>>>> sources want to add timestamp to every element. IMO, this is not
> so
> >>>>>> memory
> >>>>>>>> friendly so I prefer this design.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Thanks
> >>>>>>>>
> >>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
> >>>>>>>>
> >>>>>>>>> Hi,
> >>>>>>>>>
> >>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
> >> other
> >>>>>>>>> possible improvements. I have one proposal. Instead of having a
> >>>> method:
> >>>>>>>>>
> >>>>>>>>> boolean advance() throws IOException;
> >>>>>>>>>
> >>>>>>>>> I would replace it with
> >>>>>>>>>
> >>>>>>>>> /*
> >>>>>>>>> * Return a future, which when completed means that source has
> more
> >>>>>> data
> >>>>>>>>> and getNext() will not block.
> >>>>>>>>> * If you wish to use benefits of non blocking connectors, please
> >>>>>>>>> implement this method appropriately.
> >>>>>>>>> */
> >>>>>>>>> default CompletableFuture<?> isBlocked() {
> >>>>>>>>>      return CompletableFuture.completedFuture(null);
> >>>>>>>>> }
> >>>>>>>>>
> >>>>>>>>> And rename `getCurrent()` to `getNext()`.
> >>>>>>>>>
> >>>>>>>>> Couple of arguments:
> >>>>>>>>> 1. I don’t understand the division of work between `advance()`
> and
> >>>>>>>>> `getCurrent()`. What should be done in which, especially for
> >>>> connectors
> >>>>>>>>> that handle records in batches (like Kafka) and when should you
> >> call
> >>>>>>>>> `advance` and when `getCurrent()`.
> >>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us
> in
> >>>> the
> >>>>>>>>> future to have asynchronous/non blocking connectors and more
> >>>>>> efficiently
> >>>>>>>>> handle large number of blocked threads, without busy waiting.
> While
> >>>> at
> >>>>>> the
> >>>>>>>>> same time it doesn’t add much complexity, since naive connector
> >>>>>>>>> implementations can be always blocking.
> >>>>>>>>> 3. This also would allow us to use a fixed size thread pool of
> task
> >>>>>>>>> executors, instead of one thread per task.
> >>>>>>>>>
> >>>>>>>>> Piotrek
> >>>>>>>>>
> >>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <aljoscha@apache.org
> >
> >>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi All,
> >>>>>>>>>>
> >>>>>>>>>> In order to finally get the ball rolling on the new source
> >> interface
> >>>>>>>>> that we have discussed for so long I finally created a FLIP:
> >>>>>>>>>
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>>>>>
> >>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
> >>>> about
> >>>>>>>>> adding per-partition watermark support to the Kinesis source and
> >>>>>> because
> >>>>>>>>> this would enable generic implementation of event-time alignment
> >> for
> >>>>>> all
> >>>>>>>>> sources. Maybe we need another FLIP for the event-time alignment
> >>>> part,
> >>>>>>>>> especially the part about information sharing between operations
> >> (I'm
> >>>>>> not
> >>>>>>>>> calling it state sharing because state has a special meaning in
> >>>> Flink).
> >>>>>>>>>>
> >>>>>>>>>> Please discuss away!
> >>>>>>>>>>
> >>>>>>>>>> Aljoscha
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>
> >>>>
> >>>>
> >>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@data-artisans.com>.
Hi

Good point with select/epoll, however I do not see how they couldn’t be with Flink if we would like single task in Flink to be single-threaded (and I believe we should pursue this goal). If your connector blocks on `select`, then it can not process/handle control messages from Flink, like checkpoints, releasing resources and potentially output flushes. This would require tight integration between connector and Flink’s main event loop/selects/etc.

Looking at it from other perspective. Let’s assume that we have a connector implemented on top of `select`/`epoll`. In order to integrate it with Flink’s checkpointing/flushes/resource releasing it will have to be executed in separate thread one way or another. At least if our API will enforce/encourage non blocking implementations with some kind of notifications (`isBlocked()` or `notify()` callback), some connectors might skip one layer of wapping threads.

Btw, I don’t know if you understand my point. Having only `poll()` and `take()` is not enough for single threaded task. If our source interface doesn’t provide `notify()` callback nor `CompletableFuture<?> isBlocked(),`, there is no way to implement single threaded task that both reads the data from the source connector and can also react to system events. Ok, non blocking `poll()` would allow that, but with busy looping.

Piotrek

> On 8 Nov 2018, at 06:56, Becket Qin <be...@gmail.com> wrote:
> 
> Hi Piotrek,
> 
>> But I don’t see a reason why we should expose both blocking `take()` and
> non-blocking `poll()` methods to the Flink engine. Someone (Flink engine or
> connector) would have to do the same busy
>> looping anyway and I think it would be better to have a simpler connector
> API (that would solve our problems) and force connectors to comply one way
> or another.
> 
> If we let the block happen inside the connector, the blocking does not have
> to be a busy loop. For example, to do the block waiting efficiently, the
> connector can use java NIO selector().select which relies on OS syscall
> like epoll[1] instead of busy looping. But if Flink engine blocks outside
> the connector, it pretty much has to do the busy loop. So if there is only
> one API to get the element, a blocking getNextElement() makes more sense.
> In any case, we should avoid ambiguity. It has to be crystal clear about
> whether a method is expected to be blocking or non-blocking. Otherwise it
> would be very difficult for Flink engine to do the right thing with the
> connectors. At the first glance at getCurrent(), the expected behavior is
> not quite clear.
> 
> That said, I do agree that functionality wise, poll() and take() kind of
> overlap. But they are actually not quite different from
> isBlocked()/getNextElement(). Compared with isBlocked(), the only
> difference is that poll() also returns the next record if it is available.
> But I agree that the isBlocked() + getNextElement() is more flexible as
> users can just check the record availability, but not fetch the next
> element.
> 
>> In case of thread-less readers with only non-blocking `queue.poll()` (is
> that really a thing? I can not think about a real implementation that
> enforces such constraints)
> Right, it is pretty much a syntax sugar to allow user combine the
> check-and-take into one method. It could be achieved with isBlocked() +
> getNextElement().
> 
> [1] http://man7.org/linux/man-pages/man7/epoll.7.html
> 
> Thanks,
> 
> Jiangjie (Becket) Qin
> 
> On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <pi...@data-artisans.com>
> wrote:
> 
>> Hi Becket,
>> 
>> With my proposal, both of your examples would have to be solved by the
>> connector and solution to both problems would be the same:
>> 
>> Pretend that connector is never blocked (`isBlocked() { return
>> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking fashion (or
>> semi blocking with return of control from time to time to allow for
>> checkpointing, network flushing and other resource management things to
>> happen in the same main thread). In other words, exactly how you would
>> implement `take()` method or how the same source connector would be
>> implemented NOW with current source interface. The difference with current
>> interface would be only that main loop would be outside of the connector,
>> and instead of periodically releasing checkpointing lock, periodically
>> `return null;` or `return Optional.empty();` from `getNextElement()`.
>> 
>> In case of thread-less readers with only non-blocking `queue.poll()` (is
>> that really a thing? I can not think about a real implementation that
>> enforces such constraints), we could provide a wrapper that hides the busy
>> looping. The same applies how to solve forever blocking readers - we could
>> provider another wrapper running the connector in separate thread.
>> 
>> But I don’t see a reason why we should expose both blocking `take()` and
>> non-blocking `poll()` methods to the Flink engine. Someone (Flink engine or
>> connector) would have to do the same busy looping anyway and I think it
>> would be better to have a simpler connector API (that would solve our
>> problems) and force connectors to comply one way or another.
>> 
>> Piotrek
>> 
>>> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
>>> 
>>> Hi Piotr,
>>> 
>>> I might have misunderstood you proposal. But let me try to explain my
>>> concern. I am thinking about the following case:
>>> 1. a reader has the following two interfaces,
>>>   boolean isBlocked()
>>>   T getNextElement()
>>> 2. the implementation of getNextElement() is non-blocking.
>>> 3. The reader is thread-less, i.e. it does not have any internal thread.
>>> For example, it might just delegate the getNextElement() to a
>> queue.poll(),
>>> and isBlocked() is just queue.isEmpty().
>>> 
>>> How can Flink efficiently implement a blocking reading behavior with this
>>> reader? Either a tight loop or a backoff interval is needed. Neither of
>>> them is ideal.
>>> 
>>> Now let's say in the reader mentioned above implements a blocking
>>> getNextElement() method. Because there is no internal thread in the
>> reader,
>>> after isBlocked() returns false. Flink will still have to loop on
>>> isBlocked() to check whether the next record is available. If the next
>>> record reaches after 10 min, it is a tight loop for 10 min. You have
>>> probably noticed that in this case, even isBlocked() returns a future,
>> that
>>> future() will not be completed if Flink does not call some method from
>> the
>>> reader, because the reader has no internal thread to complete that future
>>> by itself.
>>> 
>>> Due to the above reasons, a blocking take() API would allow Flink to have
>>> an efficient way to read from a reader. There are many ways to wake up
>> the
>>> blocking thread when checkpointing is needed depending on the
>>> implementation. But I think the poll()/take() API would also work in that
>>> case.
>>> 
>>> Thanks,
>>> 
>>> Jiangjie (Becket) Qin
>>> 
>>> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <pi...@data-artisans.com>
>>> wrote:
>>> 
>>>> Hi,
>>>> 
>>>> a)
>>>> 
>>>>> BTW, regarding the isBlock() method, I have a few more questions. 21,
>> Is
>>>> a method isReady() with boolean as a return value
>>>>> equivalent? Personally I found it is a little bit confusing in what is
>>>> supposed to be returned when the future is completed. 22. if
>>>>> the implementation of isBlocked() is optional, how do the callers know
>>>> whether the method is properly implemented or not?
>>>>> Does not implemented mean it always return a completed future?
>>>> 
>>>> `CompletableFuture<?> isBlocked()` is more or less an equivalent to
>>>> `boolean hasNext()` which in case of “false” provides some kind of a
>>>> listener/callback that notifies about presence of next element. There
>> are
>>>> some minor details, like `CompletableFuture<?>` has a minimal two state
>>>> logic:
>>>> 
>>>> 1. Future is completed - we have more data
>>>> 2. Future not yet completed - we don’t have data now, but we might/we
>> will
>>>> have in the future
>>>> 
>>>> While `boolean hasNext()` and `notify()` callback are a bit more
>>>> complicated/dispersed and can lead/encourage `notify()` spam.
>>>> 
>>>> b)
>>>> 
>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>> `getNext`
>>>> the `getNext` would need return a
>>>>> `ElementWithTimestamp` because some sources want to add timestamp to
>>>> every element. IMO, this is not so memory friendly
>>>>> so I prefer this design.
>>>> 
>>>> Guowei I don’t quite understand this. Could you elaborate why having a
>>>> separate `advance()` help?
>>>> 
>>>> c)
>>>> 
>>>> Regarding advance/poll/take. What’s the value of having two separate
>>>> methods: poll and take? Which one of them should be called and which
>>>> implemented? What’s the benefit of having those methods compared to
>> having
>>>> a one single method `getNextElement()` (or `pollElement() or whatever we
>>>> name it) with following contract:
>>>> 
>>>> CompletableFuture<?> isBlocked();
>>>> 
>>>> /**
>>>> Return next element - will be called only if `isBlocked()` is completed.
>>>> Try to implement it in non blocking fashion, but if that’s impossible or
>>>> you just don’t need the effort, you can block in this method.
>>>> */
>>>> T getNextElement();
>>>> 
>>>> I mean, if the connector is implemented non-blockingly, Flink should use
>>>> it that way. If it’s not, then `poll()` will `throw new
>>>> NotImplementedException()`. Implementing both of them and providing
>> both of
>>>> them to Flink wouldn’t make a sense, thus why not merge them into a
>> single
>>>> method call that should preferably (but not necessarily need to) be
>>>> non-blocking? It’s not like we are implementing general purpose `Queue`,
>>>> which users might want to call either of `poll` or `take`. We would
>> always
>>>> prefer to call `poll`, but if it’s blocking, then still we have no
>> choice,
>>>> but to call it and block on it.
>>>> 
>>>> d)
>>>> 
>>>>> 1. I agree with Piotr and Becket that the non-blocking source is very
>>>>> important. But in addition to `Future/poll`, there may be another way
>> to
>>>>> achieve this. I think it may be not very memory friendly if every
>> advance
>>>>> call return a Future.
>>>> 
>>>> I didn’t want to mention this, to not clog my initial proposal, but
>> there
>>>> is a simple solution for the problem:
>>>> 
>>>> public interface SplitReader {
>>>> 
>>>>   (…)
>>>> 
>>>>   CompletableFuture<?> NOT_BLOCKED =
>>>> CompletableFuture.completedFuture(null);
>>>> 
>>>>   /**
>>>>    * Returns a future that will be completed when the page source
>> becomes
>>>>    * unblocked.  If the page source is not blocked, this method should
>>>> return
>>>>    * {@code NOT_BLOCKED}.
>>>>    */
>>>>   default CompletableFuture<?> isBlocked()
>>>>   {
>>>>       return NOT_BLOCKED;
>>>>   }
>>>> 
>>>> If we are blocked and we are waiting for the IO, then creating a new
>>>> Future is non-issue. Under full throttle/throughput and not blocked
>> sources
>>>> returning a static `NOT_BLOCKED` constant  should also solve the
>> problem.
>>>> 
>>>> One more remark, non-blocking sources might be a necessity in a single
>>>> threaded model without a checkpointing lock. (Currently when sources are
>>>> blocked, they can release checkpointing lock and re-acquire it again
>>>> later). Non-blocking `poll`/`getNext()` would allow for checkpoints to
>>>> happen when source is idling. In that case either `notify()` or my
>> proposed
>>>> `isBlocked()` would allow to avoid busy-looping.
>>>> 
>>>> Piotrek
>>>> 
>>>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
>>>>> 
>>>>> Hi Thomas,
>>>>> 
>>>>> The iterator-like API was also the first thing that came to me. But it
>>>>> seems a little confusing that hasNext() does not mean "the stream has
>> not
>>>>> ended", but means "the next record is ready", which is repurposing the
>>>> well
>>>>> known meaning of hasNext(). If we follow the hasNext()/next() pattern,
>> an
>>>>> additional isNextReady() method to indicate whether the next record is
>>>>> ready seems more intuitive to me.
>>>>> 
>>>>> Similarly, in poll()/take() pattern, another method of isDone() is
>> needed
>>>>> to indicate whether the stream has ended or not.
>>>>> 
>>>>> Compared with hasNext()/next()/isNextReady() pattern,
>>>>> isDone()/poll()/take() seems more flexible for the reader
>> implementation.
>>>>> When I am implementing a reader, I could have a couple of choices:
>>>>> 
>>>>> - A thread-less reader that does not have any internal thread.
>>>>> - When poll() is called, the same calling thread will perform a bunch
>>>> of
>>>>>    IO asynchronously.
>>>>>    - When take() is called, the same calling thread will perform a
>>>> bunch
>>>>>    of IO and wait until the record is ready.
>>>>> - A reader with internal threads performing network IO and put records
>>>>> into a buffer.
>>>>>    - When poll() is called, the calling thread simply reads from the
>>>>>    buffer and return empty result immediately if there is no record.
>>>>>    - When take() is called, the calling thread reads from the buffer
>>>> and
>>>>>    block waiting if the buffer is empty.
>>>>> 
>>>>> On the other hand, with the hasNext()/next()/isNextReady() API, it is
>>>> less
>>>>> intuitive for the reader developers to write the thread-less pattern.
>>>>> Although technically speaking one can still do the asynchronous IO to
>>>>> prepare the record in isNextReady(). But it is inexplicit and seems
>>>>> somewhat hacky.
>>>>> 
>>>>> Thanks,
>>>>> 
>>>>> Jiangjie (Becket) Qin
>>>>> 
>>>>> 
>>>>> 
>>>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
>>>>> 
>>>>>> Couple more points regarding discovery:
>>>>>> 
>>>>>> The proposal mentions that discovery could be outside the execution
>>>> graph.
>>>>>> Today, discovered partitions/shards are checkpointed. I believe that
>>>> will
>>>>>> also need to be the case in the future, even when discovery and
>> reading
>>>> are
>>>>>> split between different tasks.
>>>>>> 
>>>>>> For cases such as resharding of a Kinesis stream, the relationship
>>>> between
>>>>>> splits needs to be considered. Splits cannot be randomly distributed
>>>> over
>>>>>> readers in certain situations. An example was mentioned here:
>>>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>>>>>> 
>>>>>> Thomas
>>>>>> 
>>>>>> 
>>>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
>>>>>> 
>>>>>>> Thanks for getting the ball rolling on this!
>>>>>>> 
>>>>>>> Can the number of splits decrease? Yes, splits can be closed and go
>>>> away.
>>>>>>> An example would be a shard merge in Kinesis (2 existing shards will
>> be
>>>>>>> closed and replaced with a new shard).
>>>>>>> 
>>>>>>> Regarding advance/poll/take: IMO the least restrictive approach would
>>>> be
>>>>>>> the thread-less IO model (pull based, non-blocking, caller retrieves
>>>> new
>>>>>>> records when available). The current Kinesis API requires the use of
>>>>>>> threads. But that can be internal to the split reader and does not
>> need
>>>>>> to
>>>>>>> be a source API concern. In fact, that's what we are working on right
>>>> now
>>>>>>> as improvement to the existing consumer: Each shard consumer thread
>>>> will
>>>>>>> push to a queue, the consumer main thread will poll the queue(s). It
>> is
>>>>>>> essentially a mapping from threaded IO to non-blocking.
>>>>>>> 
>>>>>>> The proposed SplitReader interface would fit the thread-less IO
>> model.
>>>>>>> Similar to an iterator, we find out if there is a new element
>> (hasNext)
>>>>>> and
>>>>>>> if so, move to it (next()). Separate calls deliver the meta
>> information
>>>>>>> (timestamp, watermark). Perhaps advance call could offer a timeout
>>>>>> option,
>>>>>>> so that the caller does not end up in a busy wait. On the other
>> hand, a
>>>>>>> caller processing multiple splits may want to cycle through fast, to
>>>>>>> process elements of other splits as soon as they become available.
>> The
>>>>>> nice
>>>>>>> thing is that this "split merge" logic can now live in Flink and be
>>>>>>> optimized and shared between different sources.
>>>>>>> 
>>>>>>> Thanks,
>>>>>>> Thomas
>>>>>>> 
>>>>>>> 
>>>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
>> wrote:
>>>>>>> 
>>>>>>>> Hi,
>>>>>>>> Thanks Aljoscha for this FLIP.
>>>>>>>> 
>>>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
>> very
>>>>>>>> important. But in addition to `Future/poll`, there may be another
>> way
>>>> to
>>>>>>>> achieve this. I think it may be not very memory friendly if every
>>>>>> advance
>>>>>>>> call return a Future.
>>>>>>>> 
>>>>>>>> public interface Listener {
>>>>>>>>   public void notify();
>>>>>>>> }
>>>>>>>> 
>>>>>>>> public interface SplitReader() {
>>>>>>>>   /**
>>>>>>>>    * When there is no element temporarily, this will return false.
>>>>>>>>    * When elements is available again splitReader can call
>>>>>>>> listener.notify()
>>>>>>>>    * In addition the frame would check `advance` periodically .
>>>>>>>>    * Of course advance can always return true and ignore the
>>>> listener
>>>>>>>> argument for simplicity.
>>>>>>>>    */
>>>>>>>>   public boolean advance(Listener listener);
>>>>>>>> }
>>>>>>>> 
>>>>>>>> 2.  The FLIP tells us very clearly that how to create all Splits and
>>>> how
>>>>>>>> to create a SplitReader from a Split. But there is no strategy for
>> the
>>>>>> user
>>>>>>>> to choose how to assign the splits to the tasks. I think we could
>> add
>>>> a
>>>>>>>> Enum to let user to choose.
>>>>>>>> /**
>>>>>>>> public Enum SplitsAssignmentPolicy {
>>>>>>>>  Location,
>>>>>>>>  Workload,
>>>>>>>>  Random,
>>>>>>>>  Average
>>>>>>>> }
>>>>>>>> */
>>>>>>>> 
>>>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>>>> `getNext`
>>>>>>>> the `getNext` would need return a `ElementWithTimestamp` because
>> some
>>>>>>>> sources want to add timestamp to every element. IMO, this is not so
>>>>>> memory
>>>>>>>> friendly so I prefer this design.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Thanks
>>>>>>>> 
>>>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
>>>>>>>> 
>>>>>>>>> Hi,
>>>>>>>>> 
>>>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
>> other
>>>>>>>>> possible improvements. I have one proposal. Instead of having a
>>>> method:
>>>>>>>>> 
>>>>>>>>> boolean advance() throws IOException;
>>>>>>>>> 
>>>>>>>>> I would replace it with
>>>>>>>>> 
>>>>>>>>> /*
>>>>>>>>> * Return a future, which when completed means that source has more
>>>>>> data
>>>>>>>>> and getNext() will not block.
>>>>>>>>> * If you wish to use benefits of non blocking connectors, please
>>>>>>>>> implement this method appropriately.
>>>>>>>>> */
>>>>>>>>> default CompletableFuture<?> isBlocked() {
>>>>>>>>>      return CompletableFuture.completedFuture(null);
>>>>>>>>> }
>>>>>>>>> 
>>>>>>>>> And rename `getCurrent()` to `getNext()`.
>>>>>>>>> 
>>>>>>>>> Couple of arguments:
>>>>>>>>> 1. I don’t understand the division of work between `advance()` and
>>>>>>>>> `getCurrent()`. What should be done in which, especially for
>>>> connectors
>>>>>>>>> that handle records in batches (like Kafka) and when should you
>> call
>>>>>>>>> `advance` and when `getCurrent()`.
>>>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in
>>>> the
>>>>>>>>> future to have asynchronous/non blocking connectors and more
>>>>>> efficiently
>>>>>>>>> handle large number of blocked threads, without busy waiting. While
>>>> at
>>>>>> the
>>>>>>>>> same time it doesn’t add much complexity, since naive connector
>>>>>>>>> implementations can be always blocking.
>>>>>>>>> 3. This also would allow us to use a fixed size thread pool of task
>>>>>>>>> executors, instead of one thread per task.
>>>>>>>>> 
>>>>>>>>> Piotrek
>>>>>>>>> 
>>>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
>>>>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>> Hi All,
>>>>>>>>>> 
>>>>>>>>>> In order to finally get the ball rolling on the new source
>> interface
>>>>>>>>> that we have discussed for so long I finally created a FLIP:
>>>>>>>>> 
>>>>>> 
>>>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>>>>> 
>>>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
>>>> about
>>>>>>>>> adding per-partition watermark support to the Kinesis source and
>>>>>> because
>>>>>>>>> this would enable generic implementation of event-time alignment
>> for
>>>>>> all
>>>>>>>>> sources. Maybe we need another FLIP for the event-time alignment
>>>> part,
>>>>>>>>> especially the part about information sharing between operations
>> (I'm
>>>>>> not
>>>>>>>>> calling it state sharing because state has a special meaning in
>>>> Flink).
>>>>>>>>>> 
>>>>>>>>>> Please discuss away!
>>>>>>>>>> 
>>>>>>>>>> Aljoscha
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>> 
>>>> 
>>>> 
>> 
>> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Piotrek,

> But I don’t see a reason why we should expose both blocking `take()` and
non-blocking `poll()` methods to the Flink engine. Someone (Flink engine or
connector) would have to do the same busy
> looping anyway and I think it would be better to have a simpler connector
API (that would solve our problems) and force connectors to comply one way
or another.

If we let the block happen inside the connector, the blocking does not have
to be a busy loop. For example, to do the block waiting efficiently, the
connector can use java NIO selector().select which relies on OS syscall
like epoll[1] instead of busy looping. But if Flink engine blocks outside
the connector, it pretty much has to do the busy loop. So if there is only
one API to get the element, a blocking getNextElement() makes more sense.
In any case, we should avoid ambiguity. It has to be crystal clear about
whether a method is expected to be blocking or non-blocking. Otherwise it
would be very difficult for Flink engine to do the right thing with the
connectors. At the first glance at getCurrent(), the expected behavior is
not quite clear.

That said, I do agree that functionality wise, poll() and take() kind of
overlap. But they are actually not quite different from
isBlocked()/getNextElement(). Compared with isBlocked(), the only
difference is that poll() also returns the next record if it is available.
But I agree that the isBlocked() + getNextElement() is more flexible as
users can just check the record availability, but not fetch the next
element.

> In case of thread-less readers with only non-blocking `queue.poll()` (is
that really a thing? I can not think about a real implementation that
enforces such constraints)
Right, it is pretty much a syntax sugar to allow user combine the
check-and-take into one method. It could be achieved with isBlocked() +
getNextElement().

[1] http://man7.org/linux/man-pages/man7/epoll.7.html

Thanks,

Jiangjie (Becket) Qin

On Wed, Nov 7, 2018 at 11:58 PM Piotr Nowojski <pi...@data-artisans.com>
wrote:

> Hi Becket,
>
> With my proposal, both of your examples would have to be solved by the
> connector and solution to both problems would be the same:
>
> Pretend that connector is never blocked (`isBlocked() { return
> NOT_BLOCKED; }`) and implement `getNextElement()` in blocking fashion (or
> semi blocking with return of control from time to time to allow for
> checkpointing, network flushing and other resource management things to
> happen in the same main thread). In other words, exactly how you would
> implement `take()` method or how the same source connector would be
> implemented NOW with current source interface. The difference with current
> interface would be only that main loop would be outside of the connector,
> and instead of periodically releasing checkpointing lock, periodically
> `return null;` or `return Optional.empty();` from `getNextElement()`.
>
> In case of thread-less readers with only non-blocking `queue.poll()` (is
> that really a thing? I can not think about a real implementation that
> enforces such constraints), we could provide a wrapper that hides the busy
> looping. The same applies how to solve forever blocking readers - we could
> provider another wrapper running the connector in separate thread.
>
> But I don’t see a reason why we should expose both blocking `take()` and
> non-blocking `poll()` methods to the Flink engine. Someone (Flink engine or
> connector) would have to do the same busy looping anyway and I think it
> would be better to have a simpler connector API (that would solve our
> problems) and force connectors to comply one way or another.
>
> Piotrek
>
> > On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
> >
> > Hi Piotr,
> >
> > I might have misunderstood you proposal. But let me try to explain my
> > concern. I am thinking about the following case:
> > 1. a reader has the following two interfaces,
> >    boolean isBlocked()
> >    T getNextElement()
> > 2. the implementation of getNextElement() is non-blocking.
> > 3. The reader is thread-less, i.e. it does not have any internal thread.
> > For example, it might just delegate the getNextElement() to a
> queue.poll(),
> > and isBlocked() is just queue.isEmpty().
> >
> > How can Flink efficiently implement a blocking reading behavior with this
> > reader? Either a tight loop or a backoff interval is needed. Neither of
> > them is ideal.
> >
> > Now let's say in the reader mentioned above implements a blocking
> > getNextElement() method. Because there is no internal thread in the
> reader,
> > after isBlocked() returns false. Flink will still have to loop on
> > isBlocked() to check whether the next record is available. If the next
> > record reaches after 10 min, it is a tight loop for 10 min. You have
> > probably noticed that in this case, even isBlocked() returns a future,
> that
> > future() will not be completed if Flink does not call some method from
> the
> > reader, because the reader has no internal thread to complete that future
> > by itself.
> >
> > Due to the above reasons, a blocking take() API would allow Flink to have
> > an efficient way to read from a reader. There are many ways to wake up
> the
> > blocking thread when checkpointing is needed depending on the
> > implementation. But I think the poll()/take() API would also work in that
> > case.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <pi...@data-artisans.com>
> > wrote:
> >
> >> Hi,
> >>
> >> a)
> >>
> >>> BTW, regarding the isBlock() method, I have a few more questions. 21,
> Is
> >> a method isReady() with boolean as a return value
> >>> equivalent? Personally I found it is a little bit confusing in what is
> >> supposed to be returned when the future is completed. 22. if
> >>> the implementation of isBlocked() is optional, how do the callers know
> >> whether the method is properly implemented or not?
> >>> Does not implemented mean it always return a completed future?
> >>
> >> `CompletableFuture<?> isBlocked()` is more or less an equivalent to
> >> `boolean hasNext()` which in case of “false” provides some kind of a
> >> listener/callback that notifies about presence of next element. There
> are
> >> some minor details, like `CompletableFuture<?>` has a minimal two state
> >> logic:
> >>
> >> 1. Future is completed - we have more data
> >> 2. Future not yet completed - we don’t have data now, but we might/we
> will
> >> have in the future
> >>
> >> While `boolean hasNext()` and `notify()` callback are a bit more
> >> complicated/dispersed and can lead/encourage `notify()` spam.
> >>
> >> b)
> >>
> >>> 3. If merge the `advance` and `getCurrent`  to one method like
> `getNext`
> >> the `getNext` would need return a
> >>> `ElementWithTimestamp` because some sources want to add timestamp to
> >> every element. IMO, this is not so memory friendly
> >>> so I prefer this design.
> >>
> >> Guowei I don’t quite understand this. Could you elaborate why having a
> >> separate `advance()` help?
> >>
> >> c)
> >>
> >> Regarding advance/poll/take. What’s the value of having two separate
> >> methods: poll and take? Which one of them should be called and which
> >> implemented? What’s the benefit of having those methods compared to
> having
> >> a one single method `getNextElement()` (or `pollElement() or whatever we
> >> name it) with following contract:
> >>
> >> CompletableFuture<?> isBlocked();
> >>
> >> /**
> >> Return next element - will be called only if `isBlocked()` is completed.
> >> Try to implement it in non blocking fashion, but if that’s impossible or
> >> you just don’t need the effort, you can block in this method.
> >> */
> >> T getNextElement();
> >>
> >> I mean, if the connector is implemented non-blockingly, Flink should use
> >> it that way. If it’s not, then `poll()` will `throw new
> >> NotImplementedException()`. Implementing both of them and providing
> both of
> >> them to Flink wouldn’t make a sense, thus why not merge them into a
> single
> >> method call that should preferably (but not necessarily need to) be
> >> non-blocking? It’s not like we are implementing general purpose `Queue`,
> >> which users might want to call either of `poll` or `take`. We would
> always
> >> prefer to call `poll`, but if it’s blocking, then still we have no
> choice,
> >> but to call it and block on it.
> >>
> >> d)
> >>
> >>> 1. I agree with Piotr and Becket that the non-blocking source is very
> >>> important. But in addition to `Future/poll`, there may be another way
> to
> >>> achieve this. I think it may be not very memory friendly if every
> advance
> >>> call return a Future.
> >>
> >> I didn’t want to mention this, to not clog my initial proposal, but
> there
> >> is a simple solution for the problem:
> >>
> >> public interface SplitReader {
> >>
> >>    (…)
> >>
> >>    CompletableFuture<?> NOT_BLOCKED =
> >> CompletableFuture.completedFuture(null);
> >>
> >>    /**
> >>     * Returns a future that will be completed when the page source
> becomes
> >>     * unblocked.  If the page source is not blocked, this method should
> >> return
> >>     * {@code NOT_BLOCKED}.
> >>     */
> >>    default CompletableFuture<?> isBlocked()
> >>    {
> >>        return NOT_BLOCKED;
> >>    }
> >>
> >> If we are blocked and we are waiting for the IO, then creating a new
> >> Future is non-issue. Under full throttle/throughput and not blocked
> sources
> >> returning a static `NOT_BLOCKED` constant  should also solve the
> problem.
> >>
> >> One more remark, non-blocking sources might be a necessity in a single
> >> threaded model without a checkpointing lock. (Currently when sources are
> >> blocked, they can release checkpointing lock and re-acquire it again
> >> later). Non-blocking `poll`/`getNext()` would allow for checkpoints to
> >> happen when source is idling. In that case either `notify()` or my
> proposed
> >> `isBlocked()` would allow to avoid busy-looping.
> >>
> >> Piotrek
> >>
> >>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
> >>>
> >>> Hi Thomas,
> >>>
> >>> The iterator-like API was also the first thing that came to me. But it
> >>> seems a little confusing that hasNext() does not mean "the stream has
> not
> >>> ended", but means "the next record is ready", which is repurposing the
> >> well
> >>> known meaning of hasNext(). If we follow the hasNext()/next() pattern,
> an
> >>> additional isNextReady() method to indicate whether the next record is
> >>> ready seems more intuitive to me.
> >>>
> >>> Similarly, in poll()/take() pattern, another method of isDone() is
> needed
> >>> to indicate whether the stream has ended or not.
> >>>
> >>> Compared with hasNext()/next()/isNextReady() pattern,
> >>> isDone()/poll()/take() seems more flexible for the reader
> implementation.
> >>> When I am implementing a reader, I could have a couple of choices:
> >>>
> >>>  - A thread-less reader that does not have any internal thread.
> >>>  - When poll() is called, the same calling thread will perform a bunch
> >> of
> >>>     IO asynchronously.
> >>>     - When take() is called, the same calling thread will perform a
> >> bunch
> >>>     of IO and wait until the record is ready.
> >>>  - A reader with internal threads performing network IO and put records
> >>>  into a buffer.
> >>>     - When poll() is called, the calling thread simply reads from the
> >>>     buffer and return empty result immediately if there is no record.
> >>>     - When take() is called, the calling thread reads from the buffer
> >> and
> >>>     block waiting if the buffer is empty.
> >>>
> >>> On the other hand, with the hasNext()/next()/isNextReady() API, it is
> >> less
> >>> intuitive for the reader developers to write the thread-less pattern.
> >>> Although technically speaking one can still do the asynchronous IO to
> >>> prepare the record in isNextReady(). But it is inexplicit and seems
> >>> somewhat hacky.
> >>>
> >>> Thanks,
> >>>
> >>> Jiangjie (Becket) Qin
> >>>
> >>>
> >>>
> >>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
> >>>
> >>>> Couple more points regarding discovery:
> >>>>
> >>>> The proposal mentions that discovery could be outside the execution
> >> graph.
> >>>> Today, discovered partitions/shards are checkpointed. I believe that
> >> will
> >>>> also need to be the case in the future, even when discovery and
> reading
> >> are
> >>>> split between different tasks.
> >>>>
> >>>> For cases such as resharding of a Kinesis stream, the relationship
> >> between
> >>>> splits needs to be considered. Splits cannot be randomly distributed
> >> over
> >>>> readers in certain situations. An example was mentioned here:
> >>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >>>>
> >>>> Thomas
> >>>>
> >>>>
> >>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
> >>>>
> >>>>> Thanks for getting the ball rolling on this!
> >>>>>
> >>>>> Can the number of splits decrease? Yes, splits can be closed and go
> >> away.
> >>>>> An example would be a shard merge in Kinesis (2 existing shards will
> be
> >>>>> closed and replaced with a new shard).
> >>>>>
> >>>>> Regarding advance/poll/take: IMO the least restrictive approach would
> >> be
> >>>>> the thread-less IO model (pull based, non-blocking, caller retrieves
> >> new
> >>>>> records when available). The current Kinesis API requires the use of
> >>>>> threads. But that can be internal to the split reader and does not
> need
> >>>> to
> >>>>> be a source API concern. In fact, that's what we are working on right
> >> now
> >>>>> as improvement to the existing consumer: Each shard consumer thread
> >> will
> >>>>> push to a queue, the consumer main thread will poll the queue(s). It
> is
> >>>>> essentially a mapping from threaded IO to non-blocking.
> >>>>>
> >>>>> The proposed SplitReader interface would fit the thread-less IO
> model.
> >>>>> Similar to an iterator, we find out if there is a new element
> (hasNext)
> >>>> and
> >>>>> if so, move to it (next()). Separate calls deliver the meta
> information
> >>>>> (timestamp, watermark). Perhaps advance call could offer a timeout
> >>>> option,
> >>>>> so that the caller does not end up in a busy wait. On the other
> hand, a
> >>>>> caller processing multiple splits may want to cycle through fast, to
> >>>>> process elements of other splits as soon as they become available.
> The
> >>>> nice
> >>>>> thing is that this "split merge" logic can now live in Flink and be
> >>>>> optimized and shared between different sources.
> >>>>>
> >>>>> Thanks,
> >>>>> Thomas
> >>>>>
> >>>>>
> >>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
> wrote:
> >>>>>
> >>>>>> Hi,
> >>>>>> Thanks Aljoscha for this FLIP.
> >>>>>>
> >>>>>> 1. I agree with Piotr and Becket that the non-blocking source is
> very
> >>>>>> important. But in addition to `Future/poll`, there may be another
> way
> >> to
> >>>>>> achieve this. I think it may be not very memory friendly if every
> >>>> advance
> >>>>>> call return a Future.
> >>>>>>
> >>>>>> public interface Listener {
> >>>>>>    public void notify();
> >>>>>> }
> >>>>>>
> >>>>>> public interface SplitReader() {
> >>>>>>    /**
> >>>>>>     * When there is no element temporarily, this will return false.
> >>>>>>     * When elements is available again splitReader can call
> >>>>>> listener.notify()
> >>>>>>     * In addition the frame would check `advance` periodically .
> >>>>>>     * Of course advance can always return true and ignore the
> >> listener
> >>>>>> argument for simplicity.
> >>>>>>     */
> >>>>>>    public boolean advance(Listener listener);
> >>>>>> }
> >>>>>>
> >>>>>> 2.  The FLIP tells us very clearly that how to create all Splits and
> >> how
> >>>>>> to create a SplitReader from a Split. But there is no strategy for
> the
> >>>> user
> >>>>>> to choose how to assign the splits to the tasks. I think we could
> add
> >> a
> >>>>>> Enum to let user to choose.
> >>>>>> /**
> >>>>>> public Enum SplitsAssignmentPolicy {
> >>>>>>   Location,
> >>>>>>   Workload,
> >>>>>>   Random,
> >>>>>>   Average
> >>>>>> }
> >>>>>> */
> >>>>>>
> >>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> >> `getNext`
> >>>>>> the `getNext` would need return a `ElementWithTimestamp` because
> some
> >>>>>> sources want to add timestamp to every element. IMO, this is not so
> >>>> memory
> >>>>>> friendly so I prefer this design.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> Thanks
> >>>>>>
> >>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
> >>>>>>
> >>>>>>> Hi,
> >>>>>>>
> >>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
> other
> >>>>>>> possible improvements. I have one proposal. Instead of having a
> >> method:
> >>>>>>>
> >>>>>>> boolean advance() throws IOException;
> >>>>>>>
> >>>>>>> I would replace it with
> >>>>>>>
> >>>>>>> /*
> >>>>>>> * Return a future, which when completed means that source has more
> >>>> data
> >>>>>>> and getNext() will not block.
> >>>>>>> * If you wish to use benefits of non blocking connectors, please
> >>>>>>> implement this method appropriately.
> >>>>>>> */
> >>>>>>> default CompletableFuture<?> isBlocked() {
> >>>>>>>       return CompletableFuture.completedFuture(null);
> >>>>>>> }
> >>>>>>>
> >>>>>>> And rename `getCurrent()` to `getNext()`.
> >>>>>>>
> >>>>>>> Couple of arguments:
> >>>>>>> 1. I don’t understand the division of work between `advance()` and
> >>>>>>> `getCurrent()`. What should be done in which, especially for
> >> connectors
> >>>>>>> that handle records in batches (like Kafka) and when should you
> call
> >>>>>>> `advance` and when `getCurrent()`.
> >>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in
> >> the
> >>>>>>> future to have asynchronous/non blocking connectors and more
> >>>> efficiently
> >>>>>>> handle large number of blocked threads, without busy waiting. While
> >> at
> >>>> the
> >>>>>>> same time it doesn’t add much complexity, since naive connector
> >>>>>>> implementations can be always blocking.
> >>>>>>> 3. This also would allow us to use a fixed size thread pool of task
> >>>>>>> executors, instead of one thread per task.
> >>>>>>>
> >>>>>>> Piotrek
> >>>>>>>
> >>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
> >>>>>>> wrote:
> >>>>>>>>
> >>>>>>>> Hi All,
> >>>>>>>>
> >>>>>>>> In order to finally get the ball rolling on the new source
> interface
> >>>>>>> that we have discussed for so long I finally created a FLIP:
> >>>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>>>
> >>>>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
> >> about
> >>>>>>> adding per-partition watermark support to the Kinesis source and
> >>>> because
> >>>>>>> this would enable generic implementation of event-time alignment
> for
> >>>> all
> >>>>>>> sources. Maybe we need another FLIP for the event-time alignment
> >> part,
> >>>>>>> especially the part about information sharing between operations
> (I'm
> >>>> not
> >>>>>>> calling it state sharing because state has a special meaning in
> >> Flink).
> >>>>>>>>
> >>>>>>>> Please discuss away!
> >>>>>>>>
> >>>>>>>> Aljoscha
> >>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>
> >>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@data-artisans.com>.
Hi Becket,

With my proposal, both of your examples would have to be solved by the connector and solution to both problems would be the same:

Pretend that connector is never blocked (`isBlocked() { return NOT_BLOCKED; }`) and implement `getNextElement()` in blocking fashion (or semi blocking with return of control from time to time to allow for checkpointing, network flushing and other resource management things to happen in the same main thread). In other words, exactly how you would implement `take()` method or how the same source connector would be implemented NOW with current source interface. The difference with current interface would be only that main loop would be outside of the connector, and instead of periodically releasing checkpointing lock, periodically `return null;` or `return Optional.empty();` from `getNextElement()`.

In case of thread-less readers with only non-blocking `queue.poll()` (is that really a thing? I can not think about a real implementation that enforces such constraints), we could provide a wrapper that hides the busy looping. The same applies how to solve forever blocking readers - we could provider another wrapper running the connector in separate thread.  

But I don’t see a reason why we should expose both blocking `take()` and non-blocking `poll()` methods to the Flink engine. Someone (Flink engine or connector) would have to do the same busy looping anyway and I think it would be better to have a simpler connector API (that would solve our problems) and force connectors to comply one way or another. 

Piotrek

> On 7 Nov 2018, at 10:55, Becket Qin <be...@gmail.com> wrote:
> 
> Hi Piotr,
> 
> I might have misunderstood you proposal. But let me try to explain my
> concern. I am thinking about the following case:
> 1. a reader has the following two interfaces,
>    boolean isBlocked()
>    T getNextElement()
> 2. the implementation of getNextElement() is non-blocking.
> 3. The reader is thread-less, i.e. it does not have any internal thread.
> For example, it might just delegate the getNextElement() to a queue.poll(),
> and isBlocked() is just queue.isEmpty().
> 
> How can Flink efficiently implement a blocking reading behavior with this
> reader? Either a tight loop or a backoff interval is needed. Neither of
> them is ideal.
> 
> Now let's say in the reader mentioned above implements a blocking
> getNextElement() method. Because there is no internal thread in the reader,
> after isBlocked() returns false. Flink will still have to loop on
> isBlocked() to check whether the next record is available. If the next
> record reaches after 10 min, it is a tight loop for 10 min. You have
> probably noticed that in this case, even isBlocked() returns a future, that
> future() will not be completed if Flink does not call some method from the
> reader, because the reader has no internal thread to complete that future
> by itself.
> 
> Due to the above reasons, a blocking take() API would allow Flink to have
> an efficient way to read from a reader. There are many ways to wake up the
> blocking thread when checkpointing is needed depending on the
> implementation. But I think the poll()/take() API would also work in that
> case.
> 
> Thanks,
> 
> Jiangjie (Becket) Qin
> 
> On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <pi...@data-artisans.com>
> wrote:
> 
>> Hi,
>> 
>> a)
>> 
>>> BTW, regarding the isBlock() method, I have a few more questions. 21, Is
>> a method isReady() with boolean as a return value
>>> equivalent? Personally I found it is a little bit confusing in what is
>> supposed to be returned when the future is completed. 22. if
>>> the implementation of isBlocked() is optional, how do the callers know
>> whether the method is properly implemented or not?
>>> Does not implemented mean it always return a completed future?
>> 
>> `CompletableFuture<?> isBlocked()` is more or less an equivalent to
>> `boolean hasNext()` which in case of “false” provides some kind of a
>> listener/callback that notifies about presence of next element. There are
>> some minor details, like `CompletableFuture<?>` has a minimal two state
>> logic:
>> 
>> 1. Future is completed - we have more data
>> 2. Future not yet completed - we don’t have data now, but we might/we will
>> have in the future
>> 
>> While `boolean hasNext()` and `notify()` callback are a bit more
>> complicated/dispersed and can lead/encourage `notify()` spam.
>> 
>> b)
>> 
>>> 3. If merge the `advance` and `getCurrent`  to one method like `getNext`
>> the `getNext` would need return a
>>> `ElementWithTimestamp` because some sources want to add timestamp to
>> every element. IMO, this is not so memory friendly
>>> so I prefer this design.
>> 
>> Guowei I don’t quite understand this. Could you elaborate why having a
>> separate `advance()` help?
>> 
>> c)
>> 
>> Regarding advance/poll/take. What’s the value of having two separate
>> methods: poll and take? Which one of them should be called and which
>> implemented? What’s the benefit of having those methods compared to having
>> a one single method `getNextElement()` (or `pollElement() or whatever we
>> name it) with following contract:
>> 
>> CompletableFuture<?> isBlocked();
>> 
>> /**
>> Return next element - will be called only if `isBlocked()` is completed.
>> Try to implement it in non blocking fashion, but if that’s impossible or
>> you just don’t need the effort, you can block in this method.
>> */
>> T getNextElement();
>> 
>> I mean, if the connector is implemented non-blockingly, Flink should use
>> it that way. If it’s not, then `poll()` will `throw new
>> NotImplementedException()`. Implementing both of them and providing both of
>> them to Flink wouldn’t make a sense, thus why not merge them into a single
>> method call that should preferably (but not necessarily need to) be
>> non-blocking? It’s not like we are implementing general purpose `Queue`,
>> which users might want to call either of `poll` or `take`. We would always
>> prefer to call `poll`, but if it’s blocking, then still we have no choice,
>> but to call it and block on it.
>> 
>> d)
>> 
>>> 1. I agree with Piotr and Becket that the non-blocking source is very
>>> important. But in addition to `Future/poll`, there may be another way to
>>> achieve this. I think it may be not very memory friendly if every advance
>>> call return a Future.
>> 
>> I didn’t want to mention this, to not clog my initial proposal, but there
>> is a simple solution for the problem:
>> 
>> public interface SplitReader {
>> 
>>    (…)
>> 
>>    CompletableFuture<?> NOT_BLOCKED =
>> CompletableFuture.completedFuture(null);
>> 
>>    /**
>>     * Returns a future that will be completed when the page source becomes
>>     * unblocked.  If the page source is not blocked, this method should
>> return
>>     * {@code NOT_BLOCKED}.
>>     */
>>    default CompletableFuture<?> isBlocked()
>>    {
>>        return NOT_BLOCKED;
>>    }
>> 
>> If we are blocked and we are waiting for the IO, then creating a new
>> Future is non-issue. Under full throttle/throughput and not blocked sources
>> returning a static `NOT_BLOCKED` constant  should also solve the problem.
>> 
>> One more remark, non-blocking sources might be a necessity in a single
>> threaded model without a checkpointing lock. (Currently when sources are
>> blocked, they can release checkpointing lock and re-acquire it again
>> later). Non-blocking `poll`/`getNext()` would allow for checkpoints to
>> happen when source is idling. In that case either `notify()` or my proposed
>> `isBlocked()` would allow to avoid busy-looping.
>> 
>> Piotrek
>> 
>>> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
>>> 
>>> Hi Thomas,
>>> 
>>> The iterator-like API was also the first thing that came to me. But it
>>> seems a little confusing that hasNext() does not mean "the stream has not
>>> ended", but means "the next record is ready", which is repurposing the
>> well
>>> known meaning of hasNext(). If we follow the hasNext()/next() pattern, an
>>> additional isNextReady() method to indicate whether the next record is
>>> ready seems more intuitive to me.
>>> 
>>> Similarly, in poll()/take() pattern, another method of isDone() is needed
>>> to indicate whether the stream has ended or not.
>>> 
>>> Compared with hasNext()/next()/isNextReady() pattern,
>>> isDone()/poll()/take() seems more flexible for the reader implementation.
>>> When I am implementing a reader, I could have a couple of choices:
>>> 
>>>  - A thread-less reader that does not have any internal thread.
>>>  - When poll() is called, the same calling thread will perform a bunch
>> of
>>>     IO asynchronously.
>>>     - When take() is called, the same calling thread will perform a
>> bunch
>>>     of IO and wait until the record is ready.
>>>  - A reader with internal threads performing network IO and put records
>>>  into a buffer.
>>>     - When poll() is called, the calling thread simply reads from the
>>>     buffer and return empty result immediately if there is no record.
>>>     - When take() is called, the calling thread reads from the buffer
>> and
>>>     block waiting if the buffer is empty.
>>> 
>>> On the other hand, with the hasNext()/next()/isNextReady() API, it is
>> less
>>> intuitive for the reader developers to write the thread-less pattern.
>>> Although technically speaking one can still do the asynchronous IO to
>>> prepare the record in isNextReady(). But it is inexplicit and seems
>>> somewhat hacky.
>>> 
>>> Thanks,
>>> 
>>> Jiangjie (Becket) Qin
>>> 
>>> 
>>> 
>>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
>>> 
>>>> Couple more points regarding discovery:
>>>> 
>>>> The proposal mentions that discovery could be outside the execution
>> graph.
>>>> Today, discovered partitions/shards are checkpointed. I believe that
>> will
>>>> also need to be the case in the future, even when discovery and reading
>> are
>>>> split between different tasks.
>>>> 
>>>> For cases such as resharding of a Kinesis stream, the relationship
>> between
>>>> splits needs to be considered. Splits cannot be randomly distributed
>> over
>>>> readers in certain situations. An example was mentioned here:
>>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>>>> 
>>>> Thomas
>>>> 
>>>> 
>>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
>>>> 
>>>>> Thanks for getting the ball rolling on this!
>>>>> 
>>>>> Can the number of splits decrease? Yes, splits can be closed and go
>> away.
>>>>> An example would be a shard merge in Kinesis (2 existing shards will be
>>>>> closed and replaced with a new shard).
>>>>> 
>>>>> Regarding advance/poll/take: IMO the least restrictive approach would
>> be
>>>>> the thread-less IO model (pull based, non-blocking, caller retrieves
>> new
>>>>> records when available). The current Kinesis API requires the use of
>>>>> threads. But that can be internal to the split reader and does not need
>>>> to
>>>>> be a source API concern. In fact, that's what we are working on right
>> now
>>>>> as improvement to the existing consumer: Each shard consumer thread
>> will
>>>>> push to a queue, the consumer main thread will poll the queue(s). It is
>>>>> essentially a mapping from threaded IO to non-blocking.
>>>>> 
>>>>> The proposed SplitReader interface would fit the thread-less IO model.
>>>>> Similar to an iterator, we find out if there is a new element (hasNext)
>>>> and
>>>>> if so, move to it (next()). Separate calls deliver the meta information
>>>>> (timestamp, watermark). Perhaps advance call could offer a timeout
>>>> option,
>>>>> so that the caller does not end up in a busy wait. On the other hand, a
>>>>> caller processing multiple splits may want to cycle through fast, to
>>>>> process elements of other splits as soon as they become available. The
>>>> nice
>>>>> thing is that this "split merge" logic can now live in Flink and be
>>>>> optimized and shared between different sources.
>>>>> 
>>>>> Thanks,
>>>>> Thomas
>>>>> 
>>>>> 
>>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com> wrote:
>>>>> 
>>>>>> Hi,
>>>>>> Thanks Aljoscha for this FLIP.
>>>>>> 
>>>>>> 1. I agree with Piotr and Becket that the non-blocking source is very
>>>>>> important. But in addition to `Future/poll`, there may be another way
>> to
>>>>>> achieve this. I think it may be not very memory friendly if every
>>>> advance
>>>>>> call return a Future.
>>>>>> 
>>>>>> public interface Listener {
>>>>>>    public void notify();
>>>>>> }
>>>>>> 
>>>>>> public interface SplitReader() {
>>>>>>    /**
>>>>>>     * When there is no element temporarily, this will return false.
>>>>>>     * When elements is available again splitReader can call
>>>>>> listener.notify()
>>>>>>     * In addition the frame would check `advance` periodically .
>>>>>>     * Of course advance can always return true and ignore the
>> listener
>>>>>> argument for simplicity.
>>>>>>     */
>>>>>>    public boolean advance(Listener listener);
>>>>>> }
>>>>>> 
>>>>>> 2.  The FLIP tells us very clearly that how to create all Splits and
>> how
>>>>>> to create a SplitReader from a Split. But there is no strategy for the
>>>> user
>>>>>> to choose how to assign the splits to the tasks. I think we could add
>> a
>>>>>> Enum to let user to choose.
>>>>>> /**
>>>>>> public Enum SplitsAssignmentPolicy {
>>>>>>   Location,
>>>>>>   Workload,
>>>>>>   Random,
>>>>>>   Average
>>>>>> }
>>>>>> */
>>>>>> 
>>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>> `getNext`
>>>>>> the `getNext` would need return a `ElementWithTimestamp` because some
>>>>>> sources want to add timestamp to every element. IMO, this is not so
>>>> memory
>>>>>> friendly so I prefer this design.
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> Thanks
>>>>>> 
>>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
>>>>>> 
>>>>>>> Hi,
>>>>>>> 
>>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
>>>>>>> possible improvements. I have one proposal. Instead of having a
>> method:
>>>>>>> 
>>>>>>> boolean advance() throws IOException;
>>>>>>> 
>>>>>>> I would replace it with
>>>>>>> 
>>>>>>> /*
>>>>>>> * Return a future, which when completed means that source has more
>>>> data
>>>>>>> and getNext() will not block.
>>>>>>> * If you wish to use benefits of non blocking connectors, please
>>>>>>> implement this method appropriately.
>>>>>>> */
>>>>>>> default CompletableFuture<?> isBlocked() {
>>>>>>>       return CompletableFuture.completedFuture(null);
>>>>>>> }
>>>>>>> 
>>>>>>> And rename `getCurrent()` to `getNext()`.
>>>>>>> 
>>>>>>> Couple of arguments:
>>>>>>> 1. I don’t understand the division of work between `advance()` and
>>>>>>> `getCurrent()`. What should be done in which, especially for
>> connectors
>>>>>>> that handle records in batches (like Kafka) and when should you call
>>>>>>> `advance` and when `getCurrent()`.
>>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in
>> the
>>>>>>> future to have asynchronous/non blocking connectors and more
>>>> efficiently
>>>>>>> handle large number of blocked threads, without busy waiting. While
>> at
>>>> the
>>>>>>> same time it doesn’t add much complexity, since naive connector
>>>>>>> implementations can be always blocking.
>>>>>>> 3. This also would allow us to use a fixed size thread pool of task
>>>>>>> executors, instead of one thread per task.
>>>>>>> 
>>>>>>> Piotrek
>>>>>>> 
>>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
>>>>>>> wrote:
>>>>>>>> 
>>>>>>>> Hi All,
>>>>>>>> 
>>>>>>>> In order to finally get the ball rolling on the new source interface
>>>>>>> that we have discussed for so long I finally created a FLIP:
>>>>>>> 
>>>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>>> 
>>>>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
>> about
>>>>>>> adding per-partition watermark support to the Kinesis source and
>>>> because
>>>>>>> this would enable generic implementation of event-time alignment for
>>>> all
>>>>>>> sources. Maybe we need another FLIP for the event-time alignment
>> part,
>>>>>>> especially the part about information sharing between operations (I'm
>>>> not
>>>>>>> calling it state sharing because state has a special meaning in
>> Flink).
>>>>>>>> 
>>>>>>>> Please discuss away!
>>>>>>>> 
>>>>>>>> Aljoscha
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>> 
>> 
>> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Piotr,

I might have misunderstood you proposal. But let me try to explain my
concern. I am thinking about the following case:
1. a reader has the following two interfaces,
    boolean isBlocked()
    T getNextElement()
2. the implementation of getNextElement() is non-blocking.
3. The reader is thread-less, i.e. it does not have any internal thread.
For example, it might just delegate the getNextElement() to a queue.poll(),
and isBlocked() is just queue.isEmpty().

How can Flink efficiently implement a blocking reading behavior with this
reader? Either a tight loop or a backoff interval is needed. Neither of
them is ideal.

Now let's say in the reader mentioned above implements a blocking
getNextElement() method. Because there is no internal thread in the reader,
after isBlocked() returns false. Flink will still have to loop on
isBlocked() to check whether the next record is available. If the next
record reaches after 10 min, it is a tight loop for 10 min. You have
probably noticed that in this case, even isBlocked() returns a future, that
future() will not be completed if Flink does not call some method from the
reader, because the reader has no internal thread to complete that future
by itself.

Due to the above reasons, a blocking take() API would allow Flink to have
an efficient way to read from a reader. There are many ways to wake up the
blocking thread when checkpointing is needed depending on the
implementation. But I think the poll()/take() API would also work in that
case.

Thanks,

Jiangjie (Becket) Qin

On Wed, Nov 7, 2018 at 4:31 PM Piotr Nowojski <pi...@data-artisans.com>
wrote:

> Hi,
>
> a)
>
> > BTW, regarding the isBlock() method, I have a few more questions. 21, Is
> a method isReady() with boolean as a return value
> > equivalent? Personally I found it is a little bit confusing in what is
> supposed to be returned when the future is completed. 22. if
> > the implementation of isBlocked() is optional, how do the callers know
> whether the method is properly implemented or not?
> > Does not implemented mean it always return a completed future?
>
> `CompletableFuture<?> isBlocked()` is more or less an equivalent to
> `boolean hasNext()` which in case of “false” provides some kind of a
> listener/callback that notifies about presence of next element. There are
> some minor details, like `CompletableFuture<?>` has a minimal two state
> logic:
>
> 1. Future is completed - we have more data
> 2. Future not yet completed - we don’t have data now, but we might/we will
> have in the future
>
> While `boolean hasNext()` and `notify()` callback are a bit more
> complicated/dispersed and can lead/encourage `notify()` spam.
>
> b)
>
> > 3. If merge the `advance` and `getCurrent`  to one method like `getNext`
> the `getNext` would need return a
> >`ElementWithTimestamp` because some sources want to add timestamp to
> every element. IMO, this is not so memory friendly
> > so I prefer this design.
>
> Guowei I don’t quite understand this. Could you elaborate why having a
> separate `advance()` help?
>
> c)
>
> Regarding advance/poll/take. What’s the value of having two separate
> methods: poll and take? Which one of them should be called and which
> implemented? What’s the benefit of having those methods compared to having
> a one single method `getNextElement()` (or `pollElement() or whatever we
> name it) with following contract:
>
> CompletableFuture<?> isBlocked();
>
> /**
> Return next element - will be called only if `isBlocked()` is completed.
> Try to implement it in non blocking fashion, but if that’s impossible or
> you just don’t need the effort, you can block in this method.
> */
> T getNextElement();
>
> I mean, if the connector is implemented non-blockingly, Flink should use
> it that way. If it’s not, then `poll()` will `throw new
> NotImplementedException()`. Implementing both of them and providing both of
> them to Flink wouldn’t make a sense, thus why not merge them into a single
> method call that should preferably (but not necessarily need to) be
> non-blocking? It’s not like we are implementing general purpose `Queue`,
> which users might want to call either of `poll` or `take`. We would always
> prefer to call `poll`, but if it’s blocking, then still we have no choice,
> but to call it and block on it.
>
> d)
>
> > 1. I agree with Piotr and Becket that the non-blocking source is very
> > important. But in addition to `Future/poll`, there may be another way to
> > achieve this. I think it may be not very memory friendly if every advance
> > call return a Future.
>
> I didn’t want to mention this, to not clog my initial proposal, but there
> is a simple solution for the problem:
>
> public interface SplitReader {
>
>     (…)
>
>     CompletableFuture<?> NOT_BLOCKED =
> CompletableFuture.completedFuture(null);
>
>     /**
>      * Returns a future that will be completed when the page source becomes
>      * unblocked.  If the page source is not blocked, this method should
> return
>      * {@code NOT_BLOCKED}.
>      */
>     default CompletableFuture<?> isBlocked()
>     {
>         return NOT_BLOCKED;
>     }
>
> If we are blocked and we are waiting for the IO, then creating a new
> Future is non-issue. Under full throttle/throughput and not blocked sources
> returning a static `NOT_BLOCKED` constant  should also solve the problem.
>
> One more remark, non-blocking sources might be a necessity in a single
> threaded model without a checkpointing lock. (Currently when sources are
> blocked, they can release checkpointing lock and re-acquire it again
> later). Non-blocking `poll`/`getNext()` would allow for checkpoints to
> happen when source is idling. In that case either `notify()` or my proposed
> `isBlocked()` would allow to avoid busy-looping.
>
> Piotrek
>
> > On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
> >
> > Hi Thomas,
> >
> > The iterator-like API was also the first thing that came to me. But it
> > seems a little confusing that hasNext() does not mean "the stream has not
> > ended", but means "the next record is ready", which is repurposing the
> well
> > known meaning of hasNext(). If we follow the hasNext()/next() pattern, an
> > additional isNextReady() method to indicate whether the next record is
> > ready seems more intuitive to me.
> >
> > Similarly, in poll()/take() pattern, another method of isDone() is needed
> > to indicate whether the stream has ended or not.
> >
> > Compared with hasNext()/next()/isNextReady() pattern,
> > isDone()/poll()/take() seems more flexible for the reader implementation.
> > When I am implementing a reader, I could have a couple of choices:
> >
> >   - A thread-less reader that does not have any internal thread.
> >   - When poll() is called, the same calling thread will perform a bunch
> of
> >      IO asynchronously.
> >      - When take() is called, the same calling thread will perform a
> bunch
> >      of IO and wait until the record is ready.
> >   - A reader with internal threads performing network IO and put records
> >   into a buffer.
> >      - When poll() is called, the calling thread simply reads from the
> >      buffer and return empty result immediately if there is no record.
> >      - When take() is called, the calling thread reads from the buffer
> and
> >      block waiting if the buffer is empty.
> >
> > On the other hand, with the hasNext()/next()/isNextReady() API, it is
> less
> > intuitive for the reader developers to write the thread-less pattern.
> > Although technically speaking one can still do the asynchronous IO to
> > prepare the record in isNextReady(). But it is inexplicit and seems
> > somewhat hacky.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> >
> >
> > On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
> >
> >> Couple more points regarding discovery:
> >>
> >> The proposal mentions that discovery could be outside the execution
> graph.
> >> Today, discovered partitions/shards are checkpointed. I believe that
> will
> >> also need to be the case in the future, even when discovery and reading
> are
> >> split between different tasks.
> >>
> >> For cases such as resharding of a Kinesis stream, the relationship
> between
> >> splits needs to be considered. Splits cannot be randomly distributed
> over
> >> readers in certain situations. An example was mentioned here:
> >> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >>
> >> Thomas
> >>
> >>
> >> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
> >>
> >>> Thanks for getting the ball rolling on this!
> >>>
> >>> Can the number of splits decrease? Yes, splits can be closed and go
> away.
> >>> An example would be a shard merge in Kinesis (2 existing shards will be
> >>> closed and replaced with a new shard).
> >>>
> >>> Regarding advance/poll/take: IMO the least restrictive approach would
> be
> >>> the thread-less IO model (pull based, non-blocking, caller retrieves
> new
> >>> records when available). The current Kinesis API requires the use of
> >>> threads. But that can be internal to the split reader and does not need
> >> to
> >>> be a source API concern. In fact, that's what we are working on right
> now
> >>> as improvement to the existing consumer: Each shard consumer thread
> will
> >>> push to a queue, the consumer main thread will poll the queue(s). It is
> >>> essentially a mapping from threaded IO to non-blocking.
> >>>
> >>> The proposed SplitReader interface would fit the thread-less IO model.
> >>> Similar to an iterator, we find out if there is a new element (hasNext)
> >> and
> >>> if so, move to it (next()). Separate calls deliver the meta information
> >>> (timestamp, watermark). Perhaps advance call could offer a timeout
> >> option,
> >>> so that the caller does not end up in a busy wait. On the other hand, a
> >>> caller processing multiple splits may want to cycle through fast, to
> >>> process elements of other splits as soon as they become available. The
> >> nice
> >>> thing is that this "split merge" logic can now live in Flink and be
> >>> optimized and shared between different sources.
> >>>
> >>> Thanks,
> >>> Thomas
> >>>
> >>>
> >>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com> wrote:
> >>>
> >>>> Hi,
> >>>> Thanks Aljoscha for this FLIP.
> >>>>
> >>>> 1. I agree with Piotr and Becket that the non-blocking source is very
> >>>> important. But in addition to `Future/poll`, there may be another way
> to
> >>>> achieve this. I think it may be not very memory friendly if every
> >> advance
> >>>> call return a Future.
> >>>>
> >>>> public interface Listener {
> >>>>     public void notify();
> >>>> }
> >>>>
> >>>> public interface SplitReader() {
> >>>>     /**
> >>>>      * When there is no element temporarily, this will return false.
> >>>>      * When elements is available again splitReader can call
> >>>> listener.notify()
> >>>>      * In addition the frame would check `advance` periodically .
> >>>>      * Of course advance can always return true and ignore the
> listener
> >>>> argument for simplicity.
> >>>>      */
> >>>>     public boolean advance(Listener listener);
> >>>> }
> >>>>
> >>>> 2.  The FLIP tells us very clearly that how to create all Splits and
> how
> >>>> to create a SplitReader from a Split. But there is no strategy for the
> >> user
> >>>> to choose how to assign the splits to the tasks. I think we could add
> a
> >>>> Enum to let user to choose.
> >>>> /**
> >>>>  public Enum SplitsAssignmentPolicy {
> >>>>    Location,
> >>>>    Workload,
> >>>>    Random,
> >>>>    Average
> >>>>  }
> >>>> */
> >>>>
> >>>> 3. If merge the `advance` and `getCurrent`  to one method like
> `getNext`
> >>>> the `getNext` would need return a `ElementWithTimestamp` because some
> >>>> sources want to add timestamp to every element. IMO, this is not so
> >> memory
> >>>> friendly so I prefer this design.
> >>>>
> >>>>
> >>>>
> >>>> Thanks
> >>>>
> >>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
> >>>>
> >>>>> Hi,
> >>>>>
> >>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
> >>>>> possible improvements. I have one proposal. Instead of having a
> method:
> >>>>>
> >>>>> boolean advance() throws IOException;
> >>>>>
> >>>>> I would replace it with
> >>>>>
> >>>>> /*
> >>>>> * Return a future, which when completed means that source has more
> >> data
> >>>>> and getNext() will not block.
> >>>>> * If you wish to use benefits of non blocking connectors, please
> >>>>> implement this method appropriately.
> >>>>> */
> >>>>> default CompletableFuture<?> isBlocked() {
> >>>>>        return CompletableFuture.completedFuture(null);
> >>>>> }
> >>>>>
> >>>>> And rename `getCurrent()` to `getNext()`.
> >>>>>
> >>>>> Couple of arguments:
> >>>>> 1. I don’t understand the division of work between `advance()` and
> >>>>> `getCurrent()`. What should be done in which, especially for
> connectors
> >>>>> that handle records in batches (like Kafka) and when should you call
> >>>>> `advance` and when `getCurrent()`.
> >>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in
> the
> >>>>> future to have asynchronous/non blocking connectors and more
> >> efficiently
> >>>>> handle large number of blocked threads, without busy waiting. While
> at
> >> the
> >>>>> same time it doesn’t add much complexity, since naive connector
> >>>>> implementations can be always blocking.
> >>>>> 3. This also would allow us to use a fixed size thread pool of task
> >>>>> executors, instead of one thread per task.
> >>>>>
> >>>>> Piotrek
> >>>>>
> >>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
> >>>>> wrote:
> >>>>>>
> >>>>>> Hi All,
> >>>>>>
> >>>>>> In order to finally get the ball rolling on the new source interface
> >>>>> that we have discussed for so long I finally created a FLIP:
> >>>>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>
> >>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
> about
> >>>>> adding per-partition watermark support to the Kinesis source and
> >> because
> >>>>> this would enable generic implementation of event-time alignment for
> >> all
> >>>>> sources. Maybe we need another FLIP for the event-time alignment
> part,
> >>>>> especially the part about information sharing between operations (I'm
> >> not
> >>>>> calling it state sharing because state has a special meaning in
> Flink).
> >>>>>>
> >>>>>> Please discuss away!
> >>>>>>
> >>>>>> Aljoscha
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Dawid Wysakowicz <dw...@apache.org>.
Hi Becket,

I am not sure if I understood the last paragraph correctly, but let me
clarify my thoughts.

I would not add any bounded/batch specific methods to the DataStream.
Imo all the user facing bounded/batch specific methods should be exposed
through the new BoundedDataStream interface.

 1. Using the source() method would basically mean use the information
    from Source#getBoundedness in 2(Scheduling) and 3(Operator
    behaviors). I believe all the unbounded/stream operations can be
    executed also for a batch stream. In other words the
    stream/unbounded operations are a subset of the bounded/batch. So I
    see no problem why it would not be possible to pass a bounded source
    here if we do not care about bounded specific operations such as
    e.g. the sort you mentioned.
 2. Using the continuousSource() method would mean use
    Boundedness#CONTINUOUS_UNBOUNDED in 2 and 3 independent of what
    Source#getBoundedness says. <- not sure how useful this method would
    actually be, imo usually we do want to leverage the boundedness of a
    source
 3. Using boundedSource() would:
     1. throw and exception if Source#getBoundedness returns
        Boundedness#CONTINUOUS_UNBOUNDED
     2. if Source#getBoundedness returns Boundedness#BOUNDED it is used
        in 2 and 3 + we expose additional methods in the API as this
        would return BoundedStream

My main concern was that I think the boundedness should come out of the
source rather than from which method on ExecutionEnvironment is used. In
general I am also fine with the methods in your original proposal I
think though we should have a clear logic what happens if you:

 1. pass a bounded source to continuousSource()
 2. pass a continuous source to boundedSource()

Plus why should I think about the boundedness twice if I do not care
about the additional, bounded-specific methods. Once when instantiating
the source (e.g. the example with end timestamp) and second time when
creating the DataStream.

Best,

Dawid

On 09/12/2019 13:52, Becket Qin wrote:
> Hi Dawid,
>
> Thanks for the comments. This actually brings another relevant question
> about what does a "bounded source" imply. I actually had the same
> impression when I look at the Source API. Here is what I understand after
> some discussion with Stephan. The bounded source has the following impacts.
>
> 1. API validity.
> - A bounded source generates a bounded stream so some operations that only
> works for bounded records would be performed, e.g. sort.
> - To expose these bounded stream only APIs, there are two options:
>      a. Add them to the DataStream API and throw exception if a method is
> called on an unbounded stream.
>      b. Create a BoundedDataStream class which is returned from
> env.boundedSource(), while DataStream is returned from env.continousSource().
> Note that this cannot be done by having single env.source(theSource) even
> the Source has a getBoundedness() method.
>
> 2. Scheduling
> - A bounded source could be computed stage by stage without bringing up all
> the tasks at the same time.
>
> 3. Operator behaviors
> - A bounded source indicates the records are finite so some operators can
> wait until it receives all the records before it starts the processing.
>
> In the above impact, only 1 is relevant to the API design. And the current
> proposal in FLIP-27 is following 1.b.
>
> // boundedness depends of source property, imo this should always be
>> preferred
>>
>
> DataStream<MyType> stream = env.source(theSource);
>
>
> In your proposal, does DataStream have bounded stream only methods? It
> looks it should have, otherwise passing a bounded Source to env.source()
> would be confusing. In that case, we will essentially do 1.a if an
> unbounded Source is created from env.source(unboundedSource).
>
> If we have the methods only supported for bounded streams in DataStream, it
> seems a little weird to have a separate BoundedDataStream interface.
>
> Am I understand it correctly?
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <dw...@apache.org>
> wrote:
>
>> Hi all,
>>
>> Really well written proposal and very important one. I must admit I have
>> not understood all the intricacies of it yet.
>>
>> One question I have though is about where does the information about
>> boundedness come from. I think in most cases it is a property of the
>> source. As you described it might be e.g. end offset, a flag should it
>> monitor new splits etc. I think it would be a really nice use case to be
>> able to say:
>>
>> new KafkaSource().readUntil(long timestamp),
>>
>> which could work as an "end offset". Moreover I think all Bounded sources
>> support continuous mode, but no intrinsically continuous source support the
>> Bounded mode. If I understood the proposal correctly it suggest the
>> boundedness sort of "comes" from the outside of the source, from the
>> invokation of either boundedStream or continousSource.
>>
>> I am wondering if it would make sense to actually change the method
>>
>> boolean Source#supportsBoundedness(Boundedness)
>>
>> to
>>
>> Boundedness Source#getBoundedness().
>>
>> As for the methods #boundedSource, #continousSource, assuming the
>> boundedness is property of the source they do not affect how the enumerator
>> works, but mostly how the dag is scheduled, right? I am not against those
>> methods, but I think it is a very specific use case to actually override
>> the property of the source. In general I would expect users to only call
>> env.source(theSource), where the source tells if it is bounded or not. I
>> would suggest considering following set of methods:
>>
>> // boundedness depends of source property, imo this should always be preferred
>>
>> DataStream<MyType> stream = env.source(theSource);
>>
>>
>> // always continous execution, whether bounded or unbounded source
>>
>> DataStream<MyType> boundedStream = env.continousSource(theSource);
>>
>> // imo this would make sense if the BoundedDataStream provides additional features unavailable for continous mode
>> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
>>
>>
>> Best,
>>
>> Dawid
>>
>>
>> On 04/12/2019 11:25, Stephan Ewen wrote:
>>
>> Thanks, Becket, for updating this.
>>
>> I agree with moving the aspects you mentioned into separate FLIPs - this
>> one way becoming unwieldy in size.
>>
>> +1 to the FLIP in its current state. Its a very detailed write-up, nicely
>> done!
>>
>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <be...@gmail.com> <be...@gmail.com> wrote:
>>
>>
>> Hi all,
>>
>> Sorry for the long belated update. I have updated FLIP-27 wiki page with
>> the latest proposals. Some noticeable changes include:
>> 1. A new generic communication mechanism between SplitEnumerator and
>> SourceReader.
>> 2. Some detail API method signature changes.
>>
>> We left a few things out of this FLIP and will address them in separate
>> FLIPs. Including:
>> 1. Per split event time.
>> 2. Event time alignment.
>> 3. Fine grained failover for SplitEnumerator failure.
>>
>> Please let us know if you have any question.
>>
>> Thanks,
>>
>> Jiangjie (Becket) Qin
>>
>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> <se...@apache.org> wrote:
>>
>>
>> Hi  Łukasz!
>>
>> Becket and me are working hard on figuring out the last details and
>> implementing the first PoC. We would update the FLIP hopefully next week.
>>
>> There is a fair chance that a first version of this will be in 1.10, but
>>
>> I
>>
>> think it will take another release to battle test it and migrate the
>> connectors.
>>
>> Best,
>> Stephan
>>
>>
>>
>>
>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <lj...@touk.pl> <lj...@touk.pl>
>>
>> wrote:
>>
>> Hi,
>>
>> This proposal looks very promising for us. Do you have any plans in
>>
>> which
>>
>> Flink release it is going to be released? We are thinking on using a
>>
>> Data
>>
>> Set API for our future use cases but on the other hand Data Set API is
>> going to be deprecated so using proposed bounded data streams solution
>> could be more viable in the long term.
>>
>> Thanks,
>> Łukasz
>>
>> On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> <th...@gmail.com> wrote:
>>
>> Thanks for putting together this proposal!
>>
>> I see that the "Per Split Event Time" and "Event Time Alignment"
>>
>> sections
>>
>> are still TBD.
>>
>> It would probably be good to flesh those out a bit before proceeding
>>
>> too
>>
>> far
>>
>> as the event time alignment will probably influence the interaction
>>
>> with
>>
>> the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
>> output).
>>
>> We currently have only one implementation for event time alignment in
>>
>> the
>>
>> Kinesis consumer. The synchronization in that case takes place as the
>>
>> last
>>
>> step before records are emitted downstream (RecordEmitter). With the
>> currently proposed interfaces, the equivalent can be implemented in
>>
>> the
>>
>> reader loop, although note that in the Kinesis consumer the per shard
>> threads push records.
>>
>> Synchronization has not been implemented for the Kafka consumer yet.
>> https://issues.apache.org/jira/browse/FLINK-12675
>>
>> When I looked at it, I realized that the implementation will look
>>
>> quite
>>
>> different
>> from Kinesis because it needs to take place in the pull part, where
>>
>> records
>>
>> are taken from the Kafka client. Due to the multiplexing it cannot be
>>
>> done
>>
>> by blocking the split thread like it currently works for Kinesis.
>>
>> Reading
>>
>> from individual Kafka partitions needs to be controlled via
>>
>> pause/resume
>>
>> on the Kafka client.
>>
>> To take on that responsibility the split thread would need to be
>>
>> aware
>>
>> of
>>
>> the
>> watermarks or at least whether it should or should not continue to
>>
>> consume
>>
>> a given split and this may require a different SourceReader or
>>
>> SourceOutput
>>
>> interface.
>>
>> Thanks,
>> Thomas
>>
>>
>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com> wrote:
>>
>>
>> Hi Stephan,
>>
>> Thank you for feedback!
>> Will take a look at your branch before public discussing.
>>
>>
>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org> <se...@apache.org>
>>
>> wrote:
>>
>> Hi Biao!
>>
>> Thanks for reviving this. I would like to join this discussion,
>>
>> but
>>
>> am
>>
>> quite occupied with the 1.9 release, so can we maybe pause this
>>
>> discussion
>>
>> for a week or so?
>>
>> In the meantime I can share some suggestion based on prior
>>
>> experiments:
>>
>> How to do watermarks / timestamp extractors in a simpler and more
>>
>> flexible
>>
>> way. I think that part is quite promising should be part of the
>>
>> new
>>
>> source
>>
>> interface.
>>
>>
>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>>
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>>
>> Some experiments on how to build the source reader and its
>>
>> library
>>
>> for
>>
>> common threading/split patterns:
>>
>>
>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>>
>> Best,
>> Stephan
>>
>>
>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>>
>> wrote:
>>
>> Hi devs,
>>
>> Since 1.9 is nearly released, I think we could get back to
>>
>> FLIP-27.
>>
>> I
>>
>> believe it should be included in 1.10.
>>
>> There are so many things mentioned in document of FLIP-27. [1] I
>>
>> think
>>
>> we'd better discuss them separately. However the wiki is not a
>>
>> good
>>
>> place
>>
>> to discuss. I wrote google doc about SplitReader API which
>>
>> misses
>>
>> some
>>
>> details in the document. [2]
>>
>> 1.
>>
>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>>
>> 2.
>>
>>
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>>
>> CC Stephan, Aljoscha, Piotrek, Becket
>>
>>
>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>>
>> wrote:
>>
>> Hi Steven,
>> Thank you for the feedback. Please take a look at the document
>>
>> FLIP-27
>>
>> <
>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>
>> which
>>
>> is updated recently. A lot of details of enumerator were added
>>
>> in
>>
>> this
>>
>> document. I think it would help.
>>
>> Steven Wu <st...@gmail.com> <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
>>
>>
>> This proposal mentioned that SplitEnumerator might run on the
>> JobManager or
>> in a single task on a TaskManager.
>>
>> if enumerator is a single task on a taskmanager, then the job
>>
>> DAG
>>
>> can
>>
>> never
>> been embarrassingly parallel anymore. That will nullify the
>>
>> leverage
>>
>> of
>>
>> fine-grained recovery for embarrassingly parallel jobs.
>>
>> It's not clear to me what's the implication of running
>>
>> enumerator
>>
>> on
>>
>> the
>>
>> jobmanager. So I will leave that out for now.
>>
>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>>
>> wrote:
>>
>> Hi Stephan & Piotrek,
>>
>> Thank you for feedback.
>>
>> It seems that there are a lot of things to do in community.
>>
>> I
>>
>> am
>>
>> just
>>
>> afraid that this discussion may be forgotten since there so
>>
>> many
>>
>> proposals
>>
>> recently.
>> Anyway, wish to see the split topics soon :)
>>
>> Piotr Nowojski <pi...@da-platform.com> <pi...@da-platform.com> 于2019年1月24日周四
>>
>> 下午8:21写道:
>>
>> Hi Biao!
>>
>> This discussion was stalled because of preparations for
>>
>> the
>>
>> open
>>
>> sourcing
>>
>> & merging Blink. I think before creating the tickets we
>>
>> should
>>
>> split this
>>
>> discussion into topics/areas outlined by Stephan and
>>
>> create
>>
>> Flips
>>
>> for
>>
>> that.
>>
>> I think there is no chance for this to be completed in
>>
>> couple
>>
>> of
>>
>> remaining
>>
>> weeks/1 month before 1.8 feature freeze, however it would
>>
>> be
>>
>> good
>>
>> to aim
>>
>> with those changes for 1.9.
>>
>> Piotrek
>>
>>
>> On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>>
>> wrote:
>>
>> Hi community,
>> The summary of Stephan makes a lot sense to me. It is
>>
>> much
>>
>> clearer
>>
>> indeed
>>
>> after splitting the complex topic into small ones.
>> I was wondering is there any detail plan for next step?
>>
>> If
>>
>> not,
>>
>> I
>>
>> would
>>
>> like to push this thing forward by creating some JIRA
>>
>> issues.
>>
>> Another question is that should version 1.8 include
>>
>> these
>>
>> features?
>>
>> Stephan Ewen <se...@apache.org> <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
>>
>>
>> Thanks everyone for the lively discussion. Let me try
>>
>> to
>>
>> summarize
>>
>> where I
>>
>> see convergence in the discussion and open issues.
>> I'll try to group this by design aspect of the source.
>>
>> Please
>>
>> let me
>>
>> know
>>
>> if I got things wrong or missed something crucial here.
>>
>> For issues 1-3, if the below reflects the state of the
>>
>> discussion, I
>>
>> would
>>
>> try and update the FLIP in the next days.
>> For the remaining ones we need more discussion.
>>
>> I would suggest to fork each of these aspects into a
>>
>> separate
>>
>> mail
>>
>> thread,
>>
>> or will loose sight of the individual aspects.
>>
>> *(1) Separation of Split Enumerator and Split Reader*
>>
>>  - All seem to agree this is a good thing
>>  - Split Enumerator could in the end live on JobManager
>>
>> (and
>>
>> assign
>>
>> splits
>>
>> via RPC) or in a task (and assign splits via data
>>
>> streams)
>>
>>  - this discussion is orthogonal and should come later,
>>
>> when
>>
>> the
>>
>> interface
>>
>> is agreed upon.
>>
>> *(2) Split Readers for one or more splits*
>>
>>  - Discussion seems to agree that we need to support
>>
>> one
>>
>> reader
>>
>> that
>>
>> possibly handles multiple splits concurrently.
>>  - The requirement comes from sources where one
>>
>> poll()-style
>>
>> call
>>
>> fetches
>>
>> data from different splits / partitions
>>    --> example sources that require that would be for
>>
>> example
>>
>> Kafka,
>>
>> Pravega, Pulsar
>>
>>  - Could have one split reader per source, or multiple
>>
>> split
>>
>> readers
>>
>> that
>>
>> share the "poll()" function
>>  - To not make it too complicated, we can start with
>>
>> thinking
>>
>> about
>>
>> one
>>
>> split reader for all splits initially and see if that
>>
>> covers
>>
>> all
>>
>> requirements
>>
>> *(3) Threading model of the Split Reader*
>>
>>  - Most active part of the discussion ;-)
>>
>>  - A non-blocking way for Flink's task code to interact
>>
>> with
>>
>> the
>>
>> source
>>
>> is
>>
>> needed in order to a task runtime code based on a
>> single-threaded/actor-style task design
>>    --> I personally am a big proponent of that, it will
>>
>> help
>>
>> with
>>
>> well-behaved checkpoints, efficiency, and simpler yet
>>
>> more
>>
>> robust
>>
>> runtime
>>
>> code
>>
>>  - Users care about simple abstraction, so as a
>>
>> subclass
>>
>> of
>>
>> SplitReader
>>
>> (non-blocking / async) we need to have a
>>
>> BlockingSplitReader
>>
>> which
>>
>> will
>>
>> form the basis of most source implementations.
>>
>> BlockingSplitReader
>>
>> lets
>>
>> users do blocking simple poll() calls.
>>  - The BlockingSplitReader would spawn a thread (or
>>
>> more)
>>
>> and
>>
>> the
>>
>> thread(s) can make blocking calls and hand over data
>>
>> buffers
>>
>> via
>>
>> a
>>
>> blocking
>>
>> queue
>>  - This should allow us to cover both, a fully async
>>
>> runtime,
>>
>> and a
>>
>> simple
>>
>> blocking interface for users.
>>  - This is actually very similar to how the Kafka
>>
>> connectors
>>
>> work.
>>
>> Kafka
>>
>> 9+ with one thread, Kafka 8 with multiple threads
>>
>>  - On the base SplitReader (the async one), the
>>
>> non-blocking
>>
>> method
>>
>> that
>>
>> gets the next chunk of data would signal data
>>
>> availability
>>
>> via
>>
>> a
>>
>> CompletableFuture, because that gives the best
>>
>> flexibility
>>
>> (can
>>
>> await
>>
>> completion or register notification handlers).
>>  - The source task would register a "thenHandle()" (or
>>
>> similar)
>>
>> on the
>>
>> future to put a "take next data" task into the
>>
>> actor-style
>>
>> mailbox
>>
>> *(4) Split Enumeration and Assignment*
>>
>>  - Splits may be generated lazily, both in cases where
>>
>> there
>>
>> is a
>>
>> limited
>>
>> number of splits (but very many), or splits are
>>
>> discovered
>>
>> over
>>
>> time
>>
>>  - Assignment should also be lazy, to get better load
>>
>> balancing
>>
>>  - Assignment needs support locality preferences
>>
>>  - Possible design based on discussion so far:
>>
>>    --> SplitReader has a method "addSplits(SplitT...)"
>>
>> to
>>
>> add
>>
>> one or
>>
>> more
>>
>> splits. Some split readers might assume they have only
>>
>> one
>>
>> split
>>
>> ever,
>>
>> concurrently, others assume multiple splits. (Note:
>>
>> idea
>>
>> behind
>>
>> being
>>
>> able
>>
>> to add multiple splits at the same time is to ease
>>
>> startup
>>
>> where
>>
>> multiple
>>
>> splits may be assigned instantly.)
>>    --> SplitReader has a context object on which it can
>>
>> call
>>
>> indicate
>>
>> when
>>
>> splits are completed. The enumerator gets that
>>
>> notification and
>>
>> can
>>
>> use
>>
>> to
>>
>> decide when to assign new splits. This should help both
>>
>> in
>>
>> cases
>>
>> of
>>
>> sources
>>
>> that take splits lazily (file readers) and in case the
>>
>> source
>>
>> needs to
>>
>> preserve a partial order between splits (Kinesis,
>>
>> Pravega,
>>
>> Pulsar may
>>
>> need
>>
>> that).
>>    --> SplitEnumerator gets notification when
>>
>> SplitReaders
>>
>> start
>>
>> and
>>
>> when
>>
>> they finish splits. They can decide at that moment to
>>
>> push
>>
>> more
>>
>> splits
>>
>> to
>>
>> that reader
>>    --> The SplitEnumerator should probably be aware of
>>
>> the
>>
>> source
>>
>> parallelism, to build its initial distribution.
>>
>>  - Open question: Should the source expose something
>>
>> like
>>
>> "host
>>
>> preferences", so that yarn/mesos/k8s can take this into
>>
>> account
>>
>> when
>>
>> selecting a node to start a TM on?
>>
>> *(5) Watermarks and event time alignment*
>>
>>  - Watermark generation, as well as idleness, needs to
>>
>> be
>>
>> per
>>
>> split
>>
>> (like
>>
>> currently in the Kafka Source, per partition)
>>  - It is desirable to support optional
>>
>> event-time-alignment,
>>
>> meaning
>>
>> that
>>
>> splits that are ahead are back-pressured or temporarily
>>
>> unsubscribed
>>
>>  - I think i would be desirable to encapsulate
>>
>> watermark
>>
>> generation
>>
>> logic
>>
>> in watermark generators, for a separation of concerns.
>>
>> The
>>
>> watermark
>>
>> generators should run per split.
>>  - Using watermark generators would also help with
>>
>> another
>>
>> problem of
>>
>> the
>>
>> suggested interface, namely supporting non-periodic
>>
>> watermarks
>>
>> efficiently.
>>
>>  - Need a way to "dispatch" next record to different
>>
>> watermark
>>
>> generators
>>
>>  - Need a way to tell SplitReader to "suspend" a split
>>
>> until a
>>
>> certain
>>
>> watermark is reached (event time backpressure)
>>  - This would in fact be not needed (and thus simpler)
>>
>> if
>>
>> we
>>
>> had
>>
>> a
>>
>> SplitReader per split and may be a reason to re-open
>>
>> that
>>
>> discussion
>>
>> *(6) Watermarks across splits and in the Split
>>
>> Enumerator*
>>
>>  - The split enumerator may need some watermark
>>
>> awareness,
>>
>> which
>>
>> should
>>
>> be
>>
>> purely based on split metadata (like create timestamp
>>
>> of
>>
>> file
>>
>> splits)
>>
>>  - If there are still more splits with overlapping
>>
>> event
>>
>> time
>>
>> range
>>
>> for
>>
>> a
>>
>> split reader, then that split reader should not advance
>>
>> the
>>
>> watermark
>>
>> within the split beyond the overlap boundary. Otherwise
>>
>> future
>>
>> splits
>>
>> will
>>
>> produce late data.
>>
>>  - One way to approach this could be that the split
>>
>> enumerator
>>
>> may
>>
>> send
>>
>> watermarks to the readers, and the readers cannot emit
>>
>> watermarks
>>
>> beyond
>>
>> that received watermark.
>>  - Many split enumerators would simply immediately send
>>
>> Long.MAX
>>
>> out
>>
>> and
>>
>> leave the progress purely to the split readers.
>>
>>  - For event-time alignment / split back pressure, this
>>
>> begs
>>
>> the
>>
>> question
>>
>> how we can avoid deadlocks that may arise when splits
>>
>> are
>>
>> suspended
>>
>> for
>>
>> event time back pressure,
>>
>> *(7) Batch and streaming Unification*
>>
>>  - Functionality wise, the above design should support
>>
>> both
>>
>>  - Batch often (mostly) does not care about reading "in
>>
>> order"
>>
>> and
>>
>> generating watermarks
>>    --> Might use different enumerator logic that is
>>
>> more
>>
>> locality
>>
>> aware
>>
>> and ignores event time order
>>    --> Does not generate watermarks
>>  - Would be great if bounded sources could be
>>
>> identified
>>
>> at
>>
>> compile
>>
>> time,
>>
>> so that "env.addBoundedSource(...)" is type safe and
>>
>> can
>>
>> return a
>>
>> "BoundedDataStream".
>>  - Possible to defer this discussion until later
>>
>> *Miscellaneous Comments*
>>
>>  - Should the source have a TypeInformation for the
>>
>> produced
>>
>> type,
>>
>> instead
>>
>> of a serializer? We need a type information in the
>>
>> stream
>>
>> anyways, and
>>
>> can
>>
>> derive the serializer from that. Plus, creating the
>>
>> serializer
>>
>> should
>>
>> respect the ExecutionConfig.
>>
>>  - The TypeSerializer interface is very powerful but
>>
>> also
>>
>> not
>>
>> easy to
>>
>> implement. Its purpose is to handle data super
>>
>> efficiently,
>>
>> support
>>
>> flexible ways of evolution, etc.
>>  For metadata I would suggest to look at the
>>
>> SimpleVersionedSerializer
>>
>> instead, which is used for example for checkpoint
>>
>> master
>>
>> hooks,
>>
>> or for
>>
>> the
>>
>> streaming file sink. I think that is is a good match
>>
>> for
>>
>> cases
>>
>> where
>>
>> we
>>
>> do
>>
>> not need more than ser/deser (no copy, etc.) and don't
>>
>> need to
>>
>> push
>>
>> versioning out of the serialization paths for best
>>
>> performance
>>
>> (as in
>>
>> the
>>
>> TypeSerializer)
>>
>>
>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <k....@data-artisans.com>
>> wrote:
>>
>>
>> Hi Biao,
>>
>> Thanks for the answer!
>>
>> So given the multi-threaded readers, now we have as
>>
>> open
>>
>> questions:
>>
>> 1) How do we let the checkpoints pass through our
>>
>> multi-threaded
>>
>> reader
>>
>> operator?
>>
>> 2) Do we have separate reader and source operators or
>>
>> not? In
>>
>> the
>>
>> strategy
>>
>> that has a separate source, the source operator has a
>>
>> parallelism of
>>
>> 1
>>
>> and
>>
>> is responsible for split recovery only.
>>
>> For the first one, given also the constraints
>>
>> (blocking,
>>
>> finite
>>
>> queues,
>>
>> etc), I do not have an answer yet.
>>
>> For the 2nd, I think that we should go with separate
>>
>> operators
>>
>> for
>>
>> the
>>
>> source and the readers, for the following reasons:
>>
>> 1) This is more aligned with a potential future
>>
>> improvement
>>
>> where the
>>
>> split
>>
>> discovery becomes a responsibility of the JobManager
>>
>> and
>>
>> readers are
>>
>> pooling more work from the JM.
>>
>> 2) The source is going to be the "single point of
>>
>> truth".
>>
>> It
>>
>> will
>>
>> know
>>
>> what
>>
>> has been processed and what not. If the source and the
>>
>> readers
>>
>> are a
>>
>> single
>>
>> operator with parallelism > 1, or in general, if the
>>
>> split
>>
>> discovery
>>
>> is
>>
>> done by each task individually, then:
>>   i) we have to have a deterministic scheme for each
>>
>> reader to
>>
>> assign
>>
>> splits to itself (e.g. mod subtaskId). This is not
>>
>> necessarily
>>
>> trivial
>>
>> for
>>
>> all sources.
>>   ii) each reader would have to keep a copy of all its
>>
>> processed
>>
>> slpits
>>
>>   iii) the state has to be a union state with a
>>
>> non-trivial
>>
>> merging
>>
>> logic
>>
>> in order to support rescaling.
>>
>> Two additional points that you raised above:
>>
>> i) The point that you raised that we need to keep all
>>
>> splits
>>
>> (processed
>>
>> and
>>
>> not-processed) I think is a bit of a strong
>>
>> requirement.
>>
>> This
>>
>> would
>>
>> imply
>>
>> that for infinite sources the state will grow
>>
>> indefinitely.
>>
>> This is
>>
>> problem
>>
>> is even more pronounced if we do not have a single
>>
>> source
>>
>> that
>>
>> assigns
>>
>> splits to readers, as each reader will have its own
>>
>> copy
>>
>> of
>>
>> the
>>
>> state.
>>
>> ii) it is true that for finite sources we need to
>>
>> somehow
>>
>> not
>>
>> close
>>
>> the
>>
>> readers when the source/split discoverer finishes. The
>> ContinuousFileReaderOperator has a work-around for
>>
>> that.
>>
>> It is
>>
>> not
>>
>> elegant,
>>
>> and checkpoints are not emitted after closing the
>>
>> source,
>>
>> but
>>
>> this, I
>>
>> believe, is a bigger problem which requires more
>>
>> changes
>>
>> than
>>
>> just
>>
>> refactoring the source interface.
>>
>> Cheers,
>> Kostas
>>
>>
>>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Steven,

Unfortunately we were behind schedule and did not get this into 1.10... So
it will be in 1.11 instead.

Thanks,

Jiangjie (Becket) Qin

On Thu, Jan 16, 2020 at 10:39 AM Steven Wu <st...@gmail.com> wrote:

> Becket, is FLIP-27 still on track to be released in 1.10?
>
> On Tue, Jan 7, 2020 at 7:04 PM Becket Qin <be...@gmail.com> wrote:
>
> > Hi folks,
> >
> > Happy new year!
> >
> > Stephan and I chatted offline yesterday. After reading the email thread
> > again, I found that I have misunderstood Dawid's original proposal
> > regarding the behavior of env.source(BoundedSource) and had an incorrect
> > impression about the behavior of java covariant return type.
> > Anyways, I agree what Dawid originally proposed makes sense, which is the
> > following API:
> >
> > // Return a BoundedDataStream instance if the source is bounded.
> > // Return a DataStream instance if the source is unbounded.
> > DataStream env.source(Source);
> >
> > // Throws exception if the source is unbounded.
> > // Used when users knows the source is bounded at programming time.
> > BoundedDataStream env.boundedSource(Source);
> >
> > A BoundedDataStream only runs in batch execution mode.
> > A DataStream only runs in streaming execution mode.
> >
> > To run a bounded source in streaming execution mode, one would do the
> > following:
> >
> > // Return a DataStream instance with a source that will stop at some
> point;
> > DataStream env.source(SourceUtils.asUnbounded(myBoundedSource));
> >
> > I'll update the FLIP wiki and resume the vote if there is no further
> > concerns.
> >
> > Apologies for the misunderstanding and thanks for all the patient
> > discussions.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> >
> >
> > On Mon, Dec 23, 2019 at 8:00 AM Becket Qin <be...@gmail.com> wrote:
> >
> > > Hi Steven,
> > >
> > > I think the current proposal is what you mentioned - a Kafka source
> that
> > > can be constructed in either BOUNDED or UNBOUNDED mode. And Flink can
> get
> > > the boundedness by invoking getBoundedness().
> > >
> > > So one can create a Kafka source by doing something like the following:
> > >
> > > new KafkaSource().startOffset(),endOffset(); // A bounded instance.
> > > new KafkaSource().startOffset(); // An unbounded instance.
> > >
> > > If users want to have an UNBOUNDED Kafka source that stops at some
> point.
> > > They can wrap the BOUNDED Kafka source like below:
> > >
> > > SourceUtils.asUnbounded(new KafkaSource.startOffset().endOffset());
> > >
> > > The wrapped source would be an unbounded Kafka source that stops at the
> > > end offset.
> > >
> > > Does that make sense?
> > >
> > > Thanks,
> > >
> > > Jiangjie (Becket) Qin
> > >
> > > On Fri, Dec 20, 2019 at 1:31 PM Jark Wu <im...@gmail.com> wrote:
> > >
> > >> Hi,
> > >>
> > >> First of all, I think it is not called "UNBOUNDED", according to the
> > >> FLIP-27, it is called "CONTINUOUS_UNBOUNDED".
> > >> And from the description of the Boundedness in the FLIP-27[1] declares
> > >> clearly what Becket and I think.
> > >>
> > >> public enum Boundedness {
> > >>
> > >>     /**
> > >>      * A bounded source processes the data that is currently available
> > and
> > >> will end after that.
> > >>      *
> > >>      * <p>When a source produces a bounded stream, the runtime may
> > >> activate
> > >> additional optimizations
> > >>      * that are suitable only for bounded input. Incorrectly producing
> > >> unbounded data when the source
> > >>      * is set to produce a bounded stream will often result in
> programs
> > >> that do not output any results
> > >>      * and may eventually fail due to runtime errors (out of memory or
> > >> storage).
> > >>      */
> > >>     BOUNDED,
> > >>
> > >>     /**
> > >>      * A continuous unbounded source continuously processes all data
> as
> > it
> > >> comes.
> > >>      *
> > >>      * <p>The source may run forever (until the program is terminated)
> > or
> > >> might actually end at some point,
> > >>      * based on some source-specific conditions. Because that is not
> > >> transparent to the runtime,
> > >>      * the runtime will use an execution mode for continuous unbounded
> > >> streams whenever this mode
> > >>      * is chosen.
> > >>      */
> > >>     CONTINUOUS_UNBOUNDED
> > >> }
> > >>
> > >> Best,
> > >> Jark
> > >>
> > >> [1]:
> > >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface#FLIP-27:RefactorSourceInterface-Source
> > >>
> > >>
> > >>
> > >> On Fri, 20 Dec 2019 at 12:55, Steven Wu <st...@gmail.com> wrote:
> > >>
> > >> > Becket,
> > >> >
> > >> > Regarding "UNBOUNDED source that stops at some point", I found it
> > >> difficult
> > >> > to grasp what UNBOUNDED really mean.
> > >> >
> > >> > If we want to use Kafka source with an end/stop time, I guess you
> call
> > >> it
> > >> > UNBOUNDED kafka source that stops (aka BOUNDED-streaming). The
> > >> > terminology is a little confusing to me. Maybe BOUNDED/UNBOUNDED
> > >> shouldn't
> > >> > be used to categorize source. Just call it Kafka source and it can
> run
> > >> in
> > >> > either BOUNDED or UNBOUNDED mode.
> > >> >
> > >> > Thanks,
> > >> > Steven
> > >> >
> > >> > On Thu, Dec 19, 2019 at 7:02 PM Becket Qin <be...@gmail.com>
> > >> wrote:
> > >> >
> > >> > > I had an offline chat with Jark, and here are some more thoughts:
> > >> > >
> > >> > > 1. From SQL perspective, BOUNDED source leads to the batch
> execution
> > >> > mode,
> > >> > > UNBOUNDED source leads to the streaming execution mode.
> > >> > > 2. The semantic of UNBOUNDED source is may or may not stop. The
> > >> semantic
> > >> > of
> > >> > > BOUNDED source is will stop.
> > >> > > 3. The semantic of DataStream is may or may not terminate. The
> > >> semantic
> > >> > of
> > >> > > BoundedDataStream is will terminate.
> > >> > >
> > >> > > Given that, option 3 seems a better option because:
> > >> > > 1. SQL already has strict binding between Boundedness and
> execution
> > >> mode.
> > >> > > Letting DataStream be consistent would be good.
> > >> > > 2. The semantic of UNBOUNDED source is exactly the same as
> > >> DataStream. So
> > >> > > we should avoid breaking such semantic, i.e. turning some
> DataStream
> > >> from
> > >> > > "may or may not terminate" to "will terminate".
> > >> > >
> > >> > > For case where users want BOUNDED-streaming combination, they can
> > >> simply
> > >> > > use an UNBOUNDED source that stops at some point. We can even
> > provide
> > >> a
> > >> > > simple wrapper to wrap a BOUNDED source as an UNBOUNDED source if
> > that
> > >> > > helps. But API wise, option 3 seems telling a pretty good whole
> > story.
> > >> > >
> > >> > > Thanks,
> > >> > >
> > >> > > Jiangjie (Becket) Qin
> > >> > >
> > >> > >
> > >> > >
> > >> > >
> > >> > > On Thu, Dec 19, 2019 at 10:30 PM Becket Qin <becket.qin@gmail.com
> >
> > >> > wrote:
> > >> > >
> > >> > > > Hi Timo,
> > >> > > >
> > >> > > > Bounded is just a special case of unbounded and every bounded
> > source
> > >> > can
> > >> > > >> also be treated as an unbounded source. This would unify the
> API
> > if
> > >> > > >> people don't need a bounded operation.
> > >> > > >
> > >> > > >
> > >> > > > With option 3 users can still get a unified API with something
> > like
> > >> > > below:
> > >> > > >
> > >> > > > DataStream boundedStream = env.boundedSource(boundedSource);
> > >> > > > DataStream unboundedStream = env.source(unboundedSource);
> > >> > > >
> > >> > > > So in both cases, users can still use a unified DataStream
> without
> > >> > > > touching the bounded stream only methods.
> > >> > > > By "unify the API if people don't need the bounded operation".
> Do
> > >> you
> > >> > > > expect a DataStream with a Bounded source to have the batch
> > >> operators
> > >> > and
> > >> > > > scheduler settings as well?
> > >> > > >
> > >> > > >
> > >> > > > If we allow DataStream from BOUNDED source, we will essentially
> > pick
> > >> > > "*modified
> > >> > > > option 2*".
> > >> > > >
> > >> > > > // The source is either bounded or unbounded, but only unbounded
> > >> > > >> operations could be performed on the returned DataStream.
> > >> > > >> DataStream<Type> dataStream = env.source(someSource);
> > >> > > >
> > >> > > >
> > >> > > >> // The source must be a bounded source, otherwise exception is
> > >> thrown.
> > >> > > >> BoundedDataStream<Type> boundedDataStream =
> > >> > > >> env.boundedSource(boundedSource);
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > > // Add the following method to DataStream
> > >> > > >
> > >> > > > Boundedness DataStream#getBoundedness();
> > >> > > >
> > >> > > >
> > >> > > > From pure logical perspective, Boundedness and runtime settings
> > >> > > > (Stream/Batch) are two orthogonal dimensions. And are specified
> in
> > >> the
> > >> > > > following way.
> > >> > > >
> > >> > > > *Boundedness* - defined by the source: BOUNDED / UNBOUNDED.
> > >> > > > *Running mode* - defined by the API class: DataStream (Streaming
> > >> mode)
> > >> > /
> > >> > > > BoundedDataStream (batch mode).
> > >> > > >
> > >> > > > Excluding the UNBOUNDED-batch combination, the "*modified option
> > 2"*
> > >> > > > covers the rest three combination. Compared with "*modified
> option
> > >> 2*",
> > >> > > > the main benefit of option 3 is its simplicity and clearness, by
> > >> tying
> > >> > > > boundedness to running mode and giving up BOUNDED-streaming
> > >> > combination.
> > >> > > >
> > >> > > > Just to be clear, I am fine with either option. But I would like
> > to
> > >> > > > understand a bit more about the bounded-streaming use case and
> > when
> > >> > users
> > >> > > > would prefer this over bounded-batch case, and whether the added
> > >> value
> > >> > > > justifies the additional complexity in the API. Two cases I can
> > >> think
> > >> > of
> > >> > > > are:
> > >> > > > 1. The records in DataStream will be processed in order, while
> > >> > > > BoundedDataStream processes records without order guarantee.
> > >> > > > 2. DataStream emits intermediate results when processing a
> finite
> > >> > > dataset,
> > >> > > > while BoundedDataStream only emit the final result. In any case,
> > it
> > >> > could
> > >> > > > be supported by an UNBOUNDED source stopping at some point.
> > >> > > >
> > >> > > > Case 1 is actually misleading because DataStream in general
> > doesn't
> > >> > > really
> > >> > > > support in-order process.
> > >> > > > Case 2 seems a rare use case because the instantaneous
> > intermediate
> > >> > > result
> > >> > > > seems difficult to reason about. In any case, this can be
> > supported
> > >> by
> > >> > an
> > >> > > > UNBOUNDED source that stops at some point.
> > >> > > >
> > >> > > > Is there other use cases for bounded-streaming combination I
> > >> missed? I
> > >> > am
> > >> > > > a little hesitating to put the testing requirement here because
> > >> ideally
> > >> > > I'd
> > >> > > > avoid having public APIs for testing purpose only. And this
> could
> > be
> > >> > > > resolved by having a UNBOUNDED source stopping at some point as
> > >> well.
> > >> > > >
> > >> > > > Sorry for the long discussion, but I would really like to make
> an
> > >> API
> > >> > > > decision after knowing all the pros and cons.
> > >> > > >
> > >> > > > Thanks,
> > >> > > >
> > >> > > > Jiangjie (Becket) Qin
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > > On Thu, Dec 19, 2019 at 6:19 PM Timo Walther <
> twalthr@apache.org>
> > >> > wrote:
> > >> > > >
> > >> > > >> Hi Becket,
> > >> > > >>
> > >> > > >> regarding *Option 3* I think we can relax the constraints for
> > >> > > >> env.source():
> > >> > > >>
> > >> > > >> // MySource can be bounded or unbounded
> > >> > > >> DataStream<Type> dataStream = env.source(mySource);
> > >> > > >>
> > >> > > >> // MySource must be bounded, otherwise throws exception.
> > >> > > >> BoundedDataStream<Type> boundedDataStream =
> > >> > env.boundedSource(mySource);
> > >> > > >>
> > >> > > >> Bounded is just a special case of unbounded and every bounded
> > >> source
> > >> > can
> > >> > > >> also be treated as an unbounded source. This would unify the
> API
> > if
> > >> > > >> people don't need a bounded operation. It also addresses Jark's
> > >> > > concerns.
> > >> > > >>
> > >> > > >> Regards,
> > >> > > >> Timo
> > >> > > >>
> > >> > > >>
> > >> > > >> On 18.12.19 14:16, Becket Qin wrote:
> > >> > > >> > Hi Jark,
> > >> > > >> >
> > >> > > >> > Please see the reply below:
> > >> > > >> >
> > >> > > >> > Regarding to option#3, my concern is that if we don't support
> > >> > > streaming
> > >> > > >> >> mode for bounded source,
> > >> > > >> >> how could we create a testing source for streaming mode?
> > >> Currently,
> > >> > > >> all the
> > >> > > >> >> testing source for streaming
> > >> > > >> >> are bounded, so that the integration test will finish
> finally.
> > >> > > >> >
> > >> > > >> >
> > >> > > >> > An UNBOUNDED source does not mean it will never stops. It
> > simply
> > >> > > >> indicates
> > >> > > >> > that the source *may* run forever, so the runtime needs to be
> > >> > prepared
> > >> > > >> for
> > >> > > >> > that, but the task may still stop at some point when it hits
> > some
> > >> > > >> > source-specific condition. So an UNBOUNDED testing source can
> > >> still
> > >> > > >> stop at
> > >> > > >> > some point if needed.
> > >> > > >> >
> > >> > > >> > Regarding to Source#getRecordOrder(), could we have a
> implicit
> > >> > > contract
> > >> > > >> >> that unbounded source should
> > >> > > >> >> already read in order (i.e. reading partitions in parallel),
> > for
> > >> > > >> bounded
> > >> > > >> >> source the order is not mandatory.
> > >> > > >> >
> > >> > > >> >
> > >> > > >> >
> > >> > > >> >> This is also the behaviors of the current sources.
> > >> > > >> >
> > >> > > >> > 1) a source can't guarantee it reads in strict order, because
> > the
> > >> > > >> producer
> > >> > > >> >> may produce data not in order.
> > >> > > >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can
> > >> > reorder
> > >> > > >> data.
> > >> > > >> >
> > >> > > >> >
> > >> > > >> > It is true that sometimes the source cannot guarantee the
> > record
> > >> > > order,
> > >> > > >> but
> > >> > > >> > sometimes it can. Right now, even for stream processing,
> there
> > >> is no
> > >> > > >> > processing order guarantee. For example, a join operator may
> > >> emit a
> > >> > > >> later
> > >> > > >> > record which successfully found a join match earlier.
> > >> > > >> > Event order is one of the most important requirements for
> event
> > >> > > >> processing,
> > >> > > >> > a clear order guarantee would be necessary. That said, I
> agree
> > >> that
> > >> > > >> right
> > >> > > >> > now even if the sources provide the record order requirement,
> > the
> > >> > > >> runtime
> > >> > > >> > is not able to guarantee that out of the box. So I am OK if
> we
> > >> add
> > >> > the
> > >> > > >> > record order to the Source later. But we should avoid
> > misleading
> > >> > users
> > >> > > >> to
> > >> > > >> > make them think the processing order is guaranteed when using
> > the
> > >> > > >> unbounded
> > >> > > >> > runtime.
> > >> > > >> >
> > >> > > >> > Thanks,
> > >> > > >> >
> > >> > > >> > Jiangjie (Becket) Qin
> > >> > > >> >
> > >> > > >> > On Wed, Dec 18, 2019 at 10:29 AM Jark Wu <im...@gmail.com>
> > >> wrote:
> > >> > > >> >
> > >> > > >> >> Hi Becket,
> > >> > > >> >>
> > >> > > >> >> That's great we have reached a consensus on
> > >> > Source#getBoundedness().
> > >> > > >> >>
> > >> > > >> >> Regarding to option#3, my concern is that if we don't
> support
> > >> > > streaming
> > >> > > >> >> mode for bounded source,
> > >> > > >> >> how could we create a testing source for streaming mode?
> > >> Currently,
> > >> > > >> all the
> > >> > > >> >> testing source for streaming
> > >> > > >> >> are bounded, so that the integration test will finish
> finally.
> > >> > > >> >>
> > >> > > >> >> Regarding to Source#getRecordOrder(), could we have a
> implicit
> > >> > > contract
> > >> > > >> >> that unbounded source should
> > >> > > >> >> already read in order (i.e. reading partitions in parallel),
> > for
> > >> > > >> bounded
> > >> > > >> >> source the order is not mandatory.
> > >> > > >> >> This is also the behaviors of the current sources.
> > >> > > >> >> 1) a source can't guarantee it reads in strict order,
> because
> > >> the
> > >> > > >> producer
> > >> > > >> >> may produce data not in order.
> > >> > > >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can
> > >> > reorder
> > >> > > >> data.
> > >> > > >> >>
> > >> > > >> >> Best,
> > >> > > >> >> Jark
> > >> > > >> >>
> > >> > > >> >>
> > >> > > >> >>
> > >> > > >> >> On Tue, 17 Dec 2019 at 22:03, Becket Qin <
> > becket.qin@gmail.com>
> > >> > > wrote:
> > >> > > >> >>
> > >> > > >> >>> Hi folks,
> > >> > > >> >>>
> > >> > > >> >>> Thanks for the comments. I am convinced that the Source API
> > >> should
> > >> > > not
> > >> > > >> >> take
> > >> > > >> >>> boundedness as a parameter after it is constructed. What
> Timo
> > >> and
> > >> > > >> Dawid
> > >> > > >> >>> suggested sounds a reasonable solution to me. So the Source
> > API
> > >> > > would
> > >> > > >> >>> become:
> > >> > > >> >>>
> > >> > > >> >>> Source {
> > >> > > >> >>>      Boundedness getBoundedness();
> > >> > > >> >>> }
> > >> > > >> >>>
> > >> > > >> >>> Assuming the above Source API, in addition to the two
> options
> > >> > > >> mentioned
> > >> > > >> >> in
> > >> > > >> >>> earlier emails, I am thinking of another option:
> > >> > > >> >>>
> > >> > > >> >>> *Option 3:*
> > >> > > >> >>> // MySource must be unbounded, otherwise throws exception.
> > >> > > >> >>> DataStream<Type> dataStream = env.source(mySource);
> > >> > > >> >>>
> > >> > > >> >>> // MySource must be bounded, otherwise throws exception.
> > >> > > >> >>> BoundedDataStream<Type> boundedDataStream =
> > >> > > >> env.boundedSource(mySource);
> > >> > > >> >>>
> > >> > > >> >>> The pros of this API are:
> > >> > > >> >>>     a) It fits the requirements from Table / SQL well.
> > >> > > >> >>>     b) DataStream users still have type safety (option 2
> only
> > >> has
> > >> > > >> partial
> > >> > > >> >>> type safety).
> > >> > > >> >>>     c) Cristal clear boundedness from the API which makes
> > >> > DataStream
> > >> > > >> join
> > >> > > >> >> /
> > >> > > >> >>> connect easy to reason about.
> > >> > > >> >>> The caveats I see,
> > >> > > >> >>>     a) It is inconsistent with Table since Table has one
> > >> unified
> > >> > > >> >> interface.
> > >> > > >> >>>     b) No streaming mode for bounded source.
> > >> > > >> >>>
> > >> > > >> >>> @Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
> > >> > > >> >>> <al...@ververica.com> what do you think of the
> approach?
> > >> > > >> >>>
> > >> > > >> >>>
> > >> > > >> >>> Orthogonal to the above API, I am wondering whether
> > >> boundedness is
> > >> > > the
> > >> > > >> >> only
> > >> > > >> >>> dimension needed to describe the characteristic of the
> Source
> > >> > > >> behavior.
> > >> > > >> >> We
> > >> > > >> >>> may also need to have another dimension of *record order*.
> > >> > > >> >>>
> > >> > > >> >>> For example, when a file source is reading from a directory
> > >> with
> > >> > > >> bounded
> > >> > > >> >>> records, it may have two ways to read.
> > >> > > >> >>> 1. Read files in parallel.
> > >> > > >> >>> 2. Read files in the chronological order.
> > >> > > >> >>> In both cases, the file source is a Bounded Source.
> However,
> > >> the
> > >> > > >> >> processing
> > >> > > >> >>> requirement for downstream may be different. In the first
> > case,
> > >> > the
> > >> > > >> >>> record processing and result emitting order does not
> matter,
> > >> e.g.
> > >> > > word
> > >> > > >> >>> count. In the second case, the records may have to be
> > >> processed in
> > >> > > the
> > >> > > >> >>> order they were read, e.g. change log processing.
> > >> > > >> >>>
> > >> > > >> >>> If the Source only has a getBoundedness() method, the
> > >> downstream
> > >> > > >> >> processors
> > >> > > >> >>> would not know whether the records emitted from the Source
> > >> should
> > >> > be
> > >> > > >> >>> processed in order or not. So combining the boundedness and
> > >> record
> > >> > > >> order,
> > >> > > >> >>> we will have four scenarios:
> > >> > > >> >>>
> > >> > > >> >>> *Bounded-StrictOrder*:     A segment of change log.
> > >> > > >> >>> *Bounded-Random*:          Batch Word Count.
> > >> > > >> >>> *Unbounded-StrictOrder*: An infinite change log.
> > >> > > >> >>> *Unbounded-Random*:     Streaming Word Count.
> > >> > > >> >>>
> > >> > > >> >>> Option 2 mentioned in the previous email was kind of trying
> > to
> > >> > > handle
> > >> > > >> the
> > >> > > >> >>> Bounded-StrictOrder case by creating a DataStream from a
> > >> bounded
> > >> > > >> source,
> > >> > > >> >>> which actually does not work.
> > >> > > >> >>> It looks that we do not have strict order support in some
> > >> > operators
> > >> > > at
> > >> > > >> >> this
> > >> > > >> >>> point, e.g. join. But we may still want to add the semantic
> > to
> > >> the
> > >> > > >> Source
> > >> > > >> >>> first so later on we don't need to change all the source
> > >> > > >> implementations,
> > >> > > >> >>> especially given that many of them will be implemented by
> 3rd
> > >> > party.
> > >> > > >> >>>
> > >> > > >> >>> Given that, we need another dimension of *Record Order* in
> > the
> > >> > > Source.
> > >> > > >> >> More
> > >> > > >> >>> specifically, the API would become:
> > >> > > >> >>>
> > >> > > >> >>> Source {
> > >> > > >> >>>      Boundedness getBoundedness();
> > >> > > >> >>>      RecordOrder getRecordOrder();
> > >> > > >> >>> }
> > >> > > >> >>>
> > >> > > >> >>> public enum RecordOrder {
> > >> > > >> >>>      /** The record in the DataStream must be processed in
> > its
> > >> > > strict
> > >> > > >> >> order
> > >> > > >> >>> for correctness. */
> > >> > > >> >>>      STRICT,
> > >> > > >> >>>      /** The record in the DataStream can be processed in
> > >> > arbitrary
> > >> > > >> order.
> > >> > > >> >>> */
> > >> > > >> >>>      RANDOM;
> > >> > > >> >>> }
> > >> > > >> >>>
> > >> > > >> >>> Any thoughts?
> > >> > > >> >>>
> > >> > > >> >>> Thanks,
> > >> > > >> >>>
> > >> > > >> >>> Jiangjie (Becket) Qin
> > >> > > >> >>>
> > >> > > >> >>> On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <
> > >> twalthr@apache.org>
> > >> > > >> wrote:
> > >> > > >> >>>
> > >> > > >> >>>> Hi Becket,
> > >> > > >> >>>>
> > >> > > >> >>>> I completely agree with Dawid's suggestion. The
> information
> > >> about
> > >> > > the
> > >> > > >> >>>> boundedness should come out of the source. Because most of
> > the
> > >> > > >> >> streaming
> > >> > > >> >>>> sources can be made bounded based on some connector
> specific
> > >> > > >> criterion.
> > >> > > >> >>>> In Kafka, it would be an end offset or end timestamp but
> in
> > >> any
> > >> > > case
> > >> > > >> >>>> having just a env.boundedSource() is not enough because
> > >> > parameters
> > >> > > >> for
> > >> > > >> >>>> making the source bounded are missing.
> > >> > > >> >>>>
> > >> > > >> >>>> I suggest to have a simple `isBounded(): Boolean` flag in
> > >> every
> > >> > > >> source
> > >> > > >> >>>> that might be influenced by a connector builder as Dawid
> > >> > mentioned.
> > >> > > >> >>>>
> > >> > > >> >>>> For type safety during programming, we can still go with
> > >> *Final
> > >> > > state
> > >> > > >> >>>> 1*. By having a env.source() vs env.boundedSource(). The
> > >> latter
> > >> > > would
> > >> > > >> >>>> just enforce that the boolean flag is set to `true` and
> > could
> > >> > make
> > >> > > >> >>>> bounded operations available (if we need that actually).
> > >> > > >> >>>>
> > >> > > >> >>>> However, I don't think that we should start making a
> unified
> > >> > Table
> > >> > > >> API
> > >> > > >> >>>> ununified again. Boundedness is an optimization property.
> > >> Every
> > >> > > >> bounded
> > >> > > >> >>>> operation can also executed in an unbounded way using
> > >> > > >> >> updates/retraction
> > >> > > >> >>>> or watermarks.
> > >> > > >> >>>>
> > >> > > >> >>>> Regards,
> > >> > > >> >>>> Timo
> > >> > > >> >>>>
> > >> > > >> >>>>
> > >> > > >> >>>> On 15.12.19 14:22, Becket Qin wrote:
> > >> > > >> >>>>> Hi Dawid and Jark,
> > >> > > >> >>>>>
> > >> > > >> >>>>> I think the discussion ultimately boils down to the
> > question
> > >> > that
> > >> > > >> >> which
> > >> > > >> >>>> one
> > >> > > >> >>>>> of the following two final states do we want? Once we
> make
> > >> this
> > >> > > >> >>> decision,
> > >> > > >> >>>>> everything else can be naturally derived.
> > >> > > >> >>>>>
> > >> > > >> >>>>> *Final state 1*: Separate API for bounded / unbounded
> > >> > DataStream &
> > >> > > >> >>> Table.
> > >> > > >> >>>>> That means any code users write will be valid at the
> point
> > >> when
> > >> > > they
> > >> > > >> >>>> write
> > >> > > >> >>>>> the code. This is similar to having type safety check at
> > >> > > programming
> > >> > > >> >>>> time.
> > >> > > >> >>>>> For example,
> > >> > > >> >>>>>
> > >> > > >> >>>>> BoundedDataStream extends DataStream {
> > >> > > >> >>>>> // Operations only available for bounded data.
> > >> > > >> >>>>> BoundedDataStream sort(...);
> > >> > > >> >>>>>
> > >> > > >> >>>>> // Interaction with another BoundedStream returns a
> Bounded
> > >> > > stream.
> > >> > > >> >>>>> BoundedJoinedDataStream join(BoundedDataStream other)
> > >> > > >> >>>>>
> > >> > > >> >>>>> // Interaction with another unbounded stream returns an
> > >> > unbounded
> > >> > > >> >>> stream.
> > >> > > >> >>>>> JoinedDataStream join(DataStream other)
> > >> > > >> >>>>> }
> > >> > > >> >>>>>
> > >> > > >> >>>>> BoundedTable extends Table {
> > >> > > >> >>>>>     // Bounded only operation.
> > >> > > >> >>>>> BoundedTable sort(...);
> > >> > > >> >>>>>
> > >> > > >> >>>>> // Interaction with another BoundedTable returns a
> > >> BoundedTable.
> > >> > > >> >>>>> BoundedTable join(BoundedTable other)
> > >> > > >> >>>>>
> > >> > > >> >>>>> // Interaction with another unbounded table returns an
> > >> unbounded
> > >> > > >> >> table.
> > >> > > >> >>>>> Table join(Table other)
> > >> > > >> >>>>> }
> > >> > > >> >>>>>
> > >> > > >> >>>>> *Final state 2*: One unified API for bounded / unbounded
> > >> > > DataStream
> > >> > > >> /
> > >> > > >> >>>>> Table.
> > >> > > >> >>>>> That unified API may throw exception at DAG compilation
> > time
> > >> if
> > >> > an
> > >> > > >> >>>> invalid
> > >> > > >> >>>>> operation is tried. This is what Table API currently
> > follows.
> > >> > > >> >>>>>
> > >> > > >> >>>>> DataStream {
> > >> > > >> >>>>> // Throws exception if the DataStream is unbounded.
> > >> > > >> >>>>> DataStream sort();
> > >> > > >> >>>>> // Get boundedness.
> > >> > > >> >>>>> Boundedness getBoundedness();
> > >> > > >> >>>>> }
> > >> > > >> >>>>>
> > >> > > >> >>>>> Table {
> > >> > > >> >>>>> // Throws exception if the table has infinite rows.
> > >> > > >> >>>>> Table orderBy();
> > >> > > >> >>>>>
> > >> > > >> >>>>> // Get boundedness.
> > >> > > >> >>>>> Boundedness getBoundedness();
> > >> > > >> >>>>> }
> > >> > > >> >>>>>
> > >> > > >> >>>>> >From what I understand, there is no consensus so far on
> > this
> > >> > > >> decision
> > >> > > >> >>>> yet.
> > >> > > >> >>>>> Whichever final state we choose, we need to make it
> > >> consistent
> > >> > > >> across
> > >> > > >> >>> the
> > >> > > >> >>>>> entire project. We should avoid the case that Table
> follows
> > >> one
> > >> > > >> final
> > >> > > >> >>>> state
> > >> > > >> >>>>> while DataStream follows another. Some arguments I am
> aware
> > >> of
> > >> > > from
> > >> > > >> >>> both
> > >> > > >> >>>>> sides so far are following:
> > >> > > >> >>>>>
> > >> > > >> >>>>> Arguments for final state 1:
> > >> > > >> >>>>> 1a) Clean API with method safety check at programming
> time.
> > >> > > >> >>>>> 1b) (Counter 2b) Although SQL does not have programming
> > time
> > >> > error
> > >> > > >> >>>> check, SQL
> > >> > > >> >>>>> is not really a "programming language" per se. So SQL can
> > be
> > >> > > >> >> different
> > >> > > >> >>>> from
> > >> > > >> >>>>> Table and DataStream.
> > >> > > >> >>>>> 1c)  Although final state 2 seems making it easier for
> SQL
> > to
> > >> > use
> > >> > > >> >> given
> > >> > > >> >>>> it
> > >> > > >> >>>>> is more "config based" than "parameter based", final
> state
> > 1
> > >> can
> > >> > > >> >>> probably
> > >> > > >> >>>>> also meet what SQL wants by wrapping the Source in
> > >> TableSource /
> > >> > > >> >>>>> TableSourceFactory API if needed.
> > >> > > >> >>>>>
> > >> > > >> >>>>> Arguments for final state 2:
> > >> > > >> >>>>> 2a) The Source API itself seems already sort of following
> > the
> > >> > > >> unified
> > >> > > >> >>> API
> > >> > > >> >>>>> pattern.
> > >> > > >> >>>>> 2b) There is no "programming time" method error check in
> > SQL
> > >> > case,
> > >> > > >> so
> > >> > > >> >>> we
> > >> > > >> >>>>> cannot really achieve final state 1 across the board.
> > >> > > >> >>>>> 2c) It is an easier path given our current status, i.e.
> > >> Table is
> > >> > > >> >>> already
> > >> > > >> >>>>> following final state 2.
> > >> > > >> >>>>> 2d) Users can always explicitly check the boundedness if
> > they
> > >> > want
> > >> > > >> >> to.
> > >> > > >> >>>>>
> > >> > > >> >>>>> As I mentioned earlier, my initial thought was also to
> > have a
> > >> > > >> >>>>> "configuration based" Source rather than a "parameter
> > based"
> > >> > > Source.
> > >> > > >> >> So
> > >> > > >> >>>> it
> > >> > > >> >>>>> is completely possible that I missed some important
> > >> > consideration
> > >> > > or
> > >> > > >> >>>> design
> > >> > > >> >>>>> principles that we want to enforce for the project. It
> > would
> > >> be
> > >> > > good
> > >> > > >> >>>>> if @Stephan
> > >> > > >> >>>>> Ewen <st...@ververica.com> and @Aljoscha Krettek <
> > >> > > >> >>>> aljoscha@ververica.com> can
> > >> > > >> >>>>> also provide more thoughts on this.
> > >> > > >> >>>>>
> > >> > > >> >>>>>
> > >> > > >> >>>>> Re: Jingsong
> > >> > > >> >>>>>
> > >> > > >> >>>>> As you said, there are some batched system source, like
> > >> > > parquet/orc
> > >> > > >> >>>> source.
> > >> > > >> >>>>>> Could we have the batch emit interface to improve
> > >> performance?
> > >> > > The
> > >> > > >> >>>> queue of
> > >> > > >> >>>>>> per record may cause performance degradation.
> > >> > > >> >>>>>
> > >> > > >> >>>>>
> > >> > > >> >>>>> The current interface does not necessarily cause
> > performance
> > >> > > problem
> > >> > > >> >>> in a
> > >> > > >> >>>>> multi-threading case. In fact, the base implementation
> > allows
> > >> > > >> >>>> SplitReaders
> > >> > > >> >>>>> to add a batch <E> of records<T> to the records queue<E>,
> > so
> > >> > each
> > >> > > >> >>> element
> > >> > > >> >>>>> in the records queue would be a batch <E>. In this case,
> > when
> > >> > the
> > >> > > >> >> main
> > >> > > >> >>>>> thread polls records, it will take a batch <E> of records
> > <T>
> > >> > from
> > >> > > >> >> the
> > >> > > >> >>>>> shared records queue and process the records <T> in a
> batch
> > >> > > manner.
> > >> > > >> >>>>>
> > >> > > >> >>>>> Thanks,
> > >> > > >> >>>>>
> > >> > > >> >>>>> Jiangjie (Becket) Qin
> > >> > > >> >>>>>
> > >> > > >> >>>>> On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <
> > >> > > jingsonglee0@gmail.com
> > >> > > >> >
> > >> > > >> >>>> wrote:
> > >> > > >> >>>>>
> > >> > > >> >>>>>> Hi Becket,
> > >> > > >> >>>>>>
> > >> > > >> >>>>>> I also have some performance concerns too.
> > >> > > >> >>>>>>
> > >> > > >> >>>>>> If I understand correctly, SourceOutput will emit data
> per
> > >> > record
> > >> > > >> >> into
> > >> > > >> >>>> the
> > >> > > >> >>>>>> queue? I'm worried about the multithreading performance
> of
> > >> this
> > >> > > >> >> queue.
> > >> > > >> >>>>>>
> > >> > > >> >>>>>>> One example is some batched messaging systems which
> only
> > >> have
> > >> > an
> > >> > > >> >>> offset
> > >> > > >> >>>>>> for the entire batch instead of individual messages in
> the
> > >> > batch.
> > >> > > >> >>>>>>
> > >> > > >> >>>>>> As you said, there are some batched system source, like
> > >> > > parquet/orc
> > >> > > >> >>>> source.
> > >> > > >> >>>>>> Could we have the batch emit interface to improve
> > >> performance?
> > >> > > The
> > >> > > >> >>>> queue of
> > >> > > >> >>>>>> per record may cause performance degradation.
> > >> > > >> >>>>>>
> > >> > > >> >>>>>> Best,
> > >> > > >> >>>>>> Jingsong Lee
> > >> > > >> >>>>>>
> > >> > > >> >>>>>> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <
> imjark@gmail.com
> > >
> > >> > > wrote:
> > >> > > >> >>>>>>
> > >> > > >> >>>>>>> Hi Becket,
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>> I think Dawid explained things clearly and makes a lot
> of
> > >> > sense.
> > >> > > >> >>>>>>> I'm also in favor of #2, because #1 doesn't work for
> our
> > >> > future
> > >> > > >> >>> unified
> > >> > > >> >>>>>>> envrionment.
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>> You can see the vision in this documentation [1]. In
> the
> > >> > future,
> > >> > > >> we
> > >> > > >> >>>> would
> > >> > > >> >>>>>>> like to
> > >> > > >> >>>>>>> drop the global streaming/batch mode in SQL (i.e.
> > >> > > >> >>>>>>> EnvironmentSettings#inStreamingMode/inBatchMode).
> > >> > > >> >>>>>>> A source is bounded or unbounded once defined, so
> queries
> > >> can
> > >> > be
> > >> > > >> >>>> inferred
> > >> > > >> >>>>>>> from source to run
> > >> > > >> >>>>>>> in streaming or batch or hybrid mode. However, in #1,
> we
> > >> will
> > >> > > lose
> > >> > > >> >>> this
> > >> > > >> >>>>>>> ability because the framework
> > >> > > >> >>>>>>> doesn't know whether the source is bounded or
> unbounded.
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>> Best,
> > >> > > >> >>>>>>> Jark
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>> [1]:
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <
> > >> > > piotr@ververica.com
> > >> > > >> >
> > >> > > >> >>>>>> wrote:
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>>> Hi,
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>> Regarding the:
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>> Collection<E> getNextRecords()
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>> I’m pretty sure such design would unfortunately impact
> > the
> > >> > > >> >>> performance
> > >> > > >> >>>>>>>> (accessing and potentially creating the collection on
> > the
> > >> hot
> > >> > > >> >> path).
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>> Also the
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>> InputStatus emitNext(DataOutput<T> output) throws
> > >> Exception;
> > >> > > >> >>>>>>>> or
> > >> > > >> >>>>>>>> Status pollNext(SourceOutput<T> sourceOutput) throws
> > >> > Exception;
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>> Gives us some opportunities in the future, to allow
> > Source
> > >> > hot
> > >> > > >> >>> looping
> > >> > > >> >>>>>>>> inside, until it receives some signal “please exit
> > >> because of
> > >> > > >> some
> > >> > > >> >>>>>>> reasons”
> > >> > > >> >>>>>>>> (output collector could return such hint upon
> collecting
> > >> the
> > >> > > >> >>> result).
> > >> > > >> >>>>>> But
> > >> > > >> >>>>>>>> that’s another topic outside of this FLIP’s scope.
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>> Piotrek
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <
> > >> > trohrmann@apache.org
> > >> > > >
> > >> > > >> >>>>>> wrote:
> > >> > > >> >>>>>>>>>
> > >> > > >> >>>>>>>>> Hi Becket,
> > >> > > >> >>>>>>>>>
> > >> > > >> >>>>>>>>> quick clarification from my side because I think you
> > >> > > >> >> misunderstood
> > >> > > >> >>> my
> > >> > > >> >>>>>>>>> question. I did not suggest to let the SourceReader
> > >> return
> > >> > > only
> > >> > > >> a
> > >> > > >> >>>>>>> single
> > >> > > >> >>>>>>>>> record at a time when calling getNextRecords. As the
> > >> return
> > >> > > type
> > >> > > >> >>>>>>>> indicates,
> > >> > > >> >>>>>>>>> the method can return an arbitrary number of records.
> > >> > > >> >>>>>>>>>
> > >> > > >> >>>>>>>>> Cheers,
> > >> > > >> >>>>>>>>> Till
> > >> > > >> >>>>>>>>>
> > >> > > >> >>>>>>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> > >> > > >> >>>>>>>> dwysakowicz@apache.org <mailto:dwysakowicz@apache.org
> >>
> > >> > > >> >>>>>>>>> wrote:
> > >> > > >> >>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi Becket,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Issue #1 - Design of Source interface
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I mentioned the lack of a method like
> > >> > > >> >>>>>>>> Source#createEnumerator(Boundedness
> > >> > > >> >>>>>>>>>> boundedness, SplitEnumeratorContext context),
> because
> > >> > without
> > >> > > >> >> the
> > >> > > >> >>>>>>>> current
> > >> > > >> >>>>>>>>>> proposal is not complete/does not work.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> If we say that boundedness is an intrinsic property
> > of a
> > >> > > source
> > >> > > >> >>> imo
> > >> > > >> >>>>>> we
> > >> > > >> >>>>>>>>>> don't need the Source#createEnumerator(Boundedness
> > >> > > boundedness,
> > >> > > >> >>>>>>>>>> SplitEnumeratorContext context) method.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Assuming a source from my previous example:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
> > >> > > >> >>>>>>>>>>    ...
> > >> > > >> >>>>>>>>>>    .untilTimestamp(...)
> > >> > > >> >>>>>>>>>>    .build()
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Would the enumerator differ if created like
> > >> > > >> >>>>>>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...)
> vs
> > >> > source
> > >> > > >> >>>>>>>>>> .createEnumerator(BOUNDED, ...)? I know I am
> repeating
> > >> > > myself,
> > >> > > >> >> but
> > >> > > >> >>>>>>> this
> > >> > > >> >>>>>>>> is
> > >> > > >> >>>>>>>>>> the part that my opinion differ the most from the
> > >> current
> > >> > > >> >>> proposal.
> > >> > > >> >>>>>> I
> > >> > > >> >>>>>>>>>> really think it should always be the source that
> tells
> > >> if
> > >> > it
> > >> > > is
> > >> > > >> >>>>>>> bounded
> > >> > > >> >>>>>>>> or
> > >> > > >> >>>>>>>>>> not. In the current proposal methods
> > >> > > >> >> continousSource/boundedSource
> > >> > > >> >>>>>>>> somewhat
> > >> > > >> >>>>>>>>>> reconfigure the source, which I think is misleading.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I think a call like:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
> > >> > > >> >>>>>>>>>>    ...
> > >> > > >> >>>>>>>>>>    .readContinously() / readUntilLatestOffset() /
> > >> > > >> >>> readUntilTimestamp
> > >> > > >> >>>> /
> > >> > > >> >>>>>>>> readUntilOffsets / ...
> > >> > > >> >>>>>>>>>>    .build()
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> is way cleaner (and expressive) than
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
> > >> > > >> >>>>>>>>>>    ...
> > >> > > >> >>>>>>>>>>    .build()
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> env.continousSource(source) // which actually
> > underneath
> > >> > > would
> > >> > > >> >>> call
> > >> > > >> >>>>>>>> createEnumerator(CONTINUOUS, ctx) which would be
> > >> equivalent
> > >> > to
> > >> > > >> >>>>>>>> source.readContinously().createEnumerator(ctx)
> > >> > > >> >>>>>>>>>> // or
> > >> > > >> >>>>>>>>>> env.boundedSource(source) // which actually
> underneath
> > >> > would
> > >> > > >> >> call
> > >> > > >> >>>>>>>> createEnumerator(BOUNDED, ctx) which would be
> equivalent
> > >> to
> > >> > > >> >>>>>>>> source.readUntilLatestOffset().createEnumerator(ctx)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Sorry for the comparison, but to me it seems there
> is
> > >> too
> > >> > > much
> > >> > > >> >>> magic
> > >> > > >> >>>>>>>>>> happening underneath those two calls.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I really believe the Source interface should have
> > >> > > >> getBoundedness
> > >> > > >> >>>>>>> method
> > >> > > >> >>>>>>>>>> instead of (supportBoundedness) +
> > >> > > createEnumerator(Boundedness,
> > >> > > >> >>> ...)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Issue #2 - Design of
> > >> > > >> >>>>>>>>>>
> > >> > > >>
> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> As you might have guessed I am slightly in favor of
> > >> option
> > >> > #2
> > >> > > >> >>>>>>> modified.
> > >> > > >> >>>>>>>>>> Yes I am aware every step of the dag would have to
> be
> > >> able
> > >> > to
> > >> > > >> >> say
> > >> > > >> >>> if
> > >> > > >> >>>>>>> it
> > >> > > >> >>>>>>>> is
> > >> > > >> >>>>>>>>>> bounded or not. I have a feeling it would be easier
> to
> > >> > > express
> > >> > > >> >>> cross
> > >> > > >> >>>>>>>>>> bounded/unbounded operations, but I must admit I
> have
> > >> not
> > >> > > >> >> thought
> > >> > > >> >>> it
> > >> > > >> >>>>>>>>>> through thoroughly, In the spirit of batch is just a
> > >> > special
> > >> > > >> >> case
> > >> > > >> >>> of
> > >> > > >> >>>>>>>>>> streaming I thought BoundedStream would extend from
> > >> > > DataStream.
> > >> > > >> >>>>>>> Correct
> > >> > > >> >>>>>>>> me
> > >> > > >> >>>>>>>>>> if I am wrong. In such a setup the cross
> > >> bounded/unbounded
> > >> > > >> >>> operation
> > >> > > >> >>>>>>>> could
> > >> > > >> >>>>>>>>>> be expressed quite easily I think:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> DataStream {
> > >> > > >> >>>>>>>>>>    DataStream join(DataStream, ...); // we could not
> > >> really
> > >> > > >> tell
> > >> > > >> >> if
> > >> > > >> >>>>>> the
> > >> > > >> >>>>>>>> result is bounded or not, but because bounded stream
> is
> > a
> > >> > > special
> > >> > > >> >>> case
> > >> > > >> >>>>>> of
> > >> > > >> >>>>>>>> unbounded the API object is correct, irrespective if
> the
> > >> left
> > >> > > or
> > >> > > >> >>> right
> > >> > > >> >>>>>>> side
> > >> > > >> >>>>>>>> of the join is bounded
> > >> > > >> >>>>>>>>>> }
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> BoundedStream extends DataStream {
> > >> > > >> >>>>>>>>>>    BoundedStream join(BoundedStream, ...); // only
> if
> > >> both
> > >> > > >> sides
> > >> > > >> >>> are
> > >> > > >> >>>>>>>> bounded the result can be bounded as well. However we
> do
> > >> have
> > >> > > >> >> access
> > >> > > >> >>>> to
> > >> > > >> >>>>>>> the
> > >> > > >> >>>>>>>> DataStream#join here, so you can still join with a
> > >> DataStream
> > >> > > >> >>>>>>>>>> }
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On the other hand I also see benefits of two
> > completely
> > >> > > >> >> disjointed
> > >> > > >> >>>>>>> APIs,
> > >> > > >> >>>>>>>>>> as we could prohibit some streaming calls in the
> > bounded
> > >> > > API. I
> > >> > > >> >>>>>> can't
> > >> > > >> >>>>>>>> think
> > >> > > >> >>>>>>>>>> of any unbounded operators that could not be
> > implemented
> > >> > for
> > >> > > >> >>> bounded
> > >> > > >> >>>>>>>> stream.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Besides I think we both agree we don't like the
> > method:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> DataStream boundedStream(Source)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> suggested in the current state of the FLIP. Do we ?
> :)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Best,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Dawid
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On 10/12/2019 18:57, Becket Qin wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi folks,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks for the discussion, great feedback. Also
> thanks
> > >> > Dawid
> > >> > > >> for
> > >> > > >> >>> the
> > >> > > >> >>>>>>>>>> explanation, it is much clearer now.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> One thing that is indeed missing from the FLIP is
> how
> > >> the
> > >> > > >> >>>>>> boundedness
> > >> > > >> >>>>>>> is
> > >> > > >> >>>>>>>>>> passed to the Source implementation. So the API
> should
> > >> be
> > >> > > >> >>>>>>>>>> Source#createEnumerator(Boundedness boundedness,
> > >> > > >> >>>>>>> SplitEnumeratorContext
> > >> > > >> >>>>>>>>>> context)
> > >> > > >> >>>>>>>>>> And we can probably remove the
> > >> > > >> >>> Source#supportBoundedness(Boundedness
> > >> > > >> >>>>>>>>>> boundedness) method.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Assuming we have that, we are essentially choosing
> > from
> > >> one
> > >> > > of
> > >> > > >> >> the
> > >> > > >> >>>>>>>>>> following two options:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Option 1:
> > >> > > >> >>>>>>>>>> // The source is continuous source, and only
> unbounded
> > >> > > >> >> operations
> > >> > > >> >>>>>> can
> > >> > > >> >>>>>>> be
> > >> > > >> >>>>>>>>>> performed.
> > >> > > >> >>>>>>>>>> DataStream<Type> datastream =
> > >> > > env.continuousSource(someSource);
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> // The source is bounded source, both bounded and
> > >> unbounded
> > >> > > >> >>>>>> operations
> > >> > > >> >>>>>>>> can
> > >> > > >> >>>>>>>>>> be performed.
> > >> > > >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> > >> > > >> >>>>>>>> env.boundedSource(someSource);
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>    - Pros:
> > >> > > >> >>>>>>>>>>         a) explicit boundary between bounded /
> > unbounded
> > >> > > >> streams,
> > >> > > >> >>> it
> > >> > > >> >>>>>> is
> > >> > > >> >>>>>>>>>> quite simple and clear to the users.
> > >> > > >> >>>>>>>>>>    - Cons:
> > >> > > >> >>>>>>>>>>         a) For applications that do not involve
> > bounded
> > >> > > >> >> operations,
> > >> > > >> >>>>>> they
> > >> > > >> >>>>>>>>>> still have to call different API to distinguish
> > bounded
> > >> /
> > >> > > >> >>> unbounded
> > >> > > >> >>>>>>>> streams.
> > >> > > >> >>>>>>>>>>         b) No support for bounded stream to run in a
> > >> > > streaming
> > >> > > >> >>>> runtime
> > >> > > >> >>>>>>>>>> setting, i.e. scheduling and operators behaviors.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Option 2:
> > >> > > >> >>>>>>>>>> // The source is either bounded or unbounded, but
> only
> > >> > > >> unbounded
> > >> > > >> >>>>>>>> operations
> > >> > > >> >>>>>>>>>> could be performed on the returned DataStream.
> > >> > > >> >>>>>>>>>> DataStream<Type> dataStream =
> env.source(someSource);
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> // The source must be a bounded source, otherwise
> > >> exception
> > >> > > is
> > >> > > >> >>>>>> thrown.
> > >> > > >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> > >> > > >> >>>>>>>>>> env.boundedSource(boundedSource);
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> The pros and cons are exactly the opposite of option
> > 1.
> > >> > > >> >>>>>>>>>>    - Pros:
> > >> > > >> >>>>>>>>>>         a) For applications that do not involve
> > bounded
> > >> > > >> >> operations,
> > >> > > >> >>>>>> they
> > >> > > >> >>>>>>>>>> still have to call different API to distinguish
> > bounded
> > >> /
> > >> > > >> >>> unbounded
> > >> > > >> >>>>>>>> streams.
> > >> > > >> >>>>>>>>>>         b) Support for bounded stream to run in a
> > >> streaming
> > >> > > >> >> runtime
> > >> > > >> >>>>>>>> setting,
> > >> > > >> >>>>>>>>>> i.e. scheduling and operators behaviors.
> > >> > > >> >>>>>>>>>>    - Cons:
> > >> > > >> >>>>>>>>>>         a) Bounded / unbounded streams are kind of
> > >> mixed,
> > >> > > i.e.
> > >> > > >> >>> given
> > >> > > >> >>>> a
> > >> > > >> >>>>>>>>>> DataStream, it is not clear whether it is bounded or
> > >> not,
> > >> > > >> unless
> > >> > > >> >>> you
> > >> > > >> >>>>>>>> have
> > >> > > >> >>>>>>>>>> the access to its source.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> If we only think from the Source API perspective,
> > >> option 2
> > >> > > >> >> seems a
> > >> > > >> >>>>>>>> better
> > >> > > >> >>>>>>>>>> choice because functionality wise it is a superset
> of
> > >> > option
> > >> > > 1,
> > >> > > >> >> at
> > >> > > >> >>>>>> the
> > >> > > >> >>>>>>>> cost
> > >> > > >> >>>>>>>>>> of some seemingly acceptable ambiguity in the
> > DataStream
> > >> > API.
> > >> > > >> >>>>>>>>>> But if we look at the DataStream API as a whole,
> > option
> > >> 1
> > >> > > seems
> > >> > > >> >> a
> > >> > > >> >>>>>>>> clearer
> > >> > > >> >>>>>>>>>> choice. For example, some times a library may have
> to
> > >> know
> > >> > > >> >>> whether a
> > >> > > >> >>>>>>>>>> certain task will finish or not. And it would be
> > >> difficult
> > >> > to
> > >> > > >> >> tell
> > >> > > >> >>>>>> if
> > >> > > >> >>>>>>>> the
> > >> > > >> >>>>>>>>>> input is a DataStream, unless additional information
> > is
> > >> > > >> provided
> > >> > > >> >>> all
> > >> > > >> >>>>>>> the
> > >> > > >> >>>>>>>>>> way from the Source. One possible solution is to
> have
> > a
> > >> > > >> >> *modified
> > >> > > >> >>>>>>>> option 2*
> > >> > > >> >>>>>>>>>> which adds a method to the DataStream API to
> indicate
> > >> > > >> >> boundedness,
> > >> > > >> >>>>>>> such
> > >> > > >> >>>>>>>> as
> > >> > > >> >>>>>>>>>> getBoundedness(). It would solve the problem with a
> > >> > potential
> > >> > > >> >>>>>>> confusion
> > >> > > >> >>>>>>>> of
> > >> > > >> >>>>>>>>>> what is difference between a DataStream with
> > >> > > >> >> getBoundedness()=true
> > >> > > >> >>>>>>> and a
> > >> > > >> >>>>>>>>>> BoundedDataStream. But that seems not super
> difficult
> > to
> > >> > > >> >> explain.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> So from API's perspective, I don't have a strong
> > opinion
> > >> > > >> between
> > >> > > >> >>>>>>>> *option 1*
> > >> > > >> >>>>>>>>>> and *modified option 2. *I like the cleanness of
> > option
> > >> 1,
> > >> > > but
> > >> > > >> >>>>>>> modified
> > >> > > >> >>>>>>>>>> option 2 would be more attractive if we have
> concrete
> > >> use
> > >> > > case
> > >> > > >> >> for
> > >> > > >> >>>>>> the
> > >> > > >> >>>>>>>>>> "Bounded stream with unbounded streaming runtime
> > >> settings".
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Re: Till
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Maybe this has already been asked before but I was
> > >> > wondering
> > >> > > >> why
> > >> > > >> >>> the
> > >> > > >> >>>>>>>>>> SourceReader interface has the method pollNext which
> > >> hands
> > >> > > the
> > >> > > >> >>>>>>>>>> responsibility of outputting elements to the
> > >> SourceReader
> > >> > > >> >>>>>>>> implementation?
> > >> > > >> >>>>>>>>>> Has this been done for backwards compatibility
> reasons
> > >> with
> > >> > > the
> > >> > > >> >>> old
> > >> > > >> >>>>>>>> source
> > >> > > >> >>>>>>>>>> interface? If not, then one could define a
> > Collection<E>
> > >> > > >> >>>>>>>> getNextRecords()
> > >> > > >> >>>>>>>>>> method which returns the currently retrieved records
> > and
> > >> > then
> > >> > > >> >> the
> > >> > > >> >>>>>>> caller
> > >> > > >> >>>>>>>>>> emits them outside of the SourceReader. That way the
> > >> > > interface
> > >> > > >> >>> would
> > >> > > >> >>>>>>> not
> > >> > > >> >>>>>>>>>> allow to implement an outputting loop where we never
> > >> hand
> > >> > > back
> > >> > > >> >>>>>> control
> > >> > > >> >>>>>>>> to
> > >> > > >> >>>>>>>>>> the caller. At the moment, this contract can be
> easily
> > >> > broken
> > >> > > >> >> and
> > >> > > >> >>> is
> > >> > > >> >>>>>>>> only
> > >> > > >> >>>>>>>>>> mentioned loosely in the JavaDocs.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> The primary reason we handover the SourceOutput to
> the
> > >> > > >> >>> SourceReader
> > >> > > >> >>>>>> is
> > >> > > >> >>>>>>>>>> because sometimes it is difficult for a SourceReader
> > to
> > >> > emit
> > >> > > >> one
> > >> > > >> >>>>>>> record
> > >> > > >> >>>>>>>> at
> > >> > > >> >>>>>>>>>> a time. One example is some batched messaging
> systems
> > >> which
> > >> > > >> only
> > >> > > >> >>>>>> have
> > >> > > >> >>>>>>> an
> > >> > > >> >>>>>>>>>> offset for the entire batch instead of individual
> > >> messages
> > >> > in
> > >> > > >> >> the
> > >> > > >> >>>>>>>> batch. In
> > >> > > >> >>>>>>>>>> that case, returning one record at a time would
> leave
> > >> the
> > >> > > >> >>>>>> SourceReader
> > >> > > >> >>>>>>>> in
> > >> > > >> >>>>>>>>>> an uncheckpointable state because they can only
> > >> checkpoint
> > >> > at
> > >> > > >> >> the
> > >> > > >> >>>>>>> batch
> > >> > > >> >>>>>>>>>> boundaries.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <
> > >> > > >> >>> trohrmann@apache.org
> > >> > > >> >>>>>>>> <ma...@apache.org>> <trohrmann@apache.org
> > >> > <mailto:
> > >> > > >> >>>>>>>> trohrmann@apache.org>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi everyone,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> thanks for drafting this FLIP. It reads very well.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Concerning Dawid's proposal, I tend to agree. The
> > >> > boundedness
> > >> > > >> >>> could
> > >> > > >> >>>>>>> come
> > >> > > >> >>>>>>>>>> from the source and tell the system how to treat the
> > >> > operator
> > >> > > >> >>>>>>>> (scheduling
> > >> > > >> >>>>>>>>>> wise). From a user's perspective it should be fine
> to
> > >> get
> > >> > > back
> > >> > > >> a
> > >> > > >> >>>>>>>> DataStream
> > >> > > >> >>>>>>>>>> when calling env.source(boundedSource) if he does
> not
> > >> need
> > >> > > >> >> special
> > >> > > >> >>>>>>>>>> operations defined on a BoundedDataStream. If he
> needs
> > >> > this,
> > >> > > >> >> then
> > >> > > >> >>>>>> one
> > >> > > >> >>>>>>>> could
> > >> > > >> >>>>>>>>>> use the method BoundedDataStream
> > >> > > >> >> env.boundedSource(boundedSource).
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> If possible, we could enforce the proper usage of
> > >> > > >> >>>>>> env.boundedSource()
> > >> > > >> >>>>>>> by
> > >> > > >> >>>>>>>>>> introducing a BoundedSource type so that one cannot
> > >> pass an
> > >> > > >> >>>>>>>>>> unbounded source to it. That way users would not be
> > >> able to
> > >> > > >> >> shoot
> > >> > > >> >>>>>>>>>> themselves in the foot.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Maybe this has already been asked before but I was
> > >> > wondering
> > >> > > >> why
> > >> > > >> >>> the
> > >> > > >> >>>>>>>>>> SourceReader interface has the method pollNext which
> > >> hands
> > >> > > the
> > >> > > >> >>>>>>>>>> responsibility of outputting elements to the
> > >> SourceReader
> > >> > > >> >>>>>>>> implementation?
> > >> > > >> >>>>>>>>>> Has this been done for backwards compatibility
> reasons
> > >> with
> > >> > > the
> > >> > > >> >>> old
> > >> > > >> >>>>>>>> source
> > >> > > >> >>>>>>>>>> interface? If not, then one could define a
> > Collection<E>
> > >> > > >> >>>>>>>> getNextRecords()
> > >> > > >> >>>>>>>>>> method which returns the currently retrieved records
> > and
> > >> > then
> > >> > > >> >> the
> > >> > > >> >>>>>>> caller
> > >> > > >> >>>>>>>>>> emits them outside of the SourceReader. That way the
> > >> > > interface
> > >> > > >> >>> would
> > >> > > >> >>>>>>> not
> > >> > > >> >>>>>>>>>> allow to implement an outputting loop where we never
> > >> hand
> > >> > > back
> > >> > > >> >>>>>> control
> > >> > > >> >>>>>>>> to
> > >> > > >> >>>>>>>>>> the caller. At the moment, this contract can be
> easily
> > >> > broken
> > >> > > >> >> and
> > >> > > >> >>> is
> > >> > > >> >>>>>>>> only
> > >> > > >> >>>>>>>>>> mentioned loosely in the JavaDocs.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Cheers,
> > >> > > >> >>>>>>>>>> Till
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <
> > >> > > >> >>> jingsonglee0@gmail.com
> > >> > > >> >>>>>>>> <ma...@gmail.com>> <
> > jingsonglee0@gmail.com
> > >> > > >> <mailto:
> > >> > > >> >>>>>>>> jingsonglee0@gmail.com>>
> > >> > > >> >>>>>>>>>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi all,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I think current design is good.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> My understanding is:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> For execution mode: bounded mode and continuous
> mode,
> > >> It's
> > >> > > >> >> totally
> > >> > > >> >>>>>>>>>> different. I don't think we have the ability to
> > >> integrate
> > >> > the
> > >> > > >> >> two
> > >> > > >> >>>>>>> models
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> at
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> present. It's about scheduling, memory, algorithms,
> > >> States,
> > >> > > >> etc.
> > >> > > >> >>> we
> > >> > > >> >>>>>>>>>> shouldn't confuse them.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> For source capabilities: only bounded, only
> > continuous,
> > >> > both
> > >> > > >> >>> bounded
> > >> > > >> >>>>>>> and
> > >> > > >> >>>>>>>>>> continuous.
> > >> > > >> >>>>>>>>>> I think Kafka is a source that can be ran both
> bounded
> > >> > > >> >>>>>>>>>> and continuous execution mode.
> > >> > > >> >>>>>>>>>> And Kafka with end offset should be ran both bounded
> > >> > > >> >>>>>>>>>> and continuous execution mode.  Using apache Beam
> with
> > >> > Flink
> > >> > > >> >>>>>> runner, I
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> used
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> to run a "bounded" Kafka in streaming mode. For our
> > >> > previous
> > >> > > >> >>>>>>> DataStream,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> it
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> is not necessarily required that the source cannot
> be
> > >> > > bounded.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> So it is my thought for Dawid's question:
> > >> > > >> >>>>>>>>>> 1.pass a bounded source to continuousSource() +1
> > >> > > >> >>>>>>>>>> 2.pass a continuous source to boundedSource() -1,
> > should
> > >> > > throw
> > >> > > >> >>>>>>>> exception.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> In StreamExecutionEnvironment, continuousSource and
> > >> > > >> >> boundedSource
> > >> > > >> >>>>>>> define
> > >> > > >> >>>>>>>>>> the execution mode. It defines a clear boundary of
> > >> > execution
> > >> > > >> >> mode.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Best,
> > >> > > >> >>>>>>>>>> Jingsong Lee
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <
> > >> imjark@gmail.com
> > >> > > >> >>> <mailto:
> > >> > > >> >>>>>>>> imjark@gmail.com>> <imjark@gmail.com <mailto:
> > >> > imjark@gmail.com
> > >> > > >>
> > >> > > >> >>>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I agree with Dawid's point that the boundedness
> > >> information
> > >> > > >> >> should
> > >> > > >> >>>>>>> come
> > >> > > >> >>>>>>>>>> from the source itself (e.g. the end timestamp), not
> > >> > through
> > >> > > >> >>>>>>>>>> env.boundedSouce()/continuousSource().
> > >> > > >> >>>>>>>>>> I think if we want to support something like
> > >> `env.source()`
> > >> > > >> that
> > >> > > >> >>>>>>> derive
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> execution mode from source,
> > >> > > `supportsBoundedness(Boundedness)`
> > >> > > >> >>>>>>>>>> method is not enough, because we don't know whether
> it
> > >> is
> > >> > > >> >> bounded
> > >> > > >> >>> or
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> not.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Best,
> > >> > > >> >>>>>>>>>> Jark
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
> > >> > > >> >>>>>> dwysakowicz@apache.org
> > >> > > >> >>>>>>>> <ma...@apache.org>> <
> > dwysakowicz@apache.org
> > >> > > >> <mailto:
> > >> > > >> >>>>>>>> dwysakowicz@apache.org>>
> > >> > > >> >>>>>>>>>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> One more thing. In the current proposal, with the
> > >> > > >> >>>>>>>>>> supportsBoundedness(Boundedness) method and the
> > >> boundedness
> > >> > > >> >> coming
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> from
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> either continuousSource or boundedSource I could not
> > >> find
> > >> > how
> > >> > > >> >> this
> > >> > > >> >>>>>>>>>> information is fed back to the SplitEnumerator.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Best,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Dawid
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On 09/12/2019 13:52, Becket Qin wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi Dawid,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks for the comments. This actually brings
> another
> > >> > > relevant
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> question
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> about what does a "bounded source" imply. I actually
> > had
> > >> > the
> > >> > > >> >> same
> > >> > > >> >>>>>>>>>> impression when I look at the Source API. Here is
> > what I
> > >> > > >> >>> understand
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> after
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> some discussion with Stephan. The bounded source has
> > the
> > >> > > >> >> following
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> impacts.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 1. API validity.
> > >> > > >> >>>>>>>>>> - A bounded source generates a bounded stream so
> some
> > >> > > >> operations
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> only
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> works for bounded records would be performed, e.g.
> > sort.
> > >> > > >> >>>>>>>>>> - To expose these bounded stream only APIs, there
> are
> > >> two
> > >> > > >> >> options:
> > >> > > >> >>>>>>>>>>       a. Add them to the DataStream API and throw
> > >> exception
> > >> > > if
> > >> > > >> a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> method
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> is
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> called on an unbounded stream.
> > >> > > >> >>>>>>>>>>       b. Create a BoundedDataStream class which is
> > >> returned
> > >> > > >> from
> > >> > > >> >>>>>>>>>> env.boundedSource(), while DataStream is returned
> from
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> env.continousSource().
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Note that this cannot be done by having single
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> env.source(theSource)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> even
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the Source has a getBoundedness() method.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 2. Scheduling
> > >> > > >> >>>>>>>>>> - A bounded source could be computed stage by stage
> > >> without
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> bringing
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> up
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> all
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the tasks at the same time.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 3. Operator behaviors
> > >> > > >> >>>>>>>>>> - A bounded source indicates the records are finite
> so
> > >> some
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> operators
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> can
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> wait until it receives all the records before it
> > starts
> > >> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> processing.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> In the above impact, only 1 is relevant to the API
> > >> design.
> > >> > > And
> > >> > > >> >> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> current
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> proposal in FLIP-27 is following 1.b.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> // boundedness depends of source property, imo this
> > >> should
> > >> > > >> >> always
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> be
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> preferred
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> In your proposal, does DataStream have bounded
> stream
> > >> only
> > >> > > >> >>> methods?
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> It
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> looks it should have, otherwise passing a bounded
> > >> Source to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> env.source()
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> would be confusing. In that case, we will
> essentially
> > do
> > >> > 1.a
> > >> > > if
> > >> > > >> >> an
> > >> > > >> >>>>>>>>>> unbounded Source is created from
> > >> > env.source(unboundedSource).
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> If we have the methods only supported for bounded
> > >> streams
> > >> > in
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> DataStream,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> it
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> seems a little weird to have a separate
> > >> BoundedDataStream
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> interface.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Am I understand it correctly?
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> dwysakowicz@apache.org <mailto:
> dwysakowicz@apache.org
> > >>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi all,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Really well written proposal and very important
> one. I
> > >> must
> > >> > > >> >> admit
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> have
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> not understood all the intricacies of it yet.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> One question I have though is about where does the
> > >> > > information
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> about
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> boundedness come from. I think in most cases it is a
> > >> > property
> > >> > > >> of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> source. As you described it might be e.g. end
> offset,
> > a
> > >> > flag
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> should
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> it
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> monitor new splits etc. I think it would be a really
> > >> nice
> > >> > use
> > >> > > >> >> case
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> be
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> able to say:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> new KafkaSource().readUntil(long timestamp),
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> which could work as an "end offset". Moreover I
> think
> > >> all
> > >> > > >> >> Bounded
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> sources
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> support continuous mode, but no intrinsically
> > continuous
> > >> > > source
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> support
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Bounded mode. If I understood the proposal correctly
> > it
> > >> > > suggest
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> boundedness sort of "comes" from the outside of the
> > >> source,
> > >> > > >> from
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> invokation of either boundedStream or
> continousSource.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I am wondering if it would make sense to actually
> > change
> > >> > the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> method
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> boolean Source#supportsBoundedness(Boundedness)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Boundedness Source#getBoundedness().
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> As for the methods #boundedSource, #continousSource,
> > >> > assuming
> > >> > > >> >> the
> > >> > > >> >>>>>>>>>> boundedness is property of the source they do not
> > affect
> > >> > how
> > >> > > >> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> enumerator
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> works, but mostly how the dag is scheduled, right? I
> > am
> > >> not
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> against
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> those
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> methods, but I think it is a very specific use case
> to
> > >> > > actually
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> override
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the property of the source. In general I would
> expect
> > >> users
> > >> > > to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> only
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> call
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> env.source(theSource), where the source tells if it
> is
> > >> > > bounded
> > >> > > >> >> or
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> not. I
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> would suggest considering following set of methods:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> // boundedness depends of source property, imo this
> > >> should
> > >> > > >> >> always
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> be
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> preferred
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> // always continous execution, whether bounded or
> > >> unbounded
> > >> > > >> >> source
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> DataStream<MyType> boundedStream =
> > >> > > >> >> env.continousSource(theSource);
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> // imo this would make sense if the
> BoundedDataStream
> > >> > > provides
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> additional features unavailable for continous mode
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> BoundedDataStream<MyType> batch =
> > >> > > env.boundedSource(theSource);
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Best,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Dawid
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks, Becket, for updating this.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I agree with moving the aspects you mentioned into
> > >> separate
> > >> > > >> >> FLIPs
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> -
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> this
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> one way becoming unwieldy in size.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> +1 to the FLIP in its current state. Its a very
> > detailed
> > >> > > >> >> write-up,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> nicely
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> done!
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <
> > >> > > >> becket.qin@gmail.com
> > >> > > >> >>>>>>>> <ma...@gmail.com>> <becket.qin@gmail.com
> > >> > <mailto:
> > >> > > >> >>>>>>>> becket.qin@gmail.com>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> becket.qin@gmail.com <ma...@gmail.com>>
> > >> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi all,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Sorry for the long belated update. I have updated
> > >> FLIP-27
> > >> > > wiki
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> page
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> with
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the latest proposals. Some noticeable changes
> include:
> > >> > > >> >>>>>>>>>> 1. A new generic communication mechanism between
> > >> > > >> SplitEnumerator
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> SourceReader.
> > >> > > >> >>>>>>>>>> 2. Some detail API method signature changes.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> We left a few things out of this FLIP and will
> address
> > >> them
> > >> > > in
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> separate
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> FLIPs. Including:
> > >> > > >> >>>>>>>>>> 1. Per split event time.
> > >> > > >> >>>>>>>>>> 2. Event time alignment.
> > >> > > >> >>>>>>>>>> 3. Fine grained failover for SplitEnumerator
> failure.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Please let us know if you have any question.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <
> > >> > > sewen@apache.org
> > >> > > >> >>>>>>> <mailto:
> > >> > > >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:
> > >> > sewen@apache.org
> > >> > > >>
> > >> > > >> <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi  Łukasz!
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Becket and me are working hard on figuring out the
> > last
> > >> > > details
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> implementing the first PoC. We would update the FLIP
> > >> > > hopefully
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> next
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> week.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> There is a fair chance that a first version of this
> > >> will be
> > >> > > in
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 1.10,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> but
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> think it will take another release to battle test it
> > and
> > >> > > >> migrate
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> connectors.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Best,
> > >> > > >> >>>>>>>>>> Stephan
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz
> Jędrzejewski <
> > >> > > >> >> ljd@touk.pl
> > >> > > >> >>>>>>>> <ma...@touk.pl>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> ljd@touk.pl <ma...@touk.pl>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> This proposal looks very promising for us. Do you
> have
> > >> any
> > >> > > >> plans
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> in
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> which
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Flink release it is going to be released? We are
> > >> thinking
> > >> > on
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> using a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Data
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Set API for our future use cases but on the other
> hand
> > >> Data
> > >> > > Set
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> API
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> is
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> going to be deprecated so using proposed bounded
> data
> > >> > streams
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> solution
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> could be more viable in the long term.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks,
> > >> > > >> >>>>>>>>>> Łukasz
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On 2019/10/01 15:48:03, Thomas Weise <
> > >> > thomas.weise@gmail.com
> > >> > > >> >>>>>> <mailto:
> > >> > > >> >>>>>>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com
> > <mailto:
> > >> > > >> >>>>>>>> thomas.weise@gmail.com>> <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> thomas.weise@gmail.com <mailto:
> thomas.weise@gmail.com
> > >>
> > >> > > wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks for putting together this proposal!
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I see that the "Per Split Event Time" and "Event
> Time
> > >> > > >> Alignment"
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> sections
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> are still TBD.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> It would probably be good to flesh those out a bit
> > >> before
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> proceeding
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> too
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> far
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> as the event time alignment will probably influence
> > the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> interaction
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> with
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the split reader, specifically ReaderStatus
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> emitNext(SourceOutput<E>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> output).
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> We currently have only one implementation for event
> > time
> > >> > > >> >> alignment
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> in
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Kinesis consumer. The synchronization in that case
> > takes
> > >> > > place
> > >> > > >> >> as
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> last
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> step before records are emitted downstream
> > >> (RecordEmitter).
> > >> > > >> With
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> currently proposed interfaces, the equivalent can be
> > >> > > >> implemented
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> in
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> reader loop, although note that in the Kinesis
> > consumer
> > >> the
> > >> > > per
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> shard
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> threads push records.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Synchronization has not been implemented for the
> Kafka
> > >> > > consumer
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> yet.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
> > >> > > >> >>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> When I looked at it, I realized that the
> > implementation
> > >> > will
> > >> > > >> >> look
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> quite
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> different
> > >> > > >> >>>>>>>>>> from Kinesis because it needs to take place in the
> > pull
> > >> > part,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> where
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> records
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> are taken from the Kafka client. Due to the
> > >> multiplexing it
> > >> > > >> >> cannot
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> be
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> done
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> by blocking the split thread like it currently works
> > for
> > >> > > >> >> Kinesis.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Reading
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> from individual Kafka partitions needs to be
> > controlled
> > >> via
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> pause/resume
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> on the Kafka client.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> To take on that responsibility the split thread
> would
> > >> need
> > >> > to
> > >> > > >> be
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> aware
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>> watermarks or at least whether it should or should
> not
> > >> > > continue
> > >> > > >> >> to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> consume
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> a given split and this may require a different
> > >> SourceReader
> > >> > > or
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> SourceOutput
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> interface.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks,
> > >> > > >> >>>>>>>>>> Thomas
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <
> > >> > mmyy1110@gmail.com
> > >> > > >> >>>>>> <mailto:
> > >> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > >> > > >> >> mmyy1110@gmail.com
> > >> > > >> >>>>>
> > >> > > >> >>>>>> <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi Stephan,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thank you for feedback!
> > >> > > >> >>>>>>>>>> Will take a look at your branch before public
> > >> discussing.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <
> > >> > > >> sewen@apache.org
> > >> > > >> >>>>>>>> <ma...@apache.org>> <sewen@apache.org <mailto:
> > >> > > >> >>> sewen@apache.org
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi Biao!
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks for reviving this. I would like to join this
> > >> > > discussion,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> but
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> am
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> quite occupied with the 1.9 release, so can we maybe
> > >> pause
> > >> > > this
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> discussion
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> for a week or so?
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> In the meantime I can share some suggestion based on
> > >> prior
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> experiments:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> How to do watermarks / timestamp extractors in a
> > simpler
> > >> > and
> > >> > > >> >> more
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> flexible
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> way. I think that part is quite promising should be
> > >> part of
> > >> > > the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> new
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> source
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> interface.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > >> > > >> >>>>>>>> <
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > >> > > >> >>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > >> > > >> >>>>>>>> <
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > >> > > >> >>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Some experiments on how to build the source reader
> and
> > >> its
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> library
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> for
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> common threading/split patterns:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > >> > > >> >>>>>>>> <
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > >> > > >> >>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Best,
> > >> > > >> >>>>>>>>>> Stephan
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <
> > >> > > mmyy1110@gmail.com
> > >> > > >> >>>>>>> <mailto:
> > >> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > >> > > >> >> mmyy1110@gmail.com
> > >> > > >> >>>>>
> > >> > > >> >>>>>> <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi devs,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Since 1.9 is nearly released, I think we could get
> > back
> > >> to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> FLIP-27.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> believe it should be included in 1.10.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> There are so many things mentioned in document of
> > >> FLIP-27.
> > >> > > [1]
> > >> > > >> I
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> think
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> we'd better discuss them separately. However the
> wiki
> > is
> > >> > not
> > >> > > a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> good
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> place
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> to discuss. I wrote google doc about SplitReader API
> > >> which
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> misses
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> some
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> details in the document. [2]
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 1.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > >> > > >> >>>>>>>> <
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > >> > > >> >>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 2.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > >> > > >> >>>>>>>> <
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > >> > > >> >>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <
> > >> > mmyy1110@gmail.com
> > >> > > >> >>>>>> <mailto:
> > >> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > >> > > >> >> mmyy1110@gmail.com
> > >> > > >> >>>>>
> > >> > > >> >>>>>> <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi Steven,
> > >> > > >> >>>>>>>>>> Thank you for the feedback. Please take a look at
> the
> > >> > > document
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> FLIP-27
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >> > > >> >>>>>>>> <
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >> > > >> >>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> which
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> is updated recently. A lot of details of enumerator
> > were
> > >> > > added
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> in
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> this
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> document. I think it would help.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Steven Wu <stevenz3wu@gmail.com <mailto:
> > >> > stevenz3wu@gmail.com
> > >> > > >>
> > >> > > >> >> <
> > >> > > >> >>>>>>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
> > >> > > >> >>>>>>> stevenz3wu@gmail.com
> > >> > > >> >>>>>>>> <ma...@gmail.com>> <stevenz3wu@gmail.com
> > >> > <mailto:
> > >> > > >> >>>>>>>> stevenz3wu@gmail.com>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 于2019年3月28日周四
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 下午12:52写道:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> This proposal mentioned that SplitEnumerator might
> run
> > >> on
> > >> > the
> > >> > > >> >>>>>>>>>> JobManager or
> > >> > > >> >>>>>>>>>> in a single task on a TaskManager.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> if enumerator is a single task on a taskmanager,
> then
> > >> the
> > >> > job
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> DAG
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> can
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> never
> > >> > > >> >>>>>>>>>> been embarrassingly parallel anymore. That will
> > nullify
> > >> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> leverage
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> fine-grained recovery for embarrassingly parallel
> > jobs.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> It's not clear to me what's the implication of
> running
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> enumerator
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> on
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> jobmanager. So I will leave that out for now.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <
> > >> > mmyy1110@gmail.com
> > >> > > >> >>>>>> <mailto:
> > >> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > >> > > >> >> mmyy1110@gmail.com
> > >> > > >> >>>>>
> > >> > > >> >>>>>> <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi Stephan & Piotrek,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thank you for feedback.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> It seems that there are a lot of things to do in
> > >> community.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> am
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> just
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> afraid that this discussion may be forgotten since
> > >> there so
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> many
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> proposals
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> recently.
> > >> > > >> >>>>>>>>>> Anyway, wish to see the split topics soon :)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:
> > >> > > >> >>> piotr@da-platform.com
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>> <
> > >> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>>
> <
> > >> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>>
> <
> > >> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 于2019年1月24日周四
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 下午8:21写道:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi Biao!
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> This discussion was stalled because of preparations
> > for
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> open
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> sourcing
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> & merging Blink. I think before creating the tickets
> > we
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> should
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> split this
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> discussion into topics/areas outlined by Stephan and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> create
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Flips
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> for
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I think there is no chance for this to be completed
> in
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> couple
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> remaining
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> weeks/1 month before 1.8 feature freeze, however it
> > >> would
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> be
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> good
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> to aim
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> with those changes for 1.9.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Piotrek
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <
> > mmyy1110@gmail.com
> > >> > > >> <mailto:
> > >> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > >> > > >> >> mmyy1110@gmail.com
> > >> > > >> >>>>>
> > >> > > >> >>>>>> <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi community,
> > >> > > >> >>>>>>>>>> The summary of Stephan makes a lot sense to me. It
> is
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> much
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> clearer
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> indeed
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> after splitting the complex topic into small ones.
> > >> > > >> >>>>>>>>>> I was wondering is there any detail plan for next
> > step?
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> If
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> not,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> would
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> like to push this thing forward by creating some
> JIRA
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> issues.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Another question is that should version 1.8 include
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> these
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> features?
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Stephan Ewen <sewen@apache.org <mailto:
> > sewen@apache.org
> > >> >>
> > >> > <
> > >> > > >> >>>>>>>> sewen@apache.org <ma...@apache.org>> <
> > >> > sewen@apache.org
> > >> > > >> >>>> <mailto:
> > >> > > >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:
> > >> > sewen@apache.org
> > >> > > >>
> > >> > > >> >>>>>>>> 于2018年12月1日周六
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 上午4:20写道:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks everyone for the lively discussion. Let me
> try
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> summarize
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> where I
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> see convergence in the discussion and open issues.
> > >> > > >> >>>>>>>>>> I'll try to group this by design aspect of the
> source.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Please
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> let me
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> know
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> if I got things wrong or missed something crucial
> > here.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> For issues 1-3, if the below reflects the state of
> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> discussion, I
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> would
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> try and update the FLIP in the next days.
> > >> > > >> >>>>>>>>>> For the remaining ones we need more discussion.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> I would suggest to fork each of these aspects into a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> separate
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> mail
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> thread,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> or will loose sight of the individual aspects.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> *(1) Separation of Split Enumerator and Split
> Reader*
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - All seem to agree this is a good thing
> > >> > > >> >>>>>>>>>> - Split Enumerator could in the end live on
> JobManager
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> (and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> assign
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> splits
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> via RPC) or in a task (and assign splits via data
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> streams)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - this discussion is orthogonal and should come
> later,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> when
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> interface
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> is agreed upon.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> *(2) Split Readers for one or more splits*
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Discussion seems to agree that we need to support
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> one
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> reader
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> possibly handles multiple splits concurrently.
> > >> > > >> >>>>>>>>>> - The requirement comes from sources where one
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> poll()-style
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> call
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> fetches
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> data from different splits / partitions
> > >> > > >> >>>>>>>>>>     --> example sources that require that would be
> for
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> example
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Kafka,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Pravega, Pulsar
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Could have one split reader per source, or
> multiple
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> split
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> readers
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> share the "poll()" function
> > >> > > >> >>>>>>>>>> - To not make it too complicated, we can start with
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> thinking
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> about
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> one
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> split reader for all splits initially and see if
> that
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> covers
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> all
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> requirements
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> *(3) Threading model of the Split Reader*
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Most active part of the discussion ;-)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - A non-blocking way for Flink's task code to
> interact
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> with
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> source
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> is
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> needed in order to a task runtime code based on a
> > >> > > >> >>>>>>>>>> single-threaded/actor-style task design
> > >> > > >> >>>>>>>>>>     --> I personally am a big proponent of that, it
> > will
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> help
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> with
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> well-behaved checkpoints, efficiency, and simpler
> yet
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> more
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> robust
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> runtime
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> code
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Users care about simple abstraction, so as a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> subclass
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> SplitReader
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> (non-blocking / async) we need to have a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> BlockingSplitReader
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> which
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> will
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> form the basis of most source implementations.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> BlockingSplitReader
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> lets
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> users do blocking simple poll() calls.
> > >> > > >> >>>>>>>>>> - The BlockingSplitReader would spawn a thread (or
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> more)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> thread(s) can make blocking calls and hand over data
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> buffers
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> via
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> blocking
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> queue
> > >> > > >> >>>>>>>>>> - This should allow us to cover both, a fully async
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> runtime,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> simple
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> blocking interface for users.
> > >> > > >> >>>>>>>>>> - This is actually very similar to how the Kafka
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> connectors
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> work.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Kafka
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 9+ with one thread, Kafka 8 with multiple threads
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - On the base SplitReader (the async one), the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> non-blocking
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> method
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> gets the next chunk of data would signal data
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> availability
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> via
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> CompletableFuture, because that gives the best
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> flexibility
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> (can
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> await
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> completion or register notification handlers).
> > >> > > >> >>>>>>>>>> - The source task would register a "thenHandle()"
> (or
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> similar)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> on the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> future to put a "take next data" task into the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> actor-style
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> mailbox
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> *(4) Split Enumeration and Assignment*
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Splits may be generated lazily, both in cases
> where
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> there
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> is a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> limited
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> number of splits (but very many), or splits are
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> discovered
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> over
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> time
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Assignment should also be lazy, to get better load
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> balancing
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Assignment needs support locality preferences
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Possible design based on discussion so far:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>     --> SplitReader has a method
> > "addSplits(SplitT...)"
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> add
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> one or
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> more
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> splits. Some split readers might assume they have
> only
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> one
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> split
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> ever,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> concurrently, others assume multiple splits. (Note:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> idea
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> behind
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> being
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> able
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> to add multiple splits at the same time is to ease
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> startup
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> where
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> multiple
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> splits may be assigned instantly.)
> > >> > > >> >>>>>>>>>>     --> SplitReader has a context object on which it
> > can
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> call
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> indicate
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> when
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> splits are completed. The enumerator gets that
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> notification and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> can
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> use
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> decide when to assign new splits. This should help
> > both
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> in
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> cases
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> sources
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that take splits lazily (file readers) and in case
> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> source
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> needs to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> preserve a partial order between splits (Kinesis,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Pravega,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Pulsar may
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> need
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that).
> > >> > > >> >>>>>>>>>>     --> SplitEnumerator gets notification when
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> SplitReaders
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> start
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> when
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> they finish splits. They can decide at that moment
> to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> push
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> more
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> splits
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that reader
> > >> > > >> >>>>>>>>>>     --> The SplitEnumerator should probably be aware
> > of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> source
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> parallelism, to build its initial distribution.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Open question: Should the source expose something
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> like
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> "host
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> preferences", so that yarn/mesos/k8s can take this
> > into
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> account
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> when
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> selecting a node to start a TM on?
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> *(5) Watermarks and event time alignment*
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Watermark generation, as well as idleness, needs
> to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> be
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> per
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> split
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> (like
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> currently in the Kafka Source, per partition)
> > >> > > >> >>>>>>>>>> - It is desirable to support optional
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> event-time-alignment,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> meaning
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> splits that are ahead are back-pressured or
> > temporarily
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> unsubscribed
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - I think i would be desirable to encapsulate
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> watermark
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> generation
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> logic
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> in watermark generators, for a separation of
> concerns.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> The
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> watermark
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> generators should run per split.
> > >> > > >> >>>>>>>>>> - Using watermark generators would also help with
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> another
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> problem of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> suggested interface, namely supporting non-periodic
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> watermarks
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> efficiently.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Need a way to "dispatch" next record to different
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> watermark
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> generators
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Need a way to tell SplitReader to "suspend" a
> split
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> until a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> certain
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> watermark is reached (event time backpressure)
> > >> > > >> >>>>>>>>>> - This would in fact be not needed (and thus
> simpler)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> if
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> we
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> had
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> SplitReader per split and may be a reason to re-open
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> discussion
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> *(6) Watermarks across splits and in the Split
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Enumerator*
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - The split enumerator may need some watermark
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> awareness,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> which
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> should
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> be
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> purely based on split metadata (like create
> timestamp
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> file
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> splits)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - If there are still more splits with overlapping
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> event
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> time
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> range
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> for
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> split reader, then that split reader should not
> > advance
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> watermark
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> within the split beyond the overlap boundary.
> > Otherwise
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> future
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> splits
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> will
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> produce late data.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - One way to approach this could be that the split
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> enumerator
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> may
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> send
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> watermarks to the readers, and the readers cannot
> emit
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> watermarks
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> beyond
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that received watermark.
> > >> > > >> >>>>>>>>>> - Many split enumerators would simply immediately
> send
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Long.MAX
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> out
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> leave the progress purely to the split readers.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - For event-time alignment / split back pressure,
> this
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> begs
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> question
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> how we can avoid deadlocks that may arise when
> splits
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> are
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> suspended
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> for
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> event time back pressure,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> *(7) Batch and streaming Unification*
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Functionality wise, the above design should
> support
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> both
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Batch often (mostly) does not care about reading
> "in
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> order"
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> generating watermarks
> > >> > > >> >>>>>>>>>>     --> Might use different enumerator logic that is
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> more
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> locality
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> aware
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and ignores event time order
> > >> > > >> >>>>>>>>>>     --> Does not generate watermarks
> > >> > > >> >>>>>>>>>> - Would be great if bounded sources could be
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> identified
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> at
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> compile
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> time,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> so that "env.addBoundedSource(...)" is type safe and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> can
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> return a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> "BoundedDataStream".
> > >> > > >> >>>>>>>>>> - Possible to defer this discussion until later
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> *Miscellaneous Comments*
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - Should the source have a TypeInformation for the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> produced
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> type,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> instead
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> of a serializer? We need a type information in the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> stream
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> anyways, and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> can
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> derive the serializer from that. Plus, creating the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> serializer
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> should
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> respect the ExecutionConfig.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> - The TypeSerializer interface is very powerful but
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> also
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> not
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> easy to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> implement. Its purpose is to handle data super
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> efficiently,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> support
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> flexible ways of evolution, etc.
> > >> > > >> >>>>>>>>>> For metadata I would suggest to look at the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> SimpleVersionedSerializer
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> instead, which is used for example for checkpoint
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> master
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> hooks,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> or for
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> streaming file sink. I think that is is a good match
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> for
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> cases
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> where
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> we
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> do
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> not need more than ser/deser (no copy, etc.) and
> don't
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> need to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> push
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> versioning out of the serialization paths for best
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> performance
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> (as in
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> TypeSerializer)
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> k.kloudas@data-artisans.com>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> wrote:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Hi Biao,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Thanks for the answer!
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> So given the multi-threaded readers, now we have as
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> open
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> questions:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 1) How do we let the checkpoints pass through our
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> multi-threaded
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> reader
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> operator?
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 2) Do we have separate reader and source operators
> or
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> not? In
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> strategy
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that has a separate source, the source operator has
> a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> parallelism of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 1
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> is responsible for split recovery only.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> For the first one, given also the constraints
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> (blocking,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> finite
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> queues,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> etc), I do not have an answer yet.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> For the 2nd, I think that we should go with separate
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> operators
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> for
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> source and the readers, for the following reasons:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 1) This is more aligned with a potential future
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> improvement
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> where the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> split
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> discovery becomes a responsibility of the JobManager
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> readers are
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> pooling more work from the JM.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> 2) The source is going to be the "single point of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> truth".
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> It
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> will
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> know
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> what
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> has been processed and what not. If the source and
> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> readers
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> are a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> single
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> operator with parallelism > 1, or in general, if the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> split
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> discovery
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> is
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> done by each task individually, then:
> > >> > > >> >>>>>>>>>>    i) we have to have a deterministic scheme for
> each
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> reader to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> assign
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> splits to itself (e.g. mod subtaskId). This is not
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> necessarily
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> trivial
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> for
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> all sources.
> > >> > > >> >>>>>>>>>>    ii) each reader would have to keep a copy of all
> > its
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> processed
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> slpits
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>    iii) the state has to be a union state with a
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> non-trivial
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> merging
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> logic
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> in order to support rescaling.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Two additional points that you raised above:
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> i) The point that you raised that we need to keep
> all
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> splits
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> (processed
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> not-processed) I think is a bit of a strong
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> requirement.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> This
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> would
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> imply
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that for infinite sources the state will grow
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> indefinitely.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> This is
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> problem
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> is even more pronounced if we do not have a single
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> source
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> assigns
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> splits to readers, as each reader will have its own
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> copy
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> of
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> state.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> ii) it is true that for finite sources we need to
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> somehow
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> not
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> close
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> readers when the source/split discoverer finishes.
> The
> > >> > > >> >>>>>>>>>> ContinuousFileReaderOperator has a work-around for
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> that.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> It is
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> not
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> elegant,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> and checkpoints are not emitted after closing the
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> source,
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> but
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> this, I
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> believe, is a bigger problem which requires more
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> changes
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> than
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> just
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> refactoring the source interface.
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> Cheers,
> > >> > > >> >>>>>>>>>> Kostas
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>>
> > >> > > >> >>>>>>>>>> --
> > >> > > >> >>>>>>>>>> Best, Jingsong Lee
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>>
> > >> > > >> >>>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>>>
> > >> > > >> >>>>>> --
> > >> > > >> >>>>>> Best, Jingsong Lee
> > >> > > >> >>>>>>
> > >> > > >> >>>>>
> > >> > > >> >>>>
> > >> > > >> >>>>
> > >> > > >> >>>
> > >> > > >> >>
> > >> > > >> >
> > >> > > >>
> > >> > > >>
> > >> > >
> > >> >
> > >>
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Steven Wu <st...@gmail.com>.
Becket, is FLIP-27 still on track to be released in 1.10?

On Tue, Jan 7, 2020 at 7:04 PM Becket Qin <be...@gmail.com> wrote:

> Hi folks,
>
> Happy new year!
>
> Stephan and I chatted offline yesterday. After reading the email thread
> again, I found that I have misunderstood Dawid's original proposal
> regarding the behavior of env.source(BoundedSource) and had an incorrect
> impression about the behavior of java covariant return type.
> Anyways, I agree what Dawid originally proposed makes sense, which is the
> following API:
>
> // Return a BoundedDataStream instance if the source is bounded.
> // Return a DataStream instance if the source is unbounded.
> DataStream env.source(Source);
>
> // Throws exception if the source is unbounded.
> // Used when users knows the source is bounded at programming time.
> BoundedDataStream env.boundedSource(Source);
>
> A BoundedDataStream only runs in batch execution mode.
> A DataStream only runs in streaming execution mode.
>
> To run a bounded source in streaming execution mode, one would do the
> following:
>
> // Return a DataStream instance with a source that will stop at some point;
> DataStream env.source(SourceUtils.asUnbounded(myBoundedSource));
>
> I'll update the FLIP wiki and resume the vote if there is no further
> concerns.
>
> Apologies for the misunderstanding and thanks for all the patient
> discussions.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
> On Mon, Dec 23, 2019 at 8:00 AM Becket Qin <be...@gmail.com> wrote:
>
> > Hi Steven,
> >
> > I think the current proposal is what you mentioned - a Kafka source that
> > can be constructed in either BOUNDED or UNBOUNDED mode. And Flink can get
> > the boundedness by invoking getBoundedness().
> >
> > So one can create a Kafka source by doing something like the following:
> >
> > new KafkaSource().startOffset(),endOffset(); // A bounded instance.
> > new KafkaSource().startOffset(); // An unbounded instance.
> >
> > If users want to have an UNBOUNDED Kafka source that stops at some point.
> > They can wrap the BOUNDED Kafka source like below:
> >
> > SourceUtils.asUnbounded(new KafkaSource.startOffset().endOffset());
> >
> > The wrapped source would be an unbounded Kafka source that stops at the
> > end offset.
> >
> > Does that make sense?
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > On Fri, Dec 20, 2019 at 1:31 PM Jark Wu <im...@gmail.com> wrote:
> >
> >> Hi,
> >>
> >> First of all, I think it is not called "UNBOUNDED", according to the
> >> FLIP-27, it is called "CONTINUOUS_UNBOUNDED".
> >> And from the description of the Boundedness in the FLIP-27[1] declares
> >> clearly what Becket and I think.
> >>
> >> public enum Boundedness {
> >>
> >>     /**
> >>      * A bounded source processes the data that is currently available
> and
> >> will end after that.
> >>      *
> >>      * <p>When a source produces a bounded stream, the runtime may
> >> activate
> >> additional optimizations
> >>      * that are suitable only for bounded input. Incorrectly producing
> >> unbounded data when the source
> >>      * is set to produce a bounded stream will often result in programs
> >> that do not output any results
> >>      * and may eventually fail due to runtime errors (out of memory or
> >> storage).
> >>      */
> >>     BOUNDED,
> >>
> >>     /**
> >>      * A continuous unbounded source continuously processes all data as
> it
> >> comes.
> >>      *
> >>      * <p>The source may run forever (until the program is terminated)
> or
> >> might actually end at some point,
> >>      * based on some source-specific conditions. Because that is not
> >> transparent to the runtime,
> >>      * the runtime will use an execution mode for continuous unbounded
> >> streams whenever this mode
> >>      * is chosen.
> >>      */
> >>     CONTINUOUS_UNBOUNDED
> >> }
> >>
> >> Best,
> >> Jark
> >>
> >> [1]:
> >>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface#FLIP-27:RefactorSourceInterface-Source
> >>
> >>
> >>
> >> On Fri, 20 Dec 2019 at 12:55, Steven Wu <st...@gmail.com> wrote:
> >>
> >> > Becket,
> >> >
> >> > Regarding "UNBOUNDED source that stops at some point", I found it
> >> difficult
> >> > to grasp what UNBOUNDED really mean.
> >> >
> >> > If we want to use Kafka source with an end/stop time, I guess you call
> >> it
> >> > UNBOUNDED kafka source that stops (aka BOUNDED-streaming). The
> >> > terminology is a little confusing to me. Maybe BOUNDED/UNBOUNDED
> >> shouldn't
> >> > be used to categorize source. Just call it Kafka source and it can run
> >> in
> >> > either BOUNDED or UNBOUNDED mode.
> >> >
> >> > Thanks,
> >> > Steven
> >> >
> >> > On Thu, Dec 19, 2019 at 7:02 PM Becket Qin <be...@gmail.com>
> >> wrote:
> >> >
> >> > > I had an offline chat with Jark, and here are some more thoughts:
> >> > >
> >> > > 1. From SQL perspective, BOUNDED source leads to the batch execution
> >> > mode,
> >> > > UNBOUNDED source leads to the streaming execution mode.
> >> > > 2. The semantic of UNBOUNDED source is may or may not stop. The
> >> semantic
> >> > of
> >> > > BOUNDED source is will stop.
> >> > > 3. The semantic of DataStream is may or may not terminate. The
> >> semantic
> >> > of
> >> > > BoundedDataStream is will terminate.
> >> > >
> >> > > Given that, option 3 seems a better option because:
> >> > > 1. SQL already has strict binding between Boundedness and execution
> >> mode.
> >> > > Letting DataStream be consistent would be good.
> >> > > 2. The semantic of UNBOUNDED source is exactly the same as
> >> DataStream. So
> >> > > we should avoid breaking such semantic, i.e. turning some DataStream
> >> from
> >> > > "may or may not terminate" to "will terminate".
> >> > >
> >> > > For case where users want BOUNDED-streaming combination, they can
> >> simply
> >> > > use an UNBOUNDED source that stops at some point. We can even
> provide
> >> a
> >> > > simple wrapper to wrap a BOUNDED source as an UNBOUNDED source if
> that
> >> > > helps. But API wise, option 3 seems telling a pretty good whole
> story.
> >> > >
> >> > > Thanks,
> >> > >
> >> > > Jiangjie (Becket) Qin
> >> > >
> >> > >
> >> > >
> >> > >
> >> > > On Thu, Dec 19, 2019 at 10:30 PM Becket Qin <be...@gmail.com>
> >> > wrote:
> >> > >
> >> > > > Hi Timo,
> >> > > >
> >> > > > Bounded is just a special case of unbounded and every bounded
> source
> >> > can
> >> > > >> also be treated as an unbounded source. This would unify the API
> if
> >> > > >> people don't need a bounded operation.
> >> > > >
> >> > > >
> >> > > > With option 3 users can still get a unified API with something
> like
> >> > > below:
> >> > > >
> >> > > > DataStream boundedStream = env.boundedSource(boundedSource);
> >> > > > DataStream unboundedStream = env.source(unboundedSource);
> >> > > >
> >> > > > So in both cases, users can still use a unified DataStream without
> >> > > > touching the bounded stream only methods.
> >> > > > By "unify the API if people don't need the bounded operation". Do
> >> you
> >> > > > expect a DataStream with a Bounded source to have the batch
> >> operators
> >> > and
> >> > > > scheduler settings as well?
> >> > > >
> >> > > >
> >> > > > If we allow DataStream from BOUNDED source, we will essentially
> pick
> >> > > "*modified
> >> > > > option 2*".
> >> > > >
> >> > > > // The source is either bounded or unbounded, but only unbounded
> >> > > >> operations could be performed on the returned DataStream.
> >> > > >> DataStream<Type> dataStream = env.source(someSource);
> >> > > >
> >> > > >
> >> > > >> // The source must be a bounded source, otherwise exception is
> >> thrown.
> >> > > >> BoundedDataStream<Type> boundedDataStream =
> >> > > >> env.boundedSource(boundedSource);
> >> > > >
> >> > > >
> >> > > >
> >> > > > // Add the following method to DataStream
> >> > > >
> >> > > > Boundedness DataStream#getBoundedness();
> >> > > >
> >> > > >
> >> > > > From pure logical perspective, Boundedness and runtime settings
> >> > > > (Stream/Batch) are two orthogonal dimensions. And are specified in
> >> the
> >> > > > following way.
> >> > > >
> >> > > > *Boundedness* - defined by the source: BOUNDED / UNBOUNDED.
> >> > > > *Running mode* - defined by the API class: DataStream (Streaming
> >> mode)
> >> > /
> >> > > > BoundedDataStream (batch mode).
> >> > > >
> >> > > > Excluding the UNBOUNDED-batch combination, the "*modified option
> 2"*
> >> > > > covers the rest three combination. Compared with "*modified option
> >> 2*",
> >> > > > the main benefit of option 3 is its simplicity and clearness, by
> >> tying
> >> > > > boundedness to running mode and giving up BOUNDED-streaming
> >> > combination.
> >> > > >
> >> > > > Just to be clear, I am fine with either option. But I would like
> to
> >> > > > understand a bit more about the bounded-streaming use case and
> when
> >> > users
> >> > > > would prefer this over bounded-batch case, and whether the added
> >> value
> >> > > > justifies the additional complexity in the API. Two cases I can
> >> think
> >> > of
> >> > > > are:
> >> > > > 1. The records in DataStream will be processed in order, while
> >> > > > BoundedDataStream processes records without order guarantee.
> >> > > > 2. DataStream emits intermediate results when processing a finite
> >> > > dataset,
> >> > > > while BoundedDataStream only emit the final result. In any case,
> it
> >> > could
> >> > > > be supported by an UNBOUNDED source stopping at some point.
> >> > > >
> >> > > > Case 1 is actually misleading because DataStream in general
> doesn't
> >> > > really
> >> > > > support in-order process.
> >> > > > Case 2 seems a rare use case because the instantaneous
> intermediate
> >> > > result
> >> > > > seems difficult to reason about. In any case, this can be
> supported
> >> by
> >> > an
> >> > > > UNBOUNDED source that stops at some point.
> >> > > >
> >> > > > Is there other use cases for bounded-streaming combination I
> >> missed? I
> >> > am
> >> > > > a little hesitating to put the testing requirement here because
> >> ideally
> >> > > I'd
> >> > > > avoid having public APIs for testing purpose only. And this could
> be
> >> > > > resolved by having a UNBOUNDED source stopping at some point as
> >> well.
> >> > > >
> >> > > > Sorry for the long discussion, but I would really like to make an
> >> API
> >> > > > decision after knowing all the pros and cons.
> >> > > >
> >> > > > Thanks,
> >> > > >
> >> > > > Jiangjie (Becket) Qin
> >> > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > > On Thu, Dec 19, 2019 at 6:19 PM Timo Walther <tw...@apache.org>
> >> > wrote:
> >> > > >
> >> > > >> Hi Becket,
> >> > > >>
> >> > > >> regarding *Option 3* I think we can relax the constraints for
> >> > > >> env.source():
> >> > > >>
> >> > > >> // MySource can be bounded or unbounded
> >> > > >> DataStream<Type> dataStream = env.source(mySource);
> >> > > >>
> >> > > >> // MySource must be bounded, otherwise throws exception.
> >> > > >> BoundedDataStream<Type> boundedDataStream =
> >> > env.boundedSource(mySource);
> >> > > >>
> >> > > >> Bounded is just a special case of unbounded and every bounded
> >> source
> >> > can
> >> > > >> also be treated as an unbounded source. This would unify the API
> if
> >> > > >> people don't need a bounded operation. It also addresses Jark's
> >> > > concerns.
> >> > > >>
> >> > > >> Regards,
> >> > > >> Timo
> >> > > >>
> >> > > >>
> >> > > >> On 18.12.19 14:16, Becket Qin wrote:
> >> > > >> > Hi Jark,
> >> > > >> >
> >> > > >> > Please see the reply below:
> >> > > >> >
> >> > > >> > Regarding to option#3, my concern is that if we don't support
> >> > > streaming
> >> > > >> >> mode for bounded source,
> >> > > >> >> how could we create a testing source for streaming mode?
> >> Currently,
> >> > > >> all the
> >> > > >> >> testing source for streaming
> >> > > >> >> are bounded, so that the integration test will finish finally.
> >> > > >> >
> >> > > >> >
> >> > > >> > An UNBOUNDED source does not mean it will never stops. It
> simply
> >> > > >> indicates
> >> > > >> > that the source *may* run forever, so the runtime needs to be
> >> > prepared
> >> > > >> for
> >> > > >> > that, but the task may still stop at some point when it hits
> some
> >> > > >> > source-specific condition. So an UNBOUNDED testing source can
> >> still
> >> > > >> stop at
> >> > > >> > some point if needed.
> >> > > >> >
> >> > > >> > Regarding to Source#getRecordOrder(), could we have a implicit
> >> > > contract
> >> > > >> >> that unbounded source should
> >> > > >> >> already read in order (i.e. reading partitions in parallel),
> for
> >> > > >> bounded
> >> > > >> >> source the order is not mandatory.
> >> > > >> >
> >> > > >> >
> >> > > >> >
> >> > > >> >> This is also the behaviors of the current sources.
> >> > > >> >
> >> > > >> > 1) a source can't guarantee it reads in strict order, because
> the
> >> > > >> producer
> >> > > >> >> may produce data not in order.
> >> > > >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can
> >> > reorder
> >> > > >> data.
> >> > > >> >
> >> > > >> >
> >> > > >> > It is true that sometimes the source cannot guarantee the
> record
> >> > > order,
> >> > > >> but
> >> > > >> > sometimes it can. Right now, even for stream processing, there
> >> is no
> >> > > >> > processing order guarantee. For example, a join operator may
> >> emit a
> >> > > >> later
> >> > > >> > record which successfully found a join match earlier.
> >> > > >> > Event order is one of the most important requirements for event
> >> > > >> processing,
> >> > > >> > a clear order guarantee would be necessary. That said, I agree
> >> that
> >> > > >> right
> >> > > >> > now even if the sources provide the record order requirement,
> the
> >> > > >> runtime
> >> > > >> > is not able to guarantee that out of the box. So I am OK if we
> >> add
> >> > the
> >> > > >> > record order to the Source later. But we should avoid
> misleading
> >> > users
> >> > > >> to
> >> > > >> > make them think the processing order is guaranteed when using
> the
> >> > > >> unbounded
> >> > > >> > runtime.
> >> > > >> >
> >> > > >> > Thanks,
> >> > > >> >
> >> > > >> > Jiangjie (Becket) Qin
> >> > > >> >
> >> > > >> > On Wed, Dec 18, 2019 at 10:29 AM Jark Wu <im...@gmail.com>
> >> wrote:
> >> > > >> >
> >> > > >> >> Hi Becket,
> >> > > >> >>
> >> > > >> >> That's great we have reached a consensus on
> >> > Source#getBoundedness().
> >> > > >> >>
> >> > > >> >> Regarding to option#3, my concern is that if we don't support
> >> > > streaming
> >> > > >> >> mode for bounded source,
> >> > > >> >> how could we create a testing source for streaming mode?
> >> Currently,
> >> > > >> all the
> >> > > >> >> testing source for streaming
> >> > > >> >> are bounded, so that the integration test will finish finally.
> >> > > >> >>
> >> > > >> >> Regarding to Source#getRecordOrder(), could we have a implicit
> >> > > contract
> >> > > >> >> that unbounded source should
> >> > > >> >> already read in order (i.e. reading partitions in parallel),
> for
> >> > > >> bounded
> >> > > >> >> source the order is not mandatory.
> >> > > >> >> This is also the behaviors of the current sources.
> >> > > >> >> 1) a source can't guarantee it reads in strict order, because
> >> the
> >> > > >> producer
> >> > > >> >> may produce data not in order.
> >> > > >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can
> >> > reorder
> >> > > >> data.
> >> > > >> >>
> >> > > >> >> Best,
> >> > > >> >> Jark
> >> > > >> >>
> >> > > >> >>
> >> > > >> >>
> >> > > >> >> On Tue, 17 Dec 2019 at 22:03, Becket Qin <
> becket.qin@gmail.com>
> >> > > wrote:
> >> > > >> >>
> >> > > >> >>> Hi folks,
> >> > > >> >>>
> >> > > >> >>> Thanks for the comments. I am convinced that the Source API
> >> should
> >> > > not
> >> > > >> >> take
> >> > > >> >>> boundedness as a parameter after it is constructed. What Timo
> >> and
> >> > > >> Dawid
> >> > > >> >>> suggested sounds a reasonable solution to me. So the Source
> API
> >> > > would
> >> > > >> >>> become:
> >> > > >> >>>
> >> > > >> >>> Source {
> >> > > >> >>>      Boundedness getBoundedness();
> >> > > >> >>> }
> >> > > >> >>>
> >> > > >> >>> Assuming the above Source API, in addition to the two options
> >> > > >> mentioned
> >> > > >> >> in
> >> > > >> >>> earlier emails, I am thinking of another option:
> >> > > >> >>>
> >> > > >> >>> *Option 3:*
> >> > > >> >>> // MySource must be unbounded, otherwise throws exception.
> >> > > >> >>> DataStream<Type> dataStream = env.source(mySource);
> >> > > >> >>>
> >> > > >> >>> // MySource must be bounded, otherwise throws exception.
> >> > > >> >>> BoundedDataStream<Type> boundedDataStream =
> >> > > >> env.boundedSource(mySource);
> >> > > >> >>>
> >> > > >> >>> The pros of this API are:
> >> > > >> >>>     a) It fits the requirements from Table / SQL well.
> >> > > >> >>>     b) DataStream users still have type safety (option 2 only
> >> has
> >> > > >> partial
> >> > > >> >>> type safety).
> >> > > >> >>>     c) Cristal clear boundedness from the API which makes
> >> > DataStream
> >> > > >> join
> >> > > >> >> /
> >> > > >> >>> connect easy to reason about.
> >> > > >> >>> The caveats I see,
> >> > > >> >>>     a) It is inconsistent with Table since Table has one
> >> unified
> >> > > >> >> interface.
> >> > > >> >>>     b) No streaming mode for bounded source.
> >> > > >> >>>
> >> > > >> >>> @Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
> >> > > >> >>> <al...@ververica.com> what do you think of the approach?
> >> > > >> >>>
> >> > > >> >>>
> >> > > >> >>> Orthogonal to the above API, I am wondering whether
> >> boundedness is
> >> > > the
> >> > > >> >> only
> >> > > >> >>> dimension needed to describe the characteristic of the Source
> >> > > >> behavior.
> >> > > >> >> We
> >> > > >> >>> may also need to have another dimension of *record order*.
> >> > > >> >>>
> >> > > >> >>> For example, when a file source is reading from a directory
> >> with
> >> > > >> bounded
> >> > > >> >>> records, it may have two ways to read.
> >> > > >> >>> 1. Read files in parallel.
> >> > > >> >>> 2. Read files in the chronological order.
> >> > > >> >>> In both cases, the file source is a Bounded Source. However,
> >> the
> >> > > >> >> processing
> >> > > >> >>> requirement for downstream may be different. In the first
> case,
> >> > the
> >> > > >> >>> record processing and result emitting order does not matter,
> >> e.g.
> >> > > word
> >> > > >> >>> count. In the second case, the records may have to be
> >> processed in
> >> > > the
> >> > > >> >>> order they were read, e.g. change log processing.
> >> > > >> >>>
> >> > > >> >>> If the Source only has a getBoundedness() method, the
> >> downstream
> >> > > >> >> processors
> >> > > >> >>> would not know whether the records emitted from the Source
> >> should
> >> > be
> >> > > >> >>> processed in order or not. So combining the boundedness and
> >> record
> >> > > >> order,
> >> > > >> >>> we will have four scenarios:
> >> > > >> >>>
> >> > > >> >>> *Bounded-StrictOrder*:     A segment of change log.
> >> > > >> >>> *Bounded-Random*:          Batch Word Count.
> >> > > >> >>> *Unbounded-StrictOrder*: An infinite change log.
> >> > > >> >>> *Unbounded-Random*:     Streaming Word Count.
> >> > > >> >>>
> >> > > >> >>> Option 2 mentioned in the previous email was kind of trying
> to
> >> > > handle
> >> > > >> the
> >> > > >> >>> Bounded-StrictOrder case by creating a DataStream from a
> >> bounded
> >> > > >> source,
> >> > > >> >>> which actually does not work.
> >> > > >> >>> It looks that we do not have strict order support in some
> >> > operators
> >> > > at
> >> > > >> >> this
> >> > > >> >>> point, e.g. join. But we may still want to add the semantic
> to
> >> the
> >> > > >> Source
> >> > > >> >>> first so later on we don't need to change all the source
> >> > > >> implementations,
> >> > > >> >>> especially given that many of them will be implemented by 3rd
> >> > party.
> >> > > >> >>>
> >> > > >> >>> Given that, we need another dimension of *Record Order* in
> the
> >> > > Source.
> >> > > >> >> More
> >> > > >> >>> specifically, the API would become:
> >> > > >> >>>
> >> > > >> >>> Source {
> >> > > >> >>>      Boundedness getBoundedness();
> >> > > >> >>>      RecordOrder getRecordOrder();
> >> > > >> >>> }
> >> > > >> >>>
> >> > > >> >>> public enum RecordOrder {
> >> > > >> >>>      /** The record in the DataStream must be processed in
> its
> >> > > strict
> >> > > >> >> order
> >> > > >> >>> for correctness. */
> >> > > >> >>>      STRICT,
> >> > > >> >>>      /** The record in the DataStream can be processed in
> >> > arbitrary
> >> > > >> order.
> >> > > >> >>> */
> >> > > >> >>>      RANDOM;
> >> > > >> >>> }
> >> > > >> >>>
> >> > > >> >>> Any thoughts?
> >> > > >> >>>
> >> > > >> >>> Thanks,
> >> > > >> >>>
> >> > > >> >>> Jiangjie (Becket) Qin
> >> > > >> >>>
> >> > > >> >>> On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <
> >> twalthr@apache.org>
> >> > > >> wrote:
> >> > > >> >>>
> >> > > >> >>>> Hi Becket,
> >> > > >> >>>>
> >> > > >> >>>> I completely agree with Dawid's suggestion. The information
> >> about
> >> > > the
> >> > > >> >>>> boundedness should come out of the source. Because most of
> the
> >> > > >> >> streaming
> >> > > >> >>>> sources can be made bounded based on some connector specific
> >> > > >> criterion.
> >> > > >> >>>> In Kafka, it would be an end offset or end timestamp but in
> >> any
> >> > > case
> >> > > >> >>>> having just a env.boundedSource() is not enough because
> >> > parameters
> >> > > >> for
> >> > > >> >>>> making the source bounded are missing.
> >> > > >> >>>>
> >> > > >> >>>> I suggest to have a simple `isBounded(): Boolean` flag in
> >> every
> >> > > >> source
> >> > > >> >>>> that might be influenced by a connector builder as Dawid
> >> > mentioned.
> >> > > >> >>>>
> >> > > >> >>>> For type safety during programming, we can still go with
> >> *Final
> >> > > state
> >> > > >> >>>> 1*. By having a env.source() vs env.boundedSource(). The
> >> latter
> >> > > would
> >> > > >> >>>> just enforce that the boolean flag is set to `true` and
> could
> >> > make
> >> > > >> >>>> bounded operations available (if we need that actually).
> >> > > >> >>>>
> >> > > >> >>>> However, I don't think that we should start making a unified
> >> > Table
> >> > > >> API
> >> > > >> >>>> ununified again. Boundedness is an optimization property.
> >> Every
> >> > > >> bounded
> >> > > >> >>>> operation can also executed in an unbounded way using
> >> > > >> >> updates/retraction
> >> > > >> >>>> or watermarks.
> >> > > >> >>>>
> >> > > >> >>>> Regards,
> >> > > >> >>>> Timo
> >> > > >> >>>>
> >> > > >> >>>>
> >> > > >> >>>> On 15.12.19 14:22, Becket Qin wrote:
> >> > > >> >>>>> Hi Dawid and Jark,
> >> > > >> >>>>>
> >> > > >> >>>>> I think the discussion ultimately boils down to the
> question
> >> > that
> >> > > >> >> which
> >> > > >> >>>> one
> >> > > >> >>>>> of the following two final states do we want? Once we make
> >> this
> >> > > >> >>> decision,
> >> > > >> >>>>> everything else can be naturally derived.
> >> > > >> >>>>>
> >> > > >> >>>>> *Final state 1*: Separate API for bounded / unbounded
> >> > DataStream &
> >> > > >> >>> Table.
> >> > > >> >>>>> That means any code users write will be valid at the point
> >> when
> >> > > they
> >> > > >> >>>> write
> >> > > >> >>>>> the code. This is similar to having type safety check at
> >> > > programming
> >> > > >> >>>> time.
> >> > > >> >>>>> For example,
> >> > > >> >>>>>
> >> > > >> >>>>> BoundedDataStream extends DataStream {
> >> > > >> >>>>> // Operations only available for bounded data.
> >> > > >> >>>>> BoundedDataStream sort(...);
> >> > > >> >>>>>
> >> > > >> >>>>> // Interaction with another BoundedStream returns a Bounded
> >> > > stream.
> >> > > >> >>>>> BoundedJoinedDataStream join(BoundedDataStream other)
> >> > > >> >>>>>
> >> > > >> >>>>> // Interaction with another unbounded stream returns an
> >> > unbounded
> >> > > >> >>> stream.
> >> > > >> >>>>> JoinedDataStream join(DataStream other)
> >> > > >> >>>>> }
> >> > > >> >>>>>
> >> > > >> >>>>> BoundedTable extends Table {
> >> > > >> >>>>>     // Bounded only operation.
> >> > > >> >>>>> BoundedTable sort(...);
> >> > > >> >>>>>
> >> > > >> >>>>> // Interaction with another BoundedTable returns a
> >> BoundedTable.
> >> > > >> >>>>> BoundedTable join(BoundedTable other)
> >> > > >> >>>>>
> >> > > >> >>>>> // Interaction with another unbounded table returns an
> >> unbounded
> >> > > >> >> table.
> >> > > >> >>>>> Table join(Table other)
> >> > > >> >>>>> }
> >> > > >> >>>>>
> >> > > >> >>>>> *Final state 2*: One unified API for bounded / unbounded
> >> > > DataStream
> >> > > >> /
> >> > > >> >>>>> Table.
> >> > > >> >>>>> That unified API may throw exception at DAG compilation
> time
> >> if
> >> > an
> >> > > >> >>>> invalid
> >> > > >> >>>>> operation is tried. This is what Table API currently
> follows.
> >> > > >> >>>>>
> >> > > >> >>>>> DataStream {
> >> > > >> >>>>> // Throws exception if the DataStream is unbounded.
> >> > > >> >>>>> DataStream sort();
> >> > > >> >>>>> // Get boundedness.
> >> > > >> >>>>> Boundedness getBoundedness();
> >> > > >> >>>>> }
> >> > > >> >>>>>
> >> > > >> >>>>> Table {
> >> > > >> >>>>> // Throws exception if the table has infinite rows.
> >> > > >> >>>>> Table orderBy();
> >> > > >> >>>>>
> >> > > >> >>>>> // Get boundedness.
> >> > > >> >>>>> Boundedness getBoundedness();
> >> > > >> >>>>> }
> >> > > >> >>>>>
> >> > > >> >>>>> >From what I understand, there is no consensus so far on
> this
> >> > > >> decision
> >> > > >> >>>> yet.
> >> > > >> >>>>> Whichever final state we choose, we need to make it
> >> consistent
> >> > > >> across
> >> > > >> >>> the
> >> > > >> >>>>> entire project. We should avoid the case that Table follows
> >> one
> >> > > >> final
> >> > > >> >>>> state
> >> > > >> >>>>> while DataStream follows another. Some arguments I am aware
> >> of
> >> > > from
> >> > > >> >>> both
> >> > > >> >>>>> sides so far are following:
> >> > > >> >>>>>
> >> > > >> >>>>> Arguments for final state 1:
> >> > > >> >>>>> 1a) Clean API with method safety check at programming time.
> >> > > >> >>>>> 1b) (Counter 2b) Although SQL does not have programming
> time
> >> > error
> >> > > >> >>>> check, SQL
> >> > > >> >>>>> is not really a "programming language" per se. So SQL can
> be
> >> > > >> >> different
> >> > > >> >>>> from
> >> > > >> >>>>> Table and DataStream.
> >> > > >> >>>>> 1c)  Although final state 2 seems making it easier for SQL
> to
> >> > use
> >> > > >> >> given
> >> > > >> >>>> it
> >> > > >> >>>>> is more "config based" than "parameter based", final state
> 1
> >> can
> >> > > >> >>> probably
> >> > > >> >>>>> also meet what SQL wants by wrapping the Source in
> >> TableSource /
> >> > > >> >>>>> TableSourceFactory API if needed.
> >> > > >> >>>>>
> >> > > >> >>>>> Arguments for final state 2:
> >> > > >> >>>>> 2a) The Source API itself seems already sort of following
> the
> >> > > >> unified
> >> > > >> >>> API
> >> > > >> >>>>> pattern.
> >> > > >> >>>>> 2b) There is no "programming time" method error check in
> SQL
> >> > case,
> >> > > >> so
> >> > > >> >>> we
> >> > > >> >>>>> cannot really achieve final state 1 across the board.
> >> > > >> >>>>> 2c) It is an easier path given our current status, i.e.
> >> Table is
> >> > > >> >>> already
> >> > > >> >>>>> following final state 2.
> >> > > >> >>>>> 2d) Users can always explicitly check the boundedness if
> they
> >> > want
> >> > > >> >> to.
> >> > > >> >>>>>
> >> > > >> >>>>> As I mentioned earlier, my initial thought was also to
> have a
> >> > > >> >>>>> "configuration based" Source rather than a "parameter
> based"
> >> > > Source.
> >> > > >> >> So
> >> > > >> >>>> it
> >> > > >> >>>>> is completely possible that I missed some important
> >> > consideration
> >> > > or
> >> > > >> >>>> design
> >> > > >> >>>>> principles that we want to enforce for the project. It
> would
> >> be
> >> > > good
> >> > > >> >>>>> if @Stephan
> >> > > >> >>>>> Ewen <st...@ververica.com> and @Aljoscha Krettek <
> >> > > >> >>>> aljoscha@ververica.com> can
> >> > > >> >>>>> also provide more thoughts on this.
> >> > > >> >>>>>
> >> > > >> >>>>>
> >> > > >> >>>>> Re: Jingsong
> >> > > >> >>>>>
> >> > > >> >>>>> As you said, there are some batched system source, like
> >> > > parquet/orc
> >> > > >> >>>> source.
> >> > > >> >>>>>> Could we have the batch emit interface to improve
> >> performance?
> >> > > The
> >> > > >> >>>> queue of
> >> > > >> >>>>>> per record may cause performance degradation.
> >> > > >> >>>>>
> >> > > >> >>>>>
> >> > > >> >>>>> The current interface does not necessarily cause
> performance
> >> > > problem
> >> > > >> >>> in a
> >> > > >> >>>>> multi-threading case. In fact, the base implementation
> allows
> >> > > >> >>>> SplitReaders
> >> > > >> >>>>> to add a batch <E> of records<T> to the records queue<E>,
> so
> >> > each
> >> > > >> >>> element
> >> > > >> >>>>> in the records queue would be a batch <E>. In this case,
> when
> >> > the
> >> > > >> >> main
> >> > > >> >>>>> thread polls records, it will take a batch <E> of records
> <T>
> >> > from
> >> > > >> >> the
> >> > > >> >>>>> shared records queue and process the records <T> in a batch
> >> > > manner.
> >> > > >> >>>>>
> >> > > >> >>>>> Thanks,
> >> > > >> >>>>>
> >> > > >> >>>>> Jiangjie (Becket) Qin
> >> > > >> >>>>>
> >> > > >> >>>>> On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <
> >> > > jingsonglee0@gmail.com
> >> > > >> >
> >> > > >> >>>> wrote:
> >> > > >> >>>>>
> >> > > >> >>>>>> Hi Becket,
> >> > > >> >>>>>>
> >> > > >> >>>>>> I also have some performance concerns too.
> >> > > >> >>>>>>
> >> > > >> >>>>>> If I understand correctly, SourceOutput will emit data per
> >> > record
> >> > > >> >> into
> >> > > >> >>>> the
> >> > > >> >>>>>> queue? I'm worried about the multithreading performance of
> >> this
> >> > > >> >> queue.
> >> > > >> >>>>>>
> >> > > >> >>>>>>> One example is some batched messaging systems which only
> >> have
> >> > an
> >> > > >> >>> offset
> >> > > >> >>>>>> for the entire batch instead of individual messages in the
> >> > batch.
> >> > > >> >>>>>>
> >> > > >> >>>>>> As you said, there are some batched system source, like
> >> > > parquet/orc
> >> > > >> >>>> source.
> >> > > >> >>>>>> Could we have the batch emit interface to improve
> >> performance?
> >> > > The
> >> > > >> >>>> queue of
> >> > > >> >>>>>> per record may cause performance degradation.
> >> > > >> >>>>>>
> >> > > >> >>>>>> Best,
> >> > > >> >>>>>> Jingsong Lee
> >> > > >> >>>>>>
> >> > > >> >>>>>> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <imjark@gmail.com
> >
> >> > > wrote:
> >> > > >> >>>>>>
> >> > > >> >>>>>>> Hi Becket,
> >> > > >> >>>>>>>
> >> > > >> >>>>>>> I think Dawid explained things clearly and makes a lot of
> >> > sense.
> >> > > >> >>>>>>> I'm also in favor of #2, because #1 doesn't work for our
> >> > future
> >> > > >> >>> unified
> >> > > >> >>>>>>> envrionment.
> >> > > >> >>>>>>>
> >> > > >> >>>>>>> You can see the vision in this documentation [1]. In the
> >> > future,
> >> > > >> we
> >> > > >> >>>> would
> >> > > >> >>>>>>> like to
> >> > > >> >>>>>>> drop the global streaming/batch mode in SQL (i.e.
> >> > > >> >>>>>>> EnvironmentSettings#inStreamingMode/inBatchMode).
> >> > > >> >>>>>>> A source is bounded or unbounded once defined, so queries
> >> can
> >> > be
> >> > > >> >>>> inferred
> >> > > >> >>>>>>> from source to run
> >> > > >> >>>>>>> in streaming or batch or hybrid mode. However, in #1, we
> >> will
> >> > > lose
> >> > > >> >>> this
> >> > > >> >>>>>>> ability because the framework
> >> > > >> >>>>>>> doesn't know whether the source is bounded or unbounded.
> >> > > >> >>>>>>>
> >> > > >> >>>>>>> Best,
> >> > > >> >>>>>>> Jark
> >> > > >> >>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>> [1]:
> >> > > >> >>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
> >> > > >> >>>>>>>
> >> > > >> >>>>>>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <
> >> > > piotr@ververica.com
> >> > > >> >
> >> > > >> >>>>>> wrote:
> >> > > >> >>>>>>>
> >> > > >> >>>>>>>> Hi,
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>> Regarding the:
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>> Collection<E> getNextRecords()
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>> I’m pretty sure such design would unfortunately impact
> the
> >> > > >> >>> performance
> >> > > >> >>>>>>>> (accessing and potentially creating the collection on
> the
> >> hot
> >> > > >> >> path).
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>> Also the
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>> InputStatus emitNext(DataOutput<T> output) throws
> >> Exception;
> >> > > >> >>>>>>>> or
> >> > > >> >>>>>>>> Status pollNext(SourceOutput<T> sourceOutput) throws
> >> > Exception;
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>> Gives us some opportunities in the future, to allow
> Source
> >> > hot
> >> > > >> >>> looping
> >> > > >> >>>>>>>> inside, until it receives some signal “please exit
> >> because of
> >> > > >> some
> >> > > >> >>>>>>> reasons”
> >> > > >> >>>>>>>> (output collector could return such hint upon collecting
> >> the
> >> > > >> >>> result).
> >> > > >> >>>>>> But
> >> > > >> >>>>>>>> that’s another topic outside of this FLIP’s scope.
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>> Piotrek
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <
> >> > trohrmann@apache.org
> >> > > >
> >> > > >> >>>>>> wrote:
> >> > > >> >>>>>>>>>
> >> > > >> >>>>>>>>> Hi Becket,
> >> > > >> >>>>>>>>>
> >> > > >> >>>>>>>>> quick clarification from my side because I think you
> >> > > >> >> misunderstood
> >> > > >> >>> my
> >> > > >> >>>>>>>>> question. I did not suggest to let the SourceReader
> >> return
> >> > > only
> >> > > >> a
> >> > > >> >>>>>>> single
> >> > > >> >>>>>>>>> record at a time when calling getNextRecords. As the
> >> return
> >> > > type
> >> > > >> >>>>>>>> indicates,
> >> > > >> >>>>>>>>> the method can return an arbitrary number of records.
> >> > > >> >>>>>>>>>
> >> > > >> >>>>>>>>> Cheers,
> >> > > >> >>>>>>>>> Till
> >> > > >> >>>>>>>>>
> >> > > >> >>>>>>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> >> > > >> >>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> >> > > >> >>>>>>>>> wrote:
> >> > > >> >>>>>>>>>
> >> > > >> >>>>>>>>>> Hi Becket,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Issue #1 - Design of Source interface
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I mentioned the lack of a method like
> >> > > >> >>>>>>>> Source#createEnumerator(Boundedness
> >> > > >> >>>>>>>>>> boundedness, SplitEnumeratorContext context), because
> >> > without
> >> > > >> >> the
> >> > > >> >>>>>>>> current
> >> > > >> >>>>>>>>>> proposal is not complete/does not work.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> If we say that boundedness is an intrinsic property
> of a
> >> > > source
> >> > > >> >>> imo
> >> > > >> >>>>>> we
> >> > > >> >>>>>>>>>> don't need the Source#createEnumerator(Boundedness
> >> > > boundedness,
> >> > > >> >>>>>>>>>> SplitEnumeratorContext context) method.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Assuming a source from my previous example:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
> >> > > >> >>>>>>>>>>    ...
> >> > > >> >>>>>>>>>>    .untilTimestamp(...)
> >> > > >> >>>>>>>>>>    .build()
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Would the enumerator differ if created like
> >> > > >> >>>>>>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs
> >> > source
> >> > > >> >>>>>>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating
> >> > > myself,
> >> > > >> >> but
> >> > > >> >>>>>>> this
> >> > > >> >>>>>>>> is
> >> > > >> >>>>>>>>>> the part that my opinion differ the most from the
> >> current
> >> > > >> >>> proposal.
> >> > > >> >>>>>> I
> >> > > >> >>>>>>>>>> really think it should always be the source that tells
> >> if
> >> > it
> >> > > is
> >> > > >> >>>>>>> bounded
> >> > > >> >>>>>>>> or
> >> > > >> >>>>>>>>>> not. In the current proposal methods
> >> > > >> >> continousSource/boundedSource
> >> > > >> >>>>>>>> somewhat
> >> > > >> >>>>>>>>>> reconfigure the source, which I think is misleading.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I think a call like:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
> >> > > >> >>>>>>>>>>    ...
> >> > > >> >>>>>>>>>>    .readContinously() / readUntilLatestOffset() /
> >> > > >> >>> readUntilTimestamp
> >> > > >> >>>> /
> >> > > >> >>>>>>>> readUntilOffsets / ...
> >> > > >> >>>>>>>>>>    .build()
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> is way cleaner (and expressive) than
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
> >> > > >> >>>>>>>>>>    ...
> >> > > >> >>>>>>>>>>    .build()
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> env.continousSource(source) // which actually
> underneath
> >> > > would
> >> > > >> >>> call
> >> > > >> >>>>>>>> createEnumerator(CONTINUOUS, ctx) which would be
> >> equivalent
> >> > to
> >> > > >> >>>>>>>> source.readContinously().createEnumerator(ctx)
> >> > > >> >>>>>>>>>> // or
> >> > > >> >>>>>>>>>> env.boundedSource(source) // which actually underneath
> >> > would
> >> > > >> >> call
> >> > > >> >>>>>>>> createEnumerator(BOUNDED, ctx) which would be equivalent
> >> to
> >> > > >> >>>>>>>> source.readUntilLatestOffset().createEnumerator(ctx)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Sorry for the comparison, but to me it seems there is
> >> too
> >> > > much
> >> > > >> >>> magic
> >> > > >> >>>>>>>>>> happening underneath those two calls.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I really believe the Source interface should have
> >> > > >> getBoundedness
> >> > > >> >>>>>>> method
> >> > > >> >>>>>>>>>> instead of (supportBoundedness) +
> >> > > createEnumerator(Boundedness,
> >> > > >> >>> ...)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Issue #2 - Design of
> >> > > >> >>>>>>>>>>
> >> > > >> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> As you might have guessed I am slightly in favor of
> >> option
> >> > #2
> >> > > >> >>>>>>> modified.
> >> > > >> >>>>>>>>>> Yes I am aware every step of the dag would have to be
> >> able
> >> > to
> >> > > >> >> say
> >> > > >> >>> if
> >> > > >> >>>>>>> it
> >> > > >> >>>>>>>> is
> >> > > >> >>>>>>>>>> bounded or not. I have a feeling it would be easier to
> >> > > express
> >> > > >> >>> cross
> >> > > >> >>>>>>>>>> bounded/unbounded operations, but I must admit I have
> >> not
> >> > > >> >> thought
> >> > > >> >>> it
> >> > > >> >>>>>>>>>> through thoroughly, In the spirit of batch is just a
> >> > special
> >> > > >> >> case
> >> > > >> >>> of
> >> > > >> >>>>>>>>>> streaming I thought BoundedStream would extend from
> >> > > DataStream.
> >> > > >> >>>>>>> Correct
> >> > > >> >>>>>>>> me
> >> > > >> >>>>>>>>>> if I am wrong. In such a setup the cross
> >> bounded/unbounded
> >> > > >> >>> operation
> >> > > >> >>>>>>>> could
> >> > > >> >>>>>>>>>> be expressed quite easily I think:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> DataStream {
> >> > > >> >>>>>>>>>>    DataStream join(DataStream, ...); // we could not
> >> really
> >> > > >> tell
> >> > > >> >> if
> >> > > >> >>>>>> the
> >> > > >> >>>>>>>> result is bounded or not, but because bounded stream is
> a
> >> > > special
> >> > > >> >>> case
> >> > > >> >>>>>> of
> >> > > >> >>>>>>>> unbounded the API object is correct, irrespective if the
> >> left
> >> > > or
> >> > > >> >>> right
> >> > > >> >>>>>>> side
> >> > > >> >>>>>>>> of the join is bounded
> >> > > >> >>>>>>>>>> }
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> BoundedStream extends DataStream {
> >> > > >> >>>>>>>>>>    BoundedStream join(BoundedStream, ...); // only if
> >> both
> >> > > >> sides
> >> > > >> >>> are
> >> > > >> >>>>>>>> bounded the result can be bounded as well. However we do
> >> have
> >> > > >> >> access
> >> > > >> >>>> to
> >> > > >> >>>>>>> the
> >> > > >> >>>>>>>> DataStream#join here, so you can still join with a
> >> DataStream
> >> > > >> >>>>>>>>>> }
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On the other hand I also see benefits of two
> completely
> >> > > >> >> disjointed
> >> > > >> >>>>>>> APIs,
> >> > > >> >>>>>>>>>> as we could prohibit some streaming calls in the
> bounded
> >> > > API. I
> >> > > >> >>>>>> can't
> >> > > >> >>>>>>>> think
> >> > > >> >>>>>>>>>> of any unbounded operators that could not be
> implemented
> >> > for
> >> > > >> >>> bounded
> >> > > >> >>>>>>>> stream.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Besides I think we both agree we don't like the
> method:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> DataStream boundedStream(Source)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> suggested in the current state of the FLIP. Do we ? :)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Best,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Dawid
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On 10/12/2019 18:57, Becket Qin wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi folks,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks for the discussion, great feedback. Also thanks
> >> > Dawid
> >> > > >> for
> >> > > >> >>> the
> >> > > >> >>>>>>>>>> explanation, it is much clearer now.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> One thing that is indeed missing from the FLIP is how
> >> the
> >> > > >> >>>>>> boundedness
> >> > > >> >>>>>>> is
> >> > > >> >>>>>>>>>> passed to the Source implementation. So the API should
> >> be
> >> > > >> >>>>>>>>>> Source#createEnumerator(Boundedness boundedness,
> >> > > >> >>>>>>> SplitEnumeratorContext
> >> > > >> >>>>>>>>>> context)
> >> > > >> >>>>>>>>>> And we can probably remove the
> >> > > >> >>> Source#supportBoundedness(Boundedness
> >> > > >> >>>>>>>>>> boundedness) method.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Assuming we have that, we are essentially choosing
> from
> >> one
> >> > > of
> >> > > >> >> the
> >> > > >> >>>>>>>>>> following two options:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Option 1:
> >> > > >> >>>>>>>>>> // The source is continuous source, and only unbounded
> >> > > >> >> operations
> >> > > >> >>>>>> can
> >> > > >> >>>>>>> be
> >> > > >> >>>>>>>>>> performed.
> >> > > >> >>>>>>>>>> DataStream<Type> datastream =
> >> > > env.continuousSource(someSource);
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> // The source is bounded source, both bounded and
> >> unbounded
> >> > > >> >>>>>> operations
> >> > > >> >>>>>>>> can
> >> > > >> >>>>>>>>>> be performed.
> >> > > >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> >> > > >> >>>>>>>> env.boundedSource(someSource);
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>    - Pros:
> >> > > >> >>>>>>>>>>         a) explicit boundary between bounded /
> unbounded
> >> > > >> streams,
> >> > > >> >>> it
> >> > > >> >>>>>> is
> >> > > >> >>>>>>>>>> quite simple and clear to the users.
> >> > > >> >>>>>>>>>>    - Cons:
> >> > > >> >>>>>>>>>>         a) For applications that do not involve
> bounded
> >> > > >> >> operations,
> >> > > >> >>>>>> they
> >> > > >> >>>>>>>>>> still have to call different API to distinguish
> bounded
> >> /
> >> > > >> >>> unbounded
> >> > > >> >>>>>>>> streams.
> >> > > >> >>>>>>>>>>         b) No support for bounded stream to run in a
> >> > > streaming
> >> > > >> >>>> runtime
> >> > > >> >>>>>>>>>> setting, i.e. scheduling and operators behaviors.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Option 2:
> >> > > >> >>>>>>>>>> // The source is either bounded or unbounded, but only
> >> > > >> unbounded
> >> > > >> >>>>>>>> operations
> >> > > >> >>>>>>>>>> could be performed on the returned DataStream.
> >> > > >> >>>>>>>>>> DataStream<Type> dataStream = env.source(someSource);
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> // The source must be a bounded source, otherwise
> >> exception
> >> > > is
> >> > > >> >>>>>> thrown.
> >> > > >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> >> > > >> >>>>>>>>>> env.boundedSource(boundedSource);
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> The pros and cons are exactly the opposite of option
> 1.
> >> > > >> >>>>>>>>>>    - Pros:
> >> > > >> >>>>>>>>>>         a) For applications that do not involve
> bounded
> >> > > >> >> operations,
> >> > > >> >>>>>> they
> >> > > >> >>>>>>>>>> still have to call different API to distinguish
> bounded
> >> /
> >> > > >> >>> unbounded
> >> > > >> >>>>>>>> streams.
> >> > > >> >>>>>>>>>>         b) Support for bounded stream to run in a
> >> streaming
> >> > > >> >> runtime
> >> > > >> >>>>>>>> setting,
> >> > > >> >>>>>>>>>> i.e. scheduling and operators behaviors.
> >> > > >> >>>>>>>>>>    - Cons:
> >> > > >> >>>>>>>>>>         a) Bounded / unbounded streams are kind of
> >> mixed,
> >> > > i.e.
> >> > > >> >>> given
> >> > > >> >>>> a
> >> > > >> >>>>>>>>>> DataStream, it is not clear whether it is bounded or
> >> not,
> >> > > >> unless
> >> > > >> >>> you
> >> > > >> >>>>>>>> have
> >> > > >> >>>>>>>>>> the access to its source.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> If we only think from the Source API perspective,
> >> option 2
> >> > > >> >> seems a
> >> > > >> >>>>>>>> better
> >> > > >> >>>>>>>>>> choice because functionality wise it is a superset of
> >> > option
> >> > > 1,
> >> > > >> >> at
> >> > > >> >>>>>> the
> >> > > >> >>>>>>>> cost
> >> > > >> >>>>>>>>>> of some seemingly acceptable ambiguity in the
> DataStream
> >> > API.
> >> > > >> >>>>>>>>>> But if we look at the DataStream API as a whole,
> option
> >> 1
> >> > > seems
> >> > > >> >> a
> >> > > >> >>>>>>>> clearer
> >> > > >> >>>>>>>>>> choice. For example, some times a library may have to
> >> know
> >> > > >> >>> whether a
> >> > > >> >>>>>>>>>> certain task will finish or not. And it would be
> >> difficult
> >> > to
> >> > > >> >> tell
> >> > > >> >>>>>> if
> >> > > >> >>>>>>>> the
> >> > > >> >>>>>>>>>> input is a DataStream, unless additional information
> is
> >> > > >> provided
> >> > > >> >>> all
> >> > > >> >>>>>>> the
> >> > > >> >>>>>>>>>> way from the Source. One possible solution is to have
> a
> >> > > >> >> *modified
> >> > > >> >>>>>>>> option 2*
> >> > > >> >>>>>>>>>> which adds a method to the DataStream API to indicate
> >> > > >> >> boundedness,
> >> > > >> >>>>>>> such
> >> > > >> >>>>>>>> as
> >> > > >> >>>>>>>>>> getBoundedness(). It would solve the problem with a
> >> > potential
> >> > > >> >>>>>>> confusion
> >> > > >> >>>>>>>> of
> >> > > >> >>>>>>>>>> what is difference between a DataStream with
> >> > > >> >> getBoundedness()=true
> >> > > >> >>>>>>> and a
> >> > > >> >>>>>>>>>> BoundedDataStream. But that seems not super difficult
> to
> >> > > >> >> explain.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> So from API's perspective, I don't have a strong
> opinion
> >> > > >> between
> >> > > >> >>>>>>>> *option 1*
> >> > > >> >>>>>>>>>> and *modified option 2. *I like the cleanness of
> option
> >> 1,
> >> > > but
> >> > > >> >>>>>>> modified
> >> > > >> >>>>>>>>>> option 2 would be more attractive if we have concrete
> >> use
> >> > > case
> >> > > >> >> for
> >> > > >> >>>>>> the
> >> > > >> >>>>>>>>>> "Bounded stream with unbounded streaming runtime
> >> settings".
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Re: Till
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Maybe this has already been asked before but I was
> >> > wondering
> >> > > >> why
> >> > > >> >>> the
> >> > > >> >>>>>>>>>> SourceReader interface has the method pollNext which
> >> hands
> >> > > the
> >> > > >> >>>>>>>>>> responsibility of outputting elements to the
> >> SourceReader
> >> > > >> >>>>>>>> implementation?
> >> > > >> >>>>>>>>>> Has this been done for backwards compatibility reasons
> >> with
> >> > > the
> >> > > >> >>> old
> >> > > >> >>>>>>>> source
> >> > > >> >>>>>>>>>> interface? If not, then one could define a
> Collection<E>
> >> > > >> >>>>>>>> getNextRecords()
> >> > > >> >>>>>>>>>> method which returns the currently retrieved records
> and
> >> > then
> >> > > >> >> the
> >> > > >> >>>>>>> caller
> >> > > >> >>>>>>>>>> emits them outside of the SourceReader. That way the
> >> > > interface
> >> > > >> >>> would
> >> > > >> >>>>>>> not
> >> > > >> >>>>>>>>>> allow to implement an outputting loop where we never
> >> hand
> >> > > back
> >> > > >> >>>>>> control
> >> > > >> >>>>>>>> to
> >> > > >> >>>>>>>>>> the caller. At the moment, this contract can be easily
> >> > broken
> >> > > >> >> and
> >> > > >> >>> is
> >> > > >> >>>>>>>> only
> >> > > >> >>>>>>>>>> mentioned loosely in the JavaDocs.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> The primary reason we handover the SourceOutput to the
> >> > > >> >>> SourceReader
> >> > > >> >>>>>> is
> >> > > >> >>>>>>>>>> because sometimes it is difficult for a SourceReader
> to
> >> > emit
> >> > > >> one
> >> > > >> >>>>>>> record
> >> > > >> >>>>>>>> at
> >> > > >> >>>>>>>>>> a time. One example is some batched messaging systems
> >> which
> >> > > >> only
> >> > > >> >>>>>> have
> >> > > >> >>>>>>> an
> >> > > >> >>>>>>>>>> offset for the entire batch instead of individual
> >> messages
> >> > in
> >> > > >> >> the
> >> > > >> >>>>>>>> batch. In
> >> > > >> >>>>>>>>>> that case, returning one record at a time would leave
> >> the
> >> > > >> >>>>>> SourceReader
> >> > > >> >>>>>>>> in
> >> > > >> >>>>>>>>>> an uncheckpointable state because they can only
> >> checkpoint
> >> > at
> >> > > >> >> the
> >> > > >> >>>>>>> batch
> >> > > >> >>>>>>>>>> boundaries.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <
> >> > > >> >>> trohrmann@apache.org
> >> > > >> >>>>>>>> <ma...@apache.org>> <trohrmann@apache.org
> >> > <mailto:
> >> > > >> >>>>>>>> trohrmann@apache.org>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi everyone,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> thanks for drafting this FLIP. It reads very well.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Concerning Dawid's proposal, I tend to agree. The
> >> > boundedness
> >> > > >> >>> could
> >> > > >> >>>>>>> come
> >> > > >> >>>>>>>>>> from the source and tell the system how to treat the
> >> > operator
> >> > > >> >>>>>>>> (scheduling
> >> > > >> >>>>>>>>>> wise). From a user's perspective it should be fine to
> >> get
> >> > > back
> >> > > >> a
> >> > > >> >>>>>>>> DataStream
> >> > > >> >>>>>>>>>> when calling env.source(boundedSource) if he does not
> >> need
> >> > > >> >> special
> >> > > >> >>>>>>>>>> operations defined on a BoundedDataStream. If he needs
> >> > this,
> >> > > >> >> then
> >> > > >> >>>>>> one
> >> > > >> >>>>>>>> could
> >> > > >> >>>>>>>>>> use the method BoundedDataStream
> >> > > >> >> env.boundedSource(boundedSource).
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> If possible, we could enforce the proper usage of
> >> > > >> >>>>>> env.boundedSource()
> >> > > >> >>>>>>> by
> >> > > >> >>>>>>>>>> introducing a BoundedSource type so that one cannot
> >> pass an
> >> > > >> >>>>>>>>>> unbounded source to it. That way users would not be
> >> able to
> >> > > >> >> shoot
> >> > > >> >>>>>>>>>> themselves in the foot.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Maybe this has already been asked before but I was
> >> > wondering
> >> > > >> why
> >> > > >> >>> the
> >> > > >> >>>>>>>>>> SourceReader interface has the method pollNext which
> >> hands
> >> > > the
> >> > > >> >>>>>>>>>> responsibility of outputting elements to the
> >> SourceReader
> >> > > >> >>>>>>>> implementation?
> >> > > >> >>>>>>>>>> Has this been done for backwards compatibility reasons
> >> with
> >> > > the
> >> > > >> >>> old
> >> > > >> >>>>>>>> source
> >> > > >> >>>>>>>>>> interface? If not, then one could define a
> Collection<E>
> >> > > >> >>>>>>>> getNextRecords()
> >> > > >> >>>>>>>>>> method which returns the currently retrieved records
> and
> >> > then
> >> > > >> >> the
> >> > > >> >>>>>>> caller
> >> > > >> >>>>>>>>>> emits them outside of the SourceReader. That way the
> >> > > interface
> >> > > >> >>> would
> >> > > >> >>>>>>> not
> >> > > >> >>>>>>>>>> allow to implement an outputting loop where we never
> >> hand
> >> > > back
> >> > > >> >>>>>> control
> >> > > >> >>>>>>>> to
> >> > > >> >>>>>>>>>> the caller. At the moment, this contract can be easily
> >> > broken
> >> > > >> >> and
> >> > > >> >>> is
> >> > > >> >>>>>>>> only
> >> > > >> >>>>>>>>>> mentioned loosely in the JavaDocs.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Cheers,
> >> > > >> >>>>>>>>>> Till
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <
> >> > > >> >>> jingsonglee0@gmail.com
> >> > > >> >>>>>>>> <ma...@gmail.com>> <
> jingsonglee0@gmail.com
> >> > > >> <mailto:
> >> > > >> >>>>>>>> jingsonglee0@gmail.com>>
> >> > > >> >>>>>>>>>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi all,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I think current design is good.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> My understanding is:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> For execution mode: bounded mode and continuous mode,
> >> It's
> >> > > >> >> totally
> >> > > >> >>>>>>>>>> different. I don't think we have the ability to
> >> integrate
> >> > the
> >> > > >> >> two
> >> > > >> >>>>>>> models
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> at
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> present. It's about scheduling, memory, algorithms,
> >> States,
> >> > > >> etc.
> >> > > >> >>> we
> >> > > >> >>>>>>>>>> shouldn't confuse them.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> For source capabilities: only bounded, only
> continuous,
> >> > both
> >> > > >> >>> bounded
> >> > > >> >>>>>>> and
> >> > > >> >>>>>>>>>> continuous.
> >> > > >> >>>>>>>>>> I think Kafka is a source that can be ran both bounded
> >> > > >> >>>>>>>>>> and continuous execution mode.
> >> > > >> >>>>>>>>>> And Kafka with end offset should be ran both bounded
> >> > > >> >>>>>>>>>> and continuous execution mode.  Using apache Beam with
> >> > Flink
> >> > > >> >>>>>> runner, I
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> used
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> to run a "bounded" Kafka in streaming mode. For our
> >> > previous
> >> > > >> >>>>>>> DataStream,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> it
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> is not necessarily required that the source cannot be
> >> > > bounded.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> So it is my thought for Dawid's question:
> >> > > >> >>>>>>>>>> 1.pass a bounded source to continuousSource() +1
> >> > > >> >>>>>>>>>> 2.pass a continuous source to boundedSource() -1,
> should
> >> > > throw
> >> > > >> >>>>>>>> exception.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> In StreamExecutionEnvironment, continuousSource and
> >> > > >> >> boundedSource
> >> > > >> >>>>>>> define
> >> > > >> >>>>>>>>>> the execution mode. It defines a clear boundary of
> >> > execution
> >> > > >> >> mode.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Best,
> >> > > >> >>>>>>>>>> Jingsong Lee
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <
> >> imjark@gmail.com
> >> > > >> >>> <mailto:
> >> > > >> >>>>>>>> imjark@gmail.com>> <imjark@gmail.com <mailto:
> >> > imjark@gmail.com
> >> > > >>
> >> > > >> >>>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I agree with Dawid's point that the boundedness
> >> information
> >> > > >> >> should
> >> > > >> >>>>>>> come
> >> > > >> >>>>>>>>>> from the source itself (e.g. the end timestamp), not
> >> > through
> >> > > >> >>>>>>>>>> env.boundedSouce()/continuousSource().
> >> > > >> >>>>>>>>>> I think if we want to support something like
> >> `env.source()`
> >> > > >> that
> >> > > >> >>>>>>> derive
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> execution mode from source,
> >> > > `supportsBoundedness(Boundedness)`
> >> > > >> >>>>>>>>>> method is not enough, because we don't know whether it
> >> is
> >> > > >> >> bounded
> >> > > >> >>> or
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> not.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Best,
> >> > > >> >>>>>>>>>> Jark
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
> >> > > >> >>>>>> dwysakowicz@apache.org
> >> > > >> >>>>>>>> <ma...@apache.org>> <
> dwysakowicz@apache.org
> >> > > >> <mailto:
> >> > > >> >>>>>>>> dwysakowicz@apache.org>>
> >> > > >> >>>>>>>>>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> One more thing. In the current proposal, with the
> >> > > >> >>>>>>>>>> supportsBoundedness(Boundedness) method and the
> >> boundedness
> >> > > >> >> coming
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> from
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> either continuousSource or boundedSource I could not
> >> find
> >> > how
> >> > > >> >> this
> >> > > >> >>>>>>>>>> information is fed back to the SplitEnumerator.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Best,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Dawid
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On 09/12/2019 13:52, Becket Qin wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi Dawid,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks for the comments. This actually brings another
> >> > > relevant
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> question
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> about what does a "bounded source" imply. I actually
> had
> >> > the
> >> > > >> >> same
> >> > > >> >>>>>>>>>> impression when I look at the Source API. Here is
> what I
> >> > > >> >>> understand
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> after
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> some discussion with Stephan. The bounded source has
> the
> >> > > >> >> following
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> impacts.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 1. API validity.
> >> > > >> >>>>>>>>>> - A bounded source generates a bounded stream so some
> >> > > >> operations
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> only
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> works for bounded records would be performed, e.g.
> sort.
> >> > > >> >>>>>>>>>> - To expose these bounded stream only APIs, there are
> >> two
> >> > > >> >> options:
> >> > > >> >>>>>>>>>>       a. Add them to the DataStream API and throw
> >> exception
> >> > > if
> >> > > >> a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> method
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> is
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> called on an unbounded stream.
> >> > > >> >>>>>>>>>>       b. Create a BoundedDataStream class which is
> >> returned
> >> > > >> from
> >> > > >> >>>>>>>>>> env.boundedSource(), while DataStream is returned from
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> env.continousSource().
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Note that this cannot be done by having single
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> env.source(theSource)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> even
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the Source has a getBoundedness() method.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 2. Scheduling
> >> > > >> >>>>>>>>>> - A bounded source could be computed stage by stage
> >> without
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> bringing
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> up
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> all
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the tasks at the same time.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 3. Operator behaviors
> >> > > >> >>>>>>>>>> - A bounded source indicates the records are finite so
> >> some
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> operators
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> can
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> wait until it receives all the records before it
> starts
> >> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> processing.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> In the above impact, only 1 is relevant to the API
> >> design.
> >> > > And
> >> > > >> >> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> current
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> proposal in FLIP-27 is following 1.b.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> // boundedness depends of source property, imo this
> >> should
> >> > > >> >> always
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> be
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> preferred
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> In your proposal, does DataStream have bounded stream
> >> only
> >> > > >> >>> methods?
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> It
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> looks it should have, otherwise passing a bounded
> >> Source to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> env.source()
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> would be confusing. In that case, we will essentially
> do
> >> > 1.a
> >> > > if
> >> > > >> >> an
> >> > > >> >>>>>>>>>> unbounded Source is created from
> >> > env.source(unboundedSource).
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> If we have the methods only supported for bounded
> >> streams
> >> > in
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> DataStream,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> it
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> seems a little weird to have a separate
> >> BoundedDataStream
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> interface.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Am I understand it correctly?
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> dwysakowicz@apache.org <mailto:dwysakowicz@apache.org
> >>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi all,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Really well written proposal and very important one. I
> >> must
> >> > > >> >> admit
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> have
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> not understood all the intricacies of it yet.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> One question I have though is about where does the
> >> > > information
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> about
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> boundedness come from. I think in most cases it is a
> >> > property
> >> > > >> of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> source. As you described it might be e.g. end offset,
> a
> >> > flag
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> should
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> it
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> monitor new splits etc. I think it would be a really
> >> nice
> >> > use
> >> > > >> >> case
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> be
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> able to say:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> new KafkaSource().readUntil(long timestamp),
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> which could work as an "end offset". Moreover I think
> >> all
> >> > > >> >> Bounded
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> sources
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> support continuous mode, but no intrinsically
> continuous
> >> > > source
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> support
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Bounded mode. If I understood the proposal correctly
> it
> >> > > suggest
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> boundedness sort of "comes" from the outside of the
> >> source,
> >> > > >> from
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> invokation of either boundedStream or continousSource.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I am wondering if it would make sense to actually
> change
> >> > the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> method
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> boolean Source#supportsBoundedness(Boundedness)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Boundedness Source#getBoundedness().
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> As for the methods #boundedSource, #continousSource,
> >> > assuming
> >> > > >> >> the
> >> > > >> >>>>>>>>>> boundedness is property of the source they do not
> affect
> >> > how
> >> > > >> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> enumerator
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> works, but mostly how the dag is scheduled, right? I
> am
> >> not
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> against
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> those
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> methods, but I think it is a very specific use case to
> >> > > actually
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> override
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the property of the source. In general I would expect
> >> users
> >> > > to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> only
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> call
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> env.source(theSource), where the source tells if it is
> >> > > bounded
> >> > > >> >> or
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> not. I
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> would suggest considering following set of methods:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> // boundedness depends of source property, imo this
> >> should
> >> > > >> >> always
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> be
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> preferred
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> // always continous execution, whether bounded or
> >> unbounded
> >> > > >> >> source
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> DataStream<MyType> boundedStream =
> >> > > >> >> env.continousSource(theSource);
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> // imo this would make sense if the BoundedDataStream
> >> > > provides
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> additional features unavailable for continous mode
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> BoundedDataStream<MyType> batch =
> >> > > env.boundedSource(theSource);
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Best,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Dawid
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks, Becket, for updating this.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I agree with moving the aspects you mentioned into
> >> separate
> >> > > >> >> FLIPs
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> -
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> this
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> one way becoming unwieldy in size.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> +1 to the FLIP in its current state. Its a very
> detailed
> >> > > >> >> write-up,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> nicely
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> done!
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <
> >> > > >> becket.qin@gmail.com
> >> > > >> >>>>>>>> <ma...@gmail.com>> <becket.qin@gmail.com
> >> > <mailto:
> >> > > >> >>>>>>>> becket.qin@gmail.com>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> becket.qin@gmail.com <ma...@gmail.com>>
> >> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi all,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Sorry for the long belated update. I have updated
> >> FLIP-27
> >> > > wiki
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> page
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> with
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the latest proposals. Some noticeable changes include:
> >> > > >> >>>>>>>>>> 1. A new generic communication mechanism between
> >> > > >> SplitEnumerator
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> SourceReader.
> >> > > >> >>>>>>>>>> 2. Some detail API method signature changes.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> We left a few things out of this FLIP and will address
> >> them
> >> > > in
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> separate
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> FLIPs. Including:
> >> > > >> >>>>>>>>>> 1. Per split event time.
> >> > > >> >>>>>>>>>> 2. Event time alignment.
> >> > > >> >>>>>>>>>> 3. Fine grained failover for SplitEnumerator failure.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Please let us know if you have any question.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <
> >> > > sewen@apache.org
> >> > > >> >>>>>>> <mailto:
> >> > > >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:
> >> > sewen@apache.org
> >> > > >>
> >> > > >> <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi  Łukasz!
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Becket and me are working hard on figuring out the
> last
> >> > > details
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> implementing the first PoC. We would update the FLIP
> >> > > hopefully
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> next
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> week.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> There is a fair chance that a first version of this
> >> will be
> >> > > in
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 1.10,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> but
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> think it will take another release to battle test it
> and
> >> > > >> migrate
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> connectors.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Best,
> >> > > >> >>>>>>>>>> Stephan
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <
> >> > > >> >> ljd@touk.pl
> >> > > >> >>>>>>>> <ma...@touk.pl>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> ljd@touk.pl <ma...@touk.pl>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> This proposal looks very promising for us. Do you have
> >> any
> >> > > >> plans
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> in
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> which
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Flink release it is going to be released? We are
> >> thinking
> >> > on
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> using a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Data
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Set API for our future use cases but on the other hand
> >> Data
> >> > > Set
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> API
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> is
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> going to be deprecated so using proposed bounded data
> >> > streams
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> solution
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> could be more viable in the long term.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks,
> >> > > >> >>>>>>>>>> Łukasz
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On 2019/10/01 15:48:03, Thomas Weise <
> >> > thomas.weise@gmail.com
> >> > > >> >>>>>> <mailto:
> >> > > >> >>>>>>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com
> <mailto:
> >> > > >> >>>>>>>> thomas.weise@gmail.com>> <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> thomas.weise@gmail.com <mailto:thomas.weise@gmail.com
> >>
> >> > > wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks for putting together this proposal!
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I see that the "Per Split Event Time" and "Event Time
> >> > > >> Alignment"
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> sections
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> are still TBD.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> It would probably be good to flesh those out a bit
> >> before
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> proceeding
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> too
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> far
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> as the event time alignment will probably influence
> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> interaction
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> with
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the split reader, specifically ReaderStatus
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> emitNext(SourceOutput<E>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> output).
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> We currently have only one implementation for event
> time
> >> > > >> >> alignment
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> in
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Kinesis consumer. The synchronization in that case
> takes
> >> > > place
> >> > > >> >> as
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> last
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> step before records are emitted downstream
> >> (RecordEmitter).
> >> > > >> With
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> currently proposed interfaces, the equivalent can be
> >> > > >> implemented
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> in
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> reader loop, although note that in the Kinesis
> consumer
> >> the
> >> > > per
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> shard
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> threads push records.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Synchronization has not been implemented for the Kafka
> >> > > consumer
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> yet.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
> >> > > >> >>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> When I looked at it, I realized that the
> implementation
> >> > will
> >> > > >> >> look
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> quite
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> different
> >> > > >> >>>>>>>>>> from Kinesis because it needs to take place in the
> pull
> >> > part,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> where
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> records
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> are taken from the Kafka client. Due to the
> >> multiplexing it
> >> > > >> >> cannot
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> be
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> done
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> by blocking the split thread like it currently works
> for
> >> > > >> >> Kinesis.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Reading
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> from individual Kafka partitions needs to be
> controlled
> >> via
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> pause/resume
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> on the Kafka client.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> To take on that responsibility the split thread would
> >> need
> >> > to
> >> > > >> be
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> aware
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>> watermarks or at least whether it should or should not
> >> > > continue
> >> > > >> >> to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> consume
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> a given split and this may require a different
> >> SourceReader
> >> > > or
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> SourceOutput
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> interface.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks,
> >> > > >> >>>>>>>>>> Thomas
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <
> >> > mmyy1110@gmail.com
> >> > > >> >>>>>> <mailto:
> >> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> > > >> >> mmyy1110@gmail.com
> >> > > >> >>>>>
> >> > > >> >>>>>> <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi Stephan,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thank you for feedback!
> >> > > >> >>>>>>>>>> Will take a look at your branch before public
> >> discussing.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <
> >> > > >> sewen@apache.org
> >> > > >> >>>>>>>> <ma...@apache.org>> <sewen@apache.org <mailto:
> >> > > >> >>> sewen@apache.org
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi Biao!
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks for reviving this. I would like to join this
> >> > > discussion,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> but
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> am
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> quite occupied with the 1.9 release, so can we maybe
> >> pause
> >> > > this
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> discussion
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> for a week or so?
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> In the meantime I can share some suggestion based on
> >> prior
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> experiments:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> How to do watermarks / timestamp extractors in a
> simpler
> >> > and
> >> > > >> >> more
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> flexible
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> way. I think that part is quite promising should be
> >> part of
> >> > > the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> new
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> source
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> interface.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> >> > > >> >>>>>>>> <
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> >> > > >> >>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> >> > > >> >>>>>>>> <
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> >> > > >> >>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Some experiments on how to build the source reader and
> >> its
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> library
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> for
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> common threading/split patterns:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> >> > > >> >>>>>>>> <
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> >> > > >> >>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Best,
> >> > > >> >>>>>>>>>> Stephan
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <
> >> > > mmyy1110@gmail.com
> >> > > >> >>>>>>> <mailto:
> >> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> > > >> >> mmyy1110@gmail.com
> >> > > >> >>>>>
> >> > > >> >>>>>> <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi devs,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Since 1.9 is nearly released, I think we could get
> back
> >> to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> FLIP-27.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> believe it should be included in 1.10.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> There are so many things mentioned in document of
> >> FLIP-27.
> >> > > [1]
> >> > > >> I
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> think
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> we'd better discuss them separately. However the wiki
> is
> >> > not
> >> > > a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> good
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> place
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> to discuss. I wrote google doc about SplitReader API
> >> which
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> misses
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> some
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> details in the document. [2]
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 1.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >> > > >> >>>>>>>> <
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >> > > >> >>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 2.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> >> > > >> >>>>>>>> <
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> >> > > >> >>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <
> >> > mmyy1110@gmail.com
> >> > > >> >>>>>> <mailto:
> >> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> > > >> >> mmyy1110@gmail.com
> >> > > >> >>>>>
> >> > > >> >>>>>> <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi Steven,
> >> > > >> >>>>>>>>>> Thank you for the feedback. Please take a look at the
> >> > > document
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> FLIP-27
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >> > > >> >>>>>>>> <
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >>
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >> > > >> >>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> which
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> is updated recently. A lot of details of enumerator
> were
> >> > > added
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> in
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> this
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> document. I think it would help.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Steven Wu <stevenz3wu@gmail.com <mailto:
> >> > stevenz3wu@gmail.com
> >> > > >>
> >> > > >> >> <
> >> > > >> >>>>>>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
> >> > > >> >>>>>>> stevenz3wu@gmail.com
> >> > > >> >>>>>>>> <ma...@gmail.com>> <stevenz3wu@gmail.com
> >> > <mailto:
> >> > > >> >>>>>>>> stevenz3wu@gmail.com>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 于2019年3月28日周四
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 下午12:52写道:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> This proposal mentioned that SplitEnumerator might run
> >> on
> >> > the
> >> > > >> >>>>>>>>>> JobManager or
> >> > > >> >>>>>>>>>> in a single task on a TaskManager.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> if enumerator is a single task on a taskmanager, then
> >> the
> >> > job
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> DAG
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> can
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> never
> >> > > >> >>>>>>>>>> been embarrassingly parallel anymore. That will
> nullify
> >> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> leverage
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> fine-grained recovery for embarrassingly parallel
> jobs.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> It's not clear to me what's the implication of running
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> enumerator
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> on
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> jobmanager. So I will leave that out for now.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <
> >> > mmyy1110@gmail.com
> >> > > >> >>>>>> <mailto:
> >> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> > > >> >> mmyy1110@gmail.com
> >> > > >> >>>>>
> >> > > >> >>>>>> <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi Stephan & Piotrek,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thank you for feedback.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> It seems that there are a lot of things to do in
> >> community.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> am
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> just
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> afraid that this discussion may be forgotten since
> >> there so
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> many
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> proposals
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> recently.
> >> > > >> >>>>>>>>>> Anyway, wish to see the split topics soon :)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:
> >> > > >> >>> piotr@da-platform.com
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>> <
> >> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
> >> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
> >> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 于2019年1月24日周四
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 下午8:21写道:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi Biao!
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> This discussion was stalled because of preparations
> for
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> open
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> sourcing
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> & merging Blink. I think before creating the tickets
> we
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> should
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> split this
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> discussion into topics/areas outlined by Stephan and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> create
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Flips
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> for
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I think there is no chance for this to be completed in
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> couple
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> remaining
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> weeks/1 month before 1.8 feature freeze, however it
> >> would
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> be
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> good
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> to aim
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> with those changes for 1.9.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Piotrek
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <
> mmyy1110@gmail.com
> >> > > >> <mailto:
> >> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> > > >> >> mmyy1110@gmail.com
> >> > > >> >>>>>
> >> > > >> >>>>>> <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi community,
> >> > > >> >>>>>>>>>> The summary of Stephan makes a lot sense to me. It is
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> much
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> clearer
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> indeed
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> after splitting the complex topic into small ones.
> >> > > >> >>>>>>>>>> I was wondering is there any detail plan for next
> step?
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> If
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> not,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> would
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> like to push this thing forward by creating some JIRA
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> issues.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Another question is that should version 1.8 include
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> these
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> features?
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Stephan Ewen <sewen@apache.org <mailto:
> sewen@apache.org
> >> >>
> >> > <
> >> > > >> >>>>>>>> sewen@apache.org <ma...@apache.org>> <
> >> > sewen@apache.org
> >> > > >> >>>> <mailto:
> >> > > >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:
> >> > sewen@apache.org
> >> > > >>
> >> > > >> >>>>>>>> 于2018年12月1日周六
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 上午4:20写道:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks everyone for the lively discussion. Let me try
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> summarize
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> where I
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> see convergence in the discussion and open issues.
> >> > > >> >>>>>>>>>> I'll try to group this by design aspect of the source.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Please
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> let me
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> know
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> if I got things wrong or missed something crucial
> here.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> For issues 1-3, if the below reflects the state of the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> discussion, I
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> would
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> try and update the FLIP in the next days.
> >> > > >> >>>>>>>>>> For the remaining ones we need more discussion.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> I would suggest to fork each of these aspects into a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> separate
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> mail
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> thread,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> or will loose sight of the individual aspects.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> *(1) Separation of Split Enumerator and Split Reader*
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - All seem to agree this is a good thing
> >> > > >> >>>>>>>>>> - Split Enumerator could in the end live on JobManager
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> (and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> assign
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> splits
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> via RPC) or in a task (and assign splits via data
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> streams)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - this discussion is orthogonal and should come later,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> when
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> interface
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> is agreed upon.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> *(2) Split Readers for one or more splits*
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Discussion seems to agree that we need to support
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> one
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> reader
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> possibly handles multiple splits concurrently.
> >> > > >> >>>>>>>>>> - The requirement comes from sources where one
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> poll()-style
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> call
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> fetches
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> data from different splits / partitions
> >> > > >> >>>>>>>>>>     --> example sources that require that would be for
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> example
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Kafka,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Pravega, Pulsar
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Could have one split reader per source, or multiple
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> split
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> readers
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> share the "poll()" function
> >> > > >> >>>>>>>>>> - To not make it too complicated, we can start with
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> thinking
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> about
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> one
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> split reader for all splits initially and see if that
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> covers
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> all
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> requirements
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> *(3) Threading model of the Split Reader*
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Most active part of the discussion ;-)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - A non-blocking way for Flink's task code to interact
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> with
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> source
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> is
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> needed in order to a task runtime code based on a
> >> > > >> >>>>>>>>>> single-threaded/actor-style task design
> >> > > >> >>>>>>>>>>     --> I personally am a big proponent of that, it
> will
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> help
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> with
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> well-behaved checkpoints, efficiency, and simpler yet
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> more
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> robust
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> runtime
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> code
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Users care about simple abstraction, so as a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> subclass
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> SplitReader
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> (non-blocking / async) we need to have a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> BlockingSplitReader
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> which
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> will
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> form the basis of most source implementations.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> BlockingSplitReader
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> lets
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> users do blocking simple poll() calls.
> >> > > >> >>>>>>>>>> - The BlockingSplitReader would spawn a thread (or
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> more)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> thread(s) can make blocking calls and hand over data
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> buffers
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> via
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> blocking
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> queue
> >> > > >> >>>>>>>>>> - This should allow us to cover both, a fully async
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> runtime,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> simple
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> blocking interface for users.
> >> > > >> >>>>>>>>>> - This is actually very similar to how the Kafka
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> connectors
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> work.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Kafka
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 9+ with one thread, Kafka 8 with multiple threads
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - On the base SplitReader (the async one), the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> non-blocking
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> method
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> gets the next chunk of data would signal data
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> availability
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> via
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> CompletableFuture, because that gives the best
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> flexibility
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> (can
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> await
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> completion or register notification handlers).
> >> > > >> >>>>>>>>>> - The source task would register a "thenHandle()" (or
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> similar)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> on the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> future to put a "take next data" task into the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> actor-style
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> mailbox
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> *(4) Split Enumeration and Assignment*
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Splits may be generated lazily, both in cases where
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> there
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> is a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> limited
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> number of splits (but very many), or splits are
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> discovered
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> over
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> time
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Assignment should also be lazy, to get better load
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> balancing
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Assignment needs support locality preferences
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Possible design based on discussion so far:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>     --> SplitReader has a method
> "addSplits(SplitT...)"
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> add
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> one or
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> more
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> splits. Some split readers might assume they have only
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> one
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> split
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> ever,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> concurrently, others assume multiple splits. (Note:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> idea
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> behind
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> being
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> able
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> to add multiple splits at the same time is to ease
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> startup
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> where
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> multiple
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> splits may be assigned instantly.)
> >> > > >> >>>>>>>>>>     --> SplitReader has a context object on which it
> can
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> call
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> indicate
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> when
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> splits are completed. The enumerator gets that
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> notification and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> can
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> use
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> decide when to assign new splits. This should help
> both
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> in
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> cases
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> sources
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that take splits lazily (file readers) and in case the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> source
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> needs to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> preserve a partial order between splits (Kinesis,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Pravega,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Pulsar may
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> need
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that).
> >> > > >> >>>>>>>>>>     --> SplitEnumerator gets notification when
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> SplitReaders
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> start
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> when
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> they finish splits. They can decide at that moment to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> push
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> more
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> splits
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that reader
> >> > > >> >>>>>>>>>>     --> The SplitEnumerator should probably be aware
> of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> source
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> parallelism, to build its initial distribution.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Open question: Should the source expose something
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> like
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> "host
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> preferences", so that yarn/mesos/k8s can take this
> into
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> account
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> when
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> selecting a node to start a TM on?
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> *(5) Watermarks and event time alignment*
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Watermark generation, as well as idleness, needs to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> be
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> per
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> split
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> (like
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> currently in the Kafka Source, per partition)
> >> > > >> >>>>>>>>>> - It is desirable to support optional
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> event-time-alignment,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> meaning
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> splits that are ahead are back-pressured or
> temporarily
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> unsubscribed
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - I think i would be desirable to encapsulate
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> watermark
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> generation
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> logic
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> in watermark generators, for a separation of concerns.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> The
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> watermark
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> generators should run per split.
> >> > > >> >>>>>>>>>> - Using watermark generators would also help with
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> another
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> problem of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> suggested interface, namely supporting non-periodic
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> watermarks
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> efficiently.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Need a way to "dispatch" next record to different
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> watermark
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> generators
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Need a way to tell SplitReader to "suspend" a split
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> until a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> certain
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> watermark is reached (event time backpressure)
> >> > > >> >>>>>>>>>> - This would in fact be not needed (and thus simpler)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> if
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> we
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> had
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> SplitReader per split and may be a reason to re-open
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> discussion
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> *(6) Watermarks across splits and in the Split
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Enumerator*
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - The split enumerator may need some watermark
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> awareness,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> which
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> should
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> be
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> purely based on split metadata (like create timestamp
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> file
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> splits)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - If there are still more splits with overlapping
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> event
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> time
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> range
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> for
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> split reader, then that split reader should not
> advance
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> watermark
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> within the split beyond the overlap boundary.
> Otherwise
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> future
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> splits
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> will
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> produce late data.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - One way to approach this could be that the split
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> enumerator
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> may
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> send
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> watermarks to the readers, and the readers cannot emit
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> watermarks
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> beyond
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that received watermark.
> >> > > >> >>>>>>>>>> - Many split enumerators would simply immediately send
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Long.MAX
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> out
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> leave the progress purely to the split readers.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - For event-time alignment / split back pressure, this
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> begs
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> question
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> how we can avoid deadlocks that may arise when splits
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> are
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> suspended
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> for
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> event time back pressure,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> *(7) Batch and streaming Unification*
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Functionality wise, the above design should support
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> both
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Batch often (mostly) does not care about reading "in
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> order"
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> generating watermarks
> >> > > >> >>>>>>>>>>     --> Might use different enumerator logic that is
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> more
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> locality
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> aware
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and ignores event time order
> >> > > >> >>>>>>>>>>     --> Does not generate watermarks
> >> > > >> >>>>>>>>>> - Would be great if bounded sources could be
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> identified
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> at
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> compile
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> time,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> so that "env.addBoundedSource(...)" is type safe and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> can
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> return a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> "BoundedDataStream".
> >> > > >> >>>>>>>>>> - Possible to defer this discussion until later
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> *Miscellaneous Comments*
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - Should the source have a TypeInformation for the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> produced
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> type,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> instead
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> of a serializer? We need a type information in the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> stream
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> anyways, and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> can
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> derive the serializer from that. Plus, creating the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> serializer
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> should
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> respect the ExecutionConfig.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> - The TypeSerializer interface is very powerful but
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> also
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> not
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> easy to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> implement. Its purpose is to handle data super
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> efficiently,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> support
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> flexible ways of evolution, etc.
> >> > > >> >>>>>>>>>> For metadata I would suggest to look at the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> SimpleVersionedSerializer
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> instead, which is used for example for checkpoint
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> master
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> hooks,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> or for
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> streaming file sink. I think that is is a good match
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> for
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> cases
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> where
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> we
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> do
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> not need more than ser/deser (no copy, etc.) and don't
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> need to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> push
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> versioning out of the serialization paths for best
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> performance
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> (as in
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> TypeSerializer)
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> k.kloudas@data-artisans.com>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> wrote:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Hi Biao,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Thanks for the answer!
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> So given the multi-threaded readers, now we have as
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> open
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> questions:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 1) How do we let the checkpoints pass through our
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> multi-threaded
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> reader
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> operator?
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 2) Do we have separate reader and source operators or
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> not? In
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> strategy
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that has a separate source, the source operator has a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> parallelism of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 1
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> is responsible for split recovery only.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> For the first one, given also the constraints
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> (blocking,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> finite
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> queues,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> etc), I do not have an answer yet.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> For the 2nd, I think that we should go with separate
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> operators
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> for
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> source and the readers, for the following reasons:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 1) This is more aligned with a potential future
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> improvement
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> where the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> split
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> discovery becomes a responsibility of the JobManager
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> readers are
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> pooling more work from the JM.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> 2) The source is going to be the "single point of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> truth".
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> It
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> will
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> know
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> what
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> has been processed and what not. If the source and the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> readers
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> are a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> single
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> operator with parallelism > 1, or in general, if the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> split
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> discovery
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> is
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> done by each task individually, then:
> >> > > >> >>>>>>>>>>    i) we have to have a deterministic scheme for each
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> reader to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> assign
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> splits to itself (e.g. mod subtaskId). This is not
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> necessarily
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> trivial
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> for
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> all sources.
> >> > > >> >>>>>>>>>>    ii) each reader would have to keep a copy of all
> its
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> processed
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> slpits
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>    iii) the state has to be a union state with a
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> non-trivial
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> merging
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> logic
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> in order to support rescaling.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Two additional points that you raised above:
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> i) The point that you raised that we need to keep all
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> splits
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> (processed
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> not-processed) I think is a bit of a strong
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> requirement.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> This
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> would
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> imply
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that for infinite sources the state will grow
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> indefinitely.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> This is
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> problem
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> is even more pronounced if we do not have a single
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> source
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> assigns
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> splits to readers, as each reader will have its own
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> copy
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> of
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> state.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> ii) it is true that for finite sources we need to
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> somehow
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> not
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> close
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> readers when the source/split discoverer finishes. The
> >> > > >> >>>>>>>>>> ContinuousFileReaderOperator has a work-around for
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> that.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> It is
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> not
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> elegant,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> and checkpoints are not emitted after closing the
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> source,
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> but
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> this, I
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> believe, is a bigger problem which requires more
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> changes
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> than
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> just
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> refactoring the source interface.
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> Cheers,
> >> > > >> >>>>>>>>>> Kostas
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>>
> >> > > >> >>>>>>>>>> --
> >> > > >> >>>>>>>>>> Best, Jingsong Lee
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>>
> >> > > >> >>>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>>>
> >> > > >> >>>>>> --
> >> > > >> >>>>>> Best, Jingsong Lee
> >> > > >> >>>>>>
> >> > > >> >>>>>
> >> > > >> >>>>
> >> > > >> >>>>
> >> > > >> >>>
> >> > > >> >>
> >> > > >> >
> >> > > >>
> >> > > >>
> >> > >
> >> >
> >>
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi folks,

Happy new year!

Stephan and I chatted offline yesterday. After reading the email thread
again, I found that I have misunderstood Dawid's original proposal
regarding the behavior of env.source(BoundedSource) and had an incorrect
impression about the behavior of java covariant return type.
Anyways, I agree what Dawid originally proposed makes sense, which is the
following API:

// Return a BoundedDataStream instance if the source is bounded.
// Return a DataStream instance if the source is unbounded.
DataStream env.source(Source);

// Throws exception if the source is unbounded.
// Used when users knows the source is bounded at programming time.
BoundedDataStream env.boundedSource(Source);

A BoundedDataStream only runs in batch execution mode.
A DataStream only runs in streaming execution mode.

To run a bounded source in streaming execution mode, one would do the
following:

// Return a DataStream instance with a source that will stop at some point;
DataStream env.source(SourceUtils.asUnbounded(myBoundedSource));

I'll update the FLIP wiki and resume the vote if there is no further
concerns.

Apologies for the misunderstanding and thanks for all the patient
discussions.

Thanks,

Jiangjie (Becket) Qin



On Mon, Dec 23, 2019 at 8:00 AM Becket Qin <be...@gmail.com> wrote:

> Hi Steven,
>
> I think the current proposal is what you mentioned - a Kafka source that
> can be constructed in either BOUNDED or UNBOUNDED mode. And Flink can get
> the boundedness by invoking getBoundedness().
>
> So one can create a Kafka source by doing something like the following:
>
> new KafkaSource().startOffset(),endOffset(); // A bounded instance.
> new KafkaSource().startOffset(); // An unbounded instance.
>
> If users want to have an UNBOUNDED Kafka source that stops at some point.
> They can wrap the BOUNDED Kafka source like below:
>
> SourceUtils.asUnbounded(new KafkaSource.startOffset().endOffset());
>
> The wrapped source would be an unbounded Kafka source that stops at the
> end offset.
>
> Does that make sense?
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Fri, Dec 20, 2019 at 1:31 PM Jark Wu <im...@gmail.com> wrote:
>
>> Hi,
>>
>> First of all, I think it is not called "UNBOUNDED", according to the
>> FLIP-27, it is called "CONTINUOUS_UNBOUNDED".
>> And from the description of the Boundedness in the FLIP-27[1] declares
>> clearly what Becket and I think.
>>
>> public enum Boundedness {
>>
>>     /**
>>      * A bounded source processes the data that is currently available and
>> will end after that.
>>      *
>>      * <p>When a source produces a bounded stream, the runtime may
>> activate
>> additional optimizations
>>      * that are suitable only for bounded input. Incorrectly producing
>> unbounded data when the source
>>      * is set to produce a bounded stream will often result in programs
>> that do not output any results
>>      * and may eventually fail due to runtime errors (out of memory or
>> storage).
>>      */
>>     BOUNDED,
>>
>>     /**
>>      * A continuous unbounded source continuously processes all data as it
>> comes.
>>      *
>>      * <p>The source may run forever (until the program is terminated) or
>> might actually end at some point,
>>      * based on some source-specific conditions. Because that is not
>> transparent to the runtime,
>>      * the runtime will use an execution mode for continuous unbounded
>> streams whenever this mode
>>      * is chosen.
>>      */
>>     CONTINUOUS_UNBOUNDED
>> }
>>
>> Best,
>> Jark
>>
>> [1]:
>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface#FLIP-27:RefactorSourceInterface-Source
>>
>>
>>
>> On Fri, 20 Dec 2019 at 12:55, Steven Wu <st...@gmail.com> wrote:
>>
>> > Becket,
>> >
>> > Regarding "UNBOUNDED source that stops at some point", I found it
>> difficult
>> > to grasp what UNBOUNDED really mean.
>> >
>> > If we want to use Kafka source with an end/stop time, I guess you call
>> it
>> > UNBOUNDED kafka source that stops (aka BOUNDED-streaming). The
>> > terminology is a little confusing to me. Maybe BOUNDED/UNBOUNDED
>> shouldn't
>> > be used to categorize source. Just call it Kafka source and it can run
>> in
>> > either BOUNDED or UNBOUNDED mode.
>> >
>> > Thanks,
>> > Steven
>> >
>> > On Thu, Dec 19, 2019 at 7:02 PM Becket Qin <be...@gmail.com>
>> wrote:
>> >
>> > > I had an offline chat with Jark, and here are some more thoughts:
>> > >
>> > > 1. From SQL perspective, BOUNDED source leads to the batch execution
>> > mode,
>> > > UNBOUNDED source leads to the streaming execution mode.
>> > > 2. The semantic of UNBOUNDED source is may or may not stop. The
>> semantic
>> > of
>> > > BOUNDED source is will stop.
>> > > 3. The semantic of DataStream is may or may not terminate. The
>> semantic
>> > of
>> > > BoundedDataStream is will terminate.
>> > >
>> > > Given that, option 3 seems a better option because:
>> > > 1. SQL already has strict binding between Boundedness and execution
>> mode.
>> > > Letting DataStream be consistent would be good.
>> > > 2. The semantic of UNBOUNDED source is exactly the same as
>> DataStream. So
>> > > we should avoid breaking such semantic, i.e. turning some DataStream
>> from
>> > > "may or may not terminate" to "will terminate".
>> > >
>> > > For case where users want BOUNDED-streaming combination, they can
>> simply
>> > > use an UNBOUNDED source that stops at some point. We can even provide
>> a
>> > > simple wrapper to wrap a BOUNDED source as an UNBOUNDED source if that
>> > > helps. But API wise, option 3 seems telling a pretty good whole story.
>> > >
>> > > Thanks,
>> > >
>> > > Jiangjie (Becket) Qin
>> > >
>> > >
>> > >
>> > >
>> > > On Thu, Dec 19, 2019 at 10:30 PM Becket Qin <be...@gmail.com>
>> > wrote:
>> > >
>> > > > Hi Timo,
>> > > >
>> > > > Bounded is just a special case of unbounded and every bounded source
>> > can
>> > > >> also be treated as an unbounded source. This would unify the API if
>> > > >> people don't need a bounded operation.
>> > > >
>> > > >
>> > > > With option 3 users can still get a unified API with something like
>> > > below:
>> > > >
>> > > > DataStream boundedStream = env.boundedSource(boundedSource);
>> > > > DataStream unboundedStream = env.source(unboundedSource);
>> > > >
>> > > > So in both cases, users can still use a unified DataStream without
>> > > > touching the bounded stream only methods.
>> > > > By "unify the API if people don't need the bounded operation". Do
>> you
>> > > > expect a DataStream with a Bounded source to have the batch
>> operators
>> > and
>> > > > scheduler settings as well?
>> > > >
>> > > >
>> > > > If we allow DataStream from BOUNDED source, we will essentially pick
>> > > "*modified
>> > > > option 2*".
>> > > >
>> > > > // The source is either bounded or unbounded, but only unbounded
>> > > >> operations could be performed on the returned DataStream.
>> > > >> DataStream<Type> dataStream = env.source(someSource);
>> > > >
>> > > >
>> > > >> // The source must be a bounded source, otherwise exception is
>> thrown.
>> > > >> BoundedDataStream<Type> boundedDataStream =
>> > > >> env.boundedSource(boundedSource);
>> > > >
>> > > >
>> > > >
>> > > > // Add the following method to DataStream
>> > > >
>> > > > Boundedness DataStream#getBoundedness();
>> > > >
>> > > >
>> > > > From pure logical perspective, Boundedness and runtime settings
>> > > > (Stream/Batch) are two orthogonal dimensions. And are specified in
>> the
>> > > > following way.
>> > > >
>> > > > *Boundedness* - defined by the source: BOUNDED / UNBOUNDED.
>> > > > *Running mode* - defined by the API class: DataStream (Streaming
>> mode)
>> > /
>> > > > BoundedDataStream (batch mode).
>> > > >
>> > > > Excluding the UNBOUNDED-batch combination, the "*modified option 2"*
>> > > > covers the rest three combination. Compared with "*modified option
>> 2*",
>> > > > the main benefit of option 3 is its simplicity and clearness, by
>> tying
>> > > > boundedness to running mode and giving up BOUNDED-streaming
>> > combination.
>> > > >
>> > > > Just to be clear, I am fine with either option. But I would like to
>> > > > understand a bit more about the bounded-streaming use case and when
>> > users
>> > > > would prefer this over bounded-batch case, and whether the added
>> value
>> > > > justifies the additional complexity in the API. Two cases I can
>> think
>> > of
>> > > > are:
>> > > > 1. The records in DataStream will be processed in order, while
>> > > > BoundedDataStream processes records without order guarantee.
>> > > > 2. DataStream emits intermediate results when processing a finite
>> > > dataset,
>> > > > while BoundedDataStream only emit the final result. In any case, it
>> > could
>> > > > be supported by an UNBOUNDED source stopping at some point.
>> > > >
>> > > > Case 1 is actually misleading because DataStream in general doesn't
>> > > really
>> > > > support in-order process.
>> > > > Case 2 seems a rare use case because the instantaneous intermediate
>> > > result
>> > > > seems difficult to reason about. In any case, this can be supported
>> by
>> > an
>> > > > UNBOUNDED source that stops at some point.
>> > > >
>> > > > Is there other use cases for bounded-streaming combination I
>> missed? I
>> > am
>> > > > a little hesitating to put the testing requirement here because
>> ideally
>> > > I'd
>> > > > avoid having public APIs for testing purpose only. And this could be
>> > > > resolved by having a UNBOUNDED source stopping at some point as
>> well.
>> > > >
>> > > > Sorry for the long discussion, but I would really like to make an
>> API
>> > > > decision after knowing all the pros and cons.
>> > > >
>> > > > Thanks,
>> > > >
>> > > > Jiangjie (Becket) Qin
>> > > >
>> > > >
>> > > >
>> > > >
>> > > >
>> > > >
>> > > >
>> > > > On Thu, Dec 19, 2019 at 6:19 PM Timo Walther <tw...@apache.org>
>> > wrote:
>> > > >
>> > > >> Hi Becket,
>> > > >>
>> > > >> regarding *Option 3* I think we can relax the constraints for
>> > > >> env.source():
>> > > >>
>> > > >> // MySource can be bounded or unbounded
>> > > >> DataStream<Type> dataStream = env.source(mySource);
>> > > >>
>> > > >> // MySource must be bounded, otherwise throws exception.
>> > > >> BoundedDataStream<Type> boundedDataStream =
>> > env.boundedSource(mySource);
>> > > >>
>> > > >> Bounded is just a special case of unbounded and every bounded
>> source
>> > can
>> > > >> also be treated as an unbounded source. This would unify the API if
>> > > >> people don't need a bounded operation. It also addresses Jark's
>> > > concerns.
>> > > >>
>> > > >> Regards,
>> > > >> Timo
>> > > >>
>> > > >>
>> > > >> On 18.12.19 14:16, Becket Qin wrote:
>> > > >> > Hi Jark,
>> > > >> >
>> > > >> > Please see the reply below:
>> > > >> >
>> > > >> > Regarding to option#3, my concern is that if we don't support
>> > > streaming
>> > > >> >> mode for bounded source,
>> > > >> >> how could we create a testing source for streaming mode?
>> Currently,
>> > > >> all the
>> > > >> >> testing source for streaming
>> > > >> >> are bounded, so that the integration test will finish finally.
>> > > >> >
>> > > >> >
>> > > >> > An UNBOUNDED source does not mean it will never stops. It simply
>> > > >> indicates
>> > > >> > that the source *may* run forever, so the runtime needs to be
>> > prepared
>> > > >> for
>> > > >> > that, but the task may still stop at some point when it hits some
>> > > >> > source-specific condition. So an UNBOUNDED testing source can
>> still
>> > > >> stop at
>> > > >> > some point if needed.
>> > > >> >
>> > > >> > Regarding to Source#getRecordOrder(), could we have a implicit
>> > > contract
>> > > >> >> that unbounded source should
>> > > >> >> already read in order (i.e. reading partitions in parallel), for
>> > > >> bounded
>> > > >> >> source the order is not mandatory.
>> > > >> >
>> > > >> >
>> > > >> >
>> > > >> >> This is also the behaviors of the current sources.
>> > > >> >
>> > > >> > 1) a source can't guarantee it reads in strict order, because the
>> > > >> producer
>> > > >> >> may produce data not in order.
>> > > >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can
>> > reorder
>> > > >> data.
>> > > >> >
>> > > >> >
>> > > >> > It is true that sometimes the source cannot guarantee the record
>> > > order,
>> > > >> but
>> > > >> > sometimes it can. Right now, even for stream processing, there
>> is no
>> > > >> > processing order guarantee. For example, a join operator may
>> emit a
>> > > >> later
>> > > >> > record which successfully found a join match earlier.
>> > > >> > Event order is one of the most important requirements for event
>> > > >> processing,
>> > > >> > a clear order guarantee would be necessary. That said, I agree
>> that
>> > > >> right
>> > > >> > now even if the sources provide the record order requirement, the
>> > > >> runtime
>> > > >> > is not able to guarantee that out of the box. So I am OK if we
>> add
>> > the
>> > > >> > record order to the Source later. But we should avoid misleading
>> > users
>> > > >> to
>> > > >> > make them think the processing order is guaranteed when using the
>> > > >> unbounded
>> > > >> > runtime.
>> > > >> >
>> > > >> > Thanks,
>> > > >> >
>> > > >> > Jiangjie (Becket) Qin
>> > > >> >
>> > > >> > On Wed, Dec 18, 2019 at 10:29 AM Jark Wu <im...@gmail.com>
>> wrote:
>> > > >> >
>> > > >> >> Hi Becket,
>> > > >> >>
>> > > >> >> That's great we have reached a consensus on
>> > Source#getBoundedness().
>> > > >> >>
>> > > >> >> Regarding to option#3, my concern is that if we don't support
>> > > streaming
>> > > >> >> mode for bounded source,
>> > > >> >> how could we create a testing source for streaming mode?
>> Currently,
>> > > >> all the
>> > > >> >> testing source for streaming
>> > > >> >> are bounded, so that the integration test will finish finally.
>> > > >> >>
>> > > >> >> Regarding to Source#getRecordOrder(), could we have a implicit
>> > > contract
>> > > >> >> that unbounded source should
>> > > >> >> already read in order (i.e. reading partitions in parallel), for
>> > > >> bounded
>> > > >> >> source the order is not mandatory.
>> > > >> >> This is also the behaviors of the current sources.
>> > > >> >> 1) a source can't guarantee it reads in strict order, because
>> the
>> > > >> producer
>> > > >> >> may produce data not in order.
>> > > >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can
>> > reorder
>> > > >> data.
>> > > >> >>
>> > > >> >> Best,
>> > > >> >> Jark
>> > > >> >>
>> > > >> >>
>> > > >> >>
>> > > >> >> On Tue, 17 Dec 2019 at 22:03, Becket Qin <be...@gmail.com>
>> > > wrote:
>> > > >> >>
>> > > >> >>> Hi folks,
>> > > >> >>>
>> > > >> >>> Thanks for the comments. I am convinced that the Source API
>> should
>> > > not
>> > > >> >> take
>> > > >> >>> boundedness as a parameter after it is constructed. What Timo
>> and
>> > > >> Dawid
>> > > >> >>> suggested sounds a reasonable solution to me. So the Source API
>> > > would
>> > > >> >>> become:
>> > > >> >>>
>> > > >> >>> Source {
>> > > >> >>>      Boundedness getBoundedness();
>> > > >> >>> }
>> > > >> >>>
>> > > >> >>> Assuming the above Source API, in addition to the two options
>> > > >> mentioned
>> > > >> >> in
>> > > >> >>> earlier emails, I am thinking of another option:
>> > > >> >>>
>> > > >> >>> *Option 3:*
>> > > >> >>> // MySource must be unbounded, otherwise throws exception.
>> > > >> >>> DataStream<Type> dataStream = env.source(mySource);
>> > > >> >>>
>> > > >> >>> // MySource must be bounded, otherwise throws exception.
>> > > >> >>> BoundedDataStream<Type> boundedDataStream =
>> > > >> env.boundedSource(mySource);
>> > > >> >>>
>> > > >> >>> The pros of this API are:
>> > > >> >>>     a) It fits the requirements from Table / SQL well.
>> > > >> >>>     b) DataStream users still have type safety (option 2 only
>> has
>> > > >> partial
>> > > >> >>> type safety).
>> > > >> >>>     c) Cristal clear boundedness from the API which makes
>> > DataStream
>> > > >> join
>> > > >> >> /
>> > > >> >>> connect easy to reason about.
>> > > >> >>> The caveats I see,
>> > > >> >>>     a) It is inconsistent with Table since Table has one
>> unified
>> > > >> >> interface.
>> > > >> >>>     b) No streaming mode for bounded source.
>> > > >> >>>
>> > > >> >>> @Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
>> > > >> >>> <al...@ververica.com> what do you think of the approach?
>> > > >> >>>
>> > > >> >>>
>> > > >> >>> Orthogonal to the above API, I am wondering whether
>> boundedness is
>> > > the
>> > > >> >> only
>> > > >> >>> dimension needed to describe the characteristic of the Source
>> > > >> behavior.
>> > > >> >> We
>> > > >> >>> may also need to have another dimension of *record order*.
>> > > >> >>>
>> > > >> >>> For example, when a file source is reading from a directory
>> with
>> > > >> bounded
>> > > >> >>> records, it may have two ways to read.
>> > > >> >>> 1. Read files in parallel.
>> > > >> >>> 2. Read files in the chronological order.
>> > > >> >>> In both cases, the file source is a Bounded Source. However,
>> the
>> > > >> >> processing
>> > > >> >>> requirement for downstream may be different. In the first case,
>> > the
>> > > >> >>> record processing and result emitting order does not matter,
>> e.g.
>> > > word
>> > > >> >>> count. In the second case, the records may have to be
>> processed in
>> > > the
>> > > >> >>> order they were read, e.g. change log processing.
>> > > >> >>>
>> > > >> >>> If the Source only has a getBoundedness() method, the
>> downstream
>> > > >> >> processors
>> > > >> >>> would not know whether the records emitted from the Source
>> should
>> > be
>> > > >> >>> processed in order or not. So combining the boundedness and
>> record
>> > > >> order,
>> > > >> >>> we will have four scenarios:
>> > > >> >>>
>> > > >> >>> *Bounded-StrictOrder*:     A segment of change log.
>> > > >> >>> *Bounded-Random*:          Batch Word Count.
>> > > >> >>> *Unbounded-StrictOrder*: An infinite change log.
>> > > >> >>> *Unbounded-Random*:     Streaming Word Count.
>> > > >> >>>
>> > > >> >>> Option 2 mentioned in the previous email was kind of trying to
>> > > handle
>> > > >> the
>> > > >> >>> Bounded-StrictOrder case by creating a DataStream from a
>> bounded
>> > > >> source,
>> > > >> >>> which actually does not work.
>> > > >> >>> It looks that we do not have strict order support in some
>> > operators
>> > > at
>> > > >> >> this
>> > > >> >>> point, e.g. join. But we may still want to add the semantic to
>> the
>> > > >> Source
>> > > >> >>> first so later on we don't need to change all the source
>> > > >> implementations,
>> > > >> >>> especially given that many of them will be implemented by 3rd
>> > party.
>> > > >> >>>
>> > > >> >>> Given that, we need another dimension of *Record Order* in the
>> > > Source.
>> > > >> >> More
>> > > >> >>> specifically, the API would become:
>> > > >> >>>
>> > > >> >>> Source {
>> > > >> >>>      Boundedness getBoundedness();
>> > > >> >>>      RecordOrder getRecordOrder();
>> > > >> >>> }
>> > > >> >>>
>> > > >> >>> public enum RecordOrder {
>> > > >> >>>      /** The record in the DataStream must be processed in its
>> > > strict
>> > > >> >> order
>> > > >> >>> for correctness. */
>> > > >> >>>      STRICT,
>> > > >> >>>      /** The record in the DataStream can be processed in
>> > arbitrary
>> > > >> order.
>> > > >> >>> */
>> > > >> >>>      RANDOM;
>> > > >> >>> }
>> > > >> >>>
>> > > >> >>> Any thoughts?
>> > > >> >>>
>> > > >> >>> Thanks,
>> > > >> >>>
>> > > >> >>> Jiangjie (Becket) Qin
>> > > >> >>>
>> > > >> >>> On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <
>> twalthr@apache.org>
>> > > >> wrote:
>> > > >> >>>
>> > > >> >>>> Hi Becket,
>> > > >> >>>>
>> > > >> >>>> I completely agree with Dawid's suggestion. The information
>> about
>> > > the
>> > > >> >>>> boundedness should come out of the source. Because most of the
>> > > >> >> streaming
>> > > >> >>>> sources can be made bounded based on some connector specific
>> > > >> criterion.
>> > > >> >>>> In Kafka, it would be an end offset or end timestamp but in
>> any
>> > > case
>> > > >> >>>> having just a env.boundedSource() is not enough because
>> > parameters
>> > > >> for
>> > > >> >>>> making the source bounded are missing.
>> > > >> >>>>
>> > > >> >>>> I suggest to have a simple `isBounded(): Boolean` flag in
>> every
>> > > >> source
>> > > >> >>>> that might be influenced by a connector builder as Dawid
>> > mentioned.
>> > > >> >>>>
>> > > >> >>>> For type safety during programming, we can still go with
>> *Final
>> > > state
>> > > >> >>>> 1*. By having a env.source() vs env.boundedSource(). The
>> latter
>> > > would
>> > > >> >>>> just enforce that the boolean flag is set to `true` and could
>> > make
>> > > >> >>>> bounded operations available (if we need that actually).
>> > > >> >>>>
>> > > >> >>>> However, I don't think that we should start making a unified
>> > Table
>> > > >> API
>> > > >> >>>> ununified again. Boundedness is an optimization property.
>> Every
>> > > >> bounded
>> > > >> >>>> operation can also executed in an unbounded way using
>> > > >> >> updates/retraction
>> > > >> >>>> or watermarks.
>> > > >> >>>>
>> > > >> >>>> Regards,
>> > > >> >>>> Timo
>> > > >> >>>>
>> > > >> >>>>
>> > > >> >>>> On 15.12.19 14:22, Becket Qin wrote:
>> > > >> >>>>> Hi Dawid and Jark,
>> > > >> >>>>>
>> > > >> >>>>> I think the discussion ultimately boils down to the question
>> > that
>> > > >> >> which
>> > > >> >>>> one
>> > > >> >>>>> of the following two final states do we want? Once we make
>> this
>> > > >> >>> decision,
>> > > >> >>>>> everything else can be naturally derived.
>> > > >> >>>>>
>> > > >> >>>>> *Final state 1*: Separate API for bounded / unbounded
>> > DataStream &
>> > > >> >>> Table.
>> > > >> >>>>> That means any code users write will be valid at the point
>> when
>> > > they
>> > > >> >>>> write
>> > > >> >>>>> the code. This is similar to having type safety check at
>> > > programming
>> > > >> >>>> time.
>> > > >> >>>>> For example,
>> > > >> >>>>>
>> > > >> >>>>> BoundedDataStream extends DataStream {
>> > > >> >>>>> // Operations only available for bounded data.
>> > > >> >>>>> BoundedDataStream sort(...);
>> > > >> >>>>>
>> > > >> >>>>> // Interaction with another BoundedStream returns a Bounded
>> > > stream.
>> > > >> >>>>> BoundedJoinedDataStream join(BoundedDataStream other)
>> > > >> >>>>>
>> > > >> >>>>> // Interaction with another unbounded stream returns an
>> > unbounded
>> > > >> >>> stream.
>> > > >> >>>>> JoinedDataStream join(DataStream other)
>> > > >> >>>>> }
>> > > >> >>>>>
>> > > >> >>>>> BoundedTable extends Table {
>> > > >> >>>>>     // Bounded only operation.
>> > > >> >>>>> BoundedTable sort(...);
>> > > >> >>>>>
>> > > >> >>>>> // Interaction with another BoundedTable returns a
>> BoundedTable.
>> > > >> >>>>> BoundedTable join(BoundedTable other)
>> > > >> >>>>>
>> > > >> >>>>> // Interaction with another unbounded table returns an
>> unbounded
>> > > >> >> table.
>> > > >> >>>>> Table join(Table other)
>> > > >> >>>>> }
>> > > >> >>>>>
>> > > >> >>>>> *Final state 2*: One unified API for bounded / unbounded
>> > > DataStream
>> > > >> /
>> > > >> >>>>> Table.
>> > > >> >>>>> That unified API may throw exception at DAG compilation time
>> if
>> > an
>> > > >> >>>> invalid
>> > > >> >>>>> operation is tried. This is what Table API currently follows.
>> > > >> >>>>>
>> > > >> >>>>> DataStream {
>> > > >> >>>>> // Throws exception if the DataStream is unbounded.
>> > > >> >>>>> DataStream sort();
>> > > >> >>>>> // Get boundedness.
>> > > >> >>>>> Boundedness getBoundedness();
>> > > >> >>>>> }
>> > > >> >>>>>
>> > > >> >>>>> Table {
>> > > >> >>>>> // Throws exception if the table has infinite rows.
>> > > >> >>>>> Table orderBy();
>> > > >> >>>>>
>> > > >> >>>>> // Get boundedness.
>> > > >> >>>>> Boundedness getBoundedness();
>> > > >> >>>>> }
>> > > >> >>>>>
>> > > >> >>>>> >From what I understand, there is no consensus so far on this
>> > > >> decision
>> > > >> >>>> yet.
>> > > >> >>>>> Whichever final state we choose, we need to make it
>> consistent
>> > > >> across
>> > > >> >>> the
>> > > >> >>>>> entire project. We should avoid the case that Table follows
>> one
>> > > >> final
>> > > >> >>>> state
>> > > >> >>>>> while DataStream follows another. Some arguments I am aware
>> of
>> > > from
>> > > >> >>> both
>> > > >> >>>>> sides so far are following:
>> > > >> >>>>>
>> > > >> >>>>> Arguments for final state 1:
>> > > >> >>>>> 1a) Clean API with method safety check at programming time.
>> > > >> >>>>> 1b) (Counter 2b) Although SQL does not have programming time
>> > error
>> > > >> >>>> check, SQL
>> > > >> >>>>> is not really a "programming language" per se. So SQL can be
>> > > >> >> different
>> > > >> >>>> from
>> > > >> >>>>> Table and DataStream.
>> > > >> >>>>> 1c)  Although final state 2 seems making it easier for SQL to
>> > use
>> > > >> >> given
>> > > >> >>>> it
>> > > >> >>>>> is more "config based" than "parameter based", final state 1
>> can
>> > > >> >>> probably
>> > > >> >>>>> also meet what SQL wants by wrapping the Source in
>> TableSource /
>> > > >> >>>>> TableSourceFactory API if needed.
>> > > >> >>>>>
>> > > >> >>>>> Arguments for final state 2:
>> > > >> >>>>> 2a) The Source API itself seems already sort of following the
>> > > >> unified
>> > > >> >>> API
>> > > >> >>>>> pattern.
>> > > >> >>>>> 2b) There is no "programming time" method error check in SQL
>> > case,
>> > > >> so
>> > > >> >>> we
>> > > >> >>>>> cannot really achieve final state 1 across the board.
>> > > >> >>>>> 2c) It is an easier path given our current status, i.e.
>> Table is
>> > > >> >>> already
>> > > >> >>>>> following final state 2.
>> > > >> >>>>> 2d) Users can always explicitly check the boundedness if they
>> > want
>> > > >> >> to.
>> > > >> >>>>>
>> > > >> >>>>> As I mentioned earlier, my initial thought was also to have a
>> > > >> >>>>> "configuration based" Source rather than a "parameter based"
>> > > Source.
>> > > >> >> So
>> > > >> >>>> it
>> > > >> >>>>> is completely possible that I missed some important
>> > consideration
>> > > or
>> > > >> >>>> design
>> > > >> >>>>> principles that we want to enforce for the project. It would
>> be
>> > > good
>> > > >> >>>>> if @Stephan
>> > > >> >>>>> Ewen <st...@ververica.com> and @Aljoscha Krettek <
>> > > >> >>>> aljoscha@ververica.com> can
>> > > >> >>>>> also provide more thoughts on this.
>> > > >> >>>>>
>> > > >> >>>>>
>> > > >> >>>>> Re: Jingsong
>> > > >> >>>>>
>> > > >> >>>>> As you said, there are some batched system source, like
>> > > parquet/orc
>> > > >> >>>> source.
>> > > >> >>>>>> Could we have the batch emit interface to improve
>> performance?
>> > > The
>> > > >> >>>> queue of
>> > > >> >>>>>> per record may cause performance degradation.
>> > > >> >>>>>
>> > > >> >>>>>
>> > > >> >>>>> The current interface does not necessarily cause performance
>> > > problem
>> > > >> >>> in a
>> > > >> >>>>> multi-threading case. In fact, the base implementation allows
>> > > >> >>>> SplitReaders
>> > > >> >>>>> to add a batch <E> of records<T> to the records queue<E>, so
>> > each
>> > > >> >>> element
>> > > >> >>>>> in the records queue would be a batch <E>. In this case, when
>> > the
>> > > >> >> main
>> > > >> >>>>> thread polls records, it will take a batch <E> of records <T>
>> > from
>> > > >> >> the
>> > > >> >>>>> shared records queue and process the records <T> in a batch
>> > > manner.
>> > > >> >>>>>
>> > > >> >>>>> Thanks,
>> > > >> >>>>>
>> > > >> >>>>> Jiangjie (Becket) Qin
>> > > >> >>>>>
>> > > >> >>>>> On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <
>> > > jingsonglee0@gmail.com
>> > > >> >
>> > > >> >>>> wrote:
>> > > >> >>>>>
>> > > >> >>>>>> Hi Becket,
>> > > >> >>>>>>
>> > > >> >>>>>> I also have some performance concerns too.
>> > > >> >>>>>>
>> > > >> >>>>>> If I understand correctly, SourceOutput will emit data per
>> > record
>> > > >> >> into
>> > > >> >>>> the
>> > > >> >>>>>> queue? I'm worried about the multithreading performance of
>> this
>> > > >> >> queue.
>> > > >> >>>>>>
>> > > >> >>>>>>> One example is some batched messaging systems which only
>> have
>> > an
>> > > >> >>> offset
>> > > >> >>>>>> for the entire batch instead of individual messages in the
>> > batch.
>> > > >> >>>>>>
>> > > >> >>>>>> As you said, there are some batched system source, like
>> > > parquet/orc
>> > > >> >>>> source.
>> > > >> >>>>>> Could we have the batch emit interface to improve
>> performance?
>> > > The
>> > > >> >>>> queue of
>> > > >> >>>>>> per record may cause performance degradation.
>> > > >> >>>>>>
>> > > >> >>>>>> Best,
>> > > >> >>>>>> Jingsong Lee
>> > > >> >>>>>>
>> > > >> >>>>>> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com>
>> > > wrote:
>> > > >> >>>>>>
>> > > >> >>>>>>> Hi Becket,
>> > > >> >>>>>>>
>> > > >> >>>>>>> I think Dawid explained things clearly and makes a lot of
>> > sense.
>> > > >> >>>>>>> I'm also in favor of #2, because #1 doesn't work for our
>> > future
>> > > >> >>> unified
>> > > >> >>>>>>> envrionment.
>> > > >> >>>>>>>
>> > > >> >>>>>>> You can see the vision in this documentation [1]. In the
>> > future,
>> > > >> we
>> > > >> >>>> would
>> > > >> >>>>>>> like to
>> > > >> >>>>>>> drop the global streaming/batch mode in SQL (i.e.
>> > > >> >>>>>>> EnvironmentSettings#inStreamingMode/inBatchMode).
>> > > >> >>>>>>> A source is bounded or unbounded once defined, so queries
>> can
>> > be
>> > > >> >>>> inferred
>> > > >> >>>>>>> from source to run
>> > > >> >>>>>>> in streaming or batch or hybrid mode. However, in #1, we
>> will
>> > > lose
>> > > >> >>> this
>> > > >> >>>>>>> ability because the framework
>> > > >> >>>>>>> doesn't know whether the source is bounded or unbounded.
>> > > >> >>>>>>>
>> > > >> >>>>>>> Best,
>> > > >> >>>>>>> Jark
>> > > >> >>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>> [1]:
>> > > >> >>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
>> > > >> >>>>>>>
>> > > >> >>>>>>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <
>> > > piotr@ververica.com
>> > > >> >
>> > > >> >>>>>> wrote:
>> > > >> >>>>>>>
>> > > >> >>>>>>>> Hi,
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> Regarding the:
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> Collection<E> getNextRecords()
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> I’m pretty sure such design would unfortunately impact the
>> > > >> >>> performance
>> > > >> >>>>>>>> (accessing and potentially creating the collection on the
>> hot
>> > > >> >> path).
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> Also the
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> InputStatus emitNext(DataOutput<T> output) throws
>> Exception;
>> > > >> >>>>>>>> or
>> > > >> >>>>>>>> Status pollNext(SourceOutput<T> sourceOutput) throws
>> > Exception;
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> Gives us some opportunities in the future, to allow Source
>> > hot
>> > > >> >>> looping
>> > > >> >>>>>>>> inside, until it receives some signal “please exit
>> because of
>> > > >> some
>> > > >> >>>>>>> reasons”
>> > > >> >>>>>>>> (output collector could return such hint upon collecting
>> the
>> > > >> >>> result).
>> > > >> >>>>>> But
>> > > >> >>>>>>>> that’s another topic outside of this FLIP’s scope.
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> Piotrek
>> > > >> >>>>>>>>
>> > > >> >>>>>>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <
>> > trohrmann@apache.org
>> > > >
>> > > >> >>>>>> wrote:
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>> Hi Becket,
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>> quick clarification from my side because I think you
>> > > >> >> misunderstood
>> > > >> >>> my
>> > > >> >>>>>>>>> question. I did not suggest to let the SourceReader
>> return
>> > > only
>> > > >> a
>> > > >> >>>>>>> single
>> > > >> >>>>>>>>> record at a time when calling getNextRecords. As the
>> return
>> > > type
>> > > >> >>>>>>>> indicates,
>> > > >> >>>>>>>>> the method can return an arbitrary number of records.
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>> Cheers,
>> > > >> >>>>>>>>> Till
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
>> > > >> >>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
>> > > >> >>>>>>>>> wrote:
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>>> Hi Becket,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Issue #1 - Design of Source interface
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I mentioned the lack of a method like
>> > > >> >>>>>>>> Source#createEnumerator(Boundedness
>> > > >> >>>>>>>>>> boundedness, SplitEnumeratorContext context), because
>> > without
>> > > >> >> the
>> > > >> >>>>>>>> current
>> > > >> >>>>>>>>>> proposal is not complete/does not work.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> If we say that boundedness is an intrinsic property of a
>> > > source
>> > > >> >>> imo
>> > > >> >>>>>> we
>> > > >> >>>>>>>>>> don't need the Source#createEnumerator(Boundedness
>> > > boundedness,
>> > > >> >>>>>>>>>> SplitEnumeratorContext context) method.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Assuming a source from my previous example:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
>> > > >> >>>>>>>>>>    ...
>> > > >> >>>>>>>>>>    .untilTimestamp(...)
>> > > >> >>>>>>>>>>    .build()
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Would the enumerator differ if created like
>> > > >> >>>>>>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs
>> > source
>> > > >> >>>>>>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating
>> > > myself,
>> > > >> >> but
>> > > >> >>>>>>> this
>> > > >> >>>>>>>> is
>> > > >> >>>>>>>>>> the part that my opinion differ the most from the
>> current
>> > > >> >>> proposal.
>> > > >> >>>>>> I
>> > > >> >>>>>>>>>> really think it should always be the source that tells
>> if
>> > it
>> > > is
>> > > >> >>>>>>> bounded
>> > > >> >>>>>>>> or
>> > > >> >>>>>>>>>> not. In the current proposal methods
>> > > >> >> continousSource/boundedSource
>> > > >> >>>>>>>> somewhat
>> > > >> >>>>>>>>>> reconfigure the source, which I think is misleading.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I think a call like:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
>> > > >> >>>>>>>>>>    ...
>> > > >> >>>>>>>>>>    .readContinously() / readUntilLatestOffset() /
>> > > >> >>> readUntilTimestamp
>> > > >> >>>> /
>> > > >> >>>>>>>> readUntilOffsets / ...
>> > > >> >>>>>>>>>>    .build()
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> is way cleaner (and expressive) than
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
>> > > >> >>>>>>>>>>    ...
>> > > >> >>>>>>>>>>    .build()
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> env.continousSource(source) // which actually underneath
>> > > would
>> > > >> >>> call
>> > > >> >>>>>>>> createEnumerator(CONTINUOUS, ctx) which would be
>> equivalent
>> > to
>> > > >> >>>>>>>> source.readContinously().createEnumerator(ctx)
>> > > >> >>>>>>>>>> // or
>> > > >> >>>>>>>>>> env.boundedSource(source) // which actually underneath
>> > would
>> > > >> >> call
>> > > >> >>>>>>>> createEnumerator(BOUNDED, ctx) which would be equivalent
>> to
>> > > >> >>>>>>>> source.readUntilLatestOffset().createEnumerator(ctx)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Sorry for the comparison, but to me it seems there is
>> too
>> > > much
>> > > >> >>> magic
>> > > >> >>>>>>>>>> happening underneath those two calls.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I really believe the Source interface should have
>> > > >> getBoundedness
>> > > >> >>>>>>> method
>> > > >> >>>>>>>>>> instead of (supportBoundedness) +
>> > > createEnumerator(Boundedness,
>> > > >> >>> ...)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Issue #2 - Design of
>> > > >> >>>>>>>>>>
>> > > >> ExecutionEnvironment#source()/continuousSource()/boundedSource()
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> As you might have guessed I am slightly in favor of
>> option
>> > #2
>> > > >> >>>>>>> modified.
>> > > >> >>>>>>>>>> Yes I am aware every step of the dag would have to be
>> able
>> > to
>> > > >> >> say
>> > > >> >>> if
>> > > >> >>>>>>> it
>> > > >> >>>>>>>> is
>> > > >> >>>>>>>>>> bounded or not. I have a feeling it would be easier to
>> > > express
>> > > >> >>> cross
>> > > >> >>>>>>>>>> bounded/unbounded operations, but I must admit I have
>> not
>> > > >> >> thought
>> > > >> >>> it
>> > > >> >>>>>>>>>> through thoroughly, In the spirit of batch is just a
>> > special
>> > > >> >> case
>> > > >> >>> of
>> > > >> >>>>>>>>>> streaming I thought BoundedStream would extend from
>> > > DataStream.
>> > > >> >>>>>>> Correct
>> > > >> >>>>>>>> me
>> > > >> >>>>>>>>>> if I am wrong. In such a setup the cross
>> bounded/unbounded
>> > > >> >>> operation
>> > > >> >>>>>>>> could
>> > > >> >>>>>>>>>> be expressed quite easily I think:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> DataStream {
>> > > >> >>>>>>>>>>    DataStream join(DataStream, ...); // we could not
>> really
>> > > >> tell
>> > > >> >> if
>> > > >> >>>>>> the
>> > > >> >>>>>>>> result is bounded or not, but because bounded stream is a
>> > > special
>> > > >> >>> case
>> > > >> >>>>>> of
>> > > >> >>>>>>>> unbounded the API object is correct, irrespective if the
>> left
>> > > or
>> > > >> >>> right
>> > > >> >>>>>>> side
>> > > >> >>>>>>>> of the join is bounded
>> > > >> >>>>>>>>>> }
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> BoundedStream extends DataStream {
>> > > >> >>>>>>>>>>    BoundedStream join(BoundedStream, ...); // only if
>> both
>> > > >> sides
>> > > >> >>> are
>> > > >> >>>>>>>> bounded the result can be bounded as well. However we do
>> have
>> > > >> >> access
>> > > >> >>>> to
>> > > >> >>>>>>> the
>> > > >> >>>>>>>> DataStream#join here, so you can still join with a
>> DataStream
>> > > >> >>>>>>>>>> }
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On the other hand I also see benefits of two completely
>> > > >> >> disjointed
>> > > >> >>>>>>> APIs,
>> > > >> >>>>>>>>>> as we could prohibit some streaming calls in the bounded
>> > > API. I
>> > > >> >>>>>> can't
>> > > >> >>>>>>>> think
>> > > >> >>>>>>>>>> of any unbounded operators that could not be implemented
>> > for
>> > > >> >>> bounded
>> > > >> >>>>>>>> stream.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Besides I think we both agree we don't like the method:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> DataStream boundedStream(Source)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> suggested in the current state of the FLIP. Do we ? :)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Best,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Dawid
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On 10/12/2019 18:57, Becket Qin wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi folks,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks for the discussion, great feedback. Also thanks
>> > Dawid
>> > > >> for
>> > > >> >>> the
>> > > >> >>>>>>>>>> explanation, it is much clearer now.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> One thing that is indeed missing from the FLIP is how
>> the
>> > > >> >>>>>> boundedness
>> > > >> >>>>>>> is
>> > > >> >>>>>>>>>> passed to the Source implementation. So the API should
>> be
>> > > >> >>>>>>>>>> Source#createEnumerator(Boundedness boundedness,
>> > > >> >>>>>>> SplitEnumeratorContext
>> > > >> >>>>>>>>>> context)
>> > > >> >>>>>>>>>> And we can probably remove the
>> > > >> >>> Source#supportBoundedness(Boundedness
>> > > >> >>>>>>>>>> boundedness) method.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Assuming we have that, we are essentially choosing from
>> one
>> > > of
>> > > >> >> the
>> > > >> >>>>>>>>>> following two options:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Option 1:
>> > > >> >>>>>>>>>> // The source is continuous source, and only unbounded
>> > > >> >> operations
>> > > >> >>>>>> can
>> > > >> >>>>>>> be
>> > > >> >>>>>>>>>> performed.
>> > > >> >>>>>>>>>> DataStream<Type> datastream =
>> > > env.continuousSource(someSource);
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> // The source is bounded source, both bounded and
>> unbounded
>> > > >> >>>>>> operations
>> > > >> >>>>>>>> can
>> > > >> >>>>>>>>>> be performed.
>> > > >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
>> > > >> >>>>>>>> env.boundedSource(someSource);
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>    - Pros:
>> > > >> >>>>>>>>>>         a) explicit boundary between bounded / unbounded
>> > > >> streams,
>> > > >> >>> it
>> > > >> >>>>>> is
>> > > >> >>>>>>>>>> quite simple and clear to the users.
>> > > >> >>>>>>>>>>    - Cons:
>> > > >> >>>>>>>>>>         a) For applications that do not involve bounded
>> > > >> >> operations,
>> > > >> >>>>>> they
>> > > >> >>>>>>>>>> still have to call different API to distinguish bounded
>> /
>> > > >> >>> unbounded
>> > > >> >>>>>>>> streams.
>> > > >> >>>>>>>>>>         b) No support for bounded stream to run in a
>> > > streaming
>> > > >> >>>> runtime
>> > > >> >>>>>>>>>> setting, i.e. scheduling and operators behaviors.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Option 2:
>> > > >> >>>>>>>>>> // The source is either bounded or unbounded, but only
>> > > >> unbounded
>> > > >> >>>>>>>> operations
>> > > >> >>>>>>>>>> could be performed on the returned DataStream.
>> > > >> >>>>>>>>>> DataStream<Type> dataStream = env.source(someSource);
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> // The source must be a bounded source, otherwise
>> exception
>> > > is
>> > > >> >>>>>> thrown.
>> > > >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
>> > > >> >>>>>>>>>> env.boundedSource(boundedSource);
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> The pros and cons are exactly the opposite of option 1.
>> > > >> >>>>>>>>>>    - Pros:
>> > > >> >>>>>>>>>>         a) For applications that do not involve bounded
>> > > >> >> operations,
>> > > >> >>>>>> they
>> > > >> >>>>>>>>>> still have to call different API to distinguish bounded
>> /
>> > > >> >>> unbounded
>> > > >> >>>>>>>> streams.
>> > > >> >>>>>>>>>>         b) Support for bounded stream to run in a
>> streaming
>> > > >> >> runtime
>> > > >> >>>>>>>> setting,
>> > > >> >>>>>>>>>> i.e. scheduling and operators behaviors.
>> > > >> >>>>>>>>>>    - Cons:
>> > > >> >>>>>>>>>>         a) Bounded / unbounded streams are kind of
>> mixed,
>> > > i.e.
>> > > >> >>> given
>> > > >> >>>> a
>> > > >> >>>>>>>>>> DataStream, it is not clear whether it is bounded or
>> not,
>> > > >> unless
>> > > >> >>> you
>> > > >> >>>>>>>> have
>> > > >> >>>>>>>>>> the access to its source.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> If we only think from the Source API perspective,
>> option 2
>> > > >> >> seems a
>> > > >> >>>>>>>> better
>> > > >> >>>>>>>>>> choice because functionality wise it is a superset of
>> > option
>> > > 1,
>> > > >> >> at
>> > > >> >>>>>> the
>> > > >> >>>>>>>> cost
>> > > >> >>>>>>>>>> of some seemingly acceptable ambiguity in the DataStream
>> > API.
>> > > >> >>>>>>>>>> But if we look at the DataStream API as a whole, option
>> 1
>> > > seems
>> > > >> >> a
>> > > >> >>>>>>>> clearer
>> > > >> >>>>>>>>>> choice. For example, some times a library may have to
>> know
>> > > >> >>> whether a
>> > > >> >>>>>>>>>> certain task will finish or not. And it would be
>> difficult
>> > to
>> > > >> >> tell
>> > > >> >>>>>> if
>> > > >> >>>>>>>> the
>> > > >> >>>>>>>>>> input is a DataStream, unless additional information is
>> > > >> provided
>> > > >> >>> all
>> > > >> >>>>>>> the
>> > > >> >>>>>>>>>> way from the Source. One possible solution is to have a
>> > > >> >> *modified
>> > > >> >>>>>>>> option 2*
>> > > >> >>>>>>>>>> which adds a method to the DataStream API to indicate
>> > > >> >> boundedness,
>> > > >> >>>>>>> such
>> > > >> >>>>>>>> as
>> > > >> >>>>>>>>>> getBoundedness(). It would solve the problem with a
>> > potential
>> > > >> >>>>>>> confusion
>> > > >> >>>>>>>> of
>> > > >> >>>>>>>>>> what is difference between a DataStream with
>> > > >> >> getBoundedness()=true
>> > > >> >>>>>>> and a
>> > > >> >>>>>>>>>> BoundedDataStream. But that seems not super difficult to
>> > > >> >> explain.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> So from API's perspective, I don't have a strong opinion
>> > > >> between
>> > > >> >>>>>>>> *option 1*
>> > > >> >>>>>>>>>> and *modified option 2. *I like the cleanness of option
>> 1,
>> > > but
>> > > >> >>>>>>> modified
>> > > >> >>>>>>>>>> option 2 would be more attractive if we have concrete
>> use
>> > > case
>> > > >> >> for
>> > > >> >>>>>> the
>> > > >> >>>>>>>>>> "Bounded stream with unbounded streaming runtime
>> settings".
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Re: Till
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Maybe this has already been asked before but I was
>> > wondering
>> > > >> why
>> > > >> >>> the
>> > > >> >>>>>>>>>> SourceReader interface has the method pollNext which
>> hands
>> > > the
>> > > >> >>>>>>>>>> responsibility of outputting elements to the
>> SourceReader
>> > > >> >>>>>>>> implementation?
>> > > >> >>>>>>>>>> Has this been done for backwards compatibility reasons
>> with
>> > > the
>> > > >> >>> old
>> > > >> >>>>>>>> source
>> > > >> >>>>>>>>>> interface? If not, then one could define a Collection<E>
>> > > >> >>>>>>>> getNextRecords()
>> > > >> >>>>>>>>>> method which returns the currently retrieved records and
>> > then
>> > > >> >> the
>> > > >> >>>>>>> caller
>> > > >> >>>>>>>>>> emits them outside of the SourceReader. That way the
>> > > interface
>> > > >> >>> would
>> > > >> >>>>>>> not
>> > > >> >>>>>>>>>> allow to implement an outputting loop where we never
>> hand
>> > > back
>> > > >> >>>>>> control
>> > > >> >>>>>>>> to
>> > > >> >>>>>>>>>> the caller. At the moment, this contract can be easily
>> > broken
>> > > >> >> and
>> > > >> >>> is
>> > > >> >>>>>>>> only
>> > > >> >>>>>>>>>> mentioned loosely in the JavaDocs.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> The primary reason we handover the SourceOutput to the
>> > > >> >>> SourceReader
>> > > >> >>>>>> is
>> > > >> >>>>>>>>>> because sometimes it is difficult for a SourceReader to
>> > emit
>> > > >> one
>> > > >> >>>>>>> record
>> > > >> >>>>>>>> at
>> > > >> >>>>>>>>>> a time. One example is some batched messaging systems
>> which
>> > > >> only
>> > > >> >>>>>> have
>> > > >> >>>>>>> an
>> > > >> >>>>>>>>>> offset for the entire batch instead of individual
>> messages
>> > in
>> > > >> >> the
>> > > >> >>>>>>>> batch. In
>> > > >> >>>>>>>>>> that case, returning one record at a time would leave
>> the
>> > > >> >>>>>> SourceReader
>> > > >> >>>>>>>> in
>> > > >> >>>>>>>>>> an uncheckpointable state because they can only
>> checkpoint
>> > at
>> > > >> >> the
>> > > >> >>>>>>> batch
>> > > >> >>>>>>>>>> boundaries.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <
>> > > >> >>> trohrmann@apache.org
>> > > >> >>>>>>>> <ma...@apache.org>> <trohrmann@apache.org
>> > <mailto:
>> > > >> >>>>>>>> trohrmann@apache.org>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi everyone,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> thanks for drafting this FLIP. It reads very well.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Concerning Dawid's proposal, I tend to agree. The
>> > boundedness
>> > > >> >>> could
>> > > >> >>>>>>> come
>> > > >> >>>>>>>>>> from the source and tell the system how to treat the
>> > operator
>> > > >> >>>>>>>> (scheduling
>> > > >> >>>>>>>>>> wise). From a user's perspective it should be fine to
>> get
>> > > back
>> > > >> a
>> > > >> >>>>>>>> DataStream
>> > > >> >>>>>>>>>> when calling env.source(boundedSource) if he does not
>> need
>> > > >> >> special
>> > > >> >>>>>>>>>> operations defined on a BoundedDataStream. If he needs
>> > this,
>> > > >> >> then
>> > > >> >>>>>> one
>> > > >> >>>>>>>> could
>> > > >> >>>>>>>>>> use the method BoundedDataStream
>> > > >> >> env.boundedSource(boundedSource).
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> If possible, we could enforce the proper usage of
>> > > >> >>>>>> env.boundedSource()
>> > > >> >>>>>>> by
>> > > >> >>>>>>>>>> introducing a BoundedSource type so that one cannot
>> pass an
>> > > >> >>>>>>>>>> unbounded source to it. That way users would not be
>> able to
>> > > >> >> shoot
>> > > >> >>>>>>>>>> themselves in the foot.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Maybe this has already been asked before but I was
>> > wondering
>> > > >> why
>> > > >> >>> the
>> > > >> >>>>>>>>>> SourceReader interface has the method pollNext which
>> hands
>> > > the
>> > > >> >>>>>>>>>> responsibility of outputting elements to the
>> SourceReader
>> > > >> >>>>>>>> implementation?
>> > > >> >>>>>>>>>> Has this been done for backwards compatibility reasons
>> with
>> > > the
>> > > >> >>> old
>> > > >> >>>>>>>> source
>> > > >> >>>>>>>>>> interface? If not, then one could define a Collection<E>
>> > > >> >>>>>>>> getNextRecords()
>> > > >> >>>>>>>>>> method which returns the currently retrieved records and
>> > then
>> > > >> >> the
>> > > >> >>>>>>> caller
>> > > >> >>>>>>>>>> emits them outside of the SourceReader. That way the
>> > > interface
>> > > >> >>> would
>> > > >> >>>>>>> not
>> > > >> >>>>>>>>>> allow to implement an outputting loop where we never
>> hand
>> > > back
>> > > >> >>>>>> control
>> > > >> >>>>>>>> to
>> > > >> >>>>>>>>>> the caller. At the moment, this contract can be easily
>> > broken
>> > > >> >> and
>> > > >> >>> is
>> > > >> >>>>>>>> only
>> > > >> >>>>>>>>>> mentioned loosely in the JavaDocs.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Cheers,
>> > > >> >>>>>>>>>> Till
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <
>> > > >> >>> jingsonglee0@gmail.com
>> > > >> >>>>>>>> <ma...@gmail.com>> <jingsonglee0@gmail.com
>> > > >> <mailto:
>> > > >> >>>>>>>> jingsonglee0@gmail.com>>
>> > > >> >>>>>>>>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi all,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I think current design is good.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> My understanding is:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> For execution mode: bounded mode and continuous mode,
>> It's
>> > > >> >> totally
>> > > >> >>>>>>>>>> different. I don't think we have the ability to
>> integrate
>> > the
>> > > >> >> two
>> > > >> >>>>>>> models
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> at
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> present. It's about scheduling, memory, algorithms,
>> States,
>> > > >> etc.
>> > > >> >>> we
>> > > >> >>>>>>>>>> shouldn't confuse them.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> For source capabilities: only bounded, only continuous,
>> > both
>> > > >> >>> bounded
>> > > >> >>>>>>> and
>> > > >> >>>>>>>>>> continuous.
>> > > >> >>>>>>>>>> I think Kafka is a source that can be ran both bounded
>> > > >> >>>>>>>>>> and continuous execution mode.
>> > > >> >>>>>>>>>> And Kafka with end offset should be ran both bounded
>> > > >> >>>>>>>>>> and continuous execution mode.  Using apache Beam with
>> > Flink
>> > > >> >>>>>> runner, I
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> used
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> to run a "bounded" Kafka in streaming mode. For our
>> > previous
>> > > >> >>>>>>> DataStream,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> it
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> is not necessarily required that the source cannot be
>> > > bounded.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> So it is my thought for Dawid's question:
>> > > >> >>>>>>>>>> 1.pass a bounded source to continuousSource() +1
>> > > >> >>>>>>>>>> 2.pass a continuous source to boundedSource() -1, should
>> > > throw
>> > > >> >>>>>>>> exception.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> In StreamExecutionEnvironment, continuousSource and
>> > > >> >> boundedSource
>> > > >> >>>>>>> define
>> > > >> >>>>>>>>>> the execution mode. It defines a clear boundary of
>> > execution
>> > > >> >> mode.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Best,
>> > > >> >>>>>>>>>> Jingsong Lee
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <
>> imjark@gmail.com
>> > > >> >>> <mailto:
>> > > >> >>>>>>>> imjark@gmail.com>> <imjark@gmail.com <mailto:
>> > imjark@gmail.com
>> > > >>
>> > > >> >>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I agree with Dawid's point that the boundedness
>> information
>> > > >> >> should
>> > > >> >>>>>>> come
>> > > >> >>>>>>>>>> from the source itself (e.g. the end timestamp), not
>> > through
>> > > >> >>>>>>>>>> env.boundedSouce()/continuousSource().
>> > > >> >>>>>>>>>> I think if we want to support something like
>> `env.source()`
>> > > >> that
>> > > >> >>>>>>> derive
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> execution mode from source,
>> > > `supportsBoundedness(Boundedness)`
>> > > >> >>>>>>>>>> method is not enough, because we don't know whether it
>> is
>> > > >> >> bounded
>> > > >> >>> or
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> not.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Best,
>> > > >> >>>>>>>>>> Jark
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
>> > > >> >>>>>> dwysakowicz@apache.org
>> > > >> >>>>>>>> <ma...@apache.org>> <dwysakowicz@apache.org
>> > > >> <mailto:
>> > > >> >>>>>>>> dwysakowicz@apache.org>>
>> > > >> >>>>>>>>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> One more thing. In the current proposal, with the
>> > > >> >>>>>>>>>> supportsBoundedness(Boundedness) method and the
>> boundedness
>> > > >> >> coming
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> from
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> either continuousSource or boundedSource I could not
>> find
>> > how
>> > > >> >> this
>> > > >> >>>>>>>>>> information is fed back to the SplitEnumerator.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Best,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Dawid
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On 09/12/2019 13:52, Becket Qin wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi Dawid,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks for the comments. This actually brings another
>> > > relevant
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> question
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> about what does a "bounded source" imply. I actually had
>> > the
>> > > >> >> same
>> > > >> >>>>>>>>>> impression when I look at the Source API. Here is what I
>> > > >> >>> understand
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> after
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> some discussion with Stephan. The bounded source has the
>> > > >> >> following
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> impacts.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 1. API validity.
>> > > >> >>>>>>>>>> - A bounded source generates a bounded stream so some
>> > > >> operations
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> only
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> works for bounded records would be performed, e.g. sort.
>> > > >> >>>>>>>>>> - To expose these bounded stream only APIs, there are
>> two
>> > > >> >> options:
>> > > >> >>>>>>>>>>       a. Add them to the DataStream API and throw
>> exception
>> > > if
>> > > >> a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> method
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> is
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> called on an unbounded stream.
>> > > >> >>>>>>>>>>       b. Create a BoundedDataStream class which is
>> returned
>> > > >> from
>> > > >> >>>>>>>>>> env.boundedSource(), while DataStream is returned from
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> env.continousSource().
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Note that this cannot be done by having single
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> env.source(theSource)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> even
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the Source has a getBoundedness() method.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 2. Scheduling
>> > > >> >>>>>>>>>> - A bounded source could be computed stage by stage
>> without
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> bringing
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> up
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> all
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the tasks at the same time.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 3. Operator behaviors
>> > > >> >>>>>>>>>> - A bounded source indicates the records are finite so
>> some
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> operators
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> can
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> wait until it receives all the records before it starts
>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> processing.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> In the above impact, only 1 is relevant to the API
>> design.
>> > > And
>> > > >> >> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> current
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> proposal in FLIP-27 is following 1.b.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> // boundedness depends of source property, imo this
>> should
>> > > >> >> always
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> be
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> preferred
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> In your proposal, does DataStream have bounded stream
>> only
>> > > >> >>> methods?
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> It
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> looks it should have, otherwise passing a bounded
>> Source to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> env.source()
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> would be confusing. In that case, we will essentially do
>> > 1.a
>> > > if
>> > > >> >> an
>> > > >> >>>>>>>>>> unbounded Source is created from
>> > env.source(unboundedSource).
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> If we have the methods only supported for bounded
>> streams
>> > in
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> DataStream,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> it
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> seems a little weird to have a separate
>> BoundedDataStream
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> interface.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Am I understand it correctly?
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi all,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Really well written proposal and very important one. I
>> must
>> > > >> >> admit
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> have
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> not understood all the intricacies of it yet.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> One question I have though is about where does the
>> > > information
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> about
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> boundedness come from. I think in most cases it is a
>> > property
>> > > >> of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> source. As you described it might be e.g. end offset, a
>> > flag
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> should
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> it
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> monitor new splits etc. I think it would be a really
>> nice
>> > use
>> > > >> >> case
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> be
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> able to say:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> new KafkaSource().readUntil(long timestamp),
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> which could work as an "end offset". Moreover I think
>> all
>> > > >> >> Bounded
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> sources
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> support continuous mode, but no intrinsically continuous
>> > > source
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> support
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Bounded mode. If I understood the proposal correctly it
>> > > suggest
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> boundedness sort of "comes" from the outside of the
>> source,
>> > > >> from
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> invokation of either boundedStream or continousSource.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I am wondering if it would make sense to actually change
>> > the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> method
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> boolean Source#supportsBoundedness(Boundedness)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Boundedness Source#getBoundedness().
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> As for the methods #boundedSource, #continousSource,
>> > assuming
>> > > >> >> the
>> > > >> >>>>>>>>>> boundedness is property of the source they do not affect
>> > how
>> > > >> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> enumerator
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> works, but mostly how the dag is scheduled, right? I am
>> not
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> against
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> those
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> methods, but I think it is a very specific use case to
>> > > actually
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> override
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the property of the source. In general I would expect
>> users
>> > > to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> only
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> call
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> env.source(theSource), where the source tells if it is
>> > > bounded
>> > > >> >> or
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> not. I
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> would suggest considering following set of methods:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> // boundedness depends of source property, imo this
>> should
>> > > >> >> always
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> be
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> preferred
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> // always continous execution, whether bounded or
>> unbounded
>> > > >> >> source
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> DataStream<MyType> boundedStream =
>> > > >> >> env.continousSource(theSource);
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> // imo this would make sense if the BoundedDataStream
>> > > provides
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> additional features unavailable for continous mode
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> BoundedDataStream<MyType> batch =
>> > > env.boundedSource(theSource);
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Best,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Dawid
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks, Becket, for updating this.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I agree with moving the aspects you mentioned into
>> separate
>> > > >> >> FLIPs
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> -
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> this
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> one way becoming unwieldy in size.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> +1 to the FLIP in its current state. Its a very detailed
>> > > >> >> write-up,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> nicely
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> done!
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <
>> > > >> becket.qin@gmail.com
>> > > >> >>>>>>>> <ma...@gmail.com>> <becket.qin@gmail.com
>> > <mailto:
>> > > >> >>>>>>>> becket.qin@gmail.com>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> becket.qin@gmail.com <ma...@gmail.com>>
>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi all,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Sorry for the long belated update. I have updated
>> FLIP-27
>> > > wiki
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> page
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> with
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the latest proposals. Some noticeable changes include:
>> > > >> >>>>>>>>>> 1. A new generic communication mechanism between
>> > > >> SplitEnumerator
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> SourceReader.
>> > > >> >>>>>>>>>> 2. Some detail API method signature changes.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> We left a few things out of this FLIP and will address
>> them
>> > > in
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> separate
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> FLIPs. Including:
>> > > >> >>>>>>>>>> 1. Per split event time.
>> > > >> >>>>>>>>>> 2. Event time alignment.
>> > > >> >>>>>>>>>> 3. Fine grained failover for SplitEnumerator failure.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Please let us know if you have any question.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <
>> > > sewen@apache.org
>> > > >> >>>>>>> <mailto:
>> > > >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:
>> > sewen@apache.org
>> > > >>
>> > > >> <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi  Łukasz!
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Becket and me are working hard on figuring out the last
>> > > details
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> implementing the first PoC. We would update the FLIP
>> > > hopefully
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> next
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> week.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> There is a fair chance that a first version of this
>> will be
>> > > in
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 1.10,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> but
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> think it will take another release to battle test it and
>> > > >> migrate
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> connectors.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Best,
>> > > >> >>>>>>>>>> Stephan
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <
>> > > >> >> ljd@touk.pl
>> > > >> >>>>>>>> <ma...@touk.pl>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> ljd@touk.pl <ma...@touk.pl>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> This proposal looks very promising for us. Do you have
>> any
>> > > >> plans
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> in
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> which
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Flink release it is going to be released? We are
>> thinking
>> > on
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> using a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Data
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Set API for our future use cases but on the other hand
>> Data
>> > > Set
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> API
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> is
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> going to be deprecated so using proposed bounded data
>> > streams
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> solution
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> could be more viable in the long term.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks,
>> > > >> >>>>>>>>>> Łukasz
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On 2019/10/01 15:48:03, Thomas Weise <
>> > thomas.weise@gmail.com
>> > > >> >>>>>> <mailto:
>> > > >> >>>>>>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
>> > > >> >>>>>>>> thomas.weise@gmail.com>> <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> thomas.weise@gmail.com <ma...@gmail.com>>
>> > > wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks for putting together this proposal!
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I see that the "Per Split Event Time" and "Event Time
>> > > >> Alignment"
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> sections
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> are still TBD.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> It would probably be good to flesh those out a bit
>> before
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> proceeding
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> too
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> far
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> as the event time alignment will probably influence the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> interaction
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> with
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the split reader, specifically ReaderStatus
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> emitNext(SourceOutput<E>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> output).
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> We currently have only one implementation for event time
>> > > >> >> alignment
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> in
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Kinesis consumer. The synchronization in that case takes
>> > > place
>> > > >> >> as
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> last
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> step before records are emitted downstream
>> (RecordEmitter).
>> > > >> With
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> currently proposed interfaces, the equivalent can be
>> > > >> implemented
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> in
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> reader loop, although note that in the Kinesis consumer
>> the
>> > > per
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> shard
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> threads push records.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Synchronization has not been implemented for the Kafka
>> > > consumer
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> yet.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
>> > > >> >>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> When I looked at it, I realized that the implementation
>> > will
>> > > >> >> look
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> quite
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> different
>> > > >> >>>>>>>>>> from Kinesis because it needs to take place in the pull
>> > part,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> where
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> records
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> are taken from the Kafka client. Due to the
>> multiplexing it
>> > > >> >> cannot
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> be
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> done
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> by blocking the split thread like it currently works for
>> > > >> >> Kinesis.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Reading
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> from individual Kafka partitions needs to be controlled
>> via
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> pause/resume
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> on the Kafka client.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> To take on that responsibility the split thread would
>> need
>> > to
>> > > >> be
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> aware
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>> watermarks or at least whether it should or should not
>> > > continue
>> > > >> >> to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> consume
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> a given split and this may require a different
>> SourceReader
>> > > or
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> SourceOutput
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> interface.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks,
>> > > >> >>>>>>>>>> Thomas
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <
>> > mmyy1110@gmail.com
>> > > >> >>>>>> <mailto:
>> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> > > >> >> mmyy1110@gmail.com
>> > > >> >>>>>
>> > > >> >>>>>> <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi Stephan,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thank you for feedback!
>> > > >> >>>>>>>>>> Will take a look at your branch before public
>> discussing.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <
>> > > >> sewen@apache.org
>> > > >> >>>>>>>> <ma...@apache.org>> <sewen@apache.org <mailto:
>> > > >> >>> sewen@apache.org
>> > > >> >>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi Biao!
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks for reviving this. I would like to join this
>> > > discussion,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> but
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> am
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> quite occupied with the 1.9 release, so can we maybe
>> pause
>> > > this
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> discussion
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> for a week or so?
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> In the meantime I can share some suggestion based on
>> prior
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> experiments:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> How to do watermarks / timestamp extractors in a simpler
>> > and
>> > > >> >> more
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> flexible
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> way. I think that part is quite promising should be
>> part of
>> > > the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> new
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> source
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> interface.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>> > > >> >>>>>>>> <
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>> > > >> >>>>>>>> <
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Some experiments on how to build the source reader and
>> its
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> library
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> for
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> common threading/split patterns:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>> > > >> >>>>>>>> <
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Best,
>> > > >> >>>>>>>>>> Stephan
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <
>> > > mmyy1110@gmail.com
>> > > >> >>>>>>> <mailto:
>> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> > > >> >> mmyy1110@gmail.com
>> > > >> >>>>>
>> > > >> >>>>>> <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi devs,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Since 1.9 is nearly released, I think we could get back
>> to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> FLIP-27.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> believe it should be included in 1.10.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> There are so many things mentioned in document of
>> FLIP-27.
>> > > [1]
>> > > >> I
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> think
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> we'd better discuss them separately. However the wiki is
>> > not
>> > > a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> good
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> place
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> to discuss. I wrote google doc about SplitReader API
>> which
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> misses
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> some
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> details in the document. [2]
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 1.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>> > > >> >>>>>>>> <
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 2.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>> > > >> >>>>>>>> <
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <
>> > mmyy1110@gmail.com
>> > > >> >>>>>> <mailto:
>> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> > > >> >> mmyy1110@gmail.com
>> > > >> >>>>>
>> > > >> >>>>>> <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi Steven,
>> > > >> >>>>>>>>>> Thank you for the feedback. Please take a look at the
>> > > document
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> FLIP-27
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>> > > >> >>>>>>>> <
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >>
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> which
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> is updated recently. A lot of details of enumerator were
>> > > added
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> in
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> this
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> document. I think it would help.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Steven Wu <stevenz3wu@gmail.com <mailto:
>> > stevenz3wu@gmail.com
>> > > >>
>> > > >> >> <
>> > > >> >>>>>>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
>> > > >> >>>>>>> stevenz3wu@gmail.com
>> > > >> >>>>>>>> <ma...@gmail.com>> <stevenz3wu@gmail.com
>> > <mailto:
>> > > >> >>>>>>>> stevenz3wu@gmail.com>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 于2019年3月28日周四
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 下午12:52写道:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> This proposal mentioned that SplitEnumerator might run
>> on
>> > the
>> > > >> >>>>>>>>>> JobManager or
>> > > >> >>>>>>>>>> in a single task on a TaskManager.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> if enumerator is a single task on a taskmanager, then
>> the
>> > job
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> DAG
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> can
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> never
>> > > >> >>>>>>>>>> been embarrassingly parallel anymore. That will nullify
>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> leverage
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> fine-grained recovery for embarrassingly parallel jobs.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> It's not clear to me what's the implication of running
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> enumerator
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> on
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> jobmanager. So I will leave that out for now.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <
>> > mmyy1110@gmail.com
>> > > >> >>>>>> <mailto:
>> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> > > >> >> mmyy1110@gmail.com
>> > > >> >>>>>
>> > > >> >>>>>> <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi Stephan & Piotrek,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thank you for feedback.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> It seems that there are a lot of things to do in
>> community.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> am
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> just
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> afraid that this discussion may be forgotten since
>> there so
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> many
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> proposals
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> recently.
>> > > >> >>>>>>>>>> Anyway, wish to see the split topics soon :)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:
>> > > >> >>> piotr@da-platform.com
>> > > >> >>>>>>>>
>> > > >> >>>>>>> <
>> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
>> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
>> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 于2019年1月24日周四
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 下午8:21写道:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi Biao!
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> This discussion was stalled because of preparations for
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> open
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> sourcing
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> & merging Blink. I think before creating the tickets we
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> should
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> split this
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> discussion into topics/areas outlined by Stephan and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> create
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Flips
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> for
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I think there is no chance for this to be completed in
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> couple
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> remaining
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> weeks/1 month before 1.8 feature freeze, however it
>> would
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> be
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> good
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> to aim
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> with those changes for 1.9.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Piotrek
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com
>> > > >> <mailto:
>> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> > > >> >> mmyy1110@gmail.com
>> > > >> >>>>>
>> > > >> >>>>>> <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi community,
>> > > >> >>>>>>>>>> The summary of Stephan makes a lot sense to me. It is
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> much
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> clearer
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> indeed
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> after splitting the complex topic into small ones.
>> > > >> >>>>>>>>>> I was wondering is there any detail plan for next step?
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> If
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> not,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> would
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> like to push this thing forward by creating some JIRA
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> issues.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Another question is that should version 1.8 include
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> these
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> features?
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Stephan Ewen <sewen@apache.org <mailto:sewen@apache.org
>> >>
>> > <
>> > > >> >>>>>>>> sewen@apache.org <ma...@apache.org>> <
>> > sewen@apache.org
>> > > >> >>>> <mailto:
>> > > >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:
>> > sewen@apache.org
>> > > >>
>> > > >> >>>>>>>> 于2018年12月1日周六
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 上午4:20写道:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks everyone for the lively discussion. Let me try
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> summarize
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> where I
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> see convergence in the discussion and open issues.
>> > > >> >>>>>>>>>> I'll try to group this by design aspect of the source.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Please
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> let me
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> know
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> if I got things wrong or missed something crucial here.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> For issues 1-3, if the below reflects the state of the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> discussion, I
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> would
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> try and update the FLIP in the next days.
>> > > >> >>>>>>>>>> For the remaining ones we need more discussion.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> I would suggest to fork each of these aspects into a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> separate
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> mail
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> thread,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> or will loose sight of the individual aspects.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> *(1) Separation of Split Enumerator and Split Reader*
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - All seem to agree this is a good thing
>> > > >> >>>>>>>>>> - Split Enumerator could in the end live on JobManager
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> (and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> assign
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> splits
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> via RPC) or in a task (and assign splits via data
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> streams)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - this discussion is orthogonal and should come later,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> when
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> interface
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> is agreed upon.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> *(2) Split Readers for one or more splits*
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Discussion seems to agree that we need to support
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> one
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> reader
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> possibly handles multiple splits concurrently.
>> > > >> >>>>>>>>>> - The requirement comes from sources where one
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> poll()-style
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> call
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> fetches
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> data from different splits / partitions
>> > > >> >>>>>>>>>>     --> example sources that require that would be for
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> example
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Kafka,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Pravega, Pulsar
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Could have one split reader per source, or multiple
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> split
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> readers
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> share the "poll()" function
>> > > >> >>>>>>>>>> - To not make it too complicated, we can start with
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> thinking
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> about
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> one
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> split reader for all splits initially and see if that
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> covers
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> all
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> requirements
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> *(3) Threading model of the Split Reader*
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Most active part of the discussion ;-)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - A non-blocking way for Flink's task code to interact
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> with
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> source
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> is
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> needed in order to a task runtime code based on a
>> > > >> >>>>>>>>>> single-threaded/actor-style task design
>> > > >> >>>>>>>>>>     --> I personally am a big proponent of that, it will
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> help
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> with
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> well-behaved checkpoints, efficiency, and simpler yet
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> more
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> robust
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> runtime
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> code
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Users care about simple abstraction, so as a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> subclass
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> SplitReader
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> (non-blocking / async) we need to have a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> BlockingSplitReader
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> which
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> will
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> form the basis of most source implementations.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> BlockingSplitReader
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> lets
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> users do blocking simple poll() calls.
>> > > >> >>>>>>>>>> - The BlockingSplitReader would spawn a thread (or
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> more)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> thread(s) can make blocking calls and hand over data
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> buffers
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> via
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> blocking
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> queue
>> > > >> >>>>>>>>>> - This should allow us to cover both, a fully async
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> runtime,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> simple
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> blocking interface for users.
>> > > >> >>>>>>>>>> - This is actually very similar to how the Kafka
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> connectors
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> work.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Kafka
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 9+ with one thread, Kafka 8 with multiple threads
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - On the base SplitReader (the async one), the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> non-blocking
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> method
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> gets the next chunk of data would signal data
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> availability
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> via
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> CompletableFuture, because that gives the best
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> flexibility
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> (can
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> await
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> completion or register notification handlers).
>> > > >> >>>>>>>>>> - The source task would register a "thenHandle()" (or
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> similar)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> on the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> future to put a "take next data" task into the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> actor-style
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> mailbox
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> *(4) Split Enumeration and Assignment*
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Splits may be generated lazily, both in cases where
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> there
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> is a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> limited
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> number of splits (but very many), or splits are
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> discovered
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> over
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> time
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Assignment should also be lazy, to get better load
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> balancing
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Assignment needs support locality preferences
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Possible design based on discussion so far:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>     --> SplitReader has a method "addSplits(SplitT...)"
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> add
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> one or
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> more
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> splits. Some split readers might assume they have only
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> one
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> split
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> ever,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> concurrently, others assume multiple splits. (Note:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> idea
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> behind
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> being
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> able
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> to add multiple splits at the same time is to ease
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> startup
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> where
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> multiple
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> splits may be assigned instantly.)
>> > > >> >>>>>>>>>>     --> SplitReader has a context object on which it can
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> call
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> indicate
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> when
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> splits are completed. The enumerator gets that
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> notification and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> can
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> use
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> decide when to assign new splits. This should help both
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> in
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> cases
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> sources
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that take splits lazily (file readers) and in case the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> source
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> needs to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> preserve a partial order between splits (Kinesis,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Pravega,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Pulsar may
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> need
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that).
>> > > >> >>>>>>>>>>     --> SplitEnumerator gets notification when
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> SplitReaders
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> start
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> when
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> they finish splits. They can decide at that moment to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> push
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> more
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> splits
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that reader
>> > > >> >>>>>>>>>>     --> The SplitEnumerator should probably be aware of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> source
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> parallelism, to build its initial distribution.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Open question: Should the source expose something
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> like
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> "host
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> preferences", so that yarn/mesos/k8s can take this into
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> account
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> when
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> selecting a node to start a TM on?
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> *(5) Watermarks and event time alignment*
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Watermark generation, as well as idleness, needs to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> be
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> per
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> split
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> (like
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> currently in the Kafka Source, per partition)
>> > > >> >>>>>>>>>> - It is desirable to support optional
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> event-time-alignment,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> meaning
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> splits that are ahead are back-pressured or temporarily
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> unsubscribed
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - I think i would be desirable to encapsulate
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> watermark
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> generation
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> logic
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> in watermark generators, for a separation of concerns.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> The
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> watermark
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> generators should run per split.
>> > > >> >>>>>>>>>> - Using watermark generators would also help with
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> another
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> problem of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> suggested interface, namely supporting non-periodic
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> watermarks
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> efficiently.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Need a way to "dispatch" next record to different
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> watermark
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> generators
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Need a way to tell SplitReader to "suspend" a split
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> until a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> certain
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> watermark is reached (event time backpressure)
>> > > >> >>>>>>>>>> - This would in fact be not needed (and thus simpler)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> if
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> we
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> had
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> SplitReader per split and may be a reason to re-open
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> discussion
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> *(6) Watermarks across splits and in the Split
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Enumerator*
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - The split enumerator may need some watermark
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> awareness,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> which
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> should
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> be
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> purely based on split metadata (like create timestamp
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> file
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> splits)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - If there are still more splits with overlapping
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> event
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> time
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> range
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> for
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> split reader, then that split reader should not advance
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> watermark
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> within the split beyond the overlap boundary. Otherwise
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> future
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> splits
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> will
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> produce late data.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - One way to approach this could be that the split
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> enumerator
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> may
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> send
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> watermarks to the readers, and the readers cannot emit
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> watermarks
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> beyond
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that received watermark.
>> > > >> >>>>>>>>>> - Many split enumerators would simply immediately send
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Long.MAX
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> out
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> leave the progress purely to the split readers.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - For event-time alignment / split back pressure, this
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> begs
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> question
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> how we can avoid deadlocks that may arise when splits
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> are
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> suspended
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> for
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> event time back pressure,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> *(7) Batch and streaming Unification*
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Functionality wise, the above design should support
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> both
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Batch often (mostly) does not care about reading "in
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> order"
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> generating watermarks
>> > > >> >>>>>>>>>>     --> Might use different enumerator logic that is
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> more
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> locality
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> aware
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and ignores event time order
>> > > >> >>>>>>>>>>     --> Does not generate watermarks
>> > > >> >>>>>>>>>> - Would be great if bounded sources could be
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> identified
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> at
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> compile
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> time,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> so that "env.addBoundedSource(...)" is type safe and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> can
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> return a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> "BoundedDataStream".
>> > > >> >>>>>>>>>> - Possible to defer this discussion until later
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> *Miscellaneous Comments*
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - Should the source have a TypeInformation for the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> produced
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> type,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> instead
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> of a serializer? We need a type information in the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> stream
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> anyways, and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> can
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> derive the serializer from that. Plus, creating the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> serializer
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> should
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> respect the ExecutionConfig.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> - The TypeSerializer interface is very powerful but
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> also
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> not
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> easy to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> implement. Its purpose is to handle data super
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> efficiently,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> support
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> flexible ways of evolution, etc.
>> > > >> >>>>>>>>>> For metadata I would suggest to look at the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> SimpleVersionedSerializer
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> instead, which is used for example for checkpoint
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> master
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> hooks,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> or for
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> streaming file sink. I think that is is a good match
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> for
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> cases
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> where
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> we
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> do
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> not need more than ser/deser (no copy, etc.) and don't
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> need to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> push
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> versioning out of the serialization paths for best
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> performance
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> (as in
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> TypeSerializer)
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> k.kloudas@data-artisans.com>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Hi Biao,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Thanks for the answer!
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> So given the multi-threaded readers, now we have as
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> open
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> questions:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 1) How do we let the checkpoints pass through our
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> multi-threaded
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> reader
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> operator?
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 2) Do we have separate reader and source operators or
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> not? In
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> strategy
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that has a separate source, the source operator has a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> parallelism of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 1
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> is responsible for split recovery only.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> For the first one, given also the constraints
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> (blocking,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> finite
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> queues,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> etc), I do not have an answer yet.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> For the 2nd, I think that we should go with separate
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> operators
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> for
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> source and the readers, for the following reasons:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 1) This is more aligned with a potential future
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> improvement
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> where the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> split
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> discovery becomes a responsibility of the JobManager
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> readers are
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> pooling more work from the JM.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 2) The source is going to be the "single point of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> truth".
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> It
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> will
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> know
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> what
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> has been processed and what not. If the source and the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> readers
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> are a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> single
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> operator with parallelism > 1, or in general, if the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> split
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> discovery
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> is
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> done by each task individually, then:
>> > > >> >>>>>>>>>>    i) we have to have a deterministic scheme for each
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> reader to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> assign
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> splits to itself (e.g. mod subtaskId). This is not
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> necessarily
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> trivial
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> for
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> all sources.
>> > > >> >>>>>>>>>>    ii) each reader would have to keep a copy of all its
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> processed
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> slpits
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>    iii) the state has to be a union state with a
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> non-trivial
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> merging
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> logic
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> in order to support rescaling.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Two additional points that you raised above:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> i) The point that you raised that we need to keep all
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> splits
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> (processed
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> not-processed) I think is a bit of a strong
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> requirement.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> This
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> would
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> imply
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that for infinite sources the state will grow
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> indefinitely.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> This is
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> problem
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> is even more pronounced if we do not have a single
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> source
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> assigns
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> splits to readers, as each reader will have its own
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> copy
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> of
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> state.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> ii) it is true that for finite sources we need to
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> somehow
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> not
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> close
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> readers when the source/split discoverer finishes. The
>> > > >> >>>>>>>>>> ContinuousFileReaderOperator has a work-around for
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> that.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> It is
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> not
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> elegant,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> and checkpoints are not emitted after closing the
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> source,
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> but
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> this, I
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> believe, is a bigger problem which requires more
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> changes
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> than
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> just
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> refactoring the source interface.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Cheers,
>> > > >> >>>>>>>>>> Kostas
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> --
>> > > >> >>>>>>>>>> Best, Jingsong Lee
>> > > >> >>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>
>> > > >> >>>>>>
>> > > >> >>>>>> --
>> > > >> >>>>>> Best, Jingsong Lee
>> > > >> >>>>>>
>> > > >> >>>>>
>> > > >> >>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >> >
>> > > >>
>> > > >>
>> > >
>> >
>>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Steven,

I think the current proposal is what you mentioned - a Kafka source that
can be constructed in either BOUNDED or UNBOUNDED mode. And Flink can get
the boundedness by invoking getBoundedness().

So one can create a Kafka source by doing something like the following:

new KafkaSource().startOffset(),endOffset(); // A bounded instance.
new KafkaSource().startOffset(); // An unbounded instance.

If users want to have an UNBOUNDED Kafka source that stops at some point.
They can wrap the BOUNDED Kafka source like below:

SourceUtils.asUnbounded(new KafkaSource.startOffset().endOffset());

The wrapped source would be an unbounded Kafka source that stops at the end
offset.

Does that make sense?

Thanks,

Jiangjie (Becket) Qin

On Fri, Dec 20, 2019 at 1:31 PM Jark Wu <im...@gmail.com> wrote:

> Hi,
>
> First of all, I think it is not called "UNBOUNDED", according to the
> FLIP-27, it is called "CONTINUOUS_UNBOUNDED".
> And from the description of the Boundedness in the FLIP-27[1] declares
> clearly what Becket and I think.
>
> public enum Boundedness {
>
>     /**
>      * A bounded source processes the data that is currently available and
> will end after that.
>      *
>      * <p>When a source produces a bounded stream, the runtime may activate
> additional optimizations
>      * that are suitable only for bounded input. Incorrectly producing
> unbounded data when the source
>      * is set to produce a bounded stream will often result in programs
> that do not output any results
>      * and may eventually fail due to runtime errors (out of memory or
> storage).
>      */
>     BOUNDED,
>
>     /**
>      * A continuous unbounded source continuously processes all data as it
> comes.
>      *
>      * <p>The source may run forever (until the program is terminated) or
> might actually end at some point,
>      * based on some source-specific conditions. Because that is not
> transparent to the runtime,
>      * the runtime will use an execution mode for continuous unbounded
> streams whenever this mode
>      * is chosen.
>      */
>     CONTINUOUS_UNBOUNDED
> }
>
> Best,
> Jark
>
> [1]:
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface#FLIP-27:RefactorSourceInterface-Source
>
>
>
> On Fri, 20 Dec 2019 at 12:55, Steven Wu <st...@gmail.com> wrote:
>
> > Becket,
> >
> > Regarding "UNBOUNDED source that stops at some point", I found it
> difficult
> > to grasp what UNBOUNDED really mean.
> >
> > If we want to use Kafka source with an end/stop time, I guess you call it
> > UNBOUNDED kafka source that stops (aka BOUNDED-streaming). The
> > terminology is a little confusing to me. Maybe BOUNDED/UNBOUNDED
> shouldn't
> > be used to categorize source. Just call it Kafka source and it can run in
> > either BOUNDED or UNBOUNDED mode.
> >
> > Thanks,
> > Steven
> >
> > On Thu, Dec 19, 2019 at 7:02 PM Becket Qin <be...@gmail.com> wrote:
> >
> > > I had an offline chat with Jark, and here are some more thoughts:
> > >
> > > 1. From SQL perspective, BOUNDED source leads to the batch execution
> > mode,
> > > UNBOUNDED source leads to the streaming execution mode.
> > > 2. The semantic of UNBOUNDED source is may or may not stop. The
> semantic
> > of
> > > BOUNDED source is will stop.
> > > 3. The semantic of DataStream is may or may not terminate. The semantic
> > of
> > > BoundedDataStream is will terminate.
> > >
> > > Given that, option 3 seems a better option because:
> > > 1. SQL already has strict binding between Boundedness and execution
> mode.
> > > Letting DataStream be consistent would be good.
> > > 2. The semantic of UNBOUNDED source is exactly the same as DataStream.
> So
> > > we should avoid breaking such semantic, i.e. turning some DataStream
> from
> > > "may or may not terminate" to "will terminate".
> > >
> > > For case where users want BOUNDED-streaming combination, they can
> simply
> > > use an UNBOUNDED source that stops at some point. We can even provide a
> > > simple wrapper to wrap a BOUNDED source as an UNBOUNDED source if that
> > > helps. But API wise, option 3 seems telling a pretty good whole story.
> > >
> > > Thanks,
> > >
> > > Jiangjie (Becket) Qin
> > >
> > >
> > >
> > >
> > > On Thu, Dec 19, 2019 at 10:30 PM Becket Qin <be...@gmail.com>
> > wrote:
> > >
> > > > Hi Timo,
> > > >
> > > > Bounded is just a special case of unbounded and every bounded source
> > can
> > > >> also be treated as an unbounded source. This would unify the API if
> > > >> people don't need a bounded operation.
> > > >
> > > >
> > > > With option 3 users can still get a unified API with something like
> > > below:
> > > >
> > > > DataStream boundedStream = env.boundedSource(boundedSource);
> > > > DataStream unboundedStream = env.source(unboundedSource);
> > > >
> > > > So in both cases, users can still use a unified DataStream without
> > > > touching the bounded stream only methods.
> > > > By "unify the API if people don't need the bounded operation". Do you
> > > > expect a DataStream with a Bounded source to have the batch operators
> > and
> > > > scheduler settings as well?
> > > >
> > > >
> > > > If we allow DataStream from BOUNDED source, we will essentially pick
> > > "*modified
> > > > option 2*".
> > > >
> > > > // The source is either bounded or unbounded, but only unbounded
> > > >> operations could be performed on the returned DataStream.
> > > >> DataStream<Type> dataStream = env.source(someSource);
> > > >
> > > >
> > > >> // The source must be a bounded source, otherwise exception is
> thrown.
> > > >> BoundedDataStream<Type> boundedDataStream =
> > > >> env.boundedSource(boundedSource);
> > > >
> > > >
> > > >
> > > > // Add the following method to DataStream
> > > >
> > > > Boundedness DataStream#getBoundedness();
> > > >
> > > >
> > > > From pure logical perspective, Boundedness and runtime settings
> > > > (Stream/Batch) are two orthogonal dimensions. And are specified in
> the
> > > > following way.
> > > >
> > > > *Boundedness* - defined by the source: BOUNDED / UNBOUNDED.
> > > > *Running mode* - defined by the API class: DataStream (Streaming
> mode)
> > /
> > > > BoundedDataStream (batch mode).
> > > >
> > > > Excluding the UNBOUNDED-batch combination, the "*modified option 2"*
> > > > covers the rest three combination. Compared with "*modified option
> 2*",
> > > > the main benefit of option 3 is its simplicity and clearness, by
> tying
> > > > boundedness to running mode and giving up BOUNDED-streaming
> > combination.
> > > >
> > > > Just to be clear, I am fine with either option. But I would like to
> > > > understand a bit more about the bounded-streaming use case and when
> > users
> > > > would prefer this over bounded-batch case, and whether the added
> value
> > > > justifies the additional complexity in the API. Two cases I can think
> > of
> > > > are:
> > > > 1. The records in DataStream will be processed in order, while
> > > > BoundedDataStream processes records without order guarantee.
> > > > 2. DataStream emits intermediate results when processing a finite
> > > dataset,
> > > > while BoundedDataStream only emit the final result. In any case, it
> > could
> > > > be supported by an UNBOUNDED source stopping at some point.
> > > >
> > > > Case 1 is actually misleading because DataStream in general doesn't
> > > really
> > > > support in-order process.
> > > > Case 2 seems a rare use case because the instantaneous intermediate
> > > result
> > > > seems difficult to reason about. In any case, this can be supported
> by
> > an
> > > > UNBOUNDED source that stops at some point.
> > > >
> > > > Is there other use cases for bounded-streaming combination I missed?
> I
> > am
> > > > a little hesitating to put the testing requirement here because
> ideally
> > > I'd
> > > > avoid having public APIs for testing purpose only. And this could be
> > > > resolved by having a UNBOUNDED source stopping at some point as well.
> > > >
> > > > Sorry for the long discussion, but I would really like to make an API
> > > > decision after knowing all the pros and cons.
> > > >
> > > > Thanks,
> > > >
> > > > Jiangjie (Becket) Qin
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On Thu, Dec 19, 2019 at 6:19 PM Timo Walther <tw...@apache.org>
> > wrote:
> > > >
> > > >> Hi Becket,
> > > >>
> > > >> regarding *Option 3* I think we can relax the constraints for
> > > >> env.source():
> > > >>
> > > >> // MySource can be bounded or unbounded
> > > >> DataStream<Type> dataStream = env.source(mySource);
> > > >>
> > > >> // MySource must be bounded, otherwise throws exception.
> > > >> BoundedDataStream<Type> boundedDataStream =
> > env.boundedSource(mySource);
> > > >>
> > > >> Bounded is just a special case of unbounded and every bounded source
> > can
> > > >> also be treated as an unbounded source. This would unify the API if
> > > >> people don't need a bounded operation. It also addresses Jark's
> > > concerns.
> > > >>
> > > >> Regards,
> > > >> Timo
> > > >>
> > > >>
> > > >> On 18.12.19 14:16, Becket Qin wrote:
> > > >> > Hi Jark,
> > > >> >
> > > >> > Please see the reply below:
> > > >> >
> > > >> > Regarding to option#3, my concern is that if we don't support
> > > streaming
> > > >> >> mode for bounded source,
> > > >> >> how could we create a testing source for streaming mode?
> Currently,
> > > >> all the
> > > >> >> testing source for streaming
> > > >> >> are bounded, so that the integration test will finish finally.
> > > >> >
> > > >> >
> > > >> > An UNBOUNDED source does not mean it will never stops. It simply
> > > >> indicates
> > > >> > that the source *may* run forever, so the runtime needs to be
> > prepared
> > > >> for
> > > >> > that, but the task may still stop at some point when it hits some
> > > >> > source-specific condition. So an UNBOUNDED testing source can
> still
> > > >> stop at
> > > >> > some point if needed.
> > > >> >
> > > >> > Regarding to Source#getRecordOrder(), could we have a implicit
> > > contract
> > > >> >> that unbounded source should
> > > >> >> already read in order (i.e. reading partitions in parallel), for
> > > >> bounded
> > > >> >> source the order is not mandatory.
> > > >> >
> > > >> >
> > > >> >
> > > >> >> This is also the behaviors of the current sources.
> > > >> >
> > > >> > 1) a source can't guarantee it reads in strict order, because the
> > > >> producer
> > > >> >> may produce data not in order.
> > > >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can
> > reorder
> > > >> data.
> > > >> >
> > > >> >
> > > >> > It is true that sometimes the source cannot guarantee the record
> > > order,
> > > >> but
> > > >> > sometimes it can. Right now, even for stream processing, there is
> no
> > > >> > processing order guarantee. For example, a join operator may emit
> a
> > > >> later
> > > >> > record which successfully found a join match earlier.
> > > >> > Event order is one of the most important requirements for event
> > > >> processing,
> > > >> > a clear order guarantee would be necessary. That said, I agree
> that
> > > >> right
> > > >> > now even if the sources provide the record order requirement, the
> > > >> runtime
> > > >> > is not able to guarantee that out of the box. So I am OK if we add
> > the
> > > >> > record order to the Source later. But we should avoid misleading
> > users
> > > >> to
> > > >> > make them think the processing order is guaranteed when using the
> > > >> unbounded
> > > >> > runtime.
> > > >> >
> > > >> > Thanks,
> > > >> >
> > > >> > Jiangjie (Becket) Qin
> > > >> >
> > > >> > On Wed, Dec 18, 2019 at 10:29 AM Jark Wu <im...@gmail.com>
> wrote:
> > > >> >
> > > >> >> Hi Becket,
> > > >> >>
> > > >> >> That's great we have reached a consensus on
> > Source#getBoundedness().
> > > >> >>
> > > >> >> Regarding to option#3, my concern is that if we don't support
> > > streaming
> > > >> >> mode for bounded source,
> > > >> >> how could we create a testing source for streaming mode?
> Currently,
> > > >> all the
> > > >> >> testing source for streaming
> > > >> >> are bounded, so that the integration test will finish finally.
> > > >> >>
> > > >> >> Regarding to Source#getRecordOrder(), could we have a implicit
> > > contract
> > > >> >> that unbounded source should
> > > >> >> already read in order (i.e. reading partitions in parallel), for
> > > >> bounded
> > > >> >> source the order is not mandatory.
> > > >> >> This is also the behaviors of the current sources.
> > > >> >> 1) a source can't guarantee it reads in strict order, because the
> > > >> producer
> > > >> >> may produce data not in order.
> > > >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can
> > reorder
> > > >> data.
> > > >> >>
> > > >> >> Best,
> > > >> >> Jark
> > > >> >>
> > > >> >>
> > > >> >>
> > > >> >> On Tue, 17 Dec 2019 at 22:03, Becket Qin <be...@gmail.com>
> > > wrote:
> > > >> >>
> > > >> >>> Hi folks,
> > > >> >>>
> > > >> >>> Thanks for the comments. I am convinced that the Source API
> should
> > > not
> > > >> >> take
> > > >> >>> boundedness as a parameter after it is constructed. What Timo
> and
> > > >> Dawid
> > > >> >>> suggested sounds a reasonable solution to me. So the Source API
> > > would
> > > >> >>> become:
> > > >> >>>
> > > >> >>> Source {
> > > >> >>>      Boundedness getBoundedness();
> > > >> >>> }
> > > >> >>>
> > > >> >>> Assuming the above Source API, in addition to the two options
> > > >> mentioned
> > > >> >> in
> > > >> >>> earlier emails, I am thinking of another option:
> > > >> >>>
> > > >> >>> *Option 3:*
> > > >> >>> // MySource must be unbounded, otherwise throws exception.
> > > >> >>> DataStream<Type> dataStream = env.source(mySource);
> > > >> >>>
> > > >> >>> // MySource must be bounded, otherwise throws exception.
> > > >> >>> BoundedDataStream<Type> boundedDataStream =
> > > >> env.boundedSource(mySource);
> > > >> >>>
> > > >> >>> The pros of this API are:
> > > >> >>>     a) It fits the requirements from Table / SQL well.
> > > >> >>>     b) DataStream users still have type safety (option 2 only
> has
> > > >> partial
> > > >> >>> type safety).
> > > >> >>>     c) Cristal clear boundedness from the API which makes
> > DataStream
> > > >> join
> > > >> >> /
> > > >> >>> connect easy to reason about.
> > > >> >>> The caveats I see,
> > > >> >>>     a) It is inconsistent with Table since Table has one unified
> > > >> >> interface.
> > > >> >>>     b) No streaming mode for bounded source.
> > > >> >>>
> > > >> >>> @Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
> > > >> >>> <al...@ververica.com> what do you think of the approach?
> > > >> >>>
> > > >> >>>
> > > >> >>> Orthogonal to the above API, I am wondering whether boundedness
> is
> > > the
> > > >> >> only
> > > >> >>> dimension needed to describe the characteristic of the Source
> > > >> behavior.
> > > >> >> We
> > > >> >>> may also need to have another dimension of *record order*.
> > > >> >>>
> > > >> >>> For example, when a file source is reading from a directory with
> > > >> bounded
> > > >> >>> records, it may have two ways to read.
> > > >> >>> 1. Read files in parallel.
> > > >> >>> 2. Read files in the chronological order.
> > > >> >>> In both cases, the file source is a Bounded Source. However, the
> > > >> >> processing
> > > >> >>> requirement for downstream may be different. In the first case,
> > the
> > > >> >>> record processing and result emitting order does not matter,
> e.g.
> > > word
> > > >> >>> count. In the second case, the records may have to be processed
> in
> > > the
> > > >> >>> order they were read, e.g. change log processing.
> > > >> >>>
> > > >> >>> If the Source only has a getBoundedness() method, the downstream
> > > >> >> processors
> > > >> >>> would not know whether the records emitted from the Source
> should
> > be
> > > >> >>> processed in order or not. So combining the boundedness and
> record
> > > >> order,
> > > >> >>> we will have four scenarios:
> > > >> >>>
> > > >> >>> *Bounded-StrictOrder*:     A segment of change log.
> > > >> >>> *Bounded-Random*:          Batch Word Count.
> > > >> >>> *Unbounded-StrictOrder*: An infinite change log.
> > > >> >>> *Unbounded-Random*:     Streaming Word Count.
> > > >> >>>
> > > >> >>> Option 2 mentioned in the previous email was kind of trying to
> > > handle
> > > >> the
> > > >> >>> Bounded-StrictOrder case by creating a DataStream from a bounded
> > > >> source,
> > > >> >>> which actually does not work.
> > > >> >>> It looks that we do not have strict order support in some
> > operators
> > > at
> > > >> >> this
> > > >> >>> point, e.g. join. But we may still want to add the semantic to
> the
> > > >> Source
> > > >> >>> first so later on we don't need to change all the source
> > > >> implementations,
> > > >> >>> especially given that many of them will be implemented by 3rd
> > party.
> > > >> >>>
> > > >> >>> Given that, we need another dimension of *Record Order* in the
> > > Source.
> > > >> >> More
> > > >> >>> specifically, the API would become:
> > > >> >>>
> > > >> >>> Source {
> > > >> >>>      Boundedness getBoundedness();
> > > >> >>>      RecordOrder getRecordOrder();
> > > >> >>> }
> > > >> >>>
> > > >> >>> public enum RecordOrder {
> > > >> >>>      /** The record in the DataStream must be processed in its
> > > strict
> > > >> >> order
> > > >> >>> for correctness. */
> > > >> >>>      STRICT,
> > > >> >>>      /** The record in the DataStream can be processed in
> > arbitrary
> > > >> order.
> > > >> >>> */
> > > >> >>>      RANDOM;
> > > >> >>> }
> > > >> >>>
> > > >> >>> Any thoughts?
> > > >> >>>
> > > >> >>> Thanks,
> > > >> >>>
> > > >> >>> Jiangjie (Becket) Qin
> > > >> >>>
> > > >> >>> On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <
> twalthr@apache.org>
> > > >> wrote:
> > > >> >>>
> > > >> >>>> Hi Becket,
> > > >> >>>>
> > > >> >>>> I completely agree with Dawid's suggestion. The information
> about
> > > the
> > > >> >>>> boundedness should come out of the source. Because most of the
> > > >> >> streaming
> > > >> >>>> sources can be made bounded based on some connector specific
> > > >> criterion.
> > > >> >>>> In Kafka, it would be an end offset or end timestamp but in any
> > > case
> > > >> >>>> having just a env.boundedSource() is not enough because
> > parameters
> > > >> for
> > > >> >>>> making the source bounded are missing.
> > > >> >>>>
> > > >> >>>> I suggest to have a simple `isBounded(): Boolean` flag in every
> > > >> source
> > > >> >>>> that might be influenced by a connector builder as Dawid
> > mentioned.
> > > >> >>>>
> > > >> >>>> For type safety during programming, we can still go with *Final
> > > state
> > > >> >>>> 1*. By having a env.source() vs env.boundedSource(). The latter
> > > would
> > > >> >>>> just enforce that the boolean flag is set to `true` and could
> > make
> > > >> >>>> bounded operations available (if we need that actually).
> > > >> >>>>
> > > >> >>>> However, I don't think that we should start making a unified
> > Table
> > > >> API
> > > >> >>>> ununified again. Boundedness is an optimization property. Every
> > > >> bounded
> > > >> >>>> operation can also executed in an unbounded way using
> > > >> >> updates/retraction
> > > >> >>>> or watermarks.
> > > >> >>>>
> > > >> >>>> Regards,
> > > >> >>>> Timo
> > > >> >>>>
> > > >> >>>>
> > > >> >>>> On 15.12.19 14:22, Becket Qin wrote:
> > > >> >>>>> Hi Dawid and Jark,
> > > >> >>>>>
> > > >> >>>>> I think the discussion ultimately boils down to the question
> > that
> > > >> >> which
> > > >> >>>> one
> > > >> >>>>> of the following two final states do we want? Once we make
> this
> > > >> >>> decision,
> > > >> >>>>> everything else can be naturally derived.
> > > >> >>>>>
> > > >> >>>>> *Final state 1*: Separate API for bounded / unbounded
> > DataStream &
> > > >> >>> Table.
> > > >> >>>>> That means any code users write will be valid at the point
> when
> > > they
> > > >> >>>> write
> > > >> >>>>> the code. This is similar to having type safety check at
> > > programming
> > > >> >>>> time.
> > > >> >>>>> For example,
> > > >> >>>>>
> > > >> >>>>> BoundedDataStream extends DataStream {
> > > >> >>>>> // Operations only available for bounded data.
> > > >> >>>>> BoundedDataStream sort(...);
> > > >> >>>>>
> > > >> >>>>> // Interaction with another BoundedStream returns a Bounded
> > > stream.
> > > >> >>>>> BoundedJoinedDataStream join(BoundedDataStream other)
> > > >> >>>>>
> > > >> >>>>> // Interaction with another unbounded stream returns an
> > unbounded
> > > >> >>> stream.
> > > >> >>>>> JoinedDataStream join(DataStream other)
> > > >> >>>>> }
> > > >> >>>>>
> > > >> >>>>> BoundedTable extends Table {
> > > >> >>>>>     // Bounded only operation.
> > > >> >>>>> BoundedTable sort(...);
> > > >> >>>>>
> > > >> >>>>> // Interaction with another BoundedTable returns a
> BoundedTable.
> > > >> >>>>> BoundedTable join(BoundedTable other)
> > > >> >>>>>
> > > >> >>>>> // Interaction with another unbounded table returns an
> unbounded
> > > >> >> table.
> > > >> >>>>> Table join(Table other)
> > > >> >>>>> }
> > > >> >>>>>
> > > >> >>>>> *Final state 2*: One unified API for bounded / unbounded
> > > DataStream
> > > >> /
> > > >> >>>>> Table.
> > > >> >>>>> That unified API may throw exception at DAG compilation time
> if
> > an
> > > >> >>>> invalid
> > > >> >>>>> operation is tried. This is what Table API currently follows.
> > > >> >>>>>
> > > >> >>>>> DataStream {
> > > >> >>>>> // Throws exception if the DataStream is unbounded.
> > > >> >>>>> DataStream sort();
> > > >> >>>>> // Get boundedness.
> > > >> >>>>> Boundedness getBoundedness();
> > > >> >>>>> }
> > > >> >>>>>
> > > >> >>>>> Table {
> > > >> >>>>> // Throws exception if the table has infinite rows.
> > > >> >>>>> Table orderBy();
> > > >> >>>>>
> > > >> >>>>> // Get boundedness.
> > > >> >>>>> Boundedness getBoundedness();
> > > >> >>>>> }
> > > >> >>>>>
> > > >> >>>>> >From what I understand, there is no consensus so far on this
> > > >> decision
> > > >> >>>> yet.
> > > >> >>>>> Whichever final state we choose, we need to make it consistent
> > > >> across
> > > >> >>> the
> > > >> >>>>> entire project. We should avoid the case that Table follows
> one
> > > >> final
> > > >> >>>> state
> > > >> >>>>> while DataStream follows another. Some arguments I am aware of
> > > from
> > > >> >>> both
> > > >> >>>>> sides so far are following:
> > > >> >>>>>
> > > >> >>>>> Arguments for final state 1:
> > > >> >>>>> 1a) Clean API with method safety check at programming time.
> > > >> >>>>> 1b) (Counter 2b) Although SQL does not have programming time
> > error
> > > >> >>>> check, SQL
> > > >> >>>>> is not really a "programming language" per se. So SQL can be
> > > >> >> different
> > > >> >>>> from
> > > >> >>>>> Table and DataStream.
> > > >> >>>>> 1c)  Although final state 2 seems making it easier for SQL to
> > use
> > > >> >> given
> > > >> >>>> it
> > > >> >>>>> is more "config based" than "parameter based", final state 1
> can
> > > >> >>> probably
> > > >> >>>>> also meet what SQL wants by wrapping the Source in
> TableSource /
> > > >> >>>>> TableSourceFactory API if needed.
> > > >> >>>>>
> > > >> >>>>> Arguments for final state 2:
> > > >> >>>>> 2a) The Source API itself seems already sort of following the
> > > >> unified
> > > >> >>> API
> > > >> >>>>> pattern.
> > > >> >>>>> 2b) There is no "programming time" method error check in SQL
> > case,
> > > >> so
> > > >> >>> we
> > > >> >>>>> cannot really achieve final state 1 across the board.
> > > >> >>>>> 2c) It is an easier path given our current status, i.e. Table
> is
> > > >> >>> already
> > > >> >>>>> following final state 2.
> > > >> >>>>> 2d) Users can always explicitly check the boundedness if they
> > want
> > > >> >> to.
> > > >> >>>>>
> > > >> >>>>> As I mentioned earlier, my initial thought was also to have a
> > > >> >>>>> "configuration based" Source rather than a "parameter based"
> > > Source.
> > > >> >> So
> > > >> >>>> it
> > > >> >>>>> is completely possible that I missed some important
> > consideration
> > > or
> > > >> >>>> design
> > > >> >>>>> principles that we want to enforce for the project. It would
> be
> > > good
> > > >> >>>>> if @Stephan
> > > >> >>>>> Ewen <st...@ververica.com> and @Aljoscha Krettek <
> > > >> >>>> aljoscha@ververica.com> can
> > > >> >>>>> also provide more thoughts on this.
> > > >> >>>>>
> > > >> >>>>>
> > > >> >>>>> Re: Jingsong
> > > >> >>>>>
> > > >> >>>>> As you said, there are some batched system source, like
> > > parquet/orc
> > > >> >>>> source.
> > > >> >>>>>> Could we have the batch emit interface to improve
> performance?
> > > The
> > > >> >>>> queue of
> > > >> >>>>>> per record may cause performance degradation.
> > > >> >>>>>
> > > >> >>>>>
> > > >> >>>>> The current interface does not necessarily cause performance
> > > problem
> > > >> >>> in a
> > > >> >>>>> multi-threading case. In fact, the base implementation allows
> > > >> >>>> SplitReaders
> > > >> >>>>> to add a batch <E> of records<T> to the records queue<E>, so
> > each
> > > >> >>> element
> > > >> >>>>> in the records queue would be a batch <E>. In this case, when
> > the
> > > >> >> main
> > > >> >>>>> thread polls records, it will take a batch <E> of records <T>
> > from
> > > >> >> the
> > > >> >>>>> shared records queue and process the records <T> in a batch
> > > manner.
> > > >> >>>>>
> > > >> >>>>> Thanks,
> > > >> >>>>>
> > > >> >>>>> Jiangjie (Becket) Qin
> > > >> >>>>>
> > > >> >>>>> On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <
> > > jingsonglee0@gmail.com
> > > >> >
> > > >> >>>> wrote:
> > > >> >>>>>
> > > >> >>>>>> Hi Becket,
> > > >> >>>>>>
> > > >> >>>>>> I also have some performance concerns too.
> > > >> >>>>>>
> > > >> >>>>>> If I understand correctly, SourceOutput will emit data per
> > record
> > > >> >> into
> > > >> >>>> the
> > > >> >>>>>> queue? I'm worried about the multithreading performance of
> this
> > > >> >> queue.
> > > >> >>>>>>
> > > >> >>>>>>> One example is some batched messaging systems which only
> have
> > an
> > > >> >>> offset
> > > >> >>>>>> for the entire batch instead of individual messages in the
> > batch.
> > > >> >>>>>>
> > > >> >>>>>> As you said, there are some batched system source, like
> > > parquet/orc
> > > >> >>>> source.
> > > >> >>>>>> Could we have the batch emit interface to improve
> performance?
> > > The
> > > >> >>>> queue of
> > > >> >>>>>> per record may cause performance degradation.
> > > >> >>>>>>
> > > >> >>>>>> Best,
> > > >> >>>>>> Jingsong Lee
> > > >> >>>>>>
> > > >> >>>>>> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com>
> > > wrote:
> > > >> >>>>>>
> > > >> >>>>>>> Hi Becket,
> > > >> >>>>>>>
> > > >> >>>>>>> I think Dawid explained things clearly and makes a lot of
> > sense.
> > > >> >>>>>>> I'm also in favor of #2, because #1 doesn't work for our
> > future
> > > >> >>> unified
> > > >> >>>>>>> envrionment.
> > > >> >>>>>>>
> > > >> >>>>>>> You can see the vision in this documentation [1]. In the
> > future,
> > > >> we
> > > >> >>>> would
> > > >> >>>>>>> like to
> > > >> >>>>>>> drop the global streaming/batch mode in SQL (i.e.
> > > >> >>>>>>> EnvironmentSettings#inStreamingMode/inBatchMode).
> > > >> >>>>>>> A source is bounded or unbounded once defined, so queries
> can
> > be
> > > >> >>>> inferred
> > > >> >>>>>>> from source to run
> > > >> >>>>>>> in streaming or batch or hybrid mode. However, in #1, we
> will
> > > lose
> > > >> >>> this
> > > >> >>>>>>> ability because the framework
> > > >> >>>>>>> doesn't know whether the source is bounded or unbounded.
> > > >> >>>>>>>
> > > >> >>>>>>> Best,
> > > >> >>>>>>> Jark
> > > >> >>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>> [1]:
> > > >> >>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
> > > >> >>>>>>>
> > > >> >>>>>>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <
> > > piotr@ververica.com
> > > >> >
> > > >> >>>>>> wrote:
> > > >> >>>>>>>
> > > >> >>>>>>>> Hi,
> > > >> >>>>>>>>
> > > >> >>>>>>>> Regarding the:
> > > >> >>>>>>>>
> > > >> >>>>>>>> Collection<E> getNextRecords()
> > > >> >>>>>>>>
> > > >> >>>>>>>> I’m pretty sure such design would unfortunately impact the
> > > >> >>> performance
> > > >> >>>>>>>> (accessing and potentially creating the collection on the
> hot
> > > >> >> path).
> > > >> >>>>>>>>
> > > >> >>>>>>>> Also the
> > > >> >>>>>>>>
> > > >> >>>>>>>> InputStatus emitNext(DataOutput<T> output) throws
> Exception;
> > > >> >>>>>>>> or
> > > >> >>>>>>>> Status pollNext(SourceOutput<T> sourceOutput) throws
> > Exception;
> > > >> >>>>>>>>
> > > >> >>>>>>>> Gives us some opportunities in the future, to allow Source
> > hot
> > > >> >>> looping
> > > >> >>>>>>>> inside, until it receives some signal “please exit because
> of
> > > >> some
> > > >> >>>>>>> reasons”
> > > >> >>>>>>>> (output collector could return such hint upon collecting
> the
> > > >> >>> result).
> > > >> >>>>>> But
> > > >> >>>>>>>> that’s another topic outside of this FLIP’s scope.
> > > >> >>>>>>>>
> > > >> >>>>>>>> Piotrek
> > > >> >>>>>>>>
> > > >> >>>>>>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <
> > trohrmann@apache.org
> > > >
> > > >> >>>>>> wrote:
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> Hi Becket,
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> quick clarification from my side because I think you
> > > >> >> misunderstood
> > > >> >>> my
> > > >> >>>>>>>>> question. I did not suggest to let the SourceReader return
> > > only
> > > >> a
> > > >> >>>>>>> single
> > > >> >>>>>>>>> record at a time when calling getNextRecords. As the
> return
> > > type
> > > >> >>>>>>>> indicates,
> > > >> >>>>>>>>> the method can return an arbitrary number of records.
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> Cheers,
> > > >> >>>>>>>>> Till
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> > > >> >>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> > > >> >>>>>>>>> wrote:
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>> Hi Becket,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Issue #1 - Design of Source interface
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I mentioned the lack of a method like
> > > >> >>>>>>>> Source#createEnumerator(Boundedness
> > > >> >>>>>>>>>> boundedness, SplitEnumeratorContext context), because
> > without
> > > >> >> the
> > > >> >>>>>>>> current
> > > >> >>>>>>>>>> proposal is not complete/does not work.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> If we say that boundedness is an intrinsic property of a
> > > source
> > > >> >>> imo
> > > >> >>>>>> we
> > > >> >>>>>>>>>> don't need the Source#createEnumerator(Boundedness
> > > boundedness,
> > > >> >>>>>>>>>> SplitEnumeratorContext context) method.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Assuming a source from my previous example:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
> > > >> >>>>>>>>>>    ...
> > > >> >>>>>>>>>>    .untilTimestamp(...)
> > > >> >>>>>>>>>>    .build()
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Would the enumerator differ if created like
> > > >> >>>>>>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs
> > source
> > > >> >>>>>>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating
> > > myself,
> > > >> >> but
> > > >> >>>>>>> this
> > > >> >>>>>>>> is
> > > >> >>>>>>>>>> the part that my opinion differ the most from the current
> > > >> >>> proposal.
> > > >> >>>>>> I
> > > >> >>>>>>>>>> really think it should always be the source that tells if
> > it
> > > is
> > > >> >>>>>>> bounded
> > > >> >>>>>>>> or
> > > >> >>>>>>>>>> not. In the current proposal methods
> > > >> >> continousSource/boundedSource
> > > >> >>>>>>>> somewhat
> > > >> >>>>>>>>>> reconfigure the source, which I think is misleading.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I think a call like:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
> > > >> >>>>>>>>>>    ...
> > > >> >>>>>>>>>>    .readContinously() / readUntilLatestOffset() /
> > > >> >>> readUntilTimestamp
> > > >> >>>> /
> > > >> >>>>>>>> readUntilOffsets / ...
> > > >> >>>>>>>>>>    .build()
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> is way cleaner (and expressive) than
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Source source = KafkaSource.builder()
> > > >> >>>>>>>>>>    ...
> > > >> >>>>>>>>>>    .build()
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> env.continousSource(source) // which actually underneath
> > > would
> > > >> >>> call
> > > >> >>>>>>>> createEnumerator(CONTINUOUS, ctx) which would be equivalent
> > to
> > > >> >>>>>>>> source.readContinously().createEnumerator(ctx)
> > > >> >>>>>>>>>> // or
> > > >> >>>>>>>>>> env.boundedSource(source) // which actually underneath
> > would
> > > >> >> call
> > > >> >>>>>>>> createEnumerator(BOUNDED, ctx) which would be equivalent to
> > > >> >>>>>>>> source.readUntilLatestOffset().createEnumerator(ctx)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Sorry for the comparison, but to me it seems there is too
> > > much
> > > >> >>> magic
> > > >> >>>>>>>>>> happening underneath those two calls.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I really believe the Source interface should have
> > > >> getBoundedness
> > > >> >>>>>>> method
> > > >> >>>>>>>>>> instead of (supportBoundedness) +
> > > createEnumerator(Boundedness,
> > > >> >>> ...)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Issue #2 - Design of
> > > >> >>>>>>>>>>
> > > >> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> As you might have guessed I am slightly in favor of
> option
> > #2
> > > >> >>>>>>> modified.
> > > >> >>>>>>>>>> Yes I am aware every step of the dag would have to be
> able
> > to
> > > >> >> say
> > > >> >>> if
> > > >> >>>>>>> it
> > > >> >>>>>>>> is
> > > >> >>>>>>>>>> bounded or not. I have a feeling it would be easier to
> > > express
> > > >> >>> cross
> > > >> >>>>>>>>>> bounded/unbounded operations, but I must admit I have not
> > > >> >> thought
> > > >> >>> it
> > > >> >>>>>>>>>> through thoroughly, In the spirit of batch is just a
> > special
> > > >> >> case
> > > >> >>> of
> > > >> >>>>>>>>>> streaming I thought BoundedStream would extend from
> > > DataStream.
> > > >> >>>>>>> Correct
> > > >> >>>>>>>> me
> > > >> >>>>>>>>>> if I am wrong. In such a setup the cross
> bounded/unbounded
> > > >> >>> operation
> > > >> >>>>>>>> could
> > > >> >>>>>>>>>> be expressed quite easily I think:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> DataStream {
> > > >> >>>>>>>>>>    DataStream join(DataStream, ...); // we could not
> really
> > > >> tell
> > > >> >> if
> > > >> >>>>>> the
> > > >> >>>>>>>> result is bounded or not, but because bounded stream is a
> > > special
> > > >> >>> case
> > > >> >>>>>> of
> > > >> >>>>>>>> unbounded the API object is correct, irrespective if the
> left
> > > or
> > > >> >>> right
> > > >> >>>>>>> side
> > > >> >>>>>>>> of the join is bounded
> > > >> >>>>>>>>>> }
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> BoundedStream extends DataStream {
> > > >> >>>>>>>>>>    BoundedStream join(BoundedStream, ...); // only if
> both
> > > >> sides
> > > >> >>> are
> > > >> >>>>>>>> bounded the result can be bounded as well. However we do
> have
> > > >> >> access
> > > >> >>>> to
> > > >> >>>>>>> the
> > > >> >>>>>>>> DataStream#join here, so you can still join with a
> DataStream
> > > >> >>>>>>>>>> }
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On the other hand I also see benefits of two completely
> > > >> >> disjointed
> > > >> >>>>>>> APIs,
> > > >> >>>>>>>>>> as we could prohibit some streaming calls in the bounded
> > > API. I
> > > >> >>>>>> can't
> > > >> >>>>>>>> think
> > > >> >>>>>>>>>> of any unbounded operators that could not be implemented
> > for
> > > >> >>> bounded
> > > >> >>>>>>>> stream.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Besides I think we both agree we don't like the method:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> DataStream boundedStream(Source)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> suggested in the current state of the FLIP. Do we ? :)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Best,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Dawid
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On 10/12/2019 18:57, Becket Qin wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi folks,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks for the discussion, great feedback. Also thanks
> > Dawid
> > > >> for
> > > >> >>> the
> > > >> >>>>>>>>>> explanation, it is much clearer now.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> One thing that is indeed missing from the FLIP is how the
> > > >> >>>>>> boundedness
> > > >> >>>>>>> is
> > > >> >>>>>>>>>> passed to the Source implementation. So the API should be
> > > >> >>>>>>>>>> Source#createEnumerator(Boundedness boundedness,
> > > >> >>>>>>> SplitEnumeratorContext
> > > >> >>>>>>>>>> context)
> > > >> >>>>>>>>>> And we can probably remove the
> > > >> >>> Source#supportBoundedness(Boundedness
> > > >> >>>>>>>>>> boundedness) method.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Assuming we have that, we are essentially choosing from
> one
> > > of
> > > >> >> the
> > > >> >>>>>>>>>> following two options:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Option 1:
> > > >> >>>>>>>>>> // The source is continuous source, and only unbounded
> > > >> >> operations
> > > >> >>>>>> can
> > > >> >>>>>>> be
> > > >> >>>>>>>>>> performed.
> > > >> >>>>>>>>>> DataStream<Type> datastream =
> > > env.continuousSource(someSource);
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> // The source is bounded source, both bounded and
> unbounded
> > > >> >>>>>> operations
> > > >> >>>>>>>> can
> > > >> >>>>>>>>>> be performed.
> > > >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> > > >> >>>>>>>> env.boundedSource(someSource);
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>    - Pros:
> > > >> >>>>>>>>>>         a) explicit boundary between bounded / unbounded
> > > >> streams,
> > > >> >>> it
> > > >> >>>>>> is
> > > >> >>>>>>>>>> quite simple and clear to the users.
> > > >> >>>>>>>>>>    - Cons:
> > > >> >>>>>>>>>>         a) For applications that do not involve bounded
> > > >> >> operations,
> > > >> >>>>>> they
> > > >> >>>>>>>>>> still have to call different API to distinguish bounded /
> > > >> >>> unbounded
> > > >> >>>>>>>> streams.
> > > >> >>>>>>>>>>         b) No support for bounded stream to run in a
> > > streaming
> > > >> >>>> runtime
> > > >> >>>>>>>>>> setting, i.e. scheduling and operators behaviors.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Option 2:
> > > >> >>>>>>>>>> // The source is either bounded or unbounded, but only
> > > >> unbounded
> > > >> >>>>>>>> operations
> > > >> >>>>>>>>>> could be performed on the returned DataStream.
> > > >> >>>>>>>>>> DataStream<Type> dataStream = env.source(someSource);
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> // The source must be a bounded source, otherwise
> exception
> > > is
> > > >> >>>>>> thrown.
> > > >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> > > >> >>>>>>>>>> env.boundedSource(boundedSource);
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> The pros and cons are exactly the opposite of option 1.
> > > >> >>>>>>>>>>    - Pros:
> > > >> >>>>>>>>>>         a) For applications that do not involve bounded
> > > >> >> operations,
> > > >> >>>>>> they
> > > >> >>>>>>>>>> still have to call different API to distinguish bounded /
> > > >> >>> unbounded
> > > >> >>>>>>>> streams.
> > > >> >>>>>>>>>>         b) Support for bounded stream to run in a
> streaming
> > > >> >> runtime
> > > >> >>>>>>>> setting,
> > > >> >>>>>>>>>> i.e. scheduling and operators behaviors.
> > > >> >>>>>>>>>>    - Cons:
> > > >> >>>>>>>>>>         a) Bounded / unbounded streams are kind of mixed,
> > > i.e.
> > > >> >>> given
> > > >> >>>> a
> > > >> >>>>>>>>>> DataStream, it is not clear whether it is bounded or not,
> > > >> unless
> > > >> >>> you
> > > >> >>>>>>>> have
> > > >> >>>>>>>>>> the access to its source.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> If we only think from the Source API perspective, option
> 2
> > > >> >> seems a
> > > >> >>>>>>>> better
> > > >> >>>>>>>>>> choice because functionality wise it is a superset of
> > option
> > > 1,
> > > >> >> at
> > > >> >>>>>> the
> > > >> >>>>>>>> cost
> > > >> >>>>>>>>>> of some seemingly acceptable ambiguity in the DataStream
> > API.
> > > >> >>>>>>>>>> But if we look at the DataStream API as a whole, option 1
> > > seems
> > > >> >> a
> > > >> >>>>>>>> clearer
> > > >> >>>>>>>>>> choice. For example, some times a library may have to
> know
> > > >> >>> whether a
> > > >> >>>>>>>>>> certain task will finish or not. And it would be
> difficult
> > to
> > > >> >> tell
> > > >> >>>>>> if
> > > >> >>>>>>>> the
> > > >> >>>>>>>>>> input is a DataStream, unless additional information is
> > > >> provided
> > > >> >>> all
> > > >> >>>>>>> the
> > > >> >>>>>>>>>> way from the Source. One possible solution is to have a
> > > >> >> *modified
> > > >> >>>>>>>> option 2*
> > > >> >>>>>>>>>> which adds a method to the DataStream API to indicate
> > > >> >> boundedness,
> > > >> >>>>>>> such
> > > >> >>>>>>>> as
> > > >> >>>>>>>>>> getBoundedness(). It would solve the problem with a
> > potential
> > > >> >>>>>>> confusion
> > > >> >>>>>>>> of
> > > >> >>>>>>>>>> what is difference between a DataStream with
> > > >> >> getBoundedness()=true
> > > >> >>>>>>> and a
> > > >> >>>>>>>>>> BoundedDataStream. But that seems not super difficult to
> > > >> >> explain.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> So from API's perspective, I don't have a strong opinion
> > > >> between
> > > >> >>>>>>>> *option 1*
> > > >> >>>>>>>>>> and *modified option 2. *I like the cleanness of option
> 1,
> > > but
> > > >> >>>>>>> modified
> > > >> >>>>>>>>>> option 2 would be more attractive if we have concrete use
> > > case
> > > >> >> for
> > > >> >>>>>> the
> > > >> >>>>>>>>>> "Bounded stream with unbounded streaming runtime
> settings".
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Re: Till
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Maybe this has already been asked before but I was
> > wondering
> > > >> why
> > > >> >>> the
> > > >> >>>>>>>>>> SourceReader interface has the method pollNext which
> hands
> > > the
> > > >> >>>>>>>>>> responsibility of outputting elements to the SourceReader
> > > >> >>>>>>>> implementation?
> > > >> >>>>>>>>>> Has this been done for backwards compatibility reasons
> with
> > > the
> > > >> >>> old
> > > >> >>>>>>>> source
> > > >> >>>>>>>>>> interface? If not, then one could define a Collection<E>
> > > >> >>>>>>>> getNextRecords()
> > > >> >>>>>>>>>> method which returns the currently retrieved records and
> > then
> > > >> >> the
> > > >> >>>>>>> caller
> > > >> >>>>>>>>>> emits them outside of the SourceReader. That way the
> > > interface
> > > >> >>> would
> > > >> >>>>>>> not
> > > >> >>>>>>>>>> allow to implement an outputting loop where we never hand
> > > back
> > > >> >>>>>> control
> > > >> >>>>>>>> to
> > > >> >>>>>>>>>> the caller. At the moment, this contract can be easily
> > broken
> > > >> >> and
> > > >> >>> is
> > > >> >>>>>>>> only
> > > >> >>>>>>>>>> mentioned loosely in the JavaDocs.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> The primary reason we handover the SourceOutput to the
> > > >> >>> SourceReader
> > > >> >>>>>> is
> > > >> >>>>>>>>>> because sometimes it is difficult for a SourceReader to
> > emit
> > > >> one
> > > >> >>>>>>> record
> > > >> >>>>>>>> at
> > > >> >>>>>>>>>> a time. One example is some batched messaging systems
> which
> > > >> only
> > > >> >>>>>> have
> > > >> >>>>>>> an
> > > >> >>>>>>>>>> offset for the entire batch instead of individual
> messages
> > in
> > > >> >> the
> > > >> >>>>>>>> batch. In
> > > >> >>>>>>>>>> that case, returning one record at a time would leave the
> > > >> >>>>>> SourceReader
> > > >> >>>>>>>> in
> > > >> >>>>>>>>>> an uncheckpointable state because they can only
> checkpoint
> > at
> > > >> >> the
> > > >> >>>>>>> batch
> > > >> >>>>>>>>>> boundaries.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <
> > > >> >>> trohrmann@apache.org
> > > >> >>>>>>>> <ma...@apache.org>> <trohrmann@apache.org
> > <mailto:
> > > >> >>>>>>>> trohrmann@apache.org>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi everyone,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> thanks for drafting this FLIP. It reads very well.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Concerning Dawid's proposal, I tend to agree. The
> > boundedness
> > > >> >>> could
> > > >> >>>>>>> come
> > > >> >>>>>>>>>> from the source and tell the system how to treat the
> > operator
> > > >> >>>>>>>> (scheduling
> > > >> >>>>>>>>>> wise). From a user's perspective it should be fine to get
> > > back
> > > >> a
> > > >> >>>>>>>> DataStream
> > > >> >>>>>>>>>> when calling env.source(boundedSource) if he does not
> need
> > > >> >> special
> > > >> >>>>>>>>>> operations defined on a BoundedDataStream. If he needs
> > this,
> > > >> >> then
> > > >> >>>>>> one
> > > >> >>>>>>>> could
> > > >> >>>>>>>>>> use the method BoundedDataStream
> > > >> >> env.boundedSource(boundedSource).
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> If possible, we could enforce the proper usage of
> > > >> >>>>>> env.boundedSource()
> > > >> >>>>>>> by
> > > >> >>>>>>>>>> introducing a BoundedSource type so that one cannot pass
> an
> > > >> >>>>>>>>>> unbounded source to it. That way users would not be able
> to
> > > >> >> shoot
> > > >> >>>>>>>>>> themselves in the foot.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Maybe this has already been asked before but I was
> > wondering
> > > >> why
> > > >> >>> the
> > > >> >>>>>>>>>> SourceReader interface has the method pollNext which
> hands
> > > the
> > > >> >>>>>>>>>> responsibility of outputting elements to the SourceReader
> > > >> >>>>>>>> implementation?
> > > >> >>>>>>>>>> Has this been done for backwards compatibility reasons
> with
> > > the
> > > >> >>> old
> > > >> >>>>>>>> source
> > > >> >>>>>>>>>> interface? If not, then one could define a Collection<E>
> > > >> >>>>>>>> getNextRecords()
> > > >> >>>>>>>>>> method which returns the currently retrieved records and
> > then
> > > >> >> the
> > > >> >>>>>>> caller
> > > >> >>>>>>>>>> emits them outside of the SourceReader. That way the
> > > interface
> > > >> >>> would
> > > >> >>>>>>> not
> > > >> >>>>>>>>>> allow to implement an outputting loop where we never hand
> > > back
> > > >> >>>>>> control
> > > >> >>>>>>>> to
> > > >> >>>>>>>>>> the caller. At the moment, this contract can be easily
> > broken
> > > >> >> and
> > > >> >>> is
> > > >> >>>>>>>> only
> > > >> >>>>>>>>>> mentioned loosely in the JavaDocs.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Cheers,
> > > >> >>>>>>>>>> Till
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <
> > > >> >>> jingsonglee0@gmail.com
> > > >> >>>>>>>> <ma...@gmail.com>> <jingsonglee0@gmail.com
> > > >> <mailto:
> > > >> >>>>>>>> jingsonglee0@gmail.com>>
> > > >> >>>>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi all,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I think current design is good.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> My understanding is:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> For execution mode: bounded mode and continuous mode,
> It's
> > > >> >> totally
> > > >> >>>>>>>>>> different. I don't think we have the ability to integrate
> > the
> > > >> >> two
> > > >> >>>>>>> models
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> at
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> present. It's about scheduling, memory, algorithms,
> States,
> > > >> etc.
> > > >> >>> we
> > > >> >>>>>>>>>> shouldn't confuse them.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> For source capabilities: only bounded, only continuous,
> > both
> > > >> >>> bounded
> > > >> >>>>>>> and
> > > >> >>>>>>>>>> continuous.
> > > >> >>>>>>>>>> I think Kafka is a source that can be ran both bounded
> > > >> >>>>>>>>>> and continuous execution mode.
> > > >> >>>>>>>>>> And Kafka with end offset should be ran both bounded
> > > >> >>>>>>>>>> and continuous execution mode.  Using apache Beam with
> > Flink
> > > >> >>>>>> runner, I
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> used
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> to run a "bounded" Kafka in streaming mode. For our
> > previous
> > > >> >>>>>>> DataStream,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> it
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> is not necessarily required that the source cannot be
> > > bounded.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> So it is my thought for Dawid's question:
> > > >> >>>>>>>>>> 1.pass a bounded source to continuousSource() +1
> > > >> >>>>>>>>>> 2.pass a continuous source to boundedSource() -1, should
> > > throw
> > > >> >>>>>>>> exception.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> In StreamExecutionEnvironment, continuousSource and
> > > >> >> boundedSource
> > > >> >>>>>>> define
> > > >> >>>>>>>>>> the execution mode. It defines a clear boundary of
> > execution
> > > >> >> mode.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Best,
> > > >> >>>>>>>>>> Jingsong Lee
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <
> imjark@gmail.com
> > > >> >>> <mailto:
> > > >> >>>>>>>> imjark@gmail.com>> <imjark@gmail.com <mailto:
> > imjark@gmail.com
> > > >>
> > > >> >>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I agree with Dawid's point that the boundedness
> information
> > > >> >> should
> > > >> >>>>>>> come
> > > >> >>>>>>>>>> from the source itself (e.g. the end timestamp), not
> > through
> > > >> >>>>>>>>>> env.boundedSouce()/continuousSource().
> > > >> >>>>>>>>>> I think if we want to support something like
> `env.source()`
> > > >> that
> > > >> >>>>>>> derive
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> execution mode from source,
> > > `supportsBoundedness(Boundedness)`
> > > >> >>>>>>>>>> method is not enough, because we don't know whether it is
> > > >> >> bounded
> > > >> >>> or
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> not.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Best,
> > > >> >>>>>>>>>> Jark
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
> > > >> >>>>>> dwysakowicz@apache.org
> > > >> >>>>>>>> <ma...@apache.org>> <dwysakowicz@apache.org
> > > >> <mailto:
> > > >> >>>>>>>> dwysakowicz@apache.org>>
> > > >> >>>>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> One more thing. In the current proposal, with the
> > > >> >>>>>>>>>> supportsBoundedness(Boundedness) method and the
> boundedness
> > > >> >> coming
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> from
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> either continuousSource or boundedSource I could not find
> > how
> > > >> >> this
> > > >> >>>>>>>>>> information is fed back to the SplitEnumerator.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Best,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Dawid
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On 09/12/2019 13:52, Becket Qin wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi Dawid,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks for the comments. This actually brings another
> > > relevant
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> question
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> about what does a "bounded source" imply. I actually had
> > the
> > > >> >> same
> > > >> >>>>>>>>>> impression when I look at the Source API. Here is what I
> > > >> >>> understand
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> after
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> some discussion with Stephan. The bounded source has the
> > > >> >> following
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> impacts.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 1. API validity.
> > > >> >>>>>>>>>> - A bounded source generates a bounded stream so some
> > > >> operations
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> only
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> works for bounded records would be performed, e.g. sort.
> > > >> >>>>>>>>>> - To expose these bounded stream only APIs, there are two
> > > >> >> options:
> > > >> >>>>>>>>>>       a. Add them to the DataStream API and throw
> exception
> > > if
> > > >> a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> method
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> is
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> called on an unbounded stream.
> > > >> >>>>>>>>>>       b. Create a BoundedDataStream class which is
> returned
> > > >> from
> > > >> >>>>>>>>>> env.boundedSource(), while DataStream is returned from
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> env.continousSource().
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Note that this cannot be done by having single
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> env.source(theSource)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> even
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the Source has a getBoundedness() method.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 2. Scheduling
> > > >> >>>>>>>>>> - A bounded source could be computed stage by stage
> without
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> bringing
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> up
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> all
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the tasks at the same time.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 3. Operator behaviors
> > > >> >>>>>>>>>> - A bounded source indicates the records are finite so
> some
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> operators
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> can
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> wait until it receives all the records before it starts
> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> processing.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> In the above impact, only 1 is relevant to the API
> design.
> > > And
> > > >> >> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> current
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> proposal in FLIP-27 is following 1.b.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> // boundedness depends of source property, imo this
> should
> > > >> >> always
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> be
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> preferred
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> In your proposal, does DataStream have bounded stream
> only
> > > >> >>> methods?
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> It
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> looks it should have, otherwise passing a bounded Source
> to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> env.source()
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> would be confusing. In that case, we will essentially do
> > 1.a
> > > if
> > > >> >> an
> > > >> >>>>>>>>>> unbounded Source is created from
> > env.source(unboundedSource).
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> If we have the methods only supported for bounded streams
> > in
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> DataStream,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> it
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> seems a little weird to have a separate BoundedDataStream
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> interface.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Am I understand it correctly?
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi all,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Really well written proposal and very important one. I
> must
> > > >> >> admit
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> have
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> not understood all the intricacies of it yet.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> One question I have though is about where does the
> > > information
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> about
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> boundedness come from. I think in most cases it is a
> > property
> > > >> of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> source. As you described it might be e.g. end offset, a
> > flag
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> should
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> it
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> monitor new splits etc. I think it would be a really nice
> > use
> > > >> >> case
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> be
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> able to say:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> new KafkaSource().readUntil(long timestamp),
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> which could work as an "end offset". Moreover I think all
> > > >> >> Bounded
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> sources
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> support continuous mode, but no intrinsically continuous
> > > source
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> support
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Bounded mode. If I understood the proposal correctly it
> > > suggest
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> boundedness sort of "comes" from the outside of the
> source,
> > > >> from
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> invokation of either boundedStream or continousSource.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I am wondering if it would make sense to actually change
> > the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> method
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> boolean Source#supportsBoundedness(Boundedness)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Boundedness Source#getBoundedness().
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> As for the methods #boundedSource, #continousSource,
> > assuming
> > > >> >> the
> > > >> >>>>>>>>>> boundedness is property of the source they do not affect
> > how
> > > >> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> enumerator
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> works, but mostly how the dag is scheduled, right? I am
> not
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> against
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> those
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> methods, but I think it is a very specific use case to
> > > actually
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> override
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the property of the source. In general I would expect
> users
> > > to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> only
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> call
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> env.source(theSource), where the source tells if it is
> > > bounded
> > > >> >> or
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> not. I
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> would suggest considering following set of methods:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> // boundedness depends of source property, imo this
> should
> > > >> >> always
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> be
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> preferred
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> // always continous execution, whether bounded or
> unbounded
> > > >> >> source
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> DataStream<MyType> boundedStream =
> > > >> >> env.continousSource(theSource);
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> // imo this would make sense if the BoundedDataStream
> > > provides
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> additional features unavailable for continous mode
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> BoundedDataStream<MyType> batch =
> > > env.boundedSource(theSource);
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Best,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Dawid
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks, Becket, for updating this.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I agree with moving the aspects you mentioned into
> separate
> > > >> >> FLIPs
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> -
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> this
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> one way becoming unwieldy in size.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> +1 to the FLIP in its current state. Its a very detailed
> > > >> >> write-up,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> nicely
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> done!
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <
> > > >> becket.qin@gmail.com
> > > >> >>>>>>>> <ma...@gmail.com>> <becket.qin@gmail.com
> > <mailto:
> > > >> >>>>>>>> becket.qin@gmail.com>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> becket.qin@gmail.com <ma...@gmail.com>>
> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi all,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Sorry for the long belated update. I have updated FLIP-27
> > > wiki
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> page
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> with
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the latest proposals. Some noticeable changes include:
> > > >> >>>>>>>>>> 1. A new generic communication mechanism between
> > > >> SplitEnumerator
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> SourceReader.
> > > >> >>>>>>>>>> 2. Some detail API method signature changes.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> We left a few things out of this FLIP and will address
> them
> > > in
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> separate
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> FLIPs. Including:
> > > >> >>>>>>>>>> 1. Per split event time.
> > > >> >>>>>>>>>> 2. Event time alignment.
> > > >> >>>>>>>>>> 3. Fine grained failover for SplitEnumerator failure.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Please let us know if you have any question.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Jiangjie (Becket) Qin
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <
> > > sewen@apache.org
> > > >> >>>>>>> <mailto:
> > > >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:
> > sewen@apache.org
> > > >>
> > > >> <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi  Łukasz!
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Becket and me are working hard on figuring out the last
> > > details
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> implementing the first PoC. We would update the FLIP
> > > hopefully
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> next
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> week.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> There is a fair chance that a first version of this will
> be
> > > in
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 1.10,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> but
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> think it will take another release to battle test it and
> > > >> migrate
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> connectors.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Best,
> > > >> >>>>>>>>>> Stephan
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <
> > > >> >> ljd@touk.pl
> > > >> >>>>>>>> <ma...@touk.pl>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> ljd@touk.pl <ma...@touk.pl>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> This proposal looks very promising for us. Do you have
> any
> > > >> plans
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> in
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> which
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Flink release it is going to be released? We are thinking
> > on
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> using a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Data
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Set API for our future use cases but on the other hand
> Data
> > > Set
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> API
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> is
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> going to be deprecated so using proposed bounded data
> > streams
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> solution
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> could be more viable in the long term.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks,
> > > >> >>>>>>>>>> Łukasz
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On 2019/10/01 15:48:03, Thomas Weise <
> > thomas.weise@gmail.com
> > > >> >>>>>> <mailto:
> > > >> >>>>>>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
> > > >> >>>>>>>> thomas.weise@gmail.com>> <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> thomas.weise@gmail.com <ma...@gmail.com>>
> > > wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks for putting together this proposal!
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I see that the "Per Split Event Time" and "Event Time
> > > >> Alignment"
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> sections
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> are still TBD.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> It would probably be good to flesh those out a bit before
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> proceeding
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> too
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> far
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> as the event time alignment will probably influence the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> interaction
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> with
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the split reader, specifically ReaderStatus
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> emitNext(SourceOutput<E>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> output).
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> We currently have only one implementation for event time
> > > >> >> alignment
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> in
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Kinesis consumer. The synchronization in that case takes
> > > place
> > > >> >> as
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> last
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> step before records are emitted downstream
> (RecordEmitter).
> > > >> With
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> currently proposed interfaces, the equivalent can be
> > > >> implemented
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> in
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> reader loop, although note that in the Kinesis consumer
> the
> > > per
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> shard
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> threads push records.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Synchronization has not been implemented for the Kafka
> > > consumer
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> yet.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
> > > >> >>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> When I looked at it, I realized that the implementation
> > will
> > > >> >> look
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> quite
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> different
> > > >> >>>>>>>>>> from Kinesis because it needs to take place in the pull
> > part,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> where
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> records
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> are taken from the Kafka client. Due to the multiplexing
> it
> > > >> >> cannot
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> be
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> done
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> by blocking the split thread like it currently works for
> > > >> >> Kinesis.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Reading
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> from individual Kafka partitions needs to be controlled
> via
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> pause/resume
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> on the Kafka client.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> To take on that responsibility the split thread would
> need
> > to
> > > >> be
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> aware
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>> watermarks or at least whether it should or should not
> > > continue
> > > >> >> to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> consume
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> a given split and this may require a different
> SourceReader
> > > or
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> SourceOutput
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> interface.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks,
> > > >> >>>>>>>>>> Thomas
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <
> > mmyy1110@gmail.com
> > > >> >>>>>> <mailto:
> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > > >> >> mmyy1110@gmail.com
> > > >> >>>>>
> > > >> >>>>>> <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi Stephan,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thank you for feedback!
> > > >> >>>>>>>>>> Will take a look at your branch before public discussing.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <
> > > >> sewen@apache.org
> > > >> >>>>>>>> <ma...@apache.org>> <sewen@apache.org <mailto:
> > > >> >>> sewen@apache.org
> > > >> >>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi Biao!
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks for reviving this. I would like to join this
> > > discussion,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> but
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> am
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> quite occupied with the 1.9 release, so can we maybe
> pause
> > > this
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> discussion
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> for a week or so?
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> In the meantime I can share some suggestion based on
> prior
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> experiments:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> How to do watermarks / timestamp extractors in a simpler
> > and
> > > >> >> more
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> flexible
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> way. I think that part is quite promising should be part
> of
> > > the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> new
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> source
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> interface.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > > >> >>>>>>>> <
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > > >> >>>>>>>> <
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Some experiments on how to build the source reader and
> its
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> library
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> for
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> common threading/split patterns:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > > >> >>>>>>>> <
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Best,
> > > >> >>>>>>>>>> Stephan
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <
> > > mmyy1110@gmail.com
> > > >> >>>>>>> <mailto:
> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > > >> >> mmyy1110@gmail.com
> > > >> >>>>>
> > > >> >>>>>> <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi devs,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Since 1.9 is nearly released, I think we could get back
> to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> FLIP-27.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> believe it should be included in 1.10.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> There are so many things mentioned in document of
> FLIP-27.
> > > [1]
> > > >> I
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> think
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> we'd better discuss them separately. However the wiki is
> > not
> > > a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> good
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> place
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> to discuss. I wrote google doc about SplitReader API
> which
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> misses
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> some
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> details in the document. [2]
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 1.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > >> >>>>>>>> <
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 2.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > > >> >>>>>>>> <
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <
> > mmyy1110@gmail.com
> > > >> >>>>>> <mailto:
> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > > >> >> mmyy1110@gmail.com
> > > >> >>>>>
> > > >> >>>>>> <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi Steven,
> > > >> >>>>>>>>>> Thank you for the feedback. Please take a look at the
> > > document
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> FLIP-27
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > >> >>>>>>>> <
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> which
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> is updated recently. A lot of details of enumerator were
> > > added
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> in
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> this
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> document. I think it would help.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Steven Wu <stevenz3wu@gmail.com <mailto:
> > stevenz3wu@gmail.com
> > > >>
> > > >> >> <
> > > >> >>>>>>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
> > > >> >>>>>>> stevenz3wu@gmail.com
> > > >> >>>>>>>> <ma...@gmail.com>> <stevenz3wu@gmail.com
> > <mailto:
> > > >> >>>>>>>> stevenz3wu@gmail.com>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 于2019年3月28日周四
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 下午12:52写道:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> This proposal mentioned that SplitEnumerator might run on
> > the
> > > >> >>>>>>>>>> JobManager or
> > > >> >>>>>>>>>> in a single task on a TaskManager.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> if enumerator is a single task on a taskmanager, then the
> > job
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> DAG
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> can
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> never
> > > >> >>>>>>>>>> been embarrassingly parallel anymore. That will nullify
> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> leverage
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> fine-grained recovery for embarrassingly parallel jobs.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> It's not clear to me what's the implication of running
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> enumerator
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> on
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> jobmanager. So I will leave that out for now.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <
> > mmyy1110@gmail.com
> > > >> >>>>>> <mailto:
> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > > >> >> mmyy1110@gmail.com
> > > >> >>>>>
> > > >> >>>>>> <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi Stephan & Piotrek,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thank you for feedback.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> It seems that there are a lot of things to do in
> community.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> am
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> just
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> afraid that this discussion may be forgotten since there
> so
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> many
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> proposals
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> recently.
> > > >> >>>>>>>>>> Anyway, wish to see the split topics soon :)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:
> > > >> >>> piotr@da-platform.com
> > > >> >>>>>>>>
> > > >> >>>>>>> <
> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
> > > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 于2019年1月24日周四
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 下午8:21写道:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi Biao!
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> This discussion was stalled because of preparations for
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> open
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> sourcing
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> & merging Blink. I think before creating the tickets we
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> should
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> split this
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> discussion into topics/areas outlined by Stephan and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> create
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Flips
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> for
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I think there is no chance for this to be completed in
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> couple
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> remaining
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> weeks/1 month before 1.8 feature freeze, however it would
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> be
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> good
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> to aim
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> with those changes for 1.9.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Piotrek
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com
> > > >> <mailto:
> > > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > > >> >> mmyy1110@gmail.com
> > > >> >>>>>
> > > >> >>>>>> <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi community,
> > > >> >>>>>>>>>> The summary of Stephan makes a lot sense to me. It is
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> much
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> clearer
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> indeed
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> after splitting the complex topic into small ones.
> > > >> >>>>>>>>>> I was wondering is there any detail plan for next step?
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> If
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> not,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> would
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> like to push this thing forward by creating some JIRA
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> issues.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Another question is that should version 1.8 include
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> these
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> features?
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Stephan Ewen <sewen@apache.org <mailto:sewen@apache.org
> >>
> > <
> > > >> >>>>>>>> sewen@apache.org <ma...@apache.org>> <
> > sewen@apache.org
> > > >> >>>> <mailto:
> > > >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:
> > sewen@apache.org
> > > >>
> > > >> >>>>>>>> 于2018年12月1日周六
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 上午4:20写道:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks everyone for the lively discussion. Let me try
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> summarize
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> where I
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> see convergence in the discussion and open issues.
> > > >> >>>>>>>>>> I'll try to group this by design aspect of the source.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Please
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> let me
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> know
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> if I got things wrong or missed something crucial here.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> For issues 1-3, if the below reflects the state of the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> discussion, I
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> would
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> try and update the FLIP in the next days.
> > > >> >>>>>>>>>> For the remaining ones we need more discussion.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> I would suggest to fork each of these aspects into a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> separate
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> mail
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> thread,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> or will loose sight of the individual aspects.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> *(1) Separation of Split Enumerator and Split Reader*
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - All seem to agree this is a good thing
> > > >> >>>>>>>>>> - Split Enumerator could in the end live on JobManager
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> (and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> assign
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> splits
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> via RPC) or in a task (and assign splits via data
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> streams)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - this discussion is orthogonal and should come later,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> when
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> interface
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> is agreed upon.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> *(2) Split Readers for one or more splits*
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Discussion seems to agree that we need to support
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> one
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> reader
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> possibly handles multiple splits concurrently.
> > > >> >>>>>>>>>> - The requirement comes from sources where one
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> poll()-style
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> call
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> fetches
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> data from different splits / partitions
> > > >> >>>>>>>>>>     --> example sources that require that would be for
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> example
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Kafka,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Pravega, Pulsar
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Could have one split reader per source, or multiple
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> split
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> readers
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> share the "poll()" function
> > > >> >>>>>>>>>> - To not make it too complicated, we can start with
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> thinking
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> about
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> one
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> split reader for all splits initially and see if that
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> covers
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> all
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> requirements
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> *(3) Threading model of the Split Reader*
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Most active part of the discussion ;-)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - A non-blocking way for Flink's task code to interact
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> with
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> source
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> is
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> needed in order to a task runtime code based on a
> > > >> >>>>>>>>>> single-threaded/actor-style task design
> > > >> >>>>>>>>>>     --> I personally am a big proponent of that, it will
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> help
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> with
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> well-behaved checkpoints, efficiency, and simpler yet
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> more
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> robust
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> runtime
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> code
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Users care about simple abstraction, so as a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> subclass
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> SplitReader
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> (non-blocking / async) we need to have a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> BlockingSplitReader
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> which
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> will
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> form the basis of most source implementations.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> BlockingSplitReader
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> lets
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> users do blocking simple poll() calls.
> > > >> >>>>>>>>>> - The BlockingSplitReader would spawn a thread (or
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> more)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> thread(s) can make blocking calls and hand over data
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> buffers
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> via
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> blocking
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> queue
> > > >> >>>>>>>>>> - This should allow us to cover both, a fully async
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> runtime,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> simple
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> blocking interface for users.
> > > >> >>>>>>>>>> - This is actually very similar to how the Kafka
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> connectors
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> work.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Kafka
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 9+ with one thread, Kafka 8 with multiple threads
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - On the base SplitReader (the async one), the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> non-blocking
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> method
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> gets the next chunk of data would signal data
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> availability
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> via
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> CompletableFuture, because that gives the best
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> flexibility
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> (can
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> await
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> completion or register notification handlers).
> > > >> >>>>>>>>>> - The source task would register a "thenHandle()" (or
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> similar)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> on the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> future to put a "take next data" task into the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> actor-style
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> mailbox
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> *(4) Split Enumeration and Assignment*
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Splits may be generated lazily, both in cases where
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> there
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> is a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> limited
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> number of splits (but very many), or splits are
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> discovered
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> over
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> time
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Assignment should also be lazy, to get better load
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> balancing
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Assignment needs support locality preferences
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Possible design based on discussion so far:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>     --> SplitReader has a method "addSplits(SplitT...)"
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> add
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> one or
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> more
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> splits. Some split readers might assume they have only
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> one
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> split
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> ever,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> concurrently, others assume multiple splits. (Note:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> idea
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> behind
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> being
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> able
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> to add multiple splits at the same time is to ease
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> startup
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> where
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> multiple
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> splits may be assigned instantly.)
> > > >> >>>>>>>>>>     --> SplitReader has a context object on which it can
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> call
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> indicate
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> when
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> splits are completed. The enumerator gets that
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> notification and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> can
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> use
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> decide when to assign new splits. This should help both
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> in
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> cases
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> sources
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that take splits lazily (file readers) and in case the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> source
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> needs to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> preserve a partial order between splits (Kinesis,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Pravega,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Pulsar may
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> need
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that).
> > > >> >>>>>>>>>>     --> SplitEnumerator gets notification when
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> SplitReaders
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> start
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> when
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> they finish splits. They can decide at that moment to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> push
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> more
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> splits
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that reader
> > > >> >>>>>>>>>>     --> The SplitEnumerator should probably be aware of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> source
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> parallelism, to build its initial distribution.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Open question: Should the source expose something
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> like
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> "host
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> preferences", so that yarn/mesos/k8s can take this into
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> account
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> when
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> selecting a node to start a TM on?
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> *(5) Watermarks and event time alignment*
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Watermark generation, as well as idleness, needs to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> be
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> per
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> split
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> (like
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> currently in the Kafka Source, per partition)
> > > >> >>>>>>>>>> - It is desirable to support optional
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> event-time-alignment,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> meaning
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> splits that are ahead are back-pressured or temporarily
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> unsubscribed
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - I think i would be desirable to encapsulate
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> watermark
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> generation
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> logic
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> in watermark generators, for a separation of concerns.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> The
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> watermark
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> generators should run per split.
> > > >> >>>>>>>>>> - Using watermark generators would also help with
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> another
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> problem of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> suggested interface, namely supporting non-periodic
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> watermarks
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> efficiently.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Need a way to "dispatch" next record to different
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> watermark
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> generators
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Need a way to tell SplitReader to "suspend" a split
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> until a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> certain
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> watermark is reached (event time backpressure)
> > > >> >>>>>>>>>> - This would in fact be not needed (and thus simpler)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> if
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> we
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> had
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> SplitReader per split and may be a reason to re-open
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> discussion
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> *(6) Watermarks across splits and in the Split
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Enumerator*
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - The split enumerator may need some watermark
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> awareness,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> which
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> should
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> be
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> purely based on split metadata (like create timestamp
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> file
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> splits)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - If there are still more splits with overlapping
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> event
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> time
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> range
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> for
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> split reader, then that split reader should not advance
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> watermark
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> within the split beyond the overlap boundary. Otherwise
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> future
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> splits
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> will
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> produce late data.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - One way to approach this could be that the split
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> enumerator
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> may
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> send
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> watermarks to the readers, and the readers cannot emit
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> watermarks
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> beyond
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that received watermark.
> > > >> >>>>>>>>>> - Many split enumerators would simply immediately send
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Long.MAX
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> out
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> leave the progress purely to the split readers.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - For event-time alignment / split back pressure, this
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> begs
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> question
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> how we can avoid deadlocks that may arise when splits
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> are
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> suspended
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> for
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> event time back pressure,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> *(7) Batch and streaming Unification*
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Functionality wise, the above design should support
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> both
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Batch often (mostly) does not care about reading "in
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> order"
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> generating watermarks
> > > >> >>>>>>>>>>     --> Might use different enumerator logic that is
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> more
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> locality
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> aware
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and ignores event time order
> > > >> >>>>>>>>>>     --> Does not generate watermarks
> > > >> >>>>>>>>>> - Would be great if bounded sources could be
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> identified
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> at
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> compile
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> time,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> so that "env.addBoundedSource(...)" is type safe and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> can
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> return a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> "BoundedDataStream".
> > > >> >>>>>>>>>> - Possible to defer this discussion until later
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> *Miscellaneous Comments*
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - Should the source have a TypeInformation for the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> produced
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> type,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> instead
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> of a serializer? We need a type information in the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> stream
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> anyways, and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> can
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> derive the serializer from that. Plus, creating the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> serializer
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> should
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> respect the ExecutionConfig.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> - The TypeSerializer interface is very powerful but
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> also
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> not
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> easy to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> implement. Its purpose is to handle data super
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> efficiently,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> support
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> flexible ways of evolution, etc.
> > > >> >>>>>>>>>> For metadata I would suggest to look at the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> SimpleVersionedSerializer
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> instead, which is used for example for checkpoint
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> master
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> hooks,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> or for
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> streaming file sink. I think that is is a good match
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> for
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> cases
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> where
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> we
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> do
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> not need more than ser/deser (no copy, etc.) and don't
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> need to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> push
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> versioning out of the serialization paths for best
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> performance
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> (as in
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> TypeSerializer)
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> k.kloudas@data-artisans.com>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Hi Biao,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Thanks for the answer!
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> So given the multi-threaded readers, now we have as
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> open
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> questions:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 1) How do we let the checkpoints pass through our
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> multi-threaded
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> reader
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> operator?
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 2) Do we have separate reader and source operators or
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> not? In
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> strategy
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that has a separate source, the source operator has a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> parallelism of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 1
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> is responsible for split recovery only.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> For the first one, given also the constraints
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> (blocking,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> finite
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> queues,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> etc), I do not have an answer yet.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> For the 2nd, I think that we should go with separate
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> operators
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> for
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> source and the readers, for the following reasons:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 1) This is more aligned with a potential future
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> improvement
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> where the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> split
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> discovery becomes a responsibility of the JobManager
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> readers are
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> pooling more work from the JM.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 2) The source is going to be the "single point of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> truth".
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> It
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> will
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> know
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> what
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> has been processed and what not. If the source and the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> readers
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> are a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> single
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> operator with parallelism > 1, or in general, if the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> split
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> discovery
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> is
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> done by each task individually, then:
> > > >> >>>>>>>>>>    i) we have to have a deterministic scheme for each
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> reader to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> assign
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> splits to itself (e.g. mod subtaskId). This is not
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> necessarily
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> trivial
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> for
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> all sources.
> > > >> >>>>>>>>>>    ii) each reader would have to keep a copy of all its
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> processed
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> slpits
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>    iii) the state has to be a union state with a
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> non-trivial
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> merging
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> logic
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> in order to support rescaling.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Two additional points that you raised above:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> i) The point that you raised that we need to keep all
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> splits
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> (processed
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> not-processed) I think is a bit of a strong
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> requirement.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> This
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> would
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> imply
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that for infinite sources the state will grow
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> indefinitely.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> This is
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> problem
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> is even more pronounced if we do not have a single
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> source
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> assigns
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> splits to readers, as each reader will have its own
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> copy
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> of
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> state.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> ii) it is true that for finite sources we need to
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> somehow
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> not
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> close
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> readers when the source/split discoverer finishes. The
> > > >> >>>>>>>>>> ContinuousFileReaderOperator has a work-around for
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> that.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> It is
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> not
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> elegant,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> and checkpoints are not emitted after closing the
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> source,
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> but
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> this, I
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> believe, is a bigger problem which requires more
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> changes
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> than
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> just
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> refactoring the source interface.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Cheers,
> > > >> >>>>>>>>>> Kostas
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> --
> > > >> >>>>>>>>>> Best, Jingsong Lee
> > > >> >>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>
> > > >> >>>>>>
> > > >> >>>>>> --
> > > >> >>>>>> Best, Jingsong Lee
> > > >> >>>>>>
> > > >> >>>>>
> > > >> >>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >> >
> > > >>
> > > >>
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Jark Wu <im...@gmail.com>.
Hi,

First of all, I think it is not called "UNBOUNDED", according to the
FLIP-27, it is called "CONTINUOUS_UNBOUNDED".
And from the description of the Boundedness in the FLIP-27[1] declares
clearly what Becket and I think.

public enum Boundedness {

    /**
     * A bounded source processes the data that is currently available and
will end after that.
     *
     * <p>When a source produces a bounded stream, the runtime may activate
additional optimizations
     * that are suitable only for bounded input. Incorrectly producing
unbounded data when the source
     * is set to produce a bounded stream will often result in programs
that do not output any results
     * and may eventually fail due to runtime errors (out of memory or
storage).
     */
    BOUNDED,

    /**
     * A continuous unbounded source continuously processes all data as it
comes.
     *
     * <p>The source may run forever (until the program is terminated) or
might actually end at some point,
     * based on some source-specific conditions. Because that is not
transparent to the runtime,
     * the runtime will use an execution mode for continuous unbounded
streams whenever this mode
     * is chosen.
     */
    CONTINUOUS_UNBOUNDED
}

Best,
Jark

[1]:
https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface#FLIP-27:RefactorSourceInterface-Source



On Fri, 20 Dec 2019 at 12:55, Steven Wu <st...@gmail.com> wrote:

> Becket,
>
> Regarding "UNBOUNDED source that stops at some point", I found it difficult
> to grasp what UNBOUNDED really mean.
>
> If we want to use Kafka source with an end/stop time, I guess you call it
> UNBOUNDED kafka source that stops (aka BOUNDED-streaming). The
> terminology is a little confusing to me. Maybe BOUNDED/UNBOUNDED shouldn't
> be used to categorize source. Just call it Kafka source and it can run in
> either BOUNDED or UNBOUNDED mode.
>
> Thanks,
> Steven
>
> On Thu, Dec 19, 2019 at 7:02 PM Becket Qin <be...@gmail.com> wrote:
>
> > I had an offline chat with Jark, and here are some more thoughts:
> >
> > 1. From SQL perspective, BOUNDED source leads to the batch execution
> mode,
> > UNBOUNDED source leads to the streaming execution mode.
> > 2. The semantic of UNBOUNDED source is may or may not stop. The semantic
> of
> > BOUNDED source is will stop.
> > 3. The semantic of DataStream is may or may not terminate. The semantic
> of
> > BoundedDataStream is will terminate.
> >
> > Given that, option 3 seems a better option because:
> > 1. SQL already has strict binding between Boundedness and execution mode.
> > Letting DataStream be consistent would be good.
> > 2. The semantic of UNBOUNDED source is exactly the same as DataStream. So
> > we should avoid breaking such semantic, i.e. turning some DataStream from
> > "may or may not terminate" to "will terminate".
> >
> > For case where users want BOUNDED-streaming combination, they can simply
> > use an UNBOUNDED source that stops at some point. We can even provide a
> > simple wrapper to wrap a BOUNDED source as an UNBOUNDED source if that
> > helps. But API wise, option 3 seems telling a pretty good whole story.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> >
> >
> >
> > On Thu, Dec 19, 2019 at 10:30 PM Becket Qin <be...@gmail.com>
> wrote:
> >
> > > Hi Timo,
> > >
> > > Bounded is just a special case of unbounded and every bounded source
> can
> > >> also be treated as an unbounded source. This would unify the API if
> > >> people don't need a bounded operation.
> > >
> > >
> > > With option 3 users can still get a unified API with something like
> > below:
> > >
> > > DataStream boundedStream = env.boundedSource(boundedSource);
> > > DataStream unboundedStream = env.source(unboundedSource);
> > >
> > > So in both cases, users can still use a unified DataStream without
> > > touching the bounded stream only methods.
> > > By "unify the API if people don't need the bounded operation". Do you
> > > expect a DataStream with a Bounded source to have the batch operators
> and
> > > scheduler settings as well?
> > >
> > >
> > > If we allow DataStream from BOUNDED source, we will essentially pick
> > "*modified
> > > option 2*".
> > >
> > > // The source is either bounded or unbounded, but only unbounded
> > >> operations could be performed on the returned DataStream.
> > >> DataStream<Type> dataStream = env.source(someSource);
> > >
> > >
> > >> // The source must be a bounded source, otherwise exception is thrown.
> > >> BoundedDataStream<Type> boundedDataStream =
> > >> env.boundedSource(boundedSource);
> > >
> > >
> > >
> > > // Add the following method to DataStream
> > >
> > > Boundedness DataStream#getBoundedness();
> > >
> > >
> > > From pure logical perspective, Boundedness and runtime settings
> > > (Stream/Batch) are two orthogonal dimensions. And are specified in the
> > > following way.
> > >
> > > *Boundedness* - defined by the source: BOUNDED / UNBOUNDED.
> > > *Running mode* - defined by the API class: DataStream (Streaming mode)
> /
> > > BoundedDataStream (batch mode).
> > >
> > > Excluding the UNBOUNDED-batch combination, the "*modified option 2"*
> > > covers the rest three combination. Compared with "*modified option 2*",
> > > the main benefit of option 3 is its simplicity and clearness, by tying
> > > boundedness to running mode and giving up BOUNDED-streaming
> combination.
> > >
> > > Just to be clear, I am fine with either option. But I would like to
> > > understand a bit more about the bounded-streaming use case and when
> users
> > > would prefer this over bounded-batch case, and whether the added value
> > > justifies the additional complexity in the API. Two cases I can think
> of
> > > are:
> > > 1. The records in DataStream will be processed in order, while
> > > BoundedDataStream processes records without order guarantee.
> > > 2. DataStream emits intermediate results when processing a finite
> > dataset,
> > > while BoundedDataStream only emit the final result. In any case, it
> could
> > > be supported by an UNBOUNDED source stopping at some point.
> > >
> > > Case 1 is actually misleading because DataStream in general doesn't
> > really
> > > support in-order process.
> > > Case 2 seems a rare use case because the instantaneous intermediate
> > result
> > > seems difficult to reason about. In any case, this can be supported by
> an
> > > UNBOUNDED source that stops at some point.
> > >
> > > Is there other use cases for bounded-streaming combination I missed? I
> am
> > > a little hesitating to put the testing requirement here because ideally
> > I'd
> > > avoid having public APIs for testing purpose only. And this could be
> > > resolved by having a UNBOUNDED source stopping at some point as well.
> > >
> > > Sorry for the long discussion, but I would really like to make an API
> > > decision after knowing all the pros and cons.
> > >
> > > Thanks,
> > >
> > > Jiangjie (Becket) Qin
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > > On Thu, Dec 19, 2019 at 6:19 PM Timo Walther <tw...@apache.org>
> wrote:
> > >
> > >> Hi Becket,
> > >>
> > >> regarding *Option 3* I think we can relax the constraints for
> > >> env.source():
> > >>
> > >> // MySource can be bounded or unbounded
> > >> DataStream<Type> dataStream = env.source(mySource);
> > >>
> > >> // MySource must be bounded, otherwise throws exception.
> > >> BoundedDataStream<Type> boundedDataStream =
> env.boundedSource(mySource);
> > >>
> > >> Bounded is just a special case of unbounded and every bounded source
> can
> > >> also be treated as an unbounded source. This would unify the API if
> > >> people don't need a bounded operation. It also addresses Jark's
> > concerns.
> > >>
> > >> Regards,
> > >> Timo
> > >>
> > >>
> > >> On 18.12.19 14:16, Becket Qin wrote:
> > >> > Hi Jark,
> > >> >
> > >> > Please see the reply below:
> > >> >
> > >> > Regarding to option#3, my concern is that if we don't support
> > streaming
> > >> >> mode for bounded source,
> > >> >> how could we create a testing source for streaming mode? Currently,
> > >> all the
> > >> >> testing source for streaming
> > >> >> are bounded, so that the integration test will finish finally.
> > >> >
> > >> >
> > >> > An UNBOUNDED source does not mean it will never stops. It simply
> > >> indicates
> > >> > that the source *may* run forever, so the runtime needs to be
> prepared
> > >> for
> > >> > that, but the task may still stop at some point when it hits some
> > >> > source-specific condition. So an UNBOUNDED testing source can still
> > >> stop at
> > >> > some point if needed.
> > >> >
> > >> > Regarding to Source#getRecordOrder(), could we have a implicit
> > contract
> > >> >> that unbounded source should
> > >> >> already read in order (i.e. reading partitions in parallel), for
> > >> bounded
> > >> >> source the order is not mandatory.
> > >> >
> > >> >
> > >> >
> > >> >> This is also the behaviors of the current sources.
> > >> >
> > >> > 1) a source can't guarantee it reads in strict order, because the
> > >> producer
> > >> >> may produce data not in order.
> > >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can
> reorder
> > >> data.
> > >> >
> > >> >
> > >> > It is true that sometimes the source cannot guarantee the record
> > order,
> > >> but
> > >> > sometimes it can. Right now, even for stream processing, there is no
> > >> > processing order guarantee. For example, a join operator may emit a
> > >> later
> > >> > record which successfully found a join match earlier.
> > >> > Event order is one of the most important requirements for event
> > >> processing,
> > >> > a clear order guarantee would be necessary. That said, I agree that
> > >> right
> > >> > now even if the sources provide the record order requirement, the
> > >> runtime
> > >> > is not able to guarantee that out of the box. So I am OK if we add
> the
> > >> > record order to the Source later. But we should avoid misleading
> users
> > >> to
> > >> > make them think the processing order is guaranteed when using the
> > >> unbounded
> > >> > runtime.
> > >> >
> > >> > Thanks,
> > >> >
> > >> > Jiangjie (Becket) Qin
> > >> >
> > >> > On Wed, Dec 18, 2019 at 10:29 AM Jark Wu <im...@gmail.com> wrote:
> > >> >
> > >> >> Hi Becket,
> > >> >>
> > >> >> That's great we have reached a consensus on
> Source#getBoundedness().
> > >> >>
> > >> >> Regarding to option#3, my concern is that if we don't support
> > streaming
> > >> >> mode for bounded source,
> > >> >> how could we create a testing source for streaming mode? Currently,
> > >> all the
> > >> >> testing source for streaming
> > >> >> are bounded, so that the integration test will finish finally.
> > >> >>
> > >> >> Regarding to Source#getRecordOrder(), could we have a implicit
> > contract
> > >> >> that unbounded source should
> > >> >> already read in order (i.e. reading partitions in parallel), for
> > >> bounded
> > >> >> source the order is not mandatory.
> > >> >> This is also the behaviors of the current sources.
> > >> >> 1) a source can't guarantee it reads in strict order, because the
> > >> producer
> > >> >> may produce data not in order.
> > >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can
> reorder
> > >> data.
> > >> >>
> > >> >> Best,
> > >> >> Jark
> > >> >>
> > >> >>
> > >> >>
> > >> >> On Tue, 17 Dec 2019 at 22:03, Becket Qin <be...@gmail.com>
> > wrote:
> > >> >>
> > >> >>> Hi folks,
> > >> >>>
> > >> >>> Thanks for the comments. I am convinced that the Source API should
> > not
> > >> >> take
> > >> >>> boundedness as a parameter after it is constructed. What Timo and
> > >> Dawid
> > >> >>> suggested sounds a reasonable solution to me. So the Source API
> > would
> > >> >>> become:
> > >> >>>
> > >> >>> Source {
> > >> >>>      Boundedness getBoundedness();
> > >> >>> }
> > >> >>>
> > >> >>> Assuming the above Source API, in addition to the two options
> > >> mentioned
> > >> >> in
> > >> >>> earlier emails, I am thinking of another option:
> > >> >>>
> > >> >>> *Option 3:*
> > >> >>> // MySource must be unbounded, otherwise throws exception.
> > >> >>> DataStream<Type> dataStream = env.source(mySource);
> > >> >>>
> > >> >>> // MySource must be bounded, otherwise throws exception.
> > >> >>> BoundedDataStream<Type> boundedDataStream =
> > >> env.boundedSource(mySource);
> > >> >>>
> > >> >>> The pros of this API are:
> > >> >>>     a) It fits the requirements from Table / SQL well.
> > >> >>>     b) DataStream users still have type safety (option 2 only has
> > >> partial
> > >> >>> type safety).
> > >> >>>     c) Cristal clear boundedness from the API which makes
> DataStream
> > >> join
> > >> >> /
> > >> >>> connect easy to reason about.
> > >> >>> The caveats I see,
> > >> >>>     a) It is inconsistent with Table since Table has one unified
> > >> >> interface.
> > >> >>>     b) No streaming mode for bounded source.
> > >> >>>
> > >> >>> @Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
> > >> >>> <al...@ververica.com> what do you think of the approach?
> > >> >>>
> > >> >>>
> > >> >>> Orthogonal to the above API, I am wondering whether boundedness is
> > the
> > >> >> only
> > >> >>> dimension needed to describe the characteristic of the Source
> > >> behavior.
> > >> >> We
> > >> >>> may also need to have another dimension of *record order*.
> > >> >>>
> > >> >>> For example, when a file source is reading from a directory with
> > >> bounded
> > >> >>> records, it may have two ways to read.
> > >> >>> 1. Read files in parallel.
> > >> >>> 2. Read files in the chronological order.
> > >> >>> In both cases, the file source is a Bounded Source. However, the
> > >> >> processing
> > >> >>> requirement for downstream may be different. In the first case,
> the
> > >> >>> record processing and result emitting order does not matter, e.g.
> > word
> > >> >>> count. In the second case, the records may have to be processed in
> > the
> > >> >>> order they were read, e.g. change log processing.
> > >> >>>
> > >> >>> If the Source only has a getBoundedness() method, the downstream
> > >> >> processors
> > >> >>> would not know whether the records emitted from the Source should
> be
> > >> >>> processed in order or not. So combining the boundedness and record
> > >> order,
> > >> >>> we will have four scenarios:
> > >> >>>
> > >> >>> *Bounded-StrictOrder*:     A segment of change log.
> > >> >>> *Bounded-Random*:          Batch Word Count.
> > >> >>> *Unbounded-StrictOrder*: An infinite change log.
> > >> >>> *Unbounded-Random*:     Streaming Word Count.
> > >> >>>
> > >> >>> Option 2 mentioned in the previous email was kind of trying to
> > handle
> > >> the
> > >> >>> Bounded-StrictOrder case by creating a DataStream from a bounded
> > >> source,
> > >> >>> which actually does not work.
> > >> >>> It looks that we do not have strict order support in some
> operators
> > at
> > >> >> this
> > >> >>> point, e.g. join. But we may still want to add the semantic to the
> > >> Source
> > >> >>> first so later on we don't need to change all the source
> > >> implementations,
> > >> >>> especially given that many of them will be implemented by 3rd
> party.
> > >> >>>
> > >> >>> Given that, we need another dimension of *Record Order* in the
> > Source.
> > >> >> More
> > >> >>> specifically, the API would become:
> > >> >>>
> > >> >>> Source {
> > >> >>>      Boundedness getBoundedness();
> > >> >>>      RecordOrder getRecordOrder();
> > >> >>> }
> > >> >>>
> > >> >>> public enum RecordOrder {
> > >> >>>      /** The record in the DataStream must be processed in its
> > strict
> > >> >> order
> > >> >>> for correctness. */
> > >> >>>      STRICT,
> > >> >>>      /** The record in the DataStream can be processed in
> arbitrary
> > >> order.
> > >> >>> */
> > >> >>>      RANDOM;
> > >> >>> }
> > >> >>>
> > >> >>> Any thoughts?
> > >> >>>
> > >> >>> Thanks,
> > >> >>>
> > >> >>> Jiangjie (Becket) Qin
> > >> >>>
> > >> >>> On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <tw...@apache.org>
> > >> wrote:
> > >> >>>
> > >> >>>> Hi Becket,
> > >> >>>>
> > >> >>>> I completely agree with Dawid's suggestion. The information about
> > the
> > >> >>>> boundedness should come out of the source. Because most of the
> > >> >> streaming
> > >> >>>> sources can be made bounded based on some connector specific
> > >> criterion.
> > >> >>>> In Kafka, it would be an end offset or end timestamp but in any
> > case
> > >> >>>> having just a env.boundedSource() is not enough because
> parameters
> > >> for
> > >> >>>> making the source bounded are missing.
> > >> >>>>
> > >> >>>> I suggest to have a simple `isBounded(): Boolean` flag in every
> > >> source
> > >> >>>> that might be influenced by a connector builder as Dawid
> mentioned.
> > >> >>>>
> > >> >>>> For type safety during programming, we can still go with *Final
> > state
> > >> >>>> 1*. By having a env.source() vs env.boundedSource(). The latter
> > would
> > >> >>>> just enforce that the boolean flag is set to `true` and could
> make
> > >> >>>> bounded operations available (if we need that actually).
> > >> >>>>
> > >> >>>> However, I don't think that we should start making a unified
> Table
> > >> API
> > >> >>>> ununified again. Boundedness is an optimization property. Every
> > >> bounded
> > >> >>>> operation can also executed in an unbounded way using
> > >> >> updates/retraction
> > >> >>>> or watermarks.
> > >> >>>>
> > >> >>>> Regards,
> > >> >>>> Timo
> > >> >>>>
> > >> >>>>
> > >> >>>> On 15.12.19 14:22, Becket Qin wrote:
> > >> >>>>> Hi Dawid and Jark,
> > >> >>>>>
> > >> >>>>> I think the discussion ultimately boils down to the question
> that
> > >> >> which
> > >> >>>> one
> > >> >>>>> of the following two final states do we want? Once we make this
> > >> >>> decision,
> > >> >>>>> everything else can be naturally derived.
> > >> >>>>>
> > >> >>>>> *Final state 1*: Separate API for bounded / unbounded
> DataStream &
> > >> >>> Table.
> > >> >>>>> That means any code users write will be valid at the point when
> > they
> > >> >>>> write
> > >> >>>>> the code. This is similar to having type safety check at
> > programming
> > >> >>>> time.
> > >> >>>>> For example,
> > >> >>>>>
> > >> >>>>> BoundedDataStream extends DataStream {
> > >> >>>>> // Operations only available for bounded data.
> > >> >>>>> BoundedDataStream sort(...);
> > >> >>>>>
> > >> >>>>> // Interaction with another BoundedStream returns a Bounded
> > stream.
> > >> >>>>> BoundedJoinedDataStream join(BoundedDataStream other)
> > >> >>>>>
> > >> >>>>> // Interaction with another unbounded stream returns an
> unbounded
> > >> >>> stream.
> > >> >>>>> JoinedDataStream join(DataStream other)
> > >> >>>>> }
> > >> >>>>>
> > >> >>>>> BoundedTable extends Table {
> > >> >>>>>     // Bounded only operation.
> > >> >>>>> BoundedTable sort(...);
> > >> >>>>>
> > >> >>>>> // Interaction with another BoundedTable returns a BoundedTable.
> > >> >>>>> BoundedTable join(BoundedTable other)
> > >> >>>>>
> > >> >>>>> // Interaction with another unbounded table returns an unbounded
> > >> >> table.
> > >> >>>>> Table join(Table other)
> > >> >>>>> }
> > >> >>>>>
> > >> >>>>> *Final state 2*: One unified API for bounded / unbounded
> > DataStream
> > >> /
> > >> >>>>> Table.
> > >> >>>>> That unified API may throw exception at DAG compilation time if
> an
> > >> >>>> invalid
> > >> >>>>> operation is tried. This is what Table API currently follows.
> > >> >>>>>
> > >> >>>>> DataStream {
> > >> >>>>> // Throws exception if the DataStream is unbounded.
> > >> >>>>> DataStream sort();
> > >> >>>>> // Get boundedness.
> > >> >>>>> Boundedness getBoundedness();
> > >> >>>>> }
> > >> >>>>>
> > >> >>>>> Table {
> > >> >>>>> // Throws exception if the table has infinite rows.
> > >> >>>>> Table orderBy();
> > >> >>>>>
> > >> >>>>> // Get boundedness.
> > >> >>>>> Boundedness getBoundedness();
> > >> >>>>> }
> > >> >>>>>
> > >> >>>>> >From what I understand, there is no consensus so far on this
> > >> decision
> > >> >>>> yet.
> > >> >>>>> Whichever final state we choose, we need to make it consistent
> > >> across
> > >> >>> the
> > >> >>>>> entire project. We should avoid the case that Table follows one
> > >> final
> > >> >>>> state
> > >> >>>>> while DataStream follows another. Some arguments I am aware of
> > from
> > >> >>> both
> > >> >>>>> sides so far are following:
> > >> >>>>>
> > >> >>>>> Arguments for final state 1:
> > >> >>>>> 1a) Clean API with method safety check at programming time.
> > >> >>>>> 1b) (Counter 2b) Although SQL does not have programming time
> error
> > >> >>>> check, SQL
> > >> >>>>> is not really a "programming language" per se. So SQL can be
> > >> >> different
> > >> >>>> from
> > >> >>>>> Table and DataStream.
> > >> >>>>> 1c)  Although final state 2 seems making it easier for SQL to
> use
> > >> >> given
> > >> >>>> it
> > >> >>>>> is more "config based" than "parameter based", final state 1 can
> > >> >>> probably
> > >> >>>>> also meet what SQL wants by wrapping the Source in TableSource /
> > >> >>>>> TableSourceFactory API if needed.
> > >> >>>>>
> > >> >>>>> Arguments for final state 2:
> > >> >>>>> 2a) The Source API itself seems already sort of following the
> > >> unified
> > >> >>> API
> > >> >>>>> pattern.
> > >> >>>>> 2b) There is no "programming time" method error check in SQL
> case,
> > >> so
> > >> >>> we
> > >> >>>>> cannot really achieve final state 1 across the board.
> > >> >>>>> 2c) It is an easier path given our current status, i.e. Table is
> > >> >>> already
> > >> >>>>> following final state 2.
> > >> >>>>> 2d) Users can always explicitly check the boundedness if they
> want
> > >> >> to.
> > >> >>>>>
> > >> >>>>> As I mentioned earlier, my initial thought was also to have a
> > >> >>>>> "configuration based" Source rather than a "parameter based"
> > Source.
> > >> >> So
> > >> >>>> it
> > >> >>>>> is completely possible that I missed some important
> consideration
> > or
> > >> >>>> design
> > >> >>>>> principles that we want to enforce for the project. It would be
> > good
> > >> >>>>> if @Stephan
> > >> >>>>> Ewen <st...@ververica.com> and @Aljoscha Krettek <
> > >> >>>> aljoscha@ververica.com> can
> > >> >>>>> also provide more thoughts on this.
> > >> >>>>>
> > >> >>>>>
> > >> >>>>> Re: Jingsong
> > >> >>>>>
> > >> >>>>> As you said, there are some batched system source, like
> > parquet/orc
> > >> >>>> source.
> > >> >>>>>> Could we have the batch emit interface to improve performance?
> > The
> > >> >>>> queue of
> > >> >>>>>> per record may cause performance degradation.
> > >> >>>>>
> > >> >>>>>
> > >> >>>>> The current interface does not necessarily cause performance
> > problem
> > >> >>> in a
> > >> >>>>> multi-threading case. In fact, the base implementation allows
> > >> >>>> SplitReaders
> > >> >>>>> to add a batch <E> of records<T> to the records queue<E>, so
> each
> > >> >>> element
> > >> >>>>> in the records queue would be a batch <E>. In this case, when
> the
> > >> >> main
> > >> >>>>> thread polls records, it will take a batch <E> of records <T>
> from
> > >> >> the
> > >> >>>>> shared records queue and process the records <T> in a batch
> > manner.
> > >> >>>>>
> > >> >>>>> Thanks,
> > >> >>>>>
> > >> >>>>> Jiangjie (Becket) Qin
> > >> >>>>>
> > >> >>>>> On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <
> > jingsonglee0@gmail.com
> > >> >
> > >> >>>> wrote:
> > >> >>>>>
> > >> >>>>>> Hi Becket,
> > >> >>>>>>
> > >> >>>>>> I also have some performance concerns too.
> > >> >>>>>>
> > >> >>>>>> If I understand correctly, SourceOutput will emit data per
> record
> > >> >> into
> > >> >>>> the
> > >> >>>>>> queue? I'm worried about the multithreading performance of this
> > >> >> queue.
> > >> >>>>>>
> > >> >>>>>>> One example is some batched messaging systems which only have
> an
> > >> >>> offset
> > >> >>>>>> for the entire batch instead of individual messages in the
> batch.
> > >> >>>>>>
> > >> >>>>>> As you said, there are some batched system source, like
> > parquet/orc
> > >> >>>> source.
> > >> >>>>>> Could we have the batch emit interface to improve performance?
> > The
> > >> >>>> queue of
> > >> >>>>>> per record may cause performance degradation.
> > >> >>>>>>
> > >> >>>>>> Best,
> > >> >>>>>> Jingsong Lee
> > >> >>>>>>
> > >> >>>>>> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com>
> > wrote:
> > >> >>>>>>
> > >> >>>>>>> Hi Becket,
> > >> >>>>>>>
> > >> >>>>>>> I think Dawid explained things clearly and makes a lot of
> sense.
> > >> >>>>>>> I'm also in favor of #2, because #1 doesn't work for our
> future
> > >> >>> unified
> > >> >>>>>>> envrionment.
> > >> >>>>>>>
> > >> >>>>>>> You can see the vision in this documentation [1]. In the
> future,
> > >> we
> > >> >>>> would
> > >> >>>>>>> like to
> > >> >>>>>>> drop the global streaming/batch mode in SQL (i.e.
> > >> >>>>>>> EnvironmentSettings#inStreamingMode/inBatchMode).
> > >> >>>>>>> A source is bounded or unbounded once defined, so queries can
> be
> > >> >>>> inferred
> > >> >>>>>>> from source to run
> > >> >>>>>>> in streaming or batch or hybrid mode. However, in #1, we will
> > lose
> > >> >>> this
> > >> >>>>>>> ability because the framework
> > >> >>>>>>> doesn't know whether the source is bounded or unbounded.
> > >> >>>>>>>
> > >> >>>>>>> Best,
> > >> >>>>>>> Jark
> > >> >>>>>>>
> > >> >>>>>>>
> > >> >>>>>>> [1]:
> > >> >>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
> > >> >>>>>>>
> > >> >>>>>>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <
> > piotr@ververica.com
> > >> >
> > >> >>>>>> wrote:
> > >> >>>>>>>
> > >> >>>>>>>> Hi,
> > >> >>>>>>>>
> > >> >>>>>>>> Regarding the:
> > >> >>>>>>>>
> > >> >>>>>>>> Collection<E> getNextRecords()
> > >> >>>>>>>>
> > >> >>>>>>>> I’m pretty sure such design would unfortunately impact the
> > >> >>> performance
> > >> >>>>>>>> (accessing and potentially creating the collection on the hot
> > >> >> path).
> > >> >>>>>>>>
> > >> >>>>>>>> Also the
> > >> >>>>>>>>
> > >> >>>>>>>> InputStatus emitNext(DataOutput<T> output) throws Exception;
> > >> >>>>>>>> or
> > >> >>>>>>>> Status pollNext(SourceOutput<T> sourceOutput) throws
> Exception;
> > >> >>>>>>>>
> > >> >>>>>>>> Gives us some opportunities in the future, to allow Source
> hot
> > >> >>> looping
> > >> >>>>>>>> inside, until it receives some signal “please exit because of
> > >> some
> > >> >>>>>>> reasons”
> > >> >>>>>>>> (output collector could return such hint upon collecting the
> > >> >>> result).
> > >> >>>>>> But
> > >> >>>>>>>> that’s another topic outside of this FLIP’s scope.
> > >> >>>>>>>>
> > >> >>>>>>>> Piotrek
> > >> >>>>>>>>
> > >> >>>>>>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <
> trohrmann@apache.org
> > >
> > >> >>>>>> wrote:
> > >> >>>>>>>>>
> > >> >>>>>>>>> Hi Becket,
> > >> >>>>>>>>>
> > >> >>>>>>>>> quick clarification from my side because I think you
> > >> >> misunderstood
> > >> >>> my
> > >> >>>>>>>>> question. I did not suggest to let the SourceReader return
> > only
> > >> a
> > >> >>>>>>> single
> > >> >>>>>>>>> record at a time when calling getNextRecords. As the return
> > type
> > >> >>>>>>>> indicates,
> > >> >>>>>>>>> the method can return an arbitrary number of records.
> > >> >>>>>>>>>
> > >> >>>>>>>>> Cheers,
> > >> >>>>>>>>> Till
> > >> >>>>>>>>>
> > >> >>>>>>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> > >> >>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> > >> >>>>>>>>> wrote:
> > >> >>>>>>>>>
> > >> >>>>>>>>>> Hi Becket,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Issue #1 - Design of Source interface
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I mentioned the lack of a method like
> > >> >>>>>>>> Source#createEnumerator(Boundedness
> > >> >>>>>>>>>> boundedness, SplitEnumeratorContext context), because
> without
> > >> >> the
> > >> >>>>>>>> current
> > >> >>>>>>>>>> proposal is not complete/does not work.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> If we say that boundedness is an intrinsic property of a
> > source
> > >> >>> imo
> > >> >>>>>> we
> > >> >>>>>>>>>> don't need the Source#createEnumerator(Boundedness
> > boundedness,
> > >> >>>>>>>>>> SplitEnumeratorContext context) method.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Assuming a source from my previous example:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Source source = KafkaSource.builder()
> > >> >>>>>>>>>>    ...
> > >> >>>>>>>>>>    .untilTimestamp(...)
> > >> >>>>>>>>>>    .build()
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Would the enumerator differ if created like
> > >> >>>>>>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs
> source
> > >> >>>>>>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating
> > myself,
> > >> >> but
> > >> >>>>>>> this
> > >> >>>>>>>> is
> > >> >>>>>>>>>> the part that my opinion differ the most from the current
> > >> >>> proposal.
> > >> >>>>>> I
> > >> >>>>>>>>>> really think it should always be the source that tells if
> it
> > is
> > >> >>>>>>> bounded
> > >> >>>>>>>> or
> > >> >>>>>>>>>> not. In the current proposal methods
> > >> >> continousSource/boundedSource
> > >> >>>>>>>> somewhat
> > >> >>>>>>>>>> reconfigure the source, which I think is misleading.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I think a call like:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Source source = KafkaSource.builder()
> > >> >>>>>>>>>>    ...
> > >> >>>>>>>>>>    .readContinously() / readUntilLatestOffset() /
> > >> >>> readUntilTimestamp
> > >> >>>> /
> > >> >>>>>>>> readUntilOffsets / ...
> > >> >>>>>>>>>>    .build()
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> is way cleaner (and expressive) than
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Source source = KafkaSource.builder()
> > >> >>>>>>>>>>    ...
> > >> >>>>>>>>>>    .build()
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> env.continousSource(source) // which actually underneath
> > would
> > >> >>> call
> > >> >>>>>>>> createEnumerator(CONTINUOUS, ctx) which would be equivalent
> to
> > >> >>>>>>>> source.readContinously().createEnumerator(ctx)
> > >> >>>>>>>>>> // or
> > >> >>>>>>>>>> env.boundedSource(source) // which actually underneath
> would
> > >> >> call
> > >> >>>>>>>> createEnumerator(BOUNDED, ctx) which would be equivalent to
> > >> >>>>>>>> source.readUntilLatestOffset().createEnumerator(ctx)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Sorry for the comparison, but to me it seems there is too
> > much
> > >> >>> magic
> > >> >>>>>>>>>> happening underneath those two calls.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I really believe the Source interface should have
> > >> getBoundedness
> > >> >>>>>>> method
> > >> >>>>>>>>>> instead of (supportBoundedness) +
> > createEnumerator(Boundedness,
> > >> >>> ...)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Issue #2 - Design of
> > >> >>>>>>>>>>
> > >> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> As you might have guessed I am slightly in favor of option
> #2
> > >> >>>>>>> modified.
> > >> >>>>>>>>>> Yes I am aware every step of the dag would have to be able
> to
> > >> >> say
> > >> >>> if
> > >> >>>>>>> it
> > >> >>>>>>>> is
> > >> >>>>>>>>>> bounded or not. I have a feeling it would be easier to
> > express
> > >> >>> cross
> > >> >>>>>>>>>> bounded/unbounded operations, but I must admit I have not
> > >> >> thought
> > >> >>> it
> > >> >>>>>>>>>> through thoroughly, In the spirit of batch is just a
> special
> > >> >> case
> > >> >>> of
> > >> >>>>>>>>>> streaming I thought BoundedStream would extend from
> > DataStream.
> > >> >>>>>>> Correct
> > >> >>>>>>>> me
> > >> >>>>>>>>>> if I am wrong. In such a setup the cross bounded/unbounded
> > >> >>> operation
> > >> >>>>>>>> could
> > >> >>>>>>>>>> be expressed quite easily I think:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> DataStream {
> > >> >>>>>>>>>>    DataStream join(DataStream, ...); // we could not really
> > >> tell
> > >> >> if
> > >> >>>>>> the
> > >> >>>>>>>> result is bounded or not, but because bounded stream is a
> > special
> > >> >>> case
> > >> >>>>>> of
> > >> >>>>>>>> unbounded the API object is correct, irrespective if the left
> > or
> > >> >>> right
> > >> >>>>>>> side
> > >> >>>>>>>> of the join is bounded
> > >> >>>>>>>>>> }
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> BoundedStream extends DataStream {
> > >> >>>>>>>>>>    BoundedStream join(BoundedStream, ...); // only if both
> > >> sides
> > >> >>> are
> > >> >>>>>>>> bounded the result can be bounded as well. However we do have
> > >> >> access
> > >> >>>> to
> > >> >>>>>>> the
> > >> >>>>>>>> DataStream#join here, so you can still join with a DataStream
> > >> >>>>>>>>>> }
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On the other hand I also see benefits of two completely
> > >> >> disjointed
> > >> >>>>>>> APIs,
> > >> >>>>>>>>>> as we could prohibit some streaming calls in the bounded
> > API. I
> > >> >>>>>> can't
> > >> >>>>>>>> think
> > >> >>>>>>>>>> of any unbounded operators that could not be implemented
> for
> > >> >>> bounded
> > >> >>>>>>>> stream.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Besides I think we both agree we don't like the method:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> DataStream boundedStream(Source)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> suggested in the current state of the FLIP. Do we ? :)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Best,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Dawid
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On 10/12/2019 18:57, Becket Qin wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi folks,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks for the discussion, great feedback. Also thanks
> Dawid
> > >> for
> > >> >>> the
> > >> >>>>>>>>>> explanation, it is much clearer now.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> One thing that is indeed missing from the FLIP is how the
> > >> >>>>>> boundedness
> > >> >>>>>>> is
> > >> >>>>>>>>>> passed to the Source implementation. So the API should be
> > >> >>>>>>>>>> Source#createEnumerator(Boundedness boundedness,
> > >> >>>>>>> SplitEnumeratorContext
> > >> >>>>>>>>>> context)
> > >> >>>>>>>>>> And we can probably remove the
> > >> >>> Source#supportBoundedness(Boundedness
> > >> >>>>>>>>>> boundedness) method.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Assuming we have that, we are essentially choosing from one
> > of
> > >> >> the
> > >> >>>>>>>>>> following two options:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Option 1:
> > >> >>>>>>>>>> // The source is continuous source, and only unbounded
> > >> >> operations
> > >> >>>>>> can
> > >> >>>>>>> be
> > >> >>>>>>>>>> performed.
> > >> >>>>>>>>>> DataStream<Type> datastream =
> > env.continuousSource(someSource);
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> // The source is bounded source, both bounded and unbounded
> > >> >>>>>> operations
> > >> >>>>>>>> can
> > >> >>>>>>>>>> be performed.
> > >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> > >> >>>>>>>> env.boundedSource(someSource);
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>    - Pros:
> > >> >>>>>>>>>>         a) explicit boundary between bounded / unbounded
> > >> streams,
> > >> >>> it
> > >> >>>>>> is
> > >> >>>>>>>>>> quite simple and clear to the users.
> > >> >>>>>>>>>>    - Cons:
> > >> >>>>>>>>>>         a) For applications that do not involve bounded
> > >> >> operations,
> > >> >>>>>> they
> > >> >>>>>>>>>> still have to call different API to distinguish bounded /
> > >> >>> unbounded
> > >> >>>>>>>> streams.
> > >> >>>>>>>>>>         b) No support for bounded stream to run in a
> > streaming
> > >> >>>> runtime
> > >> >>>>>>>>>> setting, i.e. scheduling and operators behaviors.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Option 2:
> > >> >>>>>>>>>> // The source is either bounded or unbounded, but only
> > >> unbounded
> > >> >>>>>>>> operations
> > >> >>>>>>>>>> could be performed on the returned DataStream.
> > >> >>>>>>>>>> DataStream<Type> dataStream = env.source(someSource);
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> // The source must be a bounded source, otherwise exception
> > is
> > >> >>>>>> thrown.
> > >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> > >> >>>>>>>>>> env.boundedSource(boundedSource);
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> The pros and cons are exactly the opposite of option 1.
> > >> >>>>>>>>>>    - Pros:
> > >> >>>>>>>>>>         a) For applications that do not involve bounded
> > >> >> operations,
> > >> >>>>>> they
> > >> >>>>>>>>>> still have to call different API to distinguish bounded /
> > >> >>> unbounded
> > >> >>>>>>>> streams.
> > >> >>>>>>>>>>         b) Support for bounded stream to run in a streaming
> > >> >> runtime
> > >> >>>>>>>> setting,
> > >> >>>>>>>>>> i.e. scheduling and operators behaviors.
> > >> >>>>>>>>>>    - Cons:
> > >> >>>>>>>>>>         a) Bounded / unbounded streams are kind of mixed,
> > i.e.
> > >> >>> given
> > >> >>>> a
> > >> >>>>>>>>>> DataStream, it is not clear whether it is bounded or not,
> > >> unless
> > >> >>> you
> > >> >>>>>>>> have
> > >> >>>>>>>>>> the access to its source.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> If we only think from the Source API perspective, option 2
> > >> >> seems a
> > >> >>>>>>>> better
> > >> >>>>>>>>>> choice because functionality wise it is a superset of
> option
> > 1,
> > >> >> at
> > >> >>>>>> the
> > >> >>>>>>>> cost
> > >> >>>>>>>>>> of some seemingly acceptable ambiguity in the DataStream
> API.
> > >> >>>>>>>>>> But if we look at the DataStream API as a whole, option 1
> > seems
> > >> >> a
> > >> >>>>>>>> clearer
> > >> >>>>>>>>>> choice. For example, some times a library may have to know
> > >> >>> whether a
> > >> >>>>>>>>>> certain task will finish or not. And it would be difficult
> to
> > >> >> tell
> > >> >>>>>> if
> > >> >>>>>>>> the
> > >> >>>>>>>>>> input is a DataStream, unless additional information is
> > >> provided
> > >> >>> all
> > >> >>>>>>> the
> > >> >>>>>>>>>> way from the Source. One possible solution is to have a
> > >> >> *modified
> > >> >>>>>>>> option 2*
> > >> >>>>>>>>>> which adds a method to the DataStream API to indicate
> > >> >> boundedness,
> > >> >>>>>>> such
> > >> >>>>>>>> as
> > >> >>>>>>>>>> getBoundedness(). It would solve the problem with a
> potential
> > >> >>>>>>> confusion
> > >> >>>>>>>> of
> > >> >>>>>>>>>> what is difference between a DataStream with
> > >> >> getBoundedness()=true
> > >> >>>>>>> and a
> > >> >>>>>>>>>> BoundedDataStream. But that seems not super difficult to
> > >> >> explain.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> So from API's perspective, I don't have a strong opinion
> > >> between
> > >> >>>>>>>> *option 1*
> > >> >>>>>>>>>> and *modified option 2. *I like the cleanness of option 1,
> > but
> > >> >>>>>>> modified
> > >> >>>>>>>>>> option 2 would be more attractive if we have concrete use
> > case
> > >> >> for
> > >> >>>>>> the
> > >> >>>>>>>>>> "Bounded stream with unbounded streaming runtime settings".
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Re: Till
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Maybe this has already been asked before but I was
> wondering
> > >> why
> > >> >>> the
> > >> >>>>>>>>>> SourceReader interface has the method pollNext which hands
> > the
> > >> >>>>>>>>>> responsibility of outputting elements to the SourceReader
> > >> >>>>>>>> implementation?
> > >> >>>>>>>>>> Has this been done for backwards compatibility reasons with
> > the
> > >> >>> old
> > >> >>>>>>>> source
> > >> >>>>>>>>>> interface? If not, then one could define a Collection<E>
> > >> >>>>>>>> getNextRecords()
> > >> >>>>>>>>>> method which returns the currently retrieved records and
> then
> > >> >> the
> > >> >>>>>>> caller
> > >> >>>>>>>>>> emits them outside of the SourceReader. That way the
> > interface
> > >> >>> would
> > >> >>>>>>> not
> > >> >>>>>>>>>> allow to implement an outputting loop where we never hand
> > back
> > >> >>>>>> control
> > >> >>>>>>>> to
> > >> >>>>>>>>>> the caller. At the moment, this contract can be easily
> broken
> > >> >> and
> > >> >>> is
> > >> >>>>>>>> only
> > >> >>>>>>>>>> mentioned loosely in the JavaDocs.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> The primary reason we handover the SourceOutput to the
> > >> >>> SourceReader
> > >> >>>>>> is
> > >> >>>>>>>>>> because sometimes it is difficult for a SourceReader to
> emit
> > >> one
> > >> >>>>>>> record
> > >> >>>>>>>> at
> > >> >>>>>>>>>> a time. One example is some batched messaging systems which
> > >> only
> > >> >>>>>> have
> > >> >>>>>>> an
> > >> >>>>>>>>>> offset for the entire batch instead of individual messages
> in
> > >> >> the
> > >> >>>>>>>> batch. In
> > >> >>>>>>>>>> that case, returning one record at a time would leave the
> > >> >>>>>> SourceReader
> > >> >>>>>>>> in
> > >> >>>>>>>>>> an uncheckpointable state because they can only checkpoint
> at
> > >> >> the
> > >> >>>>>>> batch
> > >> >>>>>>>>>> boundaries.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Jiangjie (Becket) Qin
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <
> > >> >>> trohrmann@apache.org
> > >> >>>>>>>> <ma...@apache.org>> <trohrmann@apache.org
> <mailto:
> > >> >>>>>>>> trohrmann@apache.org>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi everyone,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> thanks for drafting this FLIP. It reads very well.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Concerning Dawid's proposal, I tend to agree. The
> boundedness
> > >> >>> could
> > >> >>>>>>> come
> > >> >>>>>>>>>> from the source and tell the system how to treat the
> operator
> > >> >>>>>>>> (scheduling
> > >> >>>>>>>>>> wise). From a user's perspective it should be fine to get
> > back
> > >> a
> > >> >>>>>>>> DataStream
> > >> >>>>>>>>>> when calling env.source(boundedSource) if he does not need
> > >> >> special
> > >> >>>>>>>>>> operations defined on a BoundedDataStream. If he needs
> this,
> > >> >> then
> > >> >>>>>> one
> > >> >>>>>>>> could
> > >> >>>>>>>>>> use the method BoundedDataStream
> > >> >> env.boundedSource(boundedSource).
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> If possible, we could enforce the proper usage of
> > >> >>>>>> env.boundedSource()
> > >> >>>>>>> by
> > >> >>>>>>>>>> introducing a BoundedSource type so that one cannot pass an
> > >> >>>>>>>>>> unbounded source to it. That way users would not be able to
> > >> >> shoot
> > >> >>>>>>>>>> themselves in the foot.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Maybe this has already been asked before but I was
> wondering
> > >> why
> > >> >>> the
> > >> >>>>>>>>>> SourceReader interface has the method pollNext which hands
> > the
> > >> >>>>>>>>>> responsibility of outputting elements to the SourceReader
> > >> >>>>>>>> implementation?
> > >> >>>>>>>>>> Has this been done for backwards compatibility reasons with
> > the
> > >> >>> old
> > >> >>>>>>>> source
> > >> >>>>>>>>>> interface? If not, then one could define a Collection<E>
> > >> >>>>>>>> getNextRecords()
> > >> >>>>>>>>>> method which returns the currently retrieved records and
> then
> > >> >> the
> > >> >>>>>>> caller
> > >> >>>>>>>>>> emits them outside of the SourceReader. That way the
> > interface
> > >> >>> would
> > >> >>>>>>> not
> > >> >>>>>>>>>> allow to implement an outputting loop where we never hand
> > back
> > >> >>>>>> control
> > >> >>>>>>>> to
> > >> >>>>>>>>>> the caller. At the moment, this contract can be easily
> broken
> > >> >> and
> > >> >>> is
> > >> >>>>>>>> only
> > >> >>>>>>>>>> mentioned loosely in the JavaDocs.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Cheers,
> > >> >>>>>>>>>> Till
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <
> > >> >>> jingsonglee0@gmail.com
> > >> >>>>>>>> <ma...@gmail.com>> <jingsonglee0@gmail.com
> > >> <mailto:
> > >> >>>>>>>> jingsonglee0@gmail.com>>
> > >> >>>>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi all,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I think current design is good.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> My understanding is:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> For execution mode: bounded mode and continuous mode, It's
> > >> >> totally
> > >> >>>>>>>>>> different. I don't think we have the ability to integrate
> the
> > >> >> two
> > >> >>>>>>> models
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> at
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> present. It's about scheduling, memory, algorithms, States,
> > >> etc.
> > >> >>> we
> > >> >>>>>>>>>> shouldn't confuse them.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> For source capabilities: only bounded, only continuous,
> both
> > >> >>> bounded
> > >> >>>>>>> and
> > >> >>>>>>>>>> continuous.
> > >> >>>>>>>>>> I think Kafka is a source that can be ran both bounded
> > >> >>>>>>>>>> and continuous execution mode.
> > >> >>>>>>>>>> And Kafka with end offset should be ran both bounded
> > >> >>>>>>>>>> and continuous execution mode.  Using apache Beam with
> Flink
> > >> >>>>>> runner, I
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> used
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> to run a "bounded" Kafka in streaming mode. For our
> previous
> > >> >>>>>>> DataStream,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> it
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> is not necessarily required that the source cannot be
> > bounded.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> So it is my thought for Dawid's question:
> > >> >>>>>>>>>> 1.pass a bounded source to continuousSource() +1
> > >> >>>>>>>>>> 2.pass a continuous source to boundedSource() -1, should
> > throw
> > >> >>>>>>>> exception.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> In StreamExecutionEnvironment, continuousSource and
> > >> >> boundedSource
> > >> >>>>>>> define
> > >> >>>>>>>>>> the execution mode. It defines a clear boundary of
> execution
> > >> >> mode.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Best,
> > >> >>>>>>>>>> Jingsong Lee
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com
> > >> >>> <mailto:
> > >> >>>>>>>> imjark@gmail.com>> <imjark@gmail.com <mailto:
> imjark@gmail.com
> > >>
> > >> >>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I agree with Dawid's point that the boundedness information
> > >> >> should
> > >> >>>>>>> come
> > >> >>>>>>>>>> from the source itself (e.g. the end timestamp), not
> through
> > >> >>>>>>>>>> env.boundedSouce()/continuousSource().
> > >> >>>>>>>>>> I think if we want to support something like `env.source()`
> > >> that
> > >> >>>>>>> derive
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> execution mode from source,
> > `supportsBoundedness(Boundedness)`
> > >> >>>>>>>>>> method is not enough, because we don't know whether it is
> > >> >> bounded
> > >> >>> or
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> not.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Best,
> > >> >>>>>>>>>> Jark
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
> > >> >>>>>> dwysakowicz@apache.org
> > >> >>>>>>>> <ma...@apache.org>> <dwysakowicz@apache.org
> > >> <mailto:
> > >> >>>>>>>> dwysakowicz@apache.org>>
> > >> >>>>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> One more thing. In the current proposal, with the
> > >> >>>>>>>>>> supportsBoundedness(Boundedness) method and the boundedness
> > >> >> coming
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> from
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> either continuousSource or boundedSource I could not find
> how
> > >> >> this
> > >> >>>>>>>>>> information is fed back to the SplitEnumerator.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Best,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Dawid
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On 09/12/2019 13:52, Becket Qin wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi Dawid,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks for the comments. This actually brings another
> > relevant
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> question
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> about what does a "bounded source" imply. I actually had
> the
> > >> >> same
> > >> >>>>>>>>>> impression when I look at the Source API. Here is what I
> > >> >>> understand
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> after
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> some discussion with Stephan. The bounded source has the
> > >> >> following
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> impacts.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 1. API validity.
> > >> >>>>>>>>>> - A bounded source generates a bounded stream so some
> > >> operations
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> only
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> works for bounded records would be performed, e.g. sort.
> > >> >>>>>>>>>> - To expose these bounded stream only APIs, there are two
> > >> >> options:
> > >> >>>>>>>>>>       a. Add them to the DataStream API and throw exception
> > if
> > >> a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> method
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> is
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> called on an unbounded stream.
> > >> >>>>>>>>>>       b. Create a BoundedDataStream class which is returned
> > >> from
> > >> >>>>>>>>>> env.boundedSource(), while DataStream is returned from
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> env.continousSource().
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Note that this cannot be done by having single
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> env.source(theSource)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> even
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the Source has a getBoundedness() method.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 2. Scheduling
> > >> >>>>>>>>>> - A bounded source could be computed stage by stage without
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> bringing
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> up
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> all
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the tasks at the same time.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 3. Operator behaviors
> > >> >>>>>>>>>> - A bounded source indicates the records are finite so some
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> operators
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> can
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> wait until it receives all the records before it starts the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> processing.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> In the above impact, only 1 is relevant to the API design.
> > And
> > >> >> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> current
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> proposal in FLIP-27 is following 1.b.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> // boundedness depends of source property, imo this should
> > >> >> always
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> be
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> preferred
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> In your proposal, does DataStream have bounded stream only
> > >> >>> methods?
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> It
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> looks it should have, otherwise passing a bounded Source to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> env.source()
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> would be confusing. In that case, we will essentially do
> 1.a
> > if
> > >> >> an
> > >> >>>>>>>>>> unbounded Source is created from
> env.source(unboundedSource).
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> If we have the methods only supported for bounded streams
> in
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> DataStream,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> it
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> seems a little weird to have a separate BoundedDataStream
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> interface.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Am I understand it correctly?
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Jiangjie (Becket) Qin
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi all,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Really well written proposal and very important one. I must
> > >> >> admit
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> have
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> not understood all the intricacies of it yet.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> One question I have though is about where does the
> > information
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> about
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> boundedness come from. I think in most cases it is a
> property
> > >> of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> source. As you described it might be e.g. end offset, a
> flag
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> should
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> it
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> monitor new splits etc. I think it would be a really nice
> use
> > >> >> case
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> be
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> able to say:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> new KafkaSource().readUntil(long timestamp),
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> which could work as an "end offset". Moreover I think all
> > >> >> Bounded
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> sources
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> support continuous mode, but no intrinsically continuous
> > source
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> support
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Bounded mode. If I understood the proposal correctly it
> > suggest
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> boundedness sort of "comes" from the outside of the source,
> > >> from
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> invokation of either boundedStream or continousSource.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I am wondering if it would make sense to actually change
> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> method
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> boolean Source#supportsBoundedness(Boundedness)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Boundedness Source#getBoundedness().
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> As for the methods #boundedSource, #continousSource,
> assuming
> > >> >> the
> > >> >>>>>>>>>> boundedness is property of the source they do not affect
> how
> > >> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> enumerator
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> works, but mostly how the dag is scheduled, right? I am not
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> against
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> those
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> methods, but I think it is a very specific use case to
> > actually
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> override
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the property of the source. In general I would expect users
> > to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> only
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> call
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> env.source(theSource), where the source tells if it is
> > bounded
> > >> >> or
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> not. I
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> would suggest considering following set of methods:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> // boundedness depends of source property, imo this should
> > >> >> always
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> be
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> preferred
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> // always continous execution, whether bounded or unbounded
> > >> >> source
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> DataStream<MyType> boundedStream =
> > >> >> env.continousSource(theSource);
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> // imo this would make sense if the BoundedDataStream
> > provides
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> additional features unavailable for continous mode
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> BoundedDataStream<MyType> batch =
> > env.boundedSource(theSource);
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Best,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Dawid
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks, Becket, for updating this.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I agree with moving the aspects you mentioned into separate
> > >> >> FLIPs
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> -
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> this
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> one way becoming unwieldy in size.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> +1 to the FLIP in its current state. Its a very detailed
> > >> >> write-up,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> nicely
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> done!
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <
> > >> becket.qin@gmail.com
> > >> >>>>>>>> <ma...@gmail.com>> <becket.qin@gmail.com
> <mailto:
> > >> >>>>>>>> becket.qin@gmail.com>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> becket.qin@gmail.com <ma...@gmail.com>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi all,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Sorry for the long belated update. I have updated FLIP-27
> > wiki
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> page
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> with
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the latest proposals. Some noticeable changes include:
> > >> >>>>>>>>>> 1. A new generic communication mechanism between
> > >> SplitEnumerator
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> SourceReader.
> > >> >>>>>>>>>> 2. Some detail API method signature changes.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> We left a few things out of this FLIP and will address them
> > in
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> separate
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> FLIPs. Including:
> > >> >>>>>>>>>> 1. Per split event time.
> > >> >>>>>>>>>> 2. Event time alignment.
> > >> >>>>>>>>>> 3. Fine grained failover for SplitEnumerator failure.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Please let us know if you have any question.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Jiangjie (Becket) Qin
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <
> > sewen@apache.org
> > >> >>>>>>> <mailto:
> > >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:
> sewen@apache.org
> > >>
> > >> <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi  Łukasz!
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Becket and me are working hard on figuring out the last
> > details
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> implementing the first PoC. We would update the FLIP
> > hopefully
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> next
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> week.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> There is a fair chance that a first version of this will be
> > in
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 1.10,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> but
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> think it will take another release to battle test it and
> > >> migrate
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> connectors.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Best,
> > >> >>>>>>>>>> Stephan
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <
> > >> >> ljd@touk.pl
> > >> >>>>>>>> <ma...@touk.pl>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> ljd@touk.pl <ma...@touk.pl>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> This proposal looks very promising for us. Do you have any
> > >> plans
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> in
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> which
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Flink release it is going to be released? We are thinking
> on
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> using a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Data
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Set API for our future use cases but on the other hand Data
> > Set
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> API
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> is
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> going to be deprecated so using proposed bounded data
> streams
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> solution
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> could be more viable in the long term.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks,
> > >> >>>>>>>>>> Łukasz
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On 2019/10/01 15:48:03, Thomas Weise <
> thomas.weise@gmail.com
> > >> >>>>>> <mailto:
> > >> >>>>>>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
> > >> >>>>>>>> thomas.weise@gmail.com>> <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> thomas.weise@gmail.com <ma...@gmail.com>>
> > wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks for putting together this proposal!
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I see that the "Per Split Event Time" and "Event Time
> > >> Alignment"
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> sections
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> are still TBD.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> It would probably be good to flesh those out a bit before
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> proceeding
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> too
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> far
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> as the event time alignment will probably influence the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> interaction
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> with
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the split reader, specifically ReaderStatus
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> emitNext(SourceOutput<E>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> output).
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> We currently have only one implementation for event time
> > >> >> alignment
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> in
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Kinesis consumer. The synchronization in that case takes
> > place
> > >> >> as
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> last
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> step before records are emitted downstream (RecordEmitter).
> > >> With
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> currently proposed interfaces, the equivalent can be
> > >> implemented
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> in
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> reader loop, although note that in the Kinesis consumer the
> > per
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> shard
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> threads push records.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Synchronization has not been implemented for the Kafka
> > consumer
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> yet.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
> > >> >>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> When I looked at it, I realized that the implementation
> will
> > >> >> look
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> quite
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> different
> > >> >>>>>>>>>> from Kinesis because it needs to take place in the pull
> part,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> where
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> records
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> are taken from the Kafka client. Due to the multiplexing it
> > >> >> cannot
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> be
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> done
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> by blocking the split thread like it currently works for
> > >> >> Kinesis.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Reading
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> from individual Kafka partitions needs to be controlled via
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> pause/resume
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> on the Kafka client.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> To take on that responsibility the split thread would need
> to
> > >> be
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> aware
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>> watermarks or at least whether it should or should not
> > continue
> > >> >> to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> consume
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> a given split and this may require a different SourceReader
> > or
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> SourceOutput
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> interface.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks,
> > >> >>>>>>>>>> Thomas
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <
> mmyy1110@gmail.com
> > >> >>>>>> <mailto:
> > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > >> >> mmyy1110@gmail.com
> > >> >>>>>
> > >> >>>>>> <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi Stephan,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thank you for feedback!
> > >> >>>>>>>>>> Will take a look at your branch before public discussing.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <
> > >> sewen@apache.org
> > >> >>>>>>>> <ma...@apache.org>> <sewen@apache.org <mailto:
> > >> >>> sewen@apache.org
> > >> >>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi Biao!
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks for reviving this. I would like to join this
> > discussion,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> but
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> am
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> quite occupied with the 1.9 release, so can we maybe pause
> > this
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> discussion
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> for a week or so?
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> In the meantime I can share some suggestion based on prior
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> experiments:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> How to do watermarks / timestamp extractors in a simpler
> and
> > >> >> more
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> flexible
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> way. I think that part is quite promising should be part of
> > the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> new
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> source
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> interface.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > >> >>>>>>>> <
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > >> >>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > >> >>>>>>>> <
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > >> >>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Some experiments on how to build the source reader and its
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> library
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> for
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> common threading/split patterns:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > >> >>>>>>>> <
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > >> >>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Best,
> > >> >>>>>>>>>> Stephan
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <
> > mmyy1110@gmail.com
> > >> >>>>>>> <mailto:
> > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > >> >> mmyy1110@gmail.com
> > >> >>>>>
> > >> >>>>>> <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi devs,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Since 1.9 is nearly released, I think we could get back to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> FLIP-27.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> believe it should be included in 1.10.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> There are so many things mentioned in document of FLIP-27.
> > [1]
> > >> I
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> think
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> we'd better discuss them separately. However the wiki is
> not
> > a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> good
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> place
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> to discuss. I wrote google doc about SplitReader API which
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> misses
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> some
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> details in the document. [2]
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 1.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > >> >>>>>>>> <
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > >> >>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 2.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > >> >>>>>>>> <
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > >> >>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <
> mmyy1110@gmail.com
> > >> >>>>>> <mailto:
> > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > >> >> mmyy1110@gmail.com
> > >> >>>>>
> > >> >>>>>> <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi Steven,
> > >> >>>>>>>>>> Thank you for the feedback. Please take a look at the
> > document
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> FLIP-27
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >> >>>>>>>> <
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >> >>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> which
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> is updated recently. A lot of details of enumerator were
> > added
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> in
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> this
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> document. I think it would help.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Steven Wu <stevenz3wu@gmail.com <mailto:
> stevenz3wu@gmail.com
> > >>
> > >> >> <
> > >> >>>>>>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
> > >> >>>>>>> stevenz3wu@gmail.com
> > >> >>>>>>>> <ma...@gmail.com>> <stevenz3wu@gmail.com
> <mailto:
> > >> >>>>>>>> stevenz3wu@gmail.com>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 于2019年3月28日周四
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 下午12:52写道:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> This proposal mentioned that SplitEnumerator might run on
> the
> > >> >>>>>>>>>> JobManager or
> > >> >>>>>>>>>> in a single task on a TaskManager.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> if enumerator is a single task on a taskmanager, then the
> job
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> DAG
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> can
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> never
> > >> >>>>>>>>>> been embarrassingly parallel anymore. That will nullify the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> leverage
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> fine-grained recovery for embarrassingly parallel jobs.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> It's not clear to me what's the implication of running
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> enumerator
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> on
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> jobmanager. So I will leave that out for now.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <
> mmyy1110@gmail.com
> > >> >>>>>> <mailto:
> > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > >> >> mmyy1110@gmail.com
> > >> >>>>>
> > >> >>>>>> <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi Stephan & Piotrek,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thank you for feedback.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> It seems that there are a lot of things to do in community.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> am
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> just
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> afraid that this discussion may be forgotten since there so
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> many
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> proposals
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> recently.
> > >> >>>>>>>>>> Anyway, wish to see the split topics soon :)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:
> > >> >>> piotr@da-platform.com
> > >> >>>>>>>>
> > >> >>>>>>> <
> > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
> > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
> > >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 于2019年1月24日周四
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 下午8:21写道:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi Biao!
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> This discussion was stalled because of preparations for
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> open
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> sourcing
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> & merging Blink. I think before creating the tickets we
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> should
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> split this
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> discussion into topics/areas outlined by Stephan and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> create
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Flips
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> for
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I think there is no chance for this to be completed in
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> couple
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> remaining
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> weeks/1 month before 1.8 feature freeze, however it would
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> be
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> good
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> to aim
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> with those changes for 1.9.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Piotrek
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com
> > >> <mailto:
> > >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> > >> >> mmyy1110@gmail.com
> > >> >>>>>
> > >> >>>>>> <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi community,
> > >> >>>>>>>>>> The summary of Stephan makes a lot sense to me. It is
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> much
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> clearer
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> indeed
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> after splitting the complex topic into small ones.
> > >> >>>>>>>>>> I was wondering is there any detail plan for next step?
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> If
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> not,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> would
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> like to push this thing forward by creating some JIRA
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> issues.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Another question is that should version 1.8 include
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> these
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> features?
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Stephan Ewen <sewen@apache.org <ma...@apache.org>>
> <
> > >> >>>>>>>> sewen@apache.org <ma...@apache.org>> <
> sewen@apache.org
> > >> >>>> <mailto:
> > >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:
> sewen@apache.org
> > >>
> > >> >>>>>>>> 于2018年12月1日周六
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 上午4:20写道:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks everyone for the lively discussion. Let me try
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> summarize
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> where I
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> see convergence in the discussion and open issues.
> > >> >>>>>>>>>> I'll try to group this by design aspect of the source.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Please
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> let me
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> know
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> if I got things wrong or missed something crucial here.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> For issues 1-3, if the below reflects the state of the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> discussion, I
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> would
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> try and update the FLIP in the next days.
> > >> >>>>>>>>>> For the remaining ones we need more discussion.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> I would suggest to fork each of these aspects into a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> separate
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> mail
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> thread,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> or will loose sight of the individual aspects.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> *(1) Separation of Split Enumerator and Split Reader*
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - All seem to agree this is a good thing
> > >> >>>>>>>>>> - Split Enumerator could in the end live on JobManager
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> (and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> assign
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> splits
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> via RPC) or in a task (and assign splits via data
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> streams)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - this discussion is orthogonal and should come later,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> when
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> interface
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> is agreed upon.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> *(2) Split Readers for one or more splits*
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Discussion seems to agree that we need to support
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> one
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> reader
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> possibly handles multiple splits concurrently.
> > >> >>>>>>>>>> - The requirement comes from sources where one
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> poll()-style
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> call
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> fetches
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> data from different splits / partitions
> > >> >>>>>>>>>>     --> example sources that require that would be for
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> example
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Kafka,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Pravega, Pulsar
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Could have one split reader per source, or multiple
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> split
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> readers
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> share the "poll()" function
> > >> >>>>>>>>>> - To not make it too complicated, we can start with
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> thinking
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> about
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> one
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> split reader for all splits initially and see if that
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> covers
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> all
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> requirements
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> *(3) Threading model of the Split Reader*
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Most active part of the discussion ;-)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - A non-blocking way for Flink's task code to interact
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> with
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> source
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> is
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> needed in order to a task runtime code based on a
> > >> >>>>>>>>>> single-threaded/actor-style task design
> > >> >>>>>>>>>>     --> I personally am a big proponent of that, it will
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> help
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> with
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> well-behaved checkpoints, efficiency, and simpler yet
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> more
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> robust
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> runtime
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> code
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Users care about simple abstraction, so as a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> subclass
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> SplitReader
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> (non-blocking / async) we need to have a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> BlockingSplitReader
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> which
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> will
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> form the basis of most source implementations.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> BlockingSplitReader
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> lets
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> users do blocking simple poll() calls.
> > >> >>>>>>>>>> - The BlockingSplitReader would spawn a thread (or
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> more)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> thread(s) can make blocking calls and hand over data
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> buffers
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> via
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> blocking
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> queue
> > >> >>>>>>>>>> - This should allow us to cover both, a fully async
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> runtime,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> simple
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> blocking interface for users.
> > >> >>>>>>>>>> - This is actually very similar to how the Kafka
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> connectors
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> work.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Kafka
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 9+ with one thread, Kafka 8 with multiple threads
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - On the base SplitReader (the async one), the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> non-blocking
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> method
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> gets the next chunk of data would signal data
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> availability
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> via
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> CompletableFuture, because that gives the best
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> flexibility
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> (can
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> await
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> completion or register notification handlers).
> > >> >>>>>>>>>> - The source task would register a "thenHandle()" (or
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> similar)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> on the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> future to put a "take next data" task into the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> actor-style
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> mailbox
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> *(4) Split Enumeration and Assignment*
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Splits may be generated lazily, both in cases where
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> there
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> is a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> limited
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> number of splits (but very many), or splits are
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> discovered
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> over
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> time
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Assignment should also be lazy, to get better load
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> balancing
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Assignment needs support locality preferences
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Possible design based on discussion so far:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>     --> SplitReader has a method "addSplits(SplitT...)"
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> add
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> one or
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> more
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> splits. Some split readers might assume they have only
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> one
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> split
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> ever,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> concurrently, others assume multiple splits. (Note:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> idea
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> behind
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> being
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> able
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> to add multiple splits at the same time is to ease
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> startup
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> where
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> multiple
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> splits may be assigned instantly.)
> > >> >>>>>>>>>>     --> SplitReader has a context object on which it can
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> call
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> indicate
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> when
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> splits are completed. The enumerator gets that
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> notification and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> can
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> use
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> decide when to assign new splits. This should help both
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> in
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> cases
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> sources
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that take splits lazily (file readers) and in case the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> source
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> needs to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> preserve a partial order between splits (Kinesis,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Pravega,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Pulsar may
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> need
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that).
> > >> >>>>>>>>>>     --> SplitEnumerator gets notification when
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> SplitReaders
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> start
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> when
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> they finish splits. They can decide at that moment to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> push
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> more
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> splits
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that reader
> > >> >>>>>>>>>>     --> The SplitEnumerator should probably be aware of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> source
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> parallelism, to build its initial distribution.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Open question: Should the source expose something
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> like
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> "host
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> preferences", so that yarn/mesos/k8s can take this into
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> account
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> when
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> selecting a node to start a TM on?
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> *(5) Watermarks and event time alignment*
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Watermark generation, as well as idleness, needs to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> be
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> per
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> split
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> (like
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> currently in the Kafka Source, per partition)
> > >> >>>>>>>>>> - It is desirable to support optional
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> event-time-alignment,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> meaning
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> splits that are ahead are back-pressured or temporarily
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> unsubscribed
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - I think i would be desirable to encapsulate
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> watermark
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> generation
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> logic
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> in watermark generators, for a separation of concerns.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> The
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> watermark
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> generators should run per split.
> > >> >>>>>>>>>> - Using watermark generators would also help with
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> another
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> problem of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> suggested interface, namely supporting non-periodic
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> watermarks
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> efficiently.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Need a way to "dispatch" next record to different
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> watermark
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> generators
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Need a way to tell SplitReader to "suspend" a split
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> until a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> certain
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> watermark is reached (event time backpressure)
> > >> >>>>>>>>>> - This would in fact be not needed (and thus simpler)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> if
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> we
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> had
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> SplitReader per split and may be a reason to re-open
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> discussion
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> *(6) Watermarks across splits and in the Split
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Enumerator*
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - The split enumerator may need some watermark
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> awareness,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> which
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> should
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> be
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> purely based on split metadata (like create timestamp
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> file
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> splits)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - If there are still more splits with overlapping
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> event
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> time
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> range
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> for
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> split reader, then that split reader should not advance
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> watermark
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> within the split beyond the overlap boundary. Otherwise
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> future
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> splits
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> will
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> produce late data.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - One way to approach this could be that the split
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> enumerator
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> may
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> send
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> watermarks to the readers, and the readers cannot emit
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> watermarks
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> beyond
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that received watermark.
> > >> >>>>>>>>>> - Many split enumerators would simply immediately send
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Long.MAX
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> out
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> leave the progress purely to the split readers.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - For event-time alignment / split back pressure, this
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> begs
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> question
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> how we can avoid deadlocks that may arise when splits
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> are
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> suspended
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> for
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> event time back pressure,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> *(7) Batch and streaming Unification*
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Functionality wise, the above design should support
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> both
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Batch often (mostly) does not care about reading "in
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> order"
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> generating watermarks
> > >> >>>>>>>>>>     --> Might use different enumerator logic that is
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> more
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> locality
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> aware
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and ignores event time order
> > >> >>>>>>>>>>     --> Does not generate watermarks
> > >> >>>>>>>>>> - Would be great if bounded sources could be
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> identified
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> at
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> compile
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> time,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> so that "env.addBoundedSource(...)" is type safe and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> can
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> return a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> "BoundedDataStream".
> > >> >>>>>>>>>> - Possible to defer this discussion until later
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> *Miscellaneous Comments*
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - Should the source have a TypeInformation for the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> produced
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> type,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> instead
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> of a serializer? We need a type information in the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> stream
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> anyways, and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> can
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> derive the serializer from that. Plus, creating the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> serializer
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> should
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> respect the ExecutionConfig.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> - The TypeSerializer interface is very powerful but
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> also
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> not
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> easy to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> implement. Its purpose is to handle data super
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> efficiently,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> support
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> flexible ways of evolution, etc.
> > >> >>>>>>>>>> For metadata I would suggest to look at the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> SimpleVersionedSerializer
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> instead, which is used for example for checkpoint
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> master
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> hooks,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> or for
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> streaming file sink. I think that is is a good match
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> for
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> cases
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> where
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> we
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> do
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> not need more than ser/deser (no copy, etc.) and don't
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> need to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> push
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> versioning out of the serialization paths for best
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> performance
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> (as in
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> TypeSerializer)
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> k.kloudas@data-artisans.com>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Hi Biao,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Thanks for the answer!
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> So given the multi-threaded readers, now we have as
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> open
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> questions:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 1) How do we let the checkpoints pass through our
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> multi-threaded
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> reader
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> operator?
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 2) Do we have separate reader and source operators or
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> not? In
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> strategy
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that has a separate source, the source operator has a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> parallelism of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 1
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> is responsible for split recovery only.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> For the first one, given also the constraints
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> (blocking,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> finite
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> queues,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> etc), I do not have an answer yet.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> For the 2nd, I think that we should go with separate
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> operators
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> for
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> source and the readers, for the following reasons:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 1) This is more aligned with a potential future
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> improvement
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> where the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> split
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> discovery becomes a responsibility of the JobManager
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> readers are
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> pooling more work from the JM.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 2) The source is going to be the "single point of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> truth".
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> It
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> will
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> know
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> what
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> has been processed and what not. If the source and the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> readers
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> are a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> single
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> operator with parallelism > 1, or in general, if the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> split
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> discovery
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> is
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> done by each task individually, then:
> > >> >>>>>>>>>>    i) we have to have a deterministic scheme for each
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> reader to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> assign
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> splits to itself (e.g. mod subtaskId). This is not
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> necessarily
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> trivial
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> for
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> all sources.
> > >> >>>>>>>>>>    ii) each reader would have to keep a copy of all its
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> processed
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> slpits
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>    iii) the state has to be a union state with a
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> non-trivial
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> merging
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> logic
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> in order to support rescaling.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Two additional points that you raised above:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> i) The point that you raised that we need to keep all
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> splits
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> (processed
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> not-processed) I think is a bit of a strong
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> requirement.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> This
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> would
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> imply
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that for infinite sources the state will grow
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> indefinitely.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> This is
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> problem
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> is even more pronounced if we do not have a single
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> source
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> assigns
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> splits to readers, as each reader will have its own
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> copy
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> of
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> state.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> ii) it is true that for finite sources we need to
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> somehow
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> not
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> close
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> readers when the source/split discoverer finishes. The
> > >> >>>>>>>>>> ContinuousFileReaderOperator has a work-around for
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> that.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> It is
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> not
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> elegant,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> and checkpoints are not emitted after closing the
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> source,
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> but
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> this, I
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> believe, is a bigger problem which requires more
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> changes
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> than
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> just
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> refactoring the source interface.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Cheers,
> > >> >>>>>>>>>> Kostas
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> --
> > >> >>>>>>>>>> Best, Jingsong Lee
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>
> > >> >>>>>>
> > >> >>>>>> --
> > >> >>>>>> Best, Jingsong Lee
> > >> >>>>>>
> > >> >>>>>
> > >> >>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >> >
> > >>
> > >>
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Steven Wu <st...@gmail.com>.
Becket,

Regarding "UNBOUNDED source that stops at some point", I found it difficult
to grasp what UNBOUNDED really mean.

If we want to use Kafka source with an end/stop time, I guess you call it
UNBOUNDED kafka source that stops (aka BOUNDED-streaming). The
terminology is a little confusing to me. Maybe BOUNDED/UNBOUNDED shouldn't
be used to categorize source. Just call it Kafka source and it can run in
either BOUNDED or UNBOUNDED mode.

Thanks,
Steven

On Thu, Dec 19, 2019 at 7:02 PM Becket Qin <be...@gmail.com> wrote:

> I had an offline chat with Jark, and here are some more thoughts:
>
> 1. From SQL perspective, BOUNDED source leads to the batch execution mode,
> UNBOUNDED source leads to the streaming execution mode.
> 2. The semantic of UNBOUNDED source is may or may not stop. The semantic of
> BOUNDED source is will stop.
> 3. The semantic of DataStream is may or may not terminate. The semantic of
> BoundedDataStream is will terminate.
>
> Given that, option 3 seems a better option because:
> 1. SQL already has strict binding between Boundedness and execution mode.
> Letting DataStream be consistent would be good.
> 2. The semantic of UNBOUNDED source is exactly the same as DataStream. So
> we should avoid breaking such semantic, i.e. turning some DataStream from
> "may or may not terminate" to "will terminate".
>
> For case where users want BOUNDED-streaming combination, they can simply
> use an UNBOUNDED source that stops at some point. We can even provide a
> simple wrapper to wrap a BOUNDED source as an UNBOUNDED source if that
> helps. But API wise, option 3 seems telling a pretty good whole story.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
>
> On Thu, Dec 19, 2019 at 10:30 PM Becket Qin <be...@gmail.com> wrote:
>
> > Hi Timo,
> >
> > Bounded is just a special case of unbounded and every bounded source can
> >> also be treated as an unbounded source. This would unify the API if
> >> people don't need a bounded operation.
> >
> >
> > With option 3 users can still get a unified API with something like
> below:
> >
> > DataStream boundedStream = env.boundedSource(boundedSource);
> > DataStream unboundedStream = env.source(unboundedSource);
> >
> > So in both cases, users can still use a unified DataStream without
> > touching the bounded stream only methods.
> > By "unify the API if people don't need the bounded operation". Do you
> > expect a DataStream with a Bounded source to have the batch operators and
> > scheduler settings as well?
> >
> >
> > If we allow DataStream from BOUNDED source, we will essentially pick
> "*modified
> > option 2*".
> >
> > // The source is either bounded or unbounded, but only unbounded
> >> operations could be performed on the returned DataStream.
> >> DataStream<Type> dataStream = env.source(someSource);
> >
> >
> >> // The source must be a bounded source, otherwise exception is thrown.
> >> BoundedDataStream<Type> boundedDataStream =
> >> env.boundedSource(boundedSource);
> >
> >
> >
> > // Add the following method to DataStream
> >
> > Boundedness DataStream#getBoundedness();
> >
> >
> > From pure logical perspective, Boundedness and runtime settings
> > (Stream/Batch) are two orthogonal dimensions. And are specified in the
> > following way.
> >
> > *Boundedness* - defined by the source: BOUNDED / UNBOUNDED.
> > *Running mode* - defined by the API class: DataStream (Streaming mode) /
> > BoundedDataStream (batch mode).
> >
> > Excluding the UNBOUNDED-batch combination, the "*modified option 2"*
> > covers the rest three combination. Compared with "*modified option 2*",
> > the main benefit of option 3 is its simplicity and clearness, by tying
> > boundedness to running mode and giving up BOUNDED-streaming combination.
> >
> > Just to be clear, I am fine with either option. But I would like to
> > understand a bit more about the bounded-streaming use case and when users
> > would prefer this over bounded-batch case, and whether the added value
> > justifies the additional complexity in the API. Two cases I can think of
> > are:
> > 1. The records in DataStream will be processed in order, while
> > BoundedDataStream processes records without order guarantee.
> > 2. DataStream emits intermediate results when processing a finite
> dataset,
> > while BoundedDataStream only emit the final result. In any case, it could
> > be supported by an UNBOUNDED source stopping at some point.
> >
> > Case 1 is actually misleading because DataStream in general doesn't
> really
> > support in-order process.
> > Case 2 seems a rare use case because the instantaneous intermediate
> result
> > seems difficult to reason about. In any case, this can be supported by an
> > UNBOUNDED source that stops at some point.
> >
> > Is there other use cases for bounded-streaming combination I missed? I am
> > a little hesitating to put the testing requirement here because ideally
> I'd
> > avoid having public APIs for testing purpose only. And this could be
> > resolved by having a UNBOUNDED source stopping at some point as well.
> >
> > Sorry for the long discussion, but I would really like to make an API
> > decision after knowing all the pros and cons.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> >
> >
> >
> >
> >
> >
> > On Thu, Dec 19, 2019 at 6:19 PM Timo Walther <tw...@apache.org> wrote:
> >
> >> Hi Becket,
> >>
> >> regarding *Option 3* I think we can relax the constraints for
> >> env.source():
> >>
> >> // MySource can be bounded or unbounded
> >> DataStream<Type> dataStream = env.source(mySource);
> >>
> >> // MySource must be bounded, otherwise throws exception.
> >> BoundedDataStream<Type> boundedDataStream = env.boundedSource(mySource);
> >>
> >> Bounded is just a special case of unbounded and every bounded source can
> >> also be treated as an unbounded source. This would unify the API if
> >> people don't need a bounded operation. It also addresses Jark's
> concerns.
> >>
> >> Regards,
> >> Timo
> >>
> >>
> >> On 18.12.19 14:16, Becket Qin wrote:
> >> > Hi Jark,
> >> >
> >> > Please see the reply below:
> >> >
> >> > Regarding to option#3, my concern is that if we don't support
> streaming
> >> >> mode for bounded source,
> >> >> how could we create a testing source for streaming mode? Currently,
> >> all the
> >> >> testing source for streaming
> >> >> are bounded, so that the integration test will finish finally.
> >> >
> >> >
> >> > An UNBOUNDED source does not mean it will never stops. It simply
> >> indicates
> >> > that the source *may* run forever, so the runtime needs to be prepared
> >> for
> >> > that, but the task may still stop at some point when it hits some
> >> > source-specific condition. So an UNBOUNDED testing source can still
> >> stop at
> >> > some point if needed.
> >> >
> >> > Regarding to Source#getRecordOrder(), could we have a implicit
> contract
> >> >> that unbounded source should
> >> >> already read in order (i.e. reading partitions in parallel), for
> >> bounded
> >> >> source the order is not mandatory.
> >> >
> >> >
> >> >
> >> >> This is also the behaviors of the current sources.
> >> >
> >> > 1) a source can't guarantee it reads in strict order, because the
> >> producer
> >> >> may produce data not in order.
> >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can reorder
> >> data.
> >> >
> >> >
> >> > It is true that sometimes the source cannot guarantee the record
> order,
> >> but
> >> > sometimes it can. Right now, even for stream processing, there is no
> >> > processing order guarantee. For example, a join operator may emit a
> >> later
> >> > record which successfully found a join match earlier.
> >> > Event order is one of the most important requirements for event
> >> processing,
> >> > a clear order guarantee would be necessary. That said, I agree that
> >> right
> >> > now even if the sources provide the record order requirement, the
> >> runtime
> >> > is not able to guarantee that out of the box. So I am OK if we add the
> >> > record order to the Source later. But we should avoid misleading users
> >> to
> >> > make them think the processing order is guaranteed when using the
> >> unbounded
> >> > runtime.
> >> >
> >> > Thanks,
> >> >
> >> > Jiangjie (Becket) Qin
> >> >
> >> > On Wed, Dec 18, 2019 at 10:29 AM Jark Wu <im...@gmail.com> wrote:
> >> >
> >> >> Hi Becket,
> >> >>
> >> >> That's great we have reached a consensus on Source#getBoundedness().
> >> >>
> >> >> Regarding to option#3, my concern is that if we don't support
> streaming
> >> >> mode for bounded source,
> >> >> how could we create a testing source for streaming mode? Currently,
> >> all the
> >> >> testing source for streaming
> >> >> are bounded, so that the integration test will finish finally.
> >> >>
> >> >> Regarding to Source#getRecordOrder(), could we have a implicit
> contract
> >> >> that unbounded source should
> >> >> already read in order (i.e. reading partitions in parallel), for
> >> bounded
> >> >> source the order is not mandatory.
> >> >> This is also the behaviors of the current sources.
> >> >> 1) a source can't guarantee it reads in strict order, because the
> >> producer
> >> >> may produce data not in order.
> >> >> 2) *Bounded-StrictOrder* is not necessary, because batch can reorder
> >> data.
> >> >>
> >> >> Best,
> >> >> Jark
> >> >>
> >> >>
> >> >>
> >> >> On Tue, 17 Dec 2019 at 22:03, Becket Qin <be...@gmail.com>
> wrote:
> >> >>
> >> >>> Hi folks,
> >> >>>
> >> >>> Thanks for the comments. I am convinced that the Source API should
> not
> >> >> take
> >> >>> boundedness as a parameter after it is constructed. What Timo and
> >> Dawid
> >> >>> suggested sounds a reasonable solution to me. So the Source API
> would
> >> >>> become:
> >> >>>
> >> >>> Source {
> >> >>>      Boundedness getBoundedness();
> >> >>> }
> >> >>>
> >> >>> Assuming the above Source API, in addition to the two options
> >> mentioned
> >> >> in
> >> >>> earlier emails, I am thinking of another option:
> >> >>>
> >> >>> *Option 3:*
> >> >>> // MySource must be unbounded, otherwise throws exception.
> >> >>> DataStream<Type> dataStream = env.source(mySource);
> >> >>>
> >> >>> // MySource must be bounded, otherwise throws exception.
> >> >>> BoundedDataStream<Type> boundedDataStream =
> >> env.boundedSource(mySource);
> >> >>>
> >> >>> The pros of this API are:
> >> >>>     a) It fits the requirements from Table / SQL well.
> >> >>>     b) DataStream users still have type safety (option 2 only has
> >> partial
> >> >>> type safety).
> >> >>>     c) Cristal clear boundedness from the API which makes DataStream
> >> join
> >> >> /
> >> >>> connect easy to reason about.
> >> >>> The caveats I see,
> >> >>>     a) It is inconsistent with Table since Table has one unified
> >> >> interface.
> >> >>>     b) No streaming mode for bounded source.
> >> >>>
> >> >>> @Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
> >> >>> <al...@ververica.com> what do you think of the approach?
> >> >>>
> >> >>>
> >> >>> Orthogonal to the above API, I am wondering whether boundedness is
> the
> >> >> only
> >> >>> dimension needed to describe the characteristic of the Source
> >> behavior.
> >> >> We
> >> >>> may also need to have another dimension of *record order*.
> >> >>>
> >> >>> For example, when a file source is reading from a directory with
> >> bounded
> >> >>> records, it may have two ways to read.
> >> >>> 1. Read files in parallel.
> >> >>> 2. Read files in the chronological order.
> >> >>> In both cases, the file source is a Bounded Source. However, the
> >> >> processing
> >> >>> requirement for downstream may be different. In the first case, the
> >> >>> record processing and result emitting order does not matter, e.g.
> word
> >> >>> count. In the second case, the records may have to be processed in
> the
> >> >>> order they were read, e.g. change log processing.
> >> >>>
> >> >>> If the Source only has a getBoundedness() method, the downstream
> >> >> processors
> >> >>> would not know whether the records emitted from the Source should be
> >> >>> processed in order or not. So combining the boundedness and record
> >> order,
> >> >>> we will have four scenarios:
> >> >>>
> >> >>> *Bounded-StrictOrder*:     A segment of change log.
> >> >>> *Bounded-Random*:          Batch Word Count.
> >> >>> *Unbounded-StrictOrder*: An infinite change log.
> >> >>> *Unbounded-Random*:     Streaming Word Count.
> >> >>>
> >> >>> Option 2 mentioned in the previous email was kind of trying to
> handle
> >> the
> >> >>> Bounded-StrictOrder case by creating a DataStream from a bounded
> >> source,
> >> >>> which actually does not work.
> >> >>> It looks that we do not have strict order support in some operators
> at
> >> >> this
> >> >>> point, e.g. join. But we may still want to add the semantic to the
> >> Source
> >> >>> first so later on we don't need to change all the source
> >> implementations,
> >> >>> especially given that many of them will be implemented by 3rd party.
> >> >>>
> >> >>> Given that, we need another dimension of *Record Order* in the
> Source.
> >> >> More
> >> >>> specifically, the API would become:
> >> >>>
> >> >>> Source {
> >> >>>      Boundedness getBoundedness();
> >> >>>      RecordOrder getRecordOrder();
> >> >>> }
> >> >>>
> >> >>> public enum RecordOrder {
> >> >>>      /** The record in the DataStream must be processed in its
> strict
> >> >> order
> >> >>> for correctness. */
> >> >>>      STRICT,
> >> >>>      /** The record in the DataStream can be processed in arbitrary
> >> order.
> >> >>> */
> >> >>>      RANDOM;
> >> >>> }
> >> >>>
> >> >>> Any thoughts?
> >> >>>
> >> >>> Thanks,
> >> >>>
> >> >>> Jiangjie (Becket) Qin
> >> >>>
> >> >>> On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <tw...@apache.org>
> >> wrote:
> >> >>>
> >> >>>> Hi Becket,
> >> >>>>
> >> >>>> I completely agree with Dawid's suggestion. The information about
> the
> >> >>>> boundedness should come out of the source. Because most of the
> >> >> streaming
> >> >>>> sources can be made bounded based on some connector specific
> >> criterion.
> >> >>>> In Kafka, it would be an end offset or end timestamp but in any
> case
> >> >>>> having just a env.boundedSource() is not enough because parameters
> >> for
> >> >>>> making the source bounded are missing.
> >> >>>>
> >> >>>> I suggest to have a simple `isBounded(): Boolean` flag in every
> >> source
> >> >>>> that might be influenced by a connector builder as Dawid mentioned.
> >> >>>>
> >> >>>> For type safety during programming, we can still go with *Final
> state
> >> >>>> 1*. By having a env.source() vs env.boundedSource(). The latter
> would
> >> >>>> just enforce that the boolean flag is set to `true` and could make
> >> >>>> bounded operations available (if we need that actually).
> >> >>>>
> >> >>>> However, I don't think that we should start making a unified Table
> >> API
> >> >>>> ununified again. Boundedness is an optimization property. Every
> >> bounded
> >> >>>> operation can also executed in an unbounded way using
> >> >> updates/retraction
> >> >>>> or watermarks.
> >> >>>>
> >> >>>> Regards,
> >> >>>> Timo
> >> >>>>
> >> >>>>
> >> >>>> On 15.12.19 14:22, Becket Qin wrote:
> >> >>>>> Hi Dawid and Jark,
> >> >>>>>
> >> >>>>> I think the discussion ultimately boils down to the question that
> >> >> which
> >> >>>> one
> >> >>>>> of the following two final states do we want? Once we make this
> >> >>> decision,
> >> >>>>> everything else can be naturally derived.
> >> >>>>>
> >> >>>>> *Final state 1*: Separate API for bounded / unbounded DataStream &
> >> >>> Table.
> >> >>>>> That means any code users write will be valid at the point when
> they
> >> >>>> write
> >> >>>>> the code. This is similar to having type safety check at
> programming
> >> >>>> time.
> >> >>>>> For example,
> >> >>>>>
> >> >>>>> BoundedDataStream extends DataStream {
> >> >>>>> // Operations only available for bounded data.
> >> >>>>> BoundedDataStream sort(...);
> >> >>>>>
> >> >>>>> // Interaction with another BoundedStream returns a Bounded
> stream.
> >> >>>>> BoundedJoinedDataStream join(BoundedDataStream other)
> >> >>>>>
> >> >>>>> // Interaction with another unbounded stream returns an unbounded
> >> >>> stream.
> >> >>>>> JoinedDataStream join(DataStream other)
> >> >>>>> }
> >> >>>>>
> >> >>>>> BoundedTable extends Table {
> >> >>>>>     // Bounded only operation.
> >> >>>>> BoundedTable sort(...);
> >> >>>>>
> >> >>>>> // Interaction with another BoundedTable returns a BoundedTable.
> >> >>>>> BoundedTable join(BoundedTable other)
> >> >>>>>
> >> >>>>> // Interaction with another unbounded table returns an unbounded
> >> >> table.
> >> >>>>> Table join(Table other)
> >> >>>>> }
> >> >>>>>
> >> >>>>> *Final state 2*: One unified API for bounded / unbounded
> DataStream
> >> /
> >> >>>>> Table.
> >> >>>>> That unified API may throw exception at DAG compilation time if an
> >> >>>> invalid
> >> >>>>> operation is tried. This is what Table API currently follows.
> >> >>>>>
> >> >>>>> DataStream {
> >> >>>>> // Throws exception if the DataStream is unbounded.
> >> >>>>> DataStream sort();
> >> >>>>> // Get boundedness.
> >> >>>>> Boundedness getBoundedness();
> >> >>>>> }
> >> >>>>>
> >> >>>>> Table {
> >> >>>>> // Throws exception if the table has infinite rows.
> >> >>>>> Table orderBy();
> >> >>>>>
> >> >>>>> // Get boundedness.
> >> >>>>> Boundedness getBoundedness();
> >> >>>>> }
> >> >>>>>
> >> >>>>> >From what I understand, there is no consensus so far on this
> >> decision
> >> >>>> yet.
> >> >>>>> Whichever final state we choose, we need to make it consistent
> >> across
> >> >>> the
> >> >>>>> entire project. We should avoid the case that Table follows one
> >> final
> >> >>>> state
> >> >>>>> while DataStream follows another. Some arguments I am aware of
> from
> >> >>> both
> >> >>>>> sides so far are following:
> >> >>>>>
> >> >>>>> Arguments for final state 1:
> >> >>>>> 1a) Clean API with method safety check at programming time.
> >> >>>>> 1b) (Counter 2b) Although SQL does not have programming time error
> >> >>>> check, SQL
> >> >>>>> is not really a "programming language" per se. So SQL can be
> >> >> different
> >> >>>> from
> >> >>>>> Table and DataStream.
> >> >>>>> 1c)  Although final state 2 seems making it easier for SQL to use
> >> >> given
> >> >>>> it
> >> >>>>> is more "config based" than "parameter based", final state 1 can
> >> >>> probably
> >> >>>>> also meet what SQL wants by wrapping the Source in TableSource /
> >> >>>>> TableSourceFactory API if needed.
> >> >>>>>
> >> >>>>> Arguments for final state 2:
> >> >>>>> 2a) The Source API itself seems already sort of following the
> >> unified
> >> >>> API
> >> >>>>> pattern.
> >> >>>>> 2b) There is no "programming time" method error check in SQL case,
> >> so
> >> >>> we
> >> >>>>> cannot really achieve final state 1 across the board.
> >> >>>>> 2c) It is an easier path given our current status, i.e. Table is
> >> >>> already
> >> >>>>> following final state 2.
> >> >>>>> 2d) Users can always explicitly check the boundedness if they want
> >> >> to.
> >> >>>>>
> >> >>>>> As I mentioned earlier, my initial thought was also to have a
> >> >>>>> "configuration based" Source rather than a "parameter based"
> Source.
> >> >> So
> >> >>>> it
> >> >>>>> is completely possible that I missed some important consideration
> or
> >> >>>> design
> >> >>>>> principles that we want to enforce for the project. It would be
> good
> >> >>>>> if @Stephan
> >> >>>>> Ewen <st...@ververica.com> and @Aljoscha Krettek <
> >> >>>> aljoscha@ververica.com> can
> >> >>>>> also provide more thoughts on this.
> >> >>>>>
> >> >>>>>
> >> >>>>> Re: Jingsong
> >> >>>>>
> >> >>>>> As you said, there are some batched system source, like
> parquet/orc
> >> >>>> source.
> >> >>>>>> Could we have the batch emit interface to improve performance?
> The
> >> >>>> queue of
> >> >>>>>> per record may cause performance degradation.
> >> >>>>>
> >> >>>>>
> >> >>>>> The current interface does not necessarily cause performance
> problem
> >> >>> in a
> >> >>>>> multi-threading case. In fact, the base implementation allows
> >> >>>> SplitReaders
> >> >>>>> to add a batch <E> of records<T> to the records queue<E>, so each
> >> >>> element
> >> >>>>> in the records queue would be a batch <E>. In this case, when the
> >> >> main
> >> >>>>> thread polls records, it will take a batch <E> of records <T> from
> >> >> the
> >> >>>>> shared records queue and process the records <T> in a batch
> manner.
> >> >>>>>
> >> >>>>> Thanks,
> >> >>>>>
> >> >>>>> Jiangjie (Becket) Qin
> >> >>>>>
> >> >>>>> On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <
> jingsonglee0@gmail.com
> >> >
> >> >>>> wrote:
> >> >>>>>
> >> >>>>>> Hi Becket,
> >> >>>>>>
> >> >>>>>> I also have some performance concerns too.
> >> >>>>>>
> >> >>>>>> If I understand correctly, SourceOutput will emit data per record
> >> >> into
> >> >>>> the
> >> >>>>>> queue? I'm worried about the multithreading performance of this
> >> >> queue.
> >> >>>>>>
> >> >>>>>>> One example is some batched messaging systems which only have an
> >> >>> offset
> >> >>>>>> for the entire batch instead of individual messages in the batch.
> >> >>>>>>
> >> >>>>>> As you said, there are some batched system source, like
> parquet/orc
> >> >>>> source.
> >> >>>>>> Could we have the batch emit interface to improve performance?
> The
> >> >>>> queue of
> >> >>>>>> per record may cause performance degradation.
> >> >>>>>>
> >> >>>>>> Best,
> >> >>>>>> Jingsong Lee
> >> >>>>>>
> >> >>>>>> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com>
> wrote:
> >> >>>>>>
> >> >>>>>>> Hi Becket,
> >> >>>>>>>
> >> >>>>>>> I think Dawid explained things clearly and makes a lot of sense.
> >> >>>>>>> I'm also in favor of #2, because #1 doesn't work for our future
> >> >>> unified
> >> >>>>>>> envrionment.
> >> >>>>>>>
> >> >>>>>>> You can see the vision in this documentation [1]. In the future,
> >> we
> >> >>>> would
> >> >>>>>>> like to
> >> >>>>>>> drop the global streaming/batch mode in SQL (i.e.
> >> >>>>>>> EnvironmentSettings#inStreamingMode/inBatchMode).
> >> >>>>>>> A source is bounded or unbounded once defined, so queries can be
> >> >>>> inferred
> >> >>>>>>> from source to run
> >> >>>>>>> in streaming or batch or hybrid mode. However, in #1, we will
> lose
> >> >>> this
> >> >>>>>>> ability because the framework
> >> >>>>>>> doesn't know whether the source is bounded or unbounded.
> >> >>>>>>>
> >> >>>>>>> Best,
> >> >>>>>>> Jark
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> [1]:
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
> >> >>>>>>>
> >> >>>>>>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <
> piotr@ververica.com
> >> >
> >> >>>>>> wrote:
> >> >>>>>>>
> >> >>>>>>>> Hi,
> >> >>>>>>>>
> >> >>>>>>>> Regarding the:
> >> >>>>>>>>
> >> >>>>>>>> Collection<E> getNextRecords()
> >> >>>>>>>>
> >> >>>>>>>> I’m pretty sure such design would unfortunately impact the
> >> >>> performance
> >> >>>>>>>> (accessing and potentially creating the collection on the hot
> >> >> path).
> >> >>>>>>>>
> >> >>>>>>>> Also the
> >> >>>>>>>>
> >> >>>>>>>> InputStatus emitNext(DataOutput<T> output) throws Exception;
> >> >>>>>>>> or
> >> >>>>>>>> Status pollNext(SourceOutput<T> sourceOutput) throws Exception;
> >> >>>>>>>>
> >> >>>>>>>> Gives us some opportunities in the future, to allow Source hot
> >> >>> looping
> >> >>>>>>>> inside, until it receives some signal “please exit because of
> >> some
> >> >>>>>>> reasons”
> >> >>>>>>>> (output collector could return such hint upon collecting the
> >> >>> result).
> >> >>>>>> But
> >> >>>>>>>> that’s another topic outside of this FLIP’s scope.
> >> >>>>>>>>
> >> >>>>>>>> Piotrek
> >> >>>>>>>>
> >> >>>>>>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <trohrmann@apache.org
> >
> >> >>>>>> wrote:
> >> >>>>>>>>>
> >> >>>>>>>>> Hi Becket,
> >> >>>>>>>>>
> >> >>>>>>>>> quick clarification from my side because I think you
> >> >> misunderstood
> >> >>> my
> >> >>>>>>>>> question. I did not suggest to let the SourceReader return
> only
> >> a
> >> >>>>>>> single
> >> >>>>>>>>> record at a time when calling getNextRecords. As the return
> type
> >> >>>>>>>> indicates,
> >> >>>>>>>>> the method can return an arbitrary number of records.
> >> >>>>>>>>>
> >> >>>>>>>>> Cheers,
> >> >>>>>>>>> Till
> >> >>>>>>>>>
> >> >>>>>>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> >> >>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> >> >>>>>>>>> wrote:
> >> >>>>>>>>>
> >> >>>>>>>>>> Hi Becket,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Issue #1 - Design of Source interface
> >> >>>>>>>>>>
> >> >>>>>>>>>> I mentioned the lack of a method like
> >> >>>>>>>> Source#createEnumerator(Boundedness
> >> >>>>>>>>>> boundedness, SplitEnumeratorContext context), because without
> >> >> the
> >> >>>>>>>> current
> >> >>>>>>>>>> proposal is not complete/does not work.
> >> >>>>>>>>>>
> >> >>>>>>>>>> If we say that boundedness is an intrinsic property of a
> source
> >> >>> imo
> >> >>>>>> we
> >> >>>>>>>>>> don't need the Source#createEnumerator(Boundedness
> boundedness,
> >> >>>>>>>>>> SplitEnumeratorContext context) method.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Assuming a source from my previous example:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Source source = KafkaSource.builder()
> >> >>>>>>>>>>    ...
> >> >>>>>>>>>>    .untilTimestamp(...)
> >> >>>>>>>>>>    .build()
> >> >>>>>>>>>>
> >> >>>>>>>>>> Would the enumerator differ if created like
> >> >>>>>>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
> >> >>>>>>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating
> myself,
> >> >> but
> >> >>>>>>> this
> >> >>>>>>>> is
> >> >>>>>>>>>> the part that my opinion differ the most from the current
> >> >>> proposal.
> >> >>>>>> I
> >> >>>>>>>>>> really think it should always be the source that tells if it
> is
> >> >>>>>>> bounded
> >> >>>>>>>> or
> >> >>>>>>>>>> not. In the current proposal methods
> >> >> continousSource/boundedSource
> >> >>>>>>>> somewhat
> >> >>>>>>>>>> reconfigure the source, which I think is misleading.
> >> >>>>>>>>>>
> >> >>>>>>>>>> I think a call like:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Source source = KafkaSource.builder()
> >> >>>>>>>>>>    ...
> >> >>>>>>>>>>    .readContinously() / readUntilLatestOffset() /
> >> >>> readUntilTimestamp
> >> >>>> /
> >> >>>>>>>> readUntilOffsets / ...
> >> >>>>>>>>>>    .build()
> >> >>>>>>>>>>
> >> >>>>>>>>>> is way cleaner (and expressive) than
> >> >>>>>>>>>>
> >> >>>>>>>>>> Source source = KafkaSource.builder()
> >> >>>>>>>>>>    ...
> >> >>>>>>>>>>    .build()
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> env.continousSource(source) // which actually underneath
> would
> >> >>> call
> >> >>>>>>>> createEnumerator(CONTINUOUS, ctx) which would be equivalent to
> >> >>>>>>>> source.readContinously().createEnumerator(ctx)
> >> >>>>>>>>>> // or
> >> >>>>>>>>>> env.boundedSource(source) // which actually underneath would
> >> >> call
> >> >>>>>>>> createEnumerator(BOUNDED, ctx) which would be equivalent to
> >> >>>>>>>> source.readUntilLatestOffset().createEnumerator(ctx)
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Sorry for the comparison, but to me it seems there is too
> much
> >> >>> magic
> >> >>>>>>>>>> happening underneath those two calls.
> >> >>>>>>>>>>
> >> >>>>>>>>>> I really believe the Source interface should have
> >> getBoundedness
> >> >>>>>>> method
> >> >>>>>>>>>> instead of (supportBoundedness) +
> createEnumerator(Boundedness,
> >> >>> ...)
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Issue #2 - Design of
> >> >>>>>>>>>>
> >> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> >> >>>>>>>>>>
> >> >>>>>>>>>> As you might have guessed I am slightly in favor of option #2
> >> >>>>>>> modified.
> >> >>>>>>>>>> Yes I am aware every step of the dag would have to be able to
> >> >> say
> >> >>> if
> >> >>>>>>> it
> >> >>>>>>>> is
> >> >>>>>>>>>> bounded or not. I have a feeling it would be easier to
> express
> >> >>> cross
> >> >>>>>>>>>> bounded/unbounded operations, but I must admit I have not
> >> >> thought
> >> >>> it
> >> >>>>>>>>>> through thoroughly, In the spirit of batch is just a special
> >> >> case
> >> >>> of
> >> >>>>>>>>>> streaming I thought BoundedStream would extend from
> DataStream.
> >> >>>>>>> Correct
> >> >>>>>>>> me
> >> >>>>>>>>>> if I am wrong. In such a setup the cross bounded/unbounded
> >> >>> operation
> >> >>>>>>>> could
> >> >>>>>>>>>> be expressed quite easily I think:
> >> >>>>>>>>>>
> >> >>>>>>>>>> DataStream {
> >> >>>>>>>>>>    DataStream join(DataStream, ...); // we could not really
> >> tell
> >> >> if
> >> >>>>>> the
> >> >>>>>>>> result is bounded or not, but because bounded stream is a
> special
> >> >>> case
> >> >>>>>> of
> >> >>>>>>>> unbounded the API object is correct, irrespective if the left
> or
> >> >>> right
> >> >>>>>>> side
> >> >>>>>>>> of the join is bounded
> >> >>>>>>>>>> }
> >> >>>>>>>>>>
> >> >>>>>>>>>> BoundedStream extends DataStream {
> >> >>>>>>>>>>    BoundedStream join(BoundedStream, ...); // only if both
> >> sides
> >> >>> are
> >> >>>>>>>> bounded the result can be bounded as well. However we do have
> >> >> access
> >> >>>> to
> >> >>>>>>> the
> >> >>>>>>>> DataStream#join here, so you can still join with a DataStream
> >> >>>>>>>>>> }
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On the other hand I also see benefits of two completely
> >> >> disjointed
> >> >>>>>>> APIs,
> >> >>>>>>>>>> as we could prohibit some streaming calls in the bounded
> API. I
> >> >>>>>> can't
> >> >>>>>>>> think
> >> >>>>>>>>>> of any unbounded operators that could not be implemented for
> >> >>> bounded
> >> >>>>>>>> stream.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Besides I think we both agree we don't like the method:
> >> >>>>>>>>>>
> >> >>>>>>>>>> DataStream boundedStream(Source)
> >> >>>>>>>>>>
> >> >>>>>>>>>> suggested in the current state of the FLIP. Do we ? :)
> >> >>>>>>>>>>
> >> >>>>>>>>>> Best,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Dawid
> >> >>>>>>>>>>
> >> >>>>>>>>>> On 10/12/2019 18:57, Becket Qin wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi folks,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks for the discussion, great feedback. Also thanks Dawid
> >> for
> >> >>> the
> >> >>>>>>>>>> explanation, it is much clearer now.
> >> >>>>>>>>>>
> >> >>>>>>>>>> One thing that is indeed missing from the FLIP is how the
> >> >>>>>> boundedness
> >> >>>>>>> is
> >> >>>>>>>>>> passed to the Source implementation. So the API should be
> >> >>>>>>>>>> Source#createEnumerator(Boundedness boundedness,
> >> >>>>>>> SplitEnumeratorContext
> >> >>>>>>>>>> context)
> >> >>>>>>>>>> And we can probably remove the
> >> >>> Source#supportBoundedness(Boundedness
> >> >>>>>>>>>> boundedness) method.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Assuming we have that, we are essentially choosing from one
> of
> >> >> the
> >> >>>>>>>>>> following two options:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Option 1:
> >> >>>>>>>>>> // The source is continuous source, and only unbounded
> >> >> operations
> >> >>>>>> can
> >> >>>>>>> be
> >> >>>>>>>>>> performed.
> >> >>>>>>>>>> DataStream<Type> datastream =
> env.continuousSource(someSource);
> >> >>>>>>>>>>
> >> >>>>>>>>>> // The source is bounded source, both bounded and unbounded
> >> >>>>>> operations
> >> >>>>>>>> can
> >> >>>>>>>>>> be performed.
> >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> >> >>>>>>>> env.boundedSource(someSource);
> >> >>>>>>>>>>
> >> >>>>>>>>>>    - Pros:
> >> >>>>>>>>>>         a) explicit boundary between bounded / unbounded
> >> streams,
> >> >>> it
> >> >>>>>> is
> >> >>>>>>>>>> quite simple and clear to the users.
> >> >>>>>>>>>>    - Cons:
> >> >>>>>>>>>>         a) For applications that do not involve bounded
> >> >> operations,
> >> >>>>>> they
> >> >>>>>>>>>> still have to call different API to distinguish bounded /
> >> >>> unbounded
> >> >>>>>>>> streams.
> >> >>>>>>>>>>         b) No support for bounded stream to run in a
> streaming
> >> >>>> runtime
> >> >>>>>>>>>> setting, i.e. scheduling and operators behaviors.
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Option 2:
> >> >>>>>>>>>> // The source is either bounded or unbounded, but only
> >> unbounded
> >> >>>>>>>> operations
> >> >>>>>>>>>> could be performed on the returned DataStream.
> >> >>>>>>>>>> DataStream<Type> dataStream = env.source(someSource);
> >> >>>>>>>>>>
> >> >>>>>>>>>> // The source must be a bounded source, otherwise exception
> is
> >> >>>>>> thrown.
> >> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> >> >>>>>>>>>> env.boundedSource(boundedSource);
> >> >>>>>>>>>>
> >> >>>>>>>>>> The pros and cons are exactly the opposite of option 1.
> >> >>>>>>>>>>    - Pros:
> >> >>>>>>>>>>         a) For applications that do not involve bounded
> >> >> operations,
> >> >>>>>> they
> >> >>>>>>>>>> still have to call different API to distinguish bounded /
> >> >>> unbounded
> >> >>>>>>>> streams.
> >> >>>>>>>>>>         b) Support for bounded stream to run in a streaming
> >> >> runtime
> >> >>>>>>>> setting,
> >> >>>>>>>>>> i.e. scheduling and operators behaviors.
> >> >>>>>>>>>>    - Cons:
> >> >>>>>>>>>>         a) Bounded / unbounded streams are kind of mixed,
> i.e.
> >> >>> given
> >> >>>> a
> >> >>>>>>>>>> DataStream, it is not clear whether it is bounded or not,
> >> unless
> >> >>> you
> >> >>>>>>>> have
> >> >>>>>>>>>> the access to its source.
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> If we only think from the Source API perspective, option 2
> >> >> seems a
> >> >>>>>>>> better
> >> >>>>>>>>>> choice because functionality wise it is a superset of option
> 1,
> >> >> at
> >> >>>>>> the
> >> >>>>>>>> cost
> >> >>>>>>>>>> of some seemingly acceptable ambiguity in the DataStream API.
> >> >>>>>>>>>> But if we look at the DataStream API as a whole, option 1
> seems
> >> >> a
> >> >>>>>>>> clearer
> >> >>>>>>>>>> choice. For example, some times a library may have to know
> >> >>> whether a
> >> >>>>>>>>>> certain task will finish or not. And it would be difficult to
> >> >> tell
> >> >>>>>> if
> >> >>>>>>>> the
> >> >>>>>>>>>> input is a DataStream, unless additional information is
> >> provided
> >> >>> all
> >> >>>>>>> the
> >> >>>>>>>>>> way from the Source. One possible solution is to have a
> >> >> *modified
> >> >>>>>>>> option 2*
> >> >>>>>>>>>> which adds a method to the DataStream API to indicate
> >> >> boundedness,
> >> >>>>>>> such
> >> >>>>>>>> as
> >> >>>>>>>>>> getBoundedness(). It would solve the problem with a potential
> >> >>>>>>> confusion
> >> >>>>>>>> of
> >> >>>>>>>>>> what is difference between a DataStream with
> >> >> getBoundedness()=true
> >> >>>>>>> and a
> >> >>>>>>>>>> BoundedDataStream. But that seems not super difficult to
> >> >> explain.
> >> >>>>>>>>>>
> >> >>>>>>>>>> So from API's perspective, I don't have a strong opinion
> >> between
> >> >>>>>>>> *option 1*
> >> >>>>>>>>>> and *modified option 2. *I like the cleanness of option 1,
> but
> >> >>>>>>> modified
> >> >>>>>>>>>> option 2 would be more attractive if we have concrete use
> case
> >> >> for
> >> >>>>>> the
> >> >>>>>>>>>> "Bounded stream with unbounded streaming runtime settings".
> >> >>>>>>>>>>
> >> >>>>>>>>>> Re: Till
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Maybe this has already been asked before but I was wondering
> >> why
> >> >>> the
> >> >>>>>>>>>> SourceReader interface has the method pollNext which hands
> the
> >> >>>>>>>>>> responsibility of outputting elements to the SourceReader
> >> >>>>>>>> implementation?
> >> >>>>>>>>>> Has this been done for backwards compatibility reasons with
> the
> >> >>> old
> >> >>>>>>>> source
> >> >>>>>>>>>> interface? If not, then one could define a Collection<E>
> >> >>>>>>>> getNextRecords()
> >> >>>>>>>>>> method which returns the currently retrieved records and then
> >> >> the
> >> >>>>>>> caller
> >> >>>>>>>>>> emits them outside of the SourceReader. That way the
> interface
> >> >>> would
> >> >>>>>>> not
> >> >>>>>>>>>> allow to implement an outputting loop where we never hand
> back
> >> >>>>>> control
> >> >>>>>>>> to
> >> >>>>>>>>>> the caller. At the moment, this contract can be easily broken
> >> >> and
> >> >>> is
> >> >>>>>>>> only
> >> >>>>>>>>>> mentioned loosely in the JavaDocs.
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> The primary reason we handover the SourceOutput to the
> >> >>> SourceReader
> >> >>>>>> is
> >> >>>>>>>>>> because sometimes it is difficult for a SourceReader to emit
> >> one
> >> >>>>>>> record
> >> >>>>>>>> at
> >> >>>>>>>>>> a time. One example is some batched messaging systems which
> >> only
> >> >>>>>> have
> >> >>>>>>> an
> >> >>>>>>>>>> offset for the entire batch instead of individual messages in
> >> >> the
> >> >>>>>>>> batch. In
> >> >>>>>>>>>> that case, returning one record at a time would leave the
> >> >>>>>> SourceReader
> >> >>>>>>>> in
> >> >>>>>>>>>> an uncheckpointable state because they can only checkpoint at
> >> >> the
> >> >>>>>>> batch
> >> >>>>>>>>>> boundaries.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Jiangjie (Becket) Qin
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <
> >> >>> trohrmann@apache.org
> >> >>>>>>>> <ma...@apache.org>> <trohrmann@apache.org <mailto:
> >> >>>>>>>> trohrmann@apache.org>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi everyone,
> >> >>>>>>>>>>
> >> >>>>>>>>>> thanks for drafting this FLIP. It reads very well.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Concerning Dawid's proposal, I tend to agree. The boundedness
> >> >>> could
> >> >>>>>>> come
> >> >>>>>>>>>> from the source and tell the system how to treat the operator
> >> >>>>>>>> (scheduling
> >> >>>>>>>>>> wise). From a user's perspective it should be fine to get
> back
> >> a
> >> >>>>>>>> DataStream
> >> >>>>>>>>>> when calling env.source(boundedSource) if he does not need
> >> >> special
> >> >>>>>>>>>> operations defined on a BoundedDataStream. If he needs this,
> >> >> then
> >> >>>>>> one
> >> >>>>>>>> could
> >> >>>>>>>>>> use the method BoundedDataStream
> >> >> env.boundedSource(boundedSource).
> >> >>>>>>>>>>
> >> >>>>>>>>>> If possible, we could enforce the proper usage of
> >> >>>>>> env.boundedSource()
> >> >>>>>>> by
> >> >>>>>>>>>> introducing a BoundedSource type so that one cannot pass an
> >> >>>>>>>>>> unbounded source to it. That way users would not be able to
> >> >> shoot
> >> >>>>>>>>>> themselves in the foot.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Maybe this has already been asked before but I was wondering
> >> why
> >> >>> the
> >> >>>>>>>>>> SourceReader interface has the method pollNext which hands
> the
> >> >>>>>>>>>> responsibility of outputting elements to the SourceReader
> >> >>>>>>>> implementation?
> >> >>>>>>>>>> Has this been done for backwards compatibility reasons with
> the
> >> >>> old
> >> >>>>>>>> source
> >> >>>>>>>>>> interface? If not, then one could define a Collection<E>
> >> >>>>>>>> getNextRecords()
> >> >>>>>>>>>> method which returns the currently retrieved records and then
> >> >> the
> >> >>>>>>> caller
> >> >>>>>>>>>> emits them outside of the SourceReader. That way the
> interface
> >> >>> would
> >> >>>>>>> not
> >> >>>>>>>>>> allow to implement an outputting loop where we never hand
> back
> >> >>>>>> control
> >> >>>>>>>> to
> >> >>>>>>>>>> the caller. At the moment, this contract can be easily broken
> >> >> and
> >> >>> is
> >> >>>>>>>> only
> >> >>>>>>>>>> mentioned loosely in the JavaDocs.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Cheers,
> >> >>>>>>>>>> Till
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <
> >> >>> jingsonglee0@gmail.com
> >> >>>>>>>> <ma...@gmail.com>> <jingsonglee0@gmail.com
> >> <mailto:
> >> >>>>>>>> jingsonglee0@gmail.com>>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi all,
> >> >>>>>>>>>>
> >> >>>>>>>>>> I think current design is good.
> >> >>>>>>>>>>
> >> >>>>>>>>>> My understanding is:
> >> >>>>>>>>>>
> >> >>>>>>>>>> For execution mode: bounded mode and continuous mode, It's
> >> >> totally
> >> >>>>>>>>>> different. I don't think we have the ability to integrate the
> >> >> two
> >> >>>>>>> models
> >> >>>>>>>>>>
> >> >>>>>>>>>> at
> >> >>>>>>>>>>
> >> >>>>>>>>>> present. It's about scheduling, memory, algorithms, States,
> >> etc.
> >> >>> we
> >> >>>>>>>>>> shouldn't confuse them.
> >> >>>>>>>>>>
> >> >>>>>>>>>> For source capabilities: only bounded, only continuous, both
> >> >>> bounded
> >> >>>>>>> and
> >> >>>>>>>>>> continuous.
> >> >>>>>>>>>> I think Kafka is a source that can be ran both bounded
> >> >>>>>>>>>> and continuous execution mode.
> >> >>>>>>>>>> And Kafka with end offset should be ran both bounded
> >> >>>>>>>>>> and continuous execution mode.  Using apache Beam with Flink
> >> >>>>>> runner, I
> >> >>>>>>>>>>
> >> >>>>>>>>>> used
> >> >>>>>>>>>>
> >> >>>>>>>>>> to run a "bounded" Kafka in streaming mode. For our previous
> >> >>>>>>> DataStream,
> >> >>>>>>>>>>
> >> >>>>>>>>>> it
> >> >>>>>>>>>>
> >> >>>>>>>>>> is not necessarily required that the source cannot be
> bounded.
> >> >>>>>>>>>>
> >> >>>>>>>>>> So it is my thought for Dawid's question:
> >> >>>>>>>>>> 1.pass a bounded source to continuousSource() +1
> >> >>>>>>>>>> 2.pass a continuous source to boundedSource() -1, should
> throw
> >> >>>>>>>> exception.
> >> >>>>>>>>>>
> >> >>>>>>>>>> In StreamExecutionEnvironment, continuousSource and
> >> >> boundedSource
> >> >>>>>>> define
> >> >>>>>>>>>> the execution mode. It defines a clear boundary of execution
> >> >> mode.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Best,
> >> >>>>>>>>>> Jingsong Lee
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com
> >> >>> <mailto:
> >> >>>>>>>> imjark@gmail.com>> <imjark@gmail.com <mailto:imjark@gmail.com
> >>
> >> >>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> I agree with Dawid's point that the boundedness information
> >> >> should
> >> >>>>>>> come
> >> >>>>>>>>>> from the source itself (e.g. the end timestamp), not through
> >> >>>>>>>>>> env.boundedSouce()/continuousSource().
> >> >>>>>>>>>> I think if we want to support something like `env.source()`
> >> that
> >> >>>>>>> derive
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> execution mode from source,
> `supportsBoundedness(Boundedness)`
> >> >>>>>>>>>> method is not enough, because we don't know whether it is
> >> >> bounded
> >> >>> or
> >> >>>>>>>>>>
> >> >>>>>>>>>> not.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Best,
> >> >>>>>>>>>> Jark
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
> >> >>>>>> dwysakowicz@apache.org
> >> >>>>>>>> <ma...@apache.org>> <dwysakowicz@apache.org
> >> <mailto:
> >> >>>>>>>> dwysakowicz@apache.org>>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> One more thing. In the current proposal, with the
> >> >>>>>>>>>> supportsBoundedness(Boundedness) method and the boundedness
> >> >> coming
> >> >>>>>>>>>>
> >> >>>>>>>>>> from
> >> >>>>>>>>>>
> >> >>>>>>>>>> either continuousSource or boundedSource I could not find how
> >> >> this
> >> >>>>>>>>>> information is fed back to the SplitEnumerator.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Best,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Dawid
> >> >>>>>>>>>>
> >> >>>>>>>>>> On 09/12/2019 13:52, Becket Qin wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi Dawid,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks for the comments. This actually brings another
> relevant
> >> >>>>>>>>>>
> >> >>>>>>>>>> question
> >> >>>>>>>>>>
> >> >>>>>>>>>> about what does a "bounded source" imply. I actually had the
> >> >> same
> >> >>>>>>>>>> impression when I look at the Source API. Here is what I
> >> >>> understand
> >> >>>>>>>>>>
> >> >>>>>>>>>> after
> >> >>>>>>>>>>
> >> >>>>>>>>>> some discussion with Stephan. The bounded source has the
> >> >> following
> >> >>>>>>>>>>
> >> >>>>>>>>>> impacts.
> >> >>>>>>>>>>
> >> >>>>>>>>>> 1. API validity.
> >> >>>>>>>>>> - A bounded source generates a bounded stream so some
> >> operations
> >> >>>>>>>>>>
> >> >>>>>>>>>> that
> >> >>>>>>>>>>
> >> >>>>>>>>>> only
> >> >>>>>>>>>>
> >> >>>>>>>>>> works for bounded records would be performed, e.g. sort.
> >> >>>>>>>>>> - To expose these bounded stream only APIs, there are two
> >> >> options:
> >> >>>>>>>>>>       a. Add them to the DataStream API and throw exception
> if
> >> a
> >> >>>>>>>>>>
> >> >>>>>>>>>> method
> >> >>>>>>>>>>
> >> >>>>>>>>>> is
> >> >>>>>>>>>>
> >> >>>>>>>>>> called on an unbounded stream.
> >> >>>>>>>>>>       b. Create a BoundedDataStream class which is returned
> >> from
> >> >>>>>>>>>> env.boundedSource(), while DataStream is returned from
> >> >>>>>>>>>>
> >> >>>>>>>>>> env.continousSource().
> >> >>>>>>>>>>
> >> >>>>>>>>>> Note that this cannot be done by having single
> >> >>>>>>>>>>
> >> >>>>>>>>>> env.source(theSource)
> >> >>>>>>>>>>
> >> >>>>>>>>>> even
> >> >>>>>>>>>>
> >> >>>>>>>>>> the Source has a getBoundedness() method.
> >> >>>>>>>>>>
> >> >>>>>>>>>> 2. Scheduling
> >> >>>>>>>>>> - A bounded source could be computed stage by stage without
> >> >>>>>>>>>>
> >> >>>>>>>>>> bringing
> >> >>>>>>>>>>
> >> >>>>>>>>>> up
> >> >>>>>>>>>>
> >> >>>>>>>>>> all
> >> >>>>>>>>>>
> >> >>>>>>>>>> the tasks at the same time.
> >> >>>>>>>>>>
> >> >>>>>>>>>> 3. Operator behaviors
> >> >>>>>>>>>> - A bounded source indicates the records are finite so some
> >> >>>>>>>>>>
> >> >>>>>>>>>> operators
> >> >>>>>>>>>>
> >> >>>>>>>>>> can
> >> >>>>>>>>>>
> >> >>>>>>>>>> wait until it receives all the records before it starts the
> >> >>>>>>>>>>
> >> >>>>>>>>>> processing.
> >> >>>>>>>>>>
> >> >>>>>>>>>> In the above impact, only 1 is relevant to the API design.
> And
> >> >> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> current
> >> >>>>>>>>>>
> >> >>>>>>>>>> proposal in FLIP-27 is following 1.b.
> >> >>>>>>>>>>
> >> >>>>>>>>>> // boundedness depends of source property, imo this should
> >> >> always
> >> >>>>>>>>>>
> >> >>>>>>>>>> be
> >> >>>>>>>>>>
> >> >>>>>>>>>> preferred
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> In your proposal, does DataStream have bounded stream only
> >> >>> methods?
> >> >>>>>>>>>>
> >> >>>>>>>>>> It
> >> >>>>>>>>>>
> >> >>>>>>>>>> looks it should have, otherwise passing a bounded Source to
> >> >>>>>>>>>>
> >> >>>>>>>>>> env.source()
> >> >>>>>>>>>>
> >> >>>>>>>>>> would be confusing. In that case, we will essentially do 1.a
> if
> >> >> an
> >> >>>>>>>>>> unbounded Source is created from env.source(unboundedSource).
> >> >>>>>>>>>>
> >> >>>>>>>>>> If we have the methods only supported for bounded streams in
> >> >>>>>>>>>>
> >> >>>>>>>>>> DataStream,
> >> >>>>>>>>>>
> >> >>>>>>>>>> it
> >> >>>>>>>>>>
> >> >>>>>>>>>> seems a little weird to have a separate BoundedDataStream
> >> >>>>>>>>>>
> >> >>>>>>>>>> interface.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Am I understand it correctly?
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Jiangjie (Becket) Qin
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> >> >>>>>>>>>>
> >> >>>>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi all,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Really well written proposal and very important one. I must
> >> >> admit
> >> >>>>>>>>>>
> >> >>>>>>>>>> I
> >> >>>>>>>>>>
> >> >>>>>>>>>> have
> >> >>>>>>>>>>
> >> >>>>>>>>>> not understood all the intricacies of it yet.
> >> >>>>>>>>>>
> >> >>>>>>>>>> One question I have though is about where does the
> information
> >> >>>>>>>>>>
> >> >>>>>>>>>> about
> >> >>>>>>>>>>
> >> >>>>>>>>>> boundedness come from. I think in most cases it is a property
> >> of
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> source. As you described it might be e.g. end offset, a flag
> >> >>>>>>>>>>
> >> >>>>>>>>>> should
> >> >>>>>>>>>>
> >> >>>>>>>>>> it
> >> >>>>>>>>>>
> >> >>>>>>>>>> monitor new splits etc. I think it would be a really nice use
> >> >> case
> >> >>>>>>>>>>
> >> >>>>>>>>>> to
> >> >>>>>>>>>>
> >> >>>>>>>>>> be
> >> >>>>>>>>>>
> >> >>>>>>>>>> able to say:
> >> >>>>>>>>>>
> >> >>>>>>>>>> new KafkaSource().readUntil(long timestamp),
> >> >>>>>>>>>>
> >> >>>>>>>>>> which could work as an "end offset". Moreover I think all
> >> >> Bounded
> >> >>>>>>>>>>
> >> >>>>>>>>>> sources
> >> >>>>>>>>>>
> >> >>>>>>>>>> support continuous mode, but no intrinsically continuous
> source
> >> >>>>>>>>>>
> >> >>>>>>>>>> support
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> Bounded mode. If I understood the proposal correctly it
> suggest
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> boundedness sort of "comes" from the outside of the source,
> >> from
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> invokation of either boundedStream or continousSource.
> >> >>>>>>>>>>
> >> >>>>>>>>>> I am wondering if it would make sense to actually change the
> >> >>>>>>>>>>
> >> >>>>>>>>>> method
> >> >>>>>>>>>>
> >> >>>>>>>>>> boolean Source#supportsBoundedness(Boundedness)
> >> >>>>>>>>>>
> >> >>>>>>>>>> to
> >> >>>>>>>>>>
> >> >>>>>>>>>> Boundedness Source#getBoundedness().
> >> >>>>>>>>>>
> >> >>>>>>>>>> As for the methods #boundedSource, #continousSource, assuming
> >> >> the
> >> >>>>>>>>>> boundedness is property of the source they do not affect how
> >> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> enumerator
> >> >>>>>>>>>>
> >> >>>>>>>>>> works, but mostly how the dag is scheduled, right? I am not
> >> >>>>>>>>>>
> >> >>>>>>>>>> against
> >> >>>>>>>>>>
> >> >>>>>>>>>> those
> >> >>>>>>>>>>
> >> >>>>>>>>>> methods, but I think it is a very specific use case to
> actually
> >> >>>>>>>>>>
> >> >>>>>>>>>> override
> >> >>>>>>>>>>
> >> >>>>>>>>>> the property of the source. In general I would expect users
> to
> >> >>>>>>>>>>
> >> >>>>>>>>>> only
> >> >>>>>>>>>>
> >> >>>>>>>>>> call
> >> >>>>>>>>>>
> >> >>>>>>>>>> env.source(theSource), where the source tells if it is
> bounded
> >> >> or
> >> >>>>>>>>>>
> >> >>>>>>>>>> not. I
> >> >>>>>>>>>>
> >> >>>>>>>>>> would suggest considering following set of methods:
> >> >>>>>>>>>>
> >> >>>>>>>>>> // boundedness depends of source property, imo this should
> >> >> always
> >> >>>>>>>>>>
> >> >>>>>>>>>> be
> >> >>>>>>>>>>
> >> >>>>>>>>>> preferred
> >> >>>>>>>>>>
> >> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> // always continous execution, whether bounded or unbounded
> >> >> source
> >> >>>>>>>>>>
> >> >>>>>>>>>> DataStream<MyType> boundedStream =
> >> >> env.continousSource(theSource);
> >> >>>>>>>>>>
> >> >>>>>>>>>> // imo this would make sense if the BoundedDataStream
> provides
> >> >>>>>>>>>>
> >> >>>>>>>>>> additional features unavailable for continous mode
> >> >>>>>>>>>>
> >> >>>>>>>>>> BoundedDataStream<MyType> batch =
> env.boundedSource(theSource);
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Best,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Dawid
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks, Becket, for updating this.
> >> >>>>>>>>>>
> >> >>>>>>>>>> I agree with moving the aspects you mentioned into separate
> >> >> FLIPs
> >> >>>>>>>>>>
> >> >>>>>>>>>> -
> >> >>>>>>>>>>
> >> >>>>>>>>>> this
> >> >>>>>>>>>>
> >> >>>>>>>>>> one way becoming unwieldy in size.
> >> >>>>>>>>>>
> >> >>>>>>>>>> +1 to the FLIP in its current state. Its a very detailed
> >> >> write-up,
> >> >>>>>>>>>>
> >> >>>>>>>>>> nicely
> >> >>>>>>>>>>
> >> >>>>>>>>>> done!
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <
> >> becket.qin@gmail.com
> >> >>>>>>>> <ma...@gmail.com>> <becket.qin@gmail.com <mailto:
> >> >>>>>>>> becket.qin@gmail.com>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> <
> >> >>>>>>>>>>
> >> >>>>>>>>>> becket.qin@gmail.com <ma...@gmail.com>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi all,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Sorry for the long belated update. I have updated FLIP-27
> wiki
> >> >>>>>>>>>>
> >> >>>>>>>>>> page
> >> >>>>>>>>>>
> >> >>>>>>>>>> with
> >> >>>>>>>>>>
> >> >>>>>>>>>> the latest proposals. Some noticeable changes include:
> >> >>>>>>>>>> 1. A new generic communication mechanism between
> >> SplitEnumerator
> >> >>>>>>>>>>
> >> >>>>>>>>>> and
> >> >>>>>>>>>>
> >> >>>>>>>>>> SourceReader.
> >> >>>>>>>>>> 2. Some detail API method signature changes.
> >> >>>>>>>>>>
> >> >>>>>>>>>> We left a few things out of this FLIP and will address them
> in
> >> >>>>>>>>>>
> >> >>>>>>>>>> separate
> >> >>>>>>>>>>
> >> >>>>>>>>>> FLIPs. Including:
> >> >>>>>>>>>> 1. Per split event time.
> >> >>>>>>>>>> 2. Event time alignment.
> >> >>>>>>>>>> 3. Fine grained failover for SplitEnumerator failure.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Please let us know if you have any question.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Jiangjie (Becket) Qin
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <
> sewen@apache.org
> >> >>>>>>> <mailto:
> >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:sewen@apache.org
> >>
> >> <
> >> >>>>>>>>>>
> >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi  Łukasz!
> >> >>>>>>>>>>
> >> >>>>>>>>>> Becket and me are working hard on figuring out the last
> details
> >> >>>>>>>>>>
> >> >>>>>>>>>> and
> >> >>>>>>>>>>
> >> >>>>>>>>>> implementing the first PoC. We would update the FLIP
> hopefully
> >> >>>>>>>>>>
> >> >>>>>>>>>> next
> >> >>>>>>>>>>
> >> >>>>>>>>>> week.
> >> >>>>>>>>>>
> >> >>>>>>>>>> There is a fair chance that a first version of this will be
> in
> >> >>>>>>>>>>
> >> >>>>>>>>>> 1.10,
> >> >>>>>>>>>>
> >> >>>>>>>>>> but
> >> >>>>>>>>>>
> >> >>>>>>>>>> I
> >> >>>>>>>>>>
> >> >>>>>>>>>> think it will take another release to battle test it and
> >> migrate
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> connectors.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Best,
> >> >>>>>>>>>> Stephan
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <
> >> >> ljd@touk.pl
> >> >>>>>>>> <ma...@touk.pl>
> >> >>>>>>>>>>
> >> >>>>>>>>>> <
> >> >>>>>>>>>>
> >> >>>>>>>>>> ljd@touk.pl <ma...@touk.pl>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi,
> >> >>>>>>>>>>
> >> >>>>>>>>>> This proposal looks very promising for us. Do you have any
> >> plans
> >> >>>>>>>>>>
> >> >>>>>>>>>> in
> >> >>>>>>>>>>
> >> >>>>>>>>>> which
> >> >>>>>>>>>>
> >> >>>>>>>>>> Flink release it is going to be released? We are thinking on
> >> >>>>>>>>>>
> >> >>>>>>>>>> using a
> >> >>>>>>>>>>
> >> >>>>>>>>>> Data
> >> >>>>>>>>>>
> >> >>>>>>>>>> Set API for our future use cases but on the other hand Data
> Set
> >> >>>>>>>>>>
> >> >>>>>>>>>> API
> >> >>>>>>>>>>
> >> >>>>>>>>>> is
> >> >>>>>>>>>>
> >> >>>>>>>>>> going to be deprecated so using proposed bounded data streams
> >> >>>>>>>>>>
> >> >>>>>>>>>> solution
> >> >>>>>>>>>>
> >> >>>>>>>>>> could be more viable in the long term.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks,
> >> >>>>>>>>>> Łukasz
> >> >>>>>>>>>>
> >> >>>>>>>>>> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com
> >> >>>>>> <mailto:
> >> >>>>>>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
> >> >>>>>>>> thomas.weise@gmail.com>> <
> >> >>>>>>>>>>
> >> >>>>>>>>>> thomas.weise@gmail.com <ma...@gmail.com>>
> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks for putting together this proposal!
> >> >>>>>>>>>>
> >> >>>>>>>>>> I see that the "Per Split Event Time" and "Event Time
> >> Alignment"
> >> >>>>>>>>>>
> >> >>>>>>>>>> sections
> >> >>>>>>>>>>
> >> >>>>>>>>>> are still TBD.
> >> >>>>>>>>>>
> >> >>>>>>>>>> It would probably be good to flesh those out a bit before
> >> >>>>>>>>>>
> >> >>>>>>>>>> proceeding
> >> >>>>>>>>>>
> >> >>>>>>>>>> too
> >> >>>>>>>>>>
> >> >>>>>>>>>> far
> >> >>>>>>>>>>
> >> >>>>>>>>>> as the event time alignment will probably influence the
> >> >>>>>>>>>>
> >> >>>>>>>>>> interaction
> >> >>>>>>>>>>
> >> >>>>>>>>>> with
> >> >>>>>>>>>>
> >> >>>>>>>>>> the split reader, specifically ReaderStatus
> >> >>>>>>>>>>
> >> >>>>>>>>>> emitNext(SourceOutput<E>
> >> >>>>>>>>>>
> >> >>>>>>>>>> output).
> >> >>>>>>>>>>
> >> >>>>>>>>>> We currently have only one implementation for event time
> >> >> alignment
> >> >>>>>>>>>>
> >> >>>>>>>>>> in
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> Kinesis consumer. The synchronization in that case takes
> place
> >> >> as
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> last
> >> >>>>>>>>>>
> >> >>>>>>>>>> step before records are emitted downstream (RecordEmitter).
> >> With
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> currently proposed interfaces, the equivalent can be
> >> implemented
> >> >>>>>>>>>>
> >> >>>>>>>>>> in
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> reader loop, although note that in the Kinesis consumer the
> per
> >> >>>>>>>>>>
> >> >>>>>>>>>> shard
> >> >>>>>>>>>>
> >> >>>>>>>>>> threads push records.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Synchronization has not been implemented for the Kafka
> consumer
> >> >>>>>>>>>>
> >> >>>>>>>>>> yet.
> >> >>>>>>>>>>
> >> >>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
> >> >>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675>
> >> >>>>>>>>>>
> >> >>>>>>>>>> When I looked at it, I realized that the implementation will
> >> >> look
> >> >>>>>>>>>>
> >> >>>>>>>>>> quite
> >> >>>>>>>>>>
> >> >>>>>>>>>> different
> >> >>>>>>>>>> from Kinesis because it needs to take place in the pull part,
> >> >>>>>>>>>>
> >> >>>>>>>>>> where
> >> >>>>>>>>>>
> >> >>>>>>>>>> records
> >> >>>>>>>>>>
> >> >>>>>>>>>> are taken from the Kafka client. Due to the multiplexing it
> >> >> cannot
> >> >>>>>>>>>>
> >> >>>>>>>>>> be
> >> >>>>>>>>>>
> >> >>>>>>>>>> done
> >> >>>>>>>>>>
> >> >>>>>>>>>> by blocking the split thread like it currently works for
> >> >> Kinesis.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Reading
> >> >>>>>>>>>>
> >> >>>>>>>>>> from individual Kafka partitions needs to be controlled via
> >> >>>>>>>>>>
> >> >>>>>>>>>> pause/resume
> >> >>>>>>>>>>
> >> >>>>>>>>>> on the Kafka client.
> >> >>>>>>>>>>
> >> >>>>>>>>>> To take on that responsibility the split thread would need to
> >> be
> >> >>>>>>>>>>
> >> >>>>>>>>>> aware
> >> >>>>>>>>>>
> >> >>>>>>>>>> of
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>> watermarks or at least whether it should or should not
> continue
> >> >> to
> >> >>>>>>>>>>
> >> >>>>>>>>>> consume
> >> >>>>>>>>>>
> >> >>>>>>>>>> a given split and this may require a different SourceReader
> or
> >> >>>>>>>>>>
> >> >>>>>>>>>> SourceOutput
> >> >>>>>>>>>>
> >> >>>>>>>>>> interface.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks,
> >> >>>>>>>>>> Thomas
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com
> >> >>>>>> <mailto:
> >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> >> mmyy1110@gmail.com
> >> >>>>>
> >> >>>>>> <
> >> >>>>>>>>>>
> >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi Stephan,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thank you for feedback!
> >> >>>>>>>>>> Will take a look at your branch before public discussing.
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <
> >> sewen@apache.org
> >> >>>>>>>> <ma...@apache.org>> <sewen@apache.org <mailto:
> >> >>> sewen@apache.org
> >> >>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> <
> >> >>>>>>>>>>
> >> >>>>>>>>>> sewen@apache.org <ma...@apache.org>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi Biao!
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks for reviving this. I would like to join this
> discussion,
> >> >>>>>>>>>>
> >> >>>>>>>>>> but
> >> >>>>>>>>>>
> >> >>>>>>>>>> am
> >> >>>>>>>>>>
> >> >>>>>>>>>> quite occupied with the 1.9 release, so can we maybe pause
> this
> >> >>>>>>>>>>
> >> >>>>>>>>>> discussion
> >> >>>>>>>>>>
> >> >>>>>>>>>> for a week or so?
> >> >>>>>>>>>>
> >> >>>>>>>>>> In the meantime I can share some suggestion based on prior
> >> >>>>>>>>>>
> >> >>>>>>>>>> experiments:
> >> >>>>>>>>>>
> >> >>>>>>>>>> How to do watermarks / timestamp extractors in a simpler and
> >> >> more
> >> >>>>>>>>>>
> >> >>>>>>>>>> flexible
> >> >>>>>>>>>>
> >> >>>>>>>>>> way. I think that part is quite promising should be part of
> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> new
> >> >>>>>>>>>>
> >> >>>>>>>>>> source
> >> >>>>>>>>>>
> >> >>>>>>>>>> interface.
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> >> >>>>>>>> <
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> >> >>>>>>>> <
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Some experiments on how to build the source reader and its
> >> >>>>>>>>>>
> >> >>>>>>>>>> library
> >> >>>>>>>>>>
> >> >>>>>>>>>> for
> >> >>>>>>>>>>
> >> >>>>>>>>>> common threading/split patterns:
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> >> >>>>>>>> <
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Best,
> >> >>>>>>>>>> Stephan
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <
> mmyy1110@gmail.com
> >> >>>>>>> <mailto:
> >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> >> mmyy1110@gmail.com
> >> >>>>>
> >> >>>>>> <
> >> >>>>>>>>>>
> >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi devs,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Since 1.9 is nearly released, I think we could get back to
> >> >>>>>>>>>>
> >> >>>>>>>>>> FLIP-27.
> >> >>>>>>>>>>
> >> >>>>>>>>>> I
> >> >>>>>>>>>>
> >> >>>>>>>>>> believe it should be included in 1.10.
> >> >>>>>>>>>>
> >> >>>>>>>>>> There are so many things mentioned in document of FLIP-27.
> [1]
> >> I
> >> >>>>>>>>>>
> >> >>>>>>>>>> think
> >> >>>>>>>>>>
> >> >>>>>>>>>> we'd better discuss them separately. However the wiki is not
> a
> >> >>>>>>>>>>
> >> >>>>>>>>>> good
> >> >>>>>>>>>>
> >> >>>>>>>>>> place
> >> >>>>>>>>>>
> >> >>>>>>>>>> to discuss. I wrote google doc about SplitReader API which
> >> >>>>>>>>>>
> >> >>>>>>>>>> misses
> >> >>>>>>>>>>
> >> >>>>>>>>>> some
> >> >>>>>>>>>>
> >> >>>>>>>>>> details in the document. [2]
> >> >>>>>>>>>>
> >> >>>>>>>>>> 1.
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >> >>>>>>>> <
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> 2.
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> >> >>>>>>>> <
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com
> >> >>>>>> <mailto:
> >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> >> mmyy1110@gmail.com
> >> >>>>>
> >> >>>>>> <
> >> >>>>>>>>>>
> >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi Steven,
> >> >>>>>>>>>> Thank you for the feedback. Please take a look at the
> document
> >> >>>>>>>>>>
> >> >>>>>>>>>> FLIP-27
> >> >>>>>>>>>>
> >> >>>>>>>>>> <
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >> >>>>>>>> <
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> which
> >> >>>>>>>>>>
> >> >>>>>>>>>> is updated recently. A lot of details of enumerator were
> added
> >> >>>>>>>>>>
> >> >>>>>>>>>> in
> >> >>>>>>>>>>
> >> >>>>>>>>>> this
> >> >>>>>>>>>>
> >> >>>>>>>>>> document. I think it would help.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Steven Wu <stevenz3wu@gmail.com <mailto:stevenz3wu@gmail.com
> >>
> >> >> <
> >> >>>>>>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
> >> >>>>>>> stevenz3wu@gmail.com
> >> >>>>>>>> <ma...@gmail.com>> <stevenz3wu@gmail.com <mailto:
> >> >>>>>>>> stevenz3wu@gmail.com>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> 于2019年3月28日周四
> >> >>>>>>>>>>
> >> >>>>>>>>>> 下午12:52写道:
> >> >>>>>>>>>>
> >> >>>>>>>>>> This proposal mentioned that SplitEnumerator might run on the
> >> >>>>>>>>>> JobManager or
> >> >>>>>>>>>> in a single task on a TaskManager.
> >> >>>>>>>>>>
> >> >>>>>>>>>> if enumerator is a single task on a taskmanager, then the job
> >> >>>>>>>>>>
> >> >>>>>>>>>> DAG
> >> >>>>>>>>>>
> >> >>>>>>>>>> can
> >> >>>>>>>>>>
> >> >>>>>>>>>> never
> >> >>>>>>>>>> been embarrassingly parallel anymore. That will nullify the
> >> >>>>>>>>>>
> >> >>>>>>>>>> leverage
> >> >>>>>>>>>>
> >> >>>>>>>>>> of
> >> >>>>>>>>>>
> >> >>>>>>>>>> fine-grained recovery for embarrassingly parallel jobs.
> >> >>>>>>>>>>
> >> >>>>>>>>>> It's not clear to me what's the implication of running
> >> >>>>>>>>>>
> >> >>>>>>>>>> enumerator
> >> >>>>>>>>>>
> >> >>>>>>>>>> on
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> jobmanager. So I will leave that out for now.
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com
> >> >>>>>> <mailto:
> >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> >> mmyy1110@gmail.com
> >> >>>>>
> >> >>>>>> <
> >> >>>>>>>>>>
> >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi Stephan & Piotrek,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thank you for feedback.
> >> >>>>>>>>>>
> >> >>>>>>>>>> It seems that there are a lot of things to do in community.
> >> >>>>>>>>>>
> >> >>>>>>>>>> I
> >> >>>>>>>>>>
> >> >>>>>>>>>> am
> >> >>>>>>>>>>
> >> >>>>>>>>>> just
> >> >>>>>>>>>>
> >> >>>>>>>>>> afraid that this discussion may be forgotten since there so
> >> >>>>>>>>>>
> >> >>>>>>>>>> many
> >> >>>>>>>>>>
> >> >>>>>>>>>> proposals
> >> >>>>>>>>>>
> >> >>>>>>>>>> recently.
> >> >>>>>>>>>> Anyway, wish to see the split topics soon :)
> >> >>>>>>>>>>
> >> >>>>>>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:
> >> >>> piotr@da-platform.com
> >> >>>>>>>>
> >> >>>>>>> <
> >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
> >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
> >> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> 于2019年1月24日周四
> >> >>>>>>>>>>
> >> >>>>>>>>>> 下午8:21写道:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi Biao!
> >> >>>>>>>>>>
> >> >>>>>>>>>> This discussion was stalled because of preparations for
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> open
> >> >>>>>>>>>>
> >> >>>>>>>>>> sourcing
> >> >>>>>>>>>>
> >> >>>>>>>>>> & merging Blink. I think before creating the tickets we
> >> >>>>>>>>>>
> >> >>>>>>>>>> should
> >> >>>>>>>>>>
> >> >>>>>>>>>> split this
> >> >>>>>>>>>>
> >> >>>>>>>>>> discussion into topics/areas outlined by Stephan and
> >> >>>>>>>>>>
> >> >>>>>>>>>> create
> >> >>>>>>>>>>
> >> >>>>>>>>>> Flips
> >> >>>>>>>>>>
> >> >>>>>>>>>> for
> >> >>>>>>>>>>
> >> >>>>>>>>>> that.
> >> >>>>>>>>>>
> >> >>>>>>>>>> I think there is no chance for this to be completed in
> >> >>>>>>>>>>
> >> >>>>>>>>>> couple
> >> >>>>>>>>>>
> >> >>>>>>>>>> of
> >> >>>>>>>>>>
> >> >>>>>>>>>> remaining
> >> >>>>>>>>>>
> >> >>>>>>>>>> weeks/1 month before 1.8 feature freeze, however it would
> >> >>>>>>>>>>
> >> >>>>>>>>>> be
> >> >>>>>>>>>>
> >> >>>>>>>>>> good
> >> >>>>>>>>>>
> >> >>>>>>>>>> to aim
> >> >>>>>>>>>>
> >> >>>>>>>>>> with those changes for 1.9.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Piotrek
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com
> >> <mailto:
> >> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> >> mmyy1110@gmail.com
> >> >>>>>
> >> >>>>>> <
> >> >>>>>>>>>>
> >> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi community,
> >> >>>>>>>>>> The summary of Stephan makes a lot sense to me. It is
> >> >>>>>>>>>>
> >> >>>>>>>>>> much
> >> >>>>>>>>>>
> >> >>>>>>>>>> clearer
> >> >>>>>>>>>>
> >> >>>>>>>>>> indeed
> >> >>>>>>>>>>
> >> >>>>>>>>>> after splitting the complex topic into small ones.
> >> >>>>>>>>>> I was wondering is there any detail plan for next step?
> >> >>>>>>>>>>
> >> >>>>>>>>>> If
> >> >>>>>>>>>>
> >> >>>>>>>>>> not,
> >> >>>>>>>>>>
> >> >>>>>>>>>> I
> >> >>>>>>>>>>
> >> >>>>>>>>>> would
> >> >>>>>>>>>>
> >> >>>>>>>>>> like to push this thing forward by creating some JIRA
> >> >>>>>>>>>>
> >> >>>>>>>>>> issues.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Another question is that should version 1.8 include
> >> >>>>>>>>>>
> >> >>>>>>>>>> these
> >> >>>>>>>>>>
> >> >>>>>>>>>> features?
> >> >>>>>>>>>>
> >> >>>>>>>>>> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <
> >> >>>>>>>> sewen@apache.org <ma...@apache.org>> <sewen@apache.org
> >> >>>> <mailto:
> >> >>>>>>>> sewen@apache.org>> <sewen@apache.org <mailto:sewen@apache.org
> >>
> >> >>>>>>>> 于2018年12月1日周六
> >> >>>>>>>>>>
> >> >>>>>>>>>> 上午4:20写道:
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks everyone for the lively discussion. Let me try
> >> >>>>>>>>>>
> >> >>>>>>>>>> to
> >> >>>>>>>>>>
> >> >>>>>>>>>> summarize
> >> >>>>>>>>>>
> >> >>>>>>>>>> where I
> >> >>>>>>>>>>
> >> >>>>>>>>>> see convergence in the discussion and open issues.
> >> >>>>>>>>>> I'll try to group this by design aspect of the source.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Please
> >> >>>>>>>>>>
> >> >>>>>>>>>> let me
> >> >>>>>>>>>>
> >> >>>>>>>>>> know
> >> >>>>>>>>>>
> >> >>>>>>>>>> if I got things wrong or missed something crucial here.
> >> >>>>>>>>>>
> >> >>>>>>>>>> For issues 1-3, if the below reflects the state of the
> >> >>>>>>>>>>
> >> >>>>>>>>>> discussion, I
> >> >>>>>>>>>>
> >> >>>>>>>>>> would
> >> >>>>>>>>>>
> >> >>>>>>>>>> try and update the FLIP in the next days.
> >> >>>>>>>>>> For the remaining ones we need more discussion.
> >> >>>>>>>>>>
> >> >>>>>>>>>> I would suggest to fork each of these aspects into a
> >> >>>>>>>>>>
> >> >>>>>>>>>> separate
> >> >>>>>>>>>>
> >> >>>>>>>>>> mail
> >> >>>>>>>>>>
> >> >>>>>>>>>> thread,
> >> >>>>>>>>>>
> >> >>>>>>>>>> or will loose sight of the individual aspects.
> >> >>>>>>>>>>
> >> >>>>>>>>>> *(1) Separation of Split Enumerator and Split Reader*
> >> >>>>>>>>>>
> >> >>>>>>>>>> - All seem to agree this is a good thing
> >> >>>>>>>>>> - Split Enumerator could in the end live on JobManager
> >> >>>>>>>>>>
> >> >>>>>>>>>> (and
> >> >>>>>>>>>>
> >> >>>>>>>>>> assign
> >> >>>>>>>>>>
> >> >>>>>>>>>> splits
> >> >>>>>>>>>>
> >> >>>>>>>>>> via RPC) or in a task (and assign splits via data
> >> >>>>>>>>>>
> >> >>>>>>>>>> streams)
> >> >>>>>>>>>>
> >> >>>>>>>>>> - this discussion is orthogonal and should come later,
> >> >>>>>>>>>>
> >> >>>>>>>>>> when
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> interface
> >> >>>>>>>>>>
> >> >>>>>>>>>> is agreed upon.
> >> >>>>>>>>>>
> >> >>>>>>>>>> *(2) Split Readers for one or more splits*
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Discussion seems to agree that we need to support
> >> >>>>>>>>>>
> >> >>>>>>>>>> one
> >> >>>>>>>>>>
> >> >>>>>>>>>> reader
> >> >>>>>>>>>>
> >> >>>>>>>>>> that
> >> >>>>>>>>>>
> >> >>>>>>>>>> possibly handles multiple splits concurrently.
> >> >>>>>>>>>> - The requirement comes from sources where one
> >> >>>>>>>>>>
> >> >>>>>>>>>> poll()-style
> >> >>>>>>>>>>
> >> >>>>>>>>>> call
> >> >>>>>>>>>>
> >> >>>>>>>>>> fetches
> >> >>>>>>>>>>
> >> >>>>>>>>>> data from different splits / partitions
> >> >>>>>>>>>>     --> example sources that require that would be for
> >> >>>>>>>>>>
> >> >>>>>>>>>> example
> >> >>>>>>>>>>
> >> >>>>>>>>>> Kafka,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Pravega, Pulsar
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Could have one split reader per source, or multiple
> >> >>>>>>>>>>
> >> >>>>>>>>>> split
> >> >>>>>>>>>>
> >> >>>>>>>>>> readers
> >> >>>>>>>>>>
> >> >>>>>>>>>> that
> >> >>>>>>>>>>
> >> >>>>>>>>>> share the "poll()" function
> >> >>>>>>>>>> - To not make it too complicated, we can start with
> >> >>>>>>>>>>
> >> >>>>>>>>>> thinking
> >> >>>>>>>>>>
> >> >>>>>>>>>> about
> >> >>>>>>>>>>
> >> >>>>>>>>>> one
> >> >>>>>>>>>>
> >> >>>>>>>>>> split reader for all splits initially and see if that
> >> >>>>>>>>>>
> >> >>>>>>>>>> covers
> >> >>>>>>>>>>
> >> >>>>>>>>>> all
> >> >>>>>>>>>>
> >> >>>>>>>>>> requirements
> >> >>>>>>>>>>
> >> >>>>>>>>>> *(3) Threading model of the Split Reader*
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Most active part of the discussion ;-)
> >> >>>>>>>>>>
> >> >>>>>>>>>> - A non-blocking way for Flink's task code to interact
> >> >>>>>>>>>>
> >> >>>>>>>>>> with
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> source
> >> >>>>>>>>>>
> >> >>>>>>>>>> is
> >> >>>>>>>>>>
> >> >>>>>>>>>> needed in order to a task runtime code based on a
> >> >>>>>>>>>> single-threaded/actor-style task design
> >> >>>>>>>>>>     --> I personally am a big proponent of that, it will
> >> >>>>>>>>>>
> >> >>>>>>>>>> help
> >> >>>>>>>>>>
> >> >>>>>>>>>> with
> >> >>>>>>>>>>
> >> >>>>>>>>>> well-behaved checkpoints, efficiency, and simpler yet
> >> >>>>>>>>>>
> >> >>>>>>>>>> more
> >> >>>>>>>>>>
> >> >>>>>>>>>> robust
> >> >>>>>>>>>>
> >> >>>>>>>>>> runtime
> >> >>>>>>>>>>
> >> >>>>>>>>>> code
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Users care about simple abstraction, so as a
> >> >>>>>>>>>>
> >> >>>>>>>>>> subclass
> >> >>>>>>>>>>
> >> >>>>>>>>>> of
> >> >>>>>>>>>>
> >> >>>>>>>>>> SplitReader
> >> >>>>>>>>>>
> >> >>>>>>>>>> (non-blocking / async) we need to have a
> >> >>>>>>>>>>
> >> >>>>>>>>>> BlockingSplitReader
> >> >>>>>>>>>>
> >> >>>>>>>>>> which
> >> >>>>>>>>>>
> >> >>>>>>>>>> will
> >> >>>>>>>>>>
> >> >>>>>>>>>> form the basis of most source implementations.
> >> >>>>>>>>>>
> >> >>>>>>>>>> BlockingSplitReader
> >> >>>>>>>>>>
> >> >>>>>>>>>> lets
> >> >>>>>>>>>>
> >> >>>>>>>>>> users do blocking simple poll() calls.
> >> >>>>>>>>>> - The BlockingSplitReader would spawn a thread (or
> >> >>>>>>>>>>
> >> >>>>>>>>>> more)
> >> >>>>>>>>>>
> >> >>>>>>>>>> and
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> thread(s) can make blocking calls and hand over data
> >> >>>>>>>>>>
> >> >>>>>>>>>> buffers
> >> >>>>>>>>>>
> >> >>>>>>>>>> via
> >> >>>>>>>>>>
> >> >>>>>>>>>> a
> >> >>>>>>>>>>
> >> >>>>>>>>>> blocking
> >> >>>>>>>>>>
> >> >>>>>>>>>> queue
> >> >>>>>>>>>> - This should allow us to cover both, a fully async
> >> >>>>>>>>>>
> >> >>>>>>>>>> runtime,
> >> >>>>>>>>>>
> >> >>>>>>>>>> and a
> >> >>>>>>>>>>
> >> >>>>>>>>>> simple
> >> >>>>>>>>>>
> >> >>>>>>>>>> blocking interface for users.
> >> >>>>>>>>>> - This is actually very similar to how the Kafka
> >> >>>>>>>>>>
> >> >>>>>>>>>> connectors
> >> >>>>>>>>>>
> >> >>>>>>>>>> work.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Kafka
> >> >>>>>>>>>>
> >> >>>>>>>>>> 9+ with one thread, Kafka 8 with multiple threads
> >> >>>>>>>>>>
> >> >>>>>>>>>> - On the base SplitReader (the async one), the
> >> >>>>>>>>>>
> >> >>>>>>>>>> non-blocking
> >> >>>>>>>>>>
> >> >>>>>>>>>> method
> >> >>>>>>>>>>
> >> >>>>>>>>>> that
> >> >>>>>>>>>>
> >> >>>>>>>>>> gets the next chunk of data would signal data
> >> >>>>>>>>>>
> >> >>>>>>>>>> availability
> >> >>>>>>>>>>
> >> >>>>>>>>>> via
> >> >>>>>>>>>>
> >> >>>>>>>>>> a
> >> >>>>>>>>>>
> >> >>>>>>>>>> CompletableFuture, because that gives the best
> >> >>>>>>>>>>
> >> >>>>>>>>>> flexibility
> >> >>>>>>>>>>
> >> >>>>>>>>>> (can
> >> >>>>>>>>>>
> >> >>>>>>>>>> await
> >> >>>>>>>>>>
> >> >>>>>>>>>> completion or register notification handlers).
> >> >>>>>>>>>> - The source task would register a "thenHandle()" (or
> >> >>>>>>>>>>
> >> >>>>>>>>>> similar)
> >> >>>>>>>>>>
> >> >>>>>>>>>> on the
> >> >>>>>>>>>>
> >> >>>>>>>>>> future to put a "take next data" task into the
> >> >>>>>>>>>>
> >> >>>>>>>>>> actor-style
> >> >>>>>>>>>>
> >> >>>>>>>>>> mailbox
> >> >>>>>>>>>>
> >> >>>>>>>>>> *(4) Split Enumeration and Assignment*
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Splits may be generated lazily, both in cases where
> >> >>>>>>>>>>
> >> >>>>>>>>>> there
> >> >>>>>>>>>>
> >> >>>>>>>>>> is a
> >> >>>>>>>>>>
> >> >>>>>>>>>> limited
> >> >>>>>>>>>>
> >> >>>>>>>>>> number of splits (but very many), or splits are
> >> >>>>>>>>>>
> >> >>>>>>>>>> discovered
> >> >>>>>>>>>>
> >> >>>>>>>>>> over
> >> >>>>>>>>>>
> >> >>>>>>>>>> time
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Assignment should also be lazy, to get better load
> >> >>>>>>>>>>
> >> >>>>>>>>>> balancing
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Assignment needs support locality preferences
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Possible design based on discussion so far:
> >> >>>>>>>>>>
> >> >>>>>>>>>>     --> SplitReader has a method "addSplits(SplitT...)"
> >> >>>>>>>>>>
> >> >>>>>>>>>> to
> >> >>>>>>>>>>
> >> >>>>>>>>>> add
> >> >>>>>>>>>>
> >> >>>>>>>>>> one or
> >> >>>>>>>>>>
> >> >>>>>>>>>> more
> >> >>>>>>>>>>
> >> >>>>>>>>>> splits. Some split readers might assume they have only
> >> >>>>>>>>>>
> >> >>>>>>>>>> one
> >> >>>>>>>>>>
> >> >>>>>>>>>> split
> >> >>>>>>>>>>
> >> >>>>>>>>>> ever,
> >> >>>>>>>>>>
> >> >>>>>>>>>> concurrently, others assume multiple splits. (Note:
> >> >>>>>>>>>>
> >> >>>>>>>>>> idea
> >> >>>>>>>>>>
> >> >>>>>>>>>> behind
> >> >>>>>>>>>>
> >> >>>>>>>>>> being
> >> >>>>>>>>>>
> >> >>>>>>>>>> able
> >> >>>>>>>>>>
> >> >>>>>>>>>> to add multiple splits at the same time is to ease
> >> >>>>>>>>>>
> >> >>>>>>>>>> startup
> >> >>>>>>>>>>
> >> >>>>>>>>>> where
> >> >>>>>>>>>>
> >> >>>>>>>>>> multiple
> >> >>>>>>>>>>
> >> >>>>>>>>>> splits may be assigned instantly.)
> >> >>>>>>>>>>     --> SplitReader has a context object on which it can
> >> >>>>>>>>>>
> >> >>>>>>>>>> call
> >> >>>>>>>>>>
> >> >>>>>>>>>> indicate
> >> >>>>>>>>>>
> >> >>>>>>>>>> when
> >> >>>>>>>>>>
> >> >>>>>>>>>> splits are completed. The enumerator gets that
> >> >>>>>>>>>>
> >> >>>>>>>>>> notification and
> >> >>>>>>>>>>
> >> >>>>>>>>>> can
> >> >>>>>>>>>>
> >> >>>>>>>>>> use
> >> >>>>>>>>>>
> >> >>>>>>>>>> to
> >> >>>>>>>>>>
> >> >>>>>>>>>> decide when to assign new splits. This should help both
> >> >>>>>>>>>>
> >> >>>>>>>>>> in
> >> >>>>>>>>>>
> >> >>>>>>>>>> cases
> >> >>>>>>>>>>
> >> >>>>>>>>>> of
> >> >>>>>>>>>>
> >> >>>>>>>>>> sources
> >> >>>>>>>>>>
> >> >>>>>>>>>> that take splits lazily (file readers) and in case the
> >> >>>>>>>>>>
> >> >>>>>>>>>> source
> >> >>>>>>>>>>
> >> >>>>>>>>>> needs to
> >> >>>>>>>>>>
> >> >>>>>>>>>> preserve a partial order between splits (Kinesis,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Pravega,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Pulsar may
> >> >>>>>>>>>>
> >> >>>>>>>>>> need
> >> >>>>>>>>>>
> >> >>>>>>>>>> that).
> >> >>>>>>>>>>     --> SplitEnumerator gets notification when
> >> >>>>>>>>>>
> >> >>>>>>>>>> SplitReaders
> >> >>>>>>>>>>
> >> >>>>>>>>>> start
> >> >>>>>>>>>>
> >> >>>>>>>>>> and
> >> >>>>>>>>>>
> >> >>>>>>>>>> when
> >> >>>>>>>>>>
> >> >>>>>>>>>> they finish splits. They can decide at that moment to
> >> >>>>>>>>>>
> >> >>>>>>>>>> push
> >> >>>>>>>>>>
> >> >>>>>>>>>> more
> >> >>>>>>>>>>
> >> >>>>>>>>>> splits
> >> >>>>>>>>>>
> >> >>>>>>>>>> to
> >> >>>>>>>>>>
> >> >>>>>>>>>> that reader
> >> >>>>>>>>>>     --> The SplitEnumerator should probably be aware of
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> source
> >> >>>>>>>>>>
> >> >>>>>>>>>> parallelism, to build its initial distribution.
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Open question: Should the source expose something
> >> >>>>>>>>>>
> >> >>>>>>>>>> like
> >> >>>>>>>>>>
> >> >>>>>>>>>> "host
> >> >>>>>>>>>>
> >> >>>>>>>>>> preferences", so that yarn/mesos/k8s can take this into
> >> >>>>>>>>>>
> >> >>>>>>>>>> account
> >> >>>>>>>>>>
> >> >>>>>>>>>> when
> >> >>>>>>>>>>
> >> >>>>>>>>>> selecting a node to start a TM on?
> >> >>>>>>>>>>
> >> >>>>>>>>>> *(5) Watermarks and event time alignment*
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Watermark generation, as well as idleness, needs to
> >> >>>>>>>>>>
> >> >>>>>>>>>> be
> >> >>>>>>>>>>
> >> >>>>>>>>>> per
> >> >>>>>>>>>>
> >> >>>>>>>>>> split
> >> >>>>>>>>>>
> >> >>>>>>>>>> (like
> >> >>>>>>>>>>
> >> >>>>>>>>>> currently in the Kafka Source, per partition)
> >> >>>>>>>>>> - It is desirable to support optional
> >> >>>>>>>>>>
> >> >>>>>>>>>> event-time-alignment,
> >> >>>>>>>>>>
> >> >>>>>>>>>> meaning
> >> >>>>>>>>>>
> >> >>>>>>>>>> that
> >> >>>>>>>>>>
> >> >>>>>>>>>> splits that are ahead are back-pressured or temporarily
> >> >>>>>>>>>>
> >> >>>>>>>>>> unsubscribed
> >> >>>>>>>>>>
> >> >>>>>>>>>> - I think i would be desirable to encapsulate
> >> >>>>>>>>>>
> >> >>>>>>>>>> watermark
> >> >>>>>>>>>>
> >> >>>>>>>>>> generation
> >> >>>>>>>>>>
> >> >>>>>>>>>> logic
> >> >>>>>>>>>>
> >> >>>>>>>>>> in watermark generators, for a separation of concerns.
> >> >>>>>>>>>>
> >> >>>>>>>>>> The
> >> >>>>>>>>>>
> >> >>>>>>>>>> watermark
> >> >>>>>>>>>>
> >> >>>>>>>>>> generators should run per split.
> >> >>>>>>>>>> - Using watermark generators would also help with
> >> >>>>>>>>>>
> >> >>>>>>>>>> another
> >> >>>>>>>>>>
> >> >>>>>>>>>> problem of
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> suggested interface, namely supporting non-periodic
> >> >>>>>>>>>>
> >> >>>>>>>>>> watermarks
> >> >>>>>>>>>>
> >> >>>>>>>>>> efficiently.
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Need a way to "dispatch" next record to different
> >> >>>>>>>>>>
> >> >>>>>>>>>> watermark
> >> >>>>>>>>>>
> >> >>>>>>>>>> generators
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Need a way to tell SplitReader to "suspend" a split
> >> >>>>>>>>>>
> >> >>>>>>>>>> until a
> >> >>>>>>>>>>
> >> >>>>>>>>>> certain
> >> >>>>>>>>>>
> >> >>>>>>>>>> watermark is reached (event time backpressure)
> >> >>>>>>>>>> - This would in fact be not needed (and thus simpler)
> >> >>>>>>>>>>
> >> >>>>>>>>>> if
> >> >>>>>>>>>>
> >> >>>>>>>>>> we
> >> >>>>>>>>>>
> >> >>>>>>>>>> had
> >> >>>>>>>>>>
> >> >>>>>>>>>> a
> >> >>>>>>>>>>
> >> >>>>>>>>>> SplitReader per split and may be a reason to re-open
> >> >>>>>>>>>>
> >> >>>>>>>>>> that
> >> >>>>>>>>>>
> >> >>>>>>>>>> discussion
> >> >>>>>>>>>>
> >> >>>>>>>>>> *(6) Watermarks across splits and in the Split
> >> >>>>>>>>>>
> >> >>>>>>>>>> Enumerator*
> >> >>>>>>>>>>
> >> >>>>>>>>>> - The split enumerator may need some watermark
> >> >>>>>>>>>>
> >> >>>>>>>>>> awareness,
> >> >>>>>>>>>>
> >> >>>>>>>>>> which
> >> >>>>>>>>>>
> >> >>>>>>>>>> should
> >> >>>>>>>>>>
> >> >>>>>>>>>> be
> >> >>>>>>>>>>
> >> >>>>>>>>>> purely based on split metadata (like create timestamp
> >> >>>>>>>>>>
> >> >>>>>>>>>> of
> >> >>>>>>>>>>
> >> >>>>>>>>>> file
> >> >>>>>>>>>>
> >> >>>>>>>>>> splits)
> >> >>>>>>>>>>
> >> >>>>>>>>>> - If there are still more splits with overlapping
> >> >>>>>>>>>>
> >> >>>>>>>>>> event
> >> >>>>>>>>>>
> >> >>>>>>>>>> time
> >> >>>>>>>>>>
> >> >>>>>>>>>> range
> >> >>>>>>>>>>
> >> >>>>>>>>>> for
> >> >>>>>>>>>>
> >> >>>>>>>>>> a
> >> >>>>>>>>>>
> >> >>>>>>>>>> split reader, then that split reader should not advance
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> watermark
> >> >>>>>>>>>>
> >> >>>>>>>>>> within the split beyond the overlap boundary. Otherwise
> >> >>>>>>>>>>
> >> >>>>>>>>>> future
> >> >>>>>>>>>>
> >> >>>>>>>>>> splits
> >> >>>>>>>>>>
> >> >>>>>>>>>> will
> >> >>>>>>>>>>
> >> >>>>>>>>>> produce late data.
> >> >>>>>>>>>>
> >> >>>>>>>>>> - One way to approach this could be that the split
> >> >>>>>>>>>>
> >> >>>>>>>>>> enumerator
> >> >>>>>>>>>>
> >> >>>>>>>>>> may
> >> >>>>>>>>>>
> >> >>>>>>>>>> send
> >> >>>>>>>>>>
> >> >>>>>>>>>> watermarks to the readers, and the readers cannot emit
> >> >>>>>>>>>>
> >> >>>>>>>>>> watermarks
> >> >>>>>>>>>>
> >> >>>>>>>>>> beyond
> >> >>>>>>>>>>
> >> >>>>>>>>>> that received watermark.
> >> >>>>>>>>>> - Many split enumerators would simply immediately send
> >> >>>>>>>>>>
> >> >>>>>>>>>> Long.MAX
> >> >>>>>>>>>>
> >> >>>>>>>>>> out
> >> >>>>>>>>>>
> >> >>>>>>>>>> and
> >> >>>>>>>>>>
> >> >>>>>>>>>> leave the progress purely to the split readers.
> >> >>>>>>>>>>
> >> >>>>>>>>>> - For event-time alignment / split back pressure, this
> >> >>>>>>>>>>
> >> >>>>>>>>>> begs
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> question
> >> >>>>>>>>>>
> >> >>>>>>>>>> how we can avoid deadlocks that may arise when splits
> >> >>>>>>>>>>
> >> >>>>>>>>>> are
> >> >>>>>>>>>>
> >> >>>>>>>>>> suspended
> >> >>>>>>>>>>
> >> >>>>>>>>>> for
> >> >>>>>>>>>>
> >> >>>>>>>>>> event time back pressure,
> >> >>>>>>>>>>
> >> >>>>>>>>>> *(7) Batch and streaming Unification*
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Functionality wise, the above design should support
> >> >>>>>>>>>>
> >> >>>>>>>>>> both
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Batch often (mostly) does not care about reading "in
> >> >>>>>>>>>>
> >> >>>>>>>>>> order"
> >> >>>>>>>>>>
> >> >>>>>>>>>> and
> >> >>>>>>>>>>
> >> >>>>>>>>>> generating watermarks
> >> >>>>>>>>>>     --> Might use different enumerator logic that is
> >> >>>>>>>>>>
> >> >>>>>>>>>> more
> >> >>>>>>>>>>
> >> >>>>>>>>>> locality
> >> >>>>>>>>>>
> >> >>>>>>>>>> aware
> >> >>>>>>>>>>
> >> >>>>>>>>>> and ignores event time order
> >> >>>>>>>>>>     --> Does not generate watermarks
> >> >>>>>>>>>> - Would be great if bounded sources could be
> >> >>>>>>>>>>
> >> >>>>>>>>>> identified
> >> >>>>>>>>>>
> >> >>>>>>>>>> at
> >> >>>>>>>>>>
> >> >>>>>>>>>> compile
> >> >>>>>>>>>>
> >> >>>>>>>>>> time,
> >> >>>>>>>>>>
> >> >>>>>>>>>> so that "env.addBoundedSource(...)" is type safe and
> >> >>>>>>>>>>
> >> >>>>>>>>>> can
> >> >>>>>>>>>>
> >> >>>>>>>>>> return a
> >> >>>>>>>>>>
> >> >>>>>>>>>> "BoundedDataStream".
> >> >>>>>>>>>> - Possible to defer this discussion until later
> >> >>>>>>>>>>
> >> >>>>>>>>>> *Miscellaneous Comments*
> >> >>>>>>>>>>
> >> >>>>>>>>>> - Should the source have a TypeInformation for the
> >> >>>>>>>>>>
> >> >>>>>>>>>> produced
> >> >>>>>>>>>>
> >> >>>>>>>>>> type,
> >> >>>>>>>>>>
> >> >>>>>>>>>> instead
> >> >>>>>>>>>>
> >> >>>>>>>>>> of a serializer? We need a type information in the
> >> >>>>>>>>>>
> >> >>>>>>>>>> stream
> >> >>>>>>>>>>
> >> >>>>>>>>>> anyways, and
> >> >>>>>>>>>>
> >> >>>>>>>>>> can
> >> >>>>>>>>>>
> >> >>>>>>>>>> derive the serializer from that. Plus, creating the
> >> >>>>>>>>>>
> >> >>>>>>>>>> serializer
> >> >>>>>>>>>>
> >> >>>>>>>>>> should
> >> >>>>>>>>>>
> >> >>>>>>>>>> respect the ExecutionConfig.
> >> >>>>>>>>>>
> >> >>>>>>>>>> - The TypeSerializer interface is very powerful but
> >> >>>>>>>>>>
> >> >>>>>>>>>> also
> >> >>>>>>>>>>
> >> >>>>>>>>>> not
> >> >>>>>>>>>>
> >> >>>>>>>>>> easy to
> >> >>>>>>>>>>
> >> >>>>>>>>>> implement. Its purpose is to handle data super
> >> >>>>>>>>>>
> >> >>>>>>>>>> efficiently,
> >> >>>>>>>>>>
> >> >>>>>>>>>> support
> >> >>>>>>>>>>
> >> >>>>>>>>>> flexible ways of evolution, etc.
> >> >>>>>>>>>> For metadata I would suggest to look at the
> >> >>>>>>>>>>
> >> >>>>>>>>>> SimpleVersionedSerializer
> >> >>>>>>>>>>
> >> >>>>>>>>>> instead, which is used for example for checkpoint
> >> >>>>>>>>>>
> >> >>>>>>>>>> master
> >> >>>>>>>>>>
> >> >>>>>>>>>> hooks,
> >> >>>>>>>>>>
> >> >>>>>>>>>> or for
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> streaming file sink. I think that is is a good match
> >> >>>>>>>>>>
> >> >>>>>>>>>> for
> >> >>>>>>>>>>
> >> >>>>>>>>>> cases
> >> >>>>>>>>>>
> >> >>>>>>>>>> where
> >> >>>>>>>>>>
> >> >>>>>>>>>> we
> >> >>>>>>>>>>
> >> >>>>>>>>>> do
> >> >>>>>>>>>>
> >> >>>>>>>>>> not need more than ser/deser (no copy, etc.) and don't
> >> >>>>>>>>>>
> >> >>>>>>>>>> need to
> >> >>>>>>>>>>
> >> >>>>>>>>>> push
> >> >>>>>>>>>>
> >> >>>>>>>>>> versioning out of the serialization paths for best
> >> >>>>>>>>>>
> >> >>>>>>>>>> performance
> >> >>>>>>>>>>
> >> >>>>>>>>>> (as in
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> TypeSerializer)
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> >> >>>>>>>>>>
> >> >>>>>>>>>> k.kloudas@data-artisans.com>
> >> >>>>>>>>>>
> >> >>>>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> Hi Biao,
> >> >>>>>>>>>>
> >> >>>>>>>>>> Thanks for the answer!
> >> >>>>>>>>>>
> >> >>>>>>>>>> So given the multi-threaded readers, now we have as
> >> >>>>>>>>>>
> >> >>>>>>>>>> open
> >> >>>>>>>>>>
> >> >>>>>>>>>> questions:
> >> >>>>>>>>>>
> >> >>>>>>>>>> 1) How do we let the checkpoints pass through our
> >> >>>>>>>>>>
> >> >>>>>>>>>> multi-threaded
> >> >>>>>>>>>>
> >> >>>>>>>>>> reader
> >> >>>>>>>>>>
> >> >>>>>>>>>> operator?
> >> >>>>>>>>>>
> >> >>>>>>>>>> 2) Do we have separate reader and source operators or
> >> >>>>>>>>>>
> >> >>>>>>>>>> not? In
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> strategy
> >> >>>>>>>>>>
> >> >>>>>>>>>> that has a separate source, the source operator has a
> >> >>>>>>>>>>
> >> >>>>>>>>>> parallelism of
> >> >>>>>>>>>>
> >> >>>>>>>>>> 1
> >> >>>>>>>>>>
> >> >>>>>>>>>> and
> >> >>>>>>>>>>
> >> >>>>>>>>>> is responsible for split recovery only.
> >> >>>>>>>>>>
> >> >>>>>>>>>> For the first one, given also the constraints
> >> >>>>>>>>>>
> >> >>>>>>>>>> (blocking,
> >> >>>>>>>>>>
> >> >>>>>>>>>> finite
> >> >>>>>>>>>>
> >> >>>>>>>>>> queues,
> >> >>>>>>>>>>
> >> >>>>>>>>>> etc), I do not have an answer yet.
> >> >>>>>>>>>>
> >> >>>>>>>>>> For the 2nd, I think that we should go with separate
> >> >>>>>>>>>>
> >> >>>>>>>>>> operators
> >> >>>>>>>>>>
> >> >>>>>>>>>> for
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> source and the readers, for the following reasons:
> >> >>>>>>>>>>
> >> >>>>>>>>>> 1) This is more aligned with a potential future
> >> >>>>>>>>>>
> >> >>>>>>>>>> improvement
> >> >>>>>>>>>>
> >> >>>>>>>>>> where the
> >> >>>>>>>>>>
> >> >>>>>>>>>> split
> >> >>>>>>>>>>
> >> >>>>>>>>>> discovery becomes a responsibility of the JobManager
> >> >>>>>>>>>>
> >> >>>>>>>>>> and
> >> >>>>>>>>>>
> >> >>>>>>>>>> readers are
> >> >>>>>>>>>>
> >> >>>>>>>>>> pooling more work from the JM.
> >> >>>>>>>>>>
> >> >>>>>>>>>> 2) The source is going to be the "single point of
> >> >>>>>>>>>>
> >> >>>>>>>>>> truth".
> >> >>>>>>>>>>
> >> >>>>>>>>>> It
> >> >>>>>>>>>>
> >> >>>>>>>>>> will
> >> >>>>>>>>>>
> >> >>>>>>>>>> know
> >> >>>>>>>>>>
> >> >>>>>>>>>> what
> >> >>>>>>>>>>
> >> >>>>>>>>>> has been processed and what not. If the source and the
> >> >>>>>>>>>>
> >> >>>>>>>>>> readers
> >> >>>>>>>>>>
> >> >>>>>>>>>> are a
> >> >>>>>>>>>>
> >> >>>>>>>>>> single
> >> >>>>>>>>>>
> >> >>>>>>>>>> operator with parallelism > 1, or in general, if the
> >> >>>>>>>>>>
> >> >>>>>>>>>> split
> >> >>>>>>>>>>
> >> >>>>>>>>>> discovery
> >> >>>>>>>>>>
> >> >>>>>>>>>> is
> >> >>>>>>>>>>
> >> >>>>>>>>>> done by each task individually, then:
> >> >>>>>>>>>>    i) we have to have a deterministic scheme for each
> >> >>>>>>>>>>
> >> >>>>>>>>>> reader to
> >> >>>>>>>>>>
> >> >>>>>>>>>> assign
> >> >>>>>>>>>>
> >> >>>>>>>>>> splits to itself (e.g. mod subtaskId). This is not
> >> >>>>>>>>>>
> >> >>>>>>>>>> necessarily
> >> >>>>>>>>>>
> >> >>>>>>>>>> trivial
> >> >>>>>>>>>>
> >> >>>>>>>>>> for
> >> >>>>>>>>>>
> >> >>>>>>>>>> all sources.
> >> >>>>>>>>>>    ii) each reader would have to keep a copy of all its
> >> >>>>>>>>>>
> >> >>>>>>>>>> processed
> >> >>>>>>>>>>
> >> >>>>>>>>>> slpits
> >> >>>>>>>>>>
> >> >>>>>>>>>>    iii) the state has to be a union state with a
> >> >>>>>>>>>>
> >> >>>>>>>>>> non-trivial
> >> >>>>>>>>>>
> >> >>>>>>>>>> merging
> >> >>>>>>>>>>
> >> >>>>>>>>>> logic
> >> >>>>>>>>>>
> >> >>>>>>>>>> in order to support rescaling.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Two additional points that you raised above:
> >> >>>>>>>>>>
> >> >>>>>>>>>> i) The point that you raised that we need to keep all
> >> >>>>>>>>>>
> >> >>>>>>>>>> splits
> >> >>>>>>>>>>
> >> >>>>>>>>>> (processed
> >> >>>>>>>>>>
> >> >>>>>>>>>> and
> >> >>>>>>>>>>
> >> >>>>>>>>>> not-processed) I think is a bit of a strong
> >> >>>>>>>>>>
> >> >>>>>>>>>> requirement.
> >> >>>>>>>>>>
> >> >>>>>>>>>> This
> >> >>>>>>>>>>
> >> >>>>>>>>>> would
> >> >>>>>>>>>>
> >> >>>>>>>>>> imply
> >> >>>>>>>>>>
> >> >>>>>>>>>> that for infinite sources the state will grow
> >> >>>>>>>>>>
> >> >>>>>>>>>> indefinitely.
> >> >>>>>>>>>>
> >> >>>>>>>>>> This is
> >> >>>>>>>>>>
> >> >>>>>>>>>> problem
> >> >>>>>>>>>>
> >> >>>>>>>>>> is even more pronounced if we do not have a single
> >> >>>>>>>>>>
> >> >>>>>>>>>> source
> >> >>>>>>>>>>
> >> >>>>>>>>>> that
> >> >>>>>>>>>>
> >> >>>>>>>>>> assigns
> >> >>>>>>>>>>
> >> >>>>>>>>>> splits to readers, as each reader will have its own
> >> >>>>>>>>>>
> >> >>>>>>>>>> copy
> >> >>>>>>>>>>
> >> >>>>>>>>>> of
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> state.
> >> >>>>>>>>>>
> >> >>>>>>>>>> ii) it is true that for finite sources we need to
> >> >>>>>>>>>>
> >> >>>>>>>>>> somehow
> >> >>>>>>>>>>
> >> >>>>>>>>>> not
> >> >>>>>>>>>>
> >> >>>>>>>>>> close
> >> >>>>>>>>>>
> >> >>>>>>>>>> the
> >> >>>>>>>>>>
> >> >>>>>>>>>> readers when the source/split discoverer finishes. The
> >> >>>>>>>>>> ContinuousFileReaderOperator has a work-around for
> >> >>>>>>>>>>
> >> >>>>>>>>>> that.
> >> >>>>>>>>>>
> >> >>>>>>>>>> It is
> >> >>>>>>>>>>
> >> >>>>>>>>>> not
> >> >>>>>>>>>>
> >> >>>>>>>>>> elegant,
> >> >>>>>>>>>>
> >> >>>>>>>>>> and checkpoints are not emitted after closing the
> >> >>>>>>>>>>
> >> >>>>>>>>>> source,
> >> >>>>>>>>>>
> >> >>>>>>>>>> but
> >> >>>>>>>>>>
> >> >>>>>>>>>> this, I
> >> >>>>>>>>>>
> >> >>>>>>>>>> believe, is a bigger problem which requires more
> >> >>>>>>>>>>
> >> >>>>>>>>>> changes
> >> >>>>>>>>>>
> >> >>>>>>>>>> than
> >> >>>>>>>>>>
> >> >>>>>>>>>> just
> >> >>>>>>>>>>
> >> >>>>>>>>>> refactoring the source interface.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Cheers,
> >> >>>>>>>>>> Kostas
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>> --
> >> >>>>>>>>>> Best, Jingsong Lee
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> --
> >> >>>>>> Best, Jingsong Lee
> >> >>>>>>
> >> >>>>>
> >> >>>>
> >> >>>>
> >> >>>
> >> >>
> >> >
> >>
> >>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
I had an offline chat with Jark, and here are some more thoughts:

1. From SQL perspective, BOUNDED source leads to the batch execution mode,
UNBOUNDED source leads to the streaming execution mode.
2. The semantic of UNBOUNDED source is may or may not stop. The semantic of
BOUNDED source is will stop.
3. The semantic of DataStream is may or may not terminate. The semantic of
BoundedDataStream is will terminate.

Given that, option 3 seems a better option because:
1. SQL already has strict binding between Boundedness and execution mode.
Letting DataStream be consistent would be good.
2. The semantic of UNBOUNDED source is exactly the same as DataStream. So
we should avoid breaking such semantic, i.e. turning some DataStream from
"may or may not terminate" to "will terminate".

For case where users want BOUNDED-streaming combination, they can simply
use an UNBOUNDED source that stops at some point. We can even provide a
simple wrapper to wrap a BOUNDED source as an UNBOUNDED source if that
helps. But API wise, option 3 seems telling a pretty good whole story.

Thanks,

Jiangjie (Becket) Qin




On Thu, Dec 19, 2019 at 10:30 PM Becket Qin <be...@gmail.com> wrote:

> Hi Timo,
>
> Bounded is just a special case of unbounded and every bounded source can
>> also be treated as an unbounded source. This would unify the API if
>> people don't need a bounded operation.
>
>
> With option 3 users can still get a unified API with something like below:
>
> DataStream boundedStream = env.boundedSource(boundedSource);
> DataStream unboundedStream = env.source(unboundedSource);
>
> So in both cases, users can still use a unified DataStream without
> touching the bounded stream only methods.
> By "unify the API if people don't need the bounded operation". Do you
> expect a DataStream with a Bounded source to have the batch operators and
> scheduler settings as well?
>
>
> If we allow DataStream from BOUNDED source, we will essentially pick "*modified
> option 2*".
>
> // The source is either bounded or unbounded, but only unbounded
>> operations could be performed on the returned DataStream.
>> DataStream<Type> dataStream = env.source(someSource);
>
>
>> // The source must be a bounded source, otherwise exception is thrown.
>> BoundedDataStream<Type> boundedDataStream =
>> env.boundedSource(boundedSource);
>
>
>
> // Add the following method to DataStream
>
> Boundedness DataStream#getBoundedness();
>
>
> From pure logical perspective, Boundedness and runtime settings
> (Stream/Batch) are two orthogonal dimensions. And are specified in the
> following way.
>
> *Boundedness* - defined by the source: BOUNDED / UNBOUNDED.
> *Running mode* - defined by the API class: DataStream (Streaming mode) /
> BoundedDataStream (batch mode).
>
> Excluding the UNBOUNDED-batch combination, the "*modified option 2"*
> covers the rest three combination. Compared with "*modified option 2*",
> the main benefit of option 3 is its simplicity and clearness, by tying
> boundedness to running mode and giving up BOUNDED-streaming combination.
>
> Just to be clear, I am fine with either option. But I would like to
> understand a bit more about the bounded-streaming use case and when users
> would prefer this over bounded-batch case, and whether the added value
> justifies the additional complexity in the API. Two cases I can think of
> are:
> 1. The records in DataStream will be processed in order, while
> BoundedDataStream processes records without order guarantee.
> 2. DataStream emits intermediate results when processing a finite dataset,
> while BoundedDataStream only emit the final result. In any case, it could
> be supported by an UNBOUNDED source stopping at some point.
>
> Case 1 is actually misleading because DataStream in general doesn't really
> support in-order process.
> Case 2 seems a rare use case because the instantaneous intermediate result
> seems difficult to reason about. In any case, this can be supported by an
> UNBOUNDED source that stops at some point.
>
> Is there other use cases for bounded-streaming combination I missed? I am
> a little hesitating to put the testing requirement here because ideally I'd
> avoid having public APIs for testing purpose only. And this could be
> resolved by having a UNBOUNDED source stopping at some point as well.
>
> Sorry for the long discussion, but I would really like to make an API
> decision after knowing all the pros and cons.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
>
>
>
>
> On Thu, Dec 19, 2019 at 6:19 PM Timo Walther <tw...@apache.org> wrote:
>
>> Hi Becket,
>>
>> regarding *Option 3* I think we can relax the constraints for
>> env.source():
>>
>> // MySource can be bounded or unbounded
>> DataStream<Type> dataStream = env.source(mySource);
>>
>> // MySource must be bounded, otherwise throws exception.
>> BoundedDataStream<Type> boundedDataStream = env.boundedSource(mySource);
>>
>> Bounded is just a special case of unbounded and every bounded source can
>> also be treated as an unbounded source. This would unify the API if
>> people don't need a bounded operation. It also addresses Jark's concerns.
>>
>> Regards,
>> Timo
>>
>>
>> On 18.12.19 14:16, Becket Qin wrote:
>> > Hi Jark,
>> >
>> > Please see the reply below:
>> >
>> > Regarding to option#3, my concern is that if we don't support streaming
>> >> mode for bounded source,
>> >> how could we create a testing source for streaming mode? Currently,
>> all the
>> >> testing source for streaming
>> >> are bounded, so that the integration test will finish finally.
>> >
>> >
>> > An UNBOUNDED source does not mean it will never stops. It simply
>> indicates
>> > that the source *may* run forever, so the runtime needs to be prepared
>> for
>> > that, but the task may still stop at some point when it hits some
>> > source-specific condition. So an UNBOUNDED testing source can still
>> stop at
>> > some point if needed.
>> >
>> > Regarding to Source#getRecordOrder(), could we have a implicit contract
>> >> that unbounded source should
>> >> already read in order (i.e. reading partitions in parallel), for
>> bounded
>> >> source the order is not mandatory.
>> >
>> >
>> >
>> >> This is also the behaviors of the current sources.
>> >
>> > 1) a source can't guarantee it reads in strict order, because the
>> producer
>> >> may produce data not in order.
>> >> 2) *Bounded-StrictOrder* is not necessary, because batch can reorder
>> data.
>> >
>> >
>> > It is true that sometimes the source cannot guarantee the record order,
>> but
>> > sometimes it can. Right now, even for stream processing, there is no
>> > processing order guarantee. For example, a join operator may emit a
>> later
>> > record which successfully found a join match earlier.
>> > Event order is one of the most important requirements for event
>> processing,
>> > a clear order guarantee would be necessary. That said, I agree that
>> right
>> > now even if the sources provide the record order requirement, the
>> runtime
>> > is not able to guarantee that out of the box. So I am OK if we add the
>> > record order to the Source later. But we should avoid misleading users
>> to
>> > make them think the processing order is guaranteed when using the
>> unbounded
>> > runtime.
>> >
>> > Thanks,
>> >
>> > Jiangjie (Becket) Qin
>> >
>> > On Wed, Dec 18, 2019 at 10:29 AM Jark Wu <im...@gmail.com> wrote:
>> >
>> >> Hi Becket,
>> >>
>> >> That's great we have reached a consensus on Source#getBoundedness().
>> >>
>> >> Regarding to option#3, my concern is that if we don't support streaming
>> >> mode for bounded source,
>> >> how could we create a testing source for streaming mode? Currently,
>> all the
>> >> testing source for streaming
>> >> are bounded, so that the integration test will finish finally.
>> >>
>> >> Regarding to Source#getRecordOrder(), could we have a implicit contract
>> >> that unbounded source should
>> >> already read in order (i.e. reading partitions in parallel), for
>> bounded
>> >> source the order is not mandatory.
>> >> This is also the behaviors of the current sources.
>> >> 1) a source can't guarantee it reads in strict order, because the
>> producer
>> >> may produce data not in order.
>> >> 2) *Bounded-StrictOrder* is not necessary, because batch can reorder
>> data.
>> >>
>> >> Best,
>> >> Jark
>> >>
>> >>
>> >>
>> >> On Tue, 17 Dec 2019 at 22:03, Becket Qin <be...@gmail.com> wrote:
>> >>
>> >>> Hi folks,
>> >>>
>> >>> Thanks for the comments. I am convinced that the Source API should not
>> >> take
>> >>> boundedness as a parameter after it is constructed. What Timo and
>> Dawid
>> >>> suggested sounds a reasonable solution to me. So the Source API would
>> >>> become:
>> >>>
>> >>> Source {
>> >>>      Boundedness getBoundedness();
>> >>> }
>> >>>
>> >>> Assuming the above Source API, in addition to the two options
>> mentioned
>> >> in
>> >>> earlier emails, I am thinking of another option:
>> >>>
>> >>> *Option 3:*
>> >>> // MySource must be unbounded, otherwise throws exception.
>> >>> DataStream<Type> dataStream = env.source(mySource);
>> >>>
>> >>> // MySource must be bounded, otherwise throws exception.
>> >>> BoundedDataStream<Type> boundedDataStream =
>> env.boundedSource(mySource);
>> >>>
>> >>> The pros of this API are:
>> >>>     a) It fits the requirements from Table / SQL well.
>> >>>     b) DataStream users still have type safety (option 2 only has
>> partial
>> >>> type safety).
>> >>>     c) Cristal clear boundedness from the API which makes DataStream
>> join
>> >> /
>> >>> connect easy to reason about.
>> >>> The caveats I see,
>> >>>     a) It is inconsistent with Table since Table has one unified
>> >> interface.
>> >>>     b) No streaming mode for bounded source.
>> >>>
>> >>> @Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
>> >>> <al...@ververica.com> what do you think of the approach?
>> >>>
>> >>>
>> >>> Orthogonal to the above API, I am wondering whether boundedness is the
>> >> only
>> >>> dimension needed to describe the characteristic of the Source
>> behavior.
>> >> We
>> >>> may also need to have another dimension of *record order*.
>> >>>
>> >>> For example, when a file source is reading from a directory with
>> bounded
>> >>> records, it may have two ways to read.
>> >>> 1. Read files in parallel.
>> >>> 2. Read files in the chronological order.
>> >>> In both cases, the file source is a Bounded Source. However, the
>> >> processing
>> >>> requirement for downstream may be different. In the first case, the
>> >>> record processing and result emitting order does not matter, e.g. word
>> >>> count. In the second case, the records may have to be processed in the
>> >>> order they were read, e.g. change log processing.
>> >>>
>> >>> If the Source only has a getBoundedness() method, the downstream
>> >> processors
>> >>> would not know whether the records emitted from the Source should be
>> >>> processed in order or not. So combining the boundedness and record
>> order,
>> >>> we will have four scenarios:
>> >>>
>> >>> *Bounded-StrictOrder*:     A segment of change log.
>> >>> *Bounded-Random*:          Batch Word Count.
>> >>> *Unbounded-StrictOrder*: An infinite change log.
>> >>> *Unbounded-Random*:     Streaming Word Count.
>> >>>
>> >>> Option 2 mentioned in the previous email was kind of trying to handle
>> the
>> >>> Bounded-StrictOrder case by creating a DataStream from a bounded
>> source,
>> >>> which actually does not work.
>> >>> It looks that we do not have strict order support in some operators at
>> >> this
>> >>> point, e.g. join. But we may still want to add the semantic to the
>> Source
>> >>> first so later on we don't need to change all the source
>> implementations,
>> >>> especially given that many of them will be implemented by 3rd party.
>> >>>
>> >>> Given that, we need another dimension of *Record Order* in the Source.
>> >> More
>> >>> specifically, the API would become:
>> >>>
>> >>> Source {
>> >>>      Boundedness getBoundedness();
>> >>>      RecordOrder getRecordOrder();
>> >>> }
>> >>>
>> >>> public enum RecordOrder {
>> >>>      /** The record in the DataStream must be processed in its strict
>> >> order
>> >>> for correctness. */
>> >>>      STRICT,
>> >>>      /** The record in the DataStream can be processed in arbitrary
>> order.
>> >>> */
>> >>>      RANDOM;
>> >>> }
>> >>>
>> >>> Any thoughts?
>> >>>
>> >>> Thanks,
>> >>>
>> >>> Jiangjie (Becket) Qin
>> >>>
>> >>> On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <tw...@apache.org>
>> wrote:
>> >>>
>> >>>> Hi Becket,
>> >>>>
>> >>>> I completely agree with Dawid's suggestion. The information about the
>> >>>> boundedness should come out of the source. Because most of the
>> >> streaming
>> >>>> sources can be made bounded based on some connector specific
>> criterion.
>> >>>> In Kafka, it would be an end offset or end timestamp but in any case
>> >>>> having just a env.boundedSource() is not enough because parameters
>> for
>> >>>> making the source bounded are missing.
>> >>>>
>> >>>> I suggest to have a simple `isBounded(): Boolean` flag in every
>> source
>> >>>> that might be influenced by a connector builder as Dawid mentioned.
>> >>>>
>> >>>> For type safety during programming, we can still go with *Final state
>> >>>> 1*. By having a env.source() vs env.boundedSource(). The latter would
>> >>>> just enforce that the boolean flag is set to `true` and could make
>> >>>> bounded operations available (if we need that actually).
>> >>>>
>> >>>> However, I don't think that we should start making a unified Table
>> API
>> >>>> ununified again. Boundedness is an optimization property. Every
>> bounded
>> >>>> operation can also executed in an unbounded way using
>> >> updates/retraction
>> >>>> or watermarks.
>> >>>>
>> >>>> Regards,
>> >>>> Timo
>> >>>>
>> >>>>
>> >>>> On 15.12.19 14:22, Becket Qin wrote:
>> >>>>> Hi Dawid and Jark,
>> >>>>>
>> >>>>> I think the discussion ultimately boils down to the question that
>> >> which
>> >>>> one
>> >>>>> of the following two final states do we want? Once we make this
>> >>> decision,
>> >>>>> everything else can be naturally derived.
>> >>>>>
>> >>>>> *Final state 1*: Separate API for bounded / unbounded DataStream &
>> >>> Table.
>> >>>>> That means any code users write will be valid at the point when they
>> >>>> write
>> >>>>> the code. This is similar to having type safety check at programming
>> >>>> time.
>> >>>>> For example,
>> >>>>>
>> >>>>> BoundedDataStream extends DataStream {
>> >>>>> // Operations only available for bounded data.
>> >>>>> BoundedDataStream sort(...);
>> >>>>>
>> >>>>> // Interaction with another BoundedStream returns a Bounded stream.
>> >>>>> BoundedJoinedDataStream join(BoundedDataStream other)
>> >>>>>
>> >>>>> // Interaction with another unbounded stream returns an unbounded
>> >>> stream.
>> >>>>> JoinedDataStream join(DataStream other)
>> >>>>> }
>> >>>>>
>> >>>>> BoundedTable extends Table {
>> >>>>>     // Bounded only operation.
>> >>>>> BoundedTable sort(...);
>> >>>>>
>> >>>>> // Interaction with another BoundedTable returns a BoundedTable.
>> >>>>> BoundedTable join(BoundedTable other)
>> >>>>>
>> >>>>> // Interaction with another unbounded table returns an unbounded
>> >> table.
>> >>>>> Table join(Table other)
>> >>>>> }
>> >>>>>
>> >>>>> *Final state 2*: One unified API for bounded / unbounded DataStream
>> /
>> >>>>> Table.
>> >>>>> That unified API may throw exception at DAG compilation time if an
>> >>>> invalid
>> >>>>> operation is tried. This is what Table API currently follows.
>> >>>>>
>> >>>>> DataStream {
>> >>>>> // Throws exception if the DataStream is unbounded.
>> >>>>> DataStream sort();
>> >>>>> // Get boundedness.
>> >>>>> Boundedness getBoundedness();
>> >>>>> }
>> >>>>>
>> >>>>> Table {
>> >>>>> // Throws exception if the table has infinite rows.
>> >>>>> Table orderBy();
>> >>>>>
>> >>>>> // Get boundedness.
>> >>>>> Boundedness getBoundedness();
>> >>>>> }
>> >>>>>
>> >>>>> >From what I understand, there is no consensus so far on this
>> decision
>> >>>> yet.
>> >>>>> Whichever final state we choose, we need to make it consistent
>> across
>> >>> the
>> >>>>> entire project. We should avoid the case that Table follows one
>> final
>> >>>> state
>> >>>>> while DataStream follows another. Some arguments I am aware of from
>> >>> both
>> >>>>> sides so far are following:
>> >>>>>
>> >>>>> Arguments for final state 1:
>> >>>>> 1a) Clean API with method safety check at programming time.
>> >>>>> 1b) (Counter 2b) Although SQL does not have programming time error
>> >>>> check, SQL
>> >>>>> is not really a "programming language" per se. So SQL can be
>> >> different
>> >>>> from
>> >>>>> Table and DataStream.
>> >>>>> 1c)  Although final state 2 seems making it easier for SQL to use
>> >> given
>> >>>> it
>> >>>>> is more "config based" than "parameter based", final state 1 can
>> >>> probably
>> >>>>> also meet what SQL wants by wrapping the Source in TableSource /
>> >>>>> TableSourceFactory API if needed.
>> >>>>>
>> >>>>> Arguments for final state 2:
>> >>>>> 2a) The Source API itself seems already sort of following the
>> unified
>> >>> API
>> >>>>> pattern.
>> >>>>> 2b) There is no "programming time" method error check in SQL case,
>> so
>> >>> we
>> >>>>> cannot really achieve final state 1 across the board.
>> >>>>> 2c) It is an easier path given our current status, i.e. Table is
>> >>> already
>> >>>>> following final state 2.
>> >>>>> 2d) Users can always explicitly check the boundedness if they want
>> >> to.
>> >>>>>
>> >>>>> As I mentioned earlier, my initial thought was also to have a
>> >>>>> "configuration based" Source rather than a "parameter based" Source.
>> >> So
>> >>>> it
>> >>>>> is completely possible that I missed some important consideration or
>> >>>> design
>> >>>>> principles that we want to enforce for the project. It would be good
>> >>>>> if @Stephan
>> >>>>> Ewen <st...@ververica.com> and @Aljoscha Krettek <
>> >>>> aljoscha@ververica.com> can
>> >>>>> also provide more thoughts on this.
>> >>>>>
>> >>>>>
>> >>>>> Re: Jingsong
>> >>>>>
>> >>>>> As you said, there are some batched system source, like parquet/orc
>> >>>> source.
>> >>>>>> Could we have the batch emit interface to improve performance? The
>> >>>> queue of
>> >>>>>> per record may cause performance degradation.
>> >>>>>
>> >>>>>
>> >>>>> The current interface does not necessarily cause performance problem
>> >>> in a
>> >>>>> multi-threading case. In fact, the base implementation allows
>> >>>> SplitReaders
>> >>>>> to add a batch <E> of records<T> to the records queue<E>, so each
>> >>> element
>> >>>>> in the records queue would be a batch <E>. In this case, when the
>> >> main
>> >>>>> thread polls records, it will take a batch <E> of records <T> from
>> >> the
>> >>>>> shared records queue and process the records <T> in a batch manner.
>> >>>>>
>> >>>>> Thanks,
>> >>>>>
>> >>>>> Jiangjie (Becket) Qin
>> >>>>>
>> >>>>> On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <jingsonglee0@gmail.com
>> >
>> >>>> wrote:
>> >>>>>
>> >>>>>> Hi Becket,
>> >>>>>>
>> >>>>>> I also have some performance concerns too.
>> >>>>>>
>> >>>>>> If I understand correctly, SourceOutput will emit data per record
>> >> into
>> >>>> the
>> >>>>>> queue? I'm worried about the multithreading performance of this
>> >> queue.
>> >>>>>>
>> >>>>>>> One example is some batched messaging systems which only have an
>> >>> offset
>> >>>>>> for the entire batch instead of individual messages in the batch.
>> >>>>>>
>> >>>>>> As you said, there are some batched system source, like parquet/orc
>> >>>> source.
>> >>>>>> Could we have the batch emit interface to improve performance? The
>> >>>> queue of
>> >>>>>> per record may cause performance degradation.
>> >>>>>>
>> >>>>>> Best,
>> >>>>>> Jingsong Lee
>> >>>>>>
>> >>>>>> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com> wrote:
>> >>>>>>
>> >>>>>>> Hi Becket,
>> >>>>>>>
>> >>>>>>> I think Dawid explained things clearly and makes a lot of sense.
>> >>>>>>> I'm also in favor of #2, because #1 doesn't work for our future
>> >>> unified
>> >>>>>>> envrionment.
>> >>>>>>>
>> >>>>>>> You can see the vision in this documentation [1]. In the future,
>> we
>> >>>> would
>> >>>>>>> like to
>> >>>>>>> drop the global streaming/batch mode in SQL (i.e.
>> >>>>>>> EnvironmentSettings#inStreamingMode/inBatchMode).
>> >>>>>>> A source is bounded or unbounded once defined, so queries can be
>> >>>> inferred
>> >>>>>>> from source to run
>> >>>>>>> in streaming or batch or hybrid mode. However, in #1, we will lose
>> >>> this
>> >>>>>>> ability because the framework
>> >>>>>>> doesn't know whether the source is bounded or unbounded.
>> >>>>>>>
>> >>>>>>> Best,
>> >>>>>>> Jark
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> [1]:
>> >>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
>> >>>>>>>
>> >>>>>>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <piotr@ververica.com
>> >
>> >>>>>> wrote:
>> >>>>>>>
>> >>>>>>>> Hi,
>> >>>>>>>>
>> >>>>>>>> Regarding the:
>> >>>>>>>>
>> >>>>>>>> Collection<E> getNextRecords()
>> >>>>>>>>
>> >>>>>>>> I’m pretty sure such design would unfortunately impact the
>> >>> performance
>> >>>>>>>> (accessing and potentially creating the collection on the hot
>> >> path).
>> >>>>>>>>
>> >>>>>>>> Also the
>> >>>>>>>>
>> >>>>>>>> InputStatus emitNext(DataOutput<T> output) throws Exception;
>> >>>>>>>> or
>> >>>>>>>> Status pollNext(SourceOutput<T> sourceOutput) throws Exception;
>> >>>>>>>>
>> >>>>>>>> Gives us some opportunities in the future, to allow Source hot
>> >>> looping
>> >>>>>>>> inside, until it receives some signal “please exit because of
>> some
>> >>>>>>> reasons”
>> >>>>>>>> (output collector could return such hint upon collecting the
>> >>> result).
>> >>>>>> But
>> >>>>>>>> that’s another topic outside of this FLIP’s scope.
>> >>>>>>>>
>> >>>>>>>> Piotrek
>> >>>>>>>>
>> >>>>>>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <tr...@apache.org>
>> >>>>>> wrote:
>> >>>>>>>>>
>> >>>>>>>>> Hi Becket,
>> >>>>>>>>>
>> >>>>>>>>> quick clarification from my side because I think you
>> >> misunderstood
>> >>> my
>> >>>>>>>>> question. I did not suggest to let the SourceReader return only
>> a
>> >>>>>>> single
>> >>>>>>>>> record at a time when calling getNextRecords. As the return type
>> >>>>>>>> indicates,
>> >>>>>>>>> the method can return an arbitrary number of records.
>> >>>>>>>>>
>> >>>>>>>>> Cheers,
>> >>>>>>>>> Till
>> >>>>>>>>>
>> >>>>>>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
>> >>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
>> >>>>>>>>> wrote:
>> >>>>>>>>>
>> >>>>>>>>>> Hi Becket,
>> >>>>>>>>>>
>> >>>>>>>>>> Issue #1 - Design of Source interface
>> >>>>>>>>>>
>> >>>>>>>>>> I mentioned the lack of a method like
>> >>>>>>>> Source#createEnumerator(Boundedness
>> >>>>>>>>>> boundedness, SplitEnumeratorContext context), because without
>> >> the
>> >>>>>>>> current
>> >>>>>>>>>> proposal is not complete/does not work.
>> >>>>>>>>>>
>> >>>>>>>>>> If we say that boundedness is an intrinsic property of a source
>> >>> imo
>> >>>>>> we
>> >>>>>>>>>> don't need the Source#createEnumerator(Boundedness boundedness,
>> >>>>>>>>>> SplitEnumeratorContext context) method.
>> >>>>>>>>>>
>> >>>>>>>>>> Assuming a source from my previous example:
>> >>>>>>>>>>
>> >>>>>>>>>> Source source = KafkaSource.builder()
>> >>>>>>>>>>    ...
>> >>>>>>>>>>    .untilTimestamp(...)
>> >>>>>>>>>>    .build()
>> >>>>>>>>>>
>> >>>>>>>>>> Would the enumerator differ if created like
>> >>>>>>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
>> >>>>>>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating myself,
>> >> but
>> >>>>>>> this
>> >>>>>>>> is
>> >>>>>>>>>> the part that my opinion differ the most from the current
>> >>> proposal.
>> >>>>>> I
>> >>>>>>>>>> really think it should always be the source that tells if it is
>> >>>>>>> bounded
>> >>>>>>>> or
>> >>>>>>>>>> not. In the current proposal methods
>> >> continousSource/boundedSource
>> >>>>>>>> somewhat
>> >>>>>>>>>> reconfigure the source, which I think is misleading.
>> >>>>>>>>>>
>> >>>>>>>>>> I think a call like:
>> >>>>>>>>>>
>> >>>>>>>>>> Source source = KafkaSource.builder()
>> >>>>>>>>>>    ...
>> >>>>>>>>>>    .readContinously() / readUntilLatestOffset() /
>> >>> readUntilTimestamp
>> >>>> /
>> >>>>>>>> readUntilOffsets / ...
>> >>>>>>>>>>    .build()
>> >>>>>>>>>>
>> >>>>>>>>>> is way cleaner (and expressive) than
>> >>>>>>>>>>
>> >>>>>>>>>> Source source = KafkaSource.builder()
>> >>>>>>>>>>    ...
>> >>>>>>>>>>    .build()
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> env.continousSource(source) // which actually underneath would
>> >>> call
>> >>>>>>>> createEnumerator(CONTINUOUS, ctx) which would be equivalent to
>> >>>>>>>> source.readContinously().createEnumerator(ctx)
>> >>>>>>>>>> // or
>> >>>>>>>>>> env.boundedSource(source) // which actually underneath would
>> >> call
>> >>>>>>>> createEnumerator(BOUNDED, ctx) which would be equivalent to
>> >>>>>>>> source.readUntilLatestOffset().createEnumerator(ctx)
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Sorry for the comparison, but to me it seems there is too much
>> >>> magic
>> >>>>>>>>>> happening underneath those two calls.
>> >>>>>>>>>>
>> >>>>>>>>>> I really believe the Source interface should have
>> getBoundedness
>> >>>>>>> method
>> >>>>>>>>>> instead of (supportBoundedness) + createEnumerator(Boundedness,
>> >>> ...)
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Issue #2 - Design of
>> >>>>>>>>>>
>> ExecutionEnvironment#source()/continuousSource()/boundedSource()
>> >>>>>>>>>>
>> >>>>>>>>>> As you might have guessed I am slightly in favor of option #2
>> >>>>>>> modified.
>> >>>>>>>>>> Yes I am aware every step of the dag would have to be able to
>> >> say
>> >>> if
>> >>>>>>> it
>> >>>>>>>> is
>> >>>>>>>>>> bounded or not. I have a feeling it would be easier to express
>> >>> cross
>> >>>>>>>>>> bounded/unbounded operations, but I must admit I have not
>> >> thought
>> >>> it
>> >>>>>>>>>> through thoroughly, In the spirit of batch is just a special
>> >> case
>> >>> of
>> >>>>>>>>>> streaming I thought BoundedStream would extend from DataStream.
>> >>>>>>> Correct
>> >>>>>>>> me
>> >>>>>>>>>> if I am wrong. In such a setup the cross bounded/unbounded
>> >>> operation
>> >>>>>>>> could
>> >>>>>>>>>> be expressed quite easily I think:
>> >>>>>>>>>>
>> >>>>>>>>>> DataStream {
>> >>>>>>>>>>    DataStream join(DataStream, ...); // we could not really
>> tell
>> >> if
>> >>>>>> the
>> >>>>>>>> result is bounded or not, but because bounded stream is a special
>> >>> case
>> >>>>>> of
>> >>>>>>>> unbounded the API object is correct, irrespective if the left or
>> >>> right
>> >>>>>>> side
>> >>>>>>>> of the join is bounded
>> >>>>>>>>>> }
>> >>>>>>>>>>
>> >>>>>>>>>> BoundedStream extends DataStream {
>> >>>>>>>>>>    BoundedStream join(BoundedStream, ...); // only if both
>> sides
>> >>> are
>> >>>>>>>> bounded the result can be bounded as well. However we do have
>> >> access
>> >>>> to
>> >>>>>>> the
>> >>>>>>>> DataStream#join here, so you can still join with a DataStream
>> >>>>>>>>>> }
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On the other hand I also see benefits of two completely
>> >> disjointed
>> >>>>>>> APIs,
>> >>>>>>>>>> as we could prohibit some streaming calls in the bounded API. I
>> >>>>>> can't
>> >>>>>>>> think
>> >>>>>>>>>> of any unbounded operators that could not be implemented for
>> >>> bounded
>> >>>>>>>> stream.
>> >>>>>>>>>>
>> >>>>>>>>>> Besides I think we both agree we don't like the method:
>> >>>>>>>>>>
>> >>>>>>>>>> DataStream boundedStream(Source)
>> >>>>>>>>>>
>> >>>>>>>>>> suggested in the current state of the FLIP. Do we ? :)
>> >>>>>>>>>>
>> >>>>>>>>>> Best,
>> >>>>>>>>>>
>> >>>>>>>>>> Dawid
>> >>>>>>>>>>
>> >>>>>>>>>> On 10/12/2019 18:57, Becket Qin wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi folks,
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks for the discussion, great feedback. Also thanks Dawid
>> for
>> >>> the
>> >>>>>>>>>> explanation, it is much clearer now.
>> >>>>>>>>>>
>> >>>>>>>>>> One thing that is indeed missing from the FLIP is how the
>> >>>>>> boundedness
>> >>>>>>> is
>> >>>>>>>>>> passed to the Source implementation. So the API should be
>> >>>>>>>>>> Source#createEnumerator(Boundedness boundedness,
>> >>>>>>> SplitEnumeratorContext
>> >>>>>>>>>> context)
>> >>>>>>>>>> And we can probably remove the
>> >>> Source#supportBoundedness(Boundedness
>> >>>>>>>>>> boundedness) method.
>> >>>>>>>>>>
>> >>>>>>>>>> Assuming we have that, we are essentially choosing from one of
>> >> the
>> >>>>>>>>>> following two options:
>> >>>>>>>>>>
>> >>>>>>>>>> Option 1:
>> >>>>>>>>>> // The source is continuous source, and only unbounded
>> >> operations
>> >>>>>> can
>> >>>>>>> be
>> >>>>>>>>>> performed.
>> >>>>>>>>>> DataStream<Type> datastream = env.continuousSource(someSource);
>> >>>>>>>>>>
>> >>>>>>>>>> // The source is bounded source, both bounded and unbounded
>> >>>>>> operations
>> >>>>>>>> can
>> >>>>>>>>>> be performed.
>> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
>> >>>>>>>> env.boundedSource(someSource);
>> >>>>>>>>>>
>> >>>>>>>>>>    - Pros:
>> >>>>>>>>>>         a) explicit boundary between bounded / unbounded
>> streams,
>> >>> it
>> >>>>>> is
>> >>>>>>>>>> quite simple and clear to the users.
>> >>>>>>>>>>    - Cons:
>> >>>>>>>>>>         a) For applications that do not involve bounded
>> >> operations,
>> >>>>>> they
>> >>>>>>>>>> still have to call different API to distinguish bounded /
>> >>> unbounded
>> >>>>>>>> streams.
>> >>>>>>>>>>         b) No support for bounded stream to run in a streaming
>> >>>> runtime
>> >>>>>>>>>> setting, i.e. scheduling and operators behaviors.
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Option 2:
>> >>>>>>>>>> // The source is either bounded or unbounded, but only
>> unbounded
>> >>>>>>>> operations
>> >>>>>>>>>> could be performed on the returned DataStream.
>> >>>>>>>>>> DataStream<Type> dataStream = env.source(someSource);
>> >>>>>>>>>>
>> >>>>>>>>>> // The source must be a bounded source, otherwise exception is
>> >>>>>> thrown.
>> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
>> >>>>>>>>>> env.boundedSource(boundedSource);
>> >>>>>>>>>>
>> >>>>>>>>>> The pros and cons are exactly the opposite of option 1.
>> >>>>>>>>>>    - Pros:
>> >>>>>>>>>>         a) For applications that do not involve bounded
>> >> operations,
>> >>>>>> they
>> >>>>>>>>>> still have to call different API to distinguish bounded /
>> >>> unbounded
>> >>>>>>>> streams.
>> >>>>>>>>>>         b) Support for bounded stream to run in a streaming
>> >> runtime
>> >>>>>>>> setting,
>> >>>>>>>>>> i.e. scheduling and operators behaviors.
>> >>>>>>>>>>    - Cons:
>> >>>>>>>>>>         a) Bounded / unbounded streams are kind of mixed, i.e.
>> >>> given
>> >>>> a
>> >>>>>>>>>> DataStream, it is not clear whether it is bounded or not,
>> unless
>> >>> you
>> >>>>>>>> have
>> >>>>>>>>>> the access to its source.
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> If we only think from the Source API perspective, option 2
>> >> seems a
>> >>>>>>>> better
>> >>>>>>>>>> choice because functionality wise it is a superset of option 1,
>> >> at
>> >>>>>> the
>> >>>>>>>> cost
>> >>>>>>>>>> of some seemingly acceptable ambiguity in the DataStream API.
>> >>>>>>>>>> But if we look at the DataStream API as a whole, option 1 seems
>> >> a
>> >>>>>>>> clearer
>> >>>>>>>>>> choice. For example, some times a library may have to know
>> >>> whether a
>> >>>>>>>>>> certain task will finish or not. And it would be difficult to
>> >> tell
>> >>>>>> if
>> >>>>>>>> the
>> >>>>>>>>>> input is a DataStream, unless additional information is
>> provided
>> >>> all
>> >>>>>>> the
>> >>>>>>>>>> way from the Source. One possible solution is to have a
>> >> *modified
>> >>>>>>>> option 2*
>> >>>>>>>>>> which adds a method to the DataStream API to indicate
>> >> boundedness,
>> >>>>>>> such
>> >>>>>>>> as
>> >>>>>>>>>> getBoundedness(). It would solve the problem with a potential
>> >>>>>>> confusion
>> >>>>>>>> of
>> >>>>>>>>>> what is difference between a DataStream with
>> >> getBoundedness()=true
>> >>>>>>> and a
>> >>>>>>>>>> BoundedDataStream. But that seems not super difficult to
>> >> explain.
>> >>>>>>>>>>
>> >>>>>>>>>> So from API's perspective, I don't have a strong opinion
>> between
>> >>>>>>>> *option 1*
>> >>>>>>>>>> and *modified option 2. *I like the cleanness of option 1, but
>> >>>>>>> modified
>> >>>>>>>>>> option 2 would be more attractive if we have concrete use case
>> >> for
>> >>>>>> the
>> >>>>>>>>>> "Bounded stream with unbounded streaming runtime settings".
>> >>>>>>>>>>
>> >>>>>>>>>> Re: Till
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Maybe this has already been asked before but I was wondering
>> why
>> >>> the
>> >>>>>>>>>> SourceReader interface has the method pollNext which hands the
>> >>>>>>>>>> responsibility of outputting elements to the SourceReader
>> >>>>>>>> implementation?
>> >>>>>>>>>> Has this been done for backwards compatibility reasons with the
>> >>> old
>> >>>>>>>> source
>> >>>>>>>>>> interface? If not, then one could define a Collection<E>
>> >>>>>>>> getNextRecords()
>> >>>>>>>>>> method which returns the currently retrieved records and then
>> >> the
>> >>>>>>> caller
>> >>>>>>>>>> emits them outside of the SourceReader. That way the interface
>> >>> would
>> >>>>>>> not
>> >>>>>>>>>> allow to implement an outputting loop where we never hand back
>> >>>>>> control
>> >>>>>>>> to
>> >>>>>>>>>> the caller. At the moment, this contract can be easily broken
>> >> and
>> >>> is
>> >>>>>>>> only
>> >>>>>>>>>> mentioned loosely in the JavaDocs.
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> The primary reason we handover the SourceOutput to the
>> >>> SourceReader
>> >>>>>> is
>> >>>>>>>>>> because sometimes it is difficult for a SourceReader to emit
>> one
>> >>>>>>> record
>> >>>>>>>> at
>> >>>>>>>>>> a time. One example is some batched messaging systems which
>> only
>> >>>>>> have
>> >>>>>>> an
>> >>>>>>>>>> offset for the entire batch instead of individual messages in
>> >> the
>> >>>>>>>> batch. In
>> >>>>>>>>>> that case, returning one record at a time would leave the
>> >>>>>> SourceReader
>> >>>>>>>> in
>> >>>>>>>>>> an uncheckpointable state because they can only checkpoint at
>> >> the
>> >>>>>>> batch
>> >>>>>>>>>> boundaries.
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks,
>> >>>>>>>>>>
>> >>>>>>>>>> Jiangjie (Becket) Qin
>> >>>>>>>>>>
>> >>>>>>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <
>> >>> trohrmann@apache.org
>> >>>>>>>> <ma...@apache.org>> <trohrmann@apache.org <mailto:
>> >>>>>>>> trohrmann@apache.org>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Hi everyone,
>> >>>>>>>>>>
>> >>>>>>>>>> thanks for drafting this FLIP. It reads very well.
>> >>>>>>>>>>
>> >>>>>>>>>> Concerning Dawid's proposal, I tend to agree. The boundedness
>> >>> could
>> >>>>>>> come
>> >>>>>>>>>> from the source and tell the system how to treat the operator
>> >>>>>>>> (scheduling
>> >>>>>>>>>> wise). From a user's perspective it should be fine to get back
>> a
>> >>>>>>>> DataStream
>> >>>>>>>>>> when calling env.source(boundedSource) if he does not need
>> >> special
>> >>>>>>>>>> operations defined on a BoundedDataStream. If he needs this,
>> >> then
>> >>>>>> one
>> >>>>>>>> could
>> >>>>>>>>>> use the method BoundedDataStream
>> >> env.boundedSource(boundedSource).
>> >>>>>>>>>>
>> >>>>>>>>>> If possible, we could enforce the proper usage of
>> >>>>>> env.boundedSource()
>> >>>>>>> by
>> >>>>>>>>>> introducing a BoundedSource type so that one cannot pass an
>> >>>>>>>>>> unbounded source to it. That way users would not be able to
>> >> shoot
>> >>>>>>>>>> themselves in the foot.
>> >>>>>>>>>>
>> >>>>>>>>>> Maybe this has already been asked before but I was wondering
>> why
>> >>> the
>> >>>>>>>>>> SourceReader interface has the method pollNext which hands the
>> >>>>>>>>>> responsibility of outputting elements to the SourceReader
>> >>>>>>>> implementation?
>> >>>>>>>>>> Has this been done for backwards compatibility reasons with the
>> >>> old
>> >>>>>>>> source
>> >>>>>>>>>> interface? If not, then one could define a Collection<E>
>> >>>>>>>> getNextRecords()
>> >>>>>>>>>> method which returns the currently retrieved records and then
>> >> the
>> >>>>>>> caller
>> >>>>>>>>>> emits them outside of the SourceReader. That way the interface
>> >>> would
>> >>>>>>> not
>> >>>>>>>>>> allow to implement an outputting loop where we never hand back
>> >>>>>> control
>> >>>>>>>> to
>> >>>>>>>>>> the caller. At the moment, this contract can be easily broken
>> >> and
>> >>> is
>> >>>>>>>> only
>> >>>>>>>>>> mentioned loosely in the JavaDocs.
>> >>>>>>>>>>
>> >>>>>>>>>> Cheers,
>> >>>>>>>>>> Till
>> >>>>>>>>>>
>> >>>>>>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <
>> >>> jingsonglee0@gmail.com
>> >>>>>>>> <ma...@gmail.com>> <jingsonglee0@gmail.com
>> <mailto:
>> >>>>>>>> jingsonglee0@gmail.com>>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Hi all,
>> >>>>>>>>>>
>> >>>>>>>>>> I think current design is good.
>> >>>>>>>>>>
>> >>>>>>>>>> My understanding is:
>> >>>>>>>>>>
>> >>>>>>>>>> For execution mode: bounded mode and continuous mode, It's
>> >> totally
>> >>>>>>>>>> different. I don't think we have the ability to integrate the
>> >> two
>> >>>>>>> models
>> >>>>>>>>>>
>> >>>>>>>>>> at
>> >>>>>>>>>>
>> >>>>>>>>>> present. It's about scheduling, memory, algorithms, States,
>> etc.
>> >>> we
>> >>>>>>>>>> shouldn't confuse them.
>> >>>>>>>>>>
>> >>>>>>>>>> For source capabilities: only bounded, only continuous, both
>> >>> bounded
>> >>>>>>> and
>> >>>>>>>>>> continuous.
>> >>>>>>>>>> I think Kafka is a source that can be ran both bounded
>> >>>>>>>>>> and continuous execution mode.
>> >>>>>>>>>> And Kafka with end offset should be ran both bounded
>> >>>>>>>>>> and continuous execution mode.  Using apache Beam with Flink
>> >>>>>> runner, I
>> >>>>>>>>>>
>> >>>>>>>>>> used
>> >>>>>>>>>>
>> >>>>>>>>>> to run a "bounded" Kafka in streaming mode. For our previous
>> >>>>>>> DataStream,
>> >>>>>>>>>>
>> >>>>>>>>>> it
>> >>>>>>>>>>
>> >>>>>>>>>> is not necessarily required that the source cannot be bounded.
>> >>>>>>>>>>
>> >>>>>>>>>> So it is my thought for Dawid's question:
>> >>>>>>>>>> 1.pass a bounded source to continuousSource() +1
>> >>>>>>>>>> 2.pass a continuous source to boundedSource() -1, should throw
>> >>>>>>>> exception.
>> >>>>>>>>>>
>> >>>>>>>>>> In StreamExecutionEnvironment, continuousSource and
>> >> boundedSource
>> >>>>>>> define
>> >>>>>>>>>> the execution mode. It defines a clear boundary of execution
>> >> mode.
>> >>>>>>>>>>
>> >>>>>>>>>> Best,
>> >>>>>>>>>> Jingsong Lee
>> >>>>>>>>>>
>> >>>>>>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com
>> >>> <mailto:
>> >>>>>>>> imjark@gmail.com>> <imjark@gmail.com <ma...@gmail.com>>
>> >>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> I agree with Dawid's point that the boundedness information
>> >> should
>> >>>>>>> come
>> >>>>>>>>>> from the source itself (e.g. the end timestamp), not through
>> >>>>>>>>>> env.boundedSouce()/continuousSource().
>> >>>>>>>>>> I think if we want to support something like `env.source()`
>> that
>> >>>>>>> derive
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> execution mode from source, `supportsBoundedness(Boundedness)`
>> >>>>>>>>>> method is not enough, because we don't know whether it is
>> >> bounded
>> >>> or
>> >>>>>>>>>>
>> >>>>>>>>>> not.
>> >>>>>>>>>>
>> >>>>>>>>>> Best,
>> >>>>>>>>>> Jark
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
>> >>>>>> dwysakowicz@apache.org
>> >>>>>>>> <ma...@apache.org>> <dwysakowicz@apache.org
>> <mailto:
>> >>>>>>>> dwysakowicz@apache.org>>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> One more thing. In the current proposal, with the
>> >>>>>>>>>> supportsBoundedness(Boundedness) method and the boundedness
>> >> coming
>> >>>>>>>>>>
>> >>>>>>>>>> from
>> >>>>>>>>>>
>> >>>>>>>>>> either continuousSource or boundedSource I could not find how
>> >> this
>> >>>>>>>>>> information is fed back to the SplitEnumerator.
>> >>>>>>>>>>
>> >>>>>>>>>> Best,
>> >>>>>>>>>>
>> >>>>>>>>>> Dawid
>> >>>>>>>>>>
>> >>>>>>>>>> On 09/12/2019 13:52, Becket Qin wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi Dawid,
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks for the comments. This actually brings another relevant
>> >>>>>>>>>>
>> >>>>>>>>>> question
>> >>>>>>>>>>
>> >>>>>>>>>> about what does a "bounded source" imply. I actually had the
>> >> same
>> >>>>>>>>>> impression when I look at the Source API. Here is what I
>> >>> understand
>> >>>>>>>>>>
>> >>>>>>>>>> after
>> >>>>>>>>>>
>> >>>>>>>>>> some discussion with Stephan. The bounded source has the
>> >> following
>> >>>>>>>>>>
>> >>>>>>>>>> impacts.
>> >>>>>>>>>>
>> >>>>>>>>>> 1. API validity.
>> >>>>>>>>>> - A bounded source generates a bounded stream so some
>> operations
>> >>>>>>>>>>
>> >>>>>>>>>> that
>> >>>>>>>>>>
>> >>>>>>>>>> only
>> >>>>>>>>>>
>> >>>>>>>>>> works for bounded records would be performed, e.g. sort.
>> >>>>>>>>>> - To expose these bounded stream only APIs, there are two
>> >> options:
>> >>>>>>>>>>       a. Add them to the DataStream API and throw exception if
>> a
>> >>>>>>>>>>
>> >>>>>>>>>> method
>> >>>>>>>>>>
>> >>>>>>>>>> is
>> >>>>>>>>>>
>> >>>>>>>>>> called on an unbounded stream.
>> >>>>>>>>>>       b. Create a BoundedDataStream class which is returned
>> from
>> >>>>>>>>>> env.boundedSource(), while DataStream is returned from
>> >>>>>>>>>>
>> >>>>>>>>>> env.continousSource().
>> >>>>>>>>>>
>> >>>>>>>>>> Note that this cannot be done by having single
>> >>>>>>>>>>
>> >>>>>>>>>> env.source(theSource)
>> >>>>>>>>>>
>> >>>>>>>>>> even
>> >>>>>>>>>>
>> >>>>>>>>>> the Source has a getBoundedness() method.
>> >>>>>>>>>>
>> >>>>>>>>>> 2. Scheduling
>> >>>>>>>>>> - A bounded source could be computed stage by stage without
>> >>>>>>>>>>
>> >>>>>>>>>> bringing
>> >>>>>>>>>>
>> >>>>>>>>>> up
>> >>>>>>>>>>
>> >>>>>>>>>> all
>> >>>>>>>>>>
>> >>>>>>>>>> the tasks at the same time.
>> >>>>>>>>>>
>> >>>>>>>>>> 3. Operator behaviors
>> >>>>>>>>>> - A bounded source indicates the records are finite so some
>> >>>>>>>>>>
>> >>>>>>>>>> operators
>> >>>>>>>>>>
>> >>>>>>>>>> can
>> >>>>>>>>>>
>> >>>>>>>>>> wait until it receives all the records before it starts the
>> >>>>>>>>>>
>> >>>>>>>>>> processing.
>> >>>>>>>>>>
>> >>>>>>>>>> In the above impact, only 1 is relevant to the API design. And
>> >> the
>> >>>>>>>>>>
>> >>>>>>>>>> current
>> >>>>>>>>>>
>> >>>>>>>>>> proposal in FLIP-27 is following 1.b.
>> >>>>>>>>>>
>> >>>>>>>>>> // boundedness depends of source property, imo this should
>> >> always
>> >>>>>>>>>>
>> >>>>>>>>>> be
>> >>>>>>>>>>
>> >>>>>>>>>> preferred
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> In your proposal, does DataStream have bounded stream only
>> >>> methods?
>> >>>>>>>>>>
>> >>>>>>>>>> It
>> >>>>>>>>>>
>> >>>>>>>>>> looks it should have, otherwise passing a bounded Source to
>> >>>>>>>>>>
>> >>>>>>>>>> env.source()
>> >>>>>>>>>>
>> >>>>>>>>>> would be confusing. In that case, we will essentially do 1.a if
>> >> an
>> >>>>>>>>>> unbounded Source is created from env.source(unboundedSource).
>> >>>>>>>>>>
>> >>>>>>>>>> If we have the methods only supported for bounded streams in
>> >>>>>>>>>>
>> >>>>>>>>>> DataStream,
>> >>>>>>>>>>
>> >>>>>>>>>> it
>> >>>>>>>>>>
>> >>>>>>>>>> seems a little weird to have a separate BoundedDataStream
>> >>>>>>>>>>
>> >>>>>>>>>> interface.
>> >>>>>>>>>>
>> >>>>>>>>>> Am I understand it correctly?
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks,
>> >>>>>>>>>>
>> >>>>>>>>>> Jiangjie (Becket) Qin
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
>> >>>>>>>>>>
>> >>>>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
>> >>>>>>>>>>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Hi all,
>> >>>>>>>>>>
>> >>>>>>>>>> Really well written proposal and very important one. I must
>> >> admit
>> >>>>>>>>>>
>> >>>>>>>>>> I
>> >>>>>>>>>>
>> >>>>>>>>>> have
>> >>>>>>>>>>
>> >>>>>>>>>> not understood all the intricacies of it yet.
>> >>>>>>>>>>
>> >>>>>>>>>> One question I have though is about where does the information
>> >>>>>>>>>>
>> >>>>>>>>>> about
>> >>>>>>>>>>
>> >>>>>>>>>> boundedness come from. I think in most cases it is a property
>> of
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> source. As you described it might be e.g. end offset, a flag
>> >>>>>>>>>>
>> >>>>>>>>>> should
>> >>>>>>>>>>
>> >>>>>>>>>> it
>> >>>>>>>>>>
>> >>>>>>>>>> monitor new splits etc. I think it would be a really nice use
>> >> case
>> >>>>>>>>>>
>> >>>>>>>>>> to
>> >>>>>>>>>>
>> >>>>>>>>>> be
>> >>>>>>>>>>
>> >>>>>>>>>> able to say:
>> >>>>>>>>>>
>> >>>>>>>>>> new KafkaSource().readUntil(long timestamp),
>> >>>>>>>>>>
>> >>>>>>>>>> which could work as an "end offset". Moreover I think all
>> >> Bounded
>> >>>>>>>>>>
>> >>>>>>>>>> sources
>> >>>>>>>>>>
>> >>>>>>>>>> support continuous mode, but no intrinsically continuous source
>> >>>>>>>>>>
>> >>>>>>>>>> support
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> Bounded mode. If I understood the proposal correctly it suggest
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> boundedness sort of "comes" from the outside of the source,
>> from
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> invokation of either boundedStream or continousSource.
>> >>>>>>>>>>
>> >>>>>>>>>> I am wondering if it would make sense to actually change the
>> >>>>>>>>>>
>> >>>>>>>>>> method
>> >>>>>>>>>>
>> >>>>>>>>>> boolean Source#supportsBoundedness(Boundedness)
>> >>>>>>>>>>
>> >>>>>>>>>> to
>> >>>>>>>>>>
>> >>>>>>>>>> Boundedness Source#getBoundedness().
>> >>>>>>>>>>
>> >>>>>>>>>> As for the methods #boundedSource, #continousSource, assuming
>> >> the
>> >>>>>>>>>> boundedness is property of the source they do not affect how
>> the
>> >>>>>>>>>>
>> >>>>>>>>>> enumerator
>> >>>>>>>>>>
>> >>>>>>>>>> works, but mostly how the dag is scheduled, right? I am not
>> >>>>>>>>>>
>> >>>>>>>>>> against
>> >>>>>>>>>>
>> >>>>>>>>>> those
>> >>>>>>>>>>
>> >>>>>>>>>> methods, but I think it is a very specific use case to actually
>> >>>>>>>>>>
>> >>>>>>>>>> override
>> >>>>>>>>>>
>> >>>>>>>>>> the property of the source. In general I would expect users to
>> >>>>>>>>>>
>> >>>>>>>>>> only
>> >>>>>>>>>>
>> >>>>>>>>>> call
>> >>>>>>>>>>
>> >>>>>>>>>> env.source(theSource), where the source tells if it is bounded
>> >> or
>> >>>>>>>>>>
>> >>>>>>>>>> not. I
>> >>>>>>>>>>
>> >>>>>>>>>> would suggest considering following set of methods:
>> >>>>>>>>>>
>> >>>>>>>>>> // boundedness depends of source property, imo this should
>> >> always
>> >>>>>>>>>>
>> >>>>>>>>>> be
>> >>>>>>>>>>
>> >>>>>>>>>> preferred
>> >>>>>>>>>>
>> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> // always continous execution, whether bounded or unbounded
>> >> source
>> >>>>>>>>>>
>> >>>>>>>>>> DataStream<MyType> boundedStream =
>> >> env.continousSource(theSource);
>> >>>>>>>>>>
>> >>>>>>>>>> // imo this would make sense if the BoundedDataStream provides
>> >>>>>>>>>>
>> >>>>>>>>>> additional features unavailable for continous mode
>> >>>>>>>>>>
>> >>>>>>>>>> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Best,
>> >>>>>>>>>>
>> >>>>>>>>>> Dawid
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks, Becket, for updating this.
>> >>>>>>>>>>
>> >>>>>>>>>> I agree with moving the aspects you mentioned into separate
>> >> FLIPs
>> >>>>>>>>>>
>> >>>>>>>>>> -
>> >>>>>>>>>>
>> >>>>>>>>>> this
>> >>>>>>>>>>
>> >>>>>>>>>> one way becoming unwieldy in size.
>> >>>>>>>>>>
>> >>>>>>>>>> +1 to the FLIP in its current state. Its a very detailed
>> >> write-up,
>> >>>>>>>>>>
>> >>>>>>>>>> nicely
>> >>>>>>>>>>
>> >>>>>>>>>> done!
>> >>>>>>>>>>
>> >>>>>>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <
>> becket.qin@gmail.com
>> >>>>>>>> <ma...@gmail.com>> <becket.qin@gmail.com <mailto:
>> >>>>>>>> becket.qin@gmail.com>>
>> >>>>>>>>>>
>> >>>>>>>>>> <
>> >>>>>>>>>>
>> >>>>>>>>>> becket.qin@gmail.com <ma...@gmail.com>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi all,
>> >>>>>>>>>>
>> >>>>>>>>>> Sorry for the long belated update. I have updated FLIP-27 wiki
>> >>>>>>>>>>
>> >>>>>>>>>> page
>> >>>>>>>>>>
>> >>>>>>>>>> with
>> >>>>>>>>>>
>> >>>>>>>>>> the latest proposals. Some noticeable changes include:
>> >>>>>>>>>> 1. A new generic communication mechanism between
>> SplitEnumerator
>> >>>>>>>>>>
>> >>>>>>>>>> and
>> >>>>>>>>>>
>> >>>>>>>>>> SourceReader.
>> >>>>>>>>>> 2. Some detail API method signature changes.
>> >>>>>>>>>>
>> >>>>>>>>>> We left a few things out of this FLIP and will address them in
>> >>>>>>>>>>
>> >>>>>>>>>> separate
>> >>>>>>>>>>
>> >>>>>>>>>> FLIPs. Including:
>> >>>>>>>>>> 1. Per split event time.
>> >>>>>>>>>> 2. Event time alignment.
>> >>>>>>>>>> 3. Fine grained failover for SplitEnumerator failure.
>> >>>>>>>>>>
>> >>>>>>>>>> Please let us know if you have any question.
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks,
>> >>>>>>>>>>
>> >>>>>>>>>> Jiangjie (Becket) Qin
>> >>>>>>>>>>
>> >>>>>>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <sewen@apache.org
>> >>>>>>> <mailto:
>> >>>>>>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>>
>> <
>> >>>>>>>>>>
>> >>>>>>>>>> sewen@apache.org <ma...@apache.org>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi  Łukasz!
>> >>>>>>>>>>
>> >>>>>>>>>> Becket and me are working hard on figuring out the last details
>> >>>>>>>>>>
>> >>>>>>>>>> and
>> >>>>>>>>>>
>> >>>>>>>>>> implementing the first PoC. We would update the FLIP hopefully
>> >>>>>>>>>>
>> >>>>>>>>>> next
>> >>>>>>>>>>
>> >>>>>>>>>> week.
>> >>>>>>>>>>
>> >>>>>>>>>> There is a fair chance that a first version of this will be in
>> >>>>>>>>>>
>> >>>>>>>>>> 1.10,
>> >>>>>>>>>>
>> >>>>>>>>>> but
>> >>>>>>>>>>
>> >>>>>>>>>> I
>> >>>>>>>>>>
>> >>>>>>>>>> think it will take another release to battle test it and
>> migrate
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> connectors.
>> >>>>>>>>>>
>> >>>>>>>>>> Best,
>> >>>>>>>>>> Stephan
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <
>> >> ljd@touk.pl
>> >>>>>>>> <ma...@touk.pl>
>> >>>>>>>>>>
>> >>>>>>>>>> <
>> >>>>>>>>>>
>> >>>>>>>>>> ljd@touk.pl <ma...@touk.pl>>
>> >>>>>>>>>>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi,
>> >>>>>>>>>>
>> >>>>>>>>>> This proposal looks very promising for us. Do you have any
>> plans
>> >>>>>>>>>>
>> >>>>>>>>>> in
>> >>>>>>>>>>
>> >>>>>>>>>> which
>> >>>>>>>>>>
>> >>>>>>>>>> Flink release it is going to be released? We are thinking on
>> >>>>>>>>>>
>> >>>>>>>>>> using a
>> >>>>>>>>>>
>> >>>>>>>>>> Data
>> >>>>>>>>>>
>> >>>>>>>>>> Set API for our future use cases but on the other hand Data Set
>> >>>>>>>>>>
>> >>>>>>>>>> API
>> >>>>>>>>>>
>> >>>>>>>>>> is
>> >>>>>>>>>>
>> >>>>>>>>>> going to be deprecated so using proposed bounded data streams
>> >>>>>>>>>>
>> >>>>>>>>>> solution
>> >>>>>>>>>>
>> >>>>>>>>>> could be more viable in the long term.
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks,
>> >>>>>>>>>> Łukasz
>> >>>>>>>>>>
>> >>>>>>>>>> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com
>> >>>>>> <mailto:
>> >>>>>>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
>> >>>>>>>> thomas.weise@gmail.com>> <
>> >>>>>>>>>>
>> >>>>>>>>>> thomas.weise@gmail.com <ma...@gmail.com>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks for putting together this proposal!
>> >>>>>>>>>>
>> >>>>>>>>>> I see that the "Per Split Event Time" and "Event Time
>> Alignment"
>> >>>>>>>>>>
>> >>>>>>>>>> sections
>> >>>>>>>>>>
>> >>>>>>>>>> are still TBD.
>> >>>>>>>>>>
>> >>>>>>>>>> It would probably be good to flesh those out a bit before
>> >>>>>>>>>>
>> >>>>>>>>>> proceeding
>> >>>>>>>>>>
>> >>>>>>>>>> too
>> >>>>>>>>>>
>> >>>>>>>>>> far
>> >>>>>>>>>>
>> >>>>>>>>>> as the event time alignment will probably influence the
>> >>>>>>>>>>
>> >>>>>>>>>> interaction
>> >>>>>>>>>>
>> >>>>>>>>>> with
>> >>>>>>>>>>
>> >>>>>>>>>> the split reader, specifically ReaderStatus
>> >>>>>>>>>>
>> >>>>>>>>>> emitNext(SourceOutput<E>
>> >>>>>>>>>>
>> >>>>>>>>>> output).
>> >>>>>>>>>>
>> >>>>>>>>>> We currently have only one implementation for event time
>> >> alignment
>> >>>>>>>>>>
>> >>>>>>>>>> in
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> Kinesis consumer. The synchronization in that case takes place
>> >> as
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> last
>> >>>>>>>>>>
>> >>>>>>>>>> step before records are emitted downstream (RecordEmitter).
>> With
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> currently proposed interfaces, the equivalent can be
>> implemented
>> >>>>>>>>>>
>> >>>>>>>>>> in
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> reader loop, although note that in the Kinesis consumer the per
>> >>>>>>>>>>
>> >>>>>>>>>> shard
>> >>>>>>>>>>
>> >>>>>>>>>> threads push records.
>> >>>>>>>>>>
>> >>>>>>>>>> Synchronization has not been implemented for the Kafka consumer
>> >>>>>>>>>>
>> >>>>>>>>>> yet.
>> >>>>>>>>>>
>> >>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
>> >>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675>
>> >>>>>>>>>>
>> >>>>>>>>>> When I looked at it, I realized that the implementation will
>> >> look
>> >>>>>>>>>>
>> >>>>>>>>>> quite
>> >>>>>>>>>>
>> >>>>>>>>>> different
>> >>>>>>>>>> from Kinesis because it needs to take place in the pull part,
>> >>>>>>>>>>
>> >>>>>>>>>> where
>> >>>>>>>>>>
>> >>>>>>>>>> records
>> >>>>>>>>>>
>> >>>>>>>>>> are taken from the Kafka client. Due to the multiplexing it
>> >> cannot
>> >>>>>>>>>>
>> >>>>>>>>>> be
>> >>>>>>>>>>
>> >>>>>>>>>> done
>> >>>>>>>>>>
>> >>>>>>>>>> by blocking the split thread like it currently works for
>> >> Kinesis.
>> >>>>>>>>>>
>> >>>>>>>>>> Reading
>> >>>>>>>>>>
>> >>>>>>>>>> from individual Kafka partitions needs to be controlled via
>> >>>>>>>>>>
>> >>>>>>>>>> pause/resume
>> >>>>>>>>>>
>> >>>>>>>>>> on the Kafka client.
>> >>>>>>>>>>
>> >>>>>>>>>> To take on that responsibility the split thread would need to
>> be
>> >>>>>>>>>>
>> >>>>>>>>>> aware
>> >>>>>>>>>>
>> >>>>>>>>>> of
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>> watermarks or at least whether it should or should not continue
>> >> to
>> >>>>>>>>>>
>> >>>>>>>>>> consume
>> >>>>>>>>>>
>> >>>>>>>>>> a given split and this may require a different SourceReader or
>> >>>>>>>>>>
>> >>>>>>>>>> SourceOutput
>> >>>>>>>>>>
>> >>>>>>>>>> interface.
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks,
>> >>>>>>>>>> Thomas
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com
>> >>>>>> <mailto:
>> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> >> mmyy1110@gmail.com
>> >>>>>
>> >>>>>> <
>> >>>>>>>>>>
>> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi Stephan,
>> >>>>>>>>>>
>> >>>>>>>>>> Thank you for feedback!
>> >>>>>>>>>> Will take a look at your branch before public discussing.
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <
>> sewen@apache.org
>> >>>>>>>> <ma...@apache.org>> <sewen@apache.org <mailto:
>> >>> sewen@apache.org
>> >>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> <
>> >>>>>>>>>>
>> >>>>>>>>>> sewen@apache.org <ma...@apache.org>>
>> >>>>>>>>>>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi Biao!
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks for reviving this. I would like to join this discussion,
>> >>>>>>>>>>
>> >>>>>>>>>> but
>> >>>>>>>>>>
>> >>>>>>>>>> am
>> >>>>>>>>>>
>> >>>>>>>>>> quite occupied with the 1.9 release, so can we maybe pause this
>> >>>>>>>>>>
>> >>>>>>>>>> discussion
>> >>>>>>>>>>
>> >>>>>>>>>> for a week or so?
>> >>>>>>>>>>
>> >>>>>>>>>> In the meantime I can share some suggestion based on prior
>> >>>>>>>>>>
>> >>>>>>>>>> experiments:
>> >>>>>>>>>>
>> >>>>>>>>>> How to do watermarks / timestamp extractors in a simpler and
>> >> more
>> >>>>>>>>>>
>> >>>>>>>>>> flexible
>> >>>>>>>>>>
>> >>>>>>>>>> way. I think that part is quite promising should be part of the
>> >>>>>>>>>>
>> >>>>>>>>>> new
>> >>>>>>>>>>
>> >>>>>>>>>> source
>> >>>>>>>>>>
>> >>>>>>>>>> interface.
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>> >>>>>>>> <
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>> >>>>>>>> <
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Some experiments on how to build the source reader and its
>> >>>>>>>>>>
>> >>>>>>>>>> library
>> >>>>>>>>>>
>> >>>>>>>>>> for
>> >>>>>>>>>>
>> >>>>>>>>>> common threading/split patterns:
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>> >>>>>>>> <
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Best,
>> >>>>>>>>>> Stephan
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mmyy1110@gmail.com
>> >>>>>>> <mailto:
>> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> >> mmyy1110@gmail.com
>> >>>>>
>> >>>>>> <
>> >>>>>>>>>>
>> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>> >>>>>>>>>>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi devs,
>> >>>>>>>>>>
>> >>>>>>>>>> Since 1.9 is nearly released, I think we could get back to
>> >>>>>>>>>>
>> >>>>>>>>>> FLIP-27.
>> >>>>>>>>>>
>> >>>>>>>>>> I
>> >>>>>>>>>>
>> >>>>>>>>>> believe it should be included in 1.10.
>> >>>>>>>>>>
>> >>>>>>>>>> There are so many things mentioned in document of FLIP-27. [1]
>> I
>> >>>>>>>>>>
>> >>>>>>>>>> think
>> >>>>>>>>>>
>> >>>>>>>>>> we'd better discuss them separately. However the wiki is not a
>> >>>>>>>>>>
>> >>>>>>>>>> good
>> >>>>>>>>>>
>> >>>>>>>>>> place
>> >>>>>>>>>>
>> >>>>>>>>>> to discuss. I wrote google doc about SplitReader API which
>> >>>>>>>>>>
>> >>>>>>>>>> misses
>> >>>>>>>>>>
>> >>>>>>>>>> some
>> >>>>>>>>>>
>> >>>>>>>>>> details in the document. [2]
>> >>>>>>>>>>
>> >>>>>>>>>> 1.
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>> >>>>>>>> <
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> 2.
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>> >>>>>>>> <
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com
>> >>>>>> <mailto:
>> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> >> mmyy1110@gmail.com
>> >>>>>
>> >>>>>> <
>> >>>>>>>>>>
>> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>> >>>>>>>>>>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi Steven,
>> >>>>>>>>>> Thank you for the feedback. Please take a look at the document
>> >>>>>>>>>>
>> >>>>>>>>>> FLIP-27
>> >>>>>>>>>>
>> >>>>>>>>>> <
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>> >>>>>>>> <
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> which
>> >>>>>>>>>>
>> >>>>>>>>>> is updated recently. A lot of details of enumerator were added
>> >>>>>>>>>>
>> >>>>>>>>>> in
>> >>>>>>>>>>
>> >>>>>>>>>> this
>> >>>>>>>>>>
>> >>>>>>>>>> document. I think it would help.
>> >>>>>>>>>>
>> >>>>>>>>>> Steven Wu <stevenz3wu@gmail.com <ma...@gmail.com>>
>> >> <
>> >>>>>>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
>> >>>>>>> stevenz3wu@gmail.com
>> >>>>>>>> <ma...@gmail.com>> <stevenz3wu@gmail.com <mailto:
>> >>>>>>>> stevenz3wu@gmail.com>>
>> >>>>>>>>>>
>> >>>>>>>>>> 于2019年3月28日周四
>> >>>>>>>>>>
>> >>>>>>>>>> 下午12:52写道:
>> >>>>>>>>>>
>> >>>>>>>>>> This proposal mentioned that SplitEnumerator might run on the
>> >>>>>>>>>> JobManager or
>> >>>>>>>>>> in a single task on a TaskManager.
>> >>>>>>>>>>
>> >>>>>>>>>> if enumerator is a single task on a taskmanager, then the job
>> >>>>>>>>>>
>> >>>>>>>>>> DAG
>> >>>>>>>>>>
>> >>>>>>>>>> can
>> >>>>>>>>>>
>> >>>>>>>>>> never
>> >>>>>>>>>> been embarrassingly parallel anymore. That will nullify the
>> >>>>>>>>>>
>> >>>>>>>>>> leverage
>> >>>>>>>>>>
>> >>>>>>>>>> of
>> >>>>>>>>>>
>> >>>>>>>>>> fine-grained recovery for embarrassingly parallel jobs.
>> >>>>>>>>>>
>> >>>>>>>>>> It's not clear to me what's the implication of running
>> >>>>>>>>>>
>> >>>>>>>>>> enumerator
>> >>>>>>>>>>
>> >>>>>>>>>> on
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> jobmanager. So I will leave that out for now.
>> >>>>>>>>>>
>> >>>>>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com
>> >>>>>> <mailto:
>> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> >> mmyy1110@gmail.com
>> >>>>>
>> >>>>>> <
>> >>>>>>>>>>
>> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>> >>>>>>>>>>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi Stephan & Piotrek,
>> >>>>>>>>>>
>> >>>>>>>>>> Thank you for feedback.
>> >>>>>>>>>>
>> >>>>>>>>>> It seems that there are a lot of things to do in community.
>> >>>>>>>>>>
>> >>>>>>>>>> I
>> >>>>>>>>>>
>> >>>>>>>>>> am
>> >>>>>>>>>>
>> >>>>>>>>>> just
>> >>>>>>>>>>
>> >>>>>>>>>> afraid that this discussion may be forgotten since there so
>> >>>>>>>>>>
>> >>>>>>>>>> many
>> >>>>>>>>>>
>> >>>>>>>>>> proposals
>> >>>>>>>>>>
>> >>>>>>>>>> recently.
>> >>>>>>>>>> Anyway, wish to see the split topics soon :)
>> >>>>>>>>>>
>> >>>>>>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:
>> >>> piotr@da-platform.com
>> >>>>>>>>
>> >>>>>>> <
>> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
>> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
>> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>>
>> >>>>>>>>>>
>> >>>>>>>>>> 于2019年1月24日周四
>> >>>>>>>>>>
>> >>>>>>>>>> 下午8:21写道:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi Biao!
>> >>>>>>>>>>
>> >>>>>>>>>> This discussion was stalled because of preparations for
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> open
>> >>>>>>>>>>
>> >>>>>>>>>> sourcing
>> >>>>>>>>>>
>> >>>>>>>>>> & merging Blink. I think before creating the tickets we
>> >>>>>>>>>>
>> >>>>>>>>>> should
>> >>>>>>>>>>
>> >>>>>>>>>> split this
>> >>>>>>>>>>
>> >>>>>>>>>> discussion into topics/areas outlined by Stephan and
>> >>>>>>>>>>
>> >>>>>>>>>> create
>> >>>>>>>>>>
>> >>>>>>>>>> Flips
>> >>>>>>>>>>
>> >>>>>>>>>> for
>> >>>>>>>>>>
>> >>>>>>>>>> that.
>> >>>>>>>>>>
>> >>>>>>>>>> I think there is no chance for this to be completed in
>> >>>>>>>>>>
>> >>>>>>>>>> couple
>> >>>>>>>>>>
>> >>>>>>>>>> of
>> >>>>>>>>>>
>> >>>>>>>>>> remaining
>> >>>>>>>>>>
>> >>>>>>>>>> weeks/1 month before 1.8 feature freeze, however it would
>> >>>>>>>>>>
>> >>>>>>>>>> be
>> >>>>>>>>>>
>> >>>>>>>>>> good
>> >>>>>>>>>>
>> >>>>>>>>>> to aim
>> >>>>>>>>>>
>> >>>>>>>>>> with those changes for 1.9.
>> >>>>>>>>>>
>> >>>>>>>>>> Piotrek
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com
>> <mailto:
>> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> >> mmyy1110@gmail.com
>> >>>>>
>> >>>>>> <
>> >>>>>>>>>>
>> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>> >>>>>>>>>>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>> Hi community,
>> >>>>>>>>>> The summary of Stephan makes a lot sense to me. It is
>> >>>>>>>>>>
>> >>>>>>>>>> much
>> >>>>>>>>>>
>> >>>>>>>>>> clearer
>> >>>>>>>>>>
>> >>>>>>>>>> indeed
>> >>>>>>>>>>
>> >>>>>>>>>> after splitting the complex topic into small ones.
>> >>>>>>>>>> I was wondering is there any detail plan for next step?
>> >>>>>>>>>>
>> >>>>>>>>>> If
>> >>>>>>>>>>
>> >>>>>>>>>> not,
>> >>>>>>>>>>
>> >>>>>>>>>> I
>> >>>>>>>>>>
>> >>>>>>>>>> would
>> >>>>>>>>>>
>> >>>>>>>>>> like to push this thing forward by creating some JIRA
>> >>>>>>>>>>
>> >>>>>>>>>> issues.
>> >>>>>>>>>>
>> >>>>>>>>>> Another question is that should version 1.8 include
>> >>>>>>>>>>
>> >>>>>>>>>> these
>> >>>>>>>>>>
>> >>>>>>>>>> features?
>> >>>>>>>>>>
>> >>>>>>>>>> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <
>> >>>>>>>> sewen@apache.org <ma...@apache.org>> <sewen@apache.org
>> >>>> <mailto:
>> >>>>>>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>>
>> >>>>>>>> 于2018年12月1日周六
>> >>>>>>>>>>
>> >>>>>>>>>> 上午4:20写道:
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks everyone for the lively discussion. Let me try
>> >>>>>>>>>>
>> >>>>>>>>>> to
>> >>>>>>>>>>
>> >>>>>>>>>> summarize
>> >>>>>>>>>>
>> >>>>>>>>>> where I
>> >>>>>>>>>>
>> >>>>>>>>>> see convergence in the discussion and open issues.
>> >>>>>>>>>> I'll try to group this by design aspect of the source.
>> >>>>>>>>>>
>> >>>>>>>>>> Please
>> >>>>>>>>>>
>> >>>>>>>>>> let me
>> >>>>>>>>>>
>> >>>>>>>>>> know
>> >>>>>>>>>>
>> >>>>>>>>>> if I got things wrong or missed something crucial here.
>> >>>>>>>>>>
>> >>>>>>>>>> For issues 1-3, if the below reflects the state of the
>> >>>>>>>>>>
>> >>>>>>>>>> discussion, I
>> >>>>>>>>>>
>> >>>>>>>>>> would
>> >>>>>>>>>>
>> >>>>>>>>>> try and update the FLIP in the next days.
>> >>>>>>>>>> For the remaining ones we need more discussion.
>> >>>>>>>>>>
>> >>>>>>>>>> I would suggest to fork each of these aspects into a
>> >>>>>>>>>>
>> >>>>>>>>>> separate
>> >>>>>>>>>>
>> >>>>>>>>>> mail
>> >>>>>>>>>>
>> >>>>>>>>>> thread,
>> >>>>>>>>>>
>> >>>>>>>>>> or will loose sight of the individual aspects.
>> >>>>>>>>>>
>> >>>>>>>>>> *(1) Separation of Split Enumerator and Split Reader*
>> >>>>>>>>>>
>> >>>>>>>>>> - All seem to agree this is a good thing
>> >>>>>>>>>> - Split Enumerator could in the end live on JobManager
>> >>>>>>>>>>
>> >>>>>>>>>> (and
>> >>>>>>>>>>
>> >>>>>>>>>> assign
>> >>>>>>>>>>
>> >>>>>>>>>> splits
>> >>>>>>>>>>
>> >>>>>>>>>> via RPC) or in a task (and assign splits via data
>> >>>>>>>>>>
>> >>>>>>>>>> streams)
>> >>>>>>>>>>
>> >>>>>>>>>> - this discussion is orthogonal and should come later,
>> >>>>>>>>>>
>> >>>>>>>>>> when
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> interface
>> >>>>>>>>>>
>> >>>>>>>>>> is agreed upon.
>> >>>>>>>>>>
>> >>>>>>>>>> *(2) Split Readers for one or more splits*
>> >>>>>>>>>>
>> >>>>>>>>>> - Discussion seems to agree that we need to support
>> >>>>>>>>>>
>> >>>>>>>>>> one
>> >>>>>>>>>>
>> >>>>>>>>>> reader
>> >>>>>>>>>>
>> >>>>>>>>>> that
>> >>>>>>>>>>
>> >>>>>>>>>> possibly handles multiple splits concurrently.
>> >>>>>>>>>> - The requirement comes from sources where one
>> >>>>>>>>>>
>> >>>>>>>>>> poll()-style
>> >>>>>>>>>>
>> >>>>>>>>>> call
>> >>>>>>>>>>
>> >>>>>>>>>> fetches
>> >>>>>>>>>>
>> >>>>>>>>>> data from different splits / partitions
>> >>>>>>>>>>     --> example sources that require that would be for
>> >>>>>>>>>>
>> >>>>>>>>>> example
>> >>>>>>>>>>
>> >>>>>>>>>> Kafka,
>> >>>>>>>>>>
>> >>>>>>>>>> Pravega, Pulsar
>> >>>>>>>>>>
>> >>>>>>>>>> - Could have one split reader per source, or multiple
>> >>>>>>>>>>
>> >>>>>>>>>> split
>> >>>>>>>>>>
>> >>>>>>>>>> readers
>> >>>>>>>>>>
>> >>>>>>>>>> that
>> >>>>>>>>>>
>> >>>>>>>>>> share the "poll()" function
>> >>>>>>>>>> - To not make it too complicated, we can start with
>> >>>>>>>>>>
>> >>>>>>>>>> thinking
>> >>>>>>>>>>
>> >>>>>>>>>> about
>> >>>>>>>>>>
>> >>>>>>>>>> one
>> >>>>>>>>>>
>> >>>>>>>>>> split reader for all splits initially and see if that
>> >>>>>>>>>>
>> >>>>>>>>>> covers
>> >>>>>>>>>>
>> >>>>>>>>>> all
>> >>>>>>>>>>
>> >>>>>>>>>> requirements
>> >>>>>>>>>>
>> >>>>>>>>>> *(3) Threading model of the Split Reader*
>> >>>>>>>>>>
>> >>>>>>>>>> - Most active part of the discussion ;-)
>> >>>>>>>>>>
>> >>>>>>>>>> - A non-blocking way for Flink's task code to interact
>> >>>>>>>>>>
>> >>>>>>>>>> with
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> source
>> >>>>>>>>>>
>> >>>>>>>>>> is
>> >>>>>>>>>>
>> >>>>>>>>>> needed in order to a task runtime code based on a
>> >>>>>>>>>> single-threaded/actor-style task design
>> >>>>>>>>>>     --> I personally am a big proponent of that, it will
>> >>>>>>>>>>
>> >>>>>>>>>> help
>> >>>>>>>>>>
>> >>>>>>>>>> with
>> >>>>>>>>>>
>> >>>>>>>>>> well-behaved checkpoints, efficiency, and simpler yet
>> >>>>>>>>>>
>> >>>>>>>>>> more
>> >>>>>>>>>>
>> >>>>>>>>>> robust
>> >>>>>>>>>>
>> >>>>>>>>>> runtime
>> >>>>>>>>>>
>> >>>>>>>>>> code
>> >>>>>>>>>>
>> >>>>>>>>>> - Users care about simple abstraction, so as a
>> >>>>>>>>>>
>> >>>>>>>>>> subclass
>> >>>>>>>>>>
>> >>>>>>>>>> of
>> >>>>>>>>>>
>> >>>>>>>>>> SplitReader
>> >>>>>>>>>>
>> >>>>>>>>>> (non-blocking / async) we need to have a
>> >>>>>>>>>>
>> >>>>>>>>>> BlockingSplitReader
>> >>>>>>>>>>
>> >>>>>>>>>> which
>> >>>>>>>>>>
>> >>>>>>>>>> will
>> >>>>>>>>>>
>> >>>>>>>>>> form the basis of most source implementations.
>> >>>>>>>>>>
>> >>>>>>>>>> BlockingSplitReader
>> >>>>>>>>>>
>> >>>>>>>>>> lets
>> >>>>>>>>>>
>> >>>>>>>>>> users do blocking simple poll() calls.
>> >>>>>>>>>> - The BlockingSplitReader would spawn a thread (or
>> >>>>>>>>>>
>> >>>>>>>>>> more)
>> >>>>>>>>>>
>> >>>>>>>>>> and
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> thread(s) can make blocking calls and hand over data
>> >>>>>>>>>>
>> >>>>>>>>>> buffers
>> >>>>>>>>>>
>> >>>>>>>>>> via
>> >>>>>>>>>>
>> >>>>>>>>>> a
>> >>>>>>>>>>
>> >>>>>>>>>> blocking
>> >>>>>>>>>>
>> >>>>>>>>>> queue
>> >>>>>>>>>> - This should allow us to cover both, a fully async
>> >>>>>>>>>>
>> >>>>>>>>>> runtime,
>> >>>>>>>>>>
>> >>>>>>>>>> and a
>> >>>>>>>>>>
>> >>>>>>>>>> simple
>> >>>>>>>>>>
>> >>>>>>>>>> blocking interface for users.
>> >>>>>>>>>> - This is actually very similar to how the Kafka
>> >>>>>>>>>>
>> >>>>>>>>>> connectors
>> >>>>>>>>>>
>> >>>>>>>>>> work.
>> >>>>>>>>>>
>> >>>>>>>>>> Kafka
>> >>>>>>>>>>
>> >>>>>>>>>> 9+ with one thread, Kafka 8 with multiple threads
>> >>>>>>>>>>
>> >>>>>>>>>> - On the base SplitReader (the async one), the
>> >>>>>>>>>>
>> >>>>>>>>>> non-blocking
>> >>>>>>>>>>
>> >>>>>>>>>> method
>> >>>>>>>>>>
>> >>>>>>>>>> that
>> >>>>>>>>>>
>> >>>>>>>>>> gets the next chunk of data would signal data
>> >>>>>>>>>>
>> >>>>>>>>>> availability
>> >>>>>>>>>>
>> >>>>>>>>>> via
>> >>>>>>>>>>
>> >>>>>>>>>> a
>> >>>>>>>>>>
>> >>>>>>>>>> CompletableFuture, because that gives the best
>> >>>>>>>>>>
>> >>>>>>>>>> flexibility
>> >>>>>>>>>>
>> >>>>>>>>>> (can
>> >>>>>>>>>>
>> >>>>>>>>>> await
>> >>>>>>>>>>
>> >>>>>>>>>> completion or register notification handlers).
>> >>>>>>>>>> - The source task would register a "thenHandle()" (or
>> >>>>>>>>>>
>> >>>>>>>>>> similar)
>> >>>>>>>>>>
>> >>>>>>>>>> on the
>> >>>>>>>>>>
>> >>>>>>>>>> future to put a "take next data" task into the
>> >>>>>>>>>>
>> >>>>>>>>>> actor-style
>> >>>>>>>>>>
>> >>>>>>>>>> mailbox
>> >>>>>>>>>>
>> >>>>>>>>>> *(4) Split Enumeration and Assignment*
>> >>>>>>>>>>
>> >>>>>>>>>> - Splits may be generated lazily, both in cases where
>> >>>>>>>>>>
>> >>>>>>>>>> there
>> >>>>>>>>>>
>> >>>>>>>>>> is a
>> >>>>>>>>>>
>> >>>>>>>>>> limited
>> >>>>>>>>>>
>> >>>>>>>>>> number of splits (but very many), or splits are
>> >>>>>>>>>>
>> >>>>>>>>>> discovered
>> >>>>>>>>>>
>> >>>>>>>>>> over
>> >>>>>>>>>>
>> >>>>>>>>>> time
>> >>>>>>>>>>
>> >>>>>>>>>> - Assignment should also be lazy, to get better load
>> >>>>>>>>>>
>> >>>>>>>>>> balancing
>> >>>>>>>>>>
>> >>>>>>>>>> - Assignment needs support locality preferences
>> >>>>>>>>>>
>> >>>>>>>>>> - Possible design based on discussion so far:
>> >>>>>>>>>>
>> >>>>>>>>>>     --> SplitReader has a method "addSplits(SplitT...)"
>> >>>>>>>>>>
>> >>>>>>>>>> to
>> >>>>>>>>>>
>> >>>>>>>>>> add
>> >>>>>>>>>>
>> >>>>>>>>>> one or
>> >>>>>>>>>>
>> >>>>>>>>>> more
>> >>>>>>>>>>
>> >>>>>>>>>> splits. Some split readers might assume they have only
>> >>>>>>>>>>
>> >>>>>>>>>> one
>> >>>>>>>>>>
>> >>>>>>>>>> split
>> >>>>>>>>>>
>> >>>>>>>>>> ever,
>> >>>>>>>>>>
>> >>>>>>>>>> concurrently, others assume multiple splits. (Note:
>> >>>>>>>>>>
>> >>>>>>>>>> idea
>> >>>>>>>>>>
>> >>>>>>>>>> behind
>> >>>>>>>>>>
>> >>>>>>>>>> being
>> >>>>>>>>>>
>> >>>>>>>>>> able
>> >>>>>>>>>>
>> >>>>>>>>>> to add multiple splits at the same time is to ease
>> >>>>>>>>>>
>> >>>>>>>>>> startup
>> >>>>>>>>>>
>> >>>>>>>>>> where
>> >>>>>>>>>>
>> >>>>>>>>>> multiple
>> >>>>>>>>>>
>> >>>>>>>>>> splits may be assigned instantly.)
>> >>>>>>>>>>     --> SplitReader has a context object on which it can
>> >>>>>>>>>>
>> >>>>>>>>>> call
>> >>>>>>>>>>
>> >>>>>>>>>> indicate
>> >>>>>>>>>>
>> >>>>>>>>>> when
>> >>>>>>>>>>
>> >>>>>>>>>> splits are completed. The enumerator gets that
>> >>>>>>>>>>
>> >>>>>>>>>> notification and
>> >>>>>>>>>>
>> >>>>>>>>>> can
>> >>>>>>>>>>
>> >>>>>>>>>> use
>> >>>>>>>>>>
>> >>>>>>>>>> to
>> >>>>>>>>>>
>> >>>>>>>>>> decide when to assign new splits. This should help both
>> >>>>>>>>>>
>> >>>>>>>>>> in
>> >>>>>>>>>>
>> >>>>>>>>>> cases
>> >>>>>>>>>>
>> >>>>>>>>>> of
>> >>>>>>>>>>
>> >>>>>>>>>> sources
>> >>>>>>>>>>
>> >>>>>>>>>> that take splits lazily (file readers) and in case the
>> >>>>>>>>>>
>> >>>>>>>>>> source
>> >>>>>>>>>>
>> >>>>>>>>>> needs to
>> >>>>>>>>>>
>> >>>>>>>>>> preserve a partial order between splits (Kinesis,
>> >>>>>>>>>>
>> >>>>>>>>>> Pravega,
>> >>>>>>>>>>
>> >>>>>>>>>> Pulsar may
>> >>>>>>>>>>
>> >>>>>>>>>> need
>> >>>>>>>>>>
>> >>>>>>>>>> that).
>> >>>>>>>>>>     --> SplitEnumerator gets notification when
>> >>>>>>>>>>
>> >>>>>>>>>> SplitReaders
>> >>>>>>>>>>
>> >>>>>>>>>> start
>> >>>>>>>>>>
>> >>>>>>>>>> and
>> >>>>>>>>>>
>> >>>>>>>>>> when
>> >>>>>>>>>>
>> >>>>>>>>>> they finish splits. They can decide at that moment to
>> >>>>>>>>>>
>> >>>>>>>>>> push
>> >>>>>>>>>>
>> >>>>>>>>>> more
>> >>>>>>>>>>
>> >>>>>>>>>> splits
>> >>>>>>>>>>
>> >>>>>>>>>> to
>> >>>>>>>>>>
>> >>>>>>>>>> that reader
>> >>>>>>>>>>     --> The SplitEnumerator should probably be aware of
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> source
>> >>>>>>>>>>
>> >>>>>>>>>> parallelism, to build its initial distribution.
>> >>>>>>>>>>
>> >>>>>>>>>> - Open question: Should the source expose something
>> >>>>>>>>>>
>> >>>>>>>>>> like
>> >>>>>>>>>>
>> >>>>>>>>>> "host
>> >>>>>>>>>>
>> >>>>>>>>>> preferences", so that yarn/mesos/k8s can take this into
>> >>>>>>>>>>
>> >>>>>>>>>> account
>> >>>>>>>>>>
>> >>>>>>>>>> when
>> >>>>>>>>>>
>> >>>>>>>>>> selecting a node to start a TM on?
>> >>>>>>>>>>
>> >>>>>>>>>> *(5) Watermarks and event time alignment*
>> >>>>>>>>>>
>> >>>>>>>>>> - Watermark generation, as well as idleness, needs to
>> >>>>>>>>>>
>> >>>>>>>>>> be
>> >>>>>>>>>>
>> >>>>>>>>>> per
>> >>>>>>>>>>
>> >>>>>>>>>> split
>> >>>>>>>>>>
>> >>>>>>>>>> (like
>> >>>>>>>>>>
>> >>>>>>>>>> currently in the Kafka Source, per partition)
>> >>>>>>>>>> - It is desirable to support optional
>> >>>>>>>>>>
>> >>>>>>>>>> event-time-alignment,
>> >>>>>>>>>>
>> >>>>>>>>>> meaning
>> >>>>>>>>>>
>> >>>>>>>>>> that
>> >>>>>>>>>>
>> >>>>>>>>>> splits that are ahead are back-pressured or temporarily
>> >>>>>>>>>>
>> >>>>>>>>>> unsubscribed
>> >>>>>>>>>>
>> >>>>>>>>>> - I think i would be desirable to encapsulate
>> >>>>>>>>>>
>> >>>>>>>>>> watermark
>> >>>>>>>>>>
>> >>>>>>>>>> generation
>> >>>>>>>>>>
>> >>>>>>>>>> logic
>> >>>>>>>>>>
>> >>>>>>>>>> in watermark generators, for a separation of concerns.
>> >>>>>>>>>>
>> >>>>>>>>>> The
>> >>>>>>>>>>
>> >>>>>>>>>> watermark
>> >>>>>>>>>>
>> >>>>>>>>>> generators should run per split.
>> >>>>>>>>>> - Using watermark generators would also help with
>> >>>>>>>>>>
>> >>>>>>>>>> another
>> >>>>>>>>>>
>> >>>>>>>>>> problem of
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> suggested interface, namely supporting non-periodic
>> >>>>>>>>>>
>> >>>>>>>>>> watermarks
>> >>>>>>>>>>
>> >>>>>>>>>> efficiently.
>> >>>>>>>>>>
>> >>>>>>>>>> - Need a way to "dispatch" next record to different
>> >>>>>>>>>>
>> >>>>>>>>>> watermark
>> >>>>>>>>>>
>> >>>>>>>>>> generators
>> >>>>>>>>>>
>> >>>>>>>>>> - Need a way to tell SplitReader to "suspend" a split
>> >>>>>>>>>>
>> >>>>>>>>>> until a
>> >>>>>>>>>>
>> >>>>>>>>>> certain
>> >>>>>>>>>>
>> >>>>>>>>>> watermark is reached (event time backpressure)
>> >>>>>>>>>> - This would in fact be not needed (and thus simpler)
>> >>>>>>>>>>
>> >>>>>>>>>> if
>> >>>>>>>>>>
>> >>>>>>>>>> we
>> >>>>>>>>>>
>> >>>>>>>>>> had
>> >>>>>>>>>>
>> >>>>>>>>>> a
>> >>>>>>>>>>
>> >>>>>>>>>> SplitReader per split and may be a reason to re-open
>> >>>>>>>>>>
>> >>>>>>>>>> that
>> >>>>>>>>>>
>> >>>>>>>>>> discussion
>> >>>>>>>>>>
>> >>>>>>>>>> *(6) Watermarks across splits and in the Split
>> >>>>>>>>>>
>> >>>>>>>>>> Enumerator*
>> >>>>>>>>>>
>> >>>>>>>>>> - The split enumerator may need some watermark
>> >>>>>>>>>>
>> >>>>>>>>>> awareness,
>> >>>>>>>>>>
>> >>>>>>>>>> which
>> >>>>>>>>>>
>> >>>>>>>>>> should
>> >>>>>>>>>>
>> >>>>>>>>>> be
>> >>>>>>>>>>
>> >>>>>>>>>> purely based on split metadata (like create timestamp
>> >>>>>>>>>>
>> >>>>>>>>>> of
>> >>>>>>>>>>
>> >>>>>>>>>> file
>> >>>>>>>>>>
>> >>>>>>>>>> splits)
>> >>>>>>>>>>
>> >>>>>>>>>> - If there are still more splits with overlapping
>> >>>>>>>>>>
>> >>>>>>>>>> event
>> >>>>>>>>>>
>> >>>>>>>>>> time
>> >>>>>>>>>>
>> >>>>>>>>>> range
>> >>>>>>>>>>
>> >>>>>>>>>> for
>> >>>>>>>>>>
>> >>>>>>>>>> a
>> >>>>>>>>>>
>> >>>>>>>>>> split reader, then that split reader should not advance
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> watermark
>> >>>>>>>>>>
>> >>>>>>>>>> within the split beyond the overlap boundary. Otherwise
>> >>>>>>>>>>
>> >>>>>>>>>> future
>> >>>>>>>>>>
>> >>>>>>>>>> splits
>> >>>>>>>>>>
>> >>>>>>>>>> will
>> >>>>>>>>>>
>> >>>>>>>>>> produce late data.
>> >>>>>>>>>>
>> >>>>>>>>>> - One way to approach this could be that the split
>> >>>>>>>>>>
>> >>>>>>>>>> enumerator
>> >>>>>>>>>>
>> >>>>>>>>>> may
>> >>>>>>>>>>
>> >>>>>>>>>> send
>> >>>>>>>>>>
>> >>>>>>>>>> watermarks to the readers, and the readers cannot emit
>> >>>>>>>>>>
>> >>>>>>>>>> watermarks
>> >>>>>>>>>>
>> >>>>>>>>>> beyond
>> >>>>>>>>>>
>> >>>>>>>>>> that received watermark.
>> >>>>>>>>>> - Many split enumerators would simply immediately send
>> >>>>>>>>>>
>> >>>>>>>>>> Long.MAX
>> >>>>>>>>>>
>> >>>>>>>>>> out
>> >>>>>>>>>>
>> >>>>>>>>>> and
>> >>>>>>>>>>
>> >>>>>>>>>> leave the progress purely to the split readers.
>> >>>>>>>>>>
>> >>>>>>>>>> - For event-time alignment / split back pressure, this
>> >>>>>>>>>>
>> >>>>>>>>>> begs
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> question
>> >>>>>>>>>>
>> >>>>>>>>>> how we can avoid deadlocks that may arise when splits
>> >>>>>>>>>>
>> >>>>>>>>>> are
>> >>>>>>>>>>
>> >>>>>>>>>> suspended
>> >>>>>>>>>>
>> >>>>>>>>>> for
>> >>>>>>>>>>
>> >>>>>>>>>> event time back pressure,
>> >>>>>>>>>>
>> >>>>>>>>>> *(7) Batch and streaming Unification*
>> >>>>>>>>>>
>> >>>>>>>>>> - Functionality wise, the above design should support
>> >>>>>>>>>>
>> >>>>>>>>>> both
>> >>>>>>>>>>
>> >>>>>>>>>> - Batch often (mostly) does not care about reading "in
>> >>>>>>>>>>
>> >>>>>>>>>> order"
>> >>>>>>>>>>
>> >>>>>>>>>> and
>> >>>>>>>>>>
>> >>>>>>>>>> generating watermarks
>> >>>>>>>>>>     --> Might use different enumerator logic that is
>> >>>>>>>>>>
>> >>>>>>>>>> more
>> >>>>>>>>>>
>> >>>>>>>>>> locality
>> >>>>>>>>>>
>> >>>>>>>>>> aware
>> >>>>>>>>>>
>> >>>>>>>>>> and ignores event time order
>> >>>>>>>>>>     --> Does not generate watermarks
>> >>>>>>>>>> - Would be great if bounded sources could be
>> >>>>>>>>>>
>> >>>>>>>>>> identified
>> >>>>>>>>>>
>> >>>>>>>>>> at
>> >>>>>>>>>>
>> >>>>>>>>>> compile
>> >>>>>>>>>>
>> >>>>>>>>>> time,
>> >>>>>>>>>>
>> >>>>>>>>>> so that "env.addBoundedSource(...)" is type safe and
>> >>>>>>>>>>
>> >>>>>>>>>> can
>> >>>>>>>>>>
>> >>>>>>>>>> return a
>> >>>>>>>>>>
>> >>>>>>>>>> "BoundedDataStream".
>> >>>>>>>>>> - Possible to defer this discussion until later
>> >>>>>>>>>>
>> >>>>>>>>>> *Miscellaneous Comments*
>> >>>>>>>>>>
>> >>>>>>>>>> - Should the source have a TypeInformation for the
>> >>>>>>>>>>
>> >>>>>>>>>> produced
>> >>>>>>>>>>
>> >>>>>>>>>> type,
>> >>>>>>>>>>
>> >>>>>>>>>> instead
>> >>>>>>>>>>
>> >>>>>>>>>> of a serializer? We need a type information in the
>> >>>>>>>>>>
>> >>>>>>>>>> stream
>> >>>>>>>>>>
>> >>>>>>>>>> anyways, and
>> >>>>>>>>>>
>> >>>>>>>>>> can
>> >>>>>>>>>>
>> >>>>>>>>>> derive the serializer from that. Plus, creating the
>> >>>>>>>>>>
>> >>>>>>>>>> serializer
>> >>>>>>>>>>
>> >>>>>>>>>> should
>> >>>>>>>>>>
>> >>>>>>>>>> respect the ExecutionConfig.
>> >>>>>>>>>>
>> >>>>>>>>>> - The TypeSerializer interface is very powerful but
>> >>>>>>>>>>
>> >>>>>>>>>> also
>> >>>>>>>>>>
>> >>>>>>>>>> not
>> >>>>>>>>>>
>> >>>>>>>>>> easy to
>> >>>>>>>>>>
>> >>>>>>>>>> implement. Its purpose is to handle data super
>> >>>>>>>>>>
>> >>>>>>>>>> efficiently,
>> >>>>>>>>>>
>> >>>>>>>>>> support
>> >>>>>>>>>>
>> >>>>>>>>>> flexible ways of evolution, etc.
>> >>>>>>>>>> For metadata I would suggest to look at the
>> >>>>>>>>>>
>> >>>>>>>>>> SimpleVersionedSerializer
>> >>>>>>>>>>
>> >>>>>>>>>> instead, which is used for example for checkpoint
>> >>>>>>>>>>
>> >>>>>>>>>> master
>> >>>>>>>>>>
>> >>>>>>>>>> hooks,
>> >>>>>>>>>>
>> >>>>>>>>>> or for
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> streaming file sink. I think that is is a good match
>> >>>>>>>>>>
>> >>>>>>>>>> for
>> >>>>>>>>>>
>> >>>>>>>>>> cases
>> >>>>>>>>>>
>> >>>>>>>>>> where
>> >>>>>>>>>>
>> >>>>>>>>>> we
>> >>>>>>>>>>
>> >>>>>>>>>> do
>> >>>>>>>>>>
>> >>>>>>>>>> not need more than ser/deser (no copy, etc.) and don't
>> >>>>>>>>>>
>> >>>>>>>>>> need to
>> >>>>>>>>>>
>> >>>>>>>>>> push
>> >>>>>>>>>>
>> >>>>>>>>>> versioning out of the serialization paths for best
>> >>>>>>>>>>
>> >>>>>>>>>> performance
>> >>>>>>>>>>
>> >>>>>>>>>> (as in
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> TypeSerializer)
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>> >>>>>>>>>>
>> >>>>>>>>>> k.kloudas@data-artisans.com>
>> >>>>>>>>>>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> Hi Biao,
>> >>>>>>>>>>
>> >>>>>>>>>> Thanks for the answer!
>> >>>>>>>>>>
>> >>>>>>>>>> So given the multi-threaded readers, now we have as
>> >>>>>>>>>>
>> >>>>>>>>>> open
>> >>>>>>>>>>
>> >>>>>>>>>> questions:
>> >>>>>>>>>>
>> >>>>>>>>>> 1) How do we let the checkpoints pass through our
>> >>>>>>>>>>
>> >>>>>>>>>> multi-threaded
>> >>>>>>>>>>
>> >>>>>>>>>> reader
>> >>>>>>>>>>
>> >>>>>>>>>> operator?
>> >>>>>>>>>>
>> >>>>>>>>>> 2) Do we have separate reader and source operators or
>> >>>>>>>>>>
>> >>>>>>>>>> not? In
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> strategy
>> >>>>>>>>>>
>> >>>>>>>>>> that has a separate source, the source operator has a
>> >>>>>>>>>>
>> >>>>>>>>>> parallelism of
>> >>>>>>>>>>
>> >>>>>>>>>> 1
>> >>>>>>>>>>
>> >>>>>>>>>> and
>> >>>>>>>>>>
>> >>>>>>>>>> is responsible for split recovery only.
>> >>>>>>>>>>
>> >>>>>>>>>> For the first one, given also the constraints
>> >>>>>>>>>>
>> >>>>>>>>>> (blocking,
>> >>>>>>>>>>
>> >>>>>>>>>> finite
>> >>>>>>>>>>
>> >>>>>>>>>> queues,
>> >>>>>>>>>>
>> >>>>>>>>>> etc), I do not have an answer yet.
>> >>>>>>>>>>
>> >>>>>>>>>> For the 2nd, I think that we should go with separate
>> >>>>>>>>>>
>> >>>>>>>>>> operators
>> >>>>>>>>>>
>> >>>>>>>>>> for
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> source and the readers, for the following reasons:
>> >>>>>>>>>>
>> >>>>>>>>>> 1) This is more aligned with a potential future
>> >>>>>>>>>>
>> >>>>>>>>>> improvement
>> >>>>>>>>>>
>> >>>>>>>>>> where the
>> >>>>>>>>>>
>> >>>>>>>>>> split
>> >>>>>>>>>>
>> >>>>>>>>>> discovery becomes a responsibility of the JobManager
>> >>>>>>>>>>
>> >>>>>>>>>> and
>> >>>>>>>>>>
>> >>>>>>>>>> readers are
>> >>>>>>>>>>
>> >>>>>>>>>> pooling more work from the JM.
>> >>>>>>>>>>
>> >>>>>>>>>> 2) The source is going to be the "single point of
>> >>>>>>>>>>
>> >>>>>>>>>> truth".
>> >>>>>>>>>>
>> >>>>>>>>>> It
>> >>>>>>>>>>
>> >>>>>>>>>> will
>> >>>>>>>>>>
>> >>>>>>>>>> know
>> >>>>>>>>>>
>> >>>>>>>>>> what
>> >>>>>>>>>>
>> >>>>>>>>>> has been processed and what not. If the source and the
>> >>>>>>>>>>
>> >>>>>>>>>> readers
>> >>>>>>>>>>
>> >>>>>>>>>> are a
>> >>>>>>>>>>
>> >>>>>>>>>> single
>> >>>>>>>>>>
>> >>>>>>>>>> operator with parallelism > 1, or in general, if the
>> >>>>>>>>>>
>> >>>>>>>>>> split
>> >>>>>>>>>>
>> >>>>>>>>>> discovery
>> >>>>>>>>>>
>> >>>>>>>>>> is
>> >>>>>>>>>>
>> >>>>>>>>>> done by each task individually, then:
>> >>>>>>>>>>    i) we have to have a deterministic scheme for each
>> >>>>>>>>>>
>> >>>>>>>>>> reader to
>> >>>>>>>>>>
>> >>>>>>>>>> assign
>> >>>>>>>>>>
>> >>>>>>>>>> splits to itself (e.g. mod subtaskId). This is not
>> >>>>>>>>>>
>> >>>>>>>>>> necessarily
>> >>>>>>>>>>
>> >>>>>>>>>> trivial
>> >>>>>>>>>>
>> >>>>>>>>>> for
>> >>>>>>>>>>
>> >>>>>>>>>> all sources.
>> >>>>>>>>>>    ii) each reader would have to keep a copy of all its
>> >>>>>>>>>>
>> >>>>>>>>>> processed
>> >>>>>>>>>>
>> >>>>>>>>>> slpits
>> >>>>>>>>>>
>> >>>>>>>>>>    iii) the state has to be a union state with a
>> >>>>>>>>>>
>> >>>>>>>>>> non-trivial
>> >>>>>>>>>>
>> >>>>>>>>>> merging
>> >>>>>>>>>>
>> >>>>>>>>>> logic
>> >>>>>>>>>>
>> >>>>>>>>>> in order to support rescaling.
>> >>>>>>>>>>
>> >>>>>>>>>> Two additional points that you raised above:
>> >>>>>>>>>>
>> >>>>>>>>>> i) The point that you raised that we need to keep all
>> >>>>>>>>>>
>> >>>>>>>>>> splits
>> >>>>>>>>>>
>> >>>>>>>>>> (processed
>> >>>>>>>>>>
>> >>>>>>>>>> and
>> >>>>>>>>>>
>> >>>>>>>>>> not-processed) I think is a bit of a strong
>> >>>>>>>>>>
>> >>>>>>>>>> requirement.
>> >>>>>>>>>>
>> >>>>>>>>>> This
>> >>>>>>>>>>
>> >>>>>>>>>> would
>> >>>>>>>>>>
>> >>>>>>>>>> imply
>> >>>>>>>>>>
>> >>>>>>>>>> that for infinite sources the state will grow
>> >>>>>>>>>>
>> >>>>>>>>>> indefinitely.
>> >>>>>>>>>>
>> >>>>>>>>>> This is
>> >>>>>>>>>>
>> >>>>>>>>>> problem
>> >>>>>>>>>>
>> >>>>>>>>>> is even more pronounced if we do not have a single
>> >>>>>>>>>>
>> >>>>>>>>>> source
>> >>>>>>>>>>
>> >>>>>>>>>> that
>> >>>>>>>>>>
>> >>>>>>>>>> assigns
>> >>>>>>>>>>
>> >>>>>>>>>> splits to readers, as each reader will have its own
>> >>>>>>>>>>
>> >>>>>>>>>> copy
>> >>>>>>>>>>
>> >>>>>>>>>> of
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> state.
>> >>>>>>>>>>
>> >>>>>>>>>> ii) it is true that for finite sources we need to
>> >>>>>>>>>>
>> >>>>>>>>>> somehow
>> >>>>>>>>>>
>> >>>>>>>>>> not
>> >>>>>>>>>>
>> >>>>>>>>>> close
>> >>>>>>>>>>
>> >>>>>>>>>> the
>> >>>>>>>>>>
>> >>>>>>>>>> readers when the source/split discoverer finishes. The
>> >>>>>>>>>> ContinuousFileReaderOperator has a work-around for
>> >>>>>>>>>>
>> >>>>>>>>>> that.
>> >>>>>>>>>>
>> >>>>>>>>>> It is
>> >>>>>>>>>>
>> >>>>>>>>>> not
>> >>>>>>>>>>
>> >>>>>>>>>> elegant,
>> >>>>>>>>>>
>> >>>>>>>>>> and checkpoints are not emitted after closing the
>> >>>>>>>>>>
>> >>>>>>>>>> source,
>> >>>>>>>>>>
>> >>>>>>>>>> but
>> >>>>>>>>>>
>> >>>>>>>>>> this, I
>> >>>>>>>>>>
>> >>>>>>>>>> believe, is a bigger problem which requires more
>> >>>>>>>>>>
>> >>>>>>>>>> changes
>> >>>>>>>>>>
>> >>>>>>>>>> than
>> >>>>>>>>>>
>> >>>>>>>>>> just
>> >>>>>>>>>>
>> >>>>>>>>>> refactoring the source interface.
>> >>>>>>>>>>
>> >>>>>>>>>> Cheers,
>> >>>>>>>>>> Kostas
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> --
>> >>>>>>>>>> Best, Jingsong Lee
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>> --
>> >>>>>> Best, Jingsong Lee
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>>
>> >>>
>> >>
>> >
>>
>>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Timo,

Bounded is just a special case of unbounded and every bounded source can
> also be treated as an unbounded source. This would unify the API if
> people don't need a bounded operation.


With option 3 users can still get a unified API with something like below:

DataStream boundedStream = env.boundedSource(boundedSource);
DataStream unboundedStream = env.source(unboundedSource);

So in both cases, users can still use a unified DataStream without touching
the bounded stream only methods.
By "unify the API if people don't need the bounded operation". Do you
expect a DataStream with a Bounded source to have the batch operators and
scheduler settings as well?


If we allow DataStream from BOUNDED source, we will essentially pick "*modified
option 2*".

// The source is either bounded or unbounded, but only unbounded operations
> could be performed on the returned DataStream.
> DataStream<Type> dataStream = env.source(someSource);


> // The source must be a bounded source, otherwise exception is thrown.
> BoundedDataStream<Type> boundedDataStream =
> env.boundedSource(boundedSource);



// Add the following method to DataStream

Boundedness DataStream#getBoundedness();


From pure logical perspective, Boundedness and runtime settings
(Stream/Batch) are two orthogonal dimensions. And are specified in the
following way.

*Boundedness* - defined by the source: BOUNDED / UNBOUNDED.
*Running mode* - defined by the API class: DataStream (Streaming mode) /
BoundedDataStream (batch mode).

Excluding the UNBOUNDED-batch combination, the "*modified option 2"* covers
the rest three combination. Compared with "*modified option 2*", the main
benefit of option 3 is its simplicity and clearness, by tying boundedness
to running mode and giving up BOUNDED-streaming combination.

Just to be clear, I am fine with either option. But I would like to
understand a bit more about the bounded-streaming use case and when users
would prefer this over bounded-batch case, and whether the added value
justifies the additional complexity in the API. Two cases I can think of
are:
1. The records in DataStream will be processed in order, while
BoundedDataStream processes records without order guarantee.
2. DataStream emits intermediate results when processing a finite dataset,
while BoundedDataStream only emit the final result. In any case, it could
be supported by an UNBOUNDED source stopping at some point.

Case 1 is actually misleading because DataStream in general doesn't really
support in-order process.
Case 2 seems a rare use case because the instantaneous intermediate result
seems difficult to reason about. In any case, this can be supported by an
UNBOUNDED source that stops at some point.

Is there other use cases for bounded-streaming combination I missed? I am a
little hesitating to put the testing requirement here because ideally I'd
avoid having public APIs for testing purpose only. And this could be
resolved by having a UNBOUNDED source stopping at some point as well.

Sorry for the long discussion, but I would really like to make an API
decision after knowing all the pros and cons.

Thanks,

Jiangjie (Becket) Qin







On Thu, Dec 19, 2019 at 6:19 PM Timo Walther <tw...@apache.org> wrote:

> Hi Becket,
>
> regarding *Option 3* I think we can relax the constraints for env.source():
>
> // MySource can be bounded or unbounded
> DataStream<Type> dataStream = env.source(mySource);
>
> // MySource must be bounded, otherwise throws exception.
> BoundedDataStream<Type> boundedDataStream = env.boundedSource(mySource);
>
> Bounded is just a special case of unbounded and every bounded source can
> also be treated as an unbounded source. This would unify the API if
> people don't need a bounded operation. It also addresses Jark's concerns.
>
> Regards,
> Timo
>
>
> On 18.12.19 14:16, Becket Qin wrote:
> > Hi Jark,
> >
> > Please see the reply below:
> >
> > Regarding to option#3, my concern is that if we don't support streaming
> >> mode for bounded source,
> >> how could we create a testing source for streaming mode? Currently, all
> the
> >> testing source for streaming
> >> are bounded, so that the integration test will finish finally.
> >
> >
> > An UNBOUNDED source does not mean it will never stops. It simply
> indicates
> > that the source *may* run forever, so the runtime needs to be prepared
> for
> > that, but the task may still stop at some point when it hits some
> > source-specific condition. So an UNBOUNDED testing source can still stop
> at
> > some point if needed.
> >
> > Regarding to Source#getRecordOrder(), could we have a implicit contract
> >> that unbounded source should
> >> already read in order (i.e. reading partitions in parallel), for bounded
> >> source the order is not mandatory.
> >
> >
> >
> >> This is also the behaviors of the current sources.
> >
> > 1) a source can't guarantee it reads in strict order, because the
> producer
> >> may produce data not in order.
> >> 2) *Bounded-StrictOrder* is not necessary, because batch can reorder
> data.
> >
> >
> > It is true that sometimes the source cannot guarantee the record order,
> but
> > sometimes it can. Right now, even for stream processing, there is no
> > processing order guarantee. For example, a join operator may emit a later
> > record which successfully found a join match earlier.
> > Event order is one of the most important requirements for event
> processing,
> > a clear order guarantee would be necessary. That said, I agree that right
> > now even if the sources provide the record order requirement, the runtime
> > is not able to guarantee that out of the box. So I am OK if we add the
> > record order to the Source later. But we should avoid misleading users to
> > make them think the processing order is guaranteed when using the
> unbounded
> > runtime.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > On Wed, Dec 18, 2019 at 10:29 AM Jark Wu <im...@gmail.com> wrote:
> >
> >> Hi Becket,
> >>
> >> That's great we have reached a consensus on Source#getBoundedness().
> >>
> >> Regarding to option#3, my concern is that if we don't support streaming
> >> mode for bounded source,
> >> how could we create a testing source for streaming mode? Currently, all
> the
> >> testing source for streaming
> >> are bounded, so that the integration test will finish finally.
> >>
> >> Regarding to Source#getRecordOrder(), could we have a implicit contract
> >> that unbounded source should
> >> already read in order (i.e. reading partitions in parallel), for bounded
> >> source the order is not mandatory.
> >> This is also the behaviors of the current sources.
> >> 1) a source can't guarantee it reads in strict order, because the
> producer
> >> may produce data not in order.
> >> 2) *Bounded-StrictOrder* is not necessary, because batch can reorder
> data.
> >>
> >> Best,
> >> Jark
> >>
> >>
> >>
> >> On Tue, 17 Dec 2019 at 22:03, Becket Qin <be...@gmail.com> wrote:
> >>
> >>> Hi folks,
> >>>
> >>> Thanks for the comments. I am convinced that the Source API should not
> >> take
> >>> boundedness as a parameter after it is constructed. What Timo and Dawid
> >>> suggested sounds a reasonable solution to me. So the Source API would
> >>> become:
> >>>
> >>> Source {
> >>>      Boundedness getBoundedness();
> >>> }
> >>>
> >>> Assuming the above Source API, in addition to the two options mentioned
> >> in
> >>> earlier emails, I am thinking of another option:
> >>>
> >>> *Option 3:*
> >>> // MySource must be unbounded, otherwise throws exception.
> >>> DataStream<Type> dataStream = env.source(mySource);
> >>>
> >>> // MySource must be bounded, otherwise throws exception.
> >>> BoundedDataStream<Type> boundedDataStream =
> env.boundedSource(mySource);
> >>>
> >>> The pros of this API are:
> >>>     a) It fits the requirements from Table / SQL well.
> >>>     b) DataStream users still have type safety (option 2 only has
> partial
> >>> type safety).
> >>>     c) Cristal clear boundedness from the API which makes DataStream
> join
> >> /
> >>> connect easy to reason about.
> >>> The caveats I see,
> >>>     a) It is inconsistent with Table since Table has one unified
> >> interface.
> >>>     b) No streaming mode for bounded source.
> >>>
> >>> @Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
> >>> <al...@ververica.com> what do you think of the approach?
> >>>
> >>>
> >>> Orthogonal to the above API, I am wondering whether boundedness is the
> >> only
> >>> dimension needed to describe the characteristic of the Source behavior.
> >> We
> >>> may also need to have another dimension of *record order*.
> >>>
> >>> For example, when a file source is reading from a directory with
> bounded
> >>> records, it may have two ways to read.
> >>> 1. Read files in parallel.
> >>> 2. Read files in the chronological order.
> >>> In both cases, the file source is a Bounded Source. However, the
> >> processing
> >>> requirement for downstream may be different. In the first case, the
> >>> record processing and result emitting order does not matter, e.g. word
> >>> count. In the second case, the records may have to be processed in the
> >>> order they were read, e.g. change log processing.
> >>>
> >>> If the Source only has a getBoundedness() method, the downstream
> >> processors
> >>> would not know whether the records emitted from the Source should be
> >>> processed in order or not. So combining the boundedness and record
> order,
> >>> we will have four scenarios:
> >>>
> >>> *Bounded-StrictOrder*:     A segment of change log.
> >>> *Bounded-Random*:          Batch Word Count.
> >>> *Unbounded-StrictOrder*: An infinite change log.
> >>> *Unbounded-Random*:     Streaming Word Count.
> >>>
> >>> Option 2 mentioned in the previous email was kind of trying to handle
> the
> >>> Bounded-StrictOrder case by creating a DataStream from a bounded
> source,
> >>> which actually does not work.
> >>> It looks that we do not have strict order support in some operators at
> >> this
> >>> point, e.g. join. But we may still want to add the semantic to the
> Source
> >>> first so later on we don't need to change all the source
> implementations,
> >>> especially given that many of them will be implemented by 3rd party.
> >>>
> >>> Given that, we need another dimension of *Record Order* in the Source.
> >> More
> >>> specifically, the API would become:
> >>>
> >>> Source {
> >>>      Boundedness getBoundedness();
> >>>      RecordOrder getRecordOrder();
> >>> }
> >>>
> >>> public enum RecordOrder {
> >>>      /** The record in the DataStream must be processed in its strict
> >> order
> >>> for correctness. */
> >>>      STRICT,
> >>>      /** The record in the DataStream can be processed in arbitrary
> order.
> >>> */
> >>>      RANDOM;
> >>> }
> >>>
> >>> Any thoughts?
> >>>
> >>> Thanks,
> >>>
> >>> Jiangjie (Becket) Qin
> >>>
> >>> On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <tw...@apache.org>
> wrote:
> >>>
> >>>> Hi Becket,
> >>>>
> >>>> I completely agree with Dawid's suggestion. The information about the
> >>>> boundedness should come out of the source. Because most of the
> >> streaming
> >>>> sources can be made bounded based on some connector specific
> criterion.
> >>>> In Kafka, it would be an end offset or end timestamp but in any case
> >>>> having just a env.boundedSource() is not enough because parameters for
> >>>> making the source bounded are missing.
> >>>>
> >>>> I suggest to have a simple `isBounded(): Boolean` flag in every source
> >>>> that might be influenced by a connector builder as Dawid mentioned.
> >>>>
> >>>> For type safety during programming, we can still go with *Final state
> >>>> 1*. By having a env.source() vs env.boundedSource(). The latter would
> >>>> just enforce that the boolean flag is set to `true` and could make
> >>>> bounded operations available (if we need that actually).
> >>>>
> >>>> However, I don't think that we should start making a unified Table API
> >>>> ununified again. Boundedness is an optimization property. Every
> bounded
> >>>> operation can also executed in an unbounded way using
> >> updates/retraction
> >>>> or watermarks.
> >>>>
> >>>> Regards,
> >>>> Timo
> >>>>
> >>>>
> >>>> On 15.12.19 14:22, Becket Qin wrote:
> >>>>> Hi Dawid and Jark,
> >>>>>
> >>>>> I think the discussion ultimately boils down to the question that
> >> which
> >>>> one
> >>>>> of the following two final states do we want? Once we make this
> >>> decision,
> >>>>> everything else can be naturally derived.
> >>>>>
> >>>>> *Final state 1*: Separate API for bounded / unbounded DataStream &
> >>> Table.
> >>>>> That means any code users write will be valid at the point when they
> >>>> write
> >>>>> the code. This is similar to having type safety check at programming
> >>>> time.
> >>>>> For example,
> >>>>>
> >>>>> BoundedDataStream extends DataStream {
> >>>>> // Operations only available for bounded data.
> >>>>> BoundedDataStream sort(...);
> >>>>>
> >>>>> // Interaction with another BoundedStream returns a Bounded stream.
> >>>>> BoundedJoinedDataStream join(BoundedDataStream other)
> >>>>>
> >>>>> // Interaction with another unbounded stream returns an unbounded
> >>> stream.
> >>>>> JoinedDataStream join(DataStream other)
> >>>>> }
> >>>>>
> >>>>> BoundedTable extends Table {
> >>>>>     // Bounded only operation.
> >>>>> BoundedTable sort(...);
> >>>>>
> >>>>> // Interaction with another BoundedTable returns a BoundedTable.
> >>>>> BoundedTable join(BoundedTable other)
> >>>>>
> >>>>> // Interaction with another unbounded table returns an unbounded
> >> table.
> >>>>> Table join(Table other)
> >>>>> }
> >>>>>
> >>>>> *Final state 2*: One unified API for bounded / unbounded DataStream /
> >>>>> Table.
> >>>>> That unified API may throw exception at DAG compilation time if an
> >>>> invalid
> >>>>> operation is tried. This is what Table API currently follows.
> >>>>>
> >>>>> DataStream {
> >>>>> // Throws exception if the DataStream is unbounded.
> >>>>> DataStream sort();
> >>>>> // Get boundedness.
> >>>>> Boundedness getBoundedness();
> >>>>> }
> >>>>>
> >>>>> Table {
> >>>>> // Throws exception if the table has infinite rows.
> >>>>> Table orderBy();
> >>>>>
> >>>>> // Get boundedness.
> >>>>> Boundedness getBoundedness();
> >>>>> }
> >>>>>
> >>>>> >From what I understand, there is no consensus so far on this
> decision
> >>>> yet.
> >>>>> Whichever final state we choose, we need to make it consistent across
> >>> the
> >>>>> entire project. We should avoid the case that Table follows one final
> >>>> state
> >>>>> while DataStream follows another. Some arguments I am aware of from
> >>> both
> >>>>> sides so far are following:
> >>>>>
> >>>>> Arguments for final state 1:
> >>>>> 1a) Clean API with method safety check at programming time.
> >>>>> 1b) (Counter 2b) Although SQL does not have programming time error
> >>>> check, SQL
> >>>>> is not really a "programming language" per se. So SQL can be
> >> different
> >>>> from
> >>>>> Table and DataStream.
> >>>>> 1c)  Although final state 2 seems making it easier for SQL to use
> >> given
> >>>> it
> >>>>> is more "config based" than "parameter based", final state 1 can
> >>> probably
> >>>>> also meet what SQL wants by wrapping the Source in TableSource /
> >>>>> TableSourceFactory API if needed.
> >>>>>
> >>>>> Arguments for final state 2:
> >>>>> 2a) The Source API itself seems already sort of following the unified
> >>> API
> >>>>> pattern.
> >>>>> 2b) There is no "programming time" method error check in SQL case, so
> >>> we
> >>>>> cannot really achieve final state 1 across the board.
> >>>>> 2c) It is an easier path given our current status, i.e. Table is
> >>> already
> >>>>> following final state 2.
> >>>>> 2d) Users can always explicitly check the boundedness if they want
> >> to.
> >>>>>
> >>>>> As I mentioned earlier, my initial thought was also to have a
> >>>>> "configuration based" Source rather than a "parameter based" Source.
> >> So
> >>>> it
> >>>>> is completely possible that I missed some important consideration or
> >>>> design
> >>>>> principles that we want to enforce for the project. It would be good
> >>>>> if @Stephan
> >>>>> Ewen <st...@ververica.com> and @Aljoscha Krettek <
> >>>> aljoscha@ververica.com> can
> >>>>> also provide more thoughts on this.
> >>>>>
> >>>>>
> >>>>> Re: Jingsong
> >>>>>
> >>>>> As you said, there are some batched system source, like parquet/orc
> >>>> source.
> >>>>>> Could we have the batch emit interface to improve performance? The
> >>>> queue of
> >>>>>> per record may cause performance degradation.
> >>>>>
> >>>>>
> >>>>> The current interface does not necessarily cause performance problem
> >>> in a
> >>>>> multi-threading case. In fact, the base implementation allows
> >>>> SplitReaders
> >>>>> to add a batch <E> of records<T> to the records queue<E>, so each
> >>> element
> >>>>> in the records queue would be a batch <E>. In this case, when the
> >> main
> >>>>> thread polls records, it will take a batch <E> of records <T> from
> >> the
> >>>>> shared records queue and process the records <T> in a batch manner.
> >>>>>
> >>>>> Thanks,
> >>>>>
> >>>>> Jiangjie (Becket) Qin
> >>>>>
> >>>>> On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <ji...@gmail.com>
> >>>> wrote:
> >>>>>
> >>>>>> Hi Becket,
> >>>>>>
> >>>>>> I also have some performance concerns too.
> >>>>>>
> >>>>>> If I understand correctly, SourceOutput will emit data per record
> >> into
> >>>> the
> >>>>>> queue? I'm worried about the multithreading performance of this
> >> queue.
> >>>>>>
> >>>>>>> One example is some batched messaging systems which only have an
> >>> offset
> >>>>>> for the entire batch instead of individual messages in the batch.
> >>>>>>
> >>>>>> As you said, there are some batched system source, like parquet/orc
> >>>> source.
> >>>>>> Could we have the batch emit interface to improve performance? The
> >>>> queue of
> >>>>>> per record may cause performance degradation.
> >>>>>>
> >>>>>> Best,
> >>>>>> Jingsong Lee
> >>>>>>
> >>>>>> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com> wrote:
> >>>>>>
> >>>>>>> Hi Becket,
> >>>>>>>
> >>>>>>> I think Dawid explained things clearly and makes a lot of sense.
> >>>>>>> I'm also in favor of #2, because #1 doesn't work for our future
> >>> unified
> >>>>>>> envrionment.
> >>>>>>>
> >>>>>>> You can see the vision in this documentation [1]. In the future, we
> >>>> would
> >>>>>>> like to
> >>>>>>> drop the global streaming/batch mode in SQL (i.e.
> >>>>>>> EnvironmentSettings#inStreamingMode/inBatchMode).
> >>>>>>> A source is bounded or unbounded once defined, so queries can be
> >>>> inferred
> >>>>>>> from source to run
> >>>>>>> in streaming or batch or hybrid mode. However, in #1, we will lose
> >>> this
> >>>>>>> ability because the framework
> >>>>>>> doesn't know whether the source is bounded or unbounded.
> >>>>>>>
> >>>>>>> Best,
> >>>>>>> Jark
> >>>>>>>
> >>>>>>>
> >>>>>>> [1]:
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
> >>>>>>>
> >>>>>>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <pi...@ververica.com>
> >>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi,
> >>>>>>>>
> >>>>>>>> Regarding the:
> >>>>>>>>
> >>>>>>>> Collection<E> getNextRecords()
> >>>>>>>>
> >>>>>>>> I’m pretty sure such design would unfortunately impact the
> >>> performance
> >>>>>>>> (accessing and potentially creating the collection on the hot
> >> path).
> >>>>>>>>
> >>>>>>>> Also the
> >>>>>>>>
> >>>>>>>> InputStatus emitNext(DataOutput<T> output) throws Exception;
> >>>>>>>> or
> >>>>>>>> Status pollNext(SourceOutput<T> sourceOutput) throws Exception;
> >>>>>>>>
> >>>>>>>> Gives us some opportunities in the future, to allow Source hot
> >>> looping
> >>>>>>>> inside, until it receives some signal “please exit because of some
> >>>>>>> reasons”
> >>>>>>>> (output collector could return such hint upon collecting the
> >>> result).
> >>>>>> But
> >>>>>>>> that’s another topic outside of this FLIP’s scope.
> >>>>>>>>
> >>>>>>>> Piotrek
> >>>>>>>>
> >>>>>>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <tr...@apache.org>
> >>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>> Hi Becket,
> >>>>>>>>>
> >>>>>>>>> quick clarification from my side because I think you
> >> misunderstood
> >>> my
> >>>>>>>>> question. I did not suggest to let the SourceReader return only a
> >>>>>>> single
> >>>>>>>>> record at a time when calling getNextRecords. As the return type
> >>>>>>>> indicates,
> >>>>>>>>> the method can return an arbitrary number of records.
> >>>>>>>>>
> >>>>>>>>> Cheers,
> >>>>>>>>> Till
> >>>>>>>>>
> >>>>>>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> >>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi Becket,
> >>>>>>>>>>
> >>>>>>>>>> Issue #1 - Design of Source interface
> >>>>>>>>>>
> >>>>>>>>>> I mentioned the lack of a method like
> >>>>>>>> Source#createEnumerator(Boundedness
> >>>>>>>>>> boundedness, SplitEnumeratorContext context), because without
> >> the
> >>>>>>>> current
> >>>>>>>>>> proposal is not complete/does not work.
> >>>>>>>>>>
> >>>>>>>>>> If we say that boundedness is an intrinsic property of a source
> >>> imo
> >>>>>> we
> >>>>>>>>>> don't need the Source#createEnumerator(Boundedness boundedness,
> >>>>>>>>>> SplitEnumeratorContext context) method.
> >>>>>>>>>>
> >>>>>>>>>> Assuming a source from my previous example:
> >>>>>>>>>>
> >>>>>>>>>> Source source = KafkaSource.builder()
> >>>>>>>>>>    ...
> >>>>>>>>>>    .untilTimestamp(...)
> >>>>>>>>>>    .build()
> >>>>>>>>>>
> >>>>>>>>>> Would the enumerator differ if created like
> >>>>>>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
> >>>>>>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating myself,
> >> but
> >>>>>>> this
> >>>>>>>> is
> >>>>>>>>>> the part that my opinion differ the most from the current
> >>> proposal.
> >>>>>> I
> >>>>>>>>>> really think it should always be the source that tells if it is
> >>>>>>> bounded
> >>>>>>>> or
> >>>>>>>>>> not. In the current proposal methods
> >> continousSource/boundedSource
> >>>>>>>> somewhat
> >>>>>>>>>> reconfigure the source, which I think is misleading.
> >>>>>>>>>>
> >>>>>>>>>> I think a call like:
> >>>>>>>>>>
> >>>>>>>>>> Source source = KafkaSource.builder()
> >>>>>>>>>>    ...
> >>>>>>>>>>    .readContinously() / readUntilLatestOffset() /
> >>> readUntilTimestamp
> >>>> /
> >>>>>>>> readUntilOffsets / ...
> >>>>>>>>>>    .build()
> >>>>>>>>>>
> >>>>>>>>>> is way cleaner (and expressive) than
> >>>>>>>>>>
> >>>>>>>>>> Source source = KafkaSource.builder()
> >>>>>>>>>>    ...
> >>>>>>>>>>    .build()
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> env.continousSource(source) // which actually underneath would
> >>> call
> >>>>>>>> createEnumerator(CONTINUOUS, ctx) which would be equivalent to
> >>>>>>>> source.readContinously().createEnumerator(ctx)
> >>>>>>>>>> // or
> >>>>>>>>>> env.boundedSource(source) // which actually underneath would
> >> call
> >>>>>>>> createEnumerator(BOUNDED, ctx) which would be equivalent to
> >>>>>>>> source.readUntilLatestOffset().createEnumerator(ctx)
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Sorry for the comparison, but to me it seems there is too much
> >>> magic
> >>>>>>>>>> happening underneath those two calls.
> >>>>>>>>>>
> >>>>>>>>>> I really believe the Source interface should have getBoundedness
> >>>>>>> method
> >>>>>>>>>> instead of (supportBoundedness) + createEnumerator(Boundedness,
> >>> ...)
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Issue #2 - Design of
> >>>>>>>>>> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> >>>>>>>>>>
> >>>>>>>>>> As you might have guessed I am slightly in favor of option #2
> >>>>>>> modified.
> >>>>>>>>>> Yes I am aware every step of the dag would have to be able to
> >> say
> >>> if
> >>>>>>> it
> >>>>>>>> is
> >>>>>>>>>> bounded or not. I have a feeling it would be easier to express
> >>> cross
> >>>>>>>>>> bounded/unbounded operations, but I must admit I have not
> >> thought
> >>> it
> >>>>>>>>>> through thoroughly, In the spirit of batch is just a special
> >> case
> >>> of
> >>>>>>>>>> streaming I thought BoundedStream would extend from DataStream.
> >>>>>>> Correct
> >>>>>>>> me
> >>>>>>>>>> if I am wrong. In such a setup the cross bounded/unbounded
> >>> operation
> >>>>>>>> could
> >>>>>>>>>> be expressed quite easily I think:
> >>>>>>>>>>
> >>>>>>>>>> DataStream {
> >>>>>>>>>>    DataStream join(DataStream, ...); // we could not really tell
> >> if
> >>>>>> the
> >>>>>>>> result is bounded or not, but because bounded stream is a special
> >>> case
> >>>>>> of
> >>>>>>>> unbounded the API object is correct, irrespective if the left or
> >>> right
> >>>>>>> side
> >>>>>>>> of the join is bounded
> >>>>>>>>>> }
> >>>>>>>>>>
> >>>>>>>>>> BoundedStream extends DataStream {
> >>>>>>>>>>    BoundedStream join(BoundedStream, ...); // only if both sides
> >>> are
> >>>>>>>> bounded the result can be bounded as well. However we do have
> >> access
> >>>> to
> >>>>>>> the
> >>>>>>>> DataStream#join here, so you can still join with a DataStream
> >>>>>>>>>> }
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On the other hand I also see benefits of two completely
> >> disjointed
> >>>>>>> APIs,
> >>>>>>>>>> as we could prohibit some streaming calls in the bounded API. I
> >>>>>> can't
> >>>>>>>> think
> >>>>>>>>>> of any unbounded operators that could not be implemented for
> >>> bounded
> >>>>>>>> stream.
> >>>>>>>>>>
> >>>>>>>>>> Besides I think we both agree we don't like the method:
> >>>>>>>>>>
> >>>>>>>>>> DataStream boundedStream(Source)
> >>>>>>>>>>
> >>>>>>>>>> suggested in the current state of the FLIP. Do we ? :)
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>>
> >>>>>>>>>> Dawid
> >>>>>>>>>>
> >>>>>>>>>> On 10/12/2019 18:57, Becket Qin wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi folks,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for the discussion, great feedback. Also thanks Dawid for
> >>> the
> >>>>>>>>>> explanation, it is much clearer now.
> >>>>>>>>>>
> >>>>>>>>>> One thing that is indeed missing from the FLIP is how the
> >>>>>> boundedness
> >>>>>>> is
> >>>>>>>>>> passed to the Source implementation. So the API should be
> >>>>>>>>>> Source#createEnumerator(Boundedness boundedness,
> >>>>>>> SplitEnumeratorContext
> >>>>>>>>>> context)
> >>>>>>>>>> And we can probably remove the
> >>> Source#supportBoundedness(Boundedness
> >>>>>>>>>> boundedness) method.
> >>>>>>>>>>
> >>>>>>>>>> Assuming we have that, we are essentially choosing from one of
> >> the
> >>>>>>>>>> following two options:
> >>>>>>>>>>
> >>>>>>>>>> Option 1:
> >>>>>>>>>> // The source is continuous source, and only unbounded
> >> operations
> >>>>>> can
> >>>>>>> be
> >>>>>>>>>> performed.
> >>>>>>>>>> DataStream<Type> datastream = env.continuousSource(someSource);
> >>>>>>>>>>
> >>>>>>>>>> // The source is bounded source, both bounded and unbounded
> >>>>>> operations
> >>>>>>>> can
> >>>>>>>>>> be performed.
> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> >>>>>>>> env.boundedSource(someSource);
> >>>>>>>>>>
> >>>>>>>>>>    - Pros:
> >>>>>>>>>>         a) explicit boundary between bounded / unbounded
> streams,
> >>> it
> >>>>>> is
> >>>>>>>>>> quite simple and clear to the users.
> >>>>>>>>>>    - Cons:
> >>>>>>>>>>         a) For applications that do not involve bounded
> >> operations,
> >>>>>> they
> >>>>>>>>>> still have to call different API to distinguish bounded /
> >>> unbounded
> >>>>>>>> streams.
> >>>>>>>>>>         b) No support for bounded stream to run in a streaming
> >>>> runtime
> >>>>>>>>>> setting, i.e. scheduling and operators behaviors.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Option 2:
> >>>>>>>>>> // The source is either bounded or unbounded, but only unbounded
> >>>>>>>> operations
> >>>>>>>>>> could be performed on the returned DataStream.
> >>>>>>>>>> DataStream<Type> dataStream = env.source(someSource);
> >>>>>>>>>>
> >>>>>>>>>> // The source must be a bounded source, otherwise exception is
> >>>>>> thrown.
> >>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
> >>>>>>>>>> env.boundedSource(boundedSource);
> >>>>>>>>>>
> >>>>>>>>>> The pros and cons are exactly the opposite of option 1.
> >>>>>>>>>>    - Pros:
> >>>>>>>>>>         a) For applications that do not involve bounded
> >> operations,
> >>>>>> they
> >>>>>>>>>> still have to call different API to distinguish bounded /
> >>> unbounded
> >>>>>>>> streams.
> >>>>>>>>>>         b) Support for bounded stream to run in a streaming
> >> runtime
> >>>>>>>> setting,
> >>>>>>>>>> i.e. scheduling and operators behaviors.
> >>>>>>>>>>    - Cons:
> >>>>>>>>>>         a) Bounded / unbounded streams are kind of mixed, i.e.
> >>> given
> >>>> a
> >>>>>>>>>> DataStream, it is not clear whether it is bounded or not, unless
> >>> you
> >>>>>>>> have
> >>>>>>>>>> the access to its source.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> If we only think from the Source API perspective, option 2
> >> seems a
> >>>>>>>> better
> >>>>>>>>>> choice because functionality wise it is a superset of option 1,
> >> at
> >>>>>> the
> >>>>>>>> cost
> >>>>>>>>>> of some seemingly acceptable ambiguity in the DataStream API.
> >>>>>>>>>> But if we look at the DataStream API as a whole, option 1 seems
> >> a
> >>>>>>>> clearer
> >>>>>>>>>> choice. For example, some times a library may have to know
> >>> whether a
> >>>>>>>>>> certain task will finish or not. And it would be difficult to
> >> tell
> >>>>>> if
> >>>>>>>> the
> >>>>>>>>>> input is a DataStream, unless additional information is provided
> >>> all
> >>>>>>> the
> >>>>>>>>>> way from the Source. One possible solution is to have a
> >> *modified
> >>>>>>>> option 2*
> >>>>>>>>>> which adds a method to the DataStream API to indicate
> >> boundedness,
> >>>>>>> such
> >>>>>>>> as
> >>>>>>>>>> getBoundedness(). It would solve the problem with a potential
> >>>>>>> confusion
> >>>>>>>> of
> >>>>>>>>>> what is difference between a DataStream with
> >> getBoundedness()=true
> >>>>>>> and a
> >>>>>>>>>> BoundedDataStream. But that seems not super difficult to
> >> explain.
> >>>>>>>>>>
> >>>>>>>>>> So from API's perspective, I don't have a strong opinion between
> >>>>>>>> *option 1*
> >>>>>>>>>> and *modified option 2. *I like the cleanness of option 1, but
> >>>>>>> modified
> >>>>>>>>>> option 2 would be more attractive if we have concrete use case
> >> for
> >>>>>> the
> >>>>>>>>>> "Bounded stream with unbounded streaming runtime settings".
> >>>>>>>>>>
> >>>>>>>>>> Re: Till
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Maybe this has already been asked before but I was wondering why
> >>> the
> >>>>>>>>>> SourceReader interface has the method pollNext which hands the
> >>>>>>>>>> responsibility of outputting elements to the SourceReader
> >>>>>>>> implementation?
> >>>>>>>>>> Has this been done for backwards compatibility reasons with the
> >>> old
> >>>>>>>> source
> >>>>>>>>>> interface? If not, then one could define a Collection<E>
> >>>>>>>> getNextRecords()
> >>>>>>>>>> method which returns the currently retrieved records and then
> >> the
> >>>>>>> caller
> >>>>>>>>>> emits them outside of the SourceReader. That way the interface
> >>> would
> >>>>>>> not
> >>>>>>>>>> allow to implement an outputting loop where we never hand back
> >>>>>> control
> >>>>>>>> to
> >>>>>>>>>> the caller. At the moment, this contract can be easily broken
> >> and
> >>> is
> >>>>>>>> only
> >>>>>>>>>> mentioned loosely in the JavaDocs.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> The primary reason we handover the SourceOutput to the
> >>> SourceReader
> >>>>>> is
> >>>>>>>>>> because sometimes it is difficult for a SourceReader to emit one
> >>>>>>> record
> >>>>>>>> at
> >>>>>>>>>> a time. One example is some batched messaging systems which only
> >>>>>> have
> >>>>>>> an
> >>>>>>>>>> offset for the entire batch instead of individual messages in
> >> the
> >>>>>>>> batch. In
> >>>>>>>>>> that case, returning one record at a time would leave the
> >>>>>> SourceReader
> >>>>>>>> in
> >>>>>>>>>> an uncheckpointable state because they can only checkpoint at
> >> the
> >>>>>>> batch
> >>>>>>>>>> boundaries.
> >>>>>>>>>>
> >>>>>>>>>> Thanks,
> >>>>>>>>>>
> >>>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>>
> >>>>>>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <
> >>> trohrmann@apache.org
> >>>>>>>> <ma...@apache.org>> <trohrmann@apache.org <mailto:
> >>>>>>>> trohrmann@apache.org>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Hi everyone,
> >>>>>>>>>>
> >>>>>>>>>> thanks for drafting this FLIP. It reads very well.
> >>>>>>>>>>
> >>>>>>>>>> Concerning Dawid's proposal, I tend to agree. The boundedness
> >>> could
> >>>>>>> come
> >>>>>>>>>> from the source and tell the system how to treat the operator
> >>>>>>>> (scheduling
> >>>>>>>>>> wise). From a user's perspective it should be fine to get back a
> >>>>>>>> DataStream
> >>>>>>>>>> when calling env.source(boundedSource) if he does not need
> >> special
> >>>>>>>>>> operations defined on a BoundedDataStream. If he needs this,
> >> then
> >>>>>> one
> >>>>>>>> could
> >>>>>>>>>> use the method BoundedDataStream
> >> env.boundedSource(boundedSource).
> >>>>>>>>>>
> >>>>>>>>>> If possible, we could enforce the proper usage of
> >>>>>> env.boundedSource()
> >>>>>>> by
> >>>>>>>>>> introducing a BoundedSource type so that one cannot pass an
> >>>>>>>>>> unbounded source to it. That way users would not be able to
> >> shoot
> >>>>>>>>>> themselves in the foot.
> >>>>>>>>>>
> >>>>>>>>>> Maybe this has already been asked before but I was wondering why
> >>> the
> >>>>>>>>>> SourceReader interface has the method pollNext which hands the
> >>>>>>>>>> responsibility of outputting elements to the SourceReader
> >>>>>>>> implementation?
> >>>>>>>>>> Has this been done for backwards compatibility reasons with the
> >>> old
> >>>>>>>> source
> >>>>>>>>>> interface? If not, then one could define a Collection<E>
> >>>>>>>> getNextRecords()
> >>>>>>>>>> method which returns the currently retrieved records and then
> >> the
> >>>>>>> caller
> >>>>>>>>>> emits them outside of the SourceReader. That way the interface
> >>> would
> >>>>>>> not
> >>>>>>>>>> allow to implement an outputting loop where we never hand back
> >>>>>> control
> >>>>>>>> to
> >>>>>>>>>> the caller. At the moment, this contract can be easily broken
> >> and
> >>> is
> >>>>>>>> only
> >>>>>>>>>> mentioned loosely in the JavaDocs.
> >>>>>>>>>>
> >>>>>>>>>> Cheers,
> >>>>>>>>>> Till
> >>>>>>>>>>
> >>>>>>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <
> >>> jingsonglee0@gmail.com
> >>>>>>>> <ma...@gmail.com>> <jingsonglee0@gmail.com <mailto:
> >>>>>>>> jingsonglee0@gmail.com>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Hi all,
> >>>>>>>>>>
> >>>>>>>>>> I think current design is good.
> >>>>>>>>>>
> >>>>>>>>>> My understanding is:
> >>>>>>>>>>
> >>>>>>>>>> For execution mode: bounded mode and continuous mode, It's
> >> totally
> >>>>>>>>>> different. I don't think we have the ability to integrate the
> >> two
> >>>>>>> models
> >>>>>>>>>>
> >>>>>>>>>> at
> >>>>>>>>>>
> >>>>>>>>>> present. It's about scheduling, memory, algorithms, States, etc.
> >>> we
> >>>>>>>>>> shouldn't confuse them.
> >>>>>>>>>>
> >>>>>>>>>> For source capabilities: only bounded, only continuous, both
> >>> bounded
> >>>>>>> and
> >>>>>>>>>> continuous.
> >>>>>>>>>> I think Kafka is a source that can be ran both bounded
> >>>>>>>>>> and continuous execution mode.
> >>>>>>>>>> And Kafka with end offset should be ran both bounded
> >>>>>>>>>> and continuous execution mode.  Using apache Beam with Flink
> >>>>>> runner, I
> >>>>>>>>>>
> >>>>>>>>>> used
> >>>>>>>>>>
> >>>>>>>>>> to run a "bounded" Kafka in streaming mode. For our previous
> >>>>>>> DataStream,
> >>>>>>>>>>
> >>>>>>>>>> it
> >>>>>>>>>>
> >>>>>>>>>> is not necessarily required that the source cannot be bounded.
> >>>>>>>>>>
> >>>>>>>>>> So it is my thought for Dawid's question:
> >>>>>>>>>> 1.pass a bounded source to continuousSource() +1
> >>>>>>>>>> 2.pass a continuous source to boundedSource() -1, should throw
> >>>>>>>> exception.
> >>>>>>>>>>
> >>>>>>>>>> In StreamExecutionEnvironment, continuousSource and
> >> boundedSource
> >>>>>>> define
> >>>>>>>>>> the execution mode. It defines a clear boundary of execution
> >> mode.
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Jingsong Lee
> >>>>>>>>>>
> >>>>>>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com
> >>> <mailto:
> >>>>>>>> imjark@gmail.com>> <imjark@gmail.com <ma...@gmail.com>>
> >>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> I agree with Dawid's point that the boundedness information
> >> should
> >>>>>>> come
> >>>>>>>>>> from the source itself (e.g. the end timestamp), not through
> >>>>>>>>>> env.boundedSouce()/continuousSource().
> >>>>>>>>>> I think if we want to support something like `env.source()` that
> >>>>>>> derive
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> execution mode from source, `supportsBoundedness(Boundedness)`
> >>>>>>>>>> method is not enough, because we don't know whether it is
> >> bounded
> >>> or
> >>>>>>>>>>
> >>>>>>>>>> not.
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Jark
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
> >>>>>> dwysakowicz@apache.org
> >>>>>>>> <ma...@apache.org>> <dwysakowicz@apache.org <mailto:
> >>>>>>>> dwysakowicz@apache.org>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> One more thing. In the current proposal, with the
> >>>>>>>>>> supportsBoundedness(Boundedness) method and the boundedness
> >> coming
> >>>>>>>>>>
> >>>>>>>>>> from
> >>>>>>>>>>
> >>>>>>>>>> either continuousSource or boundedSource I could not find how
> >> this
> >>>>>>>>>> information is fed back to the SplitEnumerator.
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>>
> >>>>>>>>>> Dawid
> >>>>>>>>>>
> >>>>>>>>>> On 09/12/2019 13:52, Becket Qin wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi Dawid,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for the comments. This actually brings another relevant
> >>>>>>>>>>
> >>>>>>>>>> question
> >>>>>>>>>>
> >>>>>>>>>> about what does a "bounded source" imply. I actually had the
> >> same
> >>>>>>>>>> impression when I look at the Source API. Here is what I
> >>> understand
> >>>>>>>>>>
> >>>>>>>>>> after
> >>>>>>>>>>
> >>>>>>>>>> some discussion with Stephan. The bounded source has the
> >> following
> >>>>>>>>>>
> >>>>>>>>>> impacts.
> >>>>>>>>>>
> >>>>>>>>>> 1. API validity.
> >>>>>>>>>> - A bounded source generates a bounded stream so some operations
> >>>>>>>>>>
> >>>>>>>>>> that
> >>>>>>>>>>
> >>>>>>>>>> only
> >>>>>>>>>>
> >>>>>>>>>> works for bounded records would be performed, e.g. sort.
> >>>>>>>>>> - To expose these bounded stream only APIs, there are two
> >> options:
> >>>>>>>>>>       a. Add them to the DataStream API and throw exception if a
> >>>>>>>>>>
> >>>>>>>>>> method
> >>>>>>>>>>
> >>>>>>>>>> is
> >>>>>>>>>>
> >>>>>>>>>> called on an unbounded stream.
> >>>>>>>>>>       b. Create a BoundedDataStream class which is returned from
> >>>>>>>>>> env.boundedSource(), while DataStream is returned from
> >>>>>>>>>>
> >>>>>>>>>> env.continousSource().
> >>>>>>>>>>
> >>>>>>>>>> Note that this cannot be done by having single
> >>>>>>>>>>
> >>>>>>>>>> env.source(theSource)
> >>>>>>>>>>
> >>>>>>>>>> even
> >>>>>>>>>>
> >>>>>>>>>> the Source has a getBoundedness() method.
> >>>>>>>>>>
> >>>>>>>>>> 2. Scheduling
> >>>>>>>>>> - A bounded source could be computed stage by stage without
> >>>>>>>>>>
> >>>>>>>>>> bringing
> >>>>>>>>>>
> >>>>>>>>>> up
> >>>>>>>>>>
> >>>>>>>>>> all
> >>>>>>>>>>
> >>>>>>>>>> the tasks at the same time.
> >>>>>>>>>>
> >>>>>>>>>> 3. Operator behaviors
> >>>>>>>>>> - A bounded source indicates the records are finite so some
> >>>>>>>>>>
> >>>>>>>>>> operators
> >>>>>>>>>>
> >>>>>>>>>> can
> >>>>>>>>>>
> >>>>>>>>>> wait until it receives all the records before it starts the
> >>>>>>>>>>
> >>>>>>>>>> processing.
> >>>>>>>>>>
> >>>>>>>>>> In the above impact, only 1 is relevant to the API design. And
> >> the
> >>>>>>>>>>
> >>>>>>>>>> current
> >>>>>>>>>>
> >>>>>>>>>> proposal in FLIP-27 is following 1.b.
> >>>>>>>>>>
> >>>>>>>>>> // boundedness depends of source property, imo this should
> >> always
> >>>>>>>>>>
> >>>>>>>>>> be
> >>>>>>>>>>
> >>>>>>>>>> preferred
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> In your proposal, does DataStream have bounded stream only
> >>> methods?
> >>>>>>>>>>
> >>>>>>>>>> It
> >>>>>>>>>>
> >>>>>>>>>> looks it should have, otherwise passing a bounded Source to
> >>>>>>>>>>
> >>>>>>>>>> env.source()
> >>>>>>>>>>
> >>>>>>>>>> would be confusing. In that case, we will essentially do 1.a if
> >> an
> >>>>>>>>>> unbounded Source is created from env.source(unboundedSource).
> >>>>>>>>>>
> >>>>>>>>>> If we have the methods only supported for bounded streams in
> >>>>>>>>>>
> >>>>>>>>>> DataStream,
> >>>>>>>>>>
> >>>>>>>>>> it
> >>>>>>>>>>
> >>>>>>>>>> seems a little weird to have a separate BoundedDataStream
> >>>>>>>>>>
> >>>>>>>>>> interface.
> >>>>>>>>>>
> >>>>>>>>>> Am I understand it correctly?
> >>>>>>>>>>
> >>>>>>>>>> Thanks,
> >>>>>>>>>>
> >>>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> >>>>>>>>>>
> >>>>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> >>>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Hi all,
> >>>>>>>>>>
> >>>>>>>>>> Really well written proposal and very important one. I must
> >> admit
> >>>>>>>>>>
> >>>>>>>>>> I
> >>>>>>>>>>
> >>>>>>>>>> have
> >>>>>>>>>>
> >>>>>>>>>> not understood all the intricacies of it yet.
> >>>>>>>>>>
> >>>>>>>>>> One question I have though is about where does the information
> >>>>>>>>>>
> >>>>>>>>>> about
> >>>>>>>>>>
> >>>>>>>>>> boundedness come from. I think in most cases it is a property of
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> source. As you described it might be e.g. end offset, a flag
> >>>>>>>>>>
> >>>>>>>>>> should
> >>>>>>>>>>
> >>>>>>>>>> it
> >>>>>>>>>>
> >>>>>>>>>> monitor new splits etc. I think it would be a really nice use
> >> case
> >>>>>>>>>>
> >>>>>>>>>> to
> >>>>>>>>>>
> >>>>>>>>>> be
> >>>>>>>>>>
> >>>>>>>>>> able to say:
> >>>>>>>>>>
> >>>>>>>>>> new KafkaSource().readUntil(long timestamp),
> >>>>>>>>>>
> >>>>>>>>>> which could work as an "end offset". Moreover I think all
> >> Bounded
> >>>>>>>>>>
> >>>>>>>>>> sources
> >>>>>>>>>>
> >>>>>>>>>> support continuous mode, but no intrinsically continuous source
> >>>>>>>>>>
> >>>>>>>>>> support
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> Bounded mode. If I understood the proposal correctly it suggest
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> boundedness sort of "comes" from the outside of the source, from
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> invokation of either boundedStream or continousSource.
> >>>>>>>>>>
> >>>>>>>>>> I am wondering if it would make sense to actually change the
> >>>>>>>>>>
> >>>>>>>>>> method
> >>>>>>>>>>
> >>>>>>>>>> boolean Source#supportsBoundedness(Boundedness)
> >>>>>>>>>>
> >>>>>>>>>> to
> >>>>>>>>>>
> >>>>>>>>>> Boundedness Source#getBoundedness().
> >>>>>>>>>>
> >>>>>>>>>> As for the methods #boundedSource, #continousSource, assuming
> >> the
> >>>>>>>>>> boundedness is property of the source they do not affect how the
> >>>>>>>>>>
> >>>>>>>>>> enumerator
> >>>>>>>>>>
> >>>>>>>>>> works, but mostly how the dag is scheduled, right? I am not
> >>>>>>>>>>
> >>>>>>>>>> against
> >>>>>>>>>>
> >>>>>>>>>> those
> >>>>>>>>>>
> >>>>>>>>>> methods, but I think it is a very specific use case to actually
> >>>>>>>>>>
> >>>>>>>>>> override
> >>>>>>>>>>
> >>>>>>>>>> the property of the source. In general I would expect users to
> >>>>>>>>>>
> >>>>>>>>>> only
> >>>>>>>>>>
> >>>>>>>>>> call
> >>>>>>>>>>
> >>>>>>>>>> env.source(theSource), where the source tells if it is bounded
> >> or
> >>>>>>>>>>
> >>>>>>>>>> not. I
> >>>>>>>>>>
> >>>>>>>>>> would suggest considering following set of methods:
> >>>>>>>>>>
> >>>>>>>>>> // boundedness depends of source property, imo this should
> >> always
> >>>>>>>>>>
> >>>>>>>>>> be
> >>>>>>>>>>
> >>>>>>>>>> preferred
> >>>>>>>>>>
> >>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> // always continous execution, whether bounded or unbounded
> >> source
> >>>>>>>>>>
> >>>>>>>>>> DataStream<MyType> boundedStream =
> >> env.continousSource(theSource);
> >>>>>>>>>>
> >>>>>>>>>> // imo this would make sense if the BoundedDataStream provides
> >>>>>>>>>>
> >>>>>>>>>> additional features unavailable for continous mode
> >>>>>>>>>>
> >>>>>>>>>> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>>
> >>>>>>>>>> Dawid
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
> >>>>>>>>>>
> >>>>>>>>>> Thanks, Becket, for updating this.
> >>>>>>>>>>
> >>>>>>>>>> I agree with moving the aspects you mentioned into separate
> >> FLIPs
> >>>>>>>>>>
> >>>>>>>>>> -
> >>>>>>>>>>
> >>>>>>>>>> this
> >>>>>>>>>>
> >>>>>>>>>> one way becoming unwieldy in size.
> >>>>>>>>>>
> >>>>>>>>>> +1 to the FLIP in its current state. Its a very detailed
> >> write-up,
> >>>>>>>>>>
> >>>>>>>>>> nicely
> >>>>>>>>>>
> >>>>>>>>>> done!
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <becket.qin@gmail.com
> >>>>>>>> <ma...@gmail.com>> <becket.qin@gmail.com <mailto:
> >>>>>>>> becket.qin@gmail.com>>
> >>>>>>>>>>
> >>>>>>>>>> <
> >>>>>>>>>>
> >>>>>>>>>> becket.qin@gmail.com <ma...@gmail.com>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi all,
> >>>>>>>>>>
> >>>>>>>>>> Sorry for the long belated update. I have updated FLIP-27 wiki
> >>>>>>>>>>
> >>>>>>>>>> page
> >>>>>>>>>>
> >>>>>>>>>> with
> >>>>>>>>>>
> >>>>>>>>>> the latest proposals. Some noticeable changes include:
> >>>>>>>>>> 1. A new generic communication mechanism between SplitEnumerator
> >>>>>>>>>>
> >>>>>>>>>> and
> >>>>>>>>>>
> >>>>>>>>>> SourceReader.
> >>>>>>>>>> 2. Some detail API method signature changes.
> >>>>>>>>>>
> >>>>>>>>>> We left a few things out of this FLIP and will address them in
> >>>>>>>>>>
> >>>>>>>>>> separate
> >>>>>>>>>>
> >>>>>>>>>> FLIPs. Including:
> >>>>>>>>>> 1. Per split event time.
> >>>>>>>>>> 2. Event time alignment.
> >>>>>>>>>> 3. Fine grained failover for SplitEnumerator failure.
> >>>>>>>>>>
> >>>>>>>>>> Please let us know if you have any question.
> >>>>>>>>>>
> >>>>>>>>>> Thanks,
> >>>>>>>>>>
> >>>>>>>>>> Jiangjie (Becket) Qin
> >>>>>>>>>>
> >>>>>>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <sewen@apache.org
> >>>>>>> <mailto:
> >>>>>>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>> <
> >>>>>>>>>>
> >>>>>>>>>> sewen@apache.org <ma...@apache.org>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi  Łukasz!
> >>>>>>>>>>
> >>>>>>>>>> Becket and me are working hard on figuring out the last details
> >>>>>>>>>>
> >>>>>>>>>> and
> >>>>>>>>>>
> >>>>>>>>>> implementing the first PoC. We would update the FLIP hopefully
> >>>>>>>>>>
> >>>>>>>>>> next
> >>>>>>>>>>
> >>>>>>>>>> week.
> >>>>>>>>>>
> >>>>>>>>>> There is a fair chance that a first version of this will be in
> >>>>>>>>>>
> >>>>>>>>>> 1.10,
> >>>>>>>>>>
> >>>>>>>>>> but
> >>>>>>>>>>
> >>>>>>>>>> I
> >>>>>>>>>>
> >>>>>>>>>> think it will take another release to battle test it and migrate
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> connectors.
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Stephan
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <
> >> ljd@touk.pl
> >>>>>>>> <ma...@touk.pl>
> >>>>>>>>>>
> >>>>>>>>>> <
> >>>>>>>>>>
> >>>>>>>>>> ljd@touk.pl <ma...@touk.pl>>
> >>>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi,
> >>>>>>>>>>
> >>>>>>>>>> This proposal looks very promising for us. Do you have any plans
> >>>>>>>>>>
> >>>>>>>>>> in
> >>>>>>>>>>
> >>>>>>>>>> which
> >>>>>>>>>>
> >>>>>>>>>> Flink release it is going to be released? We are thinking on
> >>>>>>>>>>
> >>>>>>>>>> using a
> >>>>>>>>>>
> >>>>>>>>>> Data
> >>>>>>>>>>
> >>>>>>>>>> Set API for our future use cases but on the other hand Data Set
> >>>>>>>>>>
> >>>>>>>>>> API
> >>>>>>>>>>
> >>>>>>>>>> is
> >>>>>>>>>>
> >>>>>>>>>> going to be deprecated so using proposed bounded data streams
> >>>>>>>>>>
> >>>>>>>>>> solution
> >>>>>>>>>>
> >>>>>>>>>> could be more viable in the long term.
> >>>>>>>>>>
> >>>>>>>>>> Thanks,
> >>>>>>>>>> Łukasz
> >>>>>>>>>>
> >>>>>>>>>> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com
> >>>>>> <mailto:
> >>>>>>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
> >>>>>>>> thomas.weise@gmail.com>> <
> >>>>>>>>>>
> >>>>>>>>>> thomas.weise@gmail.com <ma...@gmail.com>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Thanks for putting together this proposal!
> >>>>>>>>>>
> >>>>>>>>>> I see that the "Per Split Event Time" and "Event Time Alignment"
> >>>>>>>>>>
> >>>>>>>>>> sections
> >>>>>>>>>>
> >>>>>>>>>> are still TBD.
> >>>>>>>>>>
> >>>>>>>>>> It would probably be good to flesh those out a bit before
> >>>>>>>>>>
> >>>>>>>>>> proceeding
> >>>>>>>>>>
> >>>>>>>>>> too
> >>>>>>>>>>
> >>>>>>>>>> far
> >>>>>>>>>>
> >>>>>>>>>> as the event time alignment will probably influence the
> >>>>>>>>>>
> >>>>>>>>>> interaction
> >>>>>>>>>>
> >>>>>>>>>> with
> >>>>>>>>>>
> >>>>>>>>>> the split reader, specifically ReaderStatus
> >>>>>>>>>>
> >>>>>>>>>> emitNext(SourceOutput<E>
> >>>>>>>>>>
> >>>>>>>>>> output).
> >>>>>>>>>>
> >>>>>>>>>> We currently have only one implementation for event time
> >> alignment
> >>>>>>>>>>
> >>>>>>>>>> in
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> Kinesis consumer. The synchronization in that case takes place
> >> as
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> last
> >>>>>>>>>>
> >>>>>>>>>> step before records are emitted downstream (RecordEmitter). With
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> currently proposed interfaces, the equivalent can be implemented
> >>>>>>>>>>
> >>>>>>>>>> in
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> reader loop, although note that in the Kinesis consumer the per
> >>>>>>>>>>
> >>>>>>>>>> shard
> >>>>>>>>>>
> >>>>>>>>>> threads push records.
> >>>>>>>>>>
> >>>>>>>>>> Synchronization has not been implemented for the Kafka consumer
> >>>>>>>>>>
> >>>>>>>>>> yet.
> >>>>>>>>>>
> >>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
> >>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675>
> >>>>>>>>>>
> >>>>>>>>>> When I looked at it, I realized that the implementation will
> >> look
> >>>>>>>>>>
> >>>>>>>>>> quite
> >>>>>>>>>>
> >>>>>>>>>> different
> >>>>>>>>>> from Kinesis because it needs to take place in the pull part,
> >>>>>>>>>>
> >>>>>>>>>> where
> >>>>>>>>>>
> >>>>>>>>>> records
> >>>>>>>>>>
> >>>>>>>>>> are taken from the Kafka client. Due to the multiplexing it
> >> cannot
> >>>>>>>>>>
> >>>>>>>>>> be
> >>>>>>>>>>
> >>>>>>>>>> done
> >>>>>>>>>>
> >>>>>>>>>> by blocking the split thread like it currently works for
> >> Kinesis.
> >>>>>>>>>>
> >>>>>>>>>> Reading
> >>>>>>>>>>
> >>>>>>>>>> from individual Kafka partitions needs to be controlled via
> >>>>>>>>>>
> >>>>>>>>>> pause/resume
> >>>>>>>>>>
> >>>>>>>>>> on the Kafka client.
> >>>>>>>>>>
> >>>>>>>>>> To take on that responsibility the split thread would need to be
> >>>>>>>>>>
> >>>>>>>>>> aware
> >>>>>>>>>>
> >>>>>>>>>> of
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>> watermarks or at least whether it should or should not continue
> >> to
> >>>>>>>>>>
> >>>>>>>>>> consume
> >>>>>>>>>>
> >>>>>>>>>> a given split and this may require a different SourceReader or
> >>>>>>>>>>
> >>>>>>>>>> SourceOutput
> >>>>>>>>>>
> >>>>>>>>>> interface.
> >>>>>>>>>>
> >>>>>>>>>> Thanks,
> >>>>>>>>>> Thomas
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com
> >>>>>> <mailto:
> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> mmyy1110@gmail.com
> >>>>>
> >>>>>> <
> >>>>>>>>>>
> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi Stephan,
> >>>>>>>>>>
> >>>>>>>>>> Thank you for feedback!
> >>>>>>>>>> Will take a look at your branch before public discussing.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <sewen@apache.org
> >>>>>>>> <ma...@apache.org>> <sewen@apache.org <mailto:
> >>> sewen@apache.org
> >>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> <
> >>>>>>>>>>
> >>>>>>>>>> sewen@apache.org <ma...@apache.org>>
> >>>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi Biao!
> >>>>>>>>>>
> >>>>>>>>>> Thanks for reviving this. I would like to join this discussion,
> >>>>>>>>>>
> >>>>>>>>>> but
> >>>>>>>>>>
> >>>>>>>>>> am
> >>>>>>>>>>
> >>>>>>>>>> quite occupied with the 1.9 release, so can we maybe pause this
> >>>>>>>>>>
> >>>>>>>>>> discussion
> >>>>>>>>>>
> >>>>>>>>>> for a week or so?
> >>>>>>>>>>
> >>>>>>>>>> In the meantime I can share some suggestion based on prior
> >>>>>>>>>>
> >>>>>>>>>> experiments:
> >>>>>>>>>>
> >>>>>>>>>> How to do watermarks / timestamp extractors in a simpler and
> >> more
> >>>>>>>>>>
> >>>>>>>>>> flexible
> >>>>>>>>>>
> >>>>>>>>>> way. I think that part is quite promising should be part of the
> >>>>>>>>>>
> >>>>>>>>>> new
> >>>>>>>>>>
> >>>>>>>>>> source
> >>>>>>>>>>
> >>>>>>>>>> interface.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> >>>>>>>> <
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> >>>>>>>> <
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Some experiments on how to build the source reader and its
> >>>>>>>>>>
> >>>>>>>>>> library
> >>>>>>>>>>
> >>>>>>>>>> for
> >>>>>>>>>>
> >>>>>>>>>> common threading/split patterns:
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> >>>>>>>> <
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Stephan
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mmyy1110@gmail.com
> >>>>>>> <mailto:
> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> mmyy1110@gmail.com
> >>>>>
> >>>>>> <
> >>>>>>>>>>
> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >>>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi devs,
> >>>>>>>>>>
> >>>>>>>>>> Since 1.9 is nearly released, I think we could get back to
> >>>>>>>>>>
> >>>>>>>>>> FLIP-27.
> >>>>>>>>>>
> >>>>>>>>>> I
> >>>>>>>>>>
> >>>>>>>>>> believe it should be included in 1.10.
> >>>>>>>>>>
> >>>>>>>>>> There are so many things mentioned in document of FLIP-27. [1] I
> >>>>>>>>>>
> >>>>>>>>>> think
> >>>>>>>>>>
> >>>>>>>>>> we'd better discuss them separately. However the wiki is not a
> >>>>>>>>>>
> >>>>>>>>>> good
> >>>>>>>>>>
> >>>>>>>>>> place
> >>>>>>>>>>
> >>>>>>>>>> to discuss. I wrote google doc about SplitReader API which
> >>>>>>>>>>
> >>>>>>>>>> misses
> >>>>>>>>>>
> >>>>>>>>>> some
> >>>>>>>>>>
> >>>>>>>>>> details in the document. [2]
> >>>>>>>>>>
> >>>>>>>>>> 1.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >>>>>>>> <
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 2.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> >>>>>>>> <
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com
> >>>>>> <mailto:
> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> mmyy1110@gmail.com
> >>>>>
> >>>>>> <
> >>>>>>>>>>
> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >>>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi Steven,
> >>>>>>>>>> Thank you for the feedback. Please take a look at the document
> >>>>>>>>>>
> >>>>>>>>>> FLIP-27
> >>>>>>>>>>
> >>>>>>>>>> <
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>>> <
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> which
> >>>>>>>>>>
> >>>>>>>>>> is updated recently. A lot of details of enumerator were added
> >>>>>>>>>>
> >>>>>>>>>> in
> >>>>>>>>>>
> >>>>>>>>>> this
> >>>>>>>>>>
> >>>>>>>>>> document. I think it would help.
> >>>>>>>>>>
> >>>>>>>>>> Steven Wu <stevenz3wu@gmail.com <ma...@gmail.com>>
> >> <
> >>>>>>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
> >>>>>>> stevenz3wu@gmail.com
> >>>>>>>> <ma...@gmail.com>> <stevenz3wu@gmail.com <mailto:
> >>>>>>>> stevenz3wu@gmail.com>>
> >>>>>>>>>>
> >>>>>>>>>> 于2019年3月28日周四
> >>>>>>>>>>
> >>>>>>>>>> 下午12:52写道:
> >>>>>>>>>>
> >>>>>>>>>> This proposal mentioned that SplitEnumerator might run on the
> >>>>>>>>>> JobManager or
> >>>>>>>>>> in a single task on a TaskManager.
> >>>>>>>>>>
> >>>>>>>>>> if enumerator is a single task on a taskmanager, then the job
> >>>>>>>>>>
> >>>>>>>>>> DAG
> >>>>>>>>>>
> >>>>>>>>>> can
> >>>>>>>>>>
> >>>>>>>>>> never
> >>>>>>>>>> been embarrassingly parallel anymore. That will nullify the
> >>>>>>>>>>
> >>>>>>>>>> leverage
> >>>>>>>>>>
> >>>>>>>>>> of
> >>>>>>>>>>
> >>>>>>>>>> fine-grained recovery for embarrassingly parallel jobs.
> >>>>>>>>>>
> >>>>>>>>>> It's not clear to me what's the implication of running
> >>>>>>>>>>
> >>>>>>>>>> enumerator
> >>>>>>>>>>
> >>>>>>>>>> on
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> jobmanager. So I will leave that out for now.
> >>>>>>>>>>
> >>>>>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com
> >>>>>> <mailto:
> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> mmyy1110@gmail.com
> >>>>>
> >>>>>> <
> >>>>>>>>>>
> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >>>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi Stephan & Piotrek,
> >>>>>>>>>>
> >>>>>>>>>> Thank you for feedback.
> >>>>>>>>>>
> >>>>>>>>>> It seems that there are a lot of things to do in community.
> >>>>>>>>>>
> >>>>>>>>>> I
> >>>>>>>>>>
> >>>>>>>>>> am
> >>>>>>>>>>
> >>>>>>>>>> just
> >>>>>>>>>>
> >>>>>>>>>> afraid that this discussion may be forgotten since there so
> >>>>>>>>>>
> >>>>>>>>>> many
> >>>>>>>>>>
> >>>>>>>>>> proposals
> >>>>>>>>>>
> >>>>>>>>>> recently.
> >>>>>>>>>> Anyway, wish to see the split topics soon :)
> >>>>>>>>>>
> >>>>>>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:
> >>> piotr@da-platform.com
> >>>>>>>>
> >>>>>>> <
> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
> >>>>>>>> piotr@da-platform.com <ma...@da-platform.com>>
> >>>>>>>>>>
> >>>>>>>>>> 于2019年1月24日周四
> >>>>>>>>>>
> >>>>>>>>>> 下午8:21写道:
> >>>>>>>>>>
> >>>>>>>>>> Hi Biao!
> >>>>>>>>>>
> >>>>>>>>>> This discussion was stalled because of preparations for
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> open
> >>>>>>>>>>
> >>>>>>>>>> sourcing
> >>>>>>>>>>
> >>>>>>>>>> & merging Blink. I think before creating the tickets we
> >>>>>>>>>>
> >>>>>>>>>> should
> >>>>>>>>>>
> >>>>>>>>>> split this
> >>>>>>>>>>
> >>>>>>>>>> discussion into topics/areas outlined by Stephan and
> >>>>>>>>>>
> >>>>>>>>>> create
> >>>>>>>>>>
> >>>>>>>>>> Flips
> >>>>>>>>>>
> >>>>>>>>>> for
> >>>>>>>>>>
> >>>>>>>>>> that.
> >>>>>>>>>>
> >>>>>>>>>> I think there is no chance for this to be completed in
> >>>>>>>>>>
> >>>>>>>>>> couple
> >>>>>>>>>>
> >>>>>>>>>> of
> >>>>>>>>>>
> >>>>>>>>>> remaining
> >>>>>>>>>>
> >>>>>>>>>> weeks/1 month before 1.8 feature freeze, however it would
> >>>>>>>>>>
> >>>>>>>>>> be
> >>>>>>>>>>
> >>>>>>>>>> good
> >>>>>>>>>>
> >>>>>>>>>> to aim
> >>>>>>>>>>
> >>>>>>>>>> with those changes for 1.9.
> >>>>>>>>>>
> >>>>>>>>>> Piotrek
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com <mailto:
> >>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> >> mmyy1110@gmail.com
> >>>>>
> >>>>>> <
> >>>>>>>>>>
> >>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >>>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>> Hi community,
> >>>>>>>>>> The summary of Stephan makes a lot sense to me. It is
> >>>>>>>>>>
> >>>>>>>>>> much
> >>>>>>>>>>
> >>>>>>>>>> clearer
> >>>>>>>>>>
> >>>>>>>>>> indeed
> >>>>>>>>>>
> >>>>>>>>>> after splitting the complex topic into small ones.
> >>>>>>>>>> I was wondering is there any detail plan for next step?
> >>>>>>>>>>
> >>>>>>>>>> If
> >>>>>>>>>>
> >>>>>>>>>> not,
> >>>>>>>>>>
> >>>>>>>>>> I
> >>>>>>>>>>
> >>>>>>>>>> would
> >>>>>>>>>>
> >>>>>>>>>> like to push this thing forward by creating some JIRA
> >>>>>>>>>>
> >>>>>>>>>> issues.
> >>>>>>>>>>
> >>>>>>>>>> Another question is that should version 1.8 include
> >>>>>>>>>>
> >>>>>>>>>> these
> >>>>>>>>>>
> >>>>>>>>>> features?
> >>>>>>>>>>
> >>>>>>>>>> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <
> >>>>>>>> sewen@apache.org <ma...@apache.org>> <sewen@apache.org
> >>>> <mailto:
> >>>>>>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>>
> >>>>>>>> 于2018年12月1日周六
> >>>>>>>>>>
> >>>>>>>>>> 上午4:20写道:
> >>>>>>>>>>
> >>>>>>>>>> Thanks everyone for the lively discussion. Let me try
> >>>>>>>>>>
> >>>>>>>>>> to
> >>>>>>>>>>
> >>>>>>>>>> summarize
> >>>>>>>>>>
> >>>>>>>>>> where I
> >>>>>>>>>>
> >>>>>>>>>> see convergence in the discussion and open issues.
> >>>>>>>>>> I'll try to group this by design aspect of the source.
> >>>>>>>>>>
> >>>>>>>>>> Please
> >>>>>>>>>>
> >>>>>>>>>> let me
> >>>>>>>>>>
> >>>>>>>>>> know
> >>>>>>>>>>
> >>>>>>>>>> if I got things wrong or missed something crucial here.
> >>>>>>>>>>
> >>>>>>>>>> For issues 1-3, if the below reflects the state of the
> >>>>>>>>>>
> >>>>>>>>>> discussion, I
> >>>>>>>>>>
> >>>>>>>>>> would
> >>>>>>>>>>
> >>>>>>>>>> try and update the FLIP in the next days.
> >>>>>>>>>> For the remaining ones we need more discussion.
> >>>>>>>>>>
> >>>>>>>>>> I would suggest to fork each of these aspects into a
> >>>>>>>>>>
> >>>>>>>>>> separate
> >>>>>>>>>>
> >>>>>>>>>> mail
> >>>>>>>>>>
> >>>>>>>>>> thread,
> >>>>>>>>>>
> >>>>>>>>>> or will loose sight of the individual aspects.
> >>>>>>>>>>
> >>>>>>>>>> *(1) Separation of Split Enumerator and Split Reader*
> >>>>>>>>>>
> >>>>>>>>>> - All seem to agree this is a good thing
> >>>>>>>>>> - Split Enumerator could in the end live on JobManager
> >>>>>>>>>>
> >>>>>>>>>> (and
> >>>>>>>>>>
> >>>>>>>>>> assign
> >>>>>>>>>>
> >>>>>>>>>> splits
> >>>>>>>>>>
> >>>>>>>>>> via RPC) or in a task (and assign splits via data
> >>>>>>>>>>
> >>>>>>>>>> streams)
> >>>>>>>>>>
> >>>>>>>>>> - this discussion is orthogonal and should come later,
> >>>>>>>>>>
> >>>>>>>>>> when
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> interface
> >>>>>>>>>>
> >>>>>>>>>> is agreed upon.
> >>>>>>>>>>
> >>>>>>>>>> *(2) Split Readers for one or more splits*
> >>>>>>>>>>
> >>>>>>>>>> - Discussion seems to agree that we need to support
> >>>>>>>>>>
> >>>>>>>>>> one
> >>>>>>>>>>
> >>>>>>>>>> reader
> >>>>>>>>>>
> >>>>>>>>>> that
> >>>>>>>>>>
> >>>>>>>>>> possibly handles multiple splits concurrently.
> >>>>>>>>>> - The requirement comes from sources where one
> >>>>>>>>>>
> >>>>>>>>>> poll()-style
> >>>>>>>>>>
> >>>>>>>>>> call
> >>>>>>>>>>
> >>>>>>>>>> fetches
> >>>>>>>>>>
> >>>>>>>>>> data from different splits / partitions
> >>>>>>>>>>     --> example sources that require that would be for
> >>>>>>>>>>
> >>>>>>>>>> example
> >>>>>>>>>>
> >>>>>>>>>> Kafka,
> >>>>>>>>>>
> >>>>>>>>>> Pravega, Pulsar
> >>>>>>>>>>
> >>>>>>>>>> - Could have one split reader per source, or multiple
> >>>>>>>>>>
> >>>>>>>>>> split
> >>>>>>>>>>
> >>>>>>>>>> readers
> >>>>>>>>>>
> >>>>>>>>>> that
> >>>>>>>>>>
> >>>>>>>>>> share the "poll()" function
> >>>>>>>>>> - To not make it too complicated, we can start with
> >>>>>>>>>>
> >>>>>>>>>> thinking
> >>>>>>>>>>
> >>>>>>>>>> about
> >>>>>>>>>>
> >>>>>>>>>> one
> >>>>>>>>>>
> >>>>>>>>>> split reader for all splits initially and see if that
> >>>>>>>>>>
> >>>>>>>>>> covers
> >>>>>>>>>>
> >>>>>>>>>> all
> >>>>>>>>>>
> >>>>>>>>>> requirements
> >>>>>>>>>>
> >>>>>>>>>> *(3) Threading model of the Split Reader*
> >>>>>>>>>>
> >>>>>>>>>> - Most active part of the discussion ;-)
> >>>>>>>>>>
> >>>>>>>>>> - A non-blocking way for Flink's task code to interact
> >>>>>>>>>>
> >>>>>>>>>> with
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> source
> >>>>>>>>>>
> >>>>>>>>>> is
> >>>>>>>>>>
> >>>>>>>>>> needed in order to a task runtime code based on a
> >>>>>>>>>> single-threaded/actor-style task design
> >>>>>>>>>>     --> I personally am a big proponent of that, it will
> >>>>>>>>>>
> >>>>>>>>>> help
> >>>>>>>>>>
> >>>>>>>>>> with
> >>>>>>>>>>
> >>>>>>>>>> well-behaved checkpoints, efficiency, and simpler yet
> >>>>>>>>>>
> >>>>>>>>>> more
> >>>>>>>>>>
> >>>>>>>>>> robust
> >>>>>>>>>>
> >>>>>>>>>> runtime
> >>>>>>>>>>
> >>>>>>>>>> code
> >>>>>>>>>>
> >>>>>>>>>> - Users care about simple abstraction, so as a
> >>>>>>>>>>
> >>>>>>>>>> subclass
> >>>>>>>>>>
> >>>>>>>>>> of
> >>>>>>>>>>
> >>>>>>>>>> SplitReader
> >>>>>>>>>>
> >>>>>>>>>> (non-blocking / async) we need to have a
> >>>>>>>>>>
> >>>>>>>>>> BlockingSplitReader
> >>>>>>>>>>
> >>>>>>>>>> which
> >>>>>>>>>>
> >>>>>>>>>> will
> >>>>>>>>>>
> >>>>>>>>>> form the basis of most source implementations.
> >>>>>>>>>>
> >>>>>>>>>> BlockingSplitReader
> >>>>>>>>>>
> >>>>>>>>>> lets
> >>>>>>>>>>
> >>>>>>>>>> users do blocking simple poll() calls.
> >>>>>>>>>> - The BlockingSplitReader would spawn a thread (or
> >>>>>>>>>>
> >>>>>>>>>> more)
> >>>>>>>>>>
> >>>>>>>>>> and
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> thread(s) can make blocking calls and hand over data
> >>>>>>>>>>
> >>>>>>>>>> buffers
> >>>>>>>>>>
> >>>>>>>>>> via
> >>>>>>>>>>
> >>>>>>>>>> a
> >>>>>>>>>>
> >>>>>>>>>> blocking
> >>>>>>>>>>
> >>>>>>>>>> queue
> >>>>>>>>>> - This should allow us to cover both, a fully async
> >>>>>>>>>>
> >>>>>>>>>> runtime,
> >>>>>>>>>>
> >>>>>>>>>> and a
> >>>>>>>>>>
> >>>>>>>>>> simple
> >>>>>>>>>>
> >>>>>>>>>> blocking interface for users.
> >>>>>>>>>> - This is actually very similar to how the Kafka
> >>>>>>>>>>
> >>>>>>>>>> connectors
> >>>>>>>>>>
> >>>>>>>>>> work.
> >>>>>>>>>>
> >>>>>>>>>> Kafka
> >>>>>>>>>>
> >>>>>>>>>> 9+ with one thread, Kafka 8 with multiple threads
> >>>>>>>>>>
> >>>>>>>>>> - On the base SplitReader (the async one), the
> >>>>>>>>>>
> >>>>>>>>>> non-blocking
> >>>>>>>>>>
> >>>>>>>>>> method
> >>>>>>>>>>
> >>>>>>>>>> that
> >>>>>>>>>>
> >>>>>>>>>> gets the next chunk of data would signal data
> >>>>>>>>>>
> >>>>>>>>>> availability
> >>>>>>>>>>
> >>>>>>>>>> via
> >>>>>>>>>>
> >>>>>>>>>> a
> >>>>>>>>>>
> >>>>>>>>>> CompletableFuture, because that gives the best
> >>>>>>>>>>
> >>>>>>>>>> flexibility
> >>>>>>>>>>
> >>>>>>>>>> (can
> >>>>>>>>>>
> >>>>>>>>>> await
> >>>>>>>>>>
> >>>>>>>>>> completion or register notification handlers).
> >>>>>>>>>> - The source task would register a "thenHandle()" (or
> >>>>>>>>>>
> >>>>>>>>>> similar)
> >>>>>>>>>>
> >>>>>>>>>> on the
> >>>>>>>>>>
> >>>>>>>>>> future to put a "take next data" task into the
> >>>>>>>>>>
> >>>>>>>>>> actor-style
> >>>>>>>>>>
> >>>>>>>>>> mailbox
> >>>>>>>>>>
> >>>>>>>>>> *(4) Split Enumeration and Assignment*
> >>>>>>>>>>
> >>>>>>>>>> - Splits may be generated lazily, both in cases where
> >>>>>>>>>>
> >>>>>>>>>> there
> >>>>>>>>>>
> >>>>>>>>>> is a
> >>>>>>>>>>
> >>>>>>>>>> limited
> >>>>>>>>>>
> >>>>>>>>>> number of splits (but very many), or splits are
> >>>>>>>>>>
> >>>>>>>>>> discovered
> >>>>>>>>>>
> >>>>>>>>>> over
> >>>>>>>>>>
> >>>>>>>>>> time
> >>>>>>>>>>
> >>>>>>>>>> - Assignment should also be lazy, to get better load
> >>>>>>>>>>
> >>>>>>>>>> balancing
> >>>>>>>>>>
> >>>>>>>>>> - Assignment needs support locality preferences
> >>>>>>>>>>
> >>>>>>>>>> - Possible design based on discussion so far:
> >>>>>>>>>>
> >>>>>>>>>>     --> SplitReader has a method "addSplits(SplitT...)"
> >>>>>>>>>>
> >>>>>>>>>> to
> >>>>>>>>>>
> >>>>>>>>>> add
> >>>>>>>>>>
> >>>>>>>>>> one or
> >>>>>>>>>>
> >>>>>>>>>> more
> >>>>>>>>>>
> >>>>>>>>>> splits. Some split readers might assume they have only
> >>>>>>>>>>
> >>>>>>>>>> one
> >>>>>>>>>>
> >>>>>>>>>> split
> >>>>>>>>>>
> >>>>>>>>>> ever,
> >>>>>>>>>>
> >>>>>>>>>> concurrently, others assume multiple splits. (Note:
> >>>>>>>>>>
> >>>>>>>>>> idea
> >>>>>>>>>>
> >>>>>>>>>> behind
> >>>>>>>>>>
> >>>>>>>>>> being
> >>>>>>>>>>
> >>>>>>>>>> able
> >>>>>>>>>>
> >>>>>>>>>> to add multiple splits at the same time is to ease
> >>>>>>>>>>
> >>>>>>>>>> startup
> >>>>>>>>>>
> >>>>>>>>>> where
> >>>>>>>>>>
> >>>>>>>>>> multiple
> >>>>>>>>>>
> >>>>>>>>>> splits may be assigned instantly.)
> >>>>>>>>>>     --> SplitReader has a context object on which it can
> >>>>>>>>>>
> >>>>>>>>>> call
> >>>>>>>>>>
> >>>>>>>>>> indicate
> >>>>>>>>>>
> >>>>>>>>>> when
> >>>>>>>>>>
> >>>>>>>>>> splits are completed. The enumerator gets that
> >>>>>>>>>>
> >>>>>>>>>> notification and
> >>>>>>>>>>
> >>>>>>>>>> can
> >>>>>>>>>>
> >>>>>>>>>> use
> >>>>>>>>>>
> >>>>>>>>>> to
> >>>>>>>>>>
> >>>>>>>>>> decide when to assign new splits. This should help both
> >>>>>>>>>>
> >>>>>>>>>> in
> >>>>>>>>>>
> >>>>>>>>>> cases
> >>>>>>>>>>
> >>>>>>>>>> of
> >>>>>>>>>>
> >>>>>>>>>> sources
> >>>>>>>>>>
> >>>>>>>>>> that take splits lazily (file readers) and in case the
> >>>>>>>>>>
> >>>>>>>>>> source
> >>>>>>>>>>
> >>>>>>>>>> needs to
> >>>>>>>>>>
> >>>>>>>>>> preserve a partial order between splits (Kinesis,
> >>>>>>>>>>
> >>>>>>>>>> Pravega,
> >>>>>>>>>>
> >>>>>>>>>> Pulsar may
> >>>>>>>>>>
> >>>>>>>>>> need
> >>>>>>>>>>
> >>>>>>>>>> that).
> >>>>>>>>>>     --> SplitEnumerator gets notification when
> >>>>>>>>>>
> >>>>>>>>>> SplitReaders
> >>>>>>>>>>
> >>>>>>>>>> start
> >>>>>>>>>>
> >>>>>>>>>> and
> >>>>>>>>>>
> >>>>>>>>>> when
> >>>>>>>>>>
> >>>>>>>>>> they finish splits. They can decide at that moment to
> >>>>>>>>>>
> >>>>>>>>>> push
> >>>>>>>>>>
> >>>>>>>>>> more
> >>>>>>>>>>
> >>>>>>>>>> splits
> >>>>>>>>>>
> >>>>>>>>>> to
> >>>>>>>>>>
> >>>>>>>>>> that reader
> >>>>>>>>>>     --> The SplitEnumerator should probably be aware of
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> source
> >>>>>>>>>>
> >>>>>>>>>> parallelism, to build its initial distribution.
> >>>>>>>>>>
> >>>>>>>>>> - Open question: Should the source expose something
> >>>>>>>>>>
> >>>>>>>>>> like
> >>>>>>>>>>
> >>>>>>>>>> "host
> >>>>>>>>>>
> >>>>>>>>>> preferences", so that yarn/mesos/k8s can take this into
> >>>>>>>>>>
> >>>>>>>>>> account
> >>>>>>>>>>
> >>>>>>>>>> when
> >>>>>>>>>>
> >>>>>>>>>> selecting a node to start a TM on?
> >>>>>>>>>>
> >>>>>>>>>> *(5) Watermarks and event time alignment*
> >>>>>>>>>>
> >>>>>>>>>> - Watermark generation, as well as idleness, needs to
> >>>>>>>>>>
> >>>>>>>>>> be
> >>>>>>>>>>
> >>>>>>>>>> per
> >>>>>>>>>>
> >>>>>>>>>> split
> >>>>>>>>>>
> >>>>>>>>>> (like
> >>>>>>>>>>
> >>>>>>>>>> currently in the Kafka Source, per partition)
> >>>>>>>>>> - It is desirable to support optional
> >>>>>>>>>>
> >>>>>>>>>> event-time-alignment,
> >>>>>>>>>>
> >>>>>>>>>> meaning
> >>>>>>>>>>
> >>>>>>>>>> that
> >>>>>>>>>>
> >>>>>>>>>> splits that are ahead are back-pressured or temporarily
> >>>>>>>>>>
> >>>>>>>>>> unsubscribed
> >>>>>>>>>>
> >>>>>>>>>> - I think i would be desirable to encapsulate
> >>>>>>>>>>
> >>>>>>>>>> watermark
> >>>>>>>>>>
> >>>>>>>>>> generation
> >>>>>>>>>>
> >>>>>>>>>> logic
> >>>>>>>>>>
> >>>>>>>>>> in watermark generators, for a separation of concerns.
> >>>>>>>>>>
> >>>>>>>>>> The
> >>>>>>>>>>
> >>>>>>>>>> watermark
> >>>>>>>>>>
> >>>>>>>>>> generators should run per split.
> >>>>>>>>>> - Using watermark generators would also help with
> >>>>>>>>>>
> >>>>>>>>>> another
> >>>>>>>>>>
> >>>>>>>>>> problem of
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> suggested interface, namely supporting non-periodic
> >>>>>>>>>>
> >>>>>>>>>> watermarks
> >>>>>>>>>>
> >>>>>>>>>> efficiently.
> >>>>>>>>>>
> >>>>>>>>>> - Need a way to "dispatch" next record to different
> >>>>>>>>>>
> >>>>>>>>>> watermark
> >>>>>>>>>>
> >>>>>>>>>> generators
> >>>>>>>>>>
> >>>>>>>>>> - Need a way to tell SplitReader to "suspend" a split
> >>>>>>>>>>
> >>>>>>>>>> until a
> >>>>>>>>>>
> >>>>>>>>>> certain
> >>>>>>>>>>
> >>>>>>>>>> watermark is reached (event time backpressure)
> >>>>>>>>>> - This would in fact be not needed (and thus simpler)
> >>>>>>>>>>
> >>>>>>>>>> if
> >>>>>>>>>>
> >>>>>>>>>> we
> >>>>>>>>>>
> >>>>>>>>>> had
> >>>>>>>>>>
> >>>>>>>>>> a
> >>>>>>>>>>
> >>>>>>>>>> SplitReader per split and may be a reason to re-open
> >>>>>>>>>>
> >>>>>>>>>> that
> >>>>>>>>>>
> >>>>>>>>>> discussion
> >>>>>>>>>>
> >>>>>>>>>> *(6) Watermarks across splits and in the Split
> >>>>>>>>>>
> >>>>>>>>>> Enumerator*
> >>>>>>>>>>
> >>>>>>>>>> - The split enumerator may need some watermark
> >>>>>>>>>>
> >>>>>>>>>> awareness,
> >>>>>>>>>>
> >>>>>>>>>> which
> >>>>>>>>>>
> >>>>>>>>>> should
> >>>>>>>>>>
> >>>>>>>>>> be
> >>>>>>>>>>
> >>>>>>>>>> purely based on split metadata (like create timestamp
> >>>>>>>>>>
> >>>>>>>>>> of
> >>>>>>>>>>
> >>>>>>>>>> file
> >>>>>>>>>>
> >>>>>>>>>> splits)
> >>>>>>>>>>
> >>>>>>>>>> - If there are still more splits with overlapping
> >>>>>>>>>>
> >>>>>>>>>> event
> >>>>>>>>>>
> >>>>>>>>>> time
> >>>>>>>>>>
> >>>>>>>>>> range
> >>>>>>>>>>
> >>>>>>>>>> for
> >>>>>>>>>>
> >>>>>>>>>> a
> >>>>>>>>>>
> >>>>>>>>>> split reader, then that split reader should not advance
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> watermark
> >>>>>>>>>>
> >>>>>>>>>> within the split beyond the overlap boundary. Otherwise
> >>>>>>>>>>
> >>>>>>>>>> future
> >>>>>>>>>>
> >>>>>>>>>> splits
> >>>>>>>>>>
> >>>>>>>>>> will
> >>>>>>>>>>
> >>>>>>>>>> produce late data.
> >>>>>>>>>>
> >>>>>>>>>> - One way to approach this could be that the split
> >>>>>>>>>>
> >>>>>>>>>> enumerator
> >>>>>>>>>>
> >>>>>>>>>> may
> >>>>>>>>>>
> >>>>>>>>>> send
> >>>>>>>>>>
> >>>>>>>>>> watermarks to the readers, and the readers cannot emit
> >>>>>>>>>>
> >>>>>>>>>> watermarks
> >>>>>>>>>>
> >>>>>>>>>> beyond
> >>>>>>>>>>
> >>>>>>>>>> that received watermark.
> >>>>>>>>>> - Many split enumerators would simply immediately send
> >>>>>>>>>>
> >>>>>>>>>> Long.MAX
> >>>>>>>>>>
> >>>>>>>>>> out
> >>>>>>>>>>
> >>>>>>>>>> and
> >>>>>>>>>>
> >>>>>>>>>> leave the progress purely to the split readers.
> >>>>>>>>>>
> >>>>>>>>>> - For event-time alignment / split back pressure, this
> >>>>>>>>>>
> >>>>>>>>>> begs
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> question
> >>>>>>>>>>
> >>>>>>>>>> how we can avoid deadlocks that may arise when splits
> >>>>>>>>>>
> >>>>>>>>>> are
> >>>>>>>>>>
> >>>>>>>>>> suspended
> >>>>>>>>>>
> >>>>>>>>>> for
> >>>>>>>>>>
> >>>>>>>>>> event time back pressure,
> >>>>>>>>>>
> >>>>>>>>>> *(7) Batch and streaming Unification*
> >>>>>>>>>>
> >>>>>>>>>> - Functionality wise, the above design should support
> >>>>>>>>>>
> >>>>>>>>>> both
> >>>>>>>>>>
> >>>>>>>>>> - Batch often (mostly) does not care about reading "in
> >>>>>>>>>>
> >>>>>>>>>> order"
> >>>>>>>>>>
> >>>>>>>>>> and
> >>>>>>>>>>
> >>>>>>>>>> generating watermarks
> >>>>>>>>>>     --> Might use different enumerator logic that is
> >>>>>>>>>>
> >>>>>>>>>> more
> >>>>>>>>>>
> >>>>>>>>>> locality
> >>>>>>>>>>
> >>>>>>>>>> aware
> >>>>>>>>>>
> >>>>>>>>>> and ignores event time order
> >>>>>>>>>>     --> Does not generate watermarks
> >>>>>>>>>> - Would be great if bounded sources could be
> >>>>>>>>>>
> >>>>>>>>>> identified
> >>>>>>>>>>
> >>>>>>>>>> at
> >>>>>>>>>>
> >>>>>>>>>> compile
> >>>>>>>>>>
> >>>>>>>>>> time,
> >>>>>>>>>>
> >>>>>>>>>> so that "env.addBoundedSource(...)" is type safe and
> >>>>>>>>>>
> >>>>>>>>>> can
> >>>>>>>>>>
> >>>>>>>>>> return a
> >>>>>>>>>>
> >>>>>>>>>> "BoundedDataStream".
> >>>>>>>>>> - Possible to defer this discussion until later
> >>>>>>>>>>
> >>>>>>>>>> *Miscellaneous Comments*
> >>>>>>>>>>
> >>>>>>>>>> - Should the source have a TypeInformation for the
> >>>>>>>>>>
> >>>>>>>>>> produced
> >>>>>>>>>>
> >>>>>>>>>> type,
> >>>>>>>>>>
> >>>>>>>>>> instead
> >>>>>>>>>>
> >>>>>>>>>> of a serializer? We need a type information in the
> >>>>>>>>>>
> >>>>>>>>>> stream
> >>>>>>>>>>
> >>>>>>>>>> anyways, and
> >>>>>>>>>>
> >>>>>>>>>> can
> >>>>>>>>>>
> >>>>>>>>>> derive the serializer from that. Plus, creating the
> >>>>>>>>>>
> >>>>>>>>>> serializer
> >>>>>>>>>>
> >>>>>>>>>> should
> >>>>>>>>>>
> >>>>>>>>>> respect the ExecutionConfig.
> >>>>>>>>>>
> >>>>>>>>>> - The TypeSerializer interface is very powerful but
> >>>>>>>>>>
> >>>>>>>>>> also
> >>>>>>>>>>
> >>>>>>>>>> not
> >>>>>>>>>>
> >>>>>>>>>> easy to
> >>>>>>>>>>
> >>>>>>>>>> implement. Its purpose is to handle data super
> >>>>>>>>>>
> >>>>>>>>>> efficiently,
> >>>>>>>>>>
> >>>>>>>>>> support
> >>>>>>>>>>
> >>>>>>>>>> flexible ways of evolution, etc.
> >>>>>>>>>> For metadata I would suggest to look at the
> >>>>>>>>>>
> >>>>>>>>>> SimpleVersionedSerializer
> >>>>>>>>>>
> >>>>>>>>>> instead, which is used for example for checkpoint
> >>>>>>>>>>
> >>>>>>>>>> master
> >>>>>>>>>>
> >>>>>>>>>> hooks,
> >>>>>>>>>>
> >>>>>>>>>> or for
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> streaming file sink. I think that is is a good match
> >>>>>>>>>>
> >>>>>>>>>> for
> >>>>>>>>>>
> >>>>>>>>>> cases
> >>>>>>>>>>
> >>>>>>>>>> where
> >>>>>>>>>>
> >>>>>>>>>> we
> >>>>>>>>>>
> >>>>>>>>>> do
> >>>>>>>>>>
> >>>>>>>>>> not need more than ser/deser (no copy, etc.) and don't
> >>>>>>>>>>
> >>>>>>>>>> need to
> >>>>>>>>>>
> >>>>>>>>>> push
> >>>>>>>>>>
> >>>>>>>>>> versioning out of the serialization paths for best
> >>>>>>>>>>
> >>>>>>>>>> performance
> >>>>>>>>>>
> >>>>>>>>>> (as in
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> TypeSerializer)
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> >>>>>>>>>>
> >>>>>>>>>> k.kloudas@data-artisans.com>
> >>>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Hi Biao,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for the answer!
> >>>>>>>>>>
> >>>>>>>>>> So given the multi-threaded readers, now we have as
> >>>>>>>>>>
> >>>>>>>>>> open
> >>>>>>>>>>
> >>>>>>>>>> questions:
> >>>>>>>>>>
> >>>>>>>>>> 1) How do we let the checkpoints pass through our
> >>>>>>>>>>
> >>>>>>>>>> multi-threaded
> >>>>>>>>>>
> >>>>>>>>>> reader
> >>>>>>>>>>
> >>>>>>>>>> operator?
> >>>>>>>>>>
> >>>>>>>>>> 2) Do we have separate reader and source operators or
> >>>>>>>>>>
> >>>>>>>>>> not? In
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> strategy
> >>>>>>>>>>
> >>>>>>>>>> that has a separate source, the source operator has a
> >>>>>>>>>>
> >>>>>>>>>> parallelism of
> >>>>>>>>>>
> >>>>>>>>>> 1
> >>>>>>>>>>
> >>>>>>>>>> and
> >>>>>>>>>>
> >>>>>>>>>> is responsible for split recovery only.
> >>>>>>>>>>
> >>>>>>>>>> For the first one, given also the constraints
> >>>>>>>>>>
> >>>>>>>>>> (blocking,
> >>>>>>>>>>
> >>>>>>>>>> finite
> >>>>>>>>>>
> >>>>>>>>>> queues,
> >>>>>>>>>>
> >>>>>>>>>> etc), I do not have an answer yet.
> >>>>>>>>>>
> >>>>>>>>>> For the 2nd, I think that we should go with separate
> >>>>>>>>>>
> >>>>>>>>>> operators
> >>>>>>>>>>
> >>>>>>>>>> for
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> source and the readers, for the following reasons:
> >>>>>>>>>>
> >>>>>>>>>> 1) This is more aligned with a potential future
> >>>>>>>>>>
> >>>>>>>>>> improvement
> >>>>>>>>>>
> >>>>>>>>>> where the
> >>>>>>>>>>
> >>>>>>>>>> split
> >>>>>>>>>>
> >>>>>>>>>> discovery becomes a responsibility of the JobManager
> >>>>>>>>>>
> >>>>>>>>>> and
> >>>>>>>>>>
> >>>>>>>>>> readers are
> >>>>>>>>>>
> >>>>>>>>>> pooling more work from the JM.
> >>>>>>>>>>
> >>>>>>>>>> 2) The source is going to be the "single point of
> >>>>>>>>>>
> >>>>>>>>>> truth".
> >>>>>>>>>>
> >>>>>>>>>> It
> >>>>>>>>>>
> >>>>>>>>>> will
> >>>>>>>>>>
> >>>>>>>>>> know
> >>>>>>>>>>
> >>>>>>>>>> what
> >>>>>>>>>>
> >>>>>>>>>> has been processed and what not. If the source and the
> >>>>>>>>>>
> >>>>>>>>>> readers
> >>>>>>>>>>
> >>>>>>>>>> are a
> >>>>>>>>>>
> >>>>>>>>>> single
> >>>>>>>>>>
> >>>>>>>>>> operator with parallelism > 1, or in general, if the
> >>>>>>>>>>
> >>>>>>>>>> split
> >>>>>>>>>>
> >>>>>>>>>> discovery
> >>>>>>>>>>
> >>>>>>>>>> is
> >>>>>>>>>>
> >>>>>>>>>> done by each task individually, then:
> >>>>>>>>>>    i) we have to have a deterministic scheme for each
> >>>>>>>>>>
> >>>>>>>>>> reader to
> >>>>>>>>>>
> >>>>>>>>>> assign
> >>>>>>>>>>
> >>>>>>>>>> splits to itself (e.g. mod subtaskId). This is not
> >>>>>>>>>>
> >>>>>>>>>> necessarily
> >>>>>>>>>>
> >>>>>>>>>> trivial
> >>>>>>>>>>
> >>>>>>>>>> for
> >>>>>>>>>>
> >>>>>>>>>> all sources.
> >>>>>>>>>>    ii) each reader would have to keep a copy of all its
> >>>>>>>>>>
> >>>>>>>>>> processed
> >>>>>>>>>>
> >>>>>>>>>> slpits
> >>>>>>>>>>
> >>>>>>>>>>    iii) the state has to be a union state with a
> >>>>>>>>>>
> >>>>>>>>>> non-trivial
> >>>>>>>>>>
> >>>>>>>>>> merging
> >>>>>>>>>>
> >>>>>>>>>> logic
> >>>>>>>>>>
> >>>>>>>>>> in order to support rescaling.
> >>>>>>>>>>
> >>>>>>>>>> Two additional points that you raised above:
> >>>>>>>>>>
> >>>>>>>>>> i) The point that you raised that we need to keep all
> >>>>>>>>>>
> >>>>>>>>>> splits
> >>>>>>>>>>
> >>>>>>>>>> (processed
> >>>>>>>>>>
> >>>>>>>>>> and
> >>>>>>>>>>
> >>>>>>>>>> not-processed) I think is a bit of a strong
> >>>>>>>>>>
> >>>>>>>>>> requirement.
> >>>>>>>>>>
> >>>>>>>>>> This
> >>>>>>>>>>
> >>>>>>>>>> would
> >>>>>>>>>>
> >>>>>>>>>> imply
> >>>>>>>>>>
> >>>>>>>>>> that for infinite sources the state will grow
> >>>>>>>>>>
> >>>>>>>>>> indefinitely.
> >>>>>>>>>>
> >>>>>>>>>> This is
> >>>>>>>>>>
> >>>>>>>>>> problem
> >>>>>>>>>>
> >>>>>>>>>> is even more pronounced if we do not have a single
> >>>>>>>>>>
> >>>>>>>>>> source
> >>>>>>>>>>
> >>>>>>>>>> that
> >>>>>>>>>>
> >>>>>>>>>> assigns
> >>>>>>>>>>
> >>>>>>>>>> splits to readers, as each reader will have its own
> >>>>>>>>>>
> >>>>>>>>>> copy
> >>>>>>>>>>
> >>>>>>>>>> of
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> state.
> >>>>>>>>>>
> >>>>>>>>>> ii) it is true that for finite sources we need to
> >>>>>>>>>>
> >>>>>>>>>> somehow
> >>>>>>>>>>
> >>>>>>>>>> not
> >>>>>>>>>>
> >>>>>>>>>> close
> >>>>>>>>>>
> >>>>>>>>>> the
> >>>>>>>>>>
> >>>>>>>>>> readers when the source/split discoverer finishes. The
> >>>>>>>>>> ContinuousFileReaderOperator has a work-around for
> >>>>>>>>>>
> >>>>>>>>>> that.
> >>>>>>>>>>
> >>>>>>>>>> It is
> >>>>>>>>>>
> >>>>>>>>>> not
> >>>>>>>>>>
> >>>>>>>>>> elegant,
> >>>>>>>>>>
> >>>>>>>>>> and checkpoints are not emitted after closing the
> >>>>>>>>>>
> >>>>>>>>>> source,
> >>>>>>>>>>
> >>>>>>>>>> but
> >>>>>>>>>>
> >>>>>>>>>> this, I
> >>>>>>>>>>
> >>>>>>>>>> believe, is a bigger problem which requires more
> >>>>>>>>>>
> >>>>>>>>>> changes
> >>>>>>>>>>
> >>>>>>>>>> than
> >>>>>>>>>>
> >>>>>>>>>> just
> >>>>>>>>>>
> >>>>>>>>>> refactoring the source interface.
> >>>>>>>>>>
> >>>>>>>>>> Cheers,
> >>>>>>>>>> Kostas
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> --
> >>>>>>>>>> Best, Jingsong Lee
> >>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>> --
> >>>>>> Best, Jingsong Lee
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>
> >
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Timo Walther <tw...@apache.org>.
Hi Becket,

regarding *Option 3* I think we can relax the constraints for env.source():

// MySource can be bounded or unbounded
DataStream<Type> dataStream = env.source(mySource);

// MySource must be bounded, otherwise throws exception.
BoundedDataStream<Type> boundedDataStream = env.boundedSource(mySource);

Bounded is just a special case of unbounded and every bounded source can 
also be treated as an unbounded source. This would unify the API if 
people don't need a bounded operation. It also addresses Jark's concerns.

Regards,
Timo


On 18.12.19 14:16, Becket Qin wrote:
> Hi Jark,
> 
> Please see the reply below:
> 
> Regarding to option#3, my concern is that if we don't support streaming
>> mode for bounded source,
>> how could we create a testing source for streaming mode? Currently, all the
>> testing source for streaming
>> are bounded, so that the integration test will finish finally.
> 
> 
> An UNBOUNDED source does not mean it will never stops. It simply indicates
> that the source *may* run forever, so the runtime needs to be prepared for
> that, but the task may still stop at some point when it hits some
> source-specific condition. So an UNBOUNDED testing source can still stop at
> some point if needed.
> 
> Regarding to Source#getRecordOrder(), could we have a implicit contract
>> that unbounded source should
>> already read in order (i.e. reading partitions in parallel), for bounded
>> source the order is not mandatory.
> 
> 
> 
>> This is also the behaviors of the current sources.
> 
> 1) a source can't guarantee it reads in strict order, because the producer
>> may produce data not in order.
>> 2) *Bounded-StrictOrder* is not necessary, because batch can reorder data.
> 
> 
> It is true that sometimes the source cannot guarantee the record order, but
> sometimes it can. Right now, even for stream processing, there is no
> processing order guarantee. For example, a join operator may emit a later
> record which successfully found a join match earlier.
> Event order is one of the most important requirements for event processing,
> a clear order guarantee would be necessary. That said, I agree that right
> now even if the sources provide the record order requirement, the runtime
> is not able to guarantee that out of the box. So I am OK if we add the
> record order to the Source later. But we should avoid misleading users to
> make them think the processing order is guaranteed when using the unbounded
> runtime.
> 
> Thanks,
> 
> Jiangjie (Becket) Qin
> 
> On Wed, Dec 18, 2019 at 10:29 AM Jark Wu <im...@gmail.com> wrote:
> 
>> Hi Becket,
>>
>> That's great we have reached a consensus on Source#getBoundedness().
>>
>> Regarding to option#3, my concern is that if we don't support streaming
>> mode for bounded source,
>> how could we create a testing source for streaming mode? Currently, all the
>> testing source for streaming
>> are bounded, so that the integration test will finish finally.
>>
>> Regarding to Source#getRecordOrder(), could we have a implicit contract
>> that unbounded source should
>> already read in order (i.e. reading partitions in parallel), for bounded
>> source the order is not mandatory.
>> This is also the behaviors of the current sources.
>> 1) a source can't guarantee it reads in strict order, because the producer
>> may produce data not in order.
>> 2) *Bounded-StrictOrder* is not necessary, because batch can reorder data.
>>
>> Best,
>> Jark
>>
>>
>>
>> On Tue, 17 Dec 2019 at 22:03, Becket Qin <be...@gmail.com> wrote:
>>
>>> Hi folks,
>>>
>>> Thanks for the comments. I am convinced that the Source API should not
>> take
>>> boundedness as a parameter after it is constructed. What Timo and Dawid
>>> suggested sounds a reasonable solution to me. So the Source API would
>>> become:
>>>
>>> Source {
>>>      Boundedness getBoundedness();
>>> }
>>>
>>> Assuming the above Source API, in addition to the two options mentioned
>> in
>>> earlier emails, I am thinking of another option:
>>>
>>> *Option 3:*
>>> // MySource must be unbounded, otherwise throws exception.
>>> DataStream<Type> dataStream = env.source(mySource);
>>>
>>> // MySource must be bounded, otherwise throws exception.
>>> BoundedDataStream<Type> boundedDataStream = env.boundedSource(mySource);
>>>
>>> The pros of this API are:
>>>     a) It fits the requirements from Table / SQL well.
>>>     b) DataStream users still have type safety (option 2 only has partial
>>> type safety).
>>>     c) Cristal clear boundedness from the API which makes DataStream join
>> /
>>> connect easy to reason about.
>>> The caveats I see,
>>>     a) It is inconsistent with Table since Table has one unified
>> interface.
>>>     b) No streaming mode for bounded source.
>>>
>>> @Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
>>> <al...@ververica.com> what do you think of the approach?
>>>
>>>
>>> Orthogonal to the above API, I am wondering whether boundedness is the
>> only
>>> dimension needed to describe the characteristic of the Source behavior.
>> We
>>> may also need to have another dimension of *record order*.
>>>
>>> For example, when a file source is reading from a directory with bounded
>>> records, it may have two ways to read.
>>> 1. Read files in parallel.
>>> 2. Read files in the chronological order.
>>> In both cases, the file source is a Bounded Source. However, the
>> processing
>>> requirement for downstream may be different. In the first case, the
>>> record processing and result emitting order does not matter, e.g. word
>>> count. In the second case, the records may have to be processed in the
>>> order they were read, e.g. change log processing.
>>>
>>> If the Source only has a getBoundedness() method, the downstream
>> processors
>>> would not know whether the records emitted from the Source should be
>>> processed in order or not. So combining the boundedness and record order,
>>> we will have four scenarios:
>>>
>>> *Bounded-StrictOrder*:     A segment of change log.
>>> *Bounded-Random*:          Batch Word Count.
>>> *Unbounded-StrictOrder*: An infinite change log.
>>> *Unbounded-Random*:     Streaming Word Count.
>>>
>>> Option 2 mentioned in the previous email was kind of trying to handle the
>>> Bounded-StrictOrder case by creating a DataStream from a bounded source,
>>> which actually does not work.
>>> It looks that we do not have strict order support in some operators at
>> this
>>> point, e.g. join. But we may still want to add the semantic to the Source
>>> first so later on we don't need to change all the source implementations,
>>> especially given that many of them will be implemented by 3rd party.
>>>
>>> Given that, we need another dimension of *Record Order* in the Source.
>> More
>>> specifically, the API would become:
>>>
>>> Source {
>>>      Boundedness getBoundedness();
>>>      RecordOrder getRecordOrder();
>>> }
>>>
>>> public enum RecordOrder {
>>>      /** The record in the DataStream must be processed in its strict
>> order
>>> for correctness. */
>>>      STRICT,
>>>      /** The record in the DataStream can be processed in arbitrary order.
>>> */
>>>      RANDOM;
>>> }
>>>
>>> Any thoughts?
>>>
>>> Thanks,
>>>
>>> Jiangjie (Becket) Qin
>>>
>>> On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <tw...@apache.org> wrote:
>>>
>>>> Hi Becket,
>>>>
>>>> I completely agree with Dawid's suggestion. The information about the
>>>> boundedness should come out of the source. Because most of the
>> streaming
>>>> sources can be made bounded based on some connector specific criterion.
>>>> In Kafka, it would be an end offset or end timestamp but in any case
>>>> having just a env.boundedSource() is not enough because parameters for
>>>> making the source bounded are missing.
>>>>
>>>> I suggest to have a simple `isBounded(): Boolean` flag in every source
>>>> that might be influenced by a connector builder as Dawid mentioned.
>>>>
>>>> For type safety during programming, we can still go with *Final state
>>>> 1*. By having a env.source() vs env.boundedSource(). The latter would
>>>> just enforce that the boolean flag is set to `true` and could make
>>>> bounded operations available (if we need that actually).
>>>>
>>>> However, I don't think that we should start making a unified Table API
>>>> ununified again. Boundedness is an optimization property. Every bounded
>>>> operation can also executed in an unbounded way using
>> updates/retraction
>>>> or watermarks.
>>>>
>>>> Regards,
>>>> Timo
>>>>
>>>>
>>>> On 15.12.19 14:22, Becket Qin wrote:
>>>>> Hi Dawid and Jark,
>>>>>
>>>>> I think the discussion ultimately boils down to the question that
>> which
>>>> one
>>>>> of the following two final states do we want? Once we make this
>>> decision,
>>>>> everything else can be naturally derived.
>>>>>
>>>>> *Final state 1*: Separate API for bounded / unbounded DataStream &
>>> Table.
>>>>> That means any code users write will be valid at the point when they
>>>> write
>>>>> the code. This is similar to having type safety check at programming
>>>> time.
>>>>> For example,
>>>>>
>>>>> BoundedDataStream extends DataStream {
>>>>> // Operations only available for bounded data.
>>>>> BoundedDataStream sort(...);
>>>>>
>>>>> // Interaction with another BoundedStream returns a Bounded stream.
>>>>> BoundedJoinedDataStream join(BoundedDataStream other)
>>>>>
>>>>> // Interaction with another unbounded stream returns an unbounded
>>> stream.
>>>>> JoinedDataStream join(DataStream other)
>>>>> }
>>>>>
>>>>> BoundedTable extends Table {
>>>>>     // Bounded only operation.
>>>>> BoundedTable sort(...);
>>>>>
>>>>> // Interaction with another BoundedTable returns a BoundedTable.
>>>>> BoundedTable join(BoundedTable other)
>>>>>
>>>>> // Interaction with another unbounded table returns an unbounded
>> table.
>>>>> Table join(Table other)
>>>>> }
>>>>>
>>>>> *Final state 2*: One unified API for bounded / unbounded DataStream /
>>>>> Table.
>>>>> That unified API may throw exception at DAG compilation time if an
>>>> invalid
>>>>> operation is tried. This is what Table API currently follows.
>>>>>
>>>>> DataStream {
>>>>> // Throws exception if the DataStream is unbounded.
>>>>> DataStream sort();
>>>>> // Get boundedness.
>>>>> Boundedness getBoundedness();
>>>>> }
>>>>>
>>>>> Table {
>>>>> // Throws exception if the table has infinite rows.
>>>>> Table orderBy();
>>>>>
>>>>> // Get boundedness.
>>>>> Boundedness getBoundedness();
>>>>> }
>>>>>
>>>>> >From what I understand, there is no consensus so far on this decision
>>>> yet.
>>>>> Whichever final state we choose, we need to make it consistent across
>>> the
>>>>> entire project. We should avoid the case that Table follows one final
>>>> state
>>>>> while DataStream follows another. Some arguments I am aware of from
>>> both
>>>>> sides so far are following:
>>>>>
>>>>> Arguments for final state 1:
>>>>> 1a) Clean API with method safety check at programming time.
>>>>> 1b) (Counter 2b) Although SQL does not have programming time error
>>>> check, SQL
>>>>> is not really a "programming language" per se. So SQL can be
>> different
>>>> from
>>>>> Table and DataStream.
>>>>> 1c)  Although final state 2 seems making it easier for SQL to use
>> given
>>>> it
>>>>> is more "config based" than "parameter based", final state 1 can
>>> probably
>>>>> also meet what SQL wants by wrapping the Source in TableSource /
>>>>> TableSourceFactory API if needed.
>>>>>
>>>>> Arguments for final state 2:
>>>>> 2a) The Source API itself seems already sort of following the unified
>>> API
>>>>> pattern.
>>>>> 2b) There is no "programming time" method error check in SQL case, so
>>> we
>>>>> cannot really achieve final state 1 across the board.
>>>>> 2c) It is an easier path given our current status, i.e. Table is
>>> already
>>>>> following final state 2.
>>>>> 2d) Users can always explicitly check the boundedness if they want
>> to.
>>>>>
>>>>> As I mentioned earlier, my initial thought was also to have a
>>>>> "configuration based" Source rather than a "parameter based" Source.
>> So
>>>> it
>>>>> is completely possible that I missed some important consideration or
>>>> design
>>>>> principles that we want to enforce for the project. It would be good
>>>>> if @Stephan
>>>>> Ewen <st...@ververica.com> and @Aljoscha Krettek <
>>>> aljoscha@ververica.com> can
>>>>> also provide more thoughts on this.
>>>>>
>>>>>
>>>>> Re: Jingsong
>>>>>
>>>>> As you said, there are some batched system source, like parquet/orc
>>>> source.
>>>>>> Could we have the batch emit interface to improve performance? The
>>>> queue of
>>>>>> per record may cause performance degradation.
>>>>>
>>>>>
>>>>> The current interface does not necessarily cause performance problem
>>> in a
>>>>> multi-threading case. In fact, the base implementation allows
>>>> SplitReaders
>>>>> to add a batch <E> of records<T> to the records queue<E>, so each
>>> element
>>>>> in the records queue would be a batch <E>. In this case, when the
>> main
>>>>> thread polls records, it will take a batch <E> of records <T> from
>> the
>>>>> shared records queue and process the records <T> in a batch manner.
>>>>>
>>>>> Thanks,
>>>>>
>>>>> Jiangjie (Becket) Qin
>>>>>
>>>>> On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <ji...@gmail.com>
>>>> wrote:
>>>>>
>>>>>> Hi Becket,
>>>>>>
>>>>>> I also have some performance concerns too.
>>>>>>
>>>>>> If I understand correctly, SourceOutput will emit data per record
>> into
>>>> the
>>>>>> queue? I'm worried about the multithreading performance of this
>> queue.
>>>>>>
>>>>>>> One example is some batched messaging systems which only have an
>>> offset
>>>>>> for the entire batch instead of individual messages in the batch.
>>>>>>
>>>>>> As you said, there are some batched system source, like parquet/orc
>>>> source.
>>>>>> Could we have the batch emit interface to improve performance? The
>>>> queue of
>>>>>> per record may cause performance degradation.
>>>>>>
>>>>>> Best,
>>>>>> Jingsong Lee
>>>>>>
>>>>>> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com> wrote:
>>>>>>
>>>>>>> Hi Becket,
>>>>>>>
>>>>>>> I think Dawid explained things clearly and makes a lot of sense.
>>>>>>> I'm also in favor of #2, because #1 doesn't work for our future
>>> unified
>>>>>>> envrionment.
>>>>>>>
>>>>>>> You can see the vision in this documentation [1]. In the future, we
>>>> would
>>>>>>> like to
>>>>>>> drop the global streaming/batch mode in SQL (i.e.
>>>>>>> EnvironmentSettings#inStreamingMode/inBatchMode).
>>>>>>> A source is bounded or unbounded once defined, so queries can be
>>>> inferred
>>>>>>> from source to run
>>>>>>> in streaming or batch or hybrid mode. However, in #1, we will lose
>>> this
>>>>>>> ability because the framework
>>>>>>> doesn't know whether the source is bounded or unbounded.
>>>>>>>
>>>>>>> Best,
>>>>>>> Jark
>>>>>>>
>>>>>>>
>>>>>>> [1]:
>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
>>>>>>>
>>>>>>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <pi...@ververica.com>
>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi,
>>>>>>>>
>>>>>>>> Regarding the:
>>>>>>>>
>>>>>>>> Collection<E> getNextRecords()
>>>>>>>>
>>>>>>>> I’m pretty sure such design would unfortunately impact the
>>> performance
>>>>>>>> (accessing and potentially creating the collection on the hot
>> path).
>>>>>>>>
>>>>>>>> Also the
>>>>>>>>
>>>>>>>> InputStatus emitNext(DataOutput<T> output) throws Exception;
>>>>>>>> or
>>>>>>>> Status pollNext(SourceOutput<T> sourceOutput) throws Exception;
>>>>>>>>
>>>>>>>> Gives us some opportunities in the future, to allow Source hot
>>> looping
>>>>>>>> inside, until it receives some signal “please exit because of some
>>>>>>> reasons”
>>>>>>>> (output collector could return such hint upon collecting the
>>> result).
>>>>>> But
>>>>>>>> that’s another topic outside of this FLIP’s scope.
>>>>>>>>
>>>>>>>> Piotrek
>>>>>>>>
>>>>>>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <tr...@apache.org>
>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> Hi Becket,
>>>>>>>>>
>>>>>>>>> quick clarification from my side because I think you
>> misunderstood
>>> my
>>>>>>>>> question. I did not suggest to let the SourceReader return only a
>>>>>>> single
>>>>>>>>> record at a time when calling getNextRecords. As the return type
>>>>>>>> indicates,
>>>>>>>>> the method can return an arbitrary number of records.
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>> Till
>>>>>>>>>
>>>>>>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
>>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Becket,
>>>>>>>>>>
>>>>>>>>>> Issue #1 - Design of Source interface
>>>>>>>>>>
>>>>>>>>>> I mentioned the lack of a method like
>>>>>>>> Source#createEnumerator(Boundedness
>>>>>>>>>> boundedness, SplitEnumeratorContext context), because without
>> the
>>>>>>>> current
>>>>>>>>>> proposal is not complete/does not work.
>>>>>>>>>>
>>>>>>>>>> If we say that boundedness is an intrinsic property of a source
>>> imo
>>>>>> we
>>>>>>>>>> don't need the Source#createEnumerator(Boundedness boundedness,
>>>>>>>>>> SplitEnumeratorContext context) method.
>>>>>>>>>>
>>>>>>>>>> Assuming a source from my previous example:
>>>>>>>>>>
>>>>>>>>>> Source source = KafkaSource.builder()
>>>>>>>>>>    ...
>>>>>>>>>>    .untilTimestamp(...)
>>>>>>>>>>    .build()
>>>>>>>>>>
>>>>>>>>>> Would the enumerator differ if created like
>>>>>>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
>>>>>>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating myself,
>> but
>>>>>>> this
>>>>>>>> is
>>>>>>>>>> the part that my opinion differ the most from the current
>>> proposal.
>>>>>> I
>>>>>>>>>> really think it should always be the source that tells if it is
>>>>>>> bounded
>>>>>>>> or
>>>>>>>>>> not. In the current proposal methods
>> continousSource/boundedSource
>>>>>>>> somewhat
>>>>>>>>>> reconfigure the source, which I think is misleading.
>>>>>>>>>>
>>>>>>>>>> I think a call like:
>>>>>>>>>>
>>>>>>>>>> Source source = KafkaSource.builder()
>>>>>>>>>>    ...
>>>>>>>>>>    .readContinously() / readUntilLatestOffset() /
>>> readUntilTimestamp
>>>> /
>>>>>>>> readUntilOffsets / ...
>>>>>>>>>>    .build()
>>>>>>>>>>
>>>>>>>>>> is way cleaner (and expressive) than
>>>>>>>>>>
>>>>>>>>>> Source source = KafkaSource.builder()
>>>>>>>>>>    ...
>>>>>>>>>>    .build()
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> env.continousSource(source) // which actually underneath would
>>> call
>>>>>>>> createEnumerator(CONTINUOUS, ctx) which would be equivalent to
>>>>>>>> source.readContinously().createEnumerator(ctx)
>>>>>>>>>> // or
>>>>>>>>>> env.boundedSource(source) // which actually underneath would
>> call
>>>>>>>> createEnumerator(BOUNDED, ctx) which would be equivalent to
>>>>>>>> source.readUntilLatestOffset().createEnumerator(ctx)
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Sorry for the comparison, but to me it seems there is too much
>>> magic
>>>>>>>>>> happening underneath those two calls.
>>>>>>>>>>
>>>>>>>>>> I really believe the Source interface should have getBoundedness
>>>>>>> method
>>>>>>>>>> instead of (supportBoundedness) + createEnumerator(Boundedness,
>>> ...)
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Issue #2 - Design of
>>>>>>>>>> ExecutionEnvironment#source()/continuousSource()/boundedSource()
>>>>>>>>>>
>>>>>>>>>> As you might have guessed I am slightly in favor of option #2
>>>>>>> modified.
>>>>>>>>>> Yes I am aware every step of the dag would have to be able to
>> say
>>> if
>>>>>>> it
>>>>>>>> is
>>>>>>>>>> bounded or not. I have a feeling it would be easier to express
>>> cross
>>>>>>>>>> bounded/unbounded operations, but I must admit I have not
>> thought
>>> it
>>>>>>>>>> through thoroughly, In the spirit of batch is just a special
>> case
>>> of
>>>>>>>>>> streaming I thought BoundedStream would extend from DataStream.
>>>>>>> Correct
>>>>>>>> me
>>>>>>>>>> if I am wrong. In such a setup the cross bounded/unbounded
>>> operation
>>>>>>>> could
>>>>>>>>>> be expressed quite easily I think:
>>>>>>>>>>
>>>>>>>>>> DataStream {
>>>>>>>>>>    DataStream join(DataStream, ...); // we could not really tell
>> if
>>>>>> the
>>>>>>>> result is bounded or not, but because bounded stream is a special
>>> case
>>>>>> of
>>>>>>>> unbounded the API object is correct, irrespective if the left or
>>> right
>>>>>>> side
>>>>>>>> of the join is bounded
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> BoundedStream extends DataStream {
>>>>>>>>>>    BoundedStream join(BoundedStream, ...); // only if both sides
>>> are
>>>>>>>> bounded the result can be bounded as well. However we do have
>> access
>>>> to
>>>>>>> the
>>>>>>>> DataStream#join here, so you can still join with a DataStream
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On the other hand I also see benefits of two completely
>> disjointed
>>>>>>> APIs,
>>>>>>>>>> as we could prohibit some streaming calls in the bounded API. I
>>>>>> can't
>>>>>>>> think
>>>>>>>>>> of any unbounded operators that could not be implemented for
>>> bounded
>>>>>>>> stream.
>>>>>>>>>>
>>>>>>>>>> Besides I think we both agree we don't like the method:
>>>>>>>>>>
>>>>>>>>>> DataStream boundedStream(Source)
>>>>>>>>>>
>>>>>>>>>> suggested in the current state of the FLIP. Do we ? :)
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>>
>>>>>>>>>> Dawid
>>>>>>>>>>
>>>>>>>>>> On 10/12/2019 18:57, Becket Qin wrote:
>>>>>>>>>>
>>>>>>>>>> Hi folks,
>>>>>>>>>>
>>>>>>>>>> Thanks for the discussion, great feedback. Also thanks Dawid for
>>> the
>>>>>>>>>> explanation, it is much clearer now.
>>>>>>>>>>
>>>>>>>>>> One thing that is indeed missing from the FLIP is how the
>>>>>> boundedness
>>>>>>> is
>>>>>>>>>> passed to the Source implementation. So the API should be
>>>>>>>>>> Source#createEnumerator(Boundedness boundedness,
>>>>>>> SplitEnumeratorContext
>>>>>>>>>> context)
>>>>>>>>>> And we can probably remove the
>>> Source#supportBoundedness(Boundedness
>>>>>>>>>> boundedness) method.
>>>>>>>>>>
>>>>>>>>>> Assuming we have that, we are essentially choosing from one of
>> the
>>>>>>>>>> following two options:
>>>>>>>>>>
>>>>>>>>>> Option 1:
>>>>>>>>>> // The source is continuous source, and only unbounded
>> operations
>>>>>> can
>>>>>>> be
>>>>>>>>>> performed.
>>>>>>>>>> DataStream<Type> datastream = env.continuousSource(someSource);
>>>>>>>>>>
>>>>>>>>>> // The source is bounded source, both bounded and unbounded
>>>>>> operations
>>>>>>>> can
>>>>>>>>>> be performed.
>>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
>>>>>>>> env.boundedSource(someSource);
>>>>>>>>>>
>>>>>>>>>>    - Pros:
>>>>>>>>>>         a) explicit boundary between bounded / unbounded streams,
>>> it
>>>>>> is
>>>>>>>>>> quite simple and clear to the users.
>>>>>>>>>>    - Cons:
>>>>>>>>>>         a) For applications that do not involve bounded
>> operations,
>>>>>> they
>>>>>>>>>> still have to call different API to distinguish bounded /
>>> unbounded
>>>>>>>> streams.
>>>>>>>>>>         b) No support for bounded stream to run in a streaming
>>>> runtime
>>>>>>>>>> setting, i.e. scheduling and operators behaviors.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Option 2:
>>>>>>>>>> // The source is either bounded or unbounded, but only unbounded
>>>>>>>> operations
>>>>>>>>>> could be performed on the returned DataStream.
>>>>>>>>>> DataStream<Type> dataStream = env.source(someSource);
>>>>>>>>>>
>>>>>>>>>> // The source must be a bounded source, otherwise exception is
>>>>>> thrown.
>>>>>>>>>> BoundedDataStream<Type> boundedDataStream =
>>>>>>>>>> env.boundedSource(boundedSource);
>>>>>>>>>>
>>>>>>>>>> The pros and cons are exactly the opposite of option 1.
>>>>>>>>>>    - Pros:
>>>>>>>>>>         a) For applications that do not involve bounded
>> operations,
>>>>>> they
>>>>>>>>>> still have to call different API to distinguish bounded /
>>> unbounded
>>>>>>>> streams.
>>>>>>>>>>         b) Support for bounded stream to run in a streaming
>> runtime
>>>>>>>> setting,
>>>>>>>>>> i.e. scheduling and operators behaviors.
>>>>>>>>>>    - Cons:
>>>>>>>>>>         a) Bounded / unbounded streams are kind of mixed, i.e.
>>> given
>>>> a
>>>>>>>>>> DataStream, it is not clear whether it is bounded or not, unless
>>> you
>>>>>>>> have
>>>>>>>>>> the access to its source.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> If we only think from the Source API perspective, option 2
>> seems a
>>>>>>>> better
>>>>>>>>>> choice because functionality wise it is a superset of option 1,
>> at
>>>>>> the
>>>>>>>> cost
>>>>>>>>>> of some seemingly acceptable ambiguity in the DataStream API.
>>>>>>>>>> But if we look at the DataStream API as a whole, option 1 seems
>> a
>>>>>>>> clearer
>>>>>>>>>> choice. For example, some times a library may have to know
>>> whether a
>>>>>>>>>> certain task will finish or not. And it would be difficult to
>> tell
>>>>>> if
>>>>>>>> the
>>>>>>>>>> input is a DataStream, unless additional information is provided
>>> all
>>>>>>> the
>>>>>>>>>> way from the Source. One possible solution is to have a
>> *modified
>>>>>>>> option 2*
>>>>>>>>>> which adds a method to the DataStream API to indicate
>> boundedness,
>>>>>>> such
>>>>>>>> as
>>>>>>>>>> getBoundedness(). It would solve the problem with a potential
>>>>>>> confusion
>>>>>>>> of
>>>>>>>>>> what is difference between a DataStream with
>> getBoundedness()=true
>>>>>>> and a
>>>>>>>>>> BoundedDataStream. But that seems not super difficult to
>> explain.
>>>>>>>>>>
>>>>>>>>>> So from API's perspective, I don't have a strong opinion between
>>>>>>>> *option 1*
>>>>>>>>>> and *modified option 2. *I like the cleanness of option 1, but
>>>>>>> modified
>>>>>>>>>> option 2 would be more attractive if we have concrete use case
>> for
>>>>>> the
>>>>>>>>>> "Bounded stream with unbounded streaming runtime settings".
>>>>>>>>>>
>>>>>>>>>> Re: Till
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Maybe this has already been asked before but I was wondering why
>>> the
>>>>>>>>>> SourceReader interface has the method pollNext which hands the
>>>>>>>>>> responsibility of outputting elements to the SourceReader
>>>>>>>> implementation?
>>>>>>>>>> Has this been done for backwards compatibility reasons with the
>>> old
>>>>>>>> source
>>>>>>>>>> interface? If not, then one could define a Collection<E>
>>>>>>>> getNextRecords()
>>>>>>>>>> method which returns the currently retrieved records and then
>> the
>>>>>>> caller
>>>>>>>>>> emits them outside of the SourceReader. That way the interface
>>> would
>>>>>>> not
>>>>>>>>>> allow to implement an outputting loop where we never hand back
>>>>>> control
>>>>>>>> to
>>>>>>>>>> the caller. At the moment, this contract can be easily broken
>> and
>>> is
>>>>>>>> only
>>>>>>>>>> mentioned loosely in the JavaDocs.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> The primary reason we handover the SourceOutput to the
>>> SourceReader
>>>>>> is
>>>>>>>>>> because sometimes it is difficult for a SourceReader to emit one
>>>>>>> record
>>>>>>>> at
>>>>>>>>>> a time. One example is some batched messaging systems which only
>>>>>> have
>>>>>>> an
>>>>>>>>>> offset for the entire batch instead of individual messages in
>> the
>>>>>>>> batch. In
>>>>>>>>>> that case, returning one record at a time would leave the
>>>>>> SourceReader
>>>>>>>> in
>>>>>>>>>> an uncheckpointable state because they can only checkpoint at
>> the
>>>>>>> batch
>>>>>>>>>> boundaries.
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>>
>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>
>>>>>>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <
>>> trohrmann@apache.org
>>>>>>>> <ma...@apache.org>> <trohrmann@apache.org <mailto:
>>>>>>>> trohrmann@apache.org>> wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Hi everyone,
>>>>>>>>>>
>>>>>>>>>> thanks for drafting this FLIP. It reads very well.
>>>>>>>>>>
>>>>>>>>>> Concerning Dawid's proposal, I tend to agree. The boundedness
>>> could
>>>>>>> come
>>>>>>>>>> from the source and tell the system how to treat the operator
>>>>>>>> (scheduling
>>>>>>>>>> wise). From a user's perspective it should be fine to get back a
>>>>>>>> DataStream
>>>>>>>>>> when calling env.source(boundedSource) if he does not need
>> special
>>>>>>>>>> operations defined on a BoundedDataStream. If he needs this,
>> then
>>>>>> one
>>>>>>>> could
>>>>>>>>>> use the method BoundedDataStream
>> env.boundedSource(boundedSource).
>>>>>>>>>>
>>>>>>>>>> If possible, we could enforce the proper usage of
>>>>>> env.boundedSource()
>>>>>>> by
>>>>>>>>>> introducing a BoundedSource type so that one cannot pass an
>>>>>>>>>> unbounded source to it. That way users would not be able to
>> shoot
>>>>>>>>>> themselves in the foot.
>>>>>>>>>>
>>>>>>>>>> Maybe this has already been asked before but I was wondering why
>>> the
>>>>>>>>>> SourceReader interface has the method pollNext which hands the
>>>>>>>>>> responsibility of outputting elements to the SourceReader
>>>>>>>> implementation?
>>>>>>>>>> Has this been done for backwards compatibility reasons with the
>>> old
>>>>>>>> source
>>>>>>>>>> interface? If not, then one could define a Collection<E>
>>>>>>>> getNextRecords()
>>>>>>>>>> method which returns the currently retrieved records and then
>> the
>>>>>>> caller
>>>>>>>>>> emits them outside of the SourceReader. That way the interface
>>> would
>>>>>>> not
>>>>>>>>>> allow to implement an outputting loop where we never hand back
>>>>>> control
>>>>>>>> to
>>>>>>>>>> the caller. At the moment, this contract can be easily broken
>> and
>>> is
>>>>>>>> only
>>>>>>>>>> mentioned loosely in the JavaDocs.
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Till
>>>>>>>>>>
>>>>>>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <
>>> jingsonglee0@gmail.com
>>>>>>>> <ma...@gmail.com>> <jingsonglee0@gmail.com <mailto:
>>>>>>>> jingsonglee0@gmail.com>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Hi all,
>>>>>>>>>>
>>>>>>>>>> I think current design is good.
>>>>>>>>>>
>>>>>>>>>> My understanding is:
>>>>>>>>>>
>>>>>>>>>> For execution mode: bounded mode and continuous mode, It's
>> totally
>>>>>>>>>> different. I don't think we have the ability to integrate the
>> two
>>>>>>> models
>>>>>>>>>>
>>>>>>>>>> at
>>>>>>>>>>
>>>>>>>>>> present. It's about scheduling, memory, algorithms, States, etc.
>>> we
>>>>>>>>>> shouldn't confuse them.
>>>>>>>>>>
>>>>>>>>>> For source capabilities: only bounded, only continuous, both
>>> bounded
>>>>>>> and
>>>>>>>>>> continuous.
>>>>>>>>>> I think Kafka is a source that can be ran both bounded
>>>>>>>>>> and continuous execution mode.
>>>>>>>>>> And Kafka with end offset should be ran both bounded
>>>>>>>>>> and continuous execution mode.  Using apache Beam with Flink
>>>>>> runner, I
>>>>>>>>>>
>>>>>>>>>> used
>>>>>>>>>>
>>>>>>>>>> to run a "bounded" Kafka in streaming mode. For our previous
>>>>>>> DataStream,
>>>>>>>>>>
>>>>>>>>>> it
>>>>>>>>>>
>>>>>>>>>> is not necessarily required that the source cannot be bounded.
>>>>>>>>>>
>>>>>>>>>> So it is my thought for Dawid's question:
>>>>>>>>>> 1.pass a bounded source to continuousSource() +1
>>>>>>>>>> 2.pass a continuous source to boundedSource() -1, should throw
>>>>>>>> exception.
>>>>>>>>>>
>>>>>>>>>> In StreamExecutionEnvironment, continuousSource and
>> boundedSource
>>>>>>> define
>>>>>>>>>> the execution mode. It defines a clear boundary of execution
>> mode.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> Jingsong Lee
>>>>>>>>>>
>>>>>>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com
>>> <mailto:
>>>>>>>> imjark@gmail.com>> <imjark@gmail.com <ma...@gmail.com>>
>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> I agree with Dawid's point that the boundedness information
>> should
>>>>>>> come
>>>>>>>>>> from the source itself (e.g. the end timestamp), not through
>>>>>>>>>> env.boundedSouce()/continuousSource().
>>>>>>>>>> I think if we want to support something like `env.source()` that
>>>>>>> derive
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> execution mode from source, `supportsBoundedness(Boundedness)`
>>>>>>>>>> method is not enough, because we don't know whether it is
>> bounded
>>> or
>>>>>>>>>>
>>>>>>>>>> not.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> Jark
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
>>>>>> dwysakowicz@apache.org
>>>>>>>> <ma...@apache.org>> <dwysakowicz@apache.org <mailto:
>>>>>>>> dwysakowicz@apache.org>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> One more thing. In the current proposal, with the
>>>>>>>>>> supportsBoundedness(Boundedness) method and the boundedness
>> coming
>>>>>>>>>>
>>>>>>>>>> from
>>>>>>>>>>
>>>>>>>>>> either continuousSource or boundedSource I could not find how
>> this
>>>>>>>>>> information is fed back to the SplitEnumerator.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>>
>>>>>>>>>> Dawid
>>>>>>>>>>
>>>>>>>>>> On 09/12/2019 13:52, Becket Qin wrote:
>>>>>>>>>>
>>>>>>>>>> Hi Dawid,
>>>>>>>>>>
>>>>>>>>>> Thanks for the comments. This actually brings another relevant
>>>>>>>>>>
>>>>>>>>>> question
>>>>>>>>>>
>>>>>>>>>> about what does a "bounded source" imply. I actually had the
>> same
>>>>>>>>>> impression when I look at the Source API. Here is what I
>>> understand
>>>>>>>>>>
>>>>>>>>>> after
>>>>>>>>>>
>>>>>>>>>> some discussion with Stephan. The bounded source has the
>> following
>>>>>>>>>>
>>>>>>>>>> impacts.
>>>>>>>>>>
>>>>>>>>>> 1. API validity.
>>>>>>>>>> - A bounded source generates a bounded stream so some operations
>>>>>>>>>>
>>>>>>>>>> that
>>>>>>>>>>
>>>>>>>>>> only
>>>>>>>>>>
>>>>>>>>>> works for bounded records would be performed, e.g. sort.
>>>>>>>>>> - To expose these bounded stream only APIs, there are two
>> options:
>>>>>>>>>>       a. Add them to the DataStream API and throw exception if a
>>>>>>>>>>
>>>>>>>>>> method
>>>>>>>>>>
>>>>>>>>>> is
>>>>>>>>>>
>>>>>>>>>> called on an unbounded stream.
>>>>>>>>>>       b. Create a BoundedDataStream class which is returned from
>>>>>>>>>> env.boundedSource(), while DataStream is returned from
>>>>>>>>>>
>>>>>>>>>> env.continousSource().
>>>>>>>>>>
>>>>>>>>>> Note that this cannot be done by having single
>>>>>>>>>>
>>>>>>>>>> env.source(theSource)
>>>>>>>>>>
>>>>>>>>>> even
>>>>>>>>>>
>>>>>>>>>> the Source has a getBoundedness() method.
>>>>>>>>>>
>>>>>>>>>> 2. Scheduling
>>>>>>>>>> - A bounded source could be computed stage by stage without
>>>>>>>>>>
>>>>>>>>>> bringing
>>>>>>>>>>
>>>>>>>>>> up
>>>>>>>>>>
>>>>>>>>>> all
>>>>>>>>>>
>>>>>>>>>> the tasks at the same time.
>>>>>>>>>>
>>>>>>>>>> 3. Operator behaviors
>>>>>>>>>> - A bounded source indicates the records are finite so some
>>>>>>>>>>
>>>>>>>>>> operators
>>>>>>>>>>
>>>>>>>>>> can
>>>>>>>>>>
>>>>>>>>>> wait until it receives all the records before it starts the
>>>>>>>>>>
>>>>>>>>>> processing.
>>>>>>>>>>
>>>>>>>>>> In the above impact, only 1 is relevant to the API design. And
>> the
>>>>>>>>>>
>>>>>>>>>> current
>>>>>>>>>>
>>>>>>>>>> proposal in FLIP-27 is following 1.b.
>>>>>>>>>>
>>>>>>>>>> // boundedness depends of source property, imo this should
>> always
>>>>>>>>>>
>>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>> preferred
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> In your proposal, does DataStream have bounded stream only
>>> methods?
>>>>>>>>>>
>>>>>>>>>> It
>>>>>>>>>>
>>>>>>>>>> looks it should have, otherwise passing a bounded Source to
>>>>>>>>>>
>>>>>>>>>> env.source()
>>>>>>>>>>
>>>>>>>>>> would be confusing. In that case, we will essentially do 1.a if
>> an
>>>>>>>>>> unbounded Source is created from env.source(unboundedSource).
>>>>>>>>>>
>>>>>>>>>> If we have the methods only supported for bounded streams in
>>>>>>>>>>
>>>>>>>>>> DataStream,
>>>>>>>>>>
>>>>>>>>>> it
>>>>>>>>>>
>>>>>>>>>> seems a little weird to have a separate BoundedDataStream
>>>>>>>>>>
>>>>>>>>>> interface.
>>>>>>>>>>
>>>>>>>>>> Am I understand it correctly?
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>>
>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
>>>>>>>>>>
>>>>>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Hi all,
>>>>>>>>>>
>>>>>>>>>> Really well written proposal and very important one. I must
>> admit
>>>>>>>>>>
>>>>>>>>>> I
>>>>>>>>>>
>>>>>>>>>> have
>>>>>>>>>>
>>>>>>>>>> not understood all the intricacies of it yet.
>>>>>>>>>>
>>>>>>>>>> One question I have though is about where does the information
>>>>>>>>>>
>>>>>>>>>> about
>>>>>>>>>>
>>>>>>>>>> boundedness come from. I think in most cases it is a property of
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> source. As you described it might be e.g. end offset, a flag
>>>>>>>>>>
>>>>>>>>>> should
>>>>>>>>>>
>>>>>>>>>> it
>>>>>>>>>>
>>>>>>>>>> monitor new splits etc. I think it would be a really nice use
>> case
>>>>>>>>>>
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>> able to say:
>>>>>>>>>>
>>>>>>>>>> new KafkaSource().readUntil(long timestamp),
>>>>>>>>>>
>>>>>>>>>> which could work as an "end offset". Moreover I think all
>> Bounded
>>>>>>>>>>
>>>>>>>>>> sources
>>>>>>>>>>
>>>>>>>>>> support continuous mode, but no intrinsically continuous source
>>>>>>>>>>
>>>>>>>>>> support
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> Bounded mode. If I understood the proposal correctly it suggest
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> boundedness sort of "comes" from the outside of the source, from
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> invokation of either boundedStream or continousSource.
>>>>>>>>>>
>>>>>>>>>> I am wondering if it would make sense to actually change the
>>>>>>>>>>
>>>>>>>>>> method
>>>>>>>>>>
>>>>>>>>>> boolean Source#supportsBoundedness(Boundedness)
>>>>>>>>>>
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>> Boundedness Source#getBoundedness().
>>>>>>>>>>
>>>>>>>>>> As for the methods #boundedSource, #continousSource, assuming
>> the
>>>>>>>>>> boundedness is property of the source they do not affect how the
>>>>>>>>>>
>>>>>>>>>> enumerator
>>>>>>>>>>
>>>>>>>>>> works, but mostly how the dag is scheduled, right? I am not
>>>>>>>>>>
>>>>>>>>>> against
>>>>>>>>>>
>>>>>>>>>> those
>>>>>>>>>>
>>>>>>>>>> methods, but I think it is a very specific use case to actually
>>>>>>>>>>
>>>>>>>>>> override
>>>>>>>>>>
>>>>>>>>>> the property of the source. In general I would expect users to
>>>>>>>>>>
>>>>>>>>>> only
>>>>>>>>>>
>>>>>>>>>> call
>>>>>>>>>>
>>>>>>>>>> env.source(theSource), where the source tells if it is bounded
>> or
>>>>>>>>>>
>>>>>>>>>> not. I
>>>>>>>>>>
>>>>>>>>>> would suggest considering following set of methods:
>>>>>>>>>>
>>>>>>>>>> // boundedness depends of source property, imo this should
>> always
>>>>>>>>>>
>>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>> preferred
>>>>>>>>>>
>>>>>>>>>> DataStream<MyType> stream = env.source(theSource);
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> // always continous execution, whether bounded or unbounded
>> source
>>>>>>>>>>
>>>>>>>>>> DataStream<MyType> boundedStream =
>> env.continousSource(theSource);
>>>>>>>>>>
>>>>>>>>>> // imo this would make sense if the BoundedDataStream provides
>>>>>>>>>>
>>>>>>>>>> additional features unavailable for continous mode
>>>>>>>>>>
>>>>>>>>>> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>>
>>>>>>>>>> Dawid
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
>>>>>>>>>>
>>>>>>>>>> Thanks, Becket, for updating this.
>>>>>>>>>>
>>>>>>>>>> I agree with moving the aspects you mentioned into separate
>> FLIPs
>>>>>>>>>>
>>>>>>>>>> -
>>>>>>>>>>
>>>>>>>>>> this
>>>>>>>>>>
>>>>>>>>>> one way becoming unwieldy in size.
>>>>>>>>>>
>>>>>>>>>> +1 to the FLIP in its current state. Its a very detailed
>> write-up,
>>>>>>>>>>
>>>>>>>>>> nicely
>>>>>>>>>>
>>>>>>>>>> done!
>>>>>>>>>>
>>>>>>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <becket.qin@gmail.com
>>>>>>>> <ma...@gmail.com>> <becket.qin@gmail.com <mailto:
>>>>>>>> becket.qin@gmail.com>>
>>>>>>>>>>
>>>>>>>>>> <
>>>>>>>>>>
>>>>>>>>>> becket.qin@gmail.com <ma...@gmail.com>> wrote:
>>>>>>>>>>
>>>>>>>>>> Hi all,
>>>>>>>>>>
>>>>>>>>>> Sorry for the long belated update. I have updated FLIP-27 wiki
>>>>>>>>>>
>>>>>>>>>> page
>>>>>>>>>>
>>>>>>>>>> with
>>>>>>>>>>
>>>>>>>>>> the latest proposals. Some noticeable changes include:
>>>>>>>>>> 1. A new generic communication mechanism between SplitEnumerator
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>> SourceReader.
>>>>>>>>>> 2. Some detail API method signature changes.
>>>>>>>>>>
>>>>>>>>>> We left a few things out of this FLIP and will address them in
>>>>>>>>>>
>>>>>>>>>> separate
>>>>>>>>>>
>>>>>>>>>> FLIPs. Including:
>>>>>>>>>> 1. Per split event time.
>>>>>>>>>> 2. Event time alignment.
>>>>>>>>>> 3. Fine grained failover for SplitEnumerator failure.
>>>>>>>>>>
>>>>>>>>>> Please let us know if you have any question.
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>>
>>>>>>>>>> Jiangjie (Becket) Qin
>>>>>>>>>>
>>>>>>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <sewen@apache.org
>>>>>>> <mailto:
>>>>>>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>> <
>>>>>>>>>>
>>>>>>>>>> sewen@apache.org <ma...@apache.org>> wrote:
>>>>>>>>>>
>>>>>>>>>> Hi  Łukasz!
>>>>>>>>>>
>>>>>>>>>> Becket and me are working hard on figuring out the last details
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>> implementing the first PoC. We would update the FLIP hopefully
>>>>>>>>>>
>>>>>>>>>> next
>>>>>>>>>>
>>>>>>>>>> week.
>>>>>>>>>>
>>>>>>>>>> There is a fair chance that a first version of this will be in
>>>>>>>>>>
>>>>>>>>>> 1.10,
>>>>>>>>>>
>>>>>>>>>> but
>>>>>>>>>>
>>>>>>>>>> I
>>>>>>>>>>
>>>>>>>>>> think it will take another release to battle test it and migrate
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> connectors.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> Stephan
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <
>> ljd@touk.pl
>>>>>>>> <ma...@touk.pl>
>>>>>>>>>>
>>>>>>>>>> <
>>>>>>>>>>
>>>>>>>>>> ljd@touk.pl <ma...@touk.pl>>
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> Hi,
>>>>>>>>>>
>>>>>>>>>> This proposal looks very promising for us. Do you have any plans
>>>>>>>>>>
>>>>>>>>>> in
>>>>>>>>>>
>>>>>>>>>> which
>>>>>>>>>>
>>>>>>>>>> Flink release it is going to be released? We are thinking on
>>>>>>>>>>
>>>>>>>>>> using a
>>>>>>>>>>
>>>>>>>>>> Data
>>>>>>>>>>
>>>>>>>>>> Set API for our future use cases but on the other hand Data Set
>>>>>>>>>>
>>>>>>>>>> API
>>>>>>>>>>
>>>>>>>>>> is
>>>>>>>>>>
>>>>>>>>>> going to be deprecated so using proposed bounded data streams
>>>>>>>>>>
>>>>>>>>>> solution
>>>>>>>>>>
>>>>>>>>>> could be more viable in the long term.
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Łukasz
>>>>>>>>>>
>>>>>>>>>> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com
>>>>>> <mailto:
>>>>>>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
>>>>>>>> thomas.weise@gmail.com>> <
>>>>>>>>>>
>>>>>>>>>> thomas.weise@gmail.com <ma...@gmail.com>> wrote:
>>>>>>>>>>
>>>>>>>>>> Thanks for putting together this proposal!
>>>>>>>>>>
>>>>>>>>>> I see that the "Per Split Event Time" and "Event Time Alignment"
>>>>>>>>>>
>>>>>>>>>> sections
>>>>>>>>>>
>>>>>>>>>> are still TBD.
>>>>>>>>>>
>>>>>>>>>> It would probably be good to flesh those out a bit before
>>>>>>>>>>
>>>>>>>>>> proceeding
>>>>>>>>>>
>>>>>>>>>> too
>>>>>>>>>>
>>>>>>>>>> far
>>>>>>>>>>
>>>>>>>>>> as the event time alignment will probably influence the
>>>>>>>>>>
>>>>>>>>>> interaction
>>>>>>>>>>
>>>>>>>>>> with
>>>>>>>>>>
>>>>>>>>>> the split reader, specifically ReaderStatus
>>>>>>>>>>
>>>>>>>>>> emitNext(SourceOutput<E>
>>>>>>>>>>
>>>>>>>>>> output).
>>>>>>>>>>
>>>>>>>>>> We currently have only one implementation for event time
>> alignment
>>>>>>>>>>
>>>>>>>>>> in
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> Kinesis consumer. The synchronization in that case takes place
>> as
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> last
>>>>>>>>>>
>>>>>>>>>> step before records are emitted downstream (RecordEmitter). With
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> currently proposed interfaces, the equivalent can be implemented
>>>>>>>>>>
>>>>>>>>>> in
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> reader loop, although note that in the Kinesis consumer the per
>>>>>>>>>>
>>>>>>>>>> shard
>>>>>>>>>>
>>>>>>>>>> threads push records.
>>>>>>>>>>
>>>>>>>>>> Synchronization has not been implemented for the Kafka consumer
>>>>>>>>>>
>>>>>>>>>> yet.
>>>>>>>>>>
>>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
>>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675>
>>>>>>>>>>
>>>>>>>>>> When I looked at it, I realized that the implementation will
>> look
>>>>>>>>>>
>>>>>>>>>> quite
>>>>>>>>>>
>>>>>>>>>> different
>>>>>>>>>> from Kinesis because it needs to take place in the pull part,
>>>>>>>>>>
>>>>>>>>>> where
>>>>>>>>>>
>>>>>>>>>> records
>>>>>>>>>>
>>>>>>>>>> are taken from the Kafka client. Due to the multiplexing it
>> cannot
>>>>>>>>>>
>>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>> done
>>>>>>>>>>
>>>>>>>>>> by blocking the split thread like it currently works for
>> Kinesis.
>>>>>>>>>>
>>>>>>>>>> Reading
>>>>>>>>>>
>>>>>>>>>> from individual Kafka partitions needs to be controlled via
>>>>>>>>>>
>>>>>>>>>> pause/resume
>>>>>>>>>>
>>>>>>>>>> on the Kafka client.
>>>>>>>>>>
>>>>>>>>>> To take on that responsibility the split thread would need to be
>>>>>>>>>>
>>>>>>>>>> aware
>>>>>>>>>>
>>>>>>>>>> of
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>> watermarks or at least whether it should or should not continue
>> to
>>>>>>>>>>
>>>>>>>>>> consume
>>>>>>>>>>
>>>>>>>>>> a given split and this may require a different SourceReader or
>>>>>>>>>>
>>>>>>>>>> SourceOutput
>>>>>>>>>>
>>>>>>>>>> interface.
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Thomas
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com
>>>>>> <mailto:
>>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> mmyy1110@gmail.com
>>>>>
>>>>>> <
>>>>>>>>>>
>>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
>>>>>>>>>>
>>>>>>>>>> Hi Stephan,
>>>>>>>>>>
>>>>>>>>>> Thank you for feedback!
>>>>>>>>>> Will take a look at your branch before public discussing.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <sewen@apache.org
>>>>>>>> <ma...@apache.org>> <sewen@apache.org <mailto:
>>> sewen@apache.org
>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> <
>>>>>>>>>>
>>>>>>>>>> sewen@apache.org <ma...@apache.org>>
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> Hi Biao!
>>>>>>>>>>
>>>>>>>>>> Thanks for reviving this. I would like to join this discussion,
>>>>>>>>>>
>>>>>>>>>> but
>>>>>>>>>>
>>>>>>>>>> am
>>>>>>>>>>
>>>>>>>>>> quite occupied with the 1.9 release, so can we maybe pause this
>>>>>>>>>>
>>>>>>>>>> discussion
>>>>>>>>>>
>>>>>>>>>> for a week or so?
>>>>>>>>>>
>>>>>>>>>> In the meantime I can share some suggestion based on prior
>>>>>>>>>>
>>>>>>>>>> experiments:
>>>>>>>>>>
>>>>>>>>>> How to do watermarks / timestamp extractors in a simpler and
>> more
>>>>>>>>>>
>>>>>>>>>> flexible
>>>>>>>>>>
>>>>>>>>>> way. I think that part is quite promising should be part of the
>>>>>>>>>>
>>>>>>>>>> new
>>>>>>>>>>
>>>>>>>>>> source
>>>>>>>>>>
>>>>>>>>>> interface.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>>>>>>>> <
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>>>>>>>> <
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Some experiments on how to build the source reader and its
>>>>>>>>>>
>>>>>>>>>> library
>>>>>>>>>>
>>>>>>>>>> for
>>>>>>>>>>
>>>>>>>>>> common threading/split patterns:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>>>>>>>> <
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> Stephan
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mmyy1110@gmail.com
>>>>>>> <mailto:
>>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> mmyy1110@gmail.com
>>>>>
>>>>>> <
>>>>>>>>>>
>>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> Hi devs,
>>>>>>>>>>
>>>>>>>>>> Since 1.9 is nearly released, I think we could get back to
>>>>>>>>>>
>>>>>>>>>> FLIP-27.
>>>>>>>>>>
>>>>>>>>>> I
>>>>>>>>>>
>>>>>>>>>> believe it should be included in 1.10.
>>>>>>>>>>
>>>>>>>>>> There are so many things mentioned in document of FLIP-27. [1] I
>>>>>>>>>>
>>>>>>>>>> think
>>>>>>>>>>
>>>>>>>>>> we'd better discuss them separately. However the wiki is not a
>>>>>>>>>>
>>>>>>>>>> good
>>>>>>>>>>
>>>>>>>>>> place
>>>>>>>>>>
>>>>>>>>>> to discuss. I wrote google doc about SplitReader API which
>>>>>>>>>>
>>>>>>>>>> misses
>>>>>>>>>>
>>>>>>>>>> some
>>>>>>>>>>
>>>>>>>>>> details in the document. [2]
>>>>>>>>>>
>>>>>>>>>> 1.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>>>>>>>> <
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 2.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>>>>>>>> <
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com
>>>>>> <mailto:
>>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> mmyy1110@gmail.com
>>>>>
>>>>>> <
>>>>>>>>>>
>>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> Hi Steven,
>>>>>>>>>> Thank you for the feedback. Please take a look at the document
>>>>>>>>>>
>>>>>>>>>> FLIP-27
>>>>>>>>>>
>>>>>>>>>> <
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>>> <
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> which
>>>>>>>>>>
>>>>>>>>>> is updated recently. A lot of details of enumerator were added
>>>>>>>>>>
>>>>>>>>>> in
>>>>>>>>>>
>>>>>>>>>> this
>>>>>>>>>>
>>>>>>>>>> document. I think it would help.
>>>>>>>>>>
>>>>>>>>>> Steven Wu <stevenz3wu@gmail.com <ma...@gmail.com>>
>> <
>>>>>>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
>>>>>>> stevenz3wu@gmail.com
>>>>>>>> <ma...@gmail.com>> <stevenz3wu@gmail.com <mailto:
>>>>>>>> stevenz3wu@gmail.com>>
>>>>>>>>>>
>>>>>>>>>> 于2019年3月28日周四
>>>>>>>>>>
>>>>>>>>>> 下午12:52写道:
>>>>>>>>>>
>>>>>>>>>> This proposal mentioned that SplitEnumerator might run on the
>>>>>>>>>> JobManager or
>>>>>>>>>> in a single task on a TaskManager.
>>>>>>>>>>
>>>>>>>>>> if enumerator is a single task on a taskmanager, then the job
>>>>>>>>>>
>>>>>>>>>> DAG
>>>>>>>>>>
>>>>>>>>>> can
>>>>>>>>>>
>>>>>>>>>> never
>>>>>>>>>> been embarrassingly parallel anymore. That will nullify the
>>>>>>>>>>
>>>>>>>>>> leverage
>>>>>>>>>>
>>>>>>>>>> of
>>>>>>>>>>
>>>>>>>>>> fine-grained recovery for embarrassingly parallel jobs.
>>>>>>>>>>
>>>>>>>>>> It's not clear to me what's the implication of running
>>>>>>>>>>
>>>>>>>>>> enumerator
>>>>>>>>>>
>>>>>>>>>> on
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> jobmanager. So I will leave that out for now.
>>>>>>>>>>
>>>>>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com
>>>>>> <mailto:
>>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> mmyy1110@gmail.com
>>>>>
>>>>>> <
>>>>>>>>>>
>>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> Hi Stephan & Piotrek,
>>>>>>>>>>
>>>>>>>>>> Thank you for feedback.
>>>>>>>>>>
>>>>>>>>>> It seems that there are a lot of things to do in community.
>>>>>>>>>>
>>>>>>>>>> I
>>>>>>>>>>
>>>>>>>>>> am
>>>>>>>>>>
>>>>>>>>>> just
>>>>>>>>>>
>>>>>>>>>> afraid that this discussion may be forgotten since there so
>>>>>>>>>>
>>>>>>>>>> many
>>>>>>>>>>
>>>>>>>>>> proposals
>>>>>>>>>>
>>>>>>>>>> recently.
>>>>>>>>>> Anyway, wish to see the split topics soon :)
>>>>>>>>>>
>>>>>>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:
>>> piotr@da-platform.com
>>>>>>>>
>>>>>>> <
>>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
>>>>>>>> piotr@da-platform.com <ma...@da-platform.com>> <
>>>>>>>> piotr@da-platform.com <ma...@da-platform.com>>
>>>>>>>>>>
>>>>>>>>>> 于2019年1月24日周四
>>>>>>>>>>
>>>>>>>>>> 下午8:21写道:
>>>>>>>>>>
>>>>>>>>>> Hi Biao!
>>>>>>>>>>
>>>>>>>>>> This discussion was stalled because of preparations for
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> open
>>>>>>>>>>
>>>>>>>>>> sourcing
>>>>>>>>>>
>>>>>>>>>> & merging Blink. I think before creating the tickets we
>>>>>>>>>>
>>>>>>>>>> should
>>>>>>>>>>
>>>>>>>>>> split this
>>>>>>>>>>
>>>>>>>>>> discussion into topics/areas outlined by Stephan and
>>>>>>>>>>
>>>>>>>>>> create
>>>>>>>>>>
>>>>>>>>>> Flips
>>>>>>>>>>
>>>>>>>>>> for
>>>>>>>>>>
>>>>>>>>>> that.
>>>>>>>>>>
>>>>>>>>>> I think there is no chance for this to be completed in
>>>>>>>>>>
>>>>>>>>>> couple
>>>>>>>>>>
>>>>>>>>>> of
>>>>>>>>>>
>>>>>>>>>> remaining
>>>>>>>>>>
>>>>>>>>>> weeks/1 month before 1.8 feature freeze, however it would
>>>>>>>>>>
>>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>> good
>>>>>>>>>>
>>>>>>>>>> to aim
>>>>>>>>>>
>>>>>>>>>> with those changes for 1.9.
>>>>>>>>>>
>>>>>>>>>> Piotrek
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com <mailto:
>>>>>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
>> mmyy1110@gmail.com
>>>>>
>>>>>> <
>>>>>>>>>>
>>>>>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> Hi community,
>>>>>>>>>> The summary of Stephan makes a lot sense to me. It is
>>>>>>>>>>
>>>>>>>>>> much
>>>>>>>>>>
>>>>>>>>>> clearer
>>>>>>>>>>
>>>>>>>>>> indeed
>>>>>>>>>>
>>>>>>>>>> after splitting the complex topic into small ones.
>>>>>>>>>> I was wondering is there any detail plan for next step?
>>>>>>>>>>
>>>>>>>>>> If
>>>>>>>>>>
>>>>>>>>>> not,
>>>>>>>>>>
>>>>>>>>>> I
>>>>>>>>>>
>>>>>>>>>> would
>>>>>>>>>>
>>>>>>>>>> like to push this thing forward by creating some JIRA
>>>>>>>>>>
>>>>>>>>>> issues.
>>>>>>>>>>
>>>>>>>>>> Another question is that should version 1.8 include
>>>>>>>>>>
>>>>>>>>>> these
>>>>>>>>>>
>>>>>>>>>> features?
>>>>>>>>>>
>>>>>>>>>> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <
>>>>>>>> sewen@apache.org <ma...@apache.org>> <sewen@apache.org
>>>> <mailto:
>>>>>>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>>
>>>>>>>> 于2018年12月1日周六
>>>>>>>>>>
>>>>>>>>>> 上午4:20写道:
>>>>>>>>>>
>>>>>>>>>> Thanks everyone for the lively discussion. Let me try
>>>>>>>>>>
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>> summarize
>>>>>>>>>>
>>>>>>>>>> where I
>>>>>>>>>>
>>>>>>>>>> see convergence in the discussion and open issues.
>>>>>>>>>> I'll try to group this by design aspect of the source.
>>>>>>>>>>
>>>>>>>>>> Please
>>>>>>>>>>
>>>>>>>>>> let me
>>>>>>>>>>
>>>>>>>>>> know
>>>>>>>>>>
>>>>>>>>>> if I got things wrong or missed something crucial here.
>>>>>>>>>>
>>>>>>>>>> For issues 1-3, if the below reflects the state of the
>>>>>>>>>>
>>>>>>>>>> discussion, I
>>>>>>>>>>
>>>>>>>>>> would
>>>>>>>>>>
>>>>>>>>>> try and update the FLIP in the next days.
>>>>>>>>>> For the remaining ones we need more discussion.
>>>>>>>>>>
>>>>>>>>>> I would suggest to fork each of these aspects into a
>>>>>>>>>>
>>>>>>>>>> separate
>>>>>>>>>>
>>>>>>>>>> mail
>>>>>>>>>>
>>>>>>>>>> thread,
>>>>>>>>>>
>>>>>>>>>> or will loose sight of the individual aspects.
>>>>>>>>>>
>>>>>>>>>> *(1) Separation of Split Enumerator and Split Reader*
>>>>>>>>>>
>>>>>>>>>> - All seem to agree this is a good thing
>>>>>>>>>> - Split Enumerator could in the end live on JobManager
>>>>>>>>>>
>>>>>>>>>> (and
>>>>>>>>>>
>>>>>>>>>> assign
>>>>>>>>>>
>>>>>>>>>> splits
>>>>>>>>>>
>>>>>>>>>> via RPC) or in a task (and assign splits via data
>>>>>>>>>>
>>>>>>>>>> streams)
>>>>>>>>>>
>>>>>>>>>> - this discussion is orthogonal and should come later,
>>>>>>>>>>
>>>>>>>>>> when
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> interface
>>>>>>>>>>
>>>>>>>>>> is agreed upon.
>>>>>>>>>>
>>>>>>>>>> *(2) Split Readers for one or more splits*
>>>>>>>>>>
>>>>>>>>>> - Discussion seems to agree that we need to support
>>>>>>>>>>
>>>>>>>>>> one
>>>>>>>>>>
>>>>>>>>>> reader
>>>>>>>>>>
>>>>>>>>>> that
>>>>>>>>>>
>>>>>>>>>> possibly handles multiple splits concurrently.
>>>>>>>>>> - The requirement comes from sources where one
>>>>>>>>>>
>>>>>>>>>> poll()-style
>>>>>>>>>>
>>>>>>>>>> call
>>>>>>>>>>
>>>>>>>>>> fetches
>>>>>>>>>>
>>>>>>>>>> data from different splits / partitions
>>>>>>>>>>     --> example sources that require that would be for
>>>>>>>>>>
>>>>>>>>>> example
>>>>>>>>>>
>>>>>>>>>> Kafka,
>>>>>>>>>>
>>>>>>>>>> Pravega, Pulsar
>>>>>>>>>>
>>>>>>>>>> - Could have one split reader per source, or multiple
>>>>>>>>>>
>>>>>>>>>> split
>>>>>>>>>>
>>>>>>>>>> readers
>>>>>>>>>>
>>>>>>>>>> that
>>>>>>>>>>
>>>>>>>>>> share the "poll()" function
>>>>>>>>>> - To not make it too complicated, we can start with
>>>>>>>>>>
>>>>>>>>>> thinking
>>>>>>>>>>
>>>>>>>>>> about
>>>>>>>>>>
>>>>>>>>>> one
>>>>>>>>>>
>>>>>>>>>> split reader for all splits initially and see if that
>>>>>>>>>>
>>>>>>>>>> covers
>>>>>>>>>>
>>>>>>>>>> all
>>>>>>>>>>
>>>>>>>>>> requirements
>>>>>>>>>>
>>>>>>>>>> *(3) Threading model of the Split Reader*
>>>>>>>>>>
>>>>>>>>>> - Most active part of the discussion ;-)
>>>>>>>>>>
>>>>>>>>>> - A non-blocking way for Flink's task code to interact
>>>>>>>>>>
>>>>>>>>>> with
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> source
>>>>>>>>>>
>>>>>>>>>> is
>>>>>>>>>>
>>>>>>>>>> needed in order to a task runtime code based on a
>>>>>>>>>> single-threaded/actor-style task design
>>>>>>>>>>     --> I personally am a big proponent of that, it will
>>>>>>>>>>
>>>>>>>>>> help
>>>>>>>>>>
>>>>>>>>>> with
>>>>>>>>>>
>>>>>>>>>> well-behaved checkpoints, efficiency, and simpler yet
>>>>>>>>>>
>>>>>>>>>> more
>>>>>>>>>>
>>>>>>>>>> robust
>>>>>>>>>>
>>>>>>>>>> runtime
>>>>>>>>>>
>>>>>>>>>> code
>>>>>>>>>>
>>>>>>>>>> - Users care about simple abstraction, so as a
>>>>>>>>>>
>>>>>>>>>> subclass
>>>>>>>>>>
>>>>>>>>>> of
>>>>>>>>>>
>>>>>>>>>> SplitReader
>>>>>>>>>>
>>>>>>>>>> (non-blocking / async) we need to have a
>>>>>>>>>>
>>>>>>>>>> BlockingSplitReader
>>>>>>>>>>
>>>>>>>>>> which
>>>>>>>>>>
>>>>>>>>>> will
>>>>>>>>>>
>>>>>>>>>> form the basis of most source implementations.
>>>>>>>>>>
>>>>>>>>>> BlockingSplitReader
>>>>>>>>>>
>>>>>>>>>> lets
>>>>>>>>>>
>>>>>>>>>> users do blocking simple poll() calls.
>>>>>>>>>> - The BlockingSplitReader would spawn a thread (or
>>>>>>>>>>
>>>>>>>>>> more)
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> thread(s) can make blocking calls and hand over data
>>>>>>>>>>
>>>>>>>>>> buffers
>>>>>>>>>>
>>>>>>>>>> via
>>>>>>>>>>
>>>>>>>>>> a
>>>>>>>>>>
>>>>>>>>>> blocking
>>>>>>>>>>
>>>>>>>>>> queue
>>>>>>>>>> - This should allow us to cover both, a fully async
>>>>>>>>>>
>>>>>>>>>> runtime,
>>>>>>>>>>
>>>>>>>>>> and a
>>>>>>>>>>
>>>>>>>>>> simple
>>>>>>>>>>
>>>>>>>>>> blocking interface for users.
>>>>>>>>>> - This is actually very similar to how the Kafka
>>>>>>>>>>
>>>>>>>>>> connectors
>>>>>>>>>>
>>>>>>>>>> work.
>>>>>>>>>>
>>>>>>>>>> Kafka
>>>>>>>>>>
>>>>>>>>>> 9+ with one thread, Kafka 8 with multiple threads
>>>>>>>>>>
>>>>>>>>>> - On the base SplitReader (the async one), the
>>>>>>>>>>
>>>>>>>>>> non-blocking
>>>>>>>>>>
>>>>>>>>>> method
>>>>>>>>>>
>>>>>>>>>> that
>>>>>>>>>>
>>>>>>>>>> gets the next chunk of data would signal data
>>>>>>>>>>
>>>>>>>>>> availability
>>>>>>>>>>
>>>>>>>>>> via
>>>>>>>>>>
>>>>>>>>>> a
>>>>>>>>>>
>>>>>>>>>> CompletableFuture, because that gives the best
>>>>>>>>>>
>>>>>>>>>> flexibility
>>>>>>>>>>
>>>>>>>>>> (can
>>>>>>>>>>
>>>>>>>>>> await
>>>>>>>>>>
>>>>>>>>>> completion or register notification handlers).
>>>>>>>>>> - The source task would register a "thenHandle()" (or
>>>>>>>>>>
>>>>>>>>>> similar)
>>>>>>>>>>
>>>>>>>>>> on the
>>>>>>>>>>
>>>>>>>>>> future to put a "take next data" task into the
>>>>>>>>>>
>>>>>>>>>> actor-style
>>>>>>>>>>
>>>>>>>>>> mailbox
>>>>>>>>>>
>>>>>>>>>> *(4) Split Enumeration and Assignment*
>>>>>>>>>>
>>>>>>>>>> - Splits may be generated lazily, both in cases where
>>>>>>>>>>
>>>>>>>>>> there
>>>>>>>>>>
>>>>>>>>>> is a
>>>>>>>>>>
>>>>>>>>>> limited
>>>>>>>>>>
>>>>>>>>>> number of splits (but very many), or splits are
>>>>>>>>>>
>>>>>>>>>> discovered
>>>>>>>>>>
>>>>>>>>>> over
>>>>>>>>>>
>>>>>>>>>> time
>>>>>>>>>>
>>>>>>>>>> - Assignment should also be lazy, to get better load
>>>>>>>>>>
>>>>>>>>>> balancing
>>>>>>>>>>
>>>>>>>>>> - Assignment needs support locality preferences
>>>>>>>>>>
>>>>>>>>>> - Possible design based on discussion so far:
>>>>>>>>>>
>>>>>>>>>>     --> SplitReader has a method "addSplits(SplitT...)"
>>>>>>>>>>
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>> add
>>>>>>>>>>
>>>>>>>>>> one or
>>>>>>>>>>
>>>>>>>>>> more
>>>>>>>>>>
>>>>>>>>>> splits. Some split readers might assume they have only
>>>>>>>>>>
>>>>>>>>>> one
>>>>>>>>>>
>>>>>>>>>> split
>>>>>>>>>>
>>>>>>>>>> ever,
>>>>>>>>>>
>>>>>>>>>> concurrently, others assume multiple splits. (Note:
>>>>>>>>>>
>>>>>>>>>> idea
>>>>>>>>>>
>>>>>>>>>> behind
>>>>>>>>>>
>>>>>>>>>> being
>>>>>>>>>>
>>>>>>>>>> able
>>>>>>>>>>
>>>>>>>>>> to add multiple splits at the same time is to ease
>>>>>>>>>>
>>>>>>>>>> startup
>>>>>>>>>>
>>>>>>>>>> where
>>>>>>>>>>
>>>>>>>>>> multiple
>>>>>>>>>>
>>>>>>>>>> splits may be assigned instantly.)
>>>>>>>>>>     --> SplitReader has a context object on which it can
>>>>>>>>>>
>>>>>>>>>> call
>>>>>>>>>>
>>>>>>>>>> indicate
>>>>>>>>>>
>>>>>>>>>> when
>>>>>>>>>>
>>>>>>>>>> splits are completed. The enumerator gets that
>>>>>>>>>>
>>>>>>>>>> notification and
>>>>>>>>>>
>>>>>>>>>> can
>>>>>>>>>>
>>>>>>>>>> use
>>>>>>>>>>
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>> decide when to assign new splits. This should help both
>>>>>>>>>>
>>>>>>>>>> in
>>>>>>>>>>
>>>>>>>>>> cases
>>>>>>>>>>
>>>>>>>>>> of
>>>>>>>>>>
>>>>>>>>>> sources
>>>>>>>>>>
>>>>>>>>>> that take splits lazily (file readers) and in case the
>>>>>>>>>>
>>>>>>>>>> source
>>>>>>>>>>
>>>>>>>>>> needs to
>>>>>>>>>>
>>>>>>>>>> preserve a partial order between splits (Kinesis,
>>>>>>>>>>
>>>>>>>>>> Pravega,
>>>>>>>>>>
>>>>>>>>>> Pulsar may
>>>>>>>>>>
>>>>>>>>>> need
>>>>>>>>>>
>>>>>>>>>> that).
>>>>>>>>>>     --> SplitEnumerator gets notification when
>>>>>>>>>>
>>>>>>>>>> SplitReaders
>>>>>>>>>>
>>>>>>>>>> start
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>> when
>>>>>>>>>>
>>>>>>>>>> they finish splits. They can decide at that moment to
>>>>>>>>>>
>>>>>>>>>> push
>>>>>>>>>>
>>>>>>>>>> more
>>>>>>>>>>
>>>>>>>>>> splits
>>>>>>>>>>
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>> that reader
>>>>>>>>>>     --> The SplitEnumerator should probably be aware of
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> source
>>>>>>>>>>
>>>>>>>>>> parallelism, to build its initial distribution.
>>>>>>>>>>
>>>>>>>>>> - Open question: Should the source expose something
>>>>>>>>>>
>>>>>>>>>> like
>>>>>>>>>>
>>>>>>>>>> "host
>>>>>>>>>>
>>>>>>>>>> preferences", so that yarn/mesos/k8s can take this into
>>>>>>>>>>
>>>>>>>>>> account
>>>>>>>>>>
>>>>>>>>>> when
>>>>>>>>>>
>>>>>>>>>> selecting a node to start a TM on?
>>>>>>>>>>
>>>>>>>>>> *(5) Watermarks and event time alignment*
>>>>>>>>>>
>>>>>>>>>> - Watermark generation, as well as idleness, needs to
>>>>>>>>>>
>>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>> per
>>>>>>>>>>
>>>>>>>>>> split
>>>>>>>>>>
>>>>>>>>>> (like
>>>>>>>>>>
>>>>>>>>>> currently in the Kafka Source, per partition)
>>>>>>>>>> - It is desirable to support optional
>>>>>>>>>>
>>>>>>>>>> event-time-alignment,
>>>>>>>>>>
>>>>>>>>>> meaning
>>>>>>>>>>
>>>>>>>>>> that
>>>>>>>>>>
>>>>>>>>>> splits that are ahead are back-pressured or temporarily
>>>>>>>>>>
>>>>>>>>>> unsubscribed
>>>>>>>>>>
>>>>>>>>>> - I think i would be desirable to encapsulate
>>>>>>>>>>
>>>>>>>>>> watermark
>>>>>>>>>>
>>>>>>>>>> generation
>>>>>>>>>>
>>>>>>>>>> logic
>>>>>>>>>>
>>>>>>>>>> in watermark generators, for a separation of concerns.
>>>>>>>>>>
>>>>>>>>>> The
>>>>>>>>>>
>>>>>>>>>> watermark
>>>>>>>>>>
>>>>>>>>>> generators should run per split.
>>>>>>>>>> - Using watermark generators would also help with
>>>>>>>>>>
>>>>>>>>>> another
>>>>>>>>>>
>>>>>>>>>> problem of
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> suggested interface, namely supporting non-periodic
>>>>>>>>>>
>>>>>>>>>> watermarks
>>>>>>>>>>
>>>>>>>>>> efficiently.
>>>>>>>>>>
>>>>>>>>>> - Need a way to "dispatch" next record to different
>>>>>>>>>>
>>>>>>>>>> watermark
>>>>>>>>>>
>>>>>>>>>> generators
>>>>>>>>>>
>>>>>>>>>> - Need a way to tell SplitReader to "suspend" a split
>>>>>>>>>>
>>>>>>>>>> until a
>>>>>>>>>>
>>>>>>>>>> certain
>>>>>>>>>>
>>>>>>>>>> watermark is reached (event time backpressure)
>>>>>>>>>> - This would in fact be not needed (and thus simpler)
>>>>>>>>>>
>>>>>>>>>> if
>>>>>>>>>>
>>>>>>>>>> we
>>>>>>>>>>
>>>>>>>>>> had
>>>>>>>>>>
>>>>>>>>>> a
>>>>>>>>>>
>>>>>>>>>> SplitReader per split and may be a reason to re-open
>>>>>>>>>>
>>>>>>>>>> that
>>>>>>>>>>
>>>>>>>>>> discussion
>>>>>>>>>>
>>>>>>>>>> *(6) Watermarks across splits and in the Split
>>>>>>>>>>
>>>>>>>>>> Enumerator*
>>>>>>>>>>
>>>>>>>>>> - The split enumerator may need some watermark
>>>>>>>>>>
>>>>>>>>>> awareness,
>>>>>>>>>>
>>>>>>>>>> which
>>>>>>>>>>
>>>>>>>>>> should
>>>>>>>>>>
>>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>> purely based on split metadata (like create timestamp
>>>>>>>>>>
>>>>>>>>>> of
>>>>>>>>>>
>>>>>>>>>> file
>>>>>>>>>>
>>>>>>>>>> splits)
>>>>>>>>>>
>>>>>>>>>> - If there are still more splits with overlapping
>>>>>>>>>>
>>>>>>>>>> event
>>>>>>>>>>
>>>>>>>>>> time
>>>>>>>>>>
>>>>>>>>>> range
>>>>>>>>>>
>>>>>>>>>> for
>>>>>>>>>>
>>>>>>>>>> a
>>>>>>>>>>
>>>>>>>>>> split reader, then that split reader should not advance
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> watermark
>>>>>>>>>>
>>>>>>>>>> within the split beyond the overlap boundary. Otherwise
>>>>>>>>>>
>>>>>>>>>> future
>>>>>>>>>>
>>>>>>>>>> splits
>>>>>>>>>>
>>>>>>>>>> will
>>>>>>>>>>
>>>>>>>>>> produce late data.
>>>>>>>>>>
>>>>>>>>>> - One way to approach this could be that the split
>>>>>>>>>>
>>>>>>>>>> enumerator
>>>>>>>>>>
>>>>>>>>>> may
>>>>>>>>>>
>>>>>>>>>> send
>>>>>>>>>>
>>>>>>>>>> watermarks to the readers, and the readers cannot emit
>>>>>>>>>>
>>>>>>>>>> watermarks
>>>>>>>>>>
>>>>>>>>>> beyond
>>>>>>>>>>
>>>>>>>>>> that received watermark.
>>>>>>>>>> - Many split enumerators would simply immediately send
>>>>>>>>>>
>>>>>>>>>> Long.MAX
>>>>>>>>>>
>>>>>>>>>> out
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>> leave the progress purely to the split readers.
>>>>>>>>>>
>>>>>>>>>> - For event-time alignment / split back pressure, this
>>>>>>>>>>
>>>>>>>>>> begs
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> question
>>>>>>>>>>
>>>>>>>>>> how we can avoid deadlocks that may arise when splits
>>>>>>>>>>
>>>>>>>>>> are
>>>>>>>>>>
>>>>>>>>>> suspended
>>>>>>>>>>
>>>>>>>>>> for
>>>>>>>>>>
>>>>>>>>>> event time back pressure,
>>>>>>>>>>
>>>>>>>>>> *(7) Batch and streaming Unification*
>>>>>>>>>>
>>>>>>>>>> - Functionality wise, the above design should support
>>>>>>>>>>
>>>>>>>>>> both
>>>>>>>>>>
>>>>>>>>>> - Batch often (mostly) does not care about reading "in
>>>>>>>>>>
>>>>>>>>>> order"
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>> generating watermarks
>>>>>>>>>>     --> Might use different enumerator logic that is
>>>>>>>>>>
>>>>>>>>>> more
>>>>>>>>>>
>>>>>>>>>> locality
>>>>>>>>>>
>>>>>>>>>> aware
>>>>>>>>>>
>>>>>>>>>> and ignores event time order
>>>>>>>>>>     --> Does not generate watermarks
>>>>>>>>>> - Would be great if bounded sources could be
>>>>>>>>>>
>>>>>>>>>> identified
>>>>>>>>>>
>>>>>>>>>> at
>>>>>>>>>>
>>>>>>>>>> compile
>>>>>>>>>>
>>>>>>>>>> time,
>>>>>>>>>>
>>>>>>>>>> so that "env.addBoundedSource(...)" is type safe and
>>>>>>>>>>
>>>>>>>>>> can
>>>>>>>>>>
>>>>>>>>>> return a
>>>>>>>>>>
>>>>>>>>>> "BoundedDataStream".
>>>>>>>>>> - Possible to defer this discussion until later
>>>>>>>>>>
>>>>>>>>>> *Miscellaneous Comments*
>>>>>>>>>>
>>>>>>>>>> - Should the source have a TypeInformation for the
>>>>>>>>>>
>>>>>>>>>> produced
>>>>>>>>>>
>>>>>>>>>> type,
>>>>>>>>>>
>>>>>>>>>> instead
>>>>>>>>>>
>>>>>>>>>> of a serializer? We need a type information in the
>>>>>>>>>>
>>>>>>>>>> stream
>>>>>>>>>>
>>>>>>>>>> anyways, and
>>>>>>>>>>
>>>>>>>>>> can
>>>>>>>>>>
>>>>>>>>>> derive the serializer from that. Plus, creating the
>>>>>>>>>>
>>>>>>>>>> serializer
>>>>>>>>>>
>>>>>>>>>> should
>>>>>>>>>>
>>>>>>>>>> respect the ExecutionConfig.
>>>>>>>>>>
>>>>>>>>>> - The TypeSerializer interface is very powerful but
>>>>>>>>>>
>>>>>>>>>> also
>>>>>>>>>>
>>>>>>>>>> not
>>>>>>>>>>
>>>>>>>>>> easy to
>>>>>>>>>>
>>>>>>>>>> implement. Its purpose is to handle data super
>>>>>>>>>>
>>>>>>>>>> efficiently,
>>>>>>>>>>
>>>>>>>>>> support
>>>>>>>>>>
>>>>>>>>>> flexible ways of evolution, etc.
>>>>>>>>>> For metadata I would suggest to look at the
>>>>>>>>>>
>>>>>>>>>> SimpleVersionedSerializer
>>>>>>>>>>
>>>>>>>>>> instead, which is used for example for checkpoint
>>>>>>>>>>
>>>>>>>>>> master
>>>>>>>>>>
>>>>>>>>>> hooks,
>>>>>>>>>>
>>>>>>>>>> or for
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> streaming file sink. I think that is is a good match
>>>>>>>>>>
>>>>>>>>>> for
>>>>>>>>>>
>>>>>>>>>> cases
>>>>>>>>>>
>>>>>>>>>> where
>>>>>>>>>>
>>>>>>>>>> we
>>>>>>>>>>
>>>>>>>>>> do
>>>>>>>>>>
>>>>>>>>>> not need more than ser/deser (no copy, etc.) and don't
>>>>>>>>>>
>>>>>>>>>> need to
>>>>>>>>>>
>>>>>>>>>> push
>>>>>>>>>>
>>>>>>>>>> versioning out of the serialization paths for best
>>>>>>>>>>
>>>>>>>>>> performance
>>>>>>>>>>
>>>>>>>>>> (as in
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> TypeSerializer)
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>>>>>>>>>>
>>>>>>>>>> k.kloudas@data-artisans.com>
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Hi Biao,
>>>>>>>>>>
>>>>>>>>>> Thanks for the answer!
>>>>>>>>>>
>>>>>>>>>> So given the multi-threaded readers, now we have as
>>>>>>>>>>
>>>>>>>>>> open
>>>>>>>>>>
>>>>>>>>>> questions:
>>>>>>>>>>
>>>>>>>>>> 1) How do we let the checkpoints pass through our
>>>>>>>>>>
>>>>>>>>>> multi-threaded
>>>>>>>>>>
>>>>>>>>>> reader
>>>>>>>>>>
>>>>>>>>>> operator?
>>>>>>>>>>
>>>>>>>>>> 2) Do we have separate reader and source operators or
>>>>>>>>>>
>>>>>>>>>> not? In
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> strategy
>>>>>>>>>>
>>>>>>>>>> that has a separate source, the source operator has a
>>>>>>>>>>
>>>>>>>>>> parallelism of
>>>>>>>>>>
>>>>>>>>>> 1
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>> is responsible for split recovery only.
>>>>>>>>>>
>>>>>>>>>> For the first one, given also the constraints
>>>>>>>>>>
>>>>>>>>>> (blocking,
>>>>>>>>>>
>>>>>>>>>> finite
>>>>>>>>>>
>>>>>>>>>> queues,
>>>>>>>>>>
>>>>>>>>>> etc), I do not have an answer yet.
>>>>>>>>>>
>>>>>>>>>> For the 2nd, I think that we should go with separate
>>>>>>>>>>
>>>>>>>>>> operators
>>>>>>>>>>
>>>>>>>>>> for
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> source and the readers, for the following reasons:
>>>>>>>>>>
>>>>>>>>>> 1) This is more aligned with a potential future
>>>>>>>>>>
>>>>>>>>>> improvement
>>>>>>>>>>
>>>>>>>>>> where the
>>>>>>>>>>
>>>>>>>>>> split
>>>>>>>>>>
>>>>>>>>>> discovery becomes a responsibility of the JobManager
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>> readers are
>>>>>>>>>>
>>>>>>>>>> pooling more work from the JM.
>>>>>>>>>>
>>>>>>>>>> 2) The source is going to be the "single point of
>>>>>>>>>>
>>>>>>>>>> truth".
>>>>>>>>>>
>>>>>>>>>> It
>>>>>>>>>>
>>>>>>>>>> will
>>>>>>>>>>
>>>>>>>>>> know
>>>>>>>>>>
>>>>>>>>>> what
>>>>>>>>>>
>>>>>>>>>> has been processed and what not. If the source and the
>>>>>>>>>>
>>>>>>>>>> readers
>>>>>>>>>>
>>>>>>>>>> are a
>>>>>>>>>>
>>>>>>>>>> single
>>>>>>>>>>
>>>>>>>>>> operator with parallelism > 1, or in general, if the
>>>>>>>>>>
>>>>>>>>>> split
>>>>>>>>>>
>>>>>>>>>> discovery
>>>>>>>>>>
>>>>>>>>>> is
>>>>>>>>>>
>>>>>>>>>> done by each task individually, then:
>>>>>>>>>>    i) we have to have a deterministic scheme for each
>>>>>>>>>>
>>>>>>>>>> reader to
>>>>>>>>>>
>>>>>>>>>> assign
>>>>>>>>>>
>>>>>>>>>> splits to itself (e.g. mod subtaskId). This is not
>>>>>>>>>>
>>>>>>>>>> necessarily
>>>>>>>>>>
>>>>>>>>>> trivial
>>>>>>>>>>
>>>>>>>>>> for
>>>>>>>>>>
>>>>>>>>>> all sources.
>>>>>>>>>>    ii) each reader would have to keep a copy of all its
>>>>>>>>>>
>>>>>>>>>> processed
>>>>>>>>>>
>>>>>>>>>> slpits
>>>>>>>>>>
>>>>>>>>>>    iii) the state has to be a union state with a
>>>>>>>>>>
>>>>>>>>>> non-trivial
>>>>>>>>>>
>>>>>>>>>> merging
>>>>>>>>>>
>>>>>>>>>> logic
>>>>>>>>>>
>>>>>>>>>> in order to support rescaling.
>>>>>>>>>>
>>>>>>>>>> Two additional points that you raised above:
>>>>>>>>>>
>>>>>>>>>> i) The point that you raised that we need to keep all
>>>>>>>>>>
>>>>>>>>>> splits
>>>>>>>>>>
>>>>>>>>>> (processed
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>> not-processed) I think is a bit of a strong
>>>>>>>>>>
>>>>>>>>>> requirement.
>>>>>>>>>>
>>>>>>>>>> This
>>>>>>>>>>
>>>>>>>>>> would
>>>>>>>>>>
>>>>>>>>>> imply
>>>>>>>>>>
>>>>>>>>>> that for infinite sources the state will grow
>>>>>>>>>>
>>>>>>>>>> indefinitely.
>>>>>>>>>>
>>>>>>>>>> This is
>>>>>>>>>>
>>>>>>>>>> problem
>>>>>>>>>>
>>>>>>>>>> is even more pronounced if we do not have a single
>>>>>>>>>>
>>>>>>>>>> source
>>>>>>>>>>
>>>>>>>>>> that
>>>>>>>>>>
>>>>>>>>>> assigns
>>>>>>>>>>
>>>>>>>>>> splits to readers, as each reader will have its own
>>>>>>>>>>
>>>>>>>>>> copy
>>>>>>>>>>
>>>>>>>>>> of
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> state.
>>>>>>>>>>
>>>>>>>>>> ii) it is true that for finite sources we need to
>>>>>>>>>>
>>>>>>>>>> somehow
>>>>>>>>>>
>>>>>>>>>> not
>>>>>>>>>>
>>>>>>>>>> close
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> readers when the source/split discoverer finishes. The
>>>>>>>>>> ContinuousFileReaderOperator has a work-around for
>>>>>>>>>>
>>>>>>>>>> that.
>>>>>>>>>>
>>>>>>>>>> It is
>>>>>>>>>>
>>>>>>>>>> not
>>>>>>>>>>
>>>>>>>>>> elegant,
>>>>>>>>>>
>>>>>>>>>> and checkpoints are not emitted after closing the
>>>>>>>>>>
>>>>>>>>>> source,
>>>>>>>>>>
>>>>>>>>>> but
>>>>>>>>>>
>>>>>>>>>> this, I
>>>>>>>>>>
>>>>>>>>>> believe, is a bigger problem which requires more
>>>>>>>>>>
>>>>>>>>>> changes
>>>>>>>>>>
>>>>>>>>>> than
>>>>>>>>>>
>>>>>>>>>> just
>>>>>>>>>>
>>>>>>>>>> refactoring the source interface.
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Kostas
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> Best, Jingsong Lee
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> Best, Jingsong Lee
>>>>>>
>>>>>
>>>>
>>>>
>>>
>>
> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Jark,

Please see the reply below:

Regarding to option#3, my concern is that if we don't support streaming
> mode for bounded source,
> how could we create a testing source for streaming mode? Currently, all the
> testing source for streaming
> are bounded, so that the integration test will finish finally.


An UNBOUNDED source does not mean it will never stops. It simply indicates
that the source *may* run forever, so the runtime needs to be prepared for
that, but the task may still stop at some point when it hits some
source-specific condition. So an UNBOUNDED testing source can still stop at
some point if needed.

Regarding to Source#getRecordOrder(), could we have a implicit contract
> that unbounded source should
> already read in order (i.e. reading partitions in parallel), for bounded
> source the order is not mandatory.



> This is also the behaviors of the current sources.

1) a source can't guarantee it reads in strict order, because the producer
> may produce data not in order.
> 2) *Bounded-StrictOrder* is not necessary, because batch can reorder data.


It is true that sometimes the source cannot guarantee the record order, but
sometimes it can. Right now, even for stream processing, there is no
processing order guarantee. For example, a join operator may emit a later
record which successfully found a join match earlier.
Event order is one of the most important requirements for event processing,
a clear order guarantee would be necessary. That said, I agree that right
now even if the sources provide the record order requirement, the runtime
is not able to guarantee that out of the box. So I am OK if we add the
record order to the Source later. But we should avoid misleading users to
make them think the processing order is guaranteed when using the unbounded
runtime.

Thanks,

Jiangjie (Becket) Qin

On Wed, Dec 18, 2019 at 10:29 AM Jark Wu <im...@gmail.com> wrote:

> Hi Becket,
>
> That's great we have reached a consensus on Source#getBoundedness().
>
> Regarding to option#3, my concern is that if we don't support streaming
> mode for bounded source,
> how could we create a testing source for streaming mode? Currently, all the
> testing source for streaming
> are bounded, so that the integration test will finish finally.
>
> Regarding to Source#getRecordOrder(), could we have a implicit contract
> that unbounded source should
> already read in order (i.e. reading partitions in parallel), for bounded
> source the order is not mandatory.
> This is also the behaviors of the current sources.
> 1) a source can't guarantee it reads in strict order, because the producer
> may produce data not in order.
> 2) *Bounded-StrictOrder* is not necessary, because batch can reorder data.
>
> Best,
> Jark
>
>
>
> On Tue, 17 Dec 2019 at 22:03, Becket Qin <be...@gmail.com> wrote:
>
> > Hi folks,
> >
> > Thanks for the comments. I am convinced that the Source API should not
> take
> > boundedness as a parameter after it is constructed. What Timo and Dawid
> > suggested sounds a reasonable solution to me. So the Source API would
> > become:
> >
> > Source {
> >     Boundedness getBoundedness();
> > }
> >
> > Assuming the above Source API, in addition to the two options mentioned
> in
> > earlier emails, I am thinking of another option:
> >
> > *Option 3:*
> > // MySource must be unbounded, otherwise throws exception.
> > DataStream<Type> dataStream = env.source(mySource);
> >
> > // MySource must be bounded, otherwise throws exception.
> > BoundedDataStream<Type> boundedDataStream = env.boundedSource(mySource);
> >
> > The pros of this API are:
> >    a) It fits the requirements from Table / SQL well.
> >    b) DataStream users still have type safety (option 2 only has partial
> > type safety).
> >    c) Cristal clear boundedness from the API which makes DataStream join
> /
> > connect easy to reason about.
> > The caveats I see,
> >    a) It is inconsistent with Table since Table has one unified
> interface.
> >    b) No streaming mode for bounded source.
> >
> > @Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
> > <al...@ververica.com> what do you think of the approach?
> >
> >
> > Orthogonal to the above API, I am wondering whether boundedness is the
> only
> > dimension needed to describe the characteristic of the Source behavior.
> We
> > may also need to have another dimension of *record order*.
> >
> > For example, when a file source is reading from a directory with bounded
> > records, it may have two ways to read.
> > 1. Read files in parallel.
> > 2. Read files in the chronological order.
> > In both cases, the file source is a Bounded Source. However, the
> processing
> > requirement for downstream may be different. In the first case, the
> > record processing and result emitting order does not matter, e.g. word
> > count. In the second case, the records may have to be processed in the
> > order they were read, e.g. change log processing.
> >
> > If the Source only has a getBoundedness() method, the downstream
> processors
> > would not know whether the records emitted from the Source should be
> > processed in order or not. So combining the boundedness and record order,
> > we will have four scenarios:
> >
> > *Bounded-StrictOrder*:     A segment of change log.
> > *Bounded-Random*:          Batch Word Count.
> > *Unbounded-StrictOrder*: An infinite change log.
> > *Unbounded-Random*:     Streaming Word Count.
> >
> > Option 2 mentioned in the previous email was kind of trying to handle the
> > Bounded-StrictOrder case by creating a DataStream from a bounded source,
> > which actually does not work.
> > It looks that we do not have strict order support in some operators at
> this
> > point, e.g. join. But we may still want to add the semantic to the Source
> > first so later on we don't need to change all the source implementations,
> > especially given that many of them will be implemented by 3rd party.
> >
> > Given that, we need another dimension of *Record Order* in the Source.
> More
> > specifically, the API would become:
> >
> > Source {
> >     Boundedness getBoundedness();
> >     RecordOrder getRecordOrder();
> > }
> >
> > public enum RecordOrder {
> >     /** The record in the DataStream must be processed in its strict
> order
> > for correctness. */
> >     STRICT,
> >     /** The record in the DataStream can be processed in arbitrary order.
> > */
> >     RANDOM;
> > }
> >
> > Any thoughts?
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <tw...@apache.org> wrote:
> >
> > > Hi Becket,
> > >
> > > I completely agree with Dawid's suggestion. The information about the
> > > boundedness should come out of the source. Because most of the
> streaming
> > > sources can be made bounded based on some connector specific criterion.
> > > In Kafka, it would be an end offset or end timestamp but in any case
> > > having just a env.boundedSource() is not enough because parameters for
> > > making the source bounded are missing.
> > >
> > > I suggest to have a simple `isBounded(): Boolean` flag in every source
> > > that might be influenced by a connector builder as Dawid mentioned.
> > >
> > > For type safety during programming, we can still go with *Final state
> > > 1*. By having a env.source() vs env.boundedSource(). The latter would
> > > just enforce that the boolean flag is set to `true` and could make
> > > bounded operations available (if we need that actually).
> > >
> > > However, I don't think that we should start making a unified Table API
> > > ununified again. Boundedness is an optimization property. Every bounded
> > > operation can also executed in an unbounded way using
> updates/retraction
> > > or watermarks.
> > >
> > > Regards,
> > > Timo
> > >
> > >
> > > On 15.12.19 14:22, Becket Qin wrote:
> > > > Hi Dawid and Jark,
> > > >
> > > > I think the discussion ultimately boils down to the question that
> which
> > > one
> > > > of the following two final states do we want? Once we make this
> > decision,
> > > > everything else can be naturally derived.
> > > >
> > > > *Final state 1*: Separate API for bounded / unbounded DataStream &
> > Table.
> > > > That means any code users write will be valid at the point when they
> > > write
> > > > the code. This is similar to having type safety check at programming
> > > time.
> > > > For example,
> > > >
> > > > BoundedDataStream extends DataStream {
> > > > // Operations only available for bounded data.
> > > > BoundedDataStream sort(...);
> > > >
> > > > // Interaction with another BoundedStream returns a Bounded stream.
> > > > BoundedJoinedDataStream join(BoundedDataStream other)
> > > >
> > > > // Interaction with another unbounded stream returns an unbounded
> > stream.
> > > > JoinedDataStream join(DataStream other)
> > > > }
> > > >
> > > > BoundedTable extends Table {
> > > >    // Bounded only operation.
> > > > BoundedTable sort(...);
> > > >
> > > > // Interaction with another BoundedTable returns a BoundedTable.
> > > > BoundedTable join(BoundedTable other)
> > > >
> > > > // Interaction with another unbounded table returns an unbounded
> table.
> > > > Table join(Table other)
> > > > }
> > > >
> > > > *Final state 2*: One unified API for bounded / unbounded DataStream /
> > > > Table.
> > > > That unified API may throw exception at DAG compilation time if an
> > > invalid
> > > > operation is tried. This is what Table API currently follows.
> > > >
> > > > DataStream {
> > > > // Throws exception if the DataStream is unbounded.
> > > > DataStream sort();
> > > > // Get boundedness.
> > > > Boundedness getBoundedness();
> > > > }
> > > >
> > > > Table {
> > > > // Throws exception if the table has infinite rows.
> > > > Table orderBy();
> > > >
> > > > // Get boundedness.
> > > > Boundedness getBoundedness();
> > > > }
> > > >
> > > >>From what I understand, there is no consensus so far on this decision
> > > yet.
> > > > Whichever final state we choose, we need to make it consistent across
> > the
> > > > entire project. We should avoid the case that Table follows one final
> > > state
> > > > while DataStream follows another. Some arguments I am aware of from
> > both
> > > > sides so far are following:
> > > >
> > > > Arguments for final state 1:
> > > > 1a) Clean API with method safety check at programming time.
> > > > 1b) (Counter 2b) Although SQL does not have programming time error
> > > check, SQL
> > > > is not really a "programming language" per se. So SQL can be
> different
> > > from
> > > > Table and DataStream.
> > > > 1c)  Although final state 2 seems making it easier for SQL to use
> given
> > > it
> > > > is more "config based" than "parameter based", final state 1 can
> > probably
> > > > also meet what SQL wants by wrapping the Source in TableSource /
> > > > TableSourceFactory API if needed.
> > > >
> > > > Arguments for final state 2:
> > > > 2a) The Source API itself seems already sort of following the unified
> > API
> > > > pattern.
> > > > 2b) There is no "programming time" method error check in SQL case, so
> > we
> > > > cannot really achieve final state 1 across the board.
> > > > 2c) It is an easier path given our current status, i.e. Table is
> > already
> > > > following final state 2.
> > > > 2d) Users can always explicitly check the boundedness if they want
> to.
> > > >
> > > > As I mentioned earlier, my initial thought was also to have a
> > > > "configuration based" Source rather than a "parameter based" Source.
> So
> > > it
> > > > is completely possible that I missed some important consideration or
> > > design
> > > > principles that we want to enforce for the project. It would be good
> > > > if @Stephan
> > > > Ewen <st...@ververica.com> and @Aljoscha Krettek <
> > > aljoscha@ververica.com> can
> > > > also provide more thoughts on this.
> > > >
> > > >
> > > > Re: Jingsong
> > > >
> > > > As you said, there are some batched system source, like parquet/orc
> > > source.
> > > >> Could we have the batch emit interface to improve performance? The
> > > queue of
> > > >> per record may cause performance degradation.
> > > >
> > > >
> > > > The current interface does not necessarily cause performance problem
> > in a
> > > > multi-threading case. In fact, the base implementation allows
> > > SplitReaders
> > > > to add a batch <E> of records<T> to the records queue<E>, so each
> > element
> > > > in the records queue would be a batch <E>. In this case, when the
> main
> > > > thread polls records, it will take a batch <E> of records <T> from
> the
> > > > shared records queue and process the records <T> in a batch manner.
> > > >
> > > > Thanks,
> > > >
> > > > Jiangjie (Becket) Qin
> > > >
> > > > On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <ji...@gmail.com>
> > > wrote:
> > > >
> > > >> Hi Becket,
> > > >>
> > > >> I also have some performance concerns too.
> > > >>
> > > >> If I understand correctly, SourceOutput will emit data per record
> into
> > > the
> > > >> queue? I'm worried about the multithreading performance of this
> queue.
> > > >>
> > > >>> One example is some batched messaging systems which only have an
> > offset
> > > >> for the entire batch instead of individual messages in the batch.
> > > >>
> > > >> As you said, there are some batched system source, like parquet/orc
> > > source.
> > > >> Could we have the batch emit interface to improve performance? The
> > > queue of
> > > >> per record may cause performance degradation.
> > > >>
> > > >> Best,
> > > >> Jingsong Lee
> > > >>
> > > >> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com> wrote:
> > > >>
> > > >>> Hi Becket,
> > > >>>
> > > >>> I think Dawid explained things clearly and makes a lot of sense.
> > > >>> I'm also in favor of #2, because #1 doesn't work for our future
> > unified
> > > >>> envrionment.
> > > >>>
> > > >>> You can see the vision in this documentation [1]. In the future, we
> > > would
> > > >>> like to
> > > >>> drop the global streaming/batch mode in SQL (i.e.
> > > >>> EnvironmentSettings#inStreamingMode/inBatchMode).
> > > >>> A source is bounded or unbounded once defined, so queries can be
> > > inferred
> > > >>> from source to run
> > > >>> in streaming or batch or hybrid mode. However, in #1, we will lose
> > this
> > > >>> ability because the framework
> > > >>> doesn't know whether the source is bounded or unbounded.
> > > >>>
> > > >>> Best,
> > > >>> Jark
> > > >>>
> > > >>>
> > > >>> [1]:
> > > >>>
> > > >>>
> > > >>
> > >
> >
> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
> > > >>>
> > > >>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <pi...@ververica.com>
> > > >> wrote:
> > > >>>
> > > >>>> Hi,
> > > >>>>
> > > >>>> Regarding the:
> > > >>>>
> > > >>>> Collection<E> getNextRecords()
> > > >>>>
> > > >>>> I’m pretty sure such design would unfortunately impact the
> > performance
> > > >>>> (accessing and potentially creating the collection on the hot
> path).
> > > >>>>
> > > >>>> Also the
> > > >>>>
> > > >>>> InputStatus emitNext(DataOutput<T> output) throws Exception;
> > > >>>> or
> > > >>>> Status pollNext(SourceOutput<T> sourceOutput) throws Exception;
> > > >>>>
> > > >>>> Gives us some opportunities in the future, to allow Source hot
> > looping
> > > >>>> inside, until it receives some signal “please exit because of some
> > > >>> reasons”
> > > >>>> (output collector could return such hint upon collecting the
> > result).
> > > >> But
> > > >>>> that’s another topic outside of this FLIP’s scope.
> > > >>>>
> > > >>>> Piotrek
> > > >>>>
> > > >>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <tr...@apache.org>
> > > >> wrote:
> > > >>>>>
> > > >>>>> Hi Becket,
> > > >>>>>
> > > >>>>> quick clarification from my side because I think you
> misunderstood
> > my
> > > >>>>> question. I did not suggest to let the SourceReader return only a
> > > >>> single
> > > >>>>> record at a time when calling getNextRecords. As the return type
> > > >>>> indicates,
> > > >>>>> the method can return an arbitrary number of records.
> > > >>>>>
> > > >>>>> Cheers,
> > > >>>>> Till
> > > >>>>>
> > > >>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> > > >>>> dwysakowicz@apache.org <ma...@apache.org>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>>> Hi Becket,
> > > >>>>>>
> > > >>>>>> Issue #1 - Design of Source interface
> > > >>>>>>
> > > >>>>>> I mentioned the lack of a method like
> > > >>>> Source#createEnumerator(Boundedness
> > > >>>>>> boundedness, SplitEnumeratorContext context), because without
> the
> > > >>>> current
> > > >>>>>> proposal is not complete/does not work.
> > > >>>>>>
> > > >>>>>> If we say that boundedness is an intrinsic property of a source
> > imo
> > > >> we
> > > >>>>>> don't need the Source#createEnumerator(Boundedness boundedness,
> > > >>>>>> SplitEnumeratorContext context) method.
> > > >>>>>>
> > > >>>>>> Assuming a source from my previous example:
> > > >>>>>>
> > > >>>>>> Source source = KafkaSource.builder()
> > > >>>>>>   ...
> > > >>>>>>   .untilTimestamp(...)
> > > >>>>>>   .build()
> > > >>>>>>
> > > >>>>>> Would the enumerator differ if created like
> > > >>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
> > > >>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating myself,
> but
> > > >>> this
> > > >>>> is
> > > >>>>>> the part that my opinion differ the most from the current
> > proposal.
> > > >> I
> > > >>>>>> really think it should always be the source that tells if it is
> > > >>> bounded
> > > >>>> or
> > > >>>>>> not. In the current proposal methods
> continousSource/boundedSource
> > > >>>> somewhat
> > > >>>>>> reconfigure the source, which I think is misleading.
> > > >>>>>>
> > > >>>>>> I think a call like:
> > > >>>>>>
> > > >>>>>> Source source = KafkaSource.builder()
> > > >>>>>>   ...
> > > >>>>>>   .readContinously() / readUntilLatestOffset() /
> > readUntilTimestamp
> > > /
> > > >>>> readUntilOffsets / ...
> > > >>>>>>   .build()
> > > >>>>>>
> > > >>>>>> is way cleaner (and expressive) than
> > > >>>>>>
> > > >>>>>> Source source = KafkaSource.builder()
> > > >>>>>>   ...
> > > >>>>>>   .build()
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> env.continousSource(source) // which actually underneath would
> > call
> > > >>>> createEnumerator(CONTINUOUS, ctx) which would be equivalent to
> > > >>>> source.readContinously().createEnumerator(ctx)
> > > >>>>>> // or
> > > >>>>>> env.boundedSource(source) // which actually underneath would
> call
> > > >>>> createEnumerator(BOUNDED, ctx) which would be equivalent to
> > > >>>> source.readUntilLatestOffset().createEnumerator(ctx)
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Sorry for the comparison, but to me it seems there is too much
> > magic
> > > >>>>>> happening underneath those two calls.
> > > >>>>>>
> > > >>>>>> I really believe the Source interface should have getBoundedness
> > > >>> method
> > > >>>>>> instead of (supportBoundedness) + createEnumerator(Boundedness,
> > ...)
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Issue #2 - Design of
> > > >>>>>> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> > > >>>>>>
> > > >>>>>> As you might have guessed I am slightly in favor of option #2
> > > >>> modified.
> > > >>>>>> Yes I am aware every step of the dag would have to be able to
> say
> > if
> > > >>> it
> > > >>>> is
> > > >>>>>> bounded or not. I have a feeling it would be easier to express
> > cross
> > > >>>>>> bounded/unbounded operations, but I must admit I have not
> thought
> > it
> > > >>>>>> through thoroughly, In the spirit of batch is just a special
> case
> > of
> > > >>>>>> streaming I thought BoundedStream would extend from DataStream.
> > > >>> Correct
> > > >>>> me
> > > >>>>>> if I am wrong. In such a setup the cross bounded/unbounded
> > operation
> > > >>>> could
> > > >>>>>> be expressed quite easily I think:
> > > >>>>>>
> > > >>>>>> DataStream {
> > > >>>>>>   DataStream join(DataStream, ...); // we could not really tell
> if
> > > >> the
> > > >>>> result is bounded or not, but because bounded stream is a special
> > case
> > > >> of
> > > >>>> unbounded the API object is correct, irrespective if the left or
> > right
> > > >>> side
> > > >>>> of the join is bounded
> > > >>>>>> }
> > > >>>>>>
> > > >>>>>> BoundedStream extends DataStream {
> > > >>>>>>   BoundedStream join(BoundedStream, ...); // only if both sides
> > are
> > > >>>> bounded the result can be bounded as well. However we do have
> access
> > > to
> > > >>> the
> > > >>>> DataStream#join here, so you can still join with a DataStream
> > > >>>>>> }
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On the other hand I also see benefits of two completely
> disjointed
> > > >>> APIs,
> > > >>>>>> as we could prohibit some streaming calls in the bounded API. I
> > > >> can't
> > > >>>> think
> > > >>>>>> of any unbounded operators that could not be implemented for
> > bounded
> > > >>>> stream.
> > > >>>>>>
> > > >>>>>> Besides I think we both agree we don't like the method:
> > > >>>>>>
> > > >>>>>> DataStream boundedStream(Source)
> > > >>>>>>
> > > >>>>>> suggested in the current state of the FLIP. Do we ? :)
> > > >>>>>>
> > > >>>>>> Best,
> > > >>>>>>
> > > >>>>>> Dawid
> > > >>>>>>
> > > >>>>>> On 10/12/2019 18:57, Becket Qin wrote:
> > > >>>>>>
> > > >>>>>> Hi folks,
> > > >>>>>>
> > > >>>>>> Thanks for the discussion, great feedback. Also thanks Dawid for
> > the
> > > >>>>>> explanation, it is much clearer now.
> > > >>>>>>
> > > >>>>>> One thing that is indeed missing from the FLIP is how the
> > > >> boundedness
> > > >>> is
> > > >>>>>> passed to the Source implementation. So the API should be
> > > >>>>>> Source#createEnumerator(Boundedness boundedness,
> > > >>> SplitEnumeratorContext
> > > >>>>>> context)
> > > >>>>>> And we can probably remove the
> > Source#supportBoundedness(Boundedness
> > > >>>>>> boundedness) method.
> > > >>>>>>
> > > >>>>>> Assuming we have that, we are essentially choosing from one of
> the
> > > >>>>>> following two options:
> > > >>>>>>
> > > >>>>>> Option 1:
> > > >>>>>> // The source is continuous source, and only unbounded
> operations
> > > >> can
> > > >>> be
> > > >>>>>> performed.
> > > >>>>>> DataStream<Type> datastream = env.continuousSource(someSource);
> > > >>>>>>
> > > >>>>>> // The source is bounded source, both bounded and unbounded
> > > >> operations
> > > >>>> can
> > > >>>>>> be performed.
> > > >>>>>> BoundedDataStream<Type> boundedDataStream =
> > > >>>> env.boundedSource(someSource);
> > > >>>>>>
> > > >>>>>>   - Pros:
> > > >>>>>>        a) explicit boundary between bounded / unbounded streams,
> > it
> > > >> is
> > > >>>>>> quite simple and clear to the users.
> > > >>>>>>   - Cons:
> > > >>>>>>        a) For applications that do not involve bounded
> operations,
> > > >> they
> > > >>>>>> still have to call different API to distinguish bounded /
> > unbounded
> > > >>>> streams.
> > > >>>>>>        b) No support for bounded stream to run in a streaming
> > > runtime
> > > >>>>>> setting, i.e. scheduling and operators behaviors.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Option 2:
> > > >>>>>> // The source is either bounded or unbounded, but only unbounded
> > > >>>> operations
> > > >>>>>> could be performed on the returned DataStream.
> > > >>>>>> DataStream<Type> dataStream = env.source(someSource);
> > > >>>>>>
> > > >>>>>> // The source must be a bounded source, otherwise exception is
> > > >> thrown.
> > > >>>>>> BoundedDataStream<Type> boundedDataStream =
> > > >>>>>> env.boundedSource(boundedSource);
> > > >>>>>>
> > > >>>>>> The pros and cons are exactly the opposite of option 1.
> > > >>>>>>   - Pros:
> > > >>>>>>        a) For applications that do not involve bounded
> operations,
> > > >> they
> > > >>>>>> still have to call different API to distinguish bounded /
> > unbounded
> > > >>>> streams.
> > > >>>>>>        b) Support for bounded stream to run in a streaming
> runtime
> > > >>>> setting,
> > > >>>>>> i.e. scheduling and operators behaviors.
> > > >>>>>>   - Cons:
> > > >>>>>>        a) Bounded / unbounded streams are kind of mixed, i.e.
> > given
> > > a
> > > >>>>>> DataStream, it is not clear whether it is bounded or not, unless
> > you
> > > >>>> have
> > > >>>>>> the access to its source.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> If we only think from the Source API perspective, option 2
> seems a
> > > >>>> better
> > > >>>>>> choice because functionality wise it is a superset of option 1,
> at
> > > >> the
> > > >>>> cost
> > > >>>>>> of some seemingly acceptable ambiguity in the DataStream API.
> > > >>>>>> But if we look at the DataStream API as a whole, option 1 seems
> a
> > > >>>> clearer
> > > >>>>>> choice. For example, some times a library may have to know
> > whether a
> > > >>>>>> certain task will finish or not. And it would be difficult to
> tell
> > > >> if
> > > >>>> the
> > > >>>>>> input is a DataStream, unless additional information is provided
> > all
> > > >>> the
> > > >>>>>> way from the Source. One possible solution is to have a
> *modified
> > > >>>> option 2*
> > > >>>>>> which adds a method to the DataStream API to indicate
> boundedness,
> > > >>> such
> > > >>>> as
> > > >>>>>> getBoundedness(). It would solve the problem with a potential
> > > >>> confusion
> > > >>>> of
> > > >>>>>> what is difference between a DataStream with
> getBoundedness()=true
> > > >>> and a
> > > >>>>>> BoundedDataStream. But that seems not super difficult to
> explain.
> > > >>>>>>
> > > >>>>>> So from API's perspective, I don't have a strong opinion between
> > > >>>> *option 1*
> > > >>>>>> and *modified option 2. *I like the cleanness of option 1, but
> > > >>> modified
> > > >>>>>> option 2 would be more attractive if we have concrete use case
> for
> > > >> the
> > > >>>>>> "Bounded stream with unbounded streaming runtime settings".
> > > >>>>>>
> > > >>>>>> Re: Till
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Maybe this has already been asked before but I was wondering why
> > the
> > > >>>>>> SourceReader interface has the method pollNext which hands the
> > > >>>>>> responsibility of outputting elements to the SourceReader
> > > >>>> implementation?
> > > >>>>>> Has this been done for backwards compatibility reasons with the
> > old
> > > >>>> source
> > > >>>>>> interface? If not, then one could define a Collection<E>
> > > >>>> getNextRecords()
> > > >>>>>> method which returns the currently retrieved records and then
> the
> > > >>> caller
> > > >>>>>> emits them outside of the SourceReader. That way the interface
> > would
> > > >>> not
> > > >>>>>> allow to implement an outputting loop where we never hand back
> > > >> control
> > > >>>> to
> > > >>>>>> the caller. At the moment, this contract can be easily broken
> and
> > is
> > > >>>> only
> > > >>>>>> mentioned loosely in the JavaDocs.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> The primary reason we handover the SourceOutput to the
> > SourceReader
> > > >> is
> > > >>>>>> because sometimes it is difficult for a SourceReader to emit one
> > > >>> record
> > > >>>> at
> > > >>>>>> a time. One example is some batched messaging systems which only
> > > >> have
> > > >>> an
> > > >>>>>> offset for the entire batch instead of individual messages in
> the
> > > >>>> batch. In
> > > >>>>>> that case, returning one record at a time would leave the
> > > >> SourceReader
> > > >>>> in
> > > >>>>>> an uncheckpointable state because they can only checkpoint at
> the
> > > >>> batch
> > > >>>>>> boundaries.
> > > >>>>>>
> > > >>>>>> Thanks,
> > > >>>>>>
> > > >>>>>> Jiangjie (Becket) Qin
> > > >>>>>>
> > > >>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <
> > trohrmann@apache.org
> > > >>>> <ma...@apache.org>> <trohrmann@apache.org <mailto:
> > > >>>> trohrmann@apache.org>> wrote:
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Hi everyone,
> > > >>>>>>
> > > >>>>>> thanks for drafting this FLIP. It reads very well.
> > > >>>>>>
> > > >>>>>> Concerning Dawid's proposal, I tend to agree. The boundedness
> > could
> > > >>> come
> > > >>>>>> from the source and tell the system how to treat the operator
> > > >>>> (scheduling
> > > >>>>>> wise). From a user's perspective it should be fine to get back a
> > > >>>> DataStream
> > > >>>>>> when calling env.source(boundedSource) if he does not need
> special
> > > >>>>>> operations defined on a BoundedDataStream. If he needs this,
> then
> > > >> one
> > > >>>> could
> > > >>>>>> use the method BoundedDataStream
> env.boundedSource(boundedSource).
> > > >>>>>>
> > > >>>>>> If possible, we could enforce the proper usage of
> > > >> env.boundedSource()
> > > >>> by
> > > >>>>>> introducing a BoundedSource type so that one cannot pass an
> > > >>>>>> unbounded source to it. That way users would not be able to
> shoot
> > > >>>>>> themselves in the foot.
> > > >>>>>>
> > > >>>>>> Maybe this has already been asked before but I was wondering why
> > the
> > > >>>>>> SourceReader interface has the method pollNext which hands the
> > > >>>>>> responsibility of outputting elements to the SourceReader
> > > >>>> implementation?
> > > >>>>>> Has this been done for backwards compatibility reasons with the
> > old
> > > >>>> source
> > > >>>>>> interface? If not, then one could define a Collection<E>
> > > >>>> getNextRecords()
> > > >>>>>> method which returns the currently retrieved records and then
> the
> > > >>> caller
> > > >>>>>> emits them outside of the SourceReader. That way the interface
> > would
> > > >>> not
> > > >>>>>> allow to implement an outputting loop where we never hand back
> > > >> control
> > > >>>> to
> > > >>>>>> the caller. At the moment, this contract can be easily broken
> and
> > is
> > > >>>> only
> > > >>>>>> mentioned loosely in the JavaDocs.
> > > >>>>>>
> > > >>>>>> Cheers,
> > > >>>>>> Till
> > > >>>>>>
> > > >>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <
> > jingsonglee0@gmail.com
> > > >>>> <ma...@gmail.com>> <jingsonglee0@gmail.com <mailto:
> > > >>>> jingsonglee0@gmail.com>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Hi all,
> > > >>>>>>
> > > >>>>>> I think current design is good.
> > > >>>>>>
> > > >>>>>> My understanding is:
> > > >>>>>>
> > > >>>>>> For execution mode: bounded mode and continuous mode, It's
> totally
> > > >>>>>> different. I don't think we have the ability to integrate the
> two
> > > >>> models
> > > >>>>>>
> > > >>>>>> at
> > > >>>>>>
> > > >>>>>> present. It's about scheduling, memory, algorithms, States, etc.
> > we
> > > >>>>>> shouldn't confuse them.
> > > >>>>>>
> > > >>>>>> For source capabilities: only bounded, only continuous, both
> > bounded
> > > >>> and
> > > >>>>>> continuous.
> > > >>>>>> I think Kafka is a source that can be ran both bounded
> > > >>>>>> and continuous execution mode.
> > > >>>>>> And Kafka with end offset should be ran both bounded
> > > >>>>>> and continuous execution mode.  Using apache Beam with Flink
> > > >> runner, I
> > > >>>>>>
> > > >>>>>> used
> > > >>>>>>
> > > >>>>>> to run a "bounded" Kafka in streaming mode. For our previous
> > > >>> DataStream,
> > > >>>>>>
> > > >>>>>> it
> > > >>>>>>
> > > >>>>>> is not necessarily required that the source cannot be bounded.
> > > >>>>>>
> > > >>>>>> So it is my thought for Dawid's question:
> > > >>>>>> 1.pass a bounded source to continuousSource() +1
> > > >>>>>> 2.pass a continuous source to boundedSource() -1, should throw
> > > >>>> exception.
> > > >>>>>>
> > > >>>>>> In StreamExecutionEnvironment, continuousSource and
> boundedSource
> > > >>> define
> > > >>>>>> the execution mode. It defines a clear boundary of execution
> mode.
> > > >>>>>>
> > > >>>>>> Best,
> > > >>>>>> Jingsong Lee
> > > >>>>>>
> > > >>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com
> > <mailto:
> > > >>>> imjark@gmail.com>> <imjark@gmail.com <ma...@gmail.com>>
> > > wrote:
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> I agree with Dawid's point that the boundedness information
> should
> > > >>> come
> > > >>>>>> from the source itself (e.g. the end timestamp), not through
> > > >>>>>> env.boundedSouce()/continuousSource().
> > > >>>>>> I think if we want to support something like `env.source()` that
> > > >>> derive
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> execution mode from source, `supportsBoundedness(Boundedness)`
> > > >>>>>> method is not enough, because we don't know whether it is
> bounded
> > or
> > > >>>>>>
> > > >>>>>> not.
> > > >>>>>>
> > > >>>>>> Best,
> > > >>>>>> Jark
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
> > > >> dwysakowicz@apache.org
> > > >>>> <ma...@apache.org>> <dwysakowicz@apache.org <mailto:
> > > >>>> dwysakowicz@apache.org>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> One more thing. In the current proposal, with the
> > > >>>>>> supportsBoundedness(Boundedness) method and the boundedness
> coming
> > > >>>>>>
> > > >>>>>> from
> > > >>>>>>
> > > >>>>>> either continuousSource or boundedSource I could not find how
> this
> > > >>>>>> information is fed back to the SplitEnumerator.
> > > >>>>>>
> > > >>>>>> Best,
> > > >>>>>>
> > > >>>>>> Dawid
> > > >>>>>>
> > > >>>>>> On 09/12/2019 13:52, Becket Qin wrote:
> > > >>>>>>
> > > >>>>>> Hi Dawid,
> > > >>>>>>
> > > >>>>>> Thanks for the comments. This actually brings another relevant
> > > >>>>>>
> > > >>>>>> question
> > > >>>>>>
> > > >>>>>> about what does a "bounded source" imply. I actually had the
> same
> > > >>>>>> impression when I look at the Source API. Here is what I
> > understand
> > > >>>>>>
> > > >>>>>> after
> > > >>>>>>
> > > >>>>>> some discussion with Stephan. The bounded source has the
> following
> > > >>>>>>
> > > >>>>>> impacts.
> > > >>>>>>
> > > >>>>>> 1. API validity.
> > > >>>>>> - A bounded source generates a bounded stream so some operations
> > > >>>>>>
> > > >>>>>> that
> > > >>>>>>
> > > >>>>>> only
> > > >>>>>>
> > > >>>>>> works for bounded records would be performed, e.g. sort.
> > > >>>>>> - To expose these bounded stream only APIs, there are two
> options:
> > > >>>>>>      a. Add them to the DataStream API and throw exception if a
> > > >>>>>>
> > > >>>>>> method
> > > >>>>>>
> > > >>>>>> is
> > > >>>>>>
> > > >>>>>> called on an unbounded stream.
> > > >>>>>>      b. Create a BoundedDataStream class which is returned from
> > > >>>>>> env.boundedSource(), while DataStream is returned from
> > > >>>>>>
> > > >>>>>> env.continousSource().
> > > >>>>>>
> > > >>>>>> Note that this cannot be done by having single
> > > >>>>>>
> > > >>>>>> env.source(theSource)
> > > >>>>>>
> > > >>>>>> even
> > > >>>>>>
> > > >>>>>> the Source has a getBoundedness() method.
> > > >>>>>>
> > > >>>>>> 2. Scheduling
> > > >>>>>> - A bounded source could be computed stage by stage without
> > > >>>>>>
> > > >>>>>> bringing
> > > >>>>>>
> > > >>>>>> up
> > > >>>>>>
> > > >>>>>> all
> > > >>>>>>
> > > >>>>>> the tasks at the same time.
> > > >>>>>>
> > > >>>>>> 3. Operator behaviors
> > > >>>>>> - A bounded source indicates the records are finite so some
> > > >>>>>>
> > > >>>>>> operators
> > > >>>>>>
> > > >>>>>> can
> > > >>>>>>
> > > >>>>>> wait until it receives all the records before it starts the
> > > >>>>>>
> > > >>>>>> processing.
> > > >>>>>>
> > > >>>>>> In the above impact, only 1 is relevant to the API design. And
> the
> > > >>>>>>
> > > >>>>>> current
> > > >>>>>>
> > > >>>>>> proposal in FLIP-27 is following 1.b.
> > > >>>>>>
> > > >>>>>> // boundedness depends of source property, imo this should
> always
> > > >>>>>>
> > > >>>>>> be
> > > >>>>>>
> > > >>>>>> preferred
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> DataStream<MyType> stream = env.source(theSource);
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> In your proposal, does DataStream have bounded stream only
> > methods?
> > > >>>>>>
> > > >>>>>> It
> > > >>>>>>
> > > >>>>>> looks it should have, otherwise passing a bounded Source to
> > > >>>>>>
> > > >>>>>> env.source()
> > > >>>>>>
> > > >>>>>> would be confusing. In that case, we will essentially do 1.a if
> an
> > > >>>>>> unbounded Source is created from env.source(unboundedSource).
> > > >>>>>>
> > > >>>>>> If we have the methods only supported for bounded streams in
> > > >>>>>>
> > > >>>>>> DataStream,
> > > >>>>>>
> > > >>>>>> it
> > > >>>>>>
> > > >>>>>> seems a little weird to have a separate BoundedDataStream
> > > >>>>>>
> > > >>>>>> interface.
> > > >>>>>>
> > > >>>>>> Am I understand it correctly?
> > > >>>>>>
> > > >>>>>> Thanks,
> > > >>>>>>
> > > >>>>>> Jiangjie (Becket) Qin
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> > > >>>>>>
> > > >>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> > > >>>>>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Hi all,
> > > >>>>>>
> > > >>>>>> Really well written proposal and very important one. I must
> admit
> > > >>>>>>
> > > >>>>>> I
> > > >>>>>>
> > > >>>>>> have
> > > >>>>>>
> > > >>>>>> not understood all the intricacies of it yet.
> > > >>>>>>
> > > >>>>>> One question I have though is about where does the information
> > > >>>>>>
> > > >>>>>> about
> > > >>>>>>
> > > >>>>>> boundedness come from. I think in most cases it is a property of
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> source. As you described it might be e.g. end offset, a flag
> > > >>>>>>
> > > >>>>>> should
> > > >>>>>>
> > > >>>>>> it
> > > >>>>>>
> > > >>>>>> monitor new splits etc. I think it would be a really nice use
> case
> > > >>>>>>
> > > >>>>>> to
> > > >>>>>>
> > > >>>>>> be
> > > >>>>>>
> > > >>>>>> able to say:
> > > >>>>>>
> > > >>>>>> new KafkaSource().readUntil(long timestamp),
> > > >>>>>>
> > > >>>>>> which could work as an "end offset". Moreover I think all
> Bounded
> > > >>>>>>
> > > >>>>>> sources
> > > >>>>>>
> > > >>>>>> support continuous mode, but no intrinsically continuous source
> > > >>>>>>
> > > >>>>>> support
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> Bounded mode. If I understood the proposal correctly it suggest
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> boundedness sort of "comes" from the outside of the source, from
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> invokation of either boundedStream or continousSource.
> > > >>>>>>
> > > >>>>>> I am wondering if it would make sense to actually change the
> > > >>>>>>
> > > >>>>>> method
> > > >>>>>>
> > > >>>>>> boolean Source#supportsBoundedness(Boundedness)
> > > >>>>>>
> > > >>>>>> to
> > > >>>>>>
> > > >>>>>> Boundedness Source#getBoundedness().
> > > >>>>>>
> > > >>>>>> As for the methods #boundedSource, #continousSource, assuming
> the
> > > >>>>>> boundedness is property of the source they do not affect how the
> > > >>>>>>
> > > >>>>>> enumerator
> > > >>>>>>
> > > >>>>>> works, but mostly how the dag is scheduled, right? I am not
> > > >>>>>>
> > > >>>>>> against
> > > >>>>>>
> > > >>>>>> those
> > > >>>>>>
> > > >>>>>> methods, but I think it is a very specific use case to actually
> > > >>>>>>
> > > >>>>>> override
> > > >>>>>>
> > > >>>>>> the property of the source. In general I would expect users to
> > > >>>>>>
> > > >>>>>> only
> > > >>>>>>
> > > >>>>>> call
> > > >>>>>>
> > > >>>>>> env.source(theSource), where the source tells if it is bounded
> or
> > > >>>>>>
> > > >>>>>> not. I
> > > >>>>>>
> > > >>>>>> would suggest considering following set of methods:
> > > >>>>>>
> > > >>>>>> // boundedness depends of source property, imo this should
> always
> > > >>>>>>
> > > >>>>>> be
> > > >>>>>>
> > > >>>>>> preferred
> > > >>>>>>
> > > >>>>>> DataStream<MyType> stream = env.source(theSource);
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> // always continous execution, whether bounded or unbounded
> source
> > > >>>>>>
> > > >>>>>> DataStream<MyType> boundedStream =
> env.continousSource(theSource);
> > > >>>>>>
> > > >>>>>> // imo this would make sense if the BoundedDataStream provides
> > > >>>>>>
> > > >>>>>> additional features unavailable for continous mode
> > > >>>>>>
> > > >>>>>> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Best,
> > > >>>>>>
> > > >>>>>> Dawid
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
> > > >>>>>>
> > > >>>>>> Thanks, Becket, for updating this.
> > > >>>>>>
> > > >>>>>> I agree with moving the aspects you mentioned into separate
> FLIPs
> > > >>>>>>
> > > >>>>>> -
> > > >>>>>>
> > > >>>>>> this
> > > >>>>>>
> > > >>>>>> one way becoming unwieldy in size.
> > > >>>>>>
> > > >>>>>> +1 to the FLIP in its current state. Its a very detailed
> write-up,
> > > >>>>>>
> > > >>>>>> nicely
> > > >>>>>>
> > > >>>>>> done!
> > > >>>>>>
> > > >>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <becket.qin@gmail.com
> > > >>>> <ma...@gmail.com>> <becket.qin@gmail.com <mailto:
> > > >>>> becket.qin@gmail.com>>
> > > >>>>>>
> > > >>>>>> <
> > > >>>>>>
> > > >>>>>> becket.qin@gmail.com <ma...@gmail.com>> wrote:
> > > >>>>>>
> > > >>>>>> Hi all,
> > > >>>>>>
> > > >>>>>> Sorry for the long belated update. I have updated FLIP-27 wiki
> > > >>>>>>
> > > >>>>>> page
> > > >>>>>>
> > > >>>>>> with
> > > >>>>>>
> > > >>>>>> the latest proposals. Some noticeable changes include:
> > > >>>>>> 1. A new generic communication mechanism between SplitEnumerator
> > > >>>>>>
> > > >>>>>> and
> > > >>>>>>
> > > >>>>>> SourceReader.
> > > >>>>>> 2. Some detail API method signature changes.
> > > >>>>>>
> > > >>>>>> We left a few things out of this FLIP and will address them in
> > > >>>>>>
> > > >>>>>> separate
> > > >>>>>>
> > > >>>>>> FLIPs. Including:
> > > >>>>>> 1. Per split event time.
> > > >>>>>> 2. Event time alignment.
> > > >>>>>> 3. Fine grained failover for SplitEnumerator failure.
> > > >>>>>>
> > > >>>>>> Please let us know if you have any question.
> > > >>>>>>
> > > >>>>>> Thanks,
> > > >>>>>>
> > > >>>>>> Jiangjie (Becket) Qin
> > > >>>>>>
> > > >>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <sewen@apache.org
> > > >>> <mailto:
> > > >>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>> <
> > > >>>>>>
> > > >>>>>> sewen@apache.org <ma...@apache.org>> wrote:
> > > >>>>>>
> > > >>>>>> Hi  Łukasz!
> > > >>>>>>
> > > >>>>>> Becket and me are working hard on figuring out the last details
> > > >>>>>>
> > > >>>>>> and
> > > >>>>>>
> > > >>>>>> implementing the first PoC. We would update the FLIP hopefully
> > > >>>>>>
> > > >>>>>> next
> > > >>>>>>
> > > >>>>>> week.
> > > >>>>>>
> > > >>>>>> There is a fair chance that a first version of this will be in
> > > >>>>>>
> > > >>>>>> 1.10,
> > > >>>>>>
> > > >>>>>> but
> > > >>>>>>
> > > >>>>>> I
> > > >>>>>>
> > > >>>>>> think it will take another release to battle test it and migrate
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> connectors.
> > > >>>>>>
> > > >>>>>> Best,
> > > >>>>>> Stephan
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <
> ljd@touk.pl
> > > >>>> <ma...@touk.pl>
> > > >>>>>>
> > > >>>>>> <
> > > >>>>>>
> > > >>>>>> ljd@touk.pl <ma...@touk.pl>>
> > > >>>>>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>> Hi,
> > > >>>>>>
> > > >>>>>> This proposal looks very promising for us. Do you have any plans
> > > >>>>>>
> > > >>>>>> in
> > > >>>>>>
> > > >>>>>> which
> > > >>>>>>
> > > >>>>>> Flink release it is going to be released? We are thinking on
> > > >>>>>>
> > > >>>>>> using a
> > > >>>>>>
> > > >>>>>> Data
> > > >>>>>>
> > > >>>>>> Set API for our future use cases but on the other hand Data Set
> > > >>>>>>
> > > >>>>>> API
> > > >>>>>>
> > > >>>>>> is
> > > >>>>>>
> > > >>>>>> going to be deprecated so using proposed bounded data streams
> > > >>>>>>
> > > >>>>>> solution
> > > >>>>>>
> > > >>>>>> could be more viable in the long term.
> > > >>>>>>
> > > >>>>>> Thanks,
> > > >>>>>> Łukasz
> > > >>>>>>
> > > >>>>>> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com
> > > >> <mailto:
> > > >>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
> > > >>>> thomas.weise@gmail.com>> <
> > > >>>>>>
> > > >>>>>> thomas.weise@gmail.com <ma...@gmail.com>> wrote:
> > > >>>>>>
> > > >>>>>> Thanks for putting together this proposal!
> > > >>>>>>
> > > >>>>>> I see that the "Per Split Event Time" and "Event Time Alignment"
> > > >>>>>>
> > > >>>>>> sections
> > > >>>>>>
> > > >>>>>> are still TBD.
> > > >>>>>>
> > > >>>>>> It would probably be good to flesh those out a bit before
> > > >>>>>>
> > > >>>>>> proceeding
> > > >>>>>>
> > > >>>>>> too
> > > >>>>>>
> > > >>>>>> far
> > > >>>>>>
> > > >>>>>> as the event time alignment will probably influence the
> > > >>>>>>
> > > >>>>>> interaction
> > > >>>>>>
> > > >>>>>> with
> > > >>>>>>
> > > >>>>>> the split reader, specifically ReaderStatus
> > > >>>>>>
> > > >>>>>> emitNext(SourceOutput<E>
> > > >>>>>>
> > > >>>>>> output).
> > > >>>>>>
> > > >>>>>> We currently have only one implementation for event time
> alignment
> > > >>>>>>
> > > >>>>>> in
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> Kinesis consumer. The synchronization in that case takes place
> as
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> last
> > > >>>>>>
> > > >>>>>> step before records are emitted downstream (RecordEmitter). With
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> currently proposed interfaces, the equivalent can be implemented
> > > >>>>>>
> > > >>>>>> in
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> reader loop, although note that in the Kinesis consumer the per
> > > >>>>>>
> > > >>>>>> shard
> > > >>>>>>
> > > >>>>>> threads push records.
> > > >>>>>>
> > > >>>>>> Synchronization has not been implemented for the Kafka consumer
> > > >>>>>>
> > > >>>>>> yet.
> > > >>>>>>
> > > >>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
> > > >>>> https://issues.apache.org/jira/browse/FLINK-12675>
> > > >>>>>>
> > > >>>>>> When I looked at it, I realized that the implementation will
> look
> > > >>>>>>
> > > >>>>>> quite
> > > >>>>>>
> > > >>>>>> different
> > > >>>>>> from Kinesis because it needs to take place in the pull part,
> > > >>>>>>
> > > >>>>>> where
> > > >>>>>>
> > > >>>>>> records
> > > >>>>>>
> > > >>>>>> are taken from the Kafka client. Due to the multiplexing it
> cannot
> > > >>>>>>
> > > >>>>>> be
> > > >>>>>>
> > > >>>>>> done
> > > >>>>>>
> > > >>>>>> by blocking the split thread like it currently works for
> Kinesis.
> > > >>>>>>
> > > >>>>>> Reading
> > > >>>>>>
> > > >>>>>> from individual Kafka partitions needs to be controlled via
> > > >>>>>>
> > > >>>>>> pause/resume
> > > >>>>>>
> > > >>>>>> on the Kafka client.
> > > >>>>>>
> > > >>>>>> To take on that responsibility the split thread would need to be
> > > >>>>>>
> > > >>>>>> aware
> > > >>>>>>
> > > >>>>>> of
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>> watermarks or at least whether it should or should not continue
> to
> > > >>>>>>
> > > >>>>>> consume
> > > >>>>>>
> > > >>>>>> a given split and this may require a different SourceReader or
> > > >>>>>>
> > > >>>>>> SourceOutput
> > > >>>>>>
> > > >>>>>> interface.
> > > >>>>>>
> > > >>>>>> Thanks,
> > > >>>>>> Thomas
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com
> > > >> <mailto:
> > > >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> mmyy1110@gmail.com
> > >>
> > > >> <
> > > >>>>>>
> > > >>>>>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
> > > >>>>>>
> > > >>>>>> Hi Stephan,
> > > >>>>>>
> > > >>>>>> Thank you for feedback!
> > > >>>>>> Will take a look at your branch before public discussing.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <sewen@apache.org
> > > >>>> <ma...@apache.org>> <sewen@apache.org <mailto:
> > sewen@apache.org
> > > >>>>
> > > >>>>>>
> > > >>>>>> <
> > > >>>>>>
> > > >>>>>> sewen@apache.org <ma...@apache.org>>
> > > >>>>>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>> Hi Biao!
> > > >>>>>>
> > > >>>>>> Thanks for reviving this. I would like to join this discussion,
> > > >>>>>>
> > > >>>>>> but
> > > >>>>>>
> > > >>>>>> am
> > > >>>>>>
> > > >>>>>> quite occupied with the 1.9 release, so can we maybe pause this
> > > >>>>>>
> > > >>>>>> discussion
> > > >>>>>>
> > > >>>>>> for a week or so?
> > > >>>>>>
> > > >>>>>> In the meantime I can share some suggestion based on prior
> > > >>>>>>
> > > >>>>>> experiments:
> > > >>>>>>
> > > >>>>>> How to do watermarks / timestamp extractors in a simpler and
> more
> > > >>>>>>
> > > >>>>>> flexible
> > > >>>>>>
> > > >>>>>> way. I think that part is quite promising should be part of the
> > > >>>>>>
> > > >>>>>> new
> > > >>>>>>
> > > >>>>>> source
> > > >>>>>>
> > > >>>>>> interface.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > > >>>> <
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > > >>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > > >>>> <
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > > >>>>>
> > > >>>>>>
> > > >>>>>> Some experiments on how to build the source reader and its
> > > >>>>>>
> > > >>>>>> library
> > > >>>>>>
> > > >>>>>> for
> > > >>>>>>
> > > >>>>>> common threading/split patterns:
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > > >>>> <
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > > >>>>>
> > > >>>>>>
> > > >>>>>> Best,
> > > >>>>>> Stephan
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mmyy1110@gmail.com
> > > >>> <mailto:
> > > >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> mmyy1110@gmail.com
> > >>
> > > >> <
> > > >>>>>>
> > > >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >>>>>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>> Hi devs,
> > > >>>>>>
> > > >>>>>> Since 1.9 is nearly released, I think we could get back to
> > > >>>>>>
> > > >>>>>> FLIP-27.
> > > >>>>>>
> > > >>>>>> I
> > > >>>>>>
> > > >>>>>> believe it should be included in 1.10.
> > > >>>>>>
> > > >>>>>> There are so many things mentioned in document of FLIP-27. [1] I
> > > >>>>>>
> > > >>>>>> think
> > > >>>>>>
> > > >>>>>> we'd better discuss them separately. However the wiki is not a
> > > >>>>>>
> > > >>>>>> good
> > > >>>>>>
> > > >>>>>> place
> > > >>>>>>
> > > >>>>>> to discuss. I wrote google doc about SplitReader API which
> > > >>>>>>
> > > >>>>>> misses
> > > >>>>>>
> > > >>>>>> some
> > > >>>>>>
> > > >>>>>> details in the document. [2]
> > > >>>>>>
> > > >>>>>> 1.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > >>>> <
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > >>>>>
> > > >>>>>>
> > > >>>>>> 2.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > > >>>> <
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > > >>>>>
> > > >>>>>>
> > > >>>>>> CC Stephan, Aljoscha, Piotrek, Becket
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com
> > > >> <mailto:
> > > >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> mmyy1110@gmail.com
> > >>
> > > >> <
> > > >>>>>>
> > > >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >>>>>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>> Hi Steven,
> > > >>>>>> Thank you for the feedback. Please take a look at the document
> > > >>>>>>
> > > >>>>>> FLIP-27
> > > >>>>>>
> > > >>>>>> <
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > >>>> <
> > > >>>>
> > > >>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > >>>>>
> > > >>>>>>
> > > >>>>>> which
> > > >>>>>>
> > > >>>>>> is updated recently. A lot of details of enumerator were added
> > > >>>>>>
> > > >>>>>> in
> > > >>>>>>
> > > >>>>>> this
> > > >>>>>>
> > > >>>>>> document. I think it would help.
> > > >>>>>>
> > > >>>>>> Steven Wu <stevenz3wu@gmail.com <ma...@gmail.com>>
> <
> > > >>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
> > > >>> stevenz3wu@gmail.com
> > > >>>> <ma...@gmail.com>> <stevenz3wu@gmail.com <mailto:
> > > >>>> stevenz3wu@gmail.com>>
> > > >>>>>>
> > > >>>>>> 于2019年3月28日周四
> > > >>>>>>
> > > >>>>>> 下午12:52写道:
> > > >>>>>>
> > > >>>>>> This proposal mentioned that SplitEnumerator might run on the
> > > >>>>>> JobManager or
> > > >>>>>> in a single task on a TaskManager.
> > > >>>>>>
> > > >>>>>> if enumerator is a single task on a taskmanager, then the job
> > > >>>>>>
> > > >>>>>> DAG
> > > >>>>>>
> > > >>>>>> can
> > > >>>>>>
> > > >>>>>> never
> > > >>>>>> been embarrassingly parallel anymore. That will nullify the
> > > >>>>>>
> > > >>>>>> leverage
> > > >>>>>>
> > > >>>>>> of
> > > >>>>>>
> > > >>>>>> fine-grained recovery for embarrassingly parallel jobs.
> > > >>>>>>
> > > >>>>>> It's not clear to me what's the implication of running
> > > >>>>>>
> > > >>>>>> enumerator
> > > >>>>>>
> > > >>>>>> on
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> jobmanager. So I will leave that out for now.
> > > >>>>>>
> > > >>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com
> > > >> <mailto:
> > > >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> mmyy1110@gmail.com
> > >>
> > > >> <
> > > >>>>>>
> > > >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >>>>>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>> Hi Stephan & Piotrek,
> > > >>>>>>
> > > >>>>>> Thank you for feedback.
> > > >>>>>>
> > > >>>>>> It seems that there are a lot of things to do in community.
> > > >>>>>>
> > > >>>>>> I
> > > >>>>>>
> > > >>>>>> am
> > > >>>>>>
> > > >>>>>> just
> > > >>>>>>
> > > >>>>>> afraid that this discussion may be forgotten since there so
> > > >>>>>>
> > > >>>>>> many
> > > >>>>>>
> > > >>>>>> proposals
> > > >>>>>>
> > > >>>>>> recently.
> > > >>>>>> Anyway, wish to see the split topics soon :)
> > > >>>>>>
> > > >>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:
> > piotr@da-platform.com
> > > >>>>
> > > >>> <
> > > >>>> piotr@da-platform.com <ma...@da-platform.com>> <
> > > >>>> piotr@da-platform.com <ma...@da-platform.com>> <
> > > >>>> piotr@da-platform.com <ma...@da-platform.com>>
> > > >>>>>>
> > > >>>>>> 于2019年1月24日周四
> > > >>>>>>
> > > >>>>>> 下午8:21写道:
> > > >>>>>>
> > > >>>>>> Hi Biao!
> > > >>>>>>
> > > >>>>>> This discussion was stalled because of preparations for
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> open
> > > >>>>>>
> > > >>>>>> sourcing
> > > >>>>>>
> > > >>>>>> & merging Blink. I think before creating the tickets we
> > > >>>>>>
> > > >>>>>> should
> > > >>>>>>
> > > >>>>>> split this
> > > >>>>>>
> > > >>>>>> discussion into topics/areas outlined by Stephan and
> > > >>>>>>
> > > >>>>>> create
> > > >>>>>>
> > > >>>>>> Flips
> > > >>>>>>
> > > >>>>>> for
> > > >>>>>>
> > > >>>>>> that.
> > > >>>>>>
> > > >>>>>> I think there is no chance for this to be completed in
> > > >>>>>>
> > > >>>>>> couple
> > > >>>>>>
> > > >>>>>> of
> > > >>>>>>
> > > >>>>>> remaining
> > > >>>>>>
> > > >>>>>> weeks/1 month before 1.8 feature freeze, however it would
> > > >>>>>>
> > > >>>>>> be
> > > >>>>>>
> > > >>>>>> good
> > > >>>>>>
> > > >>>>>> to aim
> > > >>>>>>
> > > >>>>>> with those changes for 1.9.
> > > >>>>>>
> > > >>>>>> Piotrek
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com <mailto:
> > > >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:
> mmyy1110@gmail.com
> > >>
> > > >> <
> > > >>>>>>
> > > >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >>>>>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>> Hi community,
> > > >>>>>> The summary of Stephan makes a lot sense to me. It is
> > > >>>>>>
> > > >>>>>> much
> > > >>>>>>
> > > >>>>>> clearer
> > > >>>>>>
> > > >>>>>> indeed
> > > >>>>>>
> > > >>>>>> after splitting the complex topic into small ones.
> > > >>>>>> I was wondering is there any detail plan for next step?
> > > >>>>>>
> > > >>>>>> If
> > > >>>>>>
> > > >>>>>> not,
> > > >>>>>>
> > > >>>>>> I
> > > >>>>>>
> > > >>>>>> would
> > > >>>>>>
> > > >>>>>> like to push this thing forward by creating some JIRA
> > > >>>>>>
> > > >>>>>> issues.
> > > >>>>>>
> > > >>>>>> Another question is that should version 1.8 include
> > > >>>>>>
> > > >>>>>> these
> > > >>>>>>
> > > >>>>>> features?
> > > >>>>>>
> > > >>>>>> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <
> > > >>>> sewen@apache.org <ma...@apache.org>> <sewen@apache.org
> > > <mailto:
> > > >>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>>
> > > >>>> 于2018年12月1日周六
> > > >>>>>>
> > > >>>>>> 上午4:20写道:
> > > >>>>>>
> > > >>>>>> Thanks everyone for the lively discussion. Let me try
> > > >>>>>>
> > > >>>>>> to
> > > >>>>>>
> > > >>>>>> summarize
> > > >>>>>>
> > > >>>>>> where I
> > > >>>>>>
> > > >>>>>> see convergence in the discussion and open issues.
> > > >>>>>> I'll try to group this by design aspect of the source.
> > > >>>>>>
> > > >>>>>> Please
> > > >>>>>>
> > > >>>>>> let me
> > > >>>>>>
> > > >>>>>> know
> > > >>>>>>
> > > >>>>>> if I got things wrong or missed something crucial here.
> > > >>>>>>
> > > >>>>>> For issues 1-3, if the below reflects the state of the
> > > >>>>>>
> > > >>>>>> discussion, I
> > > >>>>>>
> > > >>>>>> would
> > > >>>>>>
> > > >>>>>> try and update the FLIP in the next days.
> > > >>>>>> For the remaining ones we need more discussion.
> > > >>>>>>
> > > >>>>>> I would suggest to fork each of these aspects into a
> > > >>>>>>
> > > >>>>>> separate
> > > >>>>>>
> > > >>>>>> mail
> > > >>>>>>
> > > >>>>>> thread,
> > > >>>>>>
> > > >>>>>> or will loose sight of the individual aspects.
> > > >>>>>>
> > > >>>>>> *(1) Separation of Split Enumerator and Split Reader*
> > > >>>>>>
> > > >>>>>> - All seem to agree this is a good thing
> > > >>>>>> - Split Enumerator could in the end live on JobManager
> > > >>>>>>
> > > >>>>>> (and
> > > >>>>>>
> > > >>>>>> assign
> > > >>>>>>
> > > >>>>>> splits
> > > >>>>>>
> > > >>>>>> via RPC) or in a task (and assign splits via data
> > > >>>>>>
> > > >>>>>> streams)
> > > >>>>>>
> > > >>>>>> - this discussion is orthogonal and should come later,
> > > >>>>>>
> > > >>>>>> when
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> interface
> > > >>>>>>
> > > >>>>>> is agreed upon.
> > > >>>>>>
> > > >>>>>> *(2) Split Readers for one or more splits*
> > > >>>>>>
> > > >>>>>> - Discussion seems to agree that we need to support
> > > >>>>>>
> > > >>>>>> one
> > > >>>>>>
> > > >>>>>> reader
> > > >>>>>>
> > > >>>>>> that
> > > >>>>>>
> > > >>>>>> possibly handles multiple splits concurrently.
> > > >>>>>> - The requirement comes from sources where one
> > > >>>>>>
> > > >>>>>> poll()-style
> > > >>>>>>
> > > >>>>>> call
> > > >>>>>>
> > > >>>>>> fetches
> > > >>>>>>
> > > >>>>>> data from different splits / partitions
> > > >>>>>>    --> example sources that require that would be for
> > > >>>>>>
> > > >>>>>> example
> > > >>>>>>
> > > >>>>>> Kafka,
> > > >>>>>>
> > > >>>>>> Pravega, Pulsar
> > > >>>>>>
> > > >>>>>> - Could have one split reader per source, or multiple
> > > >>>>>>
> > > >>>>>> split
> > > >>>>>>
> > > >>>>>> readers
> > > >>>>>>
> > > >>>>>> that
> > > >>>>>>
> > > >>>>>> share the "poll()" function
> > > >>>>>> - To not make it too complicated, we can start with
> > > >>>>>>
> > > >>>>>> thinking
> > > >>>>>>
> > > >>>>>> about
> > > >>>>>>
> > > >>>>>> one
> > > >>>>>>
> > > >>>>>> split reader for all splits initially and see if that
> > > >>>>>>
> > > >>>>>> covers
> > > >>>>>>
> > > >>>>>> all
> > > >>>>>>
> > > >>>>>> requirements
> > > >>>>>>
> > > >>>>>> *(3) Threading model of the Split Reader*
> > > >>>>>>
> > > >>>>>> - Most active part of the discussion ;-)
> > > >>>>>>
> > > >>>>>> - A non-blocking way for Flink's task code to interact
> > > >>>>>>
> > > >>>>>> with
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> source
> > > >>>>>>
> > > >>>>>> is
> > > >>>>>>
> > > >>>>>> needed in order to a task runtime code based on a
> > > >>>>>> single-threaded/actor-style task design
> > > >>>>>>    --> I personally am a big proponent of that, it will
> > > >>>>>>
> > > >>>>>> help
> > > >>>>>>
> > > >>>>>> with
> > > >>>>>>
> > > >>>>>> well-behaved checkpoints, efficiency, and simpler yet
> > > >>>>>>
> > > >>>>>> more
> > > >>>>>>
> > > >>>>>> robust
> > > >>>>>>
> > > >>>>>> runtime
> > > >>>>>>
> > > >>>>>> code
> > > >>>>>>
> > > >>>>>> - Users care about simple abstraction, so as a
> > > >>>>>>
> > > >>>>>> subclass
> > > >>>>>>
> > > >>>>>> of
> > > >>>>>>
> > > >>>>>> SplitReader
> > > >>>>>>
> > > >>>>>> (non-blocking / async) we need to have a
> > > >>>>>>
> > > >>>>>> BlockingSplitReader
> > > >>>>>>
> > > >>>>>> which
> > > >>>>>>
> > > >>>>>> will
> > > >>>>>>
> > > >>>>>> form the basis of most source implementations.
> > > >>>>>>
> > > >>>>>> BlockingSplitReader
> > > >>>>>>
> > > >>>>>> lets
> > > >>>>>>
> > > >>>>>> users do blocking simple poll() calls.
> > > >>>>>> - The BlockingSplitReader would spawn a thread (or
> > > >>>>>>
> > > >>>>>> more)
> > > >>>>>>
> > > >>>>>> and
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> thread(s) can make blocking calls and hand over data
> > > >>>>>>
> > > >>>>>> buffers
> > > >>>>>>
> > > >>>>>> via
> > > >>>>>>
> > > >>>>>> a
> > > >>>>>>
> > > >>>>>> blocking
> > > >>>>>>
> > > >>>>>> queue
> > > >>>>>> - This should allow us to cover both, a fully async
> > > >>>>>>
> > > >>>>>> runtime,
> > > >>>>>>
> > > >>>>>> and a
> > > >>>>>>
> > > >>>>>> simple
> > > >>>>>>
> > > >>>>>> blocking interface for users.
> > > >>>>>> - This is actually very similar to how the Kafka
> > > >>>>>>
> > > >>>>>> connectors
> > > >>>>>>
> > > >>>>>> work.
> > > >>>>>>
> > > >>>>>> Kafka
> > > >>>>>>
> > > >>>>>> 9+ with one thread, Kafka 8 with multiple threads
> > > >>>>>>
> > > >>>>>> - On the base SplitReader (the async one), the
> > > >>>>>>
> > > >>>>>> non-blocking
> > > >>>>>>
> > > >>>>>> method
> > > >>>>>>
> > > >>>>>> that
> > > >>>>>>
> > > >>>>>> gets the next chunk of data would signal data
> > > >>>>>>
> > > >>>>>> availability
> > > >>>>>>
> > > >>>>>> via
> > > >>>>>>
> > > >>>>>> a
> > > >>>>>>
> > > >>>>>> CompletableFuture, because that gives the best
> > > >>>>>>
> > > >>>>>> flexibility
> > > >>>>>>
> > > >>>>>> (can
> > > >>>>>>
> > > >>>>>> await
> > > >>>>>>
> > > >>>>>> completion or register notification handlers).
> > > >>>>>> - The source task would register a "thenHandle()" (or
> > > >>>>>>
> > > >>>>>> similar)
> > > >>>>>>
> > > >>>>>> on the
> > > >>>>>>
> > > >>>>>> future to put a "take next data" task into the
> > > >>>>>>
> > > >>>>>> actor-style
> > > >>>>>>
> > > >>>>>> mailbox
> > > >>>>>>
> > > >>>>>> *(4) Split Enumeration and Assignment*
> > > >>>>>>
> > > >>>>>> - Splits may be generated lazily, both in cases where
> > > >>>>>>
> > > >>>>>> there
> > > >>>>>>
> > > >>>>>> is a
> > > >>>>>>
> > > >>>>>> limited
> > > >>>>>>
> > > >>>>>> number of splits (but very many), or splits are
> > > >>>>>>
> > > >>>>>> discovered
> > > >>>>>>
> > > >>>>>> over
> > > >>>>>>
> > > >>>>>> time
> > > >>>>>>
> > > >>>>>> - Assignment should also be lazy, to get better load
> > > >>>>>>
> > > >>>>>> balancing
> > > >>>>>>
> > > >>>>>> - Assignment needs support locality preferences
> > > >>>>>>
> > > >>>>>> - Possible design based on discussion so far:
> > > >>>>>>
> > > >>>>>>    --> SplitReader has a method "addSplits(SplitT...)"
> > > >>>>>>
> > > >>>>>> to
> > > >>>>>>
> > > >>>>>> add
> > > >>>>>>
> > > >>>>>> one or
> > > >>>>>>
> > > >>>>>> more
> > > >>>>>>
> > > >>>>>> splits. Some split readers might assume they have only
> > > >>>>>>
> > > >>>>>> one
> > > >>>>>>
> > > >>>>>> split
> > > >>>>>>
> > > >>>>>> ever,
> > > >>>>>>
> > > >>>>>> concurrently, others assume multiple splits. (Note:
> > > >>>>>>
> > > >>>>>> idea
> > > >>>>>>
> > > >>>>>> behind
> > > >>>>>>
> > > >>>>>> being
> > > >>>>>>
> > > >>>>>> able
> > > >>>>>>
> > > >>>>>> to add multiple splits at the same time is to ease
> > > >>>>>>
> > > >>>>>> startup
> > > >>>>>>
> > > >>>>>> where
> > > >>>>>>
> > > >>>>>> multiple
> > > >>>>>>
> > > >>>>>> splits may be assigned instantly.)
> > > >>>>>>    --> SplitReader has a context object on which it can
> > > >>>>>>
> > > >>>>>> call
> > > >>>>>>
> > > >>>>>> indicate
> > > >>>>>>
> > > >>>>>> when
> > > >>>>>>
> > > >>>>>> splits are completed. The enumerator gets that
> > > >>>>>>
> > > >>>>>> notification and
> > > >>>>>>
> > > >>>>>> can
> > > >>>>>>
> > > >>>>>> use
> > > >>>>>>
> > > >>>>>> to
> > > >>>>>>
> > > >>>>>> decide when to assign new splits. This should help both
> > > >>>>>>
> > > >>>>>> in
> > > >>>>>>
> > > >>>>>> cases
> > > >>>>>>
> > > >>>>>> of
> > > >>>>>>
> > > >>>>>> sources
> > > >>>>>>
> > > >>>>>> that take splits lazily (file readers) and in case the
> > > >>>>>>
> > > >>>>>> source
> > > >>>>>>
> > > >>>>>> needs to
> > > >>>>>>
> > > >>>>>> preserve a partial order between splits (Kinesis,
> > > >>>>>>
> > > >>>>>> Pravega,
> > > >>>>>>
> > > >>>>>> Pulsar may
> > > >>>>>>
> > > >>>>>> need
> > > >>>>>>
> > > >>>>>> that).
> > > >>>>>>    --> SplitEnumerator gets notification when
> > > >>>>>>
> > > >>>>>> SplitReaders
> > > >>>>>>
> > > >>>>>> start
> > > >>>>>>
> > > >>>>>> and
> > > >>>>>>
> > > >>>>>> when
> > > >>>>>>
> > > >>>>>> they finish splits. They can decide at that moment to
> > > >>>>>>
> > > >>>>>> push
> > > >>>>>>
> > > >>>>>> more
> > > >>>>>>
> > > >>>>>> splits
> > > >>>>>>
> > > >>>>>> to
> > > >>>>>>
> > > >>>>>> that reader
> > > >>>>>>    --> The SplitEnumerator should probably be aware of
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> source
> > > >>>>>>
> > > >>>>>> parallelism, to build its initial distribution.
> > > >>>>>>
> > > >>>>>> - Open question: Should the source expose something
> > > >>>>>>
> > > >>>>>> like
> > > >>>>>>
> > > >>>>>> "host
> > > >>>>>>
> > > >>>>>> preferences", so that yarn/mesos/k8s can take this into
> > > >>>>>>
> > > >>>>>> account
> > > >>>>>>
> > > >>>>>> when
> > > >>>>>>
> > > >>>>>> selecting a node to start a TM on?
> > > >>>>>>
> > > >>>>>> *(5) Watermarks and event time alignment*
> > > >>>>>>
> > > >>>>>> - Watermark generation, as well as idleness, needs to
> > > >>>>>>
> > > >>>>>> be
> > > >>>>>>
> > > >>>>>> per
> > > >>>>>>
> > > >>>>>> split
> > > >>>>>>
> > > >>>>>> (like
> > > >>>>>>
> > > >>>>>> currently in the Kafka Source, per partition)
> > > >>>>>> - It is desirable to support optional
> > > >>>>>>
> > > >>>>>> event-time-alignment,
> > > >>>>>>
> > > >>>>>> meaning
> > > >>>>>>
> > > >>>>>> that
> > > >>>>>>
> > > >>>>>> splits that are ahead are back-pressured or temporarily
> > > >>>>>>
> > > >>>>>> unsubscribed
> > > >>>>>>
> > > >>>>>> - I think i would be desirable to encapsulate
> > > >>>>>>
> > > >>>>>> watermark
> > > >>>>>>
> > > >>>>>> generation
> > > >>>>>>
> > > >>>>>> logic
> > > >>>>>>
> > > >>>>>> in watermark generators, for a separation of concerns.
> > > >>>>>>
> > > >>>>>> The
> > > >>>>>>
> > > >>>>>> watermark
> > > >>>>>>
> > > >>>>>> generators should run per split.
> > > >>>>>> - Using watermark generators would also help with
> > > >>>>>>
> > > >>>>>> another
> > > >>>>>>
> > > >>>>>> problem of
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> suggested interface, namely supporting non-periodic
> > > >>>>>>
> > > >>>>>> watermarks
> > > >>>>>>
> > > >>>>>> efficiently.
> > > >>>>>>
> > > >>>>>> - Need a way to "dispatch" next record to different
> > > >>>>>>
> > > >>>>>> watermark
> > > >>>>>>
> > > >>>>>> generators
> > > >>>>>>
> > > >>>>>> - Need a way to tell SplitReader to "suspend" a split
> > > >>>>>>
> > > >>>>>> until a
> > > >>>>>>
> > > >>>>>> certain
> > > >>>>>>
> > > >>>>>> watermark is reached (event time backpressure)
> > > >>>>>> - This would in fact be not needed (and thus simpler)
> > > >>>>>>
> > > >>>>>> if
> > > >>>>>>
> > > >>>>>> we
> > > >>>>>>
> > > >>>>>> had
> > > >>>>>>
> > > >>>>>> a
> > > >>>>>>
> > > >>>>>> SplitReader per split and may be a reason to re-open
> > > >>>>>>
> > > >>>>>> that
> > > >>>>>>
> > > >>>>>> discussion
> > > >>>>>>
> > > >>>>>> *(6) Watermarks across splits and in the Split
> > > >>>>>>
> > > >>>>>> Enumerator*
> > > >>>>>>
> > > >>>>>> - The split enumerator may need some watermark
> > > >>>>>>
> > > >>>>>> awareness,
> > > >>>>>>
> > > >>>>>> which
> > > >>>>>>
> > > >>>>>> should
> > > >>>>>>
> > > >>>>>> be
> > > >>>>>>
> > > >>>>>> purely based on split metadata (like create timestamp
> > > >>>>>>
> > > >>>>>> of
> > > >>>>>>
> > > >>>>>> file
> > > >>>>>>
> > > >>>>>> splits)
> > > >>>>>>
> > > >>>>>> - If there are still more splits with overlapping
> > > >>>>>>
> > > >>>>>> event
> > > >>>>>>
> > > >>>>>> time
> > > >>>>>>
> > > >>>>>> range
> > > >>>>>>
> > > >>>>>> for
> > > >>>>>>
> > > >>>>>> a
> > > >>>>>>
> > > >>>>>> split reader, then that split reader should not advance
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> watermark
> > > >>>>>>
> > > >>>>>> within the split beyond the overlap boundary. Otherwise
> > > >>>>>>
> > > >>>>>> future
> > > >>>>>>
> > > >>>>>> splits
> > > >>>>>>
> > > >>>>>> will
> > > >>>>>>
> > > >>>>>> produce late data.
> > > >>>>>>
> > > >>>>>> - One way to approach this could be that the split
> > > >>>>>>
> > > >>>>>> enumerator
> > > >>>>>>
> > > >>>>>> may
> > > >>>>>>
> > > >>>>>> send
> > > >>>>>>
> > > >>>>>> watermarks to the readers, and the readers cannot emit
> > > >>>>>>
> > > >>>>>> watermarks
> > > >>>>>>
> > > >>>>>> beyond
> > > >>>>>>
> > > >>>>>> that received watermark.
> > > >>>>>> - Many split enumerators would simply immediately send
> > > >>>>>>
> > > >>>>>> Long.MAX
> > > >>>>>>
> > > >>>>>> out
> > > >>>>>>
> > > >>>>>> and
> > > >>>>>>
> > > >>>>>> leave the progress purely to the split readers.
> > > >>>>>>
> > > >>>>>> - For event-time alignment / split back pressure, this
> > > >>>>>>
> > > >>>>>> begs
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> question
> > > >>>>>>
> > > >>>>>> how we can avoid deadlocks that may arise when splits
> > > >>>>>>
> > > >>>>>> are
> > > >>>>>>
> > > >>>>>> suspended
> > > >>>>>>
> > > >>>>>> for
> > > >>>>>>
> > > >>>>>> event time back pressure,
> > > >>>>>>
> > > >>>>>> *(7) Batch and streaming Unification*
> > > >>>>>>
> > > >>>>>> - Functionality wise, the above design should support
> > > >>>>>>
> > > >>>>>> both
> > > >>>>>>
> > > >>>>>> - Batch often (mostly) does not care about reading "in
> > > >>>>>>
> > > >>>>>> order"
> > > >>>>>>
> > > >>>>>> and
> > > >>>>>>
> > > >>>>>> generating watermarks
> > > >>>>>>    --> Might use different enumerator logic that is
> > > >>>>>>
> > > >>>>>> more
> > > >>>>>>
> > > >>>>>> locality
> > > >>>>>>
> > > >>>>>> aware
> > > >>>>>>
> > > >>>>>> and ignores event time order
> > > >>>>>>    --> Does not generate watermarks
> > > >>>>>> - Would be great if bounded sources could be
> > > >>>>>>
> > > >>>>>> identified
> > > >>>>>>
> > > >>>>>> at
> > > >>>>>>
> > > >>>>>> compile
> > > >>>>>>
> > > >>>>>> time,
> > > >>>>>>
> > > >>>>>> so that "env.addBoundedSource(...)" is type safe and
> > > >>>>>>
> > > >>>>>> can
> > > >>>>>>
> > > >>>>>> return a
> > > >>>>>>
> > > >>>>>> "BoundedDataStream".
> > > >>>>>> - Possible to defer this discussion until later
> > > >>>>>>
> > > >>>>>> *Miscellaneous Comments*
> > > >>>>>>
> > > >>>>>> - Should the source have a TypeInformation for the
> > > >>>>>>
> > > >>>>>> produced
> > > >>>>>>
> > > >>>>>> type,
> > > >>>>>>
> > > >>>>>> instead
> > > >>>>>>
> > > >>>>>> of a serializer? We need a type information in the
> > > >>>>>>
> > > >>>>>> stream
> > > >>>>>>
> > > >>>>>> anyways, and
> > > >>>>>>
> > > >>>>>> can
> > > >>>>>>
> > > >>>>>> derive the serializer from that. Plus, creating the
> > > >>>>>>
> > > >>>>>> serializer
> > > >>>>>>
> > > >>>>>> should
> > > >>>>>>
> > > >>>>>> respect the ExecutionConfig.
> > > >>>>>>
> > > >>>>>> - The TypeSerializer interface is very powerful but
> > > >>>>>>
> > > >>>>>> also
> > > >>>>>>
> > > >>>>>> not
> > > >>>>>>
> > > >>>>>> easy to
> > > >>>>>>
> > > >>>>>> implement. Its purpose is to handle data super
> > > >>>>>>
> > > >>>>>> efficiently,
> > > >>>>>>
> > > >>>>>> support
> > > >>>>>>
> > > >>>>>> flexible ways of evolution, etc.
> > > >>>>>> For metadata I would suggest to look at the
> > > >>>>>>
> > > >>>>>> SimpleVersionedSerializer
> > > >>>>>>
> > > >>>>>> instead, which is used for example for checkpoint
> > > >>>>>>
> > > >>>>>> master
> > > >>>>>>
> > > >>>>>> hooks,
> > > >>>>>>
> > > >>>>>> or for
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> streaming file sink. I think that is is a good match
> > > >>>>>>
> > > >>>>>> for
> > > >>>>>>
> > > >>>>>> cases
> > > >>>>>>
> > > >>>>>> where
> > > >>>>>>
> > > >>>>>> we
> > > >>>>>>
> > > >>>>>> do
> > > >>>>>>
> > > >>>>>> not need more than ser/deser (no copy, etc.) and don't
> > > >>>>>>
> > > >>>>>> need to
> > > >>>>>>
> > > >>>>>> push
> > > >>>>>>
> > > >>>>>> versioning out of the serialization paths for best
> > > >>>>>>
> > > >>>>>> performance
> > > >>>>>>
> > > >>>>>> (as in
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> TypeSerializer)
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > > >>>>>>
> > > >>>>>> k.kloudas@data-artisans.com>
> > > >>>>>>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Hi Biao,
> > > >>>>>>
> > > >>>>>> Thanks for the answer!
> > > >>>>>>
> > > >>>>>> So given the multi-threaded readers, now we have as
> > > >>>>>>
> > > >>>>>> open
> > > >>>>>>
> > > >>>>>> questions:
> > > >>>>>>
> > > >>>>>> 1) How do we let the checkpoints pass through our
> > > >>>>>>
> > > >>>>>> multi-threaded
> > > >>>>>>
> > > >>>>>> reader
> > > >>>>>>
> > > >>>>>> operator?
> > > >>>>>>
> > > >>>>>> 2) Do we have separate reader and source operators or
> > > >>>>>>
> > > >>>>>> not? In
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> strategy
> > > >>>>>>
> > > >>>>>> that has a separate source, the source operator has a
> > > >>>>>>
> > > >>>>>> parallelism of
> > > >>>>>>
> > > >>>>>> 1
> > > >>>>>>
> > > >>>>>> and
> > > >>>>>>
> > > >>>>>> is responsible for split recovery only.
> > > >>>>>>
> > > >>>>>> For the first one, given also the constraints
> > > >>>>>>
> > > >>>>>> (blocking,
> > > >>>>>>
> > > >>>>>> finite
> > > >>>>>>
> > > >>>>>> queues,
> > > >>>>>>
> > > >>>>>> etc), I do not have an answer yet.
> > > >>>>>>
> > > >>>>>> For the 2nd, I think that we should go with separate
> > > >>>>>>
> > > >>>>>> operators
> > > >>>>>>
> > > >>>>>> for
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> source and the readers, for the following reasons:
> > > >>>>>>
> > > >>>>>> 1) This is more aligned with a potential future
> > > >>>>>>
> > > >>>>>> improvement
> > > >>>>>>
> > > >>>>>> where the
> > > >>>>>>
> > > >>>>>> split
> > > >>>>>>
> > > >>>>>> discovery becomes a responsibility of the JobManager
> > > >>>>>>
> > > >>>>>> and
> > > >>>>>>
> > > >>>>>> readers are
> > > >>>>>>
> > > >>>>>> pooling more work from the JM.
> > > >>>>>>
> > > >>>>>> 2) The source is going to be the "single point of
> > > >>>>>>
> > > >>>>>> truth".
> > > >>>>>>
> > > >>>>>> It
> > > >>>>>>
> > > >>>>>> will
> > > >>>>>>
> > > >>>>>> know
> > > >>>>>>
> > > >>>>>> what
> > > >>>>>>
> > > >>>>>> has been processed and what not. If the source and the
> > > >>>>>>
> > > >>>>>> readers
> > > >>>>>>
> > > >>>>>> are a
> > > >>>>>>
> > > >>>>>> single
> > > >>>>>>
> > > >>>>>> operator with parallelism > 1, or in general, if the
> > > >>>>>>
> > > >>>>>> split
> > > >>>>>>
> > > >>>>>> discovery
> > > >>>>>>
> > > >>>>>> is
> > > >>>>>>
> > > >>>>>> done by each task individually, then:
> > > >>>>>>   i) we have to have a deterministic scheme for each
> > > >>>>>>
> > > >>>>>> reader to
> > > >>>>>>
> > > >>>>>> assign
> > > >>>>>>
> > > >>>>>> splits to itself (e.g. mod subtaskId). This is not
> > > >>>>>>
> > > >>>>>> necessarily
> > > >>>>>>
> > > >>>>>> trivial
> > > >>>>>>
> > > >>>>>> for
> > > >>>>>>
> > > >>>>>> all sources.
> > > >>>>>>   ii) each reader would have to keep a copy of all its
> > > >>>>>>
> > > >>>>>> processed
> > > >>>>>>
> > > >>>>>> slpits
> > > >>>>>>
> > > >>>>>>   iii) the state has to be a union state with a
> > > >>>>>>
> > > >>>>>> non-trivial
> > > >>>>>>
> > > >>>>>> merging
> > > >>>>>>
> > > >>>>>> logic
> > > >>>>>>
> > > >>>>>> in order to support rescaling.
> > > >>>>>>
> > > >>>>>> Two additional points that you raised above:
> > > >>>>>>
> > > >>>>>> i) The point that you raised that we need to keep all
> > > >>>>>>
> > > >>>>>> splits
> > > >>>>>>
> > > >>>>>> (processed
> > > >>>>>>
> > > >>>>>> and
> > > >>>>>>
> > > >>>>>> not-processed) I think is a bit of a strong
> > > >>>>>>
> > > >>>>>> requirement.
> > > >>>>>>
> > > >>>>>> This
> > > >>>>>>
> > > >>>>>> would
> > > >>>>>>
> > > >>>>>> imply
> > > >>>>>>
> > > >>>>>> that for infinite sources the state will grow
> > > >>>>>>
> > > >>>>>> indefinitely.
> > > >>>>>>
> > > >>>>>> This is
> > > >>>>>>
> > > >>>>>> problem
> > > >>>>>>
> > > >>>>>> is even more pronounced if we do not have a single
> > > >>>>>>
> > > >>>>>> source
> > > >>>>>>
> > > >>>>>> that
> > > >>>>>>
> > > >>>>>> assigns
> > > >>>>>>
> > > >>>>>> splits to readers, as each reader will have its own
> > > >>>>>>
> > > >>>>>> copy
> > > >>>>>>
> > > >>>>>> of
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> state.
> > > >>>>>>
> > > >>>>>> ii) it is true that for finite sources we need to
> > > >>>>>>
> > > >>>>>> somehow
> > > >>>>>>
> > > >>>>>> not
> > > >>>>>>
> > > >>>>>> close
> > > >>>>>>
> > > >>>>>> the
> > > >>>>>>
> > > >>>>>> readers when the source/split discoverer finishes. The
> > > >>>>>> ContinuousFileReaderOperator has a work-around for
> > > >>>>>>
> > > >>>>>> that.
> > > >>>>>>
> > > >>>>>> It is
> > > >>>>>>
> > > >>>>>> not
> > > >>>>>>
> > > >>>>>> elegant,
> > > >>>>>>
> > > >>>>>> and checkpoints are not emitted after closing the
> > > >>>>>>
> > > >>>>>> source,
> > > >>>>>>
> > > >>>>>> but
> > > >>>>>>
> > > >>>>>> this, I
> > > >>>>>>
> > > >>>>>> believe, is a bigger problem which requires more
> > > >>>>>>
> > > >>>>>> changes
> > > >>>>>>
> > > >>>>>> than
> > > >>>>>>
> > > >>>>>> just
> > > >>>>>>
> > > >>>>>> refactoring the source interface.
> > > >>>>>>
> > > >>>>>> Cheers,
> > > >>>>>> Kostas
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> --
> > > >>>>>> Best, Jingsong Lee
> > > >>>>
> > > >>>>
> > > >>>
> > > >>
> > > >>
> > > >> --
> > > >> Best, Jingsong Lee
> > > >>
> > > >
> > >
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Jark Wu <im...@gmail.com>.
Hi Becket,

That's great we have reached a consensus on Source#getBoundedness().

Regarding to option#3, my concern is that if we don't support streaming
mode for bounded source,
how could we create a testing source for streaming mode? Currently, all the
testing source for streaming
are bounded, so that the integration test will finish finally.

Regarding to Source#getRecordOrder(), could we have a implicit contract
that unbounded source should
already read in order (i.e. reading partitions in parallel), for bounded
source the order is not mandatory.
This is also the behaviors of the current sources.
1) a source can't guarantee it reads in strict order, because the producer
may produce data not in order.
2) *Bounded-StrictOrder* is not necessary, because batch can reorder data.

Best,
Jark



On Tue, 17 Dec 2019 at 22:03, Becket Qin <be...@gmail.com> wrote:

> Hi folks,
>
> Thanks for the comments. I am convinced that the Source API should not take
> boundedness as a parameter after it is constructed. What Timo and Dawid
> suggested sounds a reasonable solution to me. So the Source API would
> become:
>
> Source {
>     Boundedness getBoundedness();
> }
>
> Assuming the above Source API, in addition to the two options mentioned in
> earlier emails, I am thinking of another option:
>
> *Option 3:*
> // MySource must be unbounded, otherwise throws exception.
> DataStream<Type> dataStream = env.source(mySource);
>
> // MySource must be bounded, otherwise throws exception.
> BoundedDataStream<Type> boundedDataStream = env.boundedSource(mySource);
>
> The pros of this API are:
>    a) It fits the requirements from Table / SQL well.
>    b) DataStream users still have type safety (option 2 only has partial
> type safety).
>    c) Cristal clear boundedness from the API which makes DataStream join /
> connect easy to reason about.
> The caveats I see,
>    a) It is inconsistent with Table since Table has one unified interface.
>    b) No streaming mode for bounded source.
>
> @Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
> <al...@ververica.com> what do you think of the approach?
>
>
> Orthogonal to the above API, I am wondering whether boundedness is the only
> dimension needed to describe the characteristic of the Source behavior. We
> may also need to have another dimension of *record order*.
>
> For example, when a file source is reading from a directory with bounded
> records, it may have two ways to read.
> 1. Read files in parallel.
> 2. Read files in the chronological order.
> In both cases, the file source is a Bounded Source. However, the processing
> requirement for downstream may be different. In the first case, the
> record processing and result emitting order does not matter, e.g. word
> count. In the second case, the records may have to be processed in the
> order they were read, e.g. change log processing.
>
> If the Source only has a getBoundedness() method, the downstream processors
> would not know whether the records emitted from the Source should be
> processed in order or not. So combining the boundedness and record order,
> we will have four scenarios:
>
> *Bounded-StrictOrder*:     A segment of change log.
> *Bounded-Random*:          Batch Word Count.
> *Unbounded-StrictOrder*: An infinite change log.
> *Unbounded-Random*:     Streaming Word Count.
>
> Option 2 mentioned in the previous email was kind of trying to handle the
> Bounded-StrictOrder case by creating a DataStream from a bounded source,
> which actually does not work.
> It looks that we do not have strict order support in some operators at this
> point, e.g. join. But we may still want to add the semantic to the Source
> first so later on we don't need to change all the source implementations,
> especially given that many of them will be implemented by 3rd party.
>
> Given that, we need another dimension of *Record Order* in the Source. More
> specifically, the API would become:
>
> Source {
>     Boundedness getBoundedness();
>     RecordOrder getRecordOrder();
> }
>
> public enum RecordOrder {
>     /** The record in the DataStream must be processed in its strict order
> for correctness. */
>     STRICT,
>     /** The record in the DataStream can be processed in arbitrary order.
> */
>     RANDOM;
> }
>
> Any thoughts?
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <tw...@apache.org> wrote:
>
> > Hi Becket,
> >
> > I completely agree with Dawid's suggestion. The information about the
> > boundedness should come out of the source. Because most of the streaming
> > sources can be made bounded based on some connector specific criterion.
> > In Kafka, it would be an end offset or end timestamp but in any case
> > having just a env.boundedSource() is not enough because parameters for
> > making the source bounded are missing.
> >
> > I suggest to have a simple `isBounded(): Boolean` flag in every source
> > that might be influenced by a connector builder as Dawid mentioned.
> >
> > For type safety during programming, we can still go with *Final state
> > 1*. By having a env.source() vs env.boundedSource(). The latter would
> > just enforce that the boolean flag is set to `true` and could make
> > bounded operations available (if we need that actually).
> >
> > However, I don't think that we should start making a unified Table API
> > ununified again. Boundedness is an optimization property. Every bounded
> > operation can also executed in an unbounded way using updates/retraction
> > or watermarks.
> >
> > Regards,
> > Timo
> >
> >
> > On 15.12.19 14:22, Becket Qin wrote:
> > > Hi Dawid and Jark,
> > >
> > > I think the discussion ultimately boils down to the question that which
> > one
> > > of the following two final states do we want? Once we make this
> decision,
> > > everything else can be naturally derived.
> > >
> > > *Final state 1*: Separate API for bounded / unbounded DataStream &
> Table.
> > > That means any code users write will be valid at the point when they
> > write
> > > the code. This is similar to having type safety check at programming
> > time.
> > > For example,
> > >
> > > BoundedDataStream extends DataStream {
> > > // Operations only available for bounded data.
> > > BoundedDataStream sort(...);
> > >
> > > // Interaction with another BoundedStream returns a Bounded stream.
> > > BoundedJoinedDataStream join(BoundedDataStream other)
> > >
> > > // Interaction with another unbounded stream returns an unbounded
> stream.
> > > JoinedDataStream join(DataStream other)
> > > }
> > >
> > > BoundedTable extends Table {
> > >    // Bounded only operation.
> > > BoundedTable sort(...);
> > >
> > > // Interaction with another BoundedTable returns a BoundedTable.
> > > BoundedTable join(BoundedTable other)
> > >
> > > // Interaction with another unbounded table returns an unbounded table.
> > > Table join(Table other)
> > > }
> > >
> > > *Final state 2*: One unified API for bounded / unbounded DataStream /
> > > Table.
> > > That unified API may throw exception at DAG compilation time if an
> > invalid
> > > operation is tried. This is what Table API currently follows.
> > >
> > > DataStream {
> > > // Throws exception if the DataStream is unbounded.
> > > DataStream sort();
> > > // Get boundedness.
> > > Boundedness getBoundedness();
> > > }
> > >
> > > Table {
> > > // Throws exception if the table has infinite rows.
> > > Table orderBy();
> > >
> > > // Get boundedness.
> > > Boundedness getBoundedness();
> > > }
> > >
> > >>From what I understand, there is no consensus so far on this decision
> > yet.
> > > Whichever final state we choose, we need to make it consistent across
> the
> > > entire project. We should avoid the case that Table follows one final
> > state
> > > while DataStream follows another. Some arguments I am aware of from
> both
> > > sides so far are following:
> > >
> > > Arguments for final state 1:
> > > 1a) Clean API with method safety check at programming time.
> > > 1b) (Counter 2b) Although SQL does not have programming time error
> > check, SQL
> > > is not really a "programming language" per se. So SQL can be different
> > from
> > > Table and DataStream.
> > > 1c)  Although final state 2 seems making it easier for SQL to use given
> > it
> > > is more "config based" than "parameter based", final state 1 can
> probably
> > > also meet what SQL wants by wrapping the Source in TableSource /
> > > TableSourceFactory API if needed.
> > >
> > > Arguments for final state 2:
> > > 2a) The Source API itself seems already sort of following the unified
> API
> > > pattern.
> > > 2b) There is no "programming time" method error check in SQL case, so
> we
> > > cannot really achieve final state 1 across the board.
> > > 2c) It is an easier path given our current status, i.e. Table is
> already
> > > following final state 2.
> > > 2d) Users can always explicitly check the boundedness if they want to.
> > >
> > > As I mentioned earlier, my initial thought was also to have a
> > > "configuration based" Source rather than a "parameter based" Source. So
> > it
> > > is completely possible that I missed some important consideration or
> > design
> > > principles that we want to enforce for the project. It would be good
> > > if @Stephan
> > > Ewen <st...@ververica.com> and @Aljoscha Krettek <
> > aljoscha@ververica.com> can
> > > also provide more thoughts on this.
> > >
> > >
> > > Re: Jingsong
> > >
> > > As you said, there are some batched system source, like parquet/orc
> > source.
> > >> Could we have the batch emit interface to improve performance? The
> > queue of
> > >> per record may cause performance degradation.
> > >
> > >
> > > The current interface does not necessarily cause performance problem
> in a
> > > multi-threading case. In fact, the base implementation allows
> > SplitReaders
> > > to add a batch <E> of records<T> to the records queue<E>, so each
> element
> > > in the records queue would be a batch <E>. In this case, when the main
> > > thread polls records, it will take a batch <E> of records <T> from the
> > > shared records queue and process the records <T> in a batch manner.
> > >
> > > Thanks,
> > >
> > > Jiangjie (Becket) Qin
> > >
> > > On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <ji...@gmail.com>
> > wrote:
> > >
> > >> Hi Becket,
> > >>
> > >> I also have some performance concerns too.
> > >>
> > >> If I understand correctly, SourceOutput will emit data per record into
> > the
> > >> queue? I'm worried about the multithreading performance of this queue.
> > >>
> > >>> One example is some batched messaging systems which only have an
> offset
> > >> for the entire batch instead of individual messages in the batch.
> > >>
> > >> As you said, there are some batched system source, like parquet/orc
> > source.
> > >> Could we have the batch emit interface to improve performance? The
> > queue of
> > >> per record may cause performance degradation.
> > >>
> > >> Best,
> > >> Jingsong Lee
> > >>
> > >> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com> wrote:
> > >>
> > >>> Hi Becket,
> > >>>
> > >>> I think Dawid explained things clearly and makes a lot of sense.
> > >>> I'm also in favor of #2, because #1 doesn't work for our future
> unified
> > >>> envrionment.
> > >>>
> > >>> You can see the vision in this documentation [1]. In the future, we
> > would
> > >>> like to
> > >>> drop the global streaming/batch mode in SQL (i.e.
> > >>> EnvironmentSettings#inStreamingMode/inBatchMode).
> > >>> A source is bounded or unbounded once defined, so queries can be
> > inferred
> > >>> from source to run
> > >>> in streaming or batch or hybrid mode. However, in #1, we will lose
> this
> > >>> ability because the framework
> > >>> doesn't know whether the source is bounded or unbounded.
> > >>>
> > >>> Best,
> > >>> Jark
> > >>>
> > >>>
> > >>> [1]:
> > >>>
> > >>>
> > >>
> >
> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
> > >>>
> > >>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <pi...@ververica.com>
> > >> wrote:
> > >>>
> > >>>> Hi,
> > >>>>
> > >>>> Regarding the:
> > >>>>
> > >>>> Collection<E> getNextRecords()
> > >>>>
> > >>>> I’m pretty sure such design would unfortunately impact the
> performance
> > >>>> (accessing and potentially creating the collection on the hot path).
> > >>>>
> > >>>> Also the
> > >>>>
> > >>>> InputStatus emitNext(DataOutput<T> output) throws Exception;
> > >>>> or
> > >>>> Status pollNext(SourceOutput<T> sourceOutput) throws Exception;
> > >>>>
> > >>>> Gives us some opportunities in the future, to allow Source hot
> looping
> > >>>> inside, until it receives some signal “please exit because of some
> > >>> reasons”
> > >>>> (output collector could return such hint upon collecting the
> result).
> > >> But
> > >>>> that’s another topic outside of this FLIP’s scope.
> > >>>>
> > >>>> Piotrek
> > >>>>
> > >>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <tr...@apache.org>
> > >> wrote:
> > >>>>>
> > >>>>> Hi Becket,
> > >>>>>
> > >>>>> quick clarification from my side because I think you misunderstood
> my
> > >>>>> question. I did not suggest to let the SourceReader return only a
> > >>> single
> > >>>>> record at a time when calling getNextRecords. As the return type
> > >>>> indicates,
> > >>>>> the method can return an arbitrary number of records.
> > >>>>>
> > >>>>> Cheers,
> > >>>>> Till
> > >>>>>
> > >>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> > >>>> dwysakowicz@apache.org <ma...@apache.org>>
> > >>>>> wrote:
> > >>>>>
> > >>>>>> Hi Becket,
> > >>>>>>
> > >>>>>> Issue #1 - Design of Source interface
> > >>>>>>
> > >>>>>> I mentioned the lack of a method like
> > >>>> Source#createEnumerator(Boundedness
> > >>>>>> boundedness, SplitEnumeratorContext context), because without the
> > >>>> current
> > >>>>>> proposal is not complete/does not work.
> > >>>>>>
> > >>>>>> If we say that boundedness is an intrinsic property of a source
> imo
> > >> we
> > >>>>>> don't need the Source#createEnumerator(Boundedness boundedness,
> > >>>>>> SplitEnumeratorContext context) method.
> > >>>>>>
> > >>>>>> Assuming a source from my previous example:
> > >>>>>>
> > >>>>>> Source source = KafkaSource.builder()
> > >>>>>>   ...
> > >>>>>>   .untilTimestamp(...)
> > >>>>>>   .build()
> > >>>>>>
> > >>>>>> Would the enumerator differ if created like
> > >>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
> > >>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating myself, but
> > >>> this
> > >>>> is
> > >>>>>> the part that my opinion differ the most from the current
> proposal.
> > >> I
> > >>>>>> really think it should always be the source that tells if it is
> > >>> bounded
> > >>>> or
> > >>>>>> not. In the current proposal methods continousSource/boundedSource
> > >>>> somewhat
> > >>>>>> reconfigure the source, which I think is misleading.
> > >>>>>>
> > >>>>>> I think a call like:
> > >>>>>>
> > >>>>>> Source source = KafkaSource.builder()
> > >>>>>>   ...
> > >>>>>>   .readContinously() / readUntilLatestOffset() /
> readUntilTimestamp
> > /
> > >>>> readUntilOffsets / ...
> > >>>>>>   .build()
> > >>>>>>
> > >>>>>> is way cleaner (and expressive) than
> > >>>>>>
> > >>>>>> Source source = KafkaSource.builder()
> > >>>>>>   ...
> > >>>>>>   .build()
> > >>>>>>
> > >>>>>>
> > >>>>>> env.continousSource(source) // which actually underneath would
> call
> > >>>> createEnumerator(CONTINUOUS, ctx) which would be equivalent to
> > >>>> source.readContinously().createEnumerator(ctx)
> > >>>>>> // or
> > >>>>>> env.boundedSource(source) // which actually underneath would call
> > >>>> createEnumerator(BOUNDED, ctx) which would be equivalent to
> > >>>> source.readUntilLatestOffset().createEnumerator(ctx)
> > >>>>>>
> > >>>>>>
> > >>>>>> Sorry for the comparison, but to me it seems there is too much
> magic
> > >>>>>> happening underneath those two calls.
> > >>>>>>
> > >>>>>> I really believe the Source interface should have getBoundedness
> > >>> method
> > >>>>>> instead of (supportBoundedness) + createEnumerator(Boundedness,
> ...)
> > >>>>>>
> > >>>>>>
> > >>>>>> Issue #2 - Design of
> > >>>>>> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> > >>>>>>
> > >>>>>> As you might have guessed I am slightly in favor of option #2
> > >>> modified.
> > >>>>>> Yes I am aware every step of the dag would have to be able to say
> if
> > >>> it
> > >>>> is
> > >>>>>> bounded or not. I have a feeling it would be easier to express
> cross
> > >>>>>> bounded/unbounded operations, but I must admit I have not thought
> it
> > >>>>>> through thoroughly, In the spirit of batch is just a special case
> of
> > >>>>>> streaming I thought BoundedStream would extend from DataStream.
> > >>> Correct
> > >>>> me
> > >>>>>> if I am wrong. In such a setup the cross bounded/unbounded
> operation
> > >>>> could
> > >>>>>> be expressed quite easily I think:
> > >>>>>>
> > >>>>>> DataStream {
> > >>>>>>   DataStream join(DataStream, ...); // we could not really tell if
> > >> the
> > >>>> result is bounded or not, but because bounded stream is a special
> case
> > >> of
> > >>>> unbounded the API object is correct, irrespective if the left or
> right
> > >>> side
> > >>>> of the join is bounded
> > >>>>>> }
> > >>>>>>
> > >>>>>> BoundedStream extends DataStream {
> > >>>>>>   BoundedStream join(BoundedStream, ...); // only if both sides
> are
> > >>>> bounded the result can be bounded as well. However we do have access
> > to
> > >>> the
> > >>>> DataStream#join here, so you can still join with a DataStream
> > >>>>>> }
> > >>>>>>
> > >>>>>>
> > >>>>>> On the other hand I also see benefits of two completely disjointed
> > >>> APIs,
> > >>>>>> as we could prohibit some streaming calls in the bounded API. I
> > >> can't
> > >>>> think
> > >>>>>> of any unbounded operators that could not be implemented for
> bounded
> > >>>> stream.
> > >>>>>>
> > >>>>>> Besides I think we both agree we don't like the method:
> > >>>>>>
> > >>>>>> DataStream boundedStream(Source)
> > >>>>>>
> > >>>>>> suggested in the current state of the FLIP. Do we ? :)
> > >>>>>>
> > >>>>>> Best,
> > >>>>>>
> > >>>>>> Dawid
> > >>>>>>
> > >>>>>> On 10/12/2019 18:57, Becket Qin wrote:
> > >>>>>>
> > >>>>>> Hi folks,
> > >>>>>>
> > >>>>>> Thanks for the discussion, great feedback. Also thanks Dawid for
> the
> > >>>>>> explanation, it is much clearer now.
> > >>>>>>
> > >>>>>> One thing that is indeed missing from the FLIP is how the
> > >> boundedness
> > >>> is
> > >>>>>> passed to the Source implementation. So the API should be
> > >>>>>> Source#createEnumerator(Boundedness boundedness,
> > >>> SplitEnumeratorContext
> > >>>>>> context)
> > >>>>>> And we can probably remove the
> Source#supportBoundedness(Boundedness
> > >>>>>> boundedness) method.
> > >>>>>>
> > >>>>>> Assuming we have that, we are essentially choosing from one of the
> > >>>>>> following two options:
> > >>>>>>
> > >>>>>> Option 1:
> > >>>>>> // The source is continuous source, and only unbounded operations
> > >> can
> > >>> be
> > >>>>>> performed.
> > >>>>>> DataStream<Type> datastream = env.continuousSource(someSource);
> > >>>>>>
> > >>>>>> // The source is bounded source, both bounded and unbounded
> > >> operations
> > >>>> can
> > >>>>>> be performed.
> > >>>>>> BoundedDataStream<Type> boundedDataStream =
> > >>>> env.boundedSource(someSource);
> > >>>>>>
> > >>>>>>   - Pros:
> > >>>>>>        a) explicit boundary between bounded / unbounded streams,
> it
> > >> is
> > >>>>>> quite simple and clear to the users.
> > >>>>>>   - Cons:
> > >>>>>>        a) For applications that do not involve bounded operations,
> > >> they
> > >>>>>> still have to call different API to distinguish bounded /
> unbounded
> > >>>> streams.
> > >>>>>>        b) No support for bounded stream to run in a streaming
> > runtime
> > >>>>>> setting, i.e. scheduling and operators behaviors.
> > >>>>>>
> > >>>>>>
> > >>>>>> Option 2:
> > >>>>>> // The source is either bounded or unbounded, but only unbounded
> > >>>> operations
> > >>>>>> could be performed on the returned DataStream.
> > >>>>>> DataStream<Type> dataStream = env.source(someSource);
> > >>>>>>
> > >>>>>> // The source must be a bounded source, otherwise exception is
> > >> thrown.
> > >>>>>> BoundedDataStream<Type> boundedDataStream =
> > >>>>>> env.boundedSource(boundedSource);
> > >>>>>>
> > >>>>>> The pros and cons are exactly the opposite of option 1.
> > >>>>>>   - Pros:
> > >>>>>>        a) For applications that do not involve bounded operations,
> > >> they
> > >>>>>> still have to call different API to distinguish bounded /
> unbounded
> > >>>> streams.
> > >>>>>>        b) Support for bounded stream to run in a streaming runtime
> > >>>> setting,
> > >>>>>> i.e. scheduling and operators behaviors.
> > >>>>>>   - Cons:
> > >>>>>>        a) Bounded / unbounded streams are kind of mixed, i.e.
> given
> > a
> > >>>>>> DataStream, it is not clear whether it is bounded or not, unless
> you
> > >>>> have
> > >>>>>> the access to its source.
> > >>>>>>
> > >>>>>>
> > >>>>>> If we only think from the Source API perspective, option 2 seems a
> > >>>> better
> > >>>>>> choice because functionality wise it is a superset of option 1, at
> > >> the
> > >>>> cost
> > >>>>>> of some seemingly acceptable ambiguity in the DataStream API.
> > >>>>>> But if we look at the DataStream API as a whole, option 1 seems a
> > >>>> clearer
> > >>>>>> choice. For example, some times a library may have to know
> whether a
> > >>>>>> certain task will finish or not. And it would be difficult to tell
> > >> if
> > >>>> the
> > >>>>>> input is a DataStream, unless additional information is provided
> all
> > >>> the
> > >>>>>> way from the Source. One possible solution is to have a *modified
> > >>>> option 2*
> > >>>>>> which adds a method to the DataStream API to indicate boundedness,
> > >>> such
> > >>>> as
> > >>>>>> getBoundedness(). It would solve the problem with a potential
> > >>> confusion
> > >>>> of
> > >>>>>> what is difference between a DataStream with getBoundedness()=true
> > >>> and a
> > >>>>>> BoundedDataStream. But that seems not super difficult to explain.
> > >>>>>>
> > >>>>>> So from API's perspective, I don't have a strong opinion between
> > >>>> *option 1*
> > >>>>>> and *modified option 2. *I like the cleanness of option 1, but
> > >>> modified
> > >>>>>> option 2 would be more attractive if we have concrete use case for
> > >> the
> > >>>>>> "Bounded stream with unbounded streaming runtime settings".
> > >>>>>>
> > >>>>>> Re: Till
> > >>>>>>
> > >>>>>>
> > >>>>>> Maybe this has already been asked before but I was wondering why
> the
> > >>>>>> SourceReader interface has the method pollNext which hands the
> > >>>>>> responsibility of outputting elements to the SourceReader
> > >>>> implementation?
> > >>>>>> Has this been done for backwards compatibility reasons with the
> old
> > >>>> source
> > >>>>>> interface? If not, then one could define a Collection<E>
> > >>>> getNextRecords()
> > >>>>>> method which returns the currently retrieved records and then the
> > >>> caller
> > >>>>>> emits them outside of the SourceReader. That way the interface
> would
> > >>> not
> > >>>>>> allow to implement an outputting loop where we never hand back
> > >> control
> > >>>> to
> > >>>>>> the caller. At the moment, this contract can be easily broken and
> is
> > >>>> only
> > >>>>>> mentioned loosely in the JavaDocs.
> > >>>>>>
> > >>>>>>
> > >>>>>> The primary reason we handover the SourceOutput to the
> SourceReader
> > >> is
> > >>>>>> because sometimes it is difficult for a SourceReader to emit one
> > >>> record
> > >>>> at
> > >>>>>> a time. One example is some batched messaging systems which only
> > >> have
> > >>> an
> > >>>>>> offset for the entire batch instead of individual messages in the
> > >>>> batch. In
> > >>>>>> that case, returning one record at a time would leave the
> > >> SourceReader
> > >>>> in
> > >>>>>> an uncheckpointable state because they can only checkpoint at the
> > >>> batch
> > >>>>>> boundaries.
> > >>>>>>
> > >>>>>> Thanks,
> > >>>>>>
> > >>>>>> Jiangjie (Becket) Qin
> > >>>>>>
> > >>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <
> trohrmann@apache.org
> > >>>> <ma...@apache.org>> <trohrmann@apache.org <mailto:
> > >>>> trohrmann@apache.org>> wrote:
> > >>>>>>
> > >>>>>>
> > >>>>>> Hi everyone,
> > >>>>>>
> > >>>>>> thanks for drafting this FLIP. It reads very well.
> > >>>>>>
> > >>>>>> Concerning Dawid's proposal, I tend to agree. The boundedness
> could
> > >>> come
> > >>>>>> from the source and tell the system how to treat the operator
> > >>>> (scheduling
> > >>>>>> wise). From a user's perspective it should be fine to get back a
> > >>>> DataStream
> > >>>>>> when calling env.source(boundedSource) if he does not need special
> > >>>>>> operations defined on a BoundedDataStream. If he needs this, then
> > >> one
> > >>>> could
> > >>>>>> use the method BoundedDataStream env.boundedSource(boundedSource).
> > >>>>>>
> > >>>>>> If possible, we could enforce the proper usage of
> > >> env.boundedSource()
> > >>> by
> > >>>>>> introducing a BoundedSource type so that one cannot pass an
> > >>>>>> unbounded source to it. That way users would not be able to shoot
> > >>>>>> themselves in the foot.
> > >>>>>>
> > >>>>>> Maybe this has already been asked before but I was wondering why
> the
> > >>>>>> SourceReader interface has the method pollNext which hands the
> > >>>>>> responsibility of outputting elements to the SourceReader
> > >>>> implementation?
> > >>>>>> Has this been done for backwards compatibility reasons with the
> old
> > >>>> source
> > >>>>>> interface? If not, then one could define a Collection<E>
> > >>>> getNextRecords()
> > >>>>>> method which returns the currently retrieved records and then the
> > >>> caller
> > >>>>>> emits them outside of the SourceReader. That way the interface
> would
> > >>> not
> > >>>>>> allow to implement an outputting loop where we never hand back
> > >> control
> > >>>> to
> > >>>>>> the caller. At the moment, this contract can be easily broken and
> is
> > >>>> only
> > >>>>>> mentioned loosely in the JavaDocs.
> > >>>>>>
> > >>>>>> Cheers,
> > >>>>>> Till
> > >>>>>>
> > >>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <
> jingsonglee0@gmail.com
> > >>>> <ma...@gmail.com>> <jingsonglee0@gmail.com <mailto:
> > >>>> jingsonglee0@gmail.com>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>
> > >>>>>> Hi all,
> > >>>>>>
> > >>>>>> I think current design is good.
> > >>>>>>
> > >>>>>> My understanding is:
> > >>>>>>
> > >>>>>> For execution mode: bounded mode and continuous mode, It's totally
> > >>>>>> different. I don't think we have the ability to integrate the two
> > >>> models
> > >>>>>>
> > >>>>>> at
> > >>>>>>
> > >>>>>> present. It's about scheduling, memory, algorithms, States, etc.
> we
> > >>>>>> shouldn't confuse them.
> > >>>>>>
> > >>>>>> For source capabilities: only bounded, only continuous, both
> bounded
> > >>> and
> > >>>>>> continuous.
> > >>>>>> I think Kafka is a source that can be ran both bounded
> > >>>>>> and continuous execution mode.
> > >>>>>> And Kafka with end offset should be ran both bounded
> > >>>>>> and continuous execution mode.  Using apache Beam with Flink
> > >> runner, I
> > >>>>>>
> > >>>>>> used
> > >>>>>>
> > >>>>>> to run a "bounded" Kafka in streaming mode. For our previous
> > >>> DataStream,
> > >>>>>>
> > >>>>>> it
> > >>>>>>
> > >>>>>> is not necessarily required that the source cannot be bounded.
> > >>>>>>
> > >>>>>> So it is my thought for Dawid's question:
> > >>>>>> 1.pass a bounded source to continuousSource() +1
> > >>>>>> 2.pass a continuous source to boundedSource() -1, should throw
> > >>>> exception.
> > >>>>>>
> > >>>>>> In StreamExecutionEnvironment, continuousSource and boundedSource
> > >>> define
> > >>>>>> the execution mode. It defines a clear boundary of execution mode.
> > >>>>>>
> > >>>>>> Best,
> > >>>>>> Jingsong Lee
> > >>>>>>
> > >>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com
> <mailto:
> > >>>> imjark@gmail.com>> <imjark@gmail.com <ma...@gmail.com>>
> > wrote:
> > >>>>>>
> > >>>>>>
> > >>>>>> I agree with Dawid's point that the boundedness information should
> > >>> come
> > >>>>>> from the source itself (e.g. the end timestamp), not through
> > >>>>>> env.boundedSouce()/continuousSource().
> > >>>>>> I think if we want to support something like `env.source()` that
> > >>> derive
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> execution mode from source, `supportsBoundedness(Boundedness)`
> > >>>>>> method is not enough, because we don't know whether it is bounded
> or
> > >>>>>>
> > >>>>>> not.
> > >>>>>>
> > >>>>>> Best,
> > >>>>>> Jark
> > >>>>>>
> > >>>>>>
> > >>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
> > >> dwysakowicz@apache.org
> > >>>> <ma...@apache.org>> <dwysakowicz@apache.org <mailto:
> > >>>> dwysakowicz@apache.org>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>
> > >>>>>> One more thing. In the current proposal, with the
> > >>>>>> supportsBoundedness(Boundedness) method and the boundedness coming
> > >>>>>>
> > >>>>>> from
> > >>>>>>
> > >>>>>> either continuousSource or boundedSource I could not find how this
> > >>>>>> information is fed back to the SplitEnumerator.
> > >>>>>>
> > >>>>>> Best,
> > >>>>>>
> > >>>>>> Dawid
> > >>>>>>
> > >>>>>> On 09/12/2019 13:52, Becket Qin wrote:
> > >>>>>>
> > >>>>>> Hi Dawid,
> > >>>>>>
> > >>>>>> Thanks for the comments. This actually brings another relevant
> > >>>>>>
> > >>>>>> question
> > >>>>>>
> > >>>>>> about what does a "bounded source" imply. I actually had the same
> > >>>>>> impression when I look at the Source API. Here is what I
> understand
> > >>>>>>
> > >>>>>> after
> > >>>>>>
> > >>>>>> some discussion with Stephan. The bounded source has the following
> > >>>>>>
> > >>>>>> impacts.
> > >>>>>>
> > >>>>>> 1. API validity.
> > >>>>>> - A bounded source generates a bounded stream so some operations
> > >>>>>>
> > >>>>>> that
> > >>>>>>
> > >>>>>> only
> > >>>>>>
> > >>>>>> works for bounded records would be performed, e.g. sort.
> > >>>>>> - To expose these bounded stream only APIs, there are two options:
> > >>>>>>      a. Add them to the DataStream API and throw exception if a
> > >>>>>>
> > >>>>>> method
> > >>>>>>
> > >>>>>> is
> > >>>>>>
> > >>>>>> called on an unbounded stream.
> > >>>>>>      b. Create a BoundedDataStream class which is returned from
> > >>>>>> env.boundedSource(), while DataStream is returned from
> > >>>>>>
> > >>>>>> env.continousSource().
> > >>>>>>
> > >>>>>> Note that this cannot be done by having single
> > >>>>>>
> > >>>>>> env.source(theSource)
> > >>>>>>
> > >>>>>> even
> > >>>>>>
> > >>>>>> the Source has a getBoundedness() method.
> > >>>>>>
> > >>>>>> 2. Scheduling
> > >>>>>> - A bounded source could be computed stage by stage without
> > >>>>>>
> > >>>>>> bringing
> > >>>>>>
> > >>>>>> up
> > >>>>>>
> > >>>>>> all
> > >>>>>>
> > >>>>>> the tasks at the same time.
> > >>>>>>
> > >>>>>> 3. Operator behaviors
> > >>>>>> - A bounded source indicates the records are finite so some
> > >>>>>>
> > >>>>>> operators
> > >>>>>>
> > >>>>>> can
> > >>>>>>
> > >>>>>> wait until it receives all the records before it starts the
> > >>>>>>
> > >>>>>> processing.
> > >>>>>>
> > >>>>>> In the above impact, only 1 is relevant to the API design. And the
> > >>>>>>
> > >>>>>> current
> > >>>>>>
> > >>>>>> proposal in FLIP-27 is following 1.b.
> > >>>>>>
> > >>>>>> // boundedness depends of source property, imo this should always
> > >>>>>>
> > >>>>>> be
> > >>>>>>
> > >>>>>> preferred
> > >>>>>>
> > >>>>>>
> > >>>>>> DataStream<MyType> stream = env.source(theSource);
> > >>>>>>
> > >>>>>>
> > >>>>>> In your proposal, does DataStream have bounded stream only
> methods?
> > >>>>>>
> > >>>>>> It
> > >>>>>>
> > >>>>>> looks it should have, otherwise passing a bounded Source to
> > >>>>>>
> > >>>>>> env.source()
> > >>>>>>
> > >>>>>> would be confusing. In that case, we will essentially do 1.a if an
> > >>>>>> unbounded Source is created from env.source(unboundedSource).
> > >>>>>>
> > >>>>>> If we have the methods only supported for bounded streams in
> > >>>>>>
> > >>>>>> DataStream,
> > >>>>>>
> > >>>>>> it
> > >>>>>>
> > >>>>>> seems a little weird to have a separate BoundedDataStream
> > >>>>>>
> > >>>>>> interface.
> > >>>>>>
> > >>>>>> Am I understand it correctly?
> > >>>>>>
> > >>>>>> Thanks,
> > >>>>>>
> > >>>>>> Jiangjie (Becket) Qin
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> > >>>>>>
> > >>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> > >>>>>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>
> > >>>>>> Hi all,
> > >>>>>>
> > >>>>>> Really well written proposal and very important one. I must admit
> > >>>>>>
> > >>>>>> I
> > >>>>>>
> > >>>>>> have
> > >>>>>>
> > >>>>>> not understood all the intricacies of it yet.
> > >>>>>>
> > >>>>>> One question I have though is about where does the information
> > >>>>>>
> > >>>>>> about
> > >>>>>>
> > >>>>>> boundedness come from. I think in most cases it is a property of
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> source. As you described it might be e.g. end offset, a flag
> > >>>>>>
> > >>>>>> should
> > >>>>>>
> > >>>>>> it
> > >>>>>>
> > >>>>>> monitor new splits etc. I think it would be a really nice use case
> > >>>>>>
> > >>>>>> to
> > >>>>>>
> > >>>>>> be
> > >>>>>>
> > >>>>>> able to say:
> > >>>>>>
> > >>>>>> new KafkaSource().readUntil(long timestamp),
> > >>>>>>
> > >>>>>> which could work as an "end offset". Moreover I think all Bounded
> > >>>>>>
> > >>>>>> sources
> > >>>>>>
> > >>>>>> support continuous mode, but no intrinsically continuous source
> > >>>>>>
> > >>>>>> support
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> Bounded mode. If I understood the proposal correctly it suggest
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> boundedness sort of "comes" from the outside of the source, from
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> invokation of either boundedStream or continousSource.
> > >>>>>>
> > >>>>>> I am wondering if it would make sense to actually change the
> > >>>>>>
> > >>>>>> method
> > >>>>>>
> > >>>>>> boolean Source#supportsBoundedness(Boundedness)
> > >>>>>>
> > >>>>>> to
> > >>>>>>
> > >>>>>> Boundedness Source#getBoundedness().
> > >>>>>>
> > >>>>>> As for the methods #boundedSource, #continousSource, assuming the
> > >>>>>> boundedness is property of the source they do not affect how the
> > >>>>>>
> > >>>>>> enumerator
> > >>>>>>
> > >>>>>> works, but mostly how the dag is scheduled, right? I am not
> > >>>>>>
> > >>>>>> against
> > >>>>>>
> > >>>>>> those
> > >>>>>>
> > >>>>>> methods, but I think it is a very specific use case to actually
> > >>>>>>
> > >>>>>> override
> > >>>>>>
> > >>>>>> the property of the source. In general I would expect users to
> > >>>>>>
> > >>>>>> only
> > >>>>>>
> > >>>>>> call
> > >>>>>>
> > >>>>>> env.source(theSource), where the source tells if it is bounded or
> > >>>>>>
> > >>>>>> not. I
> > >>>>>>
> > >>>>>> would suggest considering following set of methods:
> > >>>>>>
> > >>>>>> // boundedness depends of source property, imo this should always
> > >>>>>>
> > >>>>>> be
> > >>>>>>
> > >>>>>> preferred
> > >>>>>>
> > >>>>>> DataStream<MyType> stream = env.source(theSource);
> > >>>>>>
> > >>>>>>
> > >>>>>> // always continous execution, whether bounded or unbounded source
> > >>>>>>
> > >>>>>> DataStream<MyType> boundedStream = env.continousSource(theSource);
> > >>>>>>
> > >>>>>> // imo this would make sense if the BoundedDataStream provides
> > >>>>>>
> > >>>>>> additional features unavailable for continous mode
> > >>>>>>
> > >>>>>> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
> > >>>>>>
> > >>>>>>
> > >>>>>> Best,
> > >>>>>>
> > >>>>>> Dawid
> > >>>>>>
> > >>>>>>
> > >>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
> > >>>>>>
> > >>>>>> Thanks, Becket, for updating this.
> > >>>>>>
> > >>>>>> I agree with moving the aspects you mentioned into separate FLIPs
> > >>>>>>
> > >>>>>> -
> > >>>>>>
> > >>>>>> this
> > >>>>>>
> > >>>>>> one way becoming unwieldy in size.
> > >>>>>>
> > >>>>>> +1 to the FLIP in its current state. Its a very detailed write-up,
> > >>>>>>
> > >>>>>> nicely
> > >>>>>>
> > >>>>>> done!
> > >>>>>>
> > >>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <becket.qin@gmail.com
> > >>>> <ma...@gmail.com>> <becket.qin@gmail.com <mailto:
> > >>>> becket.qin@gmail.com>>
> > >>>>>>
> > >>>>>> <
> > >>>>>>
> > >>>>>> becket.qin@gmail.com <ma...@gmail.com>> wrote:
> > >>>>>>
> > >>>>>> Hi all,
> > >>>>>>
> > >>>>>> Sorry for the long belated update. I have updated FLIP-27 wiki
> > >>>>>>
> > >>>>>> page
> > >>>>>>
> > >>>>>> with
> > >>>>>>
> > >>>>>> the latest proposals. Some noticeable changes include:
> > >>>>>> 1. A new generic communication mechanism between SplitEnumerator
> > >>>>>>
> > >>>>>> and
> > >>>>>>
> > >>>>>> SourceReader.
> > >>>>>> 2. Some detail API method signature changes.
> > >>>>>>
> > >>>>>> We left a few things out of this FLIP and will address them in
> > >>>>>>
> > >>>>>> separate
> > >>>>>>
> > >>>>>> FLIPs. Including:
> > >>>>>> 1. Per split event time.
> > >>>>>> 2. Event time alignment.
> > >>>>>> 3. Fine grained failover for SplitEnumerator failure.
> > >>>>>>
> > >>>>>> Please let us know if you have any question.
> > >>>>>>
> > >>>>>> Thanks,
> > >>>>>>
> > >>>>>> Jiangjie (Becket) Qin
> > >>>>>>
> > >>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <sewen@apache.org
> > >>> <mailto:
> > >>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>> <
> > >>>>>>
> > >>>>>> sewen@apache.org <ma...@apache.org>> wrote:
> > >>>>>>
> > >>>>>> Hi  Łukasz!
> > >>>>>>
> > >>>>>> Becket and me are working hard on figuring out the last details
> > >>>>>>
> > >>>>>> and
> > >>>>>>
> > >>>>>> implementing the first PoC. We would update the FLIP hopefully
> > >>>>>>
> > >>>>>> next
> > >>>>>>
> > >>>>>> week.
> > >>>>>>
> > >>>>>> There is a fair chance that a first version of this will be in
> > >>>>>>
> > >>>>>> 1.10,
> > >>>>>>
> > >>>>>> but
> > >>>>>>
> > >>>>>> I
> > >>>>>>
> > >>>>>> think it will take another release to battle test it and migrate
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> connectors.
> > >>>>>>
> > >>>>>> Best,
> > >>>>>> Stephan
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <ljd@touk.pl
> > >>>> <ma...@touk.pl>
> > >>>>>>
> > >>>>>> <
> > >>>>>>
> > >>>>>> ljd@touk.pl <ma...@touk.pl>>
> > >>>>>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>> Hi,
> > >>>>>>
> > >>>>>> This proposal looks very promising for us. Do you have any plans
> > >>>>>>
> > >>>>>> in
> > >>>>>>
> > >>>>>> which
> > >>>>>>
> > >>>>>> Flink release it is going to be released? We are thinking on
> > >>>>>>
> > >>>>>> using a
> > >>>>>>
> > >>>>>> Data
> > >>>>>>
> > >>>>>> Set API for our future use cases but on the other hand Data Set
> > >>>>>>
> > >>>>>> API
> > >>>>>>
> > >>>>>> is
> > >>>>>>
> > >>>>>> going to be deprecated so using proposed bounded data streams
> > >>>>>>
> > >>>>>> solution
> > >>>>>>
> > >>>>>> could be more viable in the long term.
> > >>>>>>
> > >>>>>> Thanks,
> > >>>>>> Łukasz
> > >>>>>>
> > >>>>>> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com
> > >> <mailto:
> > >>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
> > >>>> thomas.weise@gmail.com>> <
> > >>>>>>
> > >>>>>> thomas.weise@gmail.com <ma...@gmail.com>> wrote:
> > >>>>>>
> > >>>>>> Thanks for putting together this proposal!
> > >>>>>>
> > >>>>>> I see that the "Per Split Event Time" and "Event Time Alignment"
> > >>>>>>
> > >>>>>> sections
> > >>>>>>
> > >>>>>> are still TBD.
> > >>>>>>
> > >>>>>> It would probably be good to flesh those out a bit before
> > >>>>>>
> > >>>>>> proceeding
> > >>>>>>
> > >>>>>> too
> > >>>>>>
> > >>>>>> far
> > >>>>>>
> > >>>>>> as the event time alignment will probably influence the
> > >>>>>>
> > >>>>>> interaction
> > >>>>>>
> > >>>>>> with
> > >>>>>>
> > >>>>>> the split reader, specifically ReaderStatus
> > >>>>>>
> > >>>>>> emitNext(SourceOutput<E>
> > >>>>>>
> > >>>>>> output).
> > >>>>>>
> > >>>>>> We currently have only one implementation for event time alignment
> > >>>>>>
> > >>>>>> in
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> Kinesis consumer. The synchronization in that case takes place as
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> last
> > >>>>>>
> > >>>>>> step before records are emitted downstream (RecordEmitter). With
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> currently proposed interfaces, the equivalent can be implemented
> > >>>>>>
> > >>>>>> in
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> reader loop, although note that in the Kinesis consumer the per
> > >>>>>>
> > >>>>>> shard
> > >>>>>>
> > >>>>>> threads push records.
> > >>>>>>
> > >>>>>> Synchronization has not been implemented for the Kafka consumer
> > >>>>>>
> > >>>>>> yet.
> > >>>>>>
> > >>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
> > >>>> https://issues.apache.org/jira/browse/FLINK-12675>
> > >>>>>>
> > >>>>>> When I looked at it, I realized that the implementation will look
> > >>>>>>
> > >>>>>> quite
> > >>>>>>
> > >>>>>> different
> > >>>>>> from Kinesis because it needs to take place in the pull part,
> > >>>>>>
> > >>>>>> where
> > >>>>>>
> > >>>>>> records
> > >>>>>>
> > >>>>>> are taken from the Kafka client. Due to the multiplexing it cannot
> > >>>>>>
> > >>>>>> be
> > >>>>>>
> > >>>>>> done
> > >>>>>>
> > >>>>>> by blocking the split thread like it currently works for Kinesis.
> > >>>>>>
> > >>>>>> Reading
> > >>>>>>
> > >>>>>> from individual Kafka partitions needs to be controlled via
> > >>>>>>
> > >>>>>> pause/resume
> > >>>>>>
> > >>>>>> on the Kafka client.
> > >>>>>>
> > >>>>>> To take on that responsibility the split thread would need to be
> > >>>>>>
> > >>>>>> aware
> > >>>>>>
> > >>>>>> of
> > >>>>>>
> > >>>>>> the
> > >>>>>> watermarks or at least whether it should or should not continue to
> > >>>>>>
> > >>>>>> consume
> > >>>>>>
> > >>>>>> a given split and this may require a different SourceReader or
> > >>>>>>
> > >>>>>> SourceOutput
> > >>>>>>
> > >>>>>> interface.
> > >>>>>>
> > >>>>>> Thanks,
> > >>>>>> Thomas
> > >>>>>>
> > >>>>>>
> > >>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com
> > >> <mailto:
> > >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:mmyy1110@gmail.com
> >>
> > >> <
> > >>>>>>
> > >>>>>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
> > >>>>>>
> > >>>>>> Hi Stephan,
> > >>>>>>
> > >>>>>> Thank you for feedback!
> > >>>>>> Will take a look at your branch before public discussing.
> > >>>>>>
> > >>>>>>
> > >>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <sewen@apache.org
> > >>>> <ma...@apache.org>> <sewen@apache.org <mailto:
> sewen@apache.org
> > >>>>
> > >>>>>>
> > >>>>>> <
> > >>>>>>
> > >>>>>> sewen@apache.org <ma...@apache.org>>
> > >>>>>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>> Hi Biao!
> > >>>>>>
> > >>>>>> Thanks for reviving this. I would like to join this discussion,
> > >>>>>>
> > >>>>>> but
> > >>>>>>
> > >>>>>> am
> > >>>>>>
> > >>>>>> quite occupied with the 1.9 release, so can we maybe pause this
> > >>>>>>
> > >>>>>> discussion
> > >>>>>>
> > >>>>>> for a week or so?
> > >>>>>>
> > >>>>>> In the meantime I can share some suggestion based on prior
> > >>>>>>
> > >>>>>> experiments:
> > >>>>>>
> > >>>>>> How to do watermarks / timestamp extractors in a simpler and more
> > >>>>>>
> > >>>>>> flexible
> > >>>>>>
> > >>>>>> way. I think that part is quite promising should be part of the
> > >>>>>>
> > >>>>>> new
> > >>>>>>
> > >>>>>> source
> > >>>>>>
> > >>>>>> interface.
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>
> > >>>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > >>>> <
> > >>>>
> > >>>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > >>>>>
> > >>>>>>
> > >>>>>>
> > >>>>
> > >>>
> > >>
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > >>>> <
> > >>>>
> > >>>
> > >>
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > >>>>>
> > >>>>>>
> > >>>>>> Some experiments on how to build the source reader and its
> > >>>>>>
> > >>>>>> library
> > >>>>>>
> > >>>>>> for
> > >>>>>>
> > >>>>>> common threading/split patterns:
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>
> > >>>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > >>>> <
> > >>>>
> > >>>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > >>>>>
> > >>>>>>
> > >>>>>> Best,
> > >>>>>> Stephan
> > >>>>>>
> > >>>>>>
> > >>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mmyy1110@gmail.com
> > >>> <mailto:
> > >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:mmyy1110@gmail.com
> >>
> > >> <
> > >>>>>>
> > >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >>>>>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>> Hi devs,
> > >>>>>>
> > >>>>>> Since 1.9 is nearly released, I think we could get back to
> > >>>>>>
> > >>>>>> FLIP-27.
> > >>>>>>
> > >>>>>> I
> > >>>>>>
> > >>>>>> believe it should be included in 1.10.
> > >>>>>>
> > >>>>>> There are so many things mentioned in document of FLIP-27. [1] I
> > >>>>>>
> > >>>>>> think
> > >>>>>>
> > >>>>>> we'd better discuss them separately. However the wiki is not a
> > >>>>>>
> > >>>>>> good
> > >>>>>>
> > >>>>>> place
> > >>>>>>
> > >>>>>> to discuss. I wrote google doc about SplitReader API which
> > >>>>>>
> > >>>>>> misses
> > >>>>>>
> > >>>>>> some
> > >>>>>>
> > >>>>>> details in the document. [2]
> > >>>>>>
> > >>>>>> 1.
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>
> > >>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > >>>> <
> > >>>>
> > >>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > >>>>>
> > >>>>>>
> > >>>>>> 2.
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>
> > >>>
> > >>
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > >>>> <
> > >>>>
> > >>>
> > >>
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > >>>>>
> > >>>>>>
> > >>>>>> CC Stephan, Aljoscha, Piotrek, Becket
> > >>>>>>
> > >>>>>>
> > >>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com
> > >> <mailto:
> > >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:mmyy1110@gmail.com
> >>
> > >> <
> > >>>>>>
> > >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >>>>>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>> Hi Steven,
> > >>>>>> Thank you for the feedback. Please take a look at the document
> > >>>>>>
> > >>>>>> FLIP-27
> > >>>>>>
> > >>>>>> <
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>
> > >>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >>>> <
> > >>>>
> > >>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >>>>>
> > >>>>>>
> > >>>>>> which
> > >>>>>>
> > >>>>>> is updated recently. A lot of details of enumerator were added
> > >>>>>>
> > >>>>>> in
> > >>>>>>
> > >>>>>> this
> > >>>>>>
> > >>>>>> document. I think it would help.
> > >>>>>>
> > >>>>>> Steven Wu <stevenz3wu@gmail.com <ma...@gmail.com>> <
> > >>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
> > >>> stevenz3wu@gmail.com
> > >>>> <ma...@gmail.com>> <stevenz3wu@gmail.com <mailto:
> > >>>> stevenz3wu@gmail.com>>
> > >>>>>>
> > >>>>>> 于2019年3月28日周四
> > >>>>>>
> > >>>>>> 下午12:52写道:
> > >>>>>>
> > >>>>>> This proposal mentioned that SplitEnumerator might run on the
> > >>>>>> JobManager or
> > >>>>>> in a single task on a TaskManager.
> > >>>>>>
> > >>>>>> if enumerator is a single task on a taskmanager, then the job
> > >>>>>>
> > >>>>>> DAG
> > >>>>>>
> > >>>>>> can
> > >>>>>>
> > >>>>>> never
> > >>>>>> been embarrassingly parallel anymore. That will nullify the
> > >>>>>>
> > >>>>>> leverage
> > >>>>>>
> > >>>>>> of
> > >>>>>>
> > >>>>>> fine-grained recovery for embarrassingly parallel jobs.
> > >>>>>>
> > >>>>>> It's not clear to me what's the implication of running
> > >>>>>>
> > >>>>>> enumerator
> > >>>>>>
> > >>>>>> on
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> jobmanager. So I will leave that out for now.
> > >>>>>>
> > >>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com
> > >> <mailto:
> > >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:mmyy1110@gmail.com
> >>
> > >> <
> > >>>>>>
> > >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >>>>>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>> Hi Stephan & Piotrek,
> > >>>>>>
> > >>>>>> Thank you for feedback.
> > >>>>>>
> > >>>>>> It seems that there are a lot of things to do in community.
> > >>>>>>
> > >>>>>> I
> > >>>>>>
> > >>>>>> am
> > >>>>>>
> > >>>>>> just
> > >>>>>>
> > >>>>>> afraid that this discussion may be forgotten since there so
> > >>>>>>
> > >>>>>> many
> > >>>>>>
> > >>>>>> proposals
> > >>>>>>
> > >>>>>> recently.
> > >>>>>> Anyway, wish to see the split topics soon :)
> > >>>>>>
> > >>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:
> piotr@da-platform.com
> > >>>>
> > >>> <
> > >>>> piotr@da-platform.com <ma...@da-platform.com>> <
> > >>>> piotr@da-platform.com <ma...@da-platform.com>> <
> > >>>> piotr@da-platform.com <ma...@da-platform.com>>
> > >>>>>>
> > >>>>>> 于2019年1月24日周四
> > >>>>>>
> > >>>>>> 下午8:21写道:
> > >>>>>>
> > >>>>>> Hi Biao!
> > >>>>>>
> > >>>>>> This discussion was stalled because of preparations for
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> open
> > >>>>>>
> > >>>>>> sourcing
> > >>>>>>
> > >>>>>> & merging Blink. I think before creating the tickets we
> > >>>>>>
> > >>>>>> should
> > >>>>>>
> > >>>>>> split this
> > >>>>>>
> > >>>>>> discussion into topics/areas outlined by Stephan and
> > >>>>>>
> > >>>>>> create
> > >>>>>>
> > >>>>>> Flips
> > >>>>>>
> > >>>>>> for
> > >>>>>>
> > >>>>>> that.
> > >>>>>>
> > >>>>>> I think there is no chance for this to be completed in
> > >>>>>>
> > >>>>>> couple
> > >>>>>>
> > >>>>>> of
> > >>>>>>
> > >>>>>> remaining
> > >>>>>>
> > >>>>>> weeks/1 month before 1.8 feature freeze, however it would
> > >>>>>>
> > >>>>>> be
> > >>>>>>
> > >>>>>> good
> > >>>>>>
> > >>>>>> to aim
> > >>>>>>
> > >>>>>> with those changes for 1.9.
> > >>>>>>
> > >>>>>> Piotrek
> > >>>>>>
> > >>>>>>
> > >>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com <mailto:
> > >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <mailto:mmyy1110@gmail.com
> >>
> > >> <
> > >>>>>>
> > >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> > >>>>>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>> Hi community,
> > >>>>>> The summary of Stephan makes a lot sense to me. It is
> > >>>>>>
> > >>>>>> much
> > >>>>>>
> > >>>>>> clearer
> > >>>>>>
> > >>>>>> indeed
> > >>>>>>
> > >>>>>> after splitting the complex topic into small ones.
> > >>>>>> I was wondering is there any detail plan for next step?
> > >>>>>>
> > >>>>>> If
> > >>>>>>
> > >>>>>> not,
> > >>>>>>
> > >>>>>> I
> > >>>>>>
> > >>>>>> would
> > >>>>>>
> > >>>>>> like to push this thing forward by creating some JIRA
> > >>>>>>
> > >>>>>> issues.
> > >>>>>>
> > >>>>>> Another question is that should version 1.8 include
> > >>>>>>
> > >>>>>> these
> > >>>>>>
> > >>>>>> features?
> > >>>>>>
> > >>>>>> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <
> > >>>> sewen@apache.org <ma...@apache.org>> <sewen@apache.org
> > <mailto:
> > >>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>>
> > >>>> 于2018年12月1日周六
> > >>>>>>
> > >>>>>> 上午4:20写道:
> > >>>>>>
> > >>>>>> Thanks everyone for the lively discussion. Let me try
> > >>>>>>
> > >>>>>> to
> > >>>>>>
> > >>>>>> summarize
> > >>>>>>
> > >>>>>> where I
> > >>>>>>
> > >>>>>> see convergence in the discussion and open issues.
> > >>>>>> I'll try to group this by design aspect of the source.
> > >>>>>>
> > >>>>>> Please
> > >>>>>>
> > >>>>>> let me
> > >>>>>>
> > >>>>>> know
> > >>>>>>
> > >>>>>> if I got things wrong or missed something crucial here.
> > >>>>>>
> > >>>>>> For issues 1-3, if the below reflects the state of the
> > >>>>>>
> > >>>>>> discussion, I
> > >>>>>>
> > >>>>>> would
> > >>>>>>
> > >>>>>> try and update the FLIP in the next days.
> > >>>>>> For the remaining ones we need more discussion.
> > >>>>>>
> > >>>>>> I would suggest to fork each of these aspects into a
> > >>>>>>
> > >>>>>> separate
> > >>>>>>
> > >>>>>> mail
> > >>>>>>
> > >>>>>> thread,
> > >>>>>>
> > >>>>>> or will loose sight of the individual aspects.
> > >>>>>>
> > >>>>>> *(1) Separation of Split Enumerator and Split Reader*
> > >>>>>>
> > >>>>>> - All seem to agree this is a good thing
> > >>>>>> - Split Enumerator could in the end live on JobManager
> > >>>>>>
> > >>>>>> (and
> > >>>>>>
> > >>>>>> assign
> > >>>>>>
> > >>>>>> splits
> > >>>>>>
> > >>>>>> via RPC) or in a task (and assign splits via data
> > >>>>>>
> > >>>>>> streams)
> > >>>>>>
> > >>>>>> - this discussion is orthogonal and should come later,
> > >>>>>>
> > >>>>>> when
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> interface
> > >>>>>>
> > >>>>>> is agreed upon.
> > >>>>>>
> > >>>>>> *(2) Split Readers for one or more splits*
> > >>>>>>
> > >>>>>> - Discussion seems to agree that we need to support
> > >>>>>>
> > >>>>>> one
> > >>>>>>
> > >>>>>> reader
> > >>>>>>
> > >>>>>> that
> > >>>>>>
> > >>>>>> possibly handles multiple splits concurrently.
> > >>>>>> - The requirement comes from sources where one
> > >>>>>>
> > >>>>>> poll()-style
> > >>>>>>
> > >>>>>> call
> > >>>>>>
> > >>>>>> fetches
> > >>>>>>
> > >>>>>> data from different splits / partitions
> > >>>>>>    --> example sources that require that would be for
> > >>>>>>
> > >>>>>> example
> > >>>>>>
> > >>>>>> Kafka,
> > >>>>>>
> > >>>>>> Pravega, Pulsar
> > >>>>>>
> > >>>>>> - Could have one split reader per source, or multiple
> > >>>>>>
> > >>>>>> split
> > >>>>>>
> > >>>>>> readers
> > >>>>>>
> > >>>>>> that
> > >>>>>>
> > >>>>>> share the "poll()" function
> > >>>>>> - To not make it too complicated, we can start with
> > >>>>>>
> > >>>>>> thinking
> > >>>>>>
> > >>>>>> about
> > >>>>>>
> > >>>>>> one
> > >>>>>>
> > >>>>>> split reader for all splits initially and see if that
> > >>>>>>
> > >>>>>> covers
> > >>>>>>
> > >>>>>> all
> > >>>>>>
> > >>>>>> requirements
> > >>>>>>
> > >>>>>> *(3) Threading model of the Split Reader*
> > >>>>>>
> > >>>>>> - Most active part of the discussion ;-)
> > >>>>>>
> > >>>>>> - A non-blocking way for Flink's task code to interact
> > >>>>>>
> > >>>>>> with
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> source
> > >>>>>>
> > >>>>>> is
> > >>>>>>
> > >>>>>> needed in order to a task runtime code based on a
> > >>>>>> single-threaded/actor-style task design
> > >>>>>>    --> I personally am a big proponent of that, it will
> > >>>>>>
> > >>>>>> help
> > >>>>>>
> > >>>>>> with
> > >>>>>>
> > >>>>>> well-behaved checkpoints, efficiency, and simpler yet
> > >>>>>>
> > >>>>>> more
> > >>>>>>
> > >>>>>> robust
> > >>>>>>
> > >>>>>> runtime
> > >>>>>>
> > >>>>>> code
> > >>>>>>
> > >>>>>> - Users care about simple abstraction, so as a
> > >>>>>>
> > >>>>>> subclass
> > >>>>>>
> > >>>>>> of
> > >>>>>>
> > >>>>>> SplitReader
> > >>>>>>
> > >>>>>> (non-blocking / async) we need to have a
> > >>>>>>
> > >>>>>> BlockingSplitReader
> > >>>>>>
> > >>>>>> which
> > >>>>>>
> > >>>>>> will
> > >>>>>>
> > >>>>>> form the basis of most source implementations.
> > >>>>>>
> > >>>>>> BlockingSplitReader
> > >>>>>>
> > >>>>>> lets
> > >>>>>>
> > >>>>>> users do blocking simple poll() calls.
> > >>>>>> - The BlockingSplitReader would spawn a thread (or
> > >>>>>>
> > >>>>>> more)
> > >>>>>>
> > >>>>>> and
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> thread(s) can make blocking calls and hand over data
> > >>>>>>
> > >>>>>> buffers
> > >>>>>>
> > >>>>>> via
> > >>>>>>
> > >>>>>> a
> > >>>>>>
> > >>>>>> blocking
> > >>>>>>
> > >>>>>> queue
> > >>>>>> - This should allow us to cover both, a fully async
> > >>>>>>
> > >>>>>> runtime,
> > >>>>>>
> > >>>>>> and a
> > >>>>>>
> > >>>>>> simple
> > >>>>>>
> > >>>>>> blocking interface for users.
> > >>>>>> - This is actually very similar to how the Kafka
> > >>>>>>
> > >>>>>> connectors
> > >>>>>>
> > >>>>>> work.
> > >>>>>>
> > >>>>>> Kafka
> > >>>>>>
> > >>>>>> 9+ with one thread, Kafka 8 with multiple threads
> > >>>>>>
> > >>>>>> - On the base SplitReader (the async one), the
> > >>>>>>
> > >>>>>> non-blocking
> > >>>>>>
> > >>>>>> method
> > >>>>>>
> > >>>>>> that
> > >>>>>>
> > >>>>>> gets the next chunk of data would signal data
> > >>>>>>
> > >>>>>> availability
> > >>>>>>
> > >>>>>> via
> > >>>>>>
> > >>>>>> a
> > >>>>>>
> > >>>>>> CompletableFuture, because that gives the best
> > >>>>>>
> > >>>>>> flexibility
> > >>>>>>
> > >>>>>> (can
> > >>>>>>
> > >>>>>> await
> > >>>>>>
> > >>>>>> completion or register notification handlers).
> > >>>>>> - The source task would register a "thenHandle()" (or
> > >>>>>>
> > >>>>>> similar)
> > >>>>>>
> > >>>>>> on the
> > >>>>>>
> > >>>>>> future to put a "take next data" task into the
> > >>>>>>
> > >>>>>> actor-style
> > >>>>>>
> > >>>>>> mailbox
> > >>>>>>
> > >>>>>> *(4) Split Enumeration and Assignment*
> > >>>>>>
> > >>>>>> - Splits may be generated lazily, both in cases where
> > >>>>>>
> > >>>>>> there
> > >>>>>>
> > >>>>>> is a
> > >>>>>>
> > >>>>>> limited
> > >>>>>>
> > >>>>>> number of splits (but very many), or splits are
> > >>>>>>
> > >>>>>> discovered
> > >>>>>>
> > >>>>>> over
> > >>>>>>
> > >>>>>> time
> > >>>>>>
> > >>>>>> - Assignment should also be lazy, to get better load
> > >>>>>>
> > >>>>>> balancing
> > >>>>>>
> > >>>>>> - Assignment needs support locality preferences
> > >>>>>>
> > >>>>>> - Possible design based on discussion so far:
> > >>>>>>
> > >>>>>>    --> SplitReader has a method "addSplits(SplitT...)"
> > >>>>>>
> > >>>>>> to
> > >>>>>>
> > >>>>>> add
> > >>>>>>
> > >>>>>> one or
> > >>>>>>
> > >>>>>> more
> > >>>>>>
> > >>>>>> splits. Some split readers might assume they have only
> > >>>>>>
> > >>>>>> one
> > >>>>>>
> > >>>>>> split
> > >>>>>>
> > >>>>>> ever,
> > >>>>>>
> > >>>>>> concurrently, others assume multiple splits. (Note:
> > >>>>>>
> > >>>>>> idea
> > >>>>>>
> > >>>>>> behind
> > >>>>>>
> > >>>>>> being
> > >>>>>>
> > >>>>>> able
> > >>>>>>
> > >>>>>> to add multiple splits at the same time is to ease
> > >>>>>>
> > >>>>>> startup
> > >>>>>>
> > >>>>>> where
> > >>>>>>
> > >>>>>> multiple
> > >>>>>>
> > >>>>>> splits may be assigned instantly.)
> > >>>>>>    --> SplitReader has a context object on which it can
> > >>>>>>
> > >>>>>> call
> > >>>>>>
> > >>>>>> indicate
> > >>>>>>
> > >>>>>> when
> > >>>>>>
> > >>>>>> splits are completed. The enumerator gets that
> > >>>>>>
> > >>>>>> notification and
> > >>>>>>
> > >>>>>> can
> > >>>>>>
> > >>>>>> use
> > >>>>>>
> > >>>>>> to
> > >>>>>>
> > >>>>>> decide when to assign new splits. This should help both
> > >>>>>>
> > >>>>>> in
> > >>>>>>
> > >>>>>> cases
> > >>>>>>
> > >>>>>> of
> > >>>>>>
> > >>>>>> sources
> > >>>>>>
> > >>>>>> that take splits lazily (file readers) and in case the
> > >>>>>>
> > >>>>>> source
> > >>>>>>
> > >>>>>> needs to
> > >>>>>>
> > >>>>>> preserve a partial order between splits (Kinesis,
> > >>>>>>
> > >>>>>> Pravega,
> > >>>>>>
> > >>>>>> Pulsar may
> > >>>>>>
> > >>>>>> need
> > >>>>>>
> > >>>>>> that).
> > >>>>>>    --> SplitEnumerator gets notification when
> > >>>>>>
> > >>>>>> SplitReaders
> > >>>>>>
> > >>>>>> start
> > >>>>>>
> > >>>>>> and
> > >>>>>>
> > >>>>>> when
> > >>>>>>
> > >>>>>> they finish splits. They can decide at that moment to
> > >>>>>>
> > >>>>>> push
> > >>>>>>
> > >>>>>> more
> > >>>>>>
> > >>>>>> splits
> > >>>>>>
> > >>>>>> to
> > >>>>>>
> > >>>>>> that reader
> > >>>>>>    --> The SplitEnumerator should probably be aware of
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> source
> > >>>>>>
> > >>>>>> parallelism, to build its initial distribution.
> > >>>>>>
> > >>>>>> - Open question: Should the source expose something
> > >>>>>>
> > >>>>>> like
> > >>>>>>
> > >>>>>> "host
> > >>>>>>
> > >>>>>> preferences", so that yarn/mesos/k8s can take this into
> > >>>>>>
> > >>>>>> account
> > >>>>>>
> > >>>>>> when
> > >>>>>>
> > >>>>>> selecting a node to start a TM on?
> > >>>>>>
> > >>>>>> *(5) Watermarks and event time alignment*
> > >>>>>>
> > >>>>>> - Watermark generation, as well as idleness, needs to
> > >>>>>>
> > >>>>>> be
> > >>>>>>
> > >>>>>> per
> > >>>>>>
> > >>>>>> split
> > >>>>>>
> > >>>>>> (like
> > >>>>>>
> > >>>>>> currently in the Kafka Source, per partition)
> > >>>>>> - It is desirable to support optional
> > >>>>>>
> > >>>>>> event-time-alignment,
> > >>>>>>
> > >>>>>> meaning
> > >>>>>>
> > >>>>>> that
> > >>>>>>
> > >>>>>> splits that are ahead are back-pressured or temporarily
> > >>>>>>
> > >>>>>> unsubscribed
> > >>>>>>
> > >>>>>> - I think i would be desirable to encapsulate
> > >>>>>>
> > >>>>>> watermark
> > >>>>>>
> > >>>>>> generation
> > >>>>>>
> > >>>>>> logic
> > >>>>>>
> > >>>>>> in watermark generators, for a separation of concerns.
> > >>>>>>
> > >>>>>> The
> > >>>>>>
> > >>>>>> watermark
> > >>>>>>
> > >>>>>> generators should run per split.
> > >>>>>> - Using watermark generators would also help with
> > >>>>>>
> > >>>>>> another
> > >>>>>>
> > >>>>>> problem of
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> suggested interface, namely supporting non-periodic
> > >>>>>>
> > >>>>>> watermarks
> > >>>>>>
> > >>>>>> efficiently.
> > >>>>>>
> > >>>>>> - Need a way to "dispatch" next record to different
> > >>>>>>
> > >>>>>> watermark
> > >>>>>>
> > >>>>>> generators
> > >>>>>>
> > >>>>>> - Need a way to tell SplitReader to "suspend" a split
> > >>>>>>
> > >>>>>> until a
> > >>>>>>
> > >>>>>> certain
> > >>>>>>
> > >>>>>> watermark is reached (event time backpressure)
> > >>>>>> - This would in fact be not needed (and thus simpler)
> > >>>>>>
> > >>>>>> if
> > >>>>>>
> > >>>>>> we
> > >>>>>>
> > >>>>>> had
> > >>>>>>
> > >>>>>> a
> > >>>>>>
> > >>>>>> SplitReader per split and may be a reason to re-open
> > >>>>>>
> > >>>>>> that
> > >>>>>>
> > >>>>>> discussion
> > >>>>>>
> > >>>>>> *(6) Watermarks across splits and in the Split
> > >>>>>>
> > >>>>>> Enumerator*
> > >>>>>>
> > >>>>>> - The split enumerator may need some watermark
> > >>>>>>
> > >>>>>> awareness,
> > >>>>>>
> > >>>>>> which
> > >>>>>>
> > >>>>>> should
> > >>>>>>
> > >>>>>> be
> > >>>>>>
> > >>>>>> purely based on split metadata (like create timestamp
> > >>>>>>
> > >>>>>> of
> > >>>>>>
> > >>>>>> file
> > >>>>>>
> > >>>>>> splits)
> > >>>>>>
> > >>>>>> - If there are still more splits with overlapping
> > >>>>>>
> > >>>>>> event
> > >>>>>>
> > >>>>>> time
> > >>>>>>
> > >>>>>> range
> > >>>>>>
> > >>>>>> for
> > >>>>>>
> > >>>>>> a
> > >>>>>>
> > >>>>>> split reader, then that split reader should not advance
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> watermark
> > >>>>>>
> > >>>>>> within the split beyond the overlap boundary. Otherwise
> > >>>>>>
> > >>>>>> future
> > >>>>>>
> > >>>>>> splits
> > >>>>>>
> > >>>>>> will
> > >>>>>>
> > >>>>>> produce late data.
> > >>>>>>
> > >>>>>> - One way to approach this could be that the split
> > >>>>>>
> > >>>>>> enumerator
> > >>>>>>
> > >>>>>> may
> > >>>>>>
> > >>>>>> send
> > >>>>>>
> > >>>>>> watermarks to the readers, and the readers cannot emit
> > >>>>>>
> > >>>>>> watermarks
> > >>>>>>
> > >>>>>> beyond
> > >>>>>>
> > >>>>>> that received watermark.
> > >>>>>> - Many split enumerators would simply immediately send
> > >>>>>>
> > >>>>>> Long.MAX
> > >>>>>>
> > >>>>>> out
> > >>>>>>
> > >>>>>> and
> > >>>>>>
> > >>>>>> leave the progress purely to the split readers.
> > >>>>>>
> > >>>>>> - For event-time alignment / split back pressure, this
> > >>>>>>
> > >>>>>> begs
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> question
> > >>>>>>
> > >>>>>> how we can avoid deadlocks that may arise when splits
> > >>>>>>
> > >>>>>> are
> > >>>>>>
> > >>>>>> suspended
> > >>>>>>
> > >>>>>> for
> > >>>>>>
> > >>>>>> event time back pressure,
> > >>>>>>
> > >>>>>> *(7) Batch and streaming Unification*
> > >>>>>>
> > >>>>>> - Functionality wise, the above design should support
> > >>>>>>
> > >>>>>> both
> > >>>>>>
> > >>>>>> - Batch often (mostly) does not care about reading "in
> > >>>>>>
> > >>>>>> order"
> > >>>>>>
> > >>>>>> and
> > >>>>>>
> > >>>>>> generating watermarks
> > >>>>>>    --> Might use different enumerator logic that is
> > >>>>>>
> > >>>>>> more
> > >>>>>>
> > >>>>>> locality
> > >>>>>>
> > >>>>>> aware
> > >>>>>>
> > >>>>>> and ignores event time order
> > >>>>>>    --> Does not generate watermarks
> > >>>>>> - Would be great if bounded sources could be
> > >>>>>>
> > >>>>>> identified
> > >>>>>>
> > >>>>>> at
> > >>>>>>
> > >>>>>> compile
> > >>>>>>
> > >>>>>> time,
> > >>>>>>
> > >>>>>> so that "env.addBoundedSource(...)" is type safe and
> > >>>>>>
> > >>>>>> can
> > >>>>>>
> > >>>>>> return a
> > >>>>>>
> > >>>>>> "BoundedDataStream".
> > >>>>>> - Possible to defer this discussion until later
> > >>>>>>
> > >>>>>> *Miscellaneous Comments*
> > >>>>>>
> > >>>>>> - Should the source have a TypeInformation for the
> > >>>>>>
> > >>>>>> produced
> > >>>>>>
> > >>>>>> type,
> > >>>>>>
> > >>>>>> instead
> > >>>>>>
> > >>>>>> of a serializer? We need a type information in the
> > >>>>>>
> > >>>>>> stream
> > >>>>>>
> > >>>>>> anyways, and
> > >>>>>>
> > >>>>>> can
> > >>>>>>
> > >>>>>> derive the serializer from that. Plus, creating the
> > >>>>>>
> > >>>>>> serializer
> > >>>>>>
> > >>>>>> should
> > >>>>>>
> > >>>>>> respect the ExecutionConfig.
> > >>>>>>
> > >>>>>> - The TypeSerializer interface is very powerful but
> > >>>>>>
> > >>>>>> also
> > >>>>>>
> > >>>>>> not
> > >>>>>>
> > >>>>>> easy to
> > >>>>>>
> > >>>>>> implement. Its purpose is to handle data super
> > >>>>>>
> > >>>>>> efficiently,
> > >>>>>>
> > >>>>>> support
> > >>>>>>
> > >>>>>> flexible ways of evolution, etc.
> > >>>>>> For metadata I would suggest to look at the
> > >>>>>>
> > >>>>>> SimpleVersionedSerializer
> > >>>>>>
> > >>>>>> instead, which is used for example for checkpoint
> > >>>>>>
> > >>>>>> master
> > >>>>>>
> > >>>>>> hooks,
> > >>>>>>
> > >>>>>> or for
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> streaming file sink. I think that is is a good match
> > >>>>>>
> > >>>>>> for
> > >>>>>>
> > >>>>>> cases
> > >>>>>>
> > >>>>>> where
> > >>>>>>
> > >>>>>> we
> > >>>>>>
> > >>>>>> do
> > >>>>>>
> > >>>>>> not need more than ser/deser (no copy, etc.) and don't
> > >>>>>>
> > >>>>>> need to
> > >>>>>>
> > >>>>>> push
> > >>>>>>
> > >>>>>> versioning out of the serialization paths for best
> > >>>>>>
> > >>>>>> performance
> > >>>>>>
> > >>>>>> (as in
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> TypeSerializer)
> > >>>>>>
> > >>>>>>
> > >>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > >>>>>>
> > >>>>>> k.kloudas@data-artisans.com>
> > >>>>>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>
> > >>>>>> Hi Biao,
> > >>>>>>
> > >>>>>> Thanks for the answer!
> > >>>>>>
> > >>>>>> So given the multi-threaded readers, now we have as
> > >>>>>>
> > >>>>>> open
> > >>>>>>
> > >>>>>> questions:
> > >>>>>>
> > >>>>>> 1) How do we let the checkpoints pass through our
> > >>>>>>
> > >>>>>> multi-threaded
> > >>>>>>
> > >>>>>> reader
> > >>>>>>
> > >>>>>> operator?
> > >>>>>>
> > >>>>>> 2) Do we have separate reader and source operators or
> > >>>>>>
> > >>>>>> not? In
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> strategy
> > >>>>>>
> > >>>>>> that has a separate source, the source operator has a
> > >>>>>>
> > >>>>>> parallelism of
> > >>>>>>
> > >>>>>> 1
> > >>>>>>
> > >>>>>> and
> > >>>>>>
> > >>>>>> is responsible for split recovery only.
> > >>>>>>
> > >>>>>> For the first one, given also the constraints
> > >>>>>>
> > >>>>>> (blocking,
> > >>>>>>
> > >>>>>> finite
> > >>>>>>
> > >>>>>> queues,
> > >>>>>>
> > >>>>>> etc), I do not have an answer yet.
> > >>>>>>
> > >>>>>> For the 2nd, I think that we should go with separate
> > >>>>>>
> > >>>>>> operators
> > >>>>>>
> > >>>>>> for
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> source and the readers, for the following reasons:
> > >>>>>>
> > >>>>>> 1) This is more aligned with a potential future
> > >>>>>>
> > >>>>>> improvement
> > >>>>>>
> > >>>>>> where the
> > >>>>>>
> > >>>>>> split
> > >>>>>>
> > >>>>>> discovery becomes a responsibility of the JobManager
> > >>>>>>
> > >>>>>> and
> > >>>>>>
> > >>>>>> readers are
> > >>>>>>
> > >>>>>> pooling more work from the JM.
> > >>>>>>
> > >>>>>> 2) The source is going to be the "single point of
> > >>>>>>
> > >>>>>> truth".
> > >>>>>>
> > >>>>>> It
> > >>>>>>
> > >>>>>> will
> > >>>>>>
> > >>>>>> know
> > >>>>>>
> > >>>>>> what
> > >>>>>>
> > >>>>>> has been processed and what not. If the source and the
> > >>>>>>
> > >>>>>> readers
> > >>>>>>
> > >>>>>> are a
> > >>>>>>
> > >>>>>> single
> > >>>>>>
> > >>>>>> operator with parallelism > 1, or in general, if the
> > >>>>>>
> > >>>>>> split
> > >>>>>>
> > >>>>>> discovery
> > >>>>>>
> > >>>>>> is
> > >>>>>>
> > >>>>>> done by each task individually, then:
> > >>>>>>   i) we have to have a deterministic scheme for each
> > >>>>>>
> > >>>>>> reader to
> > >>>>>>
> > >>>>>> assign
> > >>>>>>
> > >>>>>> splits to itself (e.g. mod subtaskId). This is not
> > >>>>>>
> > >>>>>> necessarily
> > >>>>>>
> > >>>>>> trivial
> > >>>>>>
> > >>>>>> for
> > >>>>>>
> > >>>>>> all sources.
> > >>>>>>   ii) each reader would have to keep a copy of all its
> > >>>>>>
> > >>>>>> processed
> > >>>>>>
> > >>>>>> slpits
> > >>>>>>
> > >>>>>>   iii) the state has to be a union state with a
> > >>>>>>
> > >>>>>> non-trivial
> > >>>>>>
> > >>>>>> merging
> > >>>>>>
> > >>>>>> logic
> > >>>>>>
> > >>>>>> in order to support rescaling.
> > >>>>>>
> > >>>>>> Two additional points that you raised above:
> > >>>>>>
> > >>>>>> i) The point that you raised that we need to keep all
> > >>>>>>
> > >>>>>> splits
> > >>>>>>
> > >>>>>> (processed
> > >>>>>>
> > >>>>>> and
> > >>>>>>
> > >>>>>> not-processed) I think is a bit of a strong
> > >>>>>>
> > >>>>>> requirement.
> > >>>>>>
> > >>>>>> This
> > >>>>>>
> > >>>>>> would
> > >>>>>>
> > >>>>>> imply
> > >>>>>>
> > >>>>>> that for infinite sources the state will grow
> > >>>>>>
> > >>>>>> indefinitely.
> > >>>>>>
> > >>>>>> This is
> > >>>>>>
> > >>>>>> problem
> > >>>>>>
> > >>>>>> is even more pronounced if we do not have a single
> > >>>>>>
> > >>>>>> source
> > >>>>>>
> > >>>>>> that
> > >>>>>>
> > >>>>>> assigns
> > >>>>>>
> > >>>>>> splits to readers, as each reader will have its own
> > >>>>>>
> > >>>>>> copy
> > >>>>>>
> > >>>>>> of
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> state.
> > >>>>>>
> > >>>>>> ii) it is true that for finite sources we need to
> > >>>>>>
> > >>>>>> somehow
> > >>>>>>
> > >>>>>> not
> > >>>>>>
> > >>>>>> close
> > >>>>>>
> > >>>>>> the
> > >>>>>>
> > >>>>>> readers when the source/split discoverer finishes. The
> > >>>>>> ContinuousFileReaderOperator has a work-around for
> > >>>>>>
> > >>>>>> that.
> > >>>>>>
> > >>>>>> It is
> > >>>>>>
> > >>>>>> not
> > >>>>>>
> > >>>>>> elegant,
> > >>>>>>
> > >>>>>> and checkpoints are not emitted after closing the
> > >>>>>>
> > >>>>>> source,
> > >>>>>>
> > >>>>>> but
> > >>>>>>
> > >>>>>> this, I
> > >>>>>>
> > >>>>>> believe, is a bigger problem which requires more
> > >>>>>>
> > >>>>>> changes
> > >>>>>>
> > >>>>>> than
> > >>>>>>
> > >>>>>> just
> > >>>>>>
> > >>>>>> refactoring the source interface.
> > >>>>>>
> > >>>>>> Cheers,
> > >>>>>> Kostas
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> --
> > >>>>>> Best, Jingsong Lee
> > >>>>
> > >>>>
> > >>>
> > >>
> > >>
> > >> --
> > >> Best, Jingsong Lee
> > >>
> > >
> >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi folks,

Thanks for the comments. I am convinced that the Source API should not take
boundedness as a parameter after it is constructed. What Timo and Dawid
suggested sounds a reasonable solution to me. So the Source API would
become:

Source {
    Boundedness getBoundedness();
}

Assuming the above Source API, in addition to the two options mentioned in
earlier emails, I am thinking of another option:

*Option 3:*
// MySource must be unbounded, otherwise throws exception.
DataStream<Type> dataStream = env.source(mySource);

// MySource must be bounded, otherwise throws exception.
BoundedDataStream<Type> boundedDataStream = env.boundedSource(mySource);

The pros of this API are:
   a) It fits the requirements from Table / SQL well.
   b) DataStream users still have type safety (option 2 only has partial
type safety).
   c) Cristal clear boundedness from the API which makes DataStream join /
connect easy to reason about.
The caveats I see,
   a) It is inconsistent with Table since Table has one unified interface.
   b) No streaming mode for bounded source.

@Stephan Ewen <ew...@gmail.com> @Aljoscha Krettek
<al...@ververica.com> what do you think of the approach?


Orthogonal to the above API, I am wondering whether boundedness is the only
dimension needed to describe the characteristic of the Source behavior. We
may also need to have another dimension of *record order*.

For example, when a file source is reading from a directory with bounded
records, it may have two ways to read.
1. Read files in parallel.
2. Read files in the chronological order.
In both cases, the file source is a Bounded Source. However, the processing
requirement for downstream may be different. In the first case, the
record processing and result emitting order does not matter, e.g. word
count. In the second case, the records may have to be processed in the
order they were read, e.g. change log processing.

If the Source only has a getBoundedness() method, the downstream processors
would not know whether the records emitted from the Source should be
processed in order or not. So combining the boundedness and record order,
we will have four scenarios:

*Bounded-StrictOrder*:     A segment of change log.
*Bounded-Random*:          Batch Word Count.
*Unbounded-StrictOrder*: An infinite change log.
*Unbounded-Random*:     Streaming Word Count.

Option 2 mentioned in the previous email was kind of trying to handle the
Bounded-StrictOrder case by creating a DataStream from a bounded source,
which actually does not work.
It looks that we do not have strict order support in some operators at this
point, e.g. join. But we may still want to add the semantic to the Source
first so later on we don't need to change all the source implementations,
especially given that many of them will be implemented by 3rd party.

Given that, we need another dimension of *Record Order* in the Source. More
specifically, the API would become:

Source {
    Boundedness getBoundedness();
    RecordOrder getRecordOrder();
}

public enum RecordOrder {
    /** The record in the DataStream must be processed in its strict order
for correctness. */
    STRICT,
    /** The record in the DataStream can be processed in arbitrary order. */
    RANDOM;
}

Any thoughts?

Thanks,

Jiangjie (Becket) Qin

On Tue, Dec 17, 2019 at 3:44 PM Timo Walther <tw...@apache.org> wrote:

> Hi Becket,
>
> I completely agree with Dawid's suggestion. The information about the
> boundedness should come out of the source. Because most of the streaming
> sources can be made bounded based on some connector specific criterion.
> In Kafka, it would be an end offset or end timestamp but in any case
> having just a env.boundedSource() is not enough because parameters for
> making the source bounded are missing.
>
> I suggest to have a simple `isBounded(): Boolean` flag in every source
> that might be influenced by a connector builder as Dawid mentioned.
>
> For type safety during programming, we can still go with *Final state
> 1*. By having a env.source() vs env.boundedSource(). The latter would
> just enforce that the boolean flag is set to `true` and could make
> bounded operations available (if we need that actually).
>
> However, I don't think that we should start making a unified Table API
> ununified again. Boundedness is an optimization property. Every bounded
> operation can also executed in an unbounded way using updates/retraction
> or watermarks.
>
> Regards,
> Timo
>
>
> On 15.12.19 14:22, Becket Qin wrote:
> > Hi Dawid and Jark,
> >
> > I think the discussion ultimately boils down to the question that which
> one
> > of the following two final states do we want? Once we make this decision,
> > everything else can be naturally derived.
> >
> > *Final state 1*: Separate API for bounded / unbounded DataStream & Table.
> > That means any code users write will be valid at the point when they
> write
> > the code. This is similar to having type safety check at programming
> time.
> > For example,
> >
> > BoundedDataStream extends DataStream {
> > // Operations only available for bounded data.
> > BoundedDataStream sort(...);
> >
> > // Interaction with another BoundedStream returns a Bounded stream.
> > BoundedJoinedDataStream join(BoundedDataStream other)
> >
> > // Interaction with another unbounded stream returns an unbounded stream.
> > JoinedDataStream join(DataStream other)
> > }
> >
> > BoundedTable extends Table {
> >    // Bounded only operation.
> > BoundedTable sort(...);
> >
> > // Interaction with another BoundedTable returns a BoundedTable.
> > BoundedTable join(BoundedTable other)
> >
> > // Interaction with another unbounded table returns an unbounded table.
> > Table join(Table other)
> > }
> >
> > *Final state 2*: One unified API for bounded / unbounded DataStream /
> > Table.
> > That unified API may throw exception at DAG compilation time if an
> invalid
> > operation is tried. This is what Table API currently follows.
> >
> > DataStream {
> > // Throws exception if the DataStream is unbounded.
> > DataStream sort();
> > // Get boundedness.
> > Boundedness getBoundedness();
> > }
> >
> > Table {
> > // Throws exception if the table has infinite rows.
> > Table orderBy();
> >
> > // Get boundedness.
> > Boundedness getBoundedness();
> > }
> >
> >>From what I understand, there is no consensus so far on this decision
> yet.
> > Whichever final state we choose, we need to make it consistent across the
> > entire project. We should avoid the case that Table follows one final
> state
> > while DataStream follows another. Some arguments I am aware of from both
> > sides so far are following:
> >
> > Arguments for final state 1:
> > 1a) Clean API with method safety check at programming time.
> > 1b) (Counter 2b) Although SQL does not have programming time error
> check, SQL
> > is not really a "programming language" per se. So SQL can be different
> from
> > Table and DataStream.
> > 1c)  Although final state 2 seems making it easier for SQL to use given
> it
> > is more "config based" than "parameter based", final state 1 can probably
> > also meet what SQL wants by wrapping the Source in TableSource /
> > TableSourceFactory API if needed.
> >
> > Arguments for final state 2:
> > 2a) The Source API itself seems already sort of following the unified API
> > pattern.
> > 2b) There is no "programming time" method error check in SQL case, so we
> > cannot really achieve final state 1 across the board.
> > 2c) It is an easier path given our current status, i.e. Table is already
> > following final state 2.
> > 2d) Users can always explicitly check the boundedness if they want to.
> >
> > As I mentioned earlier, my initial thought was also to have a
> > "configuration based" Source rather than a "parameter based" Source. So
> it
> > is completely possible that I missed some important consideration or
> design
> > principles that we want to enforce for the project. It would be good
> > if @Stephan
> > Ewen <st...@ververica.com> and @Aljoscha Krettek <
> aljoscha@ververica.com> can
> > also provide more thoughts on this.
> >
> >
> > Re: Jingsong
> >
> > As you said, there are some batched system source, like parquet/orc
> source.
> >> Could we have the batch emit interface to improve performance? The
> queue of
> >> per record may cause performance degradation.
> >
> >
> > The current interface does not necessarily cause performance problem in a
> > multi-threading case. In fact, the base implementation allows
> SplitReaders
> > to add a batch <E> of records<T> to the records queue<E>, so each element
> > in the records queue would be a batch <E>. In this case, when the main
> > thread polls records, it will take a batch <E> of records <T> from the
> > shared records queue and process the records <T> in a batch manner.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <ji...@gmail.com>
> wrote:
> >
> >> Hi Becket,
> >>
> >> I also have some performance concerns too.
> >>
> >> If I understand correctly, SourceOutput will emit data per record into
> the
> >> queue? I'm worried about the multithreading performance of this queue.
> >>
> >>> One example is some batched messaging systems which only have an offset
> >> for the entire batch instead of individual messages in the batch.
> >>
> >> As you said, there are some batched system source, like parquet/orc
> source.
> >> Could we have the batch emit interface to improve performance? The
> queue of
> >> per record may cause performance degradation.
> >>
> >> Best,
> >> Jingsong Lee
> >>
> >> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com> wrote:
> >>
> >>> Hi Becket,
> >>>
> >>> I think Dawid explained things clearly and makes a lot of sense.
> >>> I'm also in favor of #2, because #1 doesn't work for our future unified
> >>> envrionment.
> >>>
> >>> You can see the vision in this documentation [1]. In the future, we
> would
> >>> like to
> >>> drop the global streaming/batch mode in SQL (i.e.
> >>> EnvironmentSettings#inStreamingMode/inBatchMode).
> >>> A source is bounded or unbounded once defined, so queries can be
> inferred
> >>> from source to run
> >>> in streaming or batch or hybrid mode. However, in #1, we will lose this
> >>> ability because the framework
> >>> doesn't know whether the source is bounded or unbounded.
> >>>
> >>> Best,
> >>> Jark
> >>>
> >>>
> >>> [1]:
> >>>
> >>>
> >>
> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
> >>>
> >>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <pi...@ververica.com>
> >> wrote:
> >>>
> >>>> Hi,
> >>>>
> >>>> Regarding the:
> >>>>
> >>>> Collection<E> getNextRecords()
> >>>>
> >>>> I’m pretty sure such design would unfortunately impact the performance
> >>>> (accessing and potentially creating the collection on the hot path).
> >>>>
> >>>> Also the
> >>>>
> >>>> InputStatus emitNext(DataOutput<T> output) throws Exception;
> >>>> or
> >>>> Status pollNext(SourceOutput<T> sourceOutput) throws Exception;
> >>>>
> >>>> Gives us some opportunities in the future, to allow Source hot looping
> >>>> inside, until it receives some signal “please exit because of some
> >>> reasons”
> >>>> (output collector could return such hint upon collecting the result).
> >> But
> >>>> that’s another topic outside of this FLIP’s scope.
> >>>>
> >>>> Piotrek
> >>>>
> >>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <tr...@apache.org>
> >> wrote:
> >>>>>
> >>>>> Hi Becket,
> >>>>>
> >>>>> quick clarification from my side because I think you misunderstood my
> >>>>> question. I did not suggest to let the SourceReader return only a
> >>> single
> >>>>> record at a time when calling getNextRecords. As the return type
> >>>> indicates,
> >>>>> the method can return an arbitrary number of records.
> >>>>>
> >>>>> Cheers,
> >>>>> Till
> >>>>>
> >>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> >>>> dwysakowicz@apache.org <ma...@apache.org>>
> >>>>> wrote:
> >>>>>
> >>>>>> Hi Becket,
> >>>>>>
> >>>>>> Issue #1 - Design of Source interface
> >>>>>>
> >>>>>> I mentioned the lack of a method like
> >>>> Source#createEnumerator(Boundedness
> >>>>>> boundedness, SplitEnumeratorContext context), because without the
> >>>> current
> >>>>>> proposal is not complete/does not work.
> >>>>>>
> >>>>>> If we say that boundedness is an intrinsic property of a source imo
> >> we
> >>>>>> don't need the Source#createEnumerator(Boundedness boundedness,
> >>>>>> SplitEnumeratorContext context) method.
> >>>>>>
> >>>>>> Assuming a source from my previous example:
> >>>>>>
> >>>>>> Source source = KafkaSource.builder()
> >>>>>>   ...
> >>>>>>   .untilTimestamp(...)
> >>>>>>   .build()
> >>>>>>
> >>>>>> Would the enumerator differ if created like
> >>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
> >>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating myself, but
> >>> this
> >>>> is
> >>>>>> the part that my opinion differ the most from the current proposal.
> >> I
> >>>>>> really think it should always be the source that tells if it is
> >>> bounded
> >>>> or
> >>>>>> not. In the current proposal methods continousSource/boundedSource
> >>>> somewhat
> >>>>>> reconfigure the source, which I think is misleading.
> >>>>>>
> >>>>>> I think a call like:
> >>>>>>
> >>>>>> Source source = KafkaSource.builder()
> >>>>>>   ...
> >>>>>>   .readContinously() / readUntilLatestOffset() / readUntilTimestamp
> /
> >>>> readUntilOffsets / ...
> >>>>>>   .build()
> >>>>>>
> >>>>>> is way cleaner (and expressive) than
> >>>>>>
> >>>>>> Source source = KafkaSource.builder()
> >>>>>>   ...
> >>>>>>   .build()
> >>>>>>
> >>>>>>
> >>>>>> env.continousSource(source) // which actually underneath would call
> >>>> createEnumerator(CONTINUOUS, ctx) which would be equivalent to
> >>>> source.readContinously().createEnumerator(ctx)
> >>>>>> // or
> >>>>>> env.boundedSource(source) // which actually underneath would call
> >>>> createEnumerator(BOUNDED, ctx) which would be equivalent to
> >>>> source.readUntilLatestOffset().createEnumerator(ctx)
> >>>>>>
> >>>>>>
> >>>>>> Sorry for the comparison, but to me it seems there is too much magic
> >>>>>> happening underneath those two calls.
> >>>>>>
> >>>>>> I really believe the Source interface should have getBoundedness
> >>> method
> >>>>>> instead of (supportBoundedness) + createEnumerator(Boundedness, ...)
> >>>>>>
> >>>>>>
> >>>>>> Issue #2 - Design of
> >>>>>> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> >>>>>>
> >>>>>> As you might have guessed I am slightly in favor of option #2
> >>> modified.
> >>>>>> Yes I am aware every step of the dag would have to be able to say if
> >>> it
> >>>> is
> >>>>>> bounded or not. I have a feeling it would be easier to express cross
> >>>>>> bounded/unbounded operations, but I must admit I have not thought it
> >>>>>> through thoroughly, In the spirit of batch is just a special case of
> >>>>>> streaming I thought BoundedStream would extend from DataStream.
> >>> Correct
> >>>> me
> >>>>>> if I am wrong. In such a setup the cross bounded/unbounded operation
> >>>> could
> >>>>>> be expressed quite easily I think:
> >>>>>>
> >>>>>> DataStream {
> >>>>>>   DataStream join(DataStream, ...); // we could not really tell if
> >> the
> >>>> result is bounded or not, but because bounded stream is a special case
> >> of
> >>>> unbounded the API object is correct, irrespective if the left or right
> >>> side
> >>>> of the join is bounded
> >>>>>> }
> >>>>>>
> >>>>>> BoundedStream extends DataStream {
> >>>>>>   BoundedStream join(BoundedStream, ...); // only if both sides are
> >>>> bounded the result can be bounded as well. However we do have access
> to
> >>> the
> >>>> DataStream#join here, so you can still join with a DataStream
> >>>>>> }
> >>>>>>
> >>>>>>
> >>>>>> On the other hand I also see benefits of two completely disjointed
> >>> APIs,
> >>>>>> as we could prohibit some streaming calls in the bounded API. I
> >> can't
> >>>> think
> >>>>>> of any unbounded operators that could not be implemented for bounded
> >>>> stream.
> >>>>>>
> >>>>>> Besides I think we both agree we don't like the method:
> >>>>>>
> >>>>>> DataStream boundedStream(Source)
> >>>>>>
> >>>>>> suggested in the current state of the FLIP. Do we ? :)
> >>>>>>
> >>>>>> Best,
> >>>>>>
> >>>>>> Dawid
> >>>>>>
> >>>>>> On 10/12/2019 18:57, Becket Qin wrote:
> >>>>>>
> >>>>>> Hi folks,
> >>>>>>
> >>>>>> Thanks for the discussion, great feedback. Also thanks Dawid for the
> >>>>>> explanation, it is much clearer now.
> >>>>>>
> >>>>>> One thing that is indeed missing from the FLIP is how the
> >> boundedness
> >>> is
> >>>>>> passed to the Source implementation. So the API should be
> >>>>>> Source#createEnumerator(Boundedness boundedness,
> >>> SplitEnumeratorContext
> >>>>>> context)
> >>>>>> And we can probably remove the Source#supportBoundedness(Boundedness
> >>>>>> boundedness) method.
> >>>>>>
> >>>>>> Assuming we have that, we are essentially choosing from one of the
> >>>>>> following two options:
> >>>>>>
> >>>>>> Option 1:
> >>>>>> // The source is continuous source, and only unbounded operations
> >> can
> >>> be
> >>>>>> performed.
> >>>>>> DataStream<Type> datastream = env.continuousSource(someSource);
> >>>>>>
> >>>>>> // The source is bounded source, both bounded and unbounded
> >> operations
> >>>> can
> >>>>>> be performed.
> >>>>>> BoundedDataStream<Type> boundedDataStream =
> >>>> env.boundedSource(someSource);
> >>>>>>
> >>>>>>   - Pros:
> >>>>>>        a) explicit boundary between bounded / unbounded streams, it
> >> is
> >>>>>> quite simple and clear to the users.
> >>>>>>   - Cons:
> >>>>>>        a) For applications that do not involve bounded operations,
> >> they
> >>>>>> still have to call different API to distinguish bounded / unbounded
> >>>> streams.
> >>>>>>        b) No support for bounded stream to run in a streaming
> runtime
> >>>>>> setting, i.e. scheduling and operators behaviors.
> >>>>>>
> >>>>>>
> >>>>>> Option 2:
> >>>>>> // The source is either bounded or unbounded, but only unbounded
> >>>> operations
> >>>>>> could be performed on the returned DataStream.
> >>>>>> DataStream<Type> dataStream = env.source(someSource);
> >>>>>>
> >>>>>> // The source must be a bounded source, otherwise exception is
> >> thrown.
> >>>>>> BoundedDataStream<Type> boundedDataStream =
> >>>>>> env.boundedSource(boundedSource);
> >>>>>>
> >>>>>> The pros and cons are exactly the opposite of option 1.
> >>>>>>   - Pros:
> >>>>>>        a) For applications that do not involve bounded operations,
> >> they
> >>>>>> still have to call different API to distinguish bounded / unbounded
> >>>> streams.
> >>>>>>        b) Support for bounded stream to run in a streaming runtime
> >>>> setting,
> >>>>>> i.e. scheduling and operators behaviors.
> >>>>>>   - Cons:
> >>>>>>        a) Bounded / unbounded streams are kind of mixed, i.e. given
> a
> >>>>>> DataStream, it is not clear whether it is bounded or not, unless you
> >>>> have
> >>>>>> the access to its source.
> >>>>>>
> >>>>>>
> >>>>>> If we only think from the Source API perspective, option 2 seems a
> >>>> better
> >>>>>> choice because functionality wise it is a superset of option 1, at
> >> the
> >>>> cost
> >>>>>> of some seemingly acceptable ambiguity in the DataStream API.
> >>>>>> But if we look at the DataStream API as a whole, option 1 seems a
> >>>> clearer
> >>>>>> choice. For example, some times a library may have to know whether a
> >>>>>> certain task will finish or not. And it would be difficult to tell
> >> if
> >>>> the
> >>>>>> input is a DataStream, unless additional information is provided all
> >>> the
> >>>>>> way from the Source. One possible solution is to have a *modified
> >>>> option 2*
> >>>>>> which adds a method to the DataStream API to indicate boundedness,
> >>> such
> >>>> as
> >>>>>> getBoundedness(). It would solve the problem with a potential
> >>> confusion
> >>>> of
> >>>>>> what is difference between a DataStream with getBoundedness()=true
> >>> and a
> >>>>>> BoundedDataStream. But that seems not super difficult to explain.
> >>>>>>
> >>>>>> So from API's perspective, I don't have a strong opinion between
> >>>> *option 1*
> >>>>>> and *modified option 2. *I like the cleanness of option 1, but
> >>> modified
> >>>>>> option 2 would be more attractive if we have concrete use case for
> >> the
> >>>>>> "Bounded stream with unbounded streaming runtime settings".
> >>>>>>
> >>>>>> Re: Till
> >>>>>>
> >>>>>>
> >>>>>> Maybe this has already been asked before but I was wondering why the
> >>>>>> SourceReader interface has the method pollNext which hands the
> >>>>>> responsibility of outputting elements to the SourceReader
> >>>> implementation?
> >>>>>> Has this been done for backwards compatibility reasons with the old
> >>>> source
> >>>>>> interface? If not, then one could define a Collection<E>
> >>>> getNextRecords()
> >>>>>> method which returns the currently retrieved records and then the
> >>> caller
> >>>>>> emits them outside of the SourceReader. That way the interface would
> >>> not
> >>>>>> allow to implement an outputting loop where we never hand back
> >> control
> >>>> to
> >>>>>> the caller. At the moment, this contract can be easily broken and is
> >>>> only
> >>>>>> mentioned loosely in the JavaDocs.
> >>>>>>
> >>>>>>
> >>>>>> The primary reason we handover the SourceOutput to the SourceReader
> >> is
> >>>>>> because sometimes it is difficult for a SourceReader to emit one
> >>> record
> >>>> at
> >>>>>> a time. One example is some batched messaging systems which only
> >> have
> >>> an
> >>>>>> offset for the entire batch instead of individual messages in the
> >>>> batch. In
> >>>>>> that case, returning one record at a time would leave the
> >> SourceReader
> >>>> in
> >>>>>> an uncheckpointable state because they can only checkpoint at the
> >>> batch
> >>>>>> boundaries.
> >>>>>>
> >>>>>> Thanks,
> >>>>>>
> >>>>>> Jiangjie (Becket) Qin
> >>>>>>
> >>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <trohrmann@apache.org
> >>>> <ma...@apache.org>> <trohrmann@apache.org <mailto:
> >>>> trohrmann@apache.org>> wrote:
> >>>>>>
> >>>>>>
> >>>>>> Hi everyone,
> >>>>>>
> >>>>>> thanks for drafting this FLIP. It reads very well.
> >>>>>>
> >>>>>> Concerning Dawid's proposal, I tend to agree. The boundedness could
> >>> come
> >>>>>> from the source and tell the system how to treat the operator
> >>>> (scheduling
> >>>>>> wise). From a user's perspective it should be fine to get back a
> >>>> DataStream
> >>>>>> when calling env.source(boundedSource) if he does not need special
> >>>>>> operations defined on a BoundedDataStream. If he needs this, then
> >> one
> >>>> could
> >>>>>> use the method BoundedDataStream env.boundedSource(boundedSource).
> >>>>>>
> >>>>>> If possible, we could enforce the proper usage of
> >> env.boundedSource()
> >>> by
> >>>>>> introducing a BoundedSource type so that one cannot pass an
> >>>>>> unbounded source to it. That way users would not be able to shoot
> >>>>>> themselves in the foot.
> >>>>>>
> >>>>>> Maybe this has already been asked before but I was wondering why the
> >>>>>> SourceReader interface has the method pollNext which hands the
> >>>>>> responsibility of outputting elements to the SourceReader
> >>>> implementation?
> >>>>>> Has this been done for backwards compatibility reasons with the old
> >>>> source
> >>>>>> interface? If not, then one could define a Collection<E>
> >>>> getNextRecords()
> >>>>>> method which returns the currently retrieved records and then the
> >>> caller
> >>>>>> emits them outside of the SourceReader. That way the interface would
> >>> not
> >>>>>> allow to implement an outputting loop where we never hand back
> >> control
> >>>> to
> >>>>>> the caller. At the moment, this contract can be easily broken and is
> >>>> only
> >>>>>> mentioned loosely in the JavaDocs.
> >>>>>>
> >>>>>> Cheers,
> >>>>>> Till
> >>>>>>
> >>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <jingsonglee0@gmail.com
> >>>> <ma...@gmail.com>> <jingsonglee0@gmail.com <mailto:
> >>>> jingsonglee0@gmail.com>>
> >>>>>> wrote:
> >>>>>>
> >>>>>>
> >>>>>> Hi all,
> >>>>>>
> >>>>>> I think current design is good.
> >>>>>>
> >>>>>> My understanding is:
> >>>>>>
> >>>>>> For execution mode: bounded mode and continuous mode, It's totally
> >>>>>> different. I don't think we have the ability to integrate the two
> >>> models
> >>>>>>
> >>>>>> at
> >>>>>>
> >>>>>> present. It's about scheduling, memory, algorithms, States, etc. we
> >>>>>> shouldn't confuse them.
> >>>>>>
> >>>>>> For source capabilities: only bounded, only continuous, both bounded
> >>> and
> >>>>>> continuous.
> >>>>>> I think Kafka is a source that can be ran both bounded
> >>>>>> and continuous execution mode.
> >>>>>> And Kafka with end offset should be ran both bounded
> >>>>>> and continuous execution mode.  Using apache Beam with Flink
> >> runner, I
> >>>>>>
> >>>>>> used
> >>>>>>
> >>>>>> to run a "bounded" Kafka in streaming mode. For our previous
> >>> DataStream,
> >>>>>>
> >>>>>> it
> >>>>>>
> >>>>>> is not necessarily required that the source cannot be bounded.
> >>>>>>
> >>>>>> So it is my thought for Dawid's question:
> >>>>>> 1.pass a bounded source to continuousSource() +1
> >>>>>> 2.pass a continuous source to boundedSource() -1, should throw
> >>>> exception.
> >>>>>>
> >>>>>> In StreamExecutionEnvironment, continuousSource and boundedSource
> >>> define
> >>>>>> the execution mode. It defines a clear boundary of execution mode.
> >>>>>>
> >>>>>> Best,
> >>>>>> Jingsong Lee
> >>>>>>
> >>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com <mailto:
> >>>> imjark@gmail.com>> <imjark@gmail.com <ma...@gmail.com>>
> wrote:
> >>>>>>
> >>>>>>
> >>>>>> I agree with Dawid's point that the boundedness information should
> >>> come
> >>>>>> from the source itself (e.g. the end timestamp), not through
> >>>>>> env.boundedSouce()/continuousSource().
> >>>>>> I think if we want to support something like `env.source()` that
> >>> derive
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> execution mode from source, `supportsBoundedness(Boundedness)`
> >>>>>> method is not enough, because we don't know whether it is bounded or
> >>>>>>
> >>>>>> not.
> >>>>>>
> >>>>>> Best,
> >>>>>> Jark
> >>>>>>
> >>>>>>
> >>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
> >> dwysakowicz@apache.org
> >>>> <ma...@apache.org>> <dwysakowicz@apache.org <mailto:
> >>>> dwysakowicz@apache.org>>
> >>>>>> wrote:
> >>>>>>
> >>>>>>
> >>>>>> One more thing. In the current proposal, with the
> >>>>>> supportsBoundedness(Boundedness) method and the boundedness coming
> >>>>>>
> >>>>>> from
> >>>>>>
> >>>>>> either continuousSource or boundedSource I could not find how this
> >>>>>> information is fed back to the SplitEnumerator.
> >>>>>>
> >>>>>> Best,
> >>>>>>
> >>>>>> Dawid
> >>>>>>
> >>>>>> On 09/12/2019 13:52, Becket Qin wrote:
> >>>>>>
> >>>>>> Hi Dawid,
> >>>>>>
> >>>>>> Thanks for the comments. This actually brings another relevant
> >>>>>>
> >>>>>> question
> >>>>>>
> >>>>>> about what does a "bounded source" imply. I actually had the same
> >>>>>> impression when I look at the Source API. Here is what I understand
> >>>>>>
> >>>>>> after
> >>>>>>
> >>>>>> some discussion with Stephan. The bounded source has the following
> >>>>>>
> >>>>>> impacts.
> >>>>>>
> >>>>>> 1. API validity.
> >>>>>> - A bounded source generates a bounded stream so some operations
> >>>>>>
> >>>>>> that
> >>>>>>
> >>>>>> only
> >>>>>>
> >>>>>> works for bounded records would be performed, e.g. sort.
> >>>>>> - To expose these bounded stream only APIs, there are two options:
> >>>>>>      a. Add them to the DataStream API and throw exception if a
> >>>>>>
> >>>>>> method
> >>>>>>
> >>>>>> is
> >>>>>>
> >>>>>> called on an unbounded stream.
> >>>>>>      b. Create a BoundedDataStream class which is returned from
> >>>>>> env.boundedSource(), while DataStream is returned from
> >>>>>>
> >>>>>> env.continousSource().
> >>>>>>
> >>>>>> Note that this cannot be done by having single
> >>>>>>
> >>>>>> env.source(theSource)
> >>>>>>
> >>>>>> even
> >>>>>>
> >>>>>> the Source has a getBoundedness() method.
> >>>>>>
> >>>>>> 2. Scheduling
> >>>>>> - A bounded source could be computed stage by stage without
> >>>>>>
> >>>>>> bringing
> >>>>>>
> >>>>>> up
> >>>>>>
> >>>>>> all
> >>>>>>
> >>>>>> the tasks at the same time.
> >>>>>>
> >>>>>> 3. Operator behaviors
> >>>>>> - A bounded source indicates the records are finite so some
> >>>>>>
> >>>>>> operators
> >>>>>>
> >>>>>> can
> >>>>>>
> >>>>>> wait until it receives all the records before it starts the
> >>>>>>
> >>>>>> processing.
> >>>>>>
> >>>>>> In the above impact, only 1 is relevant to the API design. And the
> >>>>>>
> >>>>>> current
> >>>>>>
> >>>>>> proposal in FLIP-27 is following 1.b.
> >>>>>>
> >>>>>> // boundedness depends of source property, imo this should always
> >>>>>>
> >>>>>> be
> >>>>>>
> >>>>>> preferred
> >>>>>>
> >>>>>>
> >>>>>> DataStream<MyType> stream = env.source(theSource);
> >>>>>>
> >>>>>>
> >>>>>> In your proposal, does DataStream have bounded stream only methods?
> >>>>>>
> >>>>>> It
> >>>>>>
> >>>>>> looks it should have, otherwise passing a bounded Source to
> >>>>>>
> >>>>>> env.source()
> >>>>>>
> >>>>>> would be confusing. In that case, we will essentially do 1.a if an
> >>>>>> unbounded Source is created from env.source(unboundedSource).
> >>>>>>
> >>>>>> If we have the methods only supported for bounded streams in
> >>>>>>
> >>>>>> DataStream,
> >>>>>>
> >>>>>> it
> >>>>>>
> >>>>>> seems a little weird to have a separate BoundedDataStream
> >>>>>>
> >>>>>> interface.
> >>>>>>
> >>>>>> Am I understand it correctly?
> >>>>>>
> >>>>>> Thanks,
> >>>>>>
> >>>>>> Jiangjie (Becket) Qin
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> >>>>>>
> >>>>>> dwysakowicz@apache.org <ma...@apache.org>>
> >>>>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>>
> >>>>>> Hi all,
> >>>>>>
> >>>>>> Really well written proposal and very important one. I must admit
> >>>>>>
> >>>>>> I
> >>>>>>
> >>>>>> have
> >>>>>>
> >>>>>> not understood all the intricacies of it yet.
> >>>>>>
> >>>>>> One question I have though is about where does the information
> >>>>>>
> >>>>>> about
> >>>>>>
> >>>>>> boundedness come from. I think in most cases it is a property of
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> source. As you described it might be e.g. end offset, a flag
> >>>>>>
> >>>>>> should
> >>>>>>
> >>>>>> it
> >>>>>>
> >>>>>> monitor new splits etc. I think it would be a really nice use case
> >>>>>>
> >>>>>> to
> >>>>>>
> >>>>>> be
> >>>>>>
> >>>>>> able to say:
> >>>>>>
> >>>>>> new KafkaSource().readUntil(long timestamp),
> >>>>>>
> >>>>>> which could work as an "end offset". Moreover I think all Bounded
> >>>>>>
> >>>>>> sources
> >>>>>>
> >>>>>> support continuous mode, but no intrinsically continuous source
> >>>>>>
> >>>>>> support
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> Bounded mode. If I understood the proposal correctly it suggest
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> boundedness sort of "comes" from the outside of the source, from
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> invokation of either boundedStream or continousSource.
> >>>>>>
> >>>>>> I am wondering if it would make sense to actually change the
> >>>>>>
> >>>>>> method
> >>>>>>
> >>>>>> boolean Source#supportsBoundedness(Boundedness)
> >>>>>>
> >>>>>> to
> >>>>>>
> >>>>>> Boundedness Source#getBoundedness().
> >>>>>>
> >>>>>> As for the methods #boundedSource, #continousSource, assuming the
> >>>>>> boundedness is property of the source they do not affect how the
> >>>>>>
> >>>>>> enumerator
> >>>>>>
> >>>>>> works, but mostly how the dag is scheduled, right? I am not
> >>>>>>
> >>>>>> against
> >>>>>>
> >>>>>> those
> >>>>>>
> >>>>>> methods, but I think it is a very specific use case to actually
> >>>>>>
> >>>>>> override
> >>>>>>
> >>>>>> the property of the source. In general I would expect users to
> >>>>>>
> >>>>>> only
> >>>>>>
> >>>>>> call
> >>>>>>
> >>>>>> env.source(theSource), where the source tells if it is bounded or
> >>>>>>
> >>>>>> not. I
> >>>>>>
> >>>>>> would suggest considering following set of methods:
> >>>>>>
> >>>>>> // boundedness depends of source property, imo this should always
> >>>>>>
> >>>>>> be
> >>>>>>
> >>>>>> preferred
> >>>>>>
> >>>>>> DataStream<MyType> stream = env.source(theSource);
> >>>>>>
> >>>>>>
> >>>>>> // always continous execution, whether bounded or unbounded source
> >>>>>>
> >>>>>> DataStream<MyType> boundedStream = env.continousSource(theSource);
> >>>>>>
> >>>>>> // imo this would make sense if the BoundedDataStream provides
> >>>>>>
> >>>>>> additional features unavailable for continous mode
> >>>>>>
> >>>>>> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
> >>>>>>
> >>>>>>
> >>>>>> Best,
> >>>>>>
> >>>>>> Dawid
> >>>>>>
> >>>>>>
> >>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
> >>>>>>
> >>>>>> Thanks, Becket, for updating this.
> >>>>>>
> >>>>>> I agree with moving the aspects you mentioned into separate FLIPs
> >>>>>>
> >>>>>> -
> >>>>>>
> >>>>>> this
> >>>>>>
> >>>>>> one way becoming unwieldy in size.
> >>>>>>
> >>>>>> +1 to the FLIP in its current state. Its a very detailed write-up,
> >>>>>>
> >>>>>> nicely
> >>>>>>
> >>>>>> done!
> >>>>>>
> >>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <becket.qin@gmail.com
> >>>> <ma...@gmail.com>> <becket.qin@gmail.com <mailto:
> >>>> becket.qin@gmail.com>>
> >>>>>>
> >>>>>> <
> >>>>>>
> >>>>>> becket.qin@gmail.com <ma...@gmail.com>> wrote:
> >>>>>>
> >>>>>> Hi all,
> >>>>>>
> >>>>>> Sorry for the long belated update. I have updated FLIP-27 wiki
> >>>>>>
> >>>>>> page
> >>>>>>
> >>>>>> with
> >>>>>>
> >>>>>> the latest proposals. Some noticeable changes include:
> >>>>>> 1. A new generic communication mechanism between SplitEnumerator
> >>>>>>
> >>>>>> and
> >>>>>>
> >>>>>> SourceReader.
> >>>>>> 2. Some detail API method signature changes.
> >>>>>>
> >>>>>> We left a few things out of this FLIP and will address them in
> >>>>>>
> >>>>>> separate
> >>>>>>
> >>>>>> FLIPs. Including:
> >>>>>> 1. Per split event time.
> >>>>>> 2. Event time alignment.
> >>>>>> 3. Fine grained failover for SplitEnumerator failure.
> >>>>>>
> >>>>>> Please let us know if you have any question.
> >>>>>>
> >>>>>> Thanks,
> >>>>>>
> >>>>>> Jiangjie (Becket) Qin
> >>>>>>
> >>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <sewen@apache.org
> >>> <mailto:
> >>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>> <
> >>>>>>
> >>>>>> sewen@apache.org <ma...@apache.org>> wrote:
> >>>>>>
> >>>>>> Hi  Łukasz!
> >>>>>>
> >>>>>> Becket and me are working hard on figuring out the last details
> >>>>>>
> >>>>>> and
> >>>>>>
> >>>>>> implementing the first PoC. We would update the FLIP hopefully
> >>>>>>
> >>>>>> next
> >>>>>>
> >>>>>> week.
> >>>>>>
> >>>>>> There is a fair chance that a first version of this will be in
> >>>>>>
> >>>>>> 1.10,
> >>>>>>
> >>>>>> but
> >>>>>>
> >>>>>> I
> >>>>>>
> >>>>>> think it will take another release to battle test it and migrate
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> connectors.
> >>>>>>
> >>>>>> Best,
> >>>>>> Stephan
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <ljd@touk.pl
> >>>> <ma...@touk.pl>
> >>>>>>
> >>>>>> <
> >>>>>>
> >>>>>> ljd@touk.pl <ma...@touk.pl>>
> >>>>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>> Hi,
> >>>>>>
> >>>>>> This proposal looks very promising for us. Do you have any plans
> >>>>>>
> >>>>>> in
> >>>>>>
> >>>>>> which
> >>>>>>
> >>>>>> Flink release it is going to be released? We are thinking on
> >>>>>>
> >>>>>> using a
> >>>>>>
> >>>>>> Data
> >>>>>>
> >>>>>> Set API for our future use cases but on the other hand Data Set
> >>>>>>
> >>>>>> API
> >>>>>>
> >>>>>> is
> >>>>>>
> >>>>>> going to be deprecated so using proposed bounded data streams
> >>>>>>
> >>>>>> solution
> >>>>>>
> >>>>>> could be more viable in the long term.
> >>>>>>
> >>>>>> Thanks,
> >>>>>> Łukasz
> >>>>>>
> >>>>>> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com
> >> <mailto:
> >>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
> >>>> thomas.weise@gmail.com>> <
> >>>>>>
> >>>>>> thomas.weise@gmail.com <ma...@gmail.com>> wrote:
> >>>>>>
> >>>>>> Thanks for putting together this proposal!
> >>>>>>
> >>>>>> I see that the "Per Split Event Time" and "Event Time Alignment"
> >>>>>>
> >>>>>> sections
> >>>>>>
> >>>>>> are still TBD.
> >>>>>>
> >>>>>> It would probably be good to flesh those out a bit before
> >>>>>>
> >>>>>> proceeding
> >>>>>>
> >>>>>> too
> >>>>>>
> >>>>>> far
> >>>>>>
> >>>>>> as the event time alignment will probably influence the
> >>>>>>
> >>>>>> interaction
> >>>>>>
> >>>>>> with
> >>>>>>
> >>>>>> the split reader, specifically ReaderStatus
> >>>>>>
> >>>>>> emitNext(SourceOutput<E>
> >>>>>>
> >>>>>> output).
> >>>>>>
> >>>>>> We currently have only one implementation for event time alignment
> >>>>>>
> >>>>>> in
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> Kinesis consumer. The synchronization in that case takes place as
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> last
> >>>>>>
> >>>>>> step before records are emitted downstream (RecordEmitter). With
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> currently proposed interfaces, the equivalent can be implemented
> >>>>>>
> >>>>>> in
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> reader loop, although note that in the Kinesis consumer the per
> >>>>>>
> >>>>>> shard
> >>>>>>
> >>>>>> threads push records.
> >>>>>>
> >>>>>> Synchronization has not been implemented for the Kafka consumer
> >>>>>>
> >>>>>> yet.
> >>>>>>
> >>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
> >>>> https://issues.apache.org/jira/browse/FLINK-12675>
> >>>>>>
> >>>>>> When I looked at it, I realized that the implementation will look
> >>>>>>
> >>>>>> quite
> >>>>>>
> >>>>>> different
> >>>>>> from Kinesis because it needs to take place in the pull part,
> >>>>>>
> >>>>>> where
> >>>>>>
> >>>>>> records
> >>>>>>
> >>>>>> are taken from the Kafka client. Due to the multiplexing it cannot
> >>>>>>
> >>>>>> be
> >>>>>>
> >>>>>> done
> >>>>>>
> >>>>>> by blocking the split thread like it currently works for Kinesis.
> >>>>>>
> >>>>>> Reading
> >>>>>>
> >>>>>> from individual Kafka partitions needs to be controlled via
> >>>>>>
> >>>>>> pause/resume
> >>>>>>
> >>>>>> on the Kafka client.
> >>>>>>
> >>>>>> To take on that responsibility the split thread would need to be
> >>>>>>
> >>>>>> aware
> >>>>>>
> >>>>>> of
> >>>>>>
> >>>>>> the
> >>>>>> watermarks or at least whether it should or should not continue to
> >>>>>>
> >>>>>> consume
> >>>>>>
> >>>>>> a given split and this may require a different SourceReader or
> >>>>>>
> >>>>>> SourceOutput
> >>>>>>
> >>>>>> interface.
> >>>>>>
> >>>>>> Thanks,
> >>>>>> Thomas
> >>>>>>
> >>>>>>
> >>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com
> >> <mailto:
> >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
> >> <
> >>>>>>
> >>>>>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
> >>>>>>
> >>>>>> Hi Stephan,
> >>>>>>
> >>>>>> Thank you for feedback!
> >>>>>> Will take a look at your branch before public discussing.
> >>>>>>
> >>>>>>
> >>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <sewen@apache.org
> >>>> <ma...@apache.org>> <sewen@apache.org <mailto:sewen@apache.org
> >>>>
> >>>>>>
> >>>>>> <
> >>>>>>
> >>>>>> sewen@apache.org <ma...@apache.org>>
> >>>>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>> Hi Biao!
> >>>>>>
> >>>>>> Thanks for reviving this. I would like to join this discussion,
> >>>>>>
> >>>>>> but
> >>>>>>
> >>>>>> am
> >>>>>>
> >>>>>> quite occupied with the 1.9 release, so can we maybe pause this
> >>>>>>
> >>>>>> discussion
> >>>>>>
> >>>>>> for a week or so?
> >>>>>>
> >>>>>> In the meantime I can share some suggestion based on prior
> >>>>>>
> >>>>>> experiments:
> >>>>>>
> >>>>>> How to do watermarks / timestamp extractors in a simpler and more
> >>>>>>
> >>>>>> flexible
> >>>>>>
> >>>>>> way. I think that part is quite promising should be part of the
> >>>>>>
> >>>>>> new
> >>>>>>
> >>>>>> source
> >>>>>>
> >>>>>> interface.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> >>>> <
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> >>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> >>>> <
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> >>>>>
> >>>>>>
> >>>>>> Some experiments on how to build the source reader and its
> >>>>>>
> >>>>>> library
> >>>>>>
> >>>>>> for
> >>>>>>
> >>>>>> common threading/split patterns:
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> >>>> <
> >>>>
> >>>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> >>>>>
> >>>>>>
> >>>>>> Best,
> >>>>>> Stephan
> >>>>>>
> >>>>>>
> >>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mmyy1110@gmail.com
> >>> <mailto:
> >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
> >> <
> >>>>>>
> >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >>>>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>> Hi devs,
> >>>>>>
> >>>>>> Since 1.9 is nearly released, I think we could get back to
> >>>>>>
> >>>>>> FLIP-27.
> >>>>>>
> >>>>>> I
> >>>>>>
> >>>>>> believe it should be included in 1.10.
> >>>>>>
> >>>>>> There are so many things mentioned in document of FLIP-27. [1] I
> >>>>>>
> >>>>>> think
> >>>>>>
> >>>>>> we'd better discuss them separately. However the wiki is not a
> >>>>>>
> >>>>>> good
> >>>>>>
> >>>>>> place
> >>>>>>
> >>>>>> to discuss. I wrote google doc about SplitReader API which
> >>>>>>
> >>>>>> misses
> >>>>>>
> >>>>>> some
> >>>>>>
> >>>>>> details in the document. [2]
> >>>>>>
> >>>>>> 1.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >>>> <
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >>>>>
> >>>>>>
> >>>>>> 2.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> >>>> <
> >>>>
> >>>
> >>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> >>>>>
> >>>>>>
> >>>>>> CC Stephan, Aljoscha, Piotrek, Becket
> >>>>>>
> >>>>>>
> >>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com
> >> <mailto:
> >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
> >> <
> >>>>>>
> >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >>>>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>> Hi Steven,
> >>>>>> Thank you for the feedback. Please take a look at the document
> >>>>>>
> >>>>>> FLIP-27
> >>>>>>
> >>>>>> <
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>> <
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>
> >>>>>>
> >>>>>> which
> >>>>>>
> >>>>>> is updated recently. A lot of details of enumerator were added
> >>>>>>
> >>>>>> in
> >>>>>>
> >>>>>> this
> >>>>>>
> >>>>>> document. I think it would help.
> >>>>>>
> >>>>>> Steven Wu <stevenz3wu@gmail.com <ma...@gmail.com>> <
> >>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
> >>> stevenz3wu@gmail.com
> >>>> <ma...@gmail.com>> <stevenz3wu@gmail.com <mailto:
> >>>> stevenz3wu@gmail.com>>
> >>>>>>
> >>>>>> 于2019年3月28日周四
> >>>>>>
> >>>>>> 下午12:52写道:
> >>>>>>
> >>>>>> This proposal mentioned that SplitEnumerator might run on the
> >>>>>> JobManager or
> >>>>>> in a single task on a TaskManager.
> >>>>>>
> >>>>>> if enumerator is a single task on a taskmanager, then the job
> >>>>>>
> >>>>>> DAG
> >>>>>>
> >>>>>> can
> >>>>>>
> >>>>>> never
> >>>>>> been embarrassingly parallel anymore. That will nullify the
> >>>>>>
> >>>>>> leverage
> >>>>>>
> >>>>>> of
> >>>>>>
> >>>>>> fine-grained recovery for embarrassingly parallel jobs.
> >>>>>>
> >>>>>> It's not clear to me what's the implication of running
> >>>>>>
> >>>>>> enumerator
> >>>>>>
> >>>>>> on
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> jobmanager. So I will leave that out for now.
> >>>>>>
> >>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com
> >> <mailto:
> >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
> >> <
> >>>>>>
> >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >>>>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>> Hi Stephan & Piotrek,
> >>>>>>
> >>>>>> Thank you for feedback.
> >>>>>>
> >>>>>> It seems that there are a lot of things to do in community.
> >>>>>>
> >>>>>> I
> >>>>>>
> >>>>>> am
> >>>>>>
> >>>>>> just
> >>>>>>
> >>>>>> afraid that this discussion may be forgotten since there so
> >>>>>>
> >>>>>> many
> >>>>>>
> >>>>>> proposals
> >>>>>>
> >>>>>> recently.
> >>>>>> Anyway, wish to see the split topics soon :)
> >>>>>>
> >>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:piotr@da-platform.com
> >>>>
> >>> <
> >>>> piotr@da-platform.com <ma...@da-platform.com>> <
> >>>> piotr@da-platform.com <ma...@da-platform.com>> <
> >>>> piotr@da-platform.com <ma...@da-platform.com>>
> >>>>>>
> >>>>>> 于2019年1月24日周四
> >>>>>>
> >>>>>> 下午8:21写道:
> >>>>>>
> >>>>>> Hi Biao!
> >>>>>>
> >>>>>> This discussion was stalled because of preparations for
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> open
> >>>>>>
> >>>>>> sourcing
> >>>>>>
> >>>>>> & merging Blink. I think before creating the tickets we
> >>>>>>
> >>>>>> should
> >>>>>>
> >>>>>> split this
> >>>>>>
> >>>>>> discussion into topics/areas outlined by Stephan and
> >>>>>>
> >>>>>> create
> >>>>>>
> >>>>>> Flips
> >>>>>>
> >>>>>> for
> >>>>>>
> >>>>>> that.
> >>>>>>
> >>>>>> I think there is no chance for this to be completed in
> >>>>>>
> >>>>>> couple
> >>>>>>
> >>>>>> of
> >>>>>>
> >>>>>> remaining
> >>>>>>
> >>>>>> weeks/1 month before 1.8 feature freeze, however it would
> >>>>>>
> >>>>>> be
> >>>>>>
> >>>>>> good
> >>>>>>
> >>>>>> to aim
> >>>>>>
> >>>>>> with those changes for 1.9.
> >>>>>>
> >>>>>> Piotrek
> >>>>>>
> >>>>>>
> >>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com <mailto:
> >>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
> >> <
> >>>>>>
> >>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
> >>>>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>> Hi community,
> >>>>>> The summary of Stephan makes a lot sense to me. It is
> >>>>>>
> >>>>>> much
> >>>>>>
> >>>>>> clearer
> >>>>>>
> >>>>>> indeed
> >>>>>>
> >>>>>> after splitting the complex topic into small ones.
> >>>>>> I was wondering is there any detail plan for next step?
> >>>>>>
> >>>>>> If
> >>>>>>
> >>>>>> not,
> >>>>>>
> >>>>>> I
> >>>>>>
> >>>>>> would
> >>>>>>
> >>>>>> like to push this thing forward by creating some JIRA
> >>>>>>
> >>>>>> issues.
> >>>>>>
> >>>>>> Another question is that should version 1.8 include
> >>>>>>
> >>>>>> these
> >>>>>>
> >>>>>> features?
> >>>>>>
> >>>>>> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <
> >>>> sewen@apache.org <ma...@apache.org>> <sewen@apache.org
> <mailto:
> >>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>>
> >>>> 于2018年12月1日周六
> >>>>>>
> >>>>>> 上午4:20写道:
> >>>>>>
> >>>>>> Thanks everyone for the lively discussion. Let me try
> >>>>>>
> >>>>>> to
> >>>>>>
> >>>>>> summarize
> >>>>>>
> >>>>>> where I
> >>>>>>
> >>>>>> see convergence in the discussion and open issues.
> >>>>>> I'll try to group this by design aspect of the source.
> >>>>>>
> >>>>>> Please
> >>>>>>
> >>>>>> let me
> >>>>>>
> >>>>>> know
> >>>>>>
> >>>>>> if I got things wrong or missed something crucial here.
> >>>>>>
> >>>>>> For issues 1-3, if the below reflects the state of the
> >>>>>>
> >>>>>> discussion, I
> >>>>>>
> >>>>>> would
> >>>>>>
> >>>>>> try and update the FLIP in the next days.
> >>>>>> For the remaining ones we need more discussion.
> >>>>>>
> >>>>>> I would suggest to fork each of these aspects into a
> >>>>>>
> >>>>>> separate
> >>>>>>
> >>>>>> mail
> >>>>>>
> >>>>>> thread,
> >>>>>>
> >>>>>> or will loose sight of the individual aspects.
> >>>>>>
> >>>>>> *(1) Separation of Split Enumerator and Split Reader*
> >>>>>>
> >>>>>> - All seem to agree this is a good thing
> >>>>>> - Split Enumerator could in the end live on JobManager
> >>>>>>
> >>>>>> (and
> >>>>>>
> >>>>>> assign
> >>>>>>
> >>>>>> splits
> >>>>>>
> >>>>>> via RPC) or in a task (and assign splits via data
> >>>>>>
> >>>>>> streams)
> >>>>>>
> >>>>>> - this discussion is orthogonal and should come later,
> >>>>>>
> >>>>>> when
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> interface
> >>>>>>
> >>>>>> is agreed upon.
> >>>>>>
> >>>>>> *(2) Split Readers for one or more splits*
> >>>>>>
> >>>>>> - Discussion seems to agree that we need to support
> >>>>>>
> >>>>>> one
> >>>>>>
> >>>>>> reader
> >>>>>>
> >>>>>> that
> >>>>>>
> >>>>>> possibly handles multiple splits concurrently.
> >>>>>> - The requirement comes from sources where one
> >>>>>>
> >>>>>> poll()-style
> >>>>>>
> >>>>>> call
> >>>>>>
> >>>>>> fetches
> >>>>>>
> >>>>>> data from different splits / partitions
> >>>>>>    --> example sources that require that would be for
> >>>>>>
> >>>>>> example
> >>>>>>
> >>>>>> Kafka,
> >>>>>>
> >>>>>> Pravega, Pulsar
> >>>>>>
> >>>>>> - Could have one split reader per source, or multiple
> >>>>>>
> >>>>>> split
> >>>>>>
> >>>>>> readers
> >>>>>>
> >>>>>> that
> >>>>>>
> >>>>>> share the "poll()" function
> >>>>>> - To not make it too complicated, we can start with
> >>>>>>
> >>>>>> thinking
> >>>>>>
> >>>>>> about
> >>>>>>
> >>>>>> one
> >>>>>>
> >>>>>> split reader for all splits initially and see if that
> >>>>>>
> >>>>>> covers
> >>>>>>
> >>>>>> all
> >>>>>>
> >>>>>> requirements
> >>>>>>
> >>>>>> *(3) Threading model of the Split Reader*
> >>>>>>
> >>>>>> - Most active part of the discussion ;-)
> >>>>>>
> >>>>>> - A non-blocking way for Flink's task code to interact
> >>>>>>
> >>>>>> with
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> source
> >>>>>>
> >>>>>> is
> >>>>>>
> >>>>>> needed in order to a task runtime code based on a
> >>>>>> single-threaded/actor-style task design
> >>>>>>    --> I personally am a big proponent of that, it will
> >>>>>>
> >>>>>> help
> >>>>>>
> >>>>>> with
> >>>>>>
> >>>>>> well-behaved checkpoints, efficiency, and simpler yet
> >>>>>>
> >>>>>> more
> >>>>>>
> >>>>>> robust
> >>>>>>
> >>>>>> runtime
> >>>>>>
> >>>>>> code
> >>>>>>
> >>>>>> - Users care about simple abstraction, so as a
> >>>>>>
> >>>>>> subclass
> >>>>>>
> >>>>>> of
> >>>>>>
> >>>>>> SplitReader
> >>>>>>
> >>>>>> (non-blocking / async) we need to have a
> >>>>>>
> >>>>>> BlockingSplitReader
> >>>>>>
> >>>>>> which
> >>>>>>
> >>>>>> will
> >>>>>>
> >>>>>> form the basis of most source implementations.
> >>>>>>
> >>>>>> BlockingSplitReader
> >>>>>>
> >>>>>> lets
> >>>>>>
> >>>>>> users do blocking simple poll() calls.
> >>>>>> - The BlockingSplitReader would spawn a thread (or
> >>>>>>
> >>>>>> more)
> >>>>>>
> >>>>>> and
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> thread(s) can make blocking calls and hand over data
> >>>>>>
> >>>>>> buffers
> >>>>>>
> >>>>>> via
> >>>>>>
> >>>>>> a
> >>>>>>
> >>>>>> blocking
> >>>>>>
> >>>>>> queue
> >>>>>> - This should allow us to cover both, a fully async
> >>>>>>
> >>>>>> runtime,
> >>>>>>
> >>>>>> and a
> >>>>>>
> >>>>>> simple
> >>>>>>
> >>>>>> blocking interface for users.
> >>>>>> - This is actually very similar to how the Kafka
> >>>>>>
> >>>>>> connectors
> >>>>>>
> >>>>>> work.
> >>>>>>
> >>>>>> Kafka
> >>>>>>
> >>>>>> 9+ with one thread, Kafka 8 with multiple threads
> >>>>>>
> >>>>>> - On the base SplitReader (the async one), the
> >>>>>>
> >>>>>> non-blocking
> >>>>>>
> >>>>>> method
> >>>>>>
> >>>>>> that
> >>>>>>
> >>>>>> gets the next chunk of data would signal data
> >>>>>>
> >>>>>> availability
> >>>>>>
> >>>>>> via
> >>>>>>
> >>>>>> a
> >>>>>>
> >>>>>> CompletableFuture, because that gives the best
> >>>>>>
> >>>>>> flexibility
> >>>>>>
> >>>>>> (can
> >>>>>>
> >>>>>> await
> >>>>>>
> >>>>>> completion or register notification handlers).
> >>>>>> - The source task would register a "thenHandle()" (or
> >>>>>>
> >>>>>> similar)
> >>>>>>
> >>>>>> on the
> >>>>>>
> >>>>>> future to put a "take next data" task into the
> >>>>>>
> >>>>>> actor-style
> >>>>>>
> >>>>>> mailbox
> >>>>>>
> >>>>>> *(4) Split Enumeration and Assignment*
> >>>>>>
> >>>>>> - Splits may be generated lazily, both in cases where
> >>>>>>
> >>>>>> there
> >>>>>>
> >>>>>> is a
> >>>>>>
> >>>>>> limited
> >>>>>>
> >>>>>> number of splits (but very many), or splits are
> >>>>>>
> >>>>>> discovered
> >>>>>>
> >>>>>> over
> >>>>>>
> >>>>>> time
> >>>>>>
> >>>>>> - Assignment should also be lazy, to get better load
> >>>>>>
> >>>>>> balancing
> >>>>>>
> >>>>>> - Assignment needs support locality preferences
> >>>>>>
> >>>>>> - Possible design based on discussion so far:
> >>>>>>
> >>>>>>    --> SplitReader has a method "addSplits(SplitT...)"
> >>>>>>
> >>>>>> to
> >>>>>>
> >>>>>> add
> >>>>>>
> >>>>>> one or
> >>>>>>
> >>>>>> more
> >>>>>>
> >>>>>> splits. Some split readers might assume they have only
> >>>>>>
> >>>>>> one
> >>>>>>
> >>>>>> split
> >>>>>>
> >>>>>> ever,
> >>>>>>
> >>>>>> concurrently, others assume multiple splits. (Note:
> >>>>>>
> >>>>>> idea
> >>>>>>
> >>>>>> behind
> >>>>>>
> >>>>>> being
> >>>>>>
> >>>>>> able
> >>>>>>
> >>>>>> to add multiple splits at the same time is to ease
> >>>>>>
> >>>>>> startup
> >>>>>>
> >>>>>> where
> >>>>>>
> >>>>>> multiple
> >>>>>>
> >>>>>> splits may be assigned instantly.)
> >>>>>>    --> SplitReader has a context object on which it can
> >>>>>>
> >>>>>> call
> >>>>>>
> >>>>>> indicate
> >>>>>>
> >>>>>> when
> >>>>>>
> >>>>>> splits are completed. The enumerator gets that
> >>>>>>
> >>>>>> notification and
> >>>>>>
> >>>>>> can
> >>>>>>
> >>>>>> use
> >>>>>>
> >>>>>> to
> >>>>>>
> >>>>>> decide when to assign new splits. This should help both
> >>>>>>
> >>>>>> in
> >>>>>>
> >>>>>> cases
> >>>>>>
> >>>>>> of
> >>>>>>
> >>>>>> sources
> >>>>>>
> >>>>>> that take splits lazily (file readers) and in case the
> >>>>>>
> >>>>>> source
> >>>>>>
> >>>>>> needs to
> >>>>>>
> >>>>>> preserve a partial order between splits (Kinesis,
> >>>>>>
> >>>>>> Pravega,
> >>>>>>
> >>>>>> Pulsar may
> >>>>>>
> >>>>>> need
> >>>>>>
> >>>>>> that).
> >>>>>>    --> SplitEnumerator gets notification when
> >>>>>>
> >>>>>> SplitReaders
> >>>>>>
> >>>>>> start
> >>>>>>
> >>>>>> and
> >>>>>>
> >>>>>> when
> >>>>>>
> >>>>>> they finish splits. They can decide at that moment to
> >>>>>>
> >>>>>> push
> >>>>>>
> >>>>>> more
> >>>>>>
> >>>>>> splits
> >>>>>>
> >>>>>> to
> >>>>>>
> >>>>>> that reader
> >>>>>>    --> The SplitEnumerator should probably be aware of
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> source
> >>>>>>
> >>>>>> parallelism, to build its initial distribution.
> >>>>>>
> >>>>>> - Open question: Should the source expose something
> >>>>>>
> >>>>>> like
> >>>>>>
> >>>>>> "host
> >>>>>>
> >>>>>> preferences", so that yarn/mesos/k8s can take this into
> >>>>>>
> >>>>>> account
> >>>>>>
> >>>>>> when
> >>>>>>
> >>>>>> selecting a node to start a TM on?
> >>>>>>
> >>>>>> *(5) Watermarks and event time alignment*
> >>>>>>
> >>>>>> - Watermark generation, as well as idleness, needs to
> >>>>>>
> >>>>>> be
> >>>>>>
> >>>>>> per
> >>>>>>
> >>>>>> split
> >>>>>>
> >>>>>> (like
> >>>>>>
> >>>>>> currently in the Kafka Source, per partition)
> >>>>>> - It is desirable to support optional
> >>>>>>
> >>>>>> event-time-alignment,
> >>>>>>
> >>>>>> meaning
> >>>>>>
> >>>>>> that
> >>>>>>
> >>>>>> splits that are ahead are back-pressured or temporarily
> >>>>>>
> >>>>>> unsubscribed
> >>>>>>
> >>>>>> - I think i would be desirable to encapsulate
> >>>>>>
> >>>>>> watermark
> >>>>>>
> >>>>>> generation
> >>>>>>
> >>>>>> logic
> >>>>>>
> >>>>>> in watermark generators, for a separation of concerns.
> >>>>>>
> >>>>>> The
> >>>>>>
> >>>>>> watermark
> >>>>>>
> >>>>>> generators should run per split.
> >>>>>> - Using watermark generators would also help with
> >>>>>>
> >>>>>> another
> >>>>>>
> >>>>>> problem of
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> suggested interface, namely supporting non-periodic
> >>>>>>
> >>>>>> watermarks
> >>>>>>
> >>>>>> efficiently.
> >>>>>>
> >>>>>> - Need a way to "dispatch" next record to different
> >>>>>>
> >>>>>> watermark
> >>>>>>
> >>>>>> generators
> >>>>>>
> >>>>>> - Need a way to tell SplitReader to "suspend" a split
> >>>>>>
> >>>>>> until a
> >>>>>>
> >>>>>> certain
> >>>>>>
> >>>>>> watermark is reached (event time backpressure)
> >>>>>> - This would in fact be not needed (and thus simpler)
> >>>>>>
> >>>>>> if
> >>>>>>
> >>>>>> we
> >>>>>>
> >>>>>> had
> >>>>>>
> >>>>>> a
> >>>>>>
> >>>>>> SplitReader per split and may be a reason to re-open
> >>>>>>
> >>>>>> that
> >>>>>>
> >>>>>> discussion
> >>>>>>
> >>>>>> *(6) Watermarks across splits and in the Split
> >>>>>>
> >>>>>> Enumerator*
> >>>>>>
> >>>>>> - The split enumerator may need some watermark
> >>>>>>
> >>>>>> awareness,
> >>>>>>
> >>>>>> which
> >>>>>>
> >>>>>> should
> >>>>>>
> >>>>>> be
> >>>>>>
> >>>>>> purely based on split metadata (like create timestamp
> >>>>>>
> >>>>>> of
> >>>>>>
> >>>>>> file
> >>>>>>
> >>>>>> splits)
> >>>>>>
> >>>>>> - If there are still more splits with overlapping
> >>>>>>
> >>>>>> event
> >>>>>>
> >>>>>> time
> >>>>>>
> >>>>>> range
> >>>>>>
> >>>>>> for
> >>>>>>
> >>>>>> a
> >>>>>>
> >>>>>> split reader, then that split reader should not advance
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> watermark
> >>>>>>
> >>>>>> within the split beyond the overlap boundary. Otherwise
> >>>>>>
> >>>>>> future
> >>>>>>
> >>>>>> splits
> >>>>>>
> >>>>>> will
> >>>>>>
> >>>>>> produce late data.
> >>>>>>
> >>>>>> - One way to approach this could be that the split
> >>>>>>
> >>>>>> enumerator
> >>>>>>
> >>>>>> may
> >>>>>>
> >>>>>> send
> >>>>>>
> >>>>>> watermarks to the readers, and the readers cannot emit
> >>>>>>
> >>>>>> watermarks
> >>>>>>
> >>>>>> beyond
> >>>>>>
> >>>>>> that received watermark.
> >>>>>> - Many split enumerators would simply immediately send
> >>>>>>
> >>>>>> Long.MAX
> >>>>>>
> >>>>>> out
> >>>>>>
> >>>>>> and
> >>>>>>
> >>>>>> leave the progress purely to the split readers.
> >>>>>>
> >>>>>> - For event-time alignment / split back pressure, this
> >>>>>>
> >>>>>> begs
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> question
> >>>>>>
> >>>>>> how we can avoid deadlocks that may arise when splits
> >>>>>>
> >>>>>> are
> >>>>>>
> >>>>>> suspended
> >>>>>>
> >>>>>> for
> >>>>>>
> >>>>>> event time back pressure,
> >>>>>>
> >>>>>> *(7) Batch and streaming Unification*
> >>>>>>
> >>>>>> - Functionality wise, the above design should support
> >>>>>>
> >>>>>> both
> >>>>>>
> >>>>>> - Batch often (mostly) does not care about reading "in
> >>>>>>
> >>>>>> order"
> >>>>>>
> >>>>>> and
> >>>>>>
> >>>>>> generating watermarks
> >>>>>>    --> Might use different enumerator logic that is
> >>>>>>
> >>>>>> more
> >>>>>>
> >>>>>> locality
> >>>>>>
> >>>>>> aware
> >>>>>>
> >>>>>> and ignores event time order
> >>>>>>    --> Does not generate watermarks
> >>>>>> - Would be great if bounded sources could be
> >>>>>>
> >>>>>> identified
> >>>>>>
> >>>>>> at
> >>>>>>
> >>>>>> compile
> >>>>>>
> >>>>>> time,
> >>>>>>
> >>>>>> so that "env.addBoundedSource(...)" is type safe and
> >>>>>>
> >>>>>> can
> >>>>>>
> >>>>>> return a
> >>>>>>
> >>>>>> "BoundedDataStream".
> >>>>>> - Possible to defer this discussion until later
> >>>>>>
> >>>>>> *Miscellaneous Comments*
> >>>>>>
> >>>>>> - Should the source have a TypeInformation for the
> >>>>>>
> >>>>>> produced
> >>>>>>
> >>>>>> type,
> >>>>>>
> >>>>>> instead
> >>>>>>
> >>>>>> of a serializer? We need a type information in the
> >>>>>>
> >>>>>> stream
> >>>>>>
> >>>>>> anyways, and
> >>>>>>
> >>>>>> can
> >>>>>>
> >>>>>> derive the serializer from that. Plus, creating the
> >>>>>>
> >>>>>> serializer
> >>>>>>
> >>>>>> should
> >>>>>>
> >>>>>> respect the ExecutionConfig.
> >>>>>>
> >>>>>> - The TypeSerializer interface is very powerful but
> >>>>>>
> >>>>>> also
> >>>>>>
> >>>>>> not
> >>>>>>
> >>>>>> easy to
> >>>>>>
> >>>>>> implement. Its purpose is to handle data super
> >>>>>>
> >>>>>> efficiently,
> >>>>>>
> >>>>>> support
> >>>>>>
> >>>>>> flexible ways of evolution, etc.
> >>>>>> For metadata I would suggest to look at the
> >>>>>>
> >>>>>> SimpleVersionedSerializer
> >>>>>>
> >>>>>> instead, which is used for example for checkpoint
> >>>>>>
> >>>>>> master
> >>>>>>
> >>>>>> hooks,
> >>>>>>
> >>>>>> or for
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> streaming file sink. I think that is is a good match
> >>>>>>
> >>>>>> for
> >>>>>>
> >>>>>> cases
> >>>>>>
> >>>>>> where
> >>>>>>
> >>>>>> we
> >>>>>>
> >>>>>> do
> >>>>>>
> >>>>>> not need more than ser/deser (no copy, etc.) and don't
> >>>>>>
> >>>>>> need to
> >>>>>>
> >>>>>> push
> >>>>>>
> >>>>>> versioning out of the serialization paths for best
> >>>>>>
> >>>>>> performance
> >>>>>>
> >>>>>> (as in
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> TypeSerializer)
> >>>>>>
> >>>>>>
> >>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> >>>>>>
> >>>>>> k.kloudas@data-artisans.com>
> >>>>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>>
> >>>>>> Hi Biao,
> >>>>>>
> >>>>>> Thanks for the answer!
> >>>>>>
> >>>>>> So given the multi-threaded readers, now we have as
> >>>>>>
> >>>>>> open
> >>>>>>
> >>>>>> questions:
> >>>>>>
> >>>>>> 1) How do we let the checkpoints pass through our
> >>>>>>
> >>>>>> multi-threaded
> >>>>>>
> >>>>>> reader
> >>>>>>
> >>>>>> operator?
> >>>>>>
> >>>>>> 2) Do we have separate reader and source operators or
> >>>>>>
> >>>>>> not? In
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> strategy
> >>>>>>
> >>>>>> that has a separate source, the source operator has a
> >>>>>>
> >>>>>> parallelism of
> >>>>>>
> >>>>>> 1
> >>>>>>
> >>>>>> and
> >>>>>>
> >>>>>> is responsible for split recovery only.
> >>>>>>
> >>>>>> For the first one, given also the constraints
> >>>>>>
> >>>>>> (blocking,
> >>>>>>
> >>>>>> finite
> >>>>>>
> >>>>>> queues,
> >>>>>>
> >>>>>> etc), I do not have an answer yet.
> >>>>>>
> >>>>>> For the 2nd, I think that we should go with separate
> >>>>>>
> >>>>>> operators
> >>>>>>
> >>>>>> for
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> source and the readers, for the following reasons:
> >>>>>>
> >>>>>> 1) This is more aligned with a potential future
> >>>>>>
> >>>>>> improvement
> >>>>>>
> >>>>>> where the
> >>>>>>
> >>>>>> split
> >>>>>>
> >>>>>> discovery becomes a responsibility of the JobManager
> >>>>>>
> >>>>>> and
> >>>>>>
> >>>>>> readers are
> >>>>>>
> >>>>>> pooling more work from the JM.
> >>>>>>
> >>>>>> 2) The source is going to be the "single point of
> >>>>>>
> >>>>>> truth".
> >>>>>>
> >>>>>> It
> >>>>>>
> >>>>>> will
> >>>>>>
> >>>>>> know
> >>>>>>
> >>>>>> what
> >>>>>>
> >>>>>> has been processed and what not. If the source and the
> >>>>>>
> >>>>>> readers
> >>>>>>
> >>>>>> are a
> >>>>>>
> >>>>>> single
> >>>>>>
> >>>>>> operator with parallelism > 1, or in general, if the
> >>>>>>
> >>>>>> split
> >>>>>>
> >>>>>> discovery
> >>>>>>
> >>>>>> is
> >>>>>>
> >>>>>> done by each task individually, then:
> >>>>>>   i) we have to have a deterministic scheme for each
> >>>>>>
> >>>>>> reader to
> >>>>>>
> >>>>>> assign
> >>>>>>
> >>>>>> splits to itself (e.g. mod subtaskId). This is not
> >>>>>>
> >>>>>> necessarily
> >>>>>>
> >>>>>> trivial
> >>>>>>
> >>>>>> for
> >>>>>>
> >>>>>> all sources.
> >>>>>>   ii) each reader would have to keep a copy of all its
> >>>>>>
> >>>>>> processed
> >>>>>>
> >>>>>> slpits
> >>>>>>
> >>>>>>   iii) the state has to be a union state with a
> >>>>>>
> >>>>>> non-trivial
> >>>>>>
> >>>>>> merging
> >>>>>>
> >>>>>> logic
> >>>>>>
> >>>>>> in order to support rescaling.
> >>>>>>
> >>>>>> Two additional points that you raised above:
> >>>>>>
> >>>>>> i) The point that you raised that we need to keep all
> >>>>>>
> >>>>>> splits
> >>>>>>
> >>>>>> (processed
> >>>>>>
> >>>>>> and
> >>>>>>
> >>>>>> not-processed) I think is a bit of a strong
> >>>>>>
> >>>>>> requirement.
> >>>>>>
> >>>>>> This
> >>>>>>
> >>>>>> would
> >>>>>>
> >>>>>> imply
> >>>>>>
> >>>>>> that for infinite sources the state will grow
> >>>>>>
> >>>>>> indefinitely.
> >>>>>>
> >>>>>> This is
> >>>>>>
> >>>>>> problem
> >>>>>>
> >>>>>> is even more pronounced if we do not have a single
> >>>>>>
> >>>>>> source
> >>>>>>
> >>>>>> that
> >>>>>>
> >>>>>> assigns
> >>>>>>
> >>>>>> splits to readers, as each reader will have its own
> >>>>>>
> >>>>>> copy
> >>>>>>
> >>>>>> of
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> state.
> >>>>>>
> >>>>>> ii) it is true that for finite sources we need to
> >>>>>>
> >>>>>> somehow
> >>>>>>
> >>>>>> not
> >>>>>>
> >>>>>> close
> >>>>>>
> >>>>>> the
> >>>>>>
> >>>>>> readers when the source/split discoverer finishes. The
> >>>>>> ContinuousFileReaderOperator has a work-around for
> >>>>>>
> >>>>>> that.
> >>>>>>
> >>>>>> It is
> >>>>>>
> >>>>>> not
> >>>>>>
> >>>>>> elegant,
> >>>>>>
> >>>>>> and checkpoints are not emitted after closing the
> >>>>>>
> >>>>>> source,
> >>>>>>
> >>>>>> but
> >>>>>>
> >>>>>> this, I
> >>>>>>
> >>>>>> believe, is a bigger problem which requires more
> >>>>>>
> >>>>>> changes
> >>>>>>
> >>>>>> than
> >>>>>>
> >>>>>> just
> >>>>>>
> >>>>>> refactoring the source interface.
> >>>>>>
> >>>>>> Cheers,
> >>>>>> Kostas
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> --
> >>>>>> Best, Jingsong Lee
> >>>>
> >>>>
> >>>
> >>
> >>
> >> --
> >> Best, Jingsong Lee
> >>
> >
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Timo Walther <tw...@apache.org>.
Hi Becket,

I completely agree with Dawid's suggestion. The information about the 
boundedness should come out of the source. Because most of the streaming 
sources can be made bounded based on some connector specific criterion. 
In Kafka, it would be an end offset or end timestamp but in any case 
having just a env.boundedSource() is not enough because parameters for 
making the source bounded are missing.

I suggest to have a simple `isBounded(): Boolean` flag in every source 
that might be influenced by a connector builder as Dawid mentioned.

For type safety during programming, we can still go with *Final state 
1*. By having a env.source() vs env.boundedSource(). The latter would 
just enforce that the boolean flag is set to `true` and could make 
bounded operations available (if we need that actually).

However, I don't think that we should start making a unified Table API 
ununified again. Boundedness is an optimization property. Every bounded 
operation can also executed in an unbounded way using updates/retraction 
or watermarks.

Regards,
Timo


On 15.12.19 14:22, Becket Qin wrote:
> Hi Dawid and Jark,
> 
> I think the discussion ultimately boils down to the question that which one
> of the following two final states do we want? Once we make this decision,
> everything else can be naturally derived.
> 
> *Final state 1*: Separate API for bounded / unbounded DataStream & Table.
> That means any code users write will be valid at the point when they write
> the code. This is similar to having type safety check at programming time.
> For example,
> 
> BoundedDataStream extends DataStream {
> // Operations only available for bounded data.
> BoundedDataStream sort(...);
> 
> // Interaction with another BoundedStream returns a Bounded stream.
> BoundedJoinedDataStream join(BoundedDataStream other)
> 
> // Interaction with another unbounded stream returns an unbounded stream.
> JoinedDataStream join(DataStream other)
> }
> 
> BoundedTable extends Table {
>    // Bounded only operation.
> BoundedTable sort(...);
> 
> // Interaction with another BoundedTable returns a BoundedTable.
> BoundedTable join(BoundedTable other)
> 
> // Interaction with another unbounded table returns an unbounded table.
> Table join(Table other)
> }
> 
> *Final state 2*: One unified API for bounded / unbounded DataStream /
> Table.
> That unified API may throw exception at DAG compilation time if an invalid
> operation is tried. This is what Table API currently follows.
> 
> DataStream {
> // Throws exception if the DataStream is unbounded.
> DataStream sort();
> // Get boundedness.
> Boundedness getBoundedness();
> }
> 
> Table {
> // Throws exception if the table has infinite rows.
> Table orderBy();
> 
> // Get boundedness.
> Boundedness getBoundedness();
> }
> 
>>From what I understand, there is no consensus so far on this decision yet.
> Whichever final state we choose, we need to make it consistent across the
> entire project. We should avoid the case that Table follows one final state
> while DataStream follows another. Some arguments I am aware of from both
> sides so far are following:
> 
> Arguments for final state 1:
> 1a) Clean API with method safety check at programming time.
> 1b) (Counter 2b) Although SQL does not have programming time error check, SQL
> is not really a "programming language" per se. So SQL can be different from
> Table and DataStream.
> 1c)  Although final state 2 seems making it easier for SQL to use given it
> is more "config based" than "parameter based", final state 1 can probably
> also meet what SQL wants by wrapping the Source in TableSource /
> TableSourceFactory API if needed.
> 
> Arguments for final state 2:
> 2a) The Source API itself seems already sort of following the unified API
> pattern.
> 2b) There is no "programming time" method error check in SQL case, so we
> cannot really achieve final state 1 across the board.
> 2c) It is an easier path given our current status, i.e. Table is already
> following final state 2.
> 2d) Users can always explicitly check the boundedness if they want to.
> 
> As I mentioned earlier, my initial thought was also to have a
> "configuration based" Source rather than a "parameter based" Source. So it
> is completely possible that I missed some important consideration or design
> principles that we want to enforce for the project. It would be good
> if @Stephan
> Ewen <st...@ververica.com> and @Aljoscha Krettek <al...@ververica.com> can
> also provide more thoughts on this.
> 
> 
> Re: Jingsong
> 
> As you said, there are some batched system source, like parquet/orc source.
>> Could we have the batch emit interface to improve performance? The queue of
>> per record may cause performance degradation.
> 
> 
> The current interface does not necessarily cause performance problem in a
> multi-threading case. In fact, the base implementation allows SplitReaders
> to add a batch <E> of records<T> to the records queue<E>, so each element
> in the records queue would be a batch <E>. In this case, when the main
> thread polls records, it will take a batch <E> of records <T> from the
> shared records queue and process the records <T> in a batch manner.
> 
> Thanks,
> 
> Jiangjie (Becket) Qin
> 
> On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <ji...@gmail.com> wrote:
> 
>> Hi Becket,
>>
>> I also have some performance concerns too.
>>
>> If I understand correctly, SourceOutput will emit data per record into the
>> queue? I'm worried about the multithreading performance of this queue.
>>
>>> One example is some batched messaging systems which only have an offset
>> for the entire batch instead of individual messages in the batch.
>>
>> As you said, there are some batched system source, like parquet/orc source.
>> Could we have the batch emit interface to improve performance? The queue of
>> per record may cause performance degradation.
>>
>> Best,
>> Jingsong Lee
>>
>> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com> wrote:
>>
>>> Hi Becket,
>>>
>>> I think Dawid explained things clearly and makes a lot of sense.
>>> I'm also in favor of #2, because #1 doesn't work for our future unified
>>> envrionment.
>>>
>>> You can see the vision in this documentation [1]. In the future, we would
>>> like to
>>> drop the global streaming/batch mode in SQL (i.e.
>>> EnvironmentSettings#inStreamingMode/inBatchMode).
>>> A source is bounded or unbounded once defined, so queries can be inferred
>>> from source to run
>>> in streaming or batch or hybrid mode. However, in #1, we will lose this
>>> ability because the framework
>>> doesn't know whether the source is bounded or unbounded.
>>>
>>> Best,
>>> Jark
>>>
>>>
>>> [1]:
>>>
>>>
>> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
>>>
>>> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <pi...@ververica.com>
>> wrote:
>>>
>>>> Hi,
>>>>
>>>> Regarding the:
>>>>
>>>> Collection<E> getNextRecords()
>>>>
>>>> I’m pretty sure such design would unfortunately impact the performance
>>>> (accessing and potentially creating the collection on the hot path).
>>>>
>>>> Also the
>>>>
>>>> InputStatus emitNext(DataOutput<T> output) throws Exception;
>>>> or
>>>> Status pollNext(SourceOutput<T> sourceOutput) throws Exception;
>>>>
>>>> Gives us some opportunities in the future, to allow Source hot looping
>>>> inside, until it receives some signal “please exit because of some
>>> reasons”
>>>> (output collector could return such hint upon collecting the result).
>> But
>>>> that’s another topic outside of this FLIP’s scope.
>>>>
>>>> Piotrek
>>>>
>>>>> On 11 Dec 2019, at 10:41, Till Rohrmann <tr...@apache.org>
>> wrote:
>>>>>
>>>>> Hi Becket,
>>>>>
>>>>> quick clarification from my side because I think you misunderstood my
>>>>> question. I did not suggest to let the SourceReader return only a
>>> single
>>>>> record at a time when calling getNextRecords. As the return type
>>>> indicates,
>>>>> the method can return an arbitrary number of records.
>>>>>
>>>>> Cheers,
>>>>> Till
>>>>>
>>>>> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
>>>> dwysakowicz@apache.org <ma...@apache.org>>
>>>>> wrote:
>>>>>
>>>>>> Hi Becket,
>>>>>>
>>>>>> Issue #1 - Design of Source interface
>>>>>>
>>>>>> I mentioned the lack of a method like
>>>> Source#createEnumerator(Boundedness
>>>>>> boundedness, SplitEnumeratorContext context), because without the
>>>> current
>>>>>> proposal is not complete/does not work.
>>>>>>
>>>>>> If we say that boundedness is an intrinsic property of a source imo
>> we
>>>>>> don't need the Source#createEnumerator(Boundedness boundedness,
>>>>>> SplitEnumeratorContext context) method.
>>>>>>
>>>>>> Assuming a source from my previous example:
>>>>>>
>>>>>> Source source = KafkaSource.builder()
>>>>>>   ...
>>>>>>   .untilTimestamp(...)
>>>>>>   .build()
>>>>>>
>>>>>> Would the enumerator differ if created like
>>>>>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
>>>>>> .createEnumerator(BOUNDED, ...)? I know I am repeating myself, but
>>> this
>>>> is
>>>>>> the part that my opinion differ the most from the current proposal.
>> I
>>>>>> really think it should always be the source that tells if it is
>>> bounded
>>>> or
>>>>>> not. In the current proposal methods continousSource/boundedSource
>>>> somewhat
>>>>>> reconfigure the source, which I think is misleading.
>>>>>>
>>>>>> I think a call like:
>>>>>>
>>>>>> Source source = KafkaSource.builder()
>>>>>>   ...
>>>>>>   .readContinously() / readUntilLatestOffset() / readUntilTimestamp /
>>>> readUntilOffsets / ...
>>>>>>   .build()
>>>>>>
>>>>>> is way cleaner (and expressive) than
>>>>>>
>>>>>> Source source = KafkaSource.builder()
>>>>>>   ...
>>>>>>   .build()
>>>>>>
>>>>>>
>>>>>> env.continousSource(source) // which actually underneath would call
>>>> createEnumerator(CONTINUOUS, ctx) which would be equivalent to
>>>> source.readContinously().createEnumerator(ctx)
>>>>>> // or
>>>>>> env.boundedSource(source) // which actually underneath would call
>>>> createEnumerator(BOUNDED, ctx) which would be equivalent to
>>>> source.readUntilLatestOffset().createEnumerator(ctx)
>>>>>>
>>>>>>
>>>>>> Sorry for the comparison, but to me it seems there is too much magic
>>>>>> happening underneath those two calls.
>>>>>>
>>>>>> I really believe the Source interface should have getBoundedness
>>> method
>>>>>> instead of (supportBoundedness) + createEnumerator(Boundedness, ...)
>>>>>>
>>>>>>
>>>>>> Issue #2 - Design of
>>>>>> ExecutionEnvironment#source()/continuousSource()/boundedSource()
>>>>>>
>>>>>> As you might have guessed I am slightly in favor of option #2
>>> modified.
>>>>>> Yes I am aware every step of the dag would have to be able to say if
>>> it
>>>> is
>>>>>> bounded or not. I have a feeling it would be easier to express cross
>>>>>> bounded/unbounded operations, but I must admit I have not thought it
>>>>>> through thoroughly, In the spirit of batch is just a special case of
>>>>>> streaming I thought BoundedStream would extend from DataStream.
>>> Correct
>>>> me
>>>>>> if I am wrong. In such a setup the cross bounded/unbounded operation
>>>> could
>>>>>> be expressed quite easily I think:
>>>>>>
>>>>>> DataStream {
>>>>>>   DataStream join(DataStream, ...); // we could not really tell if
>> the
>>>> result is bounded or not, but because bounded stream is a special case
>> of
>>>> unbounded the API object is correct, irrespective if the left or right
>>> side
>>>> of the join is bounded
>>>>>> }
>>>>>>
>>>>>> BoundedStream extends DataStream {
>>>>>>   BoundedStream join(BoundedStream, ...); // only if both sides are
>>>> bounded the result can be bounded as well. However we do have access to
>>> the
>>>> DataStream#join here, so you can still join with a DataStream
>>>>>> }
>>>>>>
>>>>>>
>>>>>> On the other hand I also see benefits of two completely disjointed
>>> APIs,
>>>>>> as we could prohibit some streaming calls in the bounded API. I
>> can't
>>>> think
>>>>>> of any unbounded operators that could not be implemented for bounded
>>>> stream.
>>>>>>
>>>>>> Besides I think we both agree we don't like the method:
>>>>>>
>>>>>> DataStream boundedStream(Source)
>>>>>>
>>>>>> suggested in the current state of the FLIP. Do we ? :)
>>>>>>
>>>>>> Best,
>>>>>>
>>>>>> Dawid
>>>>>>
>>>>>> On 10/12/2019 18:57, Becket Qin wrote:
>>>>>>
>>>>>> Hi folks,
>>>>>>
>>>>>> Thanks for the discussion, great feedback. Also thanks Dawid for the
>>>>>> explanation, it is much clearer now.
>>>>>>
>>>>>> One thing that is indeed missing from the FLIP is how the
>> boundedness
>>> is
>>>>>> passed to the Source implementation. So the API should be
>>>>>> Source#createEnumerator(Boundedness boundedness,
>>> SplitEnumeratorContext
>>>>>> context)
>>>>>> And we can probably remove the Source#supportBoundedness(Boundedness
>>>>>> boundedness) method.
>>>>>>
>>>>>> Assuming we have that, we are essentially choosing from one of the
>>>>>> following two options:
>>>>>>
>>>>>> Option 1:
>>>>>> // The source is continuous source, and only unbounded operations
>> can
>>> be
>>>>>> performed.
>>>>>> DataStream<Type> datastream = env.continuousSource(someSource);
>>>>>>
>>>>>> // The source is bounded source, both bounded and unbounded
>> operations
>>>> can
>>>>>> be performed.
>>>>>> BoundedDataStream<Type> boundedDataStream =
>>>> env.boundedSource(someSource);
>>>>>>
>>>>>>   - Pros:
>>>>>>        a) explicit boundary between bounded / unbounded streams, it
>> is
>>>>>> quite simple and clear to the users.
>>>>>>   - Cons:
>>>>>>        a) For applications that do not involve bounded operations,
>> they
>>>>>> still have to call different API to distinguish bounded / unbounded
>>>> streams.
>>>>>>        b) No support for bounded stream to run in a streaming runtime
>>>>>> setting, i.e. scheduling and operators behaviors.
>>>>>>
>>>>>>
>>>>>> Option 2:
>>>>>> // The source is either bounded or unbounded, but only unbounded
>>>> operations
>>>>>> could be performed on the returned DataStream.
>>>>>> DataStream<Type> dataStream = env.source(someSource);
>>>>>>
>>>>>> // The source must be a bounded source, otherwise exception is
>> thrown.
>>>>>> BoundedDataStream<Type> boundedDataStream =
>>>>>> env.boundedSource(boundedSource);
>>>>>>
>>>>>> The pros and cons are exactly the opposite of option 1.
>>>>>>   - Pros:
>>>>>>        a) For applications that do not involve bounded operations,
>> they
>>>>>> still have to call different API to distinguish bounded / unbounded
>>>> streams.
>>>>>>        b) Support for bounded stream to run in a streaming runtime
>>>> setting,
>>>>>> i.e. scheduling and operators behaviors.
>>>>>>   - Cons:
>>>>>>        a) Bounded / unbounded streams are kind of mixed, i.e. given a
>>>>>> DataStream, it is not clear whether it is bounded or not, unless you
>>>> have
>>>>>> the access to its source.
>>>>>>
>>>>>>
>>>>>> If we only think from the Source API perspective, option 2 seems a
>>>> better
>>>>>> choice because functionality wise it is a superset of option 1, at
>> the
>>>> cost
>>>>>> of some seemingly acceptable ambiguity in the DataStream API.
>>>>>> But if we look at the DataStream API as a whole, option 1 seems a
>>>> clearer
>>>>>> choice. For example, some times a library may have to know whether a
>>>>>> certain task will finish or not. And it would be difficult to tell
>> if
>>>> the
>>>>>> input is a DataStream, unless additional information is provided all
>>> the
>>>>>> way from the Source. One possible solution is to have a *modified
>>>> option 2*
>>>>>> which adds a method to the DataStream API to indicate boundedness,
>>> such
>>>> as
>>>>>> getBoundedness(). It would solve the problem with a potential
>>> confusion
>>>> of
>>>>>> what is difference between a DataStream with getBoundedness()=true
>>> and a
>>>>>> BoundedDataStream. But that seems not super difficult to explain.
>>>>>>
>>>>>> So from API's perspective, I don't have a strong opinion between
>>>> *option 1*
>>>>>> and *modified option 2. *I like the cleanness of option 1, but
>>> modified
>>>>>> option 2 would be more attractive if we have concrete use case for
>> the
>>>>>> "Bounded stream with unbounded streaming runtime settings".
>>>>>>
>>>>>> Re: Till
>>>>>>
>>>>>>
>>>>>> Maybe this has already been asked before but I was wondering why the
>>>>>> SourceReader interface has the method pollNext which hands the
>>>>>> responsibility of outputting elements to the SourceReader
>>>> implementation?
>>>>>> Has this been done for backwards compatibility reasons with the old
>>>> source
>>>>>> interface? If not, then one could define a Collection<E>
>>>> getNextRecords()
>>>>>> method which returns the currently retrieved records and then the
>>> caller
>>>>>> emits them outside of the SourceReader. That way the interface would
>>> not
>>>>>> allow to implement an outputting loop where we never hand back
>> control
>>>> to
>>>>>> the caller. At the moment, this contract can be easily broken and is
>>>> only
>>>>>> mentioned loosely in the JavaDocs.
>>>>>>
>>>>>>
>>>>>> The primary reason we handover the SourceOutput to the SourceReader
>> is
>>>>>> because sometimes it is difficult for a SourceReader to emit one
>>> record
>>>> at
>>>>>> a time. One example is some batched messaging systems which only
>> have
>>> an
>>>>>> offset for the entire batch instead of individual messages in the
>>>> batch. In
>>>>>> that case, returning one record at a time would leave the
>> SourceReader
>>>> in
>>>>>> an uncheckpointable state because they can only checkpoint at the
>>> batch
>>>>>> boundaries.
>>>>>>
>>>>>> Thanks,
>>>>>>
>>>>>> Jiangjie (Becket) Qin
>>>>>>
>>>>>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <trohrmann@apache.org
>>>> <ma...@apache.org>> <trohrmann@apache.org <mailto:
>>>> trohrmann@apache.org>> wrote:
>>>>>>
>>>>>>
>>>>>> Hi everyone,
>>>>>>
>>>>>> thanks for drafting this FLIP. It reads very well.
>>>>>>
>>>>>> Concerning Dawid's proposal, I tend to agree. The boundedness could
>>> come
>>>>>> from the source and tell the system how to treat the operator
>>>> (scheduling
>>>>>> wise). From a user's perspective it should be fine to get back a
>>>> DataStream
>>>>>> when calling env.source(boundedSource) if he does not need special
>>>>>> operations defined on a BoundedDataStream. If he needs this, then
>> one
>>>> could
>>>>>> use the method BoundedDataStream env.boundedSource(boundedSource).
>>>>>>
>>>>>> If possible, we could enforce the proper usage of
>> env.boundedSource()
>>> by
>>>>>> introducing a BoundedSource type so that one cannot pass an
>>>>>> unbounded source to it. That way users would not be able to shoot
>>>>>> themselves in the foot.
>>>>>>
>>>>>> Maybe this has already been asked before but I was wondering why the
>>>>>> SourceReader interface has the method pollNext which hands the
>>>>>> responsibility of outputting elements to the SourceReader
>>>> implementation?
>>>>>> Has this been done for backwards compatibility reasons with the old
>>>> source
>>>>>> interface? If not, then one could define a Collection<E>
>>>> getNextRecords()
>>>>>> method which returns the currently retrieved records and then the
>>> caller
>>>>>> emits them outside of the SourceReader. That way the interface would
>>> not
>>>>>> allow to implement an outputting loop where we never hand back
>> control
>>>> to
>>>>>> the caller. At the moment, this contract can be easily broken and is
>>>> only
>>>>>> mentioned loosely in the JavaDocs.
>>>>>>
>>>>>> Cheers,
>>>>>> Till
>>>>>>
>>>>>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <jingsonglee0@gmail.com
>>>> <ma...@gmail.com>> <jingsonglee0@gmail.com <mailto:
>>>> jingsonglee0@gmail.com>>
>>>>>> wrote:
>>>>>>
>>>>>>
>>>>>> Hi all,
>>>>>>
>>>>>> I think current design is good.
>>>>>>
>>>>>> My understanding is:
>>>>>>
>>>>>> For execution mode: bounded mode and continuous mode, It's totally
>>>>>> different. I don't think we have the ability to integrate the two
>>> models
>>>>>>
>>>>>> at
>>>>>>
>>>>>> present. It's about scheduling, memory, algorithms, States, etc. we
>>>>>> shouldn't confuse them.
>>>>>>
>>>>>> For source capabilities: only bounded, only continuous, both bounded
>>> and
>>>>>> continuous.
>>>>>> I think Kafka is a source that can be ran both bounded
>>>>>> and continuous execution mode.
>>>>>> And Kafka with end offset should be ran both bounded
>>>>>> and continuous execution mode.  Using apache Beam with Flink
>> runner, I
>>>>>>
>>>>>> used
>>>>>>
>>>>>> to run a "bounded" Kafka in streaming mode. For our previous
>>> DataStream,
>>>>>>
>>>>>> it
>>>>>>
>>>>>> is not necessarily required that the source cannot be bounded.
>>>>>>
>>>>>> So it is my thought for Dawid's question:
>>>>>> 1.pass a bounded source to continuousSource() +1
>>>>>> 2.pass a continuous source to boundedSource() -1, should throw
>>>> exception.
>>>>>>
>>>>>> In StreamExecutionEnvironment, continuousSource and boundedSource
>>> define
>>>>>> the execution mode. It defines a clear boundary of execution mode.
>>>>>>
>>>>>> Best,
>>>>>> Jingsong Lee
>>>>>>
>>>>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com <mailto:
>>>> imjark@gmail.com>> <imjark@gmail.com <ma...@gmail.com>> wrote:
>>>>>>
>>>>>>
>>>>>> I agree with Dawid's point that the boundedness information should
>>> come
>>>>>> from the source itself (e.g. the end timestamp), not through
>>>>>> env.boundedSouce()/continuousSource().
>>>>>> I think if we want to support something like `env.source()` that
>>> derive
>>>>>>
>>>>>> the
>>>>>>
>>>>>> execution mode from source, `supportsBoundedness(Boundedness)`
>>>>>> method is not enough, because we don't know whether it is bounded or
>>>>>>
>>>>>> not.
>>>>>>
>>>>>> Best,
>>>>>> Jark
>>>>>>
>>>>>>
>>>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
>> dwysakowicz@apache.org
>>>> <ma...@apache.org>> <dwysakowicz@apache.org <mailto:
>>>> dwysakowicz@apache.org>>
>>>>>> wrote:
>>>>>>
>>>>>>
>>>>>> One more thing. In the current proposal, with the
>>>>>> supportsBoundedness(Boundedness) method and the boundedness coming
>>>>>>
>>>>>> from
>>>>>>
>>>>>> either continuousSource or boundedSource I could not find how this
>>>>>> information is fed back to the SplitEnumerator.
>>>>>>
>>>>>> Best,
>>>>>>
>>>>>> Dawid
>>>>>>
>>>>>> On 09/12/2019 13:52, Becket Qin wrote:
>>>>>>
>>>>>> Hi Dawid,
>>>>>>
>>>>>> Thanks for the comments. This actually brings another relevant
>>>>>>
>>>>>> question
>>>>>>
>>>>>> about what does a "bounded source" imply. I actually had the same
>>>>>> impression when I look at the Source API. Here is what I understand
>>>>>>
>>>>>> after
>>>>>>
>>>>>> some discussion with Stephan. The bounded source has the following
>>>>>>
>>>>>> impacts.
>>>>>>
>>>>>> 1. API validity.
>>>>>> - A bounded source generates a bounded stream so some operations
>>>>>>
>>>>>> that
>>>>>>
>>>>>> only
>>>>>>
>>>>>> works for bounded records would be performed, e.g. sort.
>>>>>> - To expose these bounded stream only APIs, there are two options:
>>>>>>      a. Add them to the DataStream API and throw exception if a
>>>>>>
>>>>>> method
>>>>>>
>>>>>> is
>>>>>>
>>>>>> called on an unbounded stream.
>>>>>>      b. Create a BoundedDataStream class which is returned from
>>>>>> env.boundedSource(), while DataStream is returned from
>>>>>>
>>>>>> env.continousSource().
>>>>>>
>>>>>> Note that this cannot be done by having single
>>>>>>
>>>>>> env.source(theSource)
>>>>>>
>>>>>> even
>>>>>>
>>>>>> the Source has a getBoundedness() method.
>>>>>>
>>>>>> 2. Scheduling
>>>>>> - A bounded source could be computed stage by stage without
>>>>>>
>>>>>> bringing
>>>>>>
>>>>>> up
>>>>>>
>>>>>> all
>>>>>>
>>>>>> the tasks at the same time.
>>>>>>
>>>>>> 3. Operator behaviors
>>>>>> - A bounded source indicates the records are finite so some
>>>>>>
>>>>>> operators
>>>>>>
>>>>>> can
>>>>>>
>>>>>> wait until it receives all the records before it starts the
>>>>>>
>>>>>> processing.
>>>>>>
>>>>>> In the above impact, only 1 is relevant to the API design. And the
>>>>>>
>>>>>> current
>>>>>>
>>>>>> proposal in FLIP-27 is following 1.b.
>>>>>>
>>>>>> // boundedness depends of source property, imo this should always
>>>>>>
>>>>>> be
>>>>>>
>>>>>> preferred
>>>>>>
>>>>>>
>>>>>> DataStream<MyType> stream = env.source(theSource);
>>>>>>
>>>>>>
>>>>>> In your proposal, does DataStream have bounded stream only methods?
>>>>>>
>>>>>> It
>>>>>>
>>>>>> looks it should have, otherwise passing a bounded Source to
>>>>>>
>>>>>> env.source()
>>>>>>
>>>>>> would be confusing. In that case, we will essentially do 1.a if an
>>>>>> unbounded Source is created from env.source(unboundedSource).
>>>>>>
>>>>>> If we have the methods only supported for bounded streams in
>>>>>>
>>>>>> DataStream,
>>>>>>
>>>>>> it
>>>>>>
>>>>>> seems a little weird to have a separate BoundedDataStream
>>>>>>
>>>>>> interface.
>>>>>>
>>>>>> Am I understand it correctly?
>>>>>>
>>>>>> Thanks,
>>>>>>
>>>>>> Jiangjie (Becket) Qin
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
>>>>>>
>>>>>> dwysakowicz@apache.org <ma...@apache.org>>
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>>
>>>>>> Hi all,
>>>>>>
>>>>>> Really well written proposal and very important one. I must admit
>>>>>>
>>>>>> I
>>>>>>
>>>>>> have
>>>>>>
>>>>>> not understood all the intricacies of it yet.
>>>>>>
>>>>>> One question I have though is about where does the information
>>>>>>
>>>>>> about
>>>>>>
>>>>>> boundedness come from. I think in most cases it is a property of
>>>>>>
>>>>>> the
>>>>>>
>>>>>> source. As you described it might be e.g. end offset, a flag
>>>>>>
>>>>>> should
>>>>>>
>>>>>> it
>>>>>>
>>>>>> monitor new splits etc. I think it would be a really nice use case
>>>>>>
>>>>>> to
>>>>>>
>>>>>> be
>>>>>>
>>>>>> able to say:
>>>>>>
>>>>>> new KafkaSource().readUntil(long timestamp),
>>>>>>
>>>>>> which could work as an "end offset". Moreover I think all Bounded
>>>>>>
>>>>>> sources
>>>>>>
>>>>>> support continuous mode, but no intrinsically continuous source
>>>>>>
>>>>>> support
>>>>>>
>>>>>> the
>>>>>>
>>>>>> Bounded mode. If I understood the proposal correctly it suggest
>>>>>>
>>>>>> the
>>>>>>
>>>>>> boundedness sort of "comes" from the outside of the source, from
>>>>>>
>>>>>> the
>>>>>>
>>>>>> invokation of either boundedStream or continousSource.
>>>>>>
>>>>>> I am wondering if it would make sense to actually change the
>>>>>>
>>>>>> method
>>>>>>
>>>>>> boolean Source#supportsBoundedness(Boundedness)
>>>>>>
>>>>>> to
>>>>>>
>>>>>> Boundedness Source#getBoundedness().
>>>>>>
>>>>>> As for the methods #boundedSource, #continousSource, assuming the
>>>>>> boundedness is property of the source they do not affect how the
>>>>>>
>>>>>> enumerator
>>>>>>
>>>>>> works, but mostly how the dag is scheduled, right? I am not
>>>>>>
>>>>>> against
>>>>>>
>>>>>> those
>>>>>>
>>>>>> methods, but I think it is a very specific use case to actually
>>>>>>
>>>>>> override
>>>>>>
>>>>>> the property of the source. In general I would expect users to
>>>>>>
>>>>>> only
>>>>>>
>>>>>> call
>>>>>>
>>>>>> env.source(theSource), where the source tells if it is bounded or
>>>>>>
>>>>>> not. I
>>>>>>
>>>>>> would suggest considering following set of methods:
>>>>>>
>>>>>> // boundedness depends of source property, imo this should always
>>>>>>
>>>>>> be
>>>>>>
>>>>>> preferred
>>>>>>
>>>>>> DataStream<MyType> stream = env.source(theSource);
>>>>>>
>>>>>>
>>>>>> // always continous execution, whether bounded or unbounded source
>>>>>>
>>>>>> DataStream<MyType> boundedStream = env.continousSource(theSource);
>>>>>>
>>>>>> // imo this would make sense if the BoundedDataStream provides
>>>>>>
>>>>>> additional features unavailable for continous mode
>>>>>>
>>>>>> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
>>>>>>
>>>>>>
>>>>>> Best,
>>>>>>
>>>>>> Dawid
>>>>>>
>>>>>>
>>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
>>>>>>
>>>>>> Thanks, Becket, for updating this.
>>>>>>
>>>>>> I agree with moving the aspects you mentioned into separate FLIPs
>>>>>>
>>>>>> -
>>>>>>
>>>>>> this
>>>>>>
>>>>>> one way becoming unwieldy in size.
>>>>>>
>>>>>> +1 to the FLIP in its current state. Its a very detailed write-up,
>>>>>>
>>>>>> nicely
>>>>>>
>>>>>> done!
>>>>>>
>>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <becket.qin@gmail.com
>>>> <ma...@gmail.com>> <becket.qin@gmail.com <mailto:
>>>> becket.qin@gmail.com>>
>>>>>>
>>>>>> <
>>>>>>
>>>>>> becket.qin@gmail.com <ma...@gmail.com>> wrote:
>>>>>>
>>>>>> Hi all,
>>>>>>
>>>>>> Sorry for the long belated update. I have updated FLIP-27 wiki
>>>>>>
>>>>>> page
>>>>>>
>>>>>> with
>>>>>>
>>>>>> the latest proposals. Some noticeable changes include:
>>>>>> 1. A new generic communication mechanism between SplitEnumerator
>>>>>>
>>>>>> and
>>>>>>
>>>>>> SourceReader.
>>>>>> 2. Some detail API method signature changes.
>>>>>>
>>>>>> We left a few things out of this FLIP and will address them in
>>>>>>
>>>>>> separate
>>>>>>
>>>>>> FLIPs. Including:
>>>>>> 1. Per split event time.
>>>>>> 2. Event time alignment.
>>>>>> 3. Fine grained failover for SplitEnumerator failure.
>>>>>>
>>>>>> Please let us know if you have any question.
>>>>>>
>>>>>> Thanks,
>>>>>>
>>>>>> Jiangjie (Becket) Qin
>>>>>>
>>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <sewen@apache.org
>>> <mailto:
>>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>> <
>>>>>>
>>>>>> sewen@apache.org <ma...@apache.org>> wrote:
>>>>>>
>>>>>> Hi  Łukasz!
>>>>>>
>>>>>> Becket and me are working hard on figuring out the last details
>>>>>>
>>>>>> and
>>>>>>
>>>>>> implementing the first PoC. We would update the FLIP hopefully
>>>>>>
>>>>>> next
>>>>>>
>>>>>> week.
>>>>>>
>>>>>> There is a fair chance that a first version of this will be in
>>>>>>
>>>>>> 1.10,
>>>>>>
>>>>>> but
>>>>>>
>>>>>> I
>>>>>>
>>>>>> think it will take another release to battle test it and migrate
>>>>>>
>>>>>> the
>>>>>>
>>>>>> connectors.
>>>>>>
>>>>>> Best,
>>>>>> Stephan
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <ljd@touk.pl
>>>> <ma...@touk.pl>
>>>>>>
>>>>>> <
>>>>>>
>>>>>> ljd@touk.pl <ma...@touk.pl>>
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>> Hi,
>>>>>>
>>>>>> This proposal looks very promising for us. Do you have any plans
>>>>>>
>>>>>> in
>>>>>>
>>>>>> which
>>>>>>
>>>>>> Flink release it is going to be released? We are thinking on
>>>>>>
>>>>>> using a
>>>>>>
>>>>>> Data
>>>>>>
>>>>>> Set API for our future use cases but on the other hand Data Set
>>>>>>
>>>>>> API
>>>>>>
>>>>>> is
>>>>>>
>>>>>> going to be deprecated so using proposed bounded data streams
>>>>>>
>>>>>> solution
>>>>>>
>>>>>> could be more viable in the long term.
>>>>>>
>>>>>> Thanks,
>>>>>> Łukasz
>>>>>>
>>>>>> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com
>> <mailto:
>>>> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
>>>> thomas.weise@gmail.com>> <
>>>>>>
>>>>>> thomas.weise@gmail.com <ma...@gmail.com>> wrote:
>>>>>>
>>>>>> Thanks for putting together this proposal!
>>>>>>
>>>>>> I see that the "Per Split Event Time" and "Event Time Alignment"
>>>>>>
>>>>>> sections
>>>>>>
>>>>>> are still TBD.
>>>>>>
>>>>>> It would probably be good to flesh those out a bit before
>>>>>>
>>>>>> proceeding
>>>>>>
>>>>>> too
>>>>>>
>>>>>> far
>>>>>>
>>>>>> as the event time alignment will probably influence the
>>>>>>
>>>>>> interaction
>>>>>>
>>>>>> with
>>>>>>
>>>>>> the split reader, specifically ReaderStatus
>>>>>>
>>>>>> emitNext(SourceOutput<E>
>>>>>>
>>>>>> output).
>>>>>>
>>>>>> We currently have only one implementation for event time alignment
>>>>>>
>>>>>> in
>>>>>>
>>>>>> the
>>>>>>
>>>>>> Kinesis consumer. The synchronization in that case takes place as
>>>>>>
>>>>>> the
>>>>>>
>>>>>> last
>>>>>>
>>>>>> step before records are emitted downstream (RecordEmitter). With
>>>>>>
>>>>>> the
>>>>>>
>>>>>> currently proposed interfaces, the equivalent can be implemented
>>>>>>
>>>>>> in
>>>>>>
>>>>>> the
>>>>>>
>>>>>> reader loop, although note that in the Kinesis consumer the per
>>>>>>
>>>>>> shard
>>>>>>
>>>>>> threads push records.
>>>>>>
>>>>>> Synchronization has not been implemented for the Kafka consumer
>>>>>>
>>>>>> yet.
>>>>>>
>>>>>> https://issues.apache.org/jira/browse/FLINK-12675 <
>>>> https://issues.apache.org/jira/browse/FLINK-12675>
>>>>>>
>>>>>> When I looked at it, I realized that the implementation will look
>>>>>>
>>>>>> quite
>>>>>>
>>>>>> different
>>>>>> from Kinesis because it needs to take place in the pull part,
>>>>>>
>>>>>> where
>>>>>>
>>>>>> records
>>>>>>
>>>>>> are taken from the Kafka client. Due to the multiplexing it cannot
>>>>>>
>>>>>> be
>>>>>>
>>>>>> done
>>>>>>
>>>>>> by blocking the split thread like it currently works for Kinesis.
>>>>>>
>>>>>> Reading
>>>>>>
>>>>>> from individual Kafka partitions needs to be controlled via
>>>>>>
>>>>>> pause/resume
>>>>>>
>>>>>> on the Kafka client.
>>>>>>
>>>>>> To take on that responsibility the split thread would need to be
>>>>>>
>>>>>> aware
>>>>>>
>>>>>> of
>>>>>>
>>>>>> the
>>>>>> watermarks or at least whether it should or should not continue to
>>>>>>
>>>>>> consume
>>>>>>
>>>>>> a given split and this may require a different SourceReader or
>>>>>>
>>>>>> SourceOutput
>>>>>>
>>>>>> interface.
>>>>>>
>>>>>> Thanks,
>>>>>> Thomas
>>>>>>
>>>>>>
>>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com
>> <mailto:
>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
>> <
>>>>>>
>>>>>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
>>>>>>
>>>>>> Hi Stephan,
>>>>>>
>>>>>> Thank you for feedback!
>>>>>> Will take a look at your branch before public discussing.
>>>>>>
>>>>>>
>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <sewen@apache.org
>>>> <ma...@apache.org>> <sewen@apache.org <mailto:sewen@apache.org
>>>>
>>>>>>
>>>>>> <
>>>>>>
>>>>>> sewen@apache.org <ma...@apache.org>>
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>> Hi Biao!
>>>>>>
>>>>>> Thanks for reviving this. I would like to join this discussion,
>>>>>>
>>>>>> but
>>>>>>
>>>>>> am
>>>>>>
>>>>>> quite occupied with the 1.9 release, so can we maybe pause this
>>>>>>
>>>>>> discussion
>>>>>>
>>>>>> for a week or so?
>>>>>>
>>>>>> In the meantime I can share some suggestion based on prior
>>>>>>
>>>>>> experiments:
>>>>>>
>>>>>> How to do watermarks / timestamp extractors in a simpler and more
>>>>>>
>>>>>> flexible
>>>>>>
>>>>>> way. I think that part is quite promising should be part of the
>>>>>>
>>>>>> new
>>>>>>
>>>>>> source
>>>>>>
>>>>>> interface.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>>>> <
>>>>
>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>>>> <
>>>>
>>>
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>>>>>
>>>>>>
>>>>>> Some experiments on how to build the source reader and its
>>>>>>
>>>>>> library
>>>>>>
>>>>>> for
>>>>>>
>>>>>> common threading/split patterns:
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>>>> <
>>>>
>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>>>>>
>>>>>>
>>>>>> Best,
>>>>>> Stephan
>>>>>>
>>>>>>
>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mmyy1110@gmail.com
>>> <mailto:
>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
>> <
>>>>>>
>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>> Hi devs,
>>>>>>
>>>>>> Since 1.9 is nearly released, I think we could get back to
>>>>>>
>>>>>> FLIP-27.
>>>>>>
>>>>>> I
>>>>>>
>>>>>> believe it should be included in 1.10.
>>>>>>
>>>>>> There are so many things mentioned in document of FLIP-27. [1] I
>>>>>>
>>>>>> think
>>>>>>
>>>>>> we'd better discuss them separately. However the wiki is not a
>>>>>>
>>>>>> good
>>>>>>
>>>>>> place
>>>>>>
>>>>>> to discuss. I wrote google doc about SplitReader API which
>>>>>>
>>>>>> misses
>>>>>>
>>>>>> some
>>>>>>
>>>>>> details in the document. [2]
>>>>>>
>>>>>> 1.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>>>> <
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>>>>>
>>>>>>
>>>>>> 2.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>>>> <
>>>>
>>>
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>>>>>
>>>>>>
>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
>>>>>>
>>>>>>
>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com
>> <mailto:
>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
>> <
>>>>>>
>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>> Hi Steven,
>>>>>> Thank you for the feedback. Please take a look at the document
>>>>>>
>>>>>> FLIP-27
>>>>>>
>>>>>> <
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>> <
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>
>>>>>>
>>>>>> which
>>>>>>
>>>>>> is updated recently. A lot of details of enumerator were added
>>>>>>
>>>>>> in
>>>>>>
>>>>>> this
>>>>>>
>>>>>> document. I think it would help.
>>>>>>
>>>>>> Steven Wu <stevenz3wu@gmail.com <ma...@gmail.com>> <
>>>> stevenz3wu@gmail.com <ma...@gmail.com>> <
>>> stevenz3wu@gmail.com
>>>> <ma...@gmail.com>> <stevenz3wu@gmail.com <mailto:
>>>> stevenz3wu@gmail.com>>
>>>>>>
>>>>>> 于2019年3月28日周四
>>>>>>
>>>>>> 下午12:52写道:
>>>>>>
>>>>>> This proposal mentioned that SplitEnumerator might run on the
>>>>>> JobManager or
>>>>>> in a single task on a TaskManager.
>>>>>>
>>>>>> if enumerator is a single task on a taskmanager, then the job
>>>>>>
>>>>>> DAG
>>>>>>
>>>>>> can
>>>>>>
>>>>>> never
>>>>>> been embarrassingly parallel anymore. That will nullify the
>>>>>>
>>>>>> leverage
>>>>>>
>>>>>> of
>>>>>>
>>>>>> fine-grained recovery for embarrassingly parallel jobs.
>>>>>>
>>>>>> It's not clear to me what's the implication of running
>>>>>>
>>>>>> enumerator
>>>>>>
>>>>>> on
>>>>>>
>>>>>> the
>>>>>>
>>>>>> jobmanager. So I will leave that out for now.
>>>>>>
>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com
>> <mailto:
>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
>> <
>>>>>>
>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>> Hi Stephan & Piotrek,
>>>>>>
>>>>>> Thank you for feedback.
>>>>>>
>>>>>> It seems that there are a lot of things to do in community.
>>>>>>
>>>>>> I
>>>>>>
>>>>>> am
>>>>>>
>>>>>> just
>>>>>>
>>>>>> afraid that this discussion may be forgotten since there so
>>>>>>
>>>>>> many
>>>>>>
>>>>>> proposals
>>>>>>
>>>>>> recently.
>>>>>> Anyway, wish to see the split topics soon :)
>>>>>>
>>>>>> Piotr Nowojski <piotr@da-platform.com <mailto:piotr@da-platform.com
>>>>
>>> <
>>>> piotr@da-platform.com <ma...@da-platform.com>> <
>>>> piotr@da-platform.com <ma...@da-platform.com>> <
>>>> piotr@da-platform.com <ma...@da-platform.com>>
>>>>>>
>>>>>> 于2019年1月24日周四
>>>>>>
>>>>>> 下午8:21写道:
>>>>>>
>>>>>> Hi Biao!
>>>>>>
>>>>>> This discussion was stalled because of preparations for
>>>>>>
>>>>>> the
>>>>>>
>>>>>> open
>>>>>>
>>>>>> sourcing
>>>>>>
>>>>>> & merging Blink. I think before creating the tickets we
>>>>>>
>>>>>> should
>>>>>>
>>>>>> split this
>>>>>>
>>>>>> discussion into topics/areas outlined by Stephan and
>>>>>>
>>>>>> create
>>>>>>
>>>>>> Flips
>>>>>>
>>>>>> for
>>>>>>
>>>>>> that.
>>>>>>
>>>>>> I think there is no chance for this to be completed in
>>>>>>
>>>>>> couple
>>>>>>
>>>>>> of
>>>>>>
>>>>>> remaining
>>>>>>
>>>>>> weeks/1 month before 1.8 feature freeze, however it would
>>>>>>
>>>>>> be
>>>>>>
>>>>>> good
>>>>>>
>>>>>> to aim
>>>>>>
>>>>>> with those changes for 1.9.
>>>>>>
>>>>>> Piotrek
>>>>>>
>>>>>>
>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com <mailto:
>>>> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
>> <
>>>>>>
>>>>>> mmyy1110@gmail.com <ma...@gmail.com>>
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>> Hi community,
>>>>>> The summary of Stephan makes a lot sense to me. It is
>>>>>>
>>>>>> much
>>>>>>
>>>>>> clearer
>>>>>>
>>>>>> indeed
>>>>>>
>>>>>> after splitting the complex topic into small ones.
>>>>>> I was wondering is there any detail plan for next step?
>>>>>>
>>>>>> If
>>>>>>
>>>>>> not,
>>>>>>
>>>>>> I
>>>>>>
>>>>>> would
>>>>>>
>>>>>> like to push this thing forward by creating some JIRA
>>>>>>
>>>>>> issues.
>>>>>>
>>>>>> Another question is that should version 1.8 include
>>>>>>
>>>>>> these
>>>>>>
>>>>>> features?
>>>>>>
>>>>>> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <
>>>> sewen@apache.org <ma...@apache.org>> <sewen@apache.org <mailto:
>>>> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>>
>>>> 于2018年12月1日周六
>>>>>>
>>>>>> 上午4:20写道:
>>>>>>
>>>>>> Thanks everyone for the lively discussion. Let me try
>>>>>>
>>>>>> to
>>>>>>
>>>>>> summarize
>>>>>>
>>>>>> where I
>>>>>>
>>>>>> see convergence in the discussion and open issues.
>>>>>> I'll try to group this by design aspect of the source.
>>>>>>
>>>>>> Please
>>>>>>
>>>>>> let me
>>>>>>
>>>>>> know
>>>>>>
>>>>>> if I got things wrong or missed something crucial here.
>>>>>>
>>>>>> For issues 1-3, if the below reflects the state of the
>>>>>>
>>>>>> discussion, I
>>>>>>
>>>>>> would
>>>>>>
>>>>>> try and update the FLIP in the next days.
>>>>>> For the remaining ones we need more discussion.
>>>>>>
>>>>>> I would suggest to fork each of these aspects into a
>>>>>>
>>>>>> separate
>>>>>>
>>>>>> mail
>>>>>>
>>>>>> thread,
>>>>>>
>>>>>> or will loose sight of the individual aspects.
>>>>>>
>>>>>> *(1) Separation of Split Enumerator and Split Reader*
>>>>>>
>>>>>> - All seem to agree this is a good thing
>>>>>> - Split Enumerator could in the end live on JobManager
>>>>>>
>>>>>> (and
>>>>>>
>>>>>> assign
>>>>>>
>>>>>> splits
>>>>>>
>>>>>> via RPC) or in a task (and assign splits via data
>>>>>>
>>>>>> streams)
>>>>>>
>>>>>> - this discussion is orthogonal and should come later,
>>>>>>
>>>>>> when
>>>>>>
>>>>>> the
>>>>>>
>>>>>> interface
>>>>>>
>>>>>> is agreed upon.
>>>>>>
>>>>>> *(2) Split Readers for one or more splits*
>>>>>>
>>>>>> - Discussion seems to agree that we need to support
>>>>>>
>>>>>> one
>>>>>>
>>>>>> reader
>>>>>>
>>>>>> that
>>>>>>
>>>>>> possibly handles multiple splits concurrently.
>>>>>> - The requirement comes from sources where one
>>>>>>
>>>>>> poll()-style
>>>>>>
>>>>>> call
>>>>>>
>>>>>> fetches
>>>>>>
>>>>>> data from different splits / partitions
>>>>>>    --> example sources that require that would be for
>>>>>>
>>>>>> example
>>>>>>
>>>>>> Kafka,
>>>>>>
>>>>>> Pravega, Pulsar
>>>>>>
>>>>>> - Could have one split reader per source, or multiple
>>>>>>
>>>>>> split
>>>>>>
>>>>>> readers
>>>>>>
>>>>>> that
>>>>>>
>>>>>> share the "poll()" function
>>>>>> - To not make it too complicated, we can start with
>>>>>>
>>>>>> thinking
>>>>>>
>>>>>> about
>>>>>>
>>>>>> one
>>>>>>
>>>>>> split reader for all splits initially and see if that
>>>>>>
>>>>>> covers
>>>>>>
>>>>>> all
>>>>>>
>>>>>> requirements
>>>>>>
>>>>>> *(3) Threading model of the Split Reader*
>>>>>>
>>>>>> - Most active part of the discussion ;-)
>>>>>>
>>>>>> - A non-blocking way for Flink's task code to interact
>>>>>>
>>>>>> with
>>>>>>
>>>>>> the
>>>>>>
>>>>>> source
>>>>>>
>>>>>> is
>>>>>>
>>>>>> needed in order to a task runtime code based on a
>>>>>> single-threaded/actor-style task design
>>>>>>    --> I personally am a big proponent of that, it will
>>>>>>
>>>>>> help
>>>>>>
>>>>>> with
>>>>>>
>>>>>> well-behaved checkpoints, efficiency, and simpler yet
>>>>>>
>>>>>> more
>>>>>>
>>>>>> robust
>>>>>>
>>>>>> runtime
>>>>>>
>>>>>> code
>>>>>>
>>>>>> - Users care about simple abstraction, so as a
>>>>>>
>>>>>> subclass
>>>>>>
>>>>>> of
>>>>>>
>>>>>> SplitReader
>>>>>>
>>>>>> (non-blocking / async) we need to have a
>>>>>>
>>>>>> BlockingSplitReader
>>>>>>
>>>>>> which
>>>>>>
>>>>>> will
>>>>>>
>>>>>> form the basis of most source implementations.
>>>>>>
>>>>>> BlockingSplitReader
>>>>>>
>>>>>> lets
>>>>>>
>>>>>> users do blocking simple poll() calls.
>>>>>> - The BlockingSplitReader would spawn a thread (or
>>>>>>
>>>>>> more)
>>>>>>
>>>>>> and
>>>>>>
>>>>>> the
>>>>>>
>>>>>> thread(s) can make blocking calls and hand over data
>>>>>>
>>>>>> buffers
>>>>>>
>>>>>> via
>>>>>>
>>>>>> a
>>>>>>
>>>>>> blocking
>>>>>>
>>>>>> queue
>>>>>> - This should allow us to cover both, a fully async
>>>>>>
>>>>>> runtime,
>>>>>>
>>>>>> and a
>>>>>>
>>>>>> simple
>>>>>>
>>>>>> blocking interface for users.
>>>>>> - This is actually very similar to how the Kafka
>>>>>>
>>>>>> connectors
>>>>>>
>>>>>> work.
>>>>>>
>>>>>> Kafka
>>>>>>
>>>>>> 9+ with one thread, Kafka 8 with multiple threads
>>>>>>
>>>>>> - On the base SplitReader (the async one), the
>>>>>>
>>>>>> non-blocking
>>>>>>
>>>>>> method
>>>>>>
>>>>>> that
>>>>>>
>>>>>> gets the next chunk of data would signal data
>>>>>>
>>>>>> availability
>>>>>>
>>>>>> via
>>>>>>
>>>>>> a
>>>>>>
>>>>>> CompletableFuture, because that gives the best
>>>>>>
>>>>>> flexibility
>>>>>>
>>>>>> (can
>>>>>>
>>>>>> await
>>>>>>
>>>>>> completion or register notification handlers).
>>>>>> - The source task would register a "thenHandle()" (or
>>>>>>
>>>>>> similar)
>>>>>>
>>>>>> on the
>>>>>>
>>>>>> future to put a "take next data" task into the
>>>>>>
>>>>>> actor-style
>>>>>>
>>>>>> mailbox
>>>>>>
>>>>>> *(4) Split Enumeration and Assignment*
>>>>>>
>>>>>> - Splits may be generated lazily, both in cases where
>>>>>>
>>>>>> there
>>>>>>
>>>>>> is a
>>>>>>
>>>>>> limited
>>>>>>
>>>>>> number of splits (but very many), or splits are
>>>>>>
>>>>>> discovered
>>>>>>
>>>>>> over
>>>>>>
>>>>>> time
>>>>>>
>>>>>> - Assignment should also be lazy, to get better load
>>>>>>
>>>>>> balancing
>>>>>>
>>>>>> - Assignment needs support locality preferences
>>>>>>
>>>>>> - Possible design based on discussion so far:
>>>>>>
>>>>>>    --> SplitReader has a method "addSplits(SplitT...)"
>>>>>>
>>>>>> to
>>>>>>
>>>>>> add
>>>>>>
>>>>>> one or
>>>>>>
>>>>>> more
>>>>>>
>>>>>> splits. Some split readers might assume they have only
>>>>>>
>>>>>> one
>>>>>>
>>>>>> split
>>>>>>
>>>>>> ever,
>>>>>>
>>>>>> concurrently, others assume multiple splits. (Note:
>>>>>>
>>>>>> idea
>>>>>>
>>>>>> behind
>>>>>>
>>>>>> being
>>>>>>
>>>>>> able
>>>>>>
>>>>>> to add multiple splits at the same time is to ease
>>>>>>
>>>>>> startup
>>>>>>
>>>>>> where
>>>>>>
>>>>>> multiple
>>>>>>
>>>>>> splits may be assigned instantly.)
>>>>>>    --> SplitReader has a context object on which it can
>>>>>>
>>>>>> call
>>>>>>
>>>>>> indicate
>>>>>>
>>>>>> when
>>>>>>
>>>>>> splits are completed. The enumerator gets that
>>>>>>
>>>>>> notification and
>>>>>>
>>>>>> can
>>>>>>
>>>>>> use
>>>>>>
>>>>>> to
>>>>>>
>>>>>> decide when to assign new splits. This should help both
>>>>>>
>>>>>> in
>>>>>>
>>>>>> cases
>>>>>>
>>>>>> of
>>>>>>
>>>>>> sources
>>>>>>
>>>>>> that take splits lazily (file readers) and in case the
>>>>>>
>>>>>> source
>>>>>>
>>>>>> needs to
>>>>>>
>>>>>> preserve a partial order between splits (Kinesis,
>>>>>>
>>>>>> Pravega,
>>>>>>
>>>>>> Pulsar may
>>>>>>
>>>>>> need
>>>>>>
>>>>>> that).
>>>>>>    --> SplitEnumerator gets notification when
>>>>>>
>>>>>> SplitReaders
>>>>>>
>>>>>> start
>>>>>>
>>>>>> and
>>>>>>
>>>>>> when
>>>>>>
>>>>>> they finish splits. They can decide at that moment to
>>>>>>
>>>>>> push
>>>>>>
>>>>>> more
>>>>>>
>>>>>> splits
>>>>>>
>>>>>> to
>>>>>>
>>>>>> that reader
>>>>>>    --> The SplitEnumerator should probably be aware of
>>>>>>
>>>>>> the
>>>>>>
>>>>>> source
>>>>>>
>>>>>> parallelism, to build its initial distribution.
>>>>>>
>>>>>> - Open question: Should the source expose something
>>>>>>
>>>>>> like
>>>>>>
>>>>>> "host
>>>>>>
>>>>>> preferences", so that yarn/mesos/k8s can take this into
>>>>>>
>>>>>> account
>>>>>>
>>>>>> when
>>>>>>
>>>>>> selecting a node to start a TM on?
>>>>>>
>>>>>> *(5) Watermarks and event time alignment*
>>>>>>
>>>>>> - Watermark generation, as well as idleness, needs to
>>>>>>
>>>>>> be
>>>>>>
>>>>>> per
>>>>>>
>>>>>> split
>>>>>>
>>>>>> (like
>>>>>>
>>>>>> currently in the Kafka Source, per partition)
>>>>>> - It is desirable to support optional
>>>>>>
>>>>>> event-time-alignment,
>>>>>>
>>>>>> meaning
>>>>>>
>>>>>> that
>>>>>>
>>>>>> splits that are ahead are back-pressured or temporarily
>>>>>>
>>>>>> unsubscribed
>>>>>>
>>>>>> - I think i would be desirable to encapsulate
>>>>>>
>>>>>> watermark
>>>>>>
>>>>>> generation
>>>>>>
>>>>>> logic
>>>>>>
>>>>>> in watermark generators, for a separation of concerns.
>>>>>>
>>>>>> The
>>>>>>
>>>>>> watermark
>>>>>>
>>>>>> generators should run per split.
>>>>>> - Using watermark generators would also help with
>>>>>>
>>>>>> another
>>>>>>
>>>>>> problem of
>>>>>>
>>>>>> the
>>>>>>
>>>>>> suggested interface, namely supporting non-periodic
>>>>>>
>>>>>> watermarks
>>>>>>
>>>>>> efficiently.
>>>>>>
>>>>>> - Need a way to "dispatch" next record to different
>>>>>>
>>>>>> watermark
>>>>>>
>>>>>> generators
>>>>>>
>>>>>> - Need a way to tell SplitReader to "suspend" a split
>>>>>>
>>>>>> until a
>>>>>>
>>>>>> certain
>>>>>>
>>>>>> watermark is reached (event time backpressure)
>>>>>> - This would in fact be not needed (and thus simpler)
>>>>>>
>>>>>> if
>>>>>>
>>>>>> we
>>>>>>
>>>>>> had
>>>>>>
>>>>>> a
>>>>>>
>>>>>> SplitReader per split and may be a reason to re-open
>>>>>>
>>>>>> that
>>>>>>
>>>>>> discussion
>>>>>>
>>>>>> *(6) Watermarks across splits and in the Split
>>>>>>
>>>>>> Enumerator*
>>>>>>
>>>>>> - The split enumerator may need some watermark
>>>>>>
>>>>>> awareness,
>>>>>>
>>>>>> which
>>>>>>
>>>>>> should
>>>>>>
>>>>>> be
>>>>>>
>>>>>> purely based on split metadata (like create timestamp
>>>>>>
>>>>>> of
>>>>>>
>>>>>> file
>>>>>>
>>>>>> splits)
>>>>>>
>>>>>> - If there are still more splits with overlapping
>>>>>>
>>>>>> event
>>>>>>
>>>>>> time
>>>>>>
>>>>>> range
>>>>>>
>>>>>> for
>>>>>>
>>>>>> a
>>>>>>
>>>>>> split reader, then that split reader should not advance
>>>>>>
>>>>>> the
>>>>>>
>>>>>> watermark
>>>>>>
>>>>>> within the split beyond the overlap boundary. Otherwise
>>>>>>
>>>>>> future
>>>>>>
>>>>>> splits
>>>>>>
>>>>>> will
>>>>>>
>>>>>> produce late data.
>>>>>>
>>>>>> - One way to approach this could be that the split
>>>>>>
>>>>>> enumerator
>>>>>>
>>>>>> may
>>>>>>
>>>>>> send
>>>>>>
>>>>>> watermarks to the readers, and the readers cannot emit
>>>>>>
>>>>>> watermarks
>>>>>>
>>>>>> beyond
>>>>>>
>>>>>> that received watermark.
>>>>>> - Many split enumerators would simply immediately send
>>>>>>
>>>>>> Long.MAX
>>>>>>
>>>>>> out
>>>>>>
>>>>>> and
>>>>>>
>>>>>> leave the progress purely to the split readers.
>>>>>>
>>>>>> - For event-time alignment / split back pressure, this
>>>>>>
>>>>>> begs
>>>>>>
>>>>>> the
>>>>>>
>>>>>> question
>>>>>>
>>>>>> how we can avoid deadlocks that may arise when splits
>>>>>>
>>>>>> are
>>>>>>
>>>>>> suspended
>>>>>>
>>>>>> for
>>>>>>
>>>>>> event time back pressure,
>>>>>>
>>>>>> *(7) Batch and streaming Unification*
>>>>>>
>>>>>> - Functionality wise, the above design should support
>>>>>>
>>>>>> both
>>>>>>
>>>>>> - Batch often (mostly) does not care about reading "in
>>>>>>
>>>>>> order"
>>>>>>
>>>>>> and
>>>>>>
>>>>>> generating watermarks
>>>>>>    --> Might use different enumerator logic that is
>>>>>>
>>>>>> more
>>>>>>
>>>>>> locality
>>>>>>
>>>>>> aware
>>>>>>
>>>>>> and ignores event time order
>>>>>>    --> Does not generate watermarks
>>>>>> - Would be great if bounded sources could be
>>>>>>
>>>>>> identified
>>>>>>
>>>>>> at
>>>>>>
>>>>>> compile
>>>>>>
>>>>>> time,
>>>>>>
>>>>>> so that "env.addBoundedSource(...)" is type safe and
>>>>>>
>>>>>> can
>>>>>>
>>>>>> return a
>>>>>>
>>>>>> "BoundedDataStream".
>>>>>> - Possible to defer this discussion until later
>>>>>>
>>>>>> *Miscellaneous Comments*
>>>>>>
>>>>>> - Should the source have a TypeInformation for the
>>>>>>
>>>>>> produced
>>>>>>
>>>>>> type,
>>>>>>
>>>>>> instead
>>>>>>
>>>>>> of a serializer? We need a type information in the
>>>>>>
>>>>>> stream
>>>>>>
>>>>>> anyways, and
>>>>>>
>>>>>> can
>>>>>>
>>>>>> derive the serializer from that. Plus, creating the
>>>>>>
>>>>>> serializer
>>>>>>
>>>>>> should
>>>>>>
>>>>>> respect the ExecutionConfig.
>>>>>>
>>>>>> - The TypeSerializer interface is very powerful but
>>>>>>
>>>>>> also
>>>>>>
>>>>>> not
>>>>>>
>>>>>> easy to
>>>>>>
>>>>>> implement. Its purpose is to handle data super
>>>>>>
>>>>>> efficiently,
>>>>>>
>>>>>> support
>>>>>>
>>>>>> flexible ways of evolution, etc.
>>>>>> For metadata I would suggest to look at the
>>>>>>
>>>>>> SimpleVersionedSerializer
>>>>>>
>>>>>> instead, which is used for example for checkpoint
>>>>>>
>>>>>> master
>>>>>>
>>>>>> hooks,
>>>>>>
>>>>>> or for
>>>>>>
>>>>>> the
>>>>>>
>>>>>> streaming file sink. I think that is is a good match
>>>>>>
>>>>>> for
>>>>>>
>>>>>> cases
>>>>>>
>>>>>> where
>>>>>>
>>>>>> we
>>>>>>
>>>>>> do
>>>>>>
>>>>>> not need more than ser/deser (no copy, etc.) and don't
>>>>>>
>>>>>> need to
>>>>>>
>>>>>> push
>>>>>>
>>>>>> versioning out of the serialization paths for best
>>>>>>
>>>>>> performance
>>>>>>
>>>>>> (as in
>>>>>>
>>>>>> the
>>>>>>
>>>>>> TypeSerializer)
>>>>>>
>>>>>>
>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>>>>>>
>>>>>> k.kloudas@data-artisans.com>
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>>
>>>>>> Hi Biao,
>>>>>>
>>>>>> Thanks for the answer!
>>>>>>
>>>>>> So given the multi-threaded readers, now we have as
>>>>>>
>>>>>> open
>>>>>>
>>>>>> questions:
>>>>>>
>>>>>> 1) How do we let the checkpoints pass through our
>>>>>>
>>>>>> multi-threaded
>>>>>>
>>>>>> reader
>>>>>>
>>>>>> operator?
>>>>>>
>>>>>> 2) Do we have separate reader and source operators or
>>>>>>
>>>>>> not? In
>>>>>>
>>>>>> the
>>>>>>
>>>>>> strategy
>>>>>>
>>>>>> that has a separate source, the source operator has a
>>>>>>
>>>>>> parallelism of
>>>>>>
>>>>>> 1
>>>>>>
>>>>>> and
>>>>>>
>>>>>> is responsible for split recovery only.
>>>>>>
>>>>>> For the first one, given also the constraints
>>>>>>
>>>>>> (blocking,
>>>>>>
>>>>>> finite
>>>>>>
>>>>>> queues,
>>>>>>
>>>>>> etc), I do not have an answer yet.
>>>>>>
>>>>>> For the 2nd, I think that we should go with separate
>>>>>>
>>>>>> operators
>>>>>>
>>>>>> for
>>>>>>
>>>>>> the
>>>>>>
>>>>>> source and the readers, for the following reasons:
>>>>>>
>>>>>> 1) This is more aligned with a potential future
>>>>>>
>>>>>> improvement
>>>>>>
>>>>>> where the
>>>>>>
>>>>>> split
>>>>>>
>>>>>> discovery becomes a responsibility of the JobManager
>>>>>>
>>>>>> and
>>>>>>
>>>>>> readers are
>>>>>>
>>>>>> pooling more work from the JM.
>>>>>>
>>>>>> 2) The source is going to be the "single point of
>>>>>>
>>>>>> truth".
>>>>>>
>>>>>> It
>>>>>>
>>>>>> will
>>>>>>
>>>>>> know
>>>>>>
>>>>>> what
>>>>>>
>>>>>> has been processed and what not. If the source and the
>>>>>>
>>>>>> readers
>>>>>>
>>>>>> are a
>>>>>>
>>>>>> single
>>>>>>
>>>>>> operator with parallelism > 1, or in general, if the
>>>>>>
>>>>>> split
>>>>>>
>>>>>> discovery
>>>>>>
>>>>>> is
>>>>>>
>>>>>> done by each task individually, then:
>>>>>>   i) we have to have a deterministic scheme for each
>>>>>>
>>>>>> reader to
>>>>>>
>>>>>> assign
>>>>>>
>>>>>> splits to itself (e.g. mod subtaskId). This is not
>>>>>>
>>>>>> necessarily
>>>>>>
>>>>>> trivial
>>>>>>
>>>>>> for
>>>>>>
>>>>>> all sources.
>>>>>>   ii) each reader would have to keep a copy of all its
>>>>>>
>>>>>> processed
>>>>>>
>>>>>> slpits
>>>>>>
>>>>>>   iii) the state has to be a union state with a
>>>>>>
>>>>>> non-trivial
>>>>>>
>>>>>> merging
>>>>>>
>>>>>> logic
>>>>>>
>>>>>> in order to support rescaling.
>>>>>>
>>>>>> Two additional points that you raised above:
>>>>>>
>>>>>> i) The point that you raised that we need to keep all
>>>>>>
>>>>>> splits
>>>>>>
>>>>>> (processed
>>>>>>
>>>>>> and
>>>>>>
>>>>>> not-processed) I think is a bit of a strong
>>>>>>
>>>>>> requirement.
>>>>>>
>>>>>> This
>>>>>>
>>>>>> would
>>>>>>
>>>>>> imply
>>>>>>
>>>>>> that for infinite sources the state will grow
>>>>>>
>>>>>> indefinitely.
>>>>>>
>>>>>> This is
>>>>>>
>>>>>> problem
>>>>>>
>>>>>> is even more pronounced if we do not have a single
>>>>>>
>>>>>> source
>>>>>>
>>>>>> that
>>>>>>
>>>>>> assigns
>>>>>>
>>>>>> splits to readers, as each reader will have its own
>>>>>>
>>>>>> copy
>>>>>>
>>>>>> of
>>>>>>
>>>>>> the
>>>>>>
>>>>>> state.
>>>>>>
>>>>>> ii) it is true that for finite sources we need to
>>>>>>
>>>>>> somehow
>>>>>>
>>>>>> not
>>>>>>
>>>>>> close
>>>>>>
>>>>>> the
>>>>>>
>>>>>> readers when the source/split discoverer finishes. The
>>>>>> ContinuousFileReaderOperator has a work-around for
>>>>>>
>>>>>> that.
>>>>>>
>>>>>> It is
>>>>>>
>>>>>> not
>>>>>>
>>>>>> elegant,
>>>>>>
>>>>>> and checkpoints are not emitted after closing the
>>>>>>
>>>>>> source,
>>>>>>
>>>>>> but
>>>>>>
>>>>>> this, I
>>>>>>
>>>>>> believe, is a bigger problem which requires more
>>>>>>
>>>>>> changes
>>>>>>
>>>>>> than
>>>>>>
>>>>>> just
>>>>>>
>>>>>> refactoring the source interface.
>>>>>>
>>>>>> Cheers,
>>>>>> Kostas
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> Best, Jingsong Lee
>>>>
>>>>
>>>
>>
>>
>> --
>> Best, Jingsong Lee
>>
> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Dawid and Jark,

I think the discussion ultimately boils down to the question that which one
of the following two final states do we want? Once we make this decision,
everything else can be naturally derived.

*Final state 1*: Separate API for bounded / unbounded DataStream & Table.
That means any code users write will be valid at the point when they write
the code. This is similar to having type safety check at programming time.
For example,

BoundedDataStream extends DataStream {
// Operations only available for bounded data.
BoundedDataStream sort(...);

// Interaction with another BoundedStream returns a Bounded stream.
BoundedJoinedDataStream join(BoundedDataStream other)

// Interaction with another unbounded stream returns an unbounded stream.
JoinedDataStream join(DataStream other)
}

BoundedTable extends Table {
  // Bounded only operation.
BoundedTable sort(...);

// Interaction with another BoundedTable returns a BoundedTable.
BoundedTable join(BoundedTable other)

// Interaction with another unbounded table returns an unbounded table.
Table join(Table other)
}

*Final state 2*: One unified API for bounded / unbounded DataStream /
Table.
That unified API may throw exception at DAG compilation time if an invalid
operation is tried. This is what Table API currently follows.

DataStream {
// Throws exception if the DataStream is unbounded.
DataStream sort();
// Get boundedness.
Boundedness getBoundedness();
}

Table {
// Throws exception if the table has infinite rows.
Table orderBy();

// Get boundedness.
Boundedness getBoundedness();
}

From what I understand, there is no consensus so far on this decision yet.
Whichever final state we choose, we need to make it consistent across the
entire project. We should avoid the case that Table follows one final state
while DataStream follows another. Some arguments I am aware of from both
sides so far are following:

Arguments for final state 1:
1a) Clean API with method safety check at programming time.
1b) (Counter 2b) Although SQL does not have programming time error check, SQL
is not really a "programming language" per se. So SQL can be different from
Table and DataStream.
1c)  Although final state 2 seems making it easier for SQL to use given it
is more "config based" than "parameter based", final state 1 can probably
also meet what SQL wants by wrapping the Source in TableSource /
TableSourceFactory API if needed.

Arguments for final state 2:
2a) The Source API itself seems already sort of following the unified API
pattern.
2b) There is no "programming time" method error check in SQL case, so we
cannot really achieve final state 1 across the board.
2c) It is an easier path given our current status, i.e. Table is already
following final state 2.
2d) Users can always explicitly check the boundedness if they want to.

As I mentioned earlier, my initial thought was also to have a
"configuration based" Source rather than a "parameter based" Source. So it
is completely possible that I missed some important consideration or design
principles that we want to enforce for the project. It would be good
if @Stephan
Ewen <st...@ververica.com> and @Aljoscha Krettek <al...@ververica.com> can
also provide more thoughts on this.


Re: Jingsong

As you said, there are some batched system source, like parquet/orc source.
> Could we have the batch emit interface to improve performance? The queue of
> per record may cause performance degradation.


The current interface does not necessarily cause performance problem in a
multi-threading case. In fact, the base implementation allows SplitReaders
to add a batch <E> of records<T> to the records queue<E>, so each element
in the records queue would be a batch <E>. In this case, when the main
thread polls records, it will take a batch <E> of records <T> from the
shared records queue and process the records <T> in a batch manner.

Thanks,

Jiangjie (Becket) Qin

On Thu, Dec 12, 2019 at 1:29 PM Jingsong Li <ji...@gmail.com> wrote:

> Hi Becket,
>
> I also have some performance concerns too.
>
> If I understand correctly, SourceOutput will emit data per record into the
> queue? I'm worried about the multithreading performance of this queue.
>
> > One example is some batched messaging systems which only have an offset
> for the entire batch instead of individual messages in the batch.
>
> As you said, there are some batched system source, like parquet/orc source.
> Could we have the batch emit interface to improve performance? The queue of
> per record may cause performance degradation.
>
> Best,
> Jingsong Lee
>
> On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com> wrote:
>
> > Hi Becket,
> >
> > I think Dawid explained things clearly and makes a lot of sense.
> > I'm also in favor of #2, because #1 doesn't work for our future unified
> > envrionment.
> >
> > You can see the vision in this documentation [1]. In the future, we would
> > like to
> > drop the global streaming/batch mode in SQL (i.e.
> > EnvironmentSettings#inStreamingMode/inBatchMode).
> > A source is bounded or unbounded once defined, so queries can be inferred
> > from source to run
> > in streaming or batch or hybrid mode. However, in #1, we will lose this
> > ability because the framework
> > doesn't know whether the source is bounded or unbounded.
> >
> > Best,
> > Jark
> >
> >
> > [1]:
> >
> >
> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
> >
> > On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <pi...@ververica.com>
> wrote:
> >
> > > Hi,
> > >
> > > Regarding the:
> > >
> > > Collection<E> getNextRecords()
> > >
> > > I’m pretty sure such design would unfortunately impact the performance
> > > (accessing and potentially creating the collection on the hot path).
> > >
> > > Also the
> > >
> > > InputStatus emitNext(DataOutput<T> output) throws Exception;
> > > or
> > > Status pollNext(SourceOutput<T> sourceOutput) throws Exception;
> > >
> > > Gives us some opportunities in the future, to allow Source hot looping
> > > inside, until it receives some signal “please exit because of some
> > reasons”
> > > (output collector could return such hint upon collecting the result).
> But
> > > that’s another topic outside of this FLIP’s scope.
> > >
> > > Piotrek
> > >
> > > > On 11 Dec 2019, at 10:41, Till Rohrmann <tr...@apache.org>
> wrote:
> > > >
> > > > Hi Becket,
> > > >
> > > > quick clarification from my side because I think you misunderstood my
> > > > question. I did not suggest to let the SourceReader return only a
> > single
> > > > record at a time when calling getNextRecords. As the return type
> > > indicates,
> > > > the method can return an arbitrary number of records.
> > > >
> > > > Cheers,
> > > > Till
> > > >
> > > > On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> > > dwysakowicz@apache.org <ma...@apache.org>>
> > > > wrote:
> > > >
> > > >> Hi Becket,
> > > >>
> > > >> Issue #1 - Design of Source interface
> > > >>
> > > >> I mentioned the lack of a method like
> > > Source#createEnumerator(Boundedness
> > > >> boundedness, SplitEnumeratorContext context), because without the
> > > current
> > > >> proposal is not complete/does not work.
> > > >>
> > > >> If we say that boundedness is an intrinsic property of a source imo
> we
> > > >> don't need the Source#createEnumerator(Boundedness boundedness,
> > > >> SplitEnumeratorContext context) method.
> > > >>
> > > >> Assuming a source from my previous example:
> > > >>
> > > >> Source source = KafkaSource.builder()
> > > >>  ...
> > > >>  .untilTimestamp(...)
> > > >>  .build()
> > > >>
> > > >> Would the enumerator differ if created like
> > > >> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
> > > >> .createEnumerator(BOUNDED, ...)? I know I am repeating myself, but
> > this
> > > is
> > > >> the part that my opinion differ the most from the current proposal.
> I
> > > >> really think it should always be the source that tells if it is
> > bounded
> > > or
> > > >> not. In the current proposal methods continousSource/boundedSource
> > > somewhat
> > > >> reconfigure the source, which I think is misleading.
> > > >>
> > > >> I think a call like:
> > > >>
> > > >> Source source = KafkaSource.builder()
> > > >>  ...
> > > >>  .readContinously() / readUntilLatestOffset() / readUntilTimestamp /
> > > readUntilOffsets / ...
> > > >>  .build()
> > > >>
> > > >> is way cleaner (and expressive) than
> > > >>
> > > >> Source source = KafkaSource.builder()
> > > >>  ...
> > > >>  .build()
> > > >>
> > > >>
> > > >> env.continousSource(source) // which actually underneath would call
> > > createEnumerator(CONTINUOUS, ctx) which would be equivalent to
> > > source.readContinously().createEnumerator(ctx)
> > > >> // or
> > > >> env.boundedSource(source) // which actually underneath would call
> > > createEnumerator(BOUNDED, ctx) which would be equivalent to
> > > source.readUntilLatestOffset().createEnumerator(ctx)
> > > >>
> > > >>
> > > >> Sorry for the comparison, but to me it seems there is too much magic
> > > >> happening underneath those two calls.
> > > >>
> > > >> I really believe the Source interface should have getBoundedness
> > method
> > > >> instead of (supportBoundedness) + createEnumerator(Boundedness, ...)
> > > >>
> > > >>
> > > >> Issue #2 - Design of
> > > >> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> > > >>
> > > >> As you might have guessed I am slightly in favor of option #2
> > modified.
> > > >> Yes I am aware every step of the dag would have to be able to say if
> > it
> > > is
> > > >> bounded or not. I have a feeling it would be easier to express cross
> > > >> bounded/unbounded operations, but I must admit I have not thought it
> > > >> through thoroughly, In the spirit of batch is just a special case of
> > > >> streaming I thought BoundedStream would extend from DataStream.
> > Correct
> > > me
> > > >> if I am wrong. In such a setup the cross bounded/unbounded operation
> > > could
> > > >> be expressed quite easily I think:
> > > >>
> > > >> DataStream {
> > > >>  DataStream join(DataStream, ...); // we could not really tell if
> the
> > > result is bounded or not, but because bounded stream is a special case
> of
> > > unbounded the API object is correct, irrespective if the left or right
> > side
> > > of the join is bounded
> > > >> }
> > > >>
> > > >> BoundedStream extends DataStream {
> > > >>  BoundedStream join(BoundedStream, ...); // only if both sides are
> > > bounded the result can be bounded as well. However we do have access to
> > the
> > > DataStream#join here, so you can still join with a DataStream
> > > >> }
> > > >>
> > > >>
> > > >> On the other hand I also see benefits of two completely disjointed
> > APIs,
> > > >> as we could prohibit some streaming calls in the bounded API. I
> can't
> > > think
> > > >> of any unbounded operators that could not be implemented for bounded
> > > stream.
> > > >>
> > > >> Besides I think we both agree we don't like the method:
> > > >>
> > > >> DataStream boundedStream(Source)
> > > >>
> > > >> suggested in the current state of the FLIP. Do we ? :)
> > > >>
> > > >> Best,
> > > >>
> > > >> Dawid
> > > >>
> > > >> On 10/12/2019 18:57, Becket Qin wrote:
> > > >>
> > > >> Hi folks,
> > > >>
> > > >> Thanks for the discussion, great feedback. Also thanks Dawid for the
> > > >> explanation, it is much clearer now.
> > > >>
> > > >> One thing that is indeed missing from the FLIP is how the
> boundedness
> > is
> > > >> passed to the Source implementation. So the API should be
> > > >> Source#createEnumerator(Boundedness boundedness,
> > SplitEnumeratorContext
> > > >> context)
> > > >> And we can probably remove the Source#supportBoundedness(Boundedness
> > > >> boundedness) method.
> > > >>
> > > >> Assuming we have that, we are essentially choosing from one of the
> > > >> following two options:
> > > >>
> > > >> Option 1:
> > > >> // The source is continuous source, and only unbounded operations
> can
> > be
> > > >> performed.
> > > >> DataStream<Type> datastream = env.continuousSource(someSource);
> > > >>
> > > >> // The source is bounded source, both bounded and unbounded
> operations
> > > can
> > > >> be performed.
> > > >> BoundedDataStream<Type> boundedDataStream =
> > > env.boundedSource(someSource);
> > > >>
> > > >>  - Pros:
> > > >>       a) explicit boundary between bounded / unbounded streams, it
> is
> > > >> quite simple and clear to the users.
> > > >>  - Cons:
> > > >>       a) For applications that do not involve bounded operations,
> they
> > > >> still have to call different API to distinguish bounded / unbounded
> > > streams.
> > > >>       b) No support for bounded stream to run in a streaming runtime
> > > >> setting, i.e. scheduling and operators behaviors.
> > > >>
> > > >>
> > > >> Option 2:
> > > >> // The source is either bounded or unbounded, but only unbounded
> > > operations
> > > >> could be performed on the returned DataStream.
> > > >> DataStream<Type> dataStream = env.source(someSource);
> > > >>
> > > >> // The source must be a bounded source, otherwise exception is
> thrown.
> > > >> BoundedDataStream<Type> boundedDataStream =
> > > >> env.boundedSource(boundedSource);
> > > >>
> > > >> The pros and cons are exactly the opposite of option 1.
> > > >>  - Pros:
> > > >>       a) For applications that do not involve bounded operations,
> they
> > > >> still have to call different API to distinguish bounded / unbounded
> > > streams.
> > > >>       b) Support for bounded stream to run in a streaming runtime
> > > setting,
> > > >> i.e. scheduling and operators behaviors.
> > > >>  - Cons:
> > > >>       a) Bounded / unbounded streams are kind of mixed, i.e. given a
> > > >> DataStream, it is not clear whether it is bounded or not, unless you
> > > have
> > > >> the access to its source.
> > > >>
> > > >>
> > > >> If we only think from the Source API perspective, option 2 seems a
> > > better
> > > >> choice because functionality wise it is a superset of option 1, at
> the
> > > cost
> > > >> of some seemingly acceptable ambiguity in the DataStream API.
> > > >> But if we look at the DataStream API as a whole, option 1 seems a
> > > clearer
> > > >> choice. For example, some times a library may have to know whether a
> > > >> certain task will finish or not. And it would be difficult to tell
> if
> > > the
> > > >> input is a DataStream, unless additional information is provided all
> > the
> > > >> way from the Source. One possible solution is to have a *modified
> > > option 2*
> > > >> which adds a method to the DataStream API to indicate boundedness,
> > such
> > > as
> > > >> getBoundedness(). It would solve the problem with a potential
> > confusion
> > > of
> > > >> what is difference between a DataStream with getBoundedness()=true
> > and a
> > > >> BoundedDataStream. But that seems not super difficult to explain.
> > > >>
> > > >> So from API's perspective, I don't have a strong opinion between
> > > *option 1*
> > > >> and *modified option 2. *I like the cleanness of option 1, but
> > modified
> > > >> option 2 would be more attractive if we have concrete use case for
> the
> > > >> "Bounded stream with unbounded streaming runtime settings".
> > > >>
> > > >> Re: Till
> > > >>
> > > >>
> > > >> Maybe this has already been asked before but I was wondering why the
> > > >> SourceReader interface has the method pollNext which hands the
> > > >> responsibility of outputting elements to the SourceReader
> > > implementation?
> > > >> Has this been done for backwards compatibility reasons with the old
> > > source
> > > >> interface? If not, then one could define a Collection<E>
> > > getNextRecords()
> > > >> method which returns the currently retrieved records and then the
> > caller
> > > >> emits them outside of the SourceReader. That way the interface would
> > not
> > > >> allow to implement an outputting loop where we never hand back
> control
> > > to
> > > >> the caller. At the moment, this contract can be easily broken and is
> > > only
> > > >> mentioned loosely in the JavaDocs.
> > > >>
> > > >>
> > > >> The primary reason we handover the SourceOutput to the SourceReader
> is
> > > >> because sometimes it is difficult for a SourceReader to emit one
> > record
> > > at
> > > >> a time. One example is some batched messaging systems which only
> have
> > an
> > > >> offset for the entire batch instead of individual messages in the
> > > batch. In
> > > >> that case, returning one record at a time would leave the
> SourceReader
> > > in
> > > >> an uncheckpointable state because they can only checkpoint at the
> > batch
> > > >> boundaries.
> > > >>
> > > >> Thanks,
> > > >>
> > > >> Jiangjie (Becket) Qin
> > > >>
> > > >> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <trohrmann@apache.org
> > > <ma...@apache.org>> <trohrmann@apache.org <mailto:
> > > trohrmann@apache.org>> wrote:
> > > >>
> > > >>
> > > >> Hi everyone,
> > > >>
> > > >> thanks for drafting this FLIP. It reads very well.
> > > >>
> > > >> Concerning Dawid's proposal, I tend to agree. The boundedness could
> > come
> > > >> from the source and tell the system how to treat the operator
> > > (scheduling
> > > >> wise). From a user's perspective it should be fine to get back a
> > > DataStream
> > > >> when calling env.source(boundedSource) if he does not need special
> > > >> operations defined on a BoundedDataStream. If he needs this, then
> one
> > > could
> > > >> use the method BoundedDataStream env.boundedSource(boundedSource).
> > > >>
> > > >> If possible, we could enforce the proper usage of
> env.boundedSource()
> > by
> > > >> introducing a BoundedSource type so that one cannot pass an
> > > >> unbounded source to it. That way users would not be able to shoot
> > > >> themselves in the foot.
> > > >>
> > > >> Maybe this has already been asked before but I was wondering why the
> > > >> SourceReader interface has the method pollNext which hands the
> > > >> responsibility of outputting elements to the SourceReader
> > > implementation?
> > > >> Has this been done for backwards compatibility reasons with the old
> > > source
> > > >> interface? If not, then one could define a Collection<E>
> > > getNextRecords()
> > > >> method which returns the currently retrieved records and then the
> > caller
> > > >> emits them outside of the SourceReader. That way the interface would
> > not
> > > >> allow to implement an outputting loop where we never hand back
> control
> > > to
> > > >> the caller. At the moment, this contract can be easily broken and is
> > > only
> > > >> mentioned loosely in the JavaDocs.
> > > >>
> > > >> Cheers,
> > > >> Till
> > > >>
> > > >> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <jingsonglee0@gmail.com
> > > <ma...@gmail.com>> <jingsonglee0@gmail.com <mailto:
> > > jingsonglee0@gmail.com>>
> > > >> wrote:
> > > >>
> > > >>
> > > >> Hi all,
> > > >>
> > > >> I think current design is good.
> > > >>
> > > >> My understanding is:
> > > >>
> > > >> For execution mode: bounded mode and continuous mode, It's totally
> > > >> different. I don't think we have the ability to integrate the two
> > models
> > > >>
> > > >> at
> > > >>
> > > >> present. It's about scheduling, memory, algorithms, States, etc. we
> > > >> shouldn't confuse them.
> > > >>
> > > >> For source capabilities: only bounded, only continuous, both bounded
> > and
> > > >> continuous.
> > > >> I think Kafka is a source that can be ran both bounded
> > > >> and continuous execution mode.
> > > >> And Kafka with end offset should be ran both bounded
> > > >> and continuous execution mode.  Using apache Beam with Flink
> runner, I
> > > >>
> > > >> used
> > > >>
> > > >> to run a "bounded" Kafka in streaming mode. For our previous
> > DataStream,
> > > >>
> > > >> it
> > > >>
> > > >> is not necessarily required that the source cannot be bounded.
> > > >>
> > > >> So it is my thought for Dawid's question:
> > > >> 1.pass a bounded source to continuousSource() +1
> > > >> 2.pass a continuous source to boundedSource() -1, should throw
> > > exception.
> > > >>
> > > >> In StreamExecutionEnvironment, continuousSource and boundedSource
> > define
> > > >> the execution mode. It defines a clear boundary of execution mode.
> > > >>
> > > >> Best,
> > > >> Jingsong Lee
> > > >>
> > > >> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com <mailto:
> > > imjark@gmail.com>> <imjark@gmail.com <ma...@gmail.com>> wrote:
> > > >>
> > > >>
> > > >> I agree with Dawid's point that the boundedness information should
> > come
> > > >> from the source itself (e.g. the end timestamp), not through
> > > >> env.boundedSouce()/continuousSource().
> > > >> I think if we want to support something like `env.source()` that
> > derive
> > > >>
> > > >> the
> > > >>
> > > >> execution mode from source, `supportsBoundedness(Boundedness)`
> > > >> method is not enough, because we don't know whether it is bounded or
> > > >>
> > > >> not.
> > > >>
> > > >> Best,
> > > >> Jark
> > > >>
> > > >>
> > > >> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <
> dwysakowicz@apache.org
> > > <ma...@apache.org>> <dwysakowicz@apache.org <mailto:
> > > dwysakowicz@apache.org>>
> > > >> wrote:
> > > >>
> > > >>
> > > >> One more thing. In the current proposal, with the
> > > >> supportsBoundedness(Boundedness) method and the boundedness coming
> > > >>
> > > >> from
> > > >>
> > > >> either continuousSource or boundedSource I could not find how this
> > > >> information is fed back to the SplitEnumerator.
> > > >>
> > > >> Best,
> > > >>
> > > >> Dawid
> > > >>
> > > >> On 09/12/2019 13:52, Becket Qin wrote:
> > > >>
> > > >> Hi Dawid,
> > > >>
> > > >> Thanks for the comments. This actually brings another relevant
> > > >>
> > > >> question
> > > >>
> > > >> about what does a "bounded source" imply. I actually had the same
> > > >> impression when I look at the Source API. Here is what I understand
> > > >>
> > > >> after
> > > >>
> > > >> some discussion with Stephan. The bounded source has the following
> > > >>
> > > >> impacts.
> > > >>
> > > >> 1. API validity.
> > > >> - A bounded source generates a bounded stream so some operations
> > > >>
> > > >> that
> > > >>
> > > >> only
> > > >>
> > > >> works for bounded records would be performed, e.g. sort.
> > > >> - To expose these bounded stream only APIs, there are two options:
> > > >>     a. Add them to the DataStream API and throw exception if a
> > > >>
> > > >> method
> > > >>
> > > >> is
> > > >>
> > > >> called on an unbounded stream.
> > > >>     b. Create a BoundedDataStream class which is returned from
> > > >> env.boundedSource(), while DataStream is returned from
> > > >>
> > > >> env.continousSource().
> > > >>
> > > >> Note that this cannot be done by having single
> > > >>
> > > >> env.source(theSource)
> > > >>
> > > >> even
> > > >>
> > > >> the Source has a getBoundedness() method.
> > > >>
> > > >> 2. Scheduling
> > > >> - A bounded source could be computed stage by stage without
> > > >>
> > > >> bringing
> > > >>
> > > >> up
> > > >>
> > > >> all
> > > >>
> > > >> the tasks at the same time.
> > > >>
> > > >> 3. Operator behaviors
> > > >> - A bounded source indicates the records are finite so some
> > > >>
> > > >> operators
> > > >>
> > > >> can
> > > >>
> > > >> wait until it receives all the records before it starts the
> > > >>
> > > >> processing.
> > > >>
> > > >> In the above impact, only 1 is relevant to the API design. And the
> > > >>
> > > >> current
> > > >>
> > > >> proposal in FLIP-27 is following 1.b.
> > > >>
> > > >> // boundedness depends of source property, imo this should always
> > > >>
> > > >> be
> > > >>
> > > >> preferred
> > > >>
> > > >>
> > > >> DataStream<MyType> stream = env.source(theSource);
> > > >>
> > > >>
> > > >> In your proposal, does DataStream have bounded stream only methods?
> > > >>
> > > >> It
> > > >>
> > > >> looks it should have, otherwise passing a bounded Source to
> > > >>
> > > >> env.source()
> > > >>
> > > >> would be confusing. In that case, we will essentially do 1.a if an
> > > >> unbounded Source is created from env.source(unboundedSource).
> > > >>
> > > >> If we have the methods only supported for bounded streams in
> > > >>
> > > >> DataStream,
> > > >>
> > > >> it
> > > >>
> > > >> seems a little weird to have a separate BoundedDataStream
> > > >>
> > > >> interface.
> > > >>
> > > >> Am I understand it correctly?
> > > >>
> > > >> Thanks,
> > > >>
> > > >> Jiangjie (Becket) Qin
> > > >>
> > > >>
> > > >>
> > > >> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> > > >>
> > > >> dwysakowicz@apache.org <ma...@apache.org>>
> > > >>
> > > >> wrote:
> > > >>
> > > >>
> > > >> Hi all,
> > > >>
> > > >> Really well written proposal and very important one. I must admit
> > > >>
> > > >> I
> > > >>
> > > >> have
> > > >>
> > > >> not understood all the intricacies of it yet.
> > > >>
> > > >> One question I have though is about where does the information
> > > >>
> > > >> about
> > > >>
> > > >> boundedness come from. I think in most cases it is a property of
> > > >>
> > > >> the
> > > >>
> > > >> source. As you described it might be e.g. end offset, a flag
> > > >>
> > > >> should
> > > >>
> > > >> it
> > > >>
> > > >> monitor new splits etc. I think it would be a really nice use case
> > > >>
> > > >> to
> > > >>
> > > >> be
> > > >>
> > > >> able to say:
> > > >>
> > > >> new KafkaSource().readUntil(long timestamp),
> > > >>
> > > >> which could work as an "end offset". Moreover I think all Bounded
> > > >>
> > > >> sources
> > > >>
> > > >> support continuous mode, but no intrinsically continuous source
> > > >>
> > > >> support
> > > >>
> > > >> the
> > > >>
> > > >> Bounded mode. If I understood the proposal correctly it suggest
> > > >>
> > > >> the
> > > >>
> > > >> boundedness sort of "comes" from the outside of the source, from
> > > >>
> > > >> the
> > > >>
> > > >> invokation of either boundedStream or continousSource.
> > > >>
> > > >> I am wondering if it would make sense to actually change the
> > > >>
> > > >> method
> > > >>
> > > >> boolean Source#supportsBoundedness(Boundedness)
> > > >>
> > > >> to
> > > >>
> > > >> Boundedness Source#getBoundedness().
> > > >>
> > > >> As for the methods #boundedSource, #continousSource, assuming the
> > > >> boundedness is property of the source they do not affect how the
> > > >>
> > > >> enumerator
> > > >>
> > > >> works, but mostly how the dag is scheduled, right? I am not
> > > >>
> > > >> against
> > > >>
> > > >> those
> > > >>
> > > >> methods, but I think it is a very specific use case to actually
> > > >>
> > > >> override
> > > >>
> > > >> the property of the source. In general I would expect users to
> > > >>
> > > >> only
> > > >>
> > > >> call
> > > >>
> > > >> env.source(theSource), where the source tells if it is bounded or
> > > >>
> > > >> not. I
> > > >>
> > > >> would suggest considering following set of methods:
> > > >>
> > > >> // boundedness depends of source property, imo this should always
> > > >>
> > > >> be
> > > >>
> > > >> preferred
> > > >>
> > > >> DataStream<MyType> stream = env.source(theSource);
> > > >>
> > > >>
> > > >> // always continous execution, whether bounded or unbounded source
> > > >>
> > > >> DataStream<MyType> boundedStream = env.continousSource(theSource);
> > > >>
> > > >> // imo this would make sense if the BoundedDataStream provides
> > > >>
> > > >> additional features unavailable for continous mode
> > > >>
> > > >> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
> > > >>
> > > >>
> > > >> Best,
> > > >>
> > > >> Dawid
> > > >>
> > > >>
> > > >> On 04/12/2019 11:25, Stephan Ewen wrote:
> > > >>
> > > >> Thanks, Becket, for updating this.
> > > >>
> > > >> I agree with moving the aspects you mentioned into separate FLIPs
> > > >>
> > > >> -
> > > >>
> > > >> this
> > > >>
> > > >> one way becoming unwieldy in size.
> > > >>
> > > >> +1 to the FLIP in its current state. Its a very detailed write-up,
> > > >>
> > > >> nicely
> > > >>
> > > >> done!
> > > >>
> > > >> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <becket.qin@gmail.com
> > > <ma...@gmail.com>> <becket.qin@gmail.com <mailto:
> > > becket.qin@gmail.com>>
> > > >>
> > > >> <
> > > >>
> > > >> becket.qin@gmail.com <ma...@gmail.com>> wrote:
> > > >>
> > > >> Hi all,
> > > >>
> > > >> Sorry for the long belated update. I have updated FLIP-27 wiki
> > > >>
> > > >> page
> > > >>
> > > >> with
> > > >>
> > > >> the latest proposals. Some noticeable changes include:
> > > >> 1. A new generic communication mechanism between SplitEnumerator
> > > >>
> > > >> and
> > > >>
> > > >> SourceReader.
> > > >> 2. Some detail API method signature changes.
> > > >>
> > > >> We left a few things out of this FLIP and will address them in
> > > >>
> > > >> separate
> > > >>
> > > >> FLIPs. Including:
> > > >> 1. Per split event time.
> > > >> 2. Event time alignment.
> > > >> 3. Fine grained failover for SplitEnumerator failure.
> > > >>
> > > >> Please let us know if you have any question.
> > > >>
> > > >> Thanks,
> > > >>
> > > >> Jiangjie (Becket) Qin
> > > >>
> > > >> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <sewen@apache.org
> > <mailto:
> > > sewen@apache.org>> <sewen@apache.org <ma...@apache.org>> <
> > > >>
> > > >> sewen@apache.org <ma...@apache.org>> wrote:
> > > >>
> > > >> Hi  Łukasz!
> > > >>
> > > >> Becket and me are working hard on figuring out the last details
> > > >>
> > > >> and
> > > >>
> > > >> implementing the first PoC. We would update the FLIP hopefully
> > > >>
> > > >> next
> > > >>
> > > >> week.
> > > >>
> > > >> There is a fair chance that a first version of this will be in
> > > >>
> > > >> 1.10,
> > > >>
> > > >> but
> > > >>
> > > >> I
> > > >>
> > > >> think it will take another release to battle test it and migrate
> > > >>
> > > >> the
> > > >>
> > > >> connectors.
> > > >>
> > > >> Best,
> > > >> Stephan
> > > >>
> > > >>
> > > >>
> > > >>
> > > >> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <ljd@touk.pl
> > > <ma...@touk.pl>
> > > >>
> > > >> <
> > > >>
> > > >> ljd@touk.pl <ma...@touk.pl>>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi,
> > > >>
> > > >> This proposal looks very promising for us. Do you have any plans
> > > >>
> > > >> in
> > > >>
> > > >> which
> > > >>
> > > >> Flink release it is going to be released? We are thinking on
> > > >>
> > > >> using a
> > > >>
> > > >> Data
> > > >>
> > > >> Set API for our future use cases but on the other hand Data Set
> > > >>
> > > >> API
> > > >>
> > > >> is
> > > >>
> > > >> going to be deprecated so using proposed bounded data streams
> > > >>
> > > >> solution
> > > >>
> > > >> could be more viable in the long term.
> > > >>
> > > >> Thanks,
> > > >> Łukasz
> > > >>
> > > >> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com
> <mailto:
> > > thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
> > > thomas.weise@gmail.com>> <
> > > >>
> > > >> thomas.weise@gmail.com <ma...@gmail.com>> wrote:
> > > >>
> > > >> Thanks for putting together this proposal!
> > > >>
> > > >> I see that the "Per Split Event Time" and "Event Time Alignment"
> > > >>
> > > >> sections
> > > >>
> > > >> are still TBD.
> > > >>
> > > >> It would probably be good to flesh those out a bit before
> > > >>
> > > >> proceeding
> > > >>
> > > >> too
> > > >>
> > > >> far
> > > >>
> > > >> as the event time alignment will probably influence the
> > > >>
> > > >> interaction
> > > >>
> > > >> with
> > > >>
> > > >> the split reader, specifically ReaderStatus
> > > >>
> > > >> emitNext(SourceOutput<E>
> > > >>
> > > >> output).
> > > >>
> > > >> We currently have only one implementation for event time alignment
> > > >>
> > > >> in
> > > >>
> > > >> the
> > > >>
> > > >> Kinesis consumer. The synchronization in that case takes place as
> > > >>
> > > >> the
> > > >>
> > > >> last
> > > >>
> > > >> step before records are emitted downstream (RecordEmitter). With
> > > >>
> > > >> the
> > > >>
> > > >> currently proposed interfaces, the equivalent can be implemented
> > > >>
> > > >> in
> > > >>
> > > >> the
> > > >>
> > > >> reader loop, although note that in the Kinesis consumer the per
> > > >>
> > > >> shard
> > > >>
> > > >> threads push records.
> > > >>
> > > >> Synchronization has not been implemented for the Kafka consumer
> > > >>
> > > >> yet.
> > > >>
> > > >> https://issues.apache.org/jira/browse/FLINK-12675 <
> > > https://issues.apache.org/jira/browse/FLINK-12675>
> > > >>
> > > >> When I looked at it, I realized that the implementation will look
> > > >>
> > > >> quite
> > > >>
> > > >> different
> > > >> from Kinesis because it needs to take place in the pull part,
> > > >>
> > > >> where
> > > >>
> > > >> records
> > > >>
> > > >> are taken from the Kafka client. Due to the multiplexing it cannot
> > > >>
> > > >> be
> > > >>
> > > >> done
> > > >>
> > > >> by blocking the split thread like it currently works for Kinesis.
> > > >>
> > > >> Reading
> > > >>
> > > >> from individual Kafka partitions needs to be controlled via
> > > >>
> > > >> pause/resume
> > > >>
> > > >> on the Kafka client.
> > > >>
> > > >> To take on that responsibility the split thread would need to be
> > > >>
> > > >> aware
> > > >>
> > > >> of
> > > >>
> > > >> the
> > > >> watermarks or at least whether it should or should not continue to
> > > >>
> > > >> consume
> > > >>
> > > >> a given split and this may require a different SourceReader or
> > > >>
> > > >> SourceOutput
> > > >>
> > > >> interface.
> > > >>
> > > >> Thanks,
> > > >> Thomas
> > > >>
> > > >>
> > > >> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com
> <mailto:
> > > mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
> <
> > > >>
> > > >> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
> > > >>
> > > >> Hi Stephan,
> > > >>
> > > >> Thank you for feedback!
> > > >> Will take a look at your branch before public discussing.
> > > >>
> > > >>
> > > >> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <sewen@apache.org
> > > <ma...@apache.org>> <sewen@apache.org <mailto:sewen@apache.org
> >>
> > > >>
> > > >> <
> > > >>
> > > >> sewen@apache.org <ma...@apache.org>>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi Biao!
> > > >>
> > > >> Thanks for reviving this. I would like to join this discussion,
> > > >>
> > > >> but
> > > >>
> > > >> am
> > > >>
> > > >> quite occupied with the 1.9 release, so can we maybe pause this
> > > >>
> > > >> discussion
> > > >>
> > > >> for a week or so?
> > > >>
> > > >> In the meantime I can share some suggestion based on prior
> > > >>
> > > >> experiments:
> > > >>
> > > >> How to do watermarks / timestamp extractors in a simpler and more
> > > >>
> > > >> flexible
> > > >>
> > > >> way. I think that part is quite promising should be part of the
> > > >>
> > > >> new
> > > >>
> > > >> source
> > > >>
> > > >> interface.
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > > <
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > > >
> > > >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > > <
> > >
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > > >
> > > >>
> > > >> Some experiments on how to build the source reader and its
> > > >>
> > > >> library
> > > >>
> > > >> for
> > > >>
> > > >> common threading/split patterns:
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > > <
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > > >
> > > >>
> > > >> Best,
> > > >> Stephan
> > > >>
> > > >>
> > > >> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mmyy1110@gmail.com
> > <mailto:
> > > mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
> <
> > > >>
> > > >> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi devs,
> > > >>
> > > >> Since 1.9 is nearly released, I think we could get back to
> > > >>
> > > >> FLIP-27.
> > > >>
> > > >> I
> > > >>
> > > >> believe it should be included in 1.10.
> > > >>
> > > >> There are so many things mentioned in document of FLIP-27. [1] I
> > > >>
> > > >> think
> > > >>
> > > >> we'd better discuss them separately. However the wiki is not a
> > > >>
> > > >> good
> > > >>
> > > >> place
> > > >>
> > > >> to discuss. I wrote google doc about SplitReader API which
> > > >>
> > > >> misses
> > > >>
> > > >> some
> > > >>
> > > >> details in the document. [2]
> > > >>
> > > >> 1.
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > <
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > >
> > > >>
> > > >> 2.
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > >
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > > <
> > >
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > > >
> > > >>
> > > >> CC Stephan, Aljoscha, Piotrek, Becket
> > > >>
> > > >>
> > > >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com
> <mailto:
> > > mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
> <
> > > >>
> > > >> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi Steven,
> > > >> Thank you for the feedback. Please take a look at the document
> > > >>
> > > >> FLIP-27
> > > >>
> > > >> <
> > > >>
> > > >>
> > > >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > <
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > >
> > > >>
> > > >> which
> > > >>
> > > >> is updated recently. A lot of details of enumerator were added
> > > >>
> > > >> in
> > > >>
> > > >> this
> > > >>
> > > >> document. I think it would help.
> > > >>
> > > >> Steven Wu <stevenz3wu@gmail.com <ma...@gmail.com>> <
> > > stevenz3wu@gmail.com <ma...@gmail.com>> <
> > stevenz3wu@gmail.com
> > > <ma...@gmail.com>> <stevenz3wu@gmail.com <mailto:
> > > stevenz3wu@gmail.com>>
> > > >>
> > > >> 于2019年3月28日周四
> > > >>
> > > >> 下午12:52写道:
> > > >>
> > > >> This proposal mentioned that SplitEnumerator might run on the
> > > >> JobManager or
> > > >> in a single task on a TaskManager.
> > > >>
> > > >> if enumerator is a single task on a taskmanager, then the job
> > > >>
> > > >> DAG
> > > >>
> > > >> can
> > > >>
> > > >> never
> > > >> been embarrassingly parallel anymore. That will nullify the
> > > >>
> > > >> leverage
> > > >>
> > > >> of
> > > >>
> > > >> fine-grained recovery for embarrassingly parallel jobs.
> > > >>
> > > >> It's not clear to me what's the implication of running
> > > >>
> > > >> enumerator
> > > >>
> > > >> on
> > > >>
> > > >> the
> > > >>
> > > >> jobmanager. So I will leave that out for now.
> > > >>
> > > >> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com
> <mailto:
> > > mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
> <
> > > >>
> > > >> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi Stephan & Piotrek,
> > > >>
> > > >> Thank you for feedback.
> > > >>
> > > >> It seems that there are a lot of things to do in community.
> > > >>
> > > >> I
> > > >>
> > > >> am
> > > >>
> > > >> just
> > > >>
> > > >> afraid that this discussion may be forgotten since there so
> > > >>
> > > >> many
> > > >>
> > > >> proposals
> > > >>
> > > >> recently.
> > > >> Anyway, wish to see the split topics soon :)
> > > >>
> > > >> Piotr Nowojski <piotr@da-platform.com <mailto:piotr@da-platform.com
> >>
> > <
> > > piotr@da-platform.com <ma...@da-platform.com>> <
> > > piotr@da-platform.com <ma...@da-platform.com>> <
> > > piotr@da-platform.com <ma...@da-platform.com>>
> > > >>
> > > >> 于2019年1月24日周四
> > > >>
> > > >> 下午8:21写道:
> > > >>
> > > >> Hi Biao!
> > > >>
> > > >> This discussion was stalled because of preparations for
> > > >>
> > > >> the
> > > >>
> > > >> open
> > > >>
> > > >> sourcing
> > > >>
> > > >> & merging Blink. I think before creating the tickets we
> > > >>
> > > >> should
> > > >>
> > > >> split this
> > > >>
> > > >> discussion into topics/areas outlined by Stephan and
> > > >>
> > > >> create
> > > >>
> > > >> Flips
> > > >>
> > > >> for
> > > >>
> > > >> that.
> > > >>
> > > >> I think there is no chance for this to be completed in
> > > >>
> > > >> couple
> > > >>
> > > >> of
> > > >>
> > > >> remaining
> > > >>
> > > >> weeks/1 month before 1.8 feature freeze, however it would
> > > >>
> > > >> be
> > > >>
> > > >> good
> > > >>
> > > >> to aim
> > > >>
> > > >> with those changes for 1.9.
> > > >>
> > > >> Piotrek
> > > >>
> > > >>
> > > >> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com <mailto:
> > > mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>>
> <
> > > >>
> > > >> mmyy1110@gmail.com <ma...@gmail.com>>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi community,
> > > >> The summary of Stephan makes a lot sense to me. It is
> > > >>
> > > >> much
> > > >>
> > > >> clearer
> > > >>
> > > >> indeed
> > > >>
> > > >> after splitting the complex topic into small ones.
> > > >> I was wondering is there any detail plan for next step?
> > > >>
> > > >> If
> > > >>
> > > >> not,
> > > >>
> > > >> I
> > > >>
> > > >> would
> > > >>
> > > >> like to push this thing forward by creating some JIRA
> > > >>
> > > >> issues.
> > > >>
> > > >> Another question is that should version 1.8 include
> > > >>
> > > >> these
> > > >>
> > > >> features?
> > > >>
> > > >> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <
> > > sewen@apache.org <ma...@apache.org>> <sewen@apache.org <mailto:
> > > sewen@apache.org>> <sewen@apache.org <ma...@apache.org>>
> > > 于2018年12月1日周六
> > > >>
> > > >> 上午4:20写道:
> > > >>
> > > >> Thanks everyone for the lively discussion. Let me try
> > > >>
> > > >> to
> > > >>
> > > >> summarize
> > > >>
> > > >> where I
> > > >>
> > > >> see convergence in the discussion and open issues.
> > > >> I'll try to group this by design aspect of the source.
> > > >>
> > > >> Please
> > > >>
> > > >> let me
> > > >>
> > > >> know
> > > >>
> > > >> if I got things wrong or missed something crucial here.
> > > >>
> > > >> For issues 1-3, if the below reflects the state of the
> > > >>
> > > >> discussion, I
> > > >>
> > > >> would
> > > >>
> > > >> try and update the FLIP in the next days.
> > > >> For the remaining ones we need more discussion.
> > > >>
> > > >> I would suggest to fork each of these aspects into a
> > > >>
> > > >> separate
> > > >>
> > > >> mail
> > > >>
> > > >> thread,
> > > >>
> > > >> or will loose sight of the individual aspects.
> > > >>
> > > >> *(1) Separation of Split Enumerator and Split Reader*
> > > >>
> > > >> - All seem to agree this is a good thing
> > > >> - Split Enumerator could in the end live on JobManager
> > > >>
> > > >> (and
> > > >>
> > > >> assign
> > > >>
> > > >> splits
> > > >>
> > > >> via RPC) or in a task (and assign splits via data
> > > >>
> > > >> streams)
> > > >>
> > > >> - this discussion is orthogonal and should come later,
> > > >>
> > > >> when
> > > >>
> > > >> the
> > > >>
> > > >> interface
> > > >>
> > > >> is agreed upon.
> > > >>
> > > >> *(2) Split Readers for one or more splits*
> > > >>
> > > >> - Discussion seems to agree that we need to support
> > > >>
> > > >> one
> > > >>
> > > >> reader
> > > >>
> > > >> that
> > > >>
> > > >> possibly handles multiple splits concurrently.
> > > >> - The requirement comes from sources where one
> > > >>
> > > >> poll()-style
> > > >>
> > > >> call
> > > >>
> > > >> fetches
> > > >>
> > > >> data from different splits / partitions
> > > >>   --> example sources that require that would be for
> > > >>
> > > >> example
> > > >>
> > > >> Kafka,
> > > >>
> > > >> Pravega, Pulsar
> > > >>
> > > >> - Could have one split reader per source, or multiple
> > > >>
> > > >> split
> > > >>
> > > >> readers
> > > >>
> > > >> that
> > > >>
> > > >> share the "poll()" function
> > > >> - To not make it too complicated, we can start with
> > > >>
> > > >> thinking
> > > >>
> > > >> about
> > > >>
> > > >> one
> > > >>
> > > >> split reader for all splits initially and see if that
> > > >>
> > > >> covers
> > > >>
> > > >> all
> > > >>
> > > >> requirements
> > > >>
> > > >> *(3) Threading model of the Split Reader*
> > > >>
> > > >> - Most active part of the discussion ;-)
> > > >>
> > > >> - A non-blocking way for Flink's task code to interact
> > > >>
> > > >> with
> > > >>
> > > >> the
> > > >>
> > > >> source
> > > >>
> > > >> is
> > > >>
> > > >> needed in order to a task runtime code based on a
> > > >> single-threaded/actor-style task design
> > > >>   --> I personally am a big proponent of that, it will
> > > >>
> > > >> help
> > > >>
> > > >> with
> > > >>
> > > >> well-behaved checkpoints, efficiency, and simpler yet
> > > >>
> > > >> more
> > > >>
> > > >> robust
> > > >>
> > > >> runtime
> > > >>
> > > >> code
> > > >>
> > > >> - Users care about simple abstraction, so as a
> > > >>
> > > >> subclass
> > > >>
> > > >> of
> > > >>
> > > >> SplitReader
> > > >>
> > > >> (non-blocking / async) we need to have a
> > > >>
> > > >> BlockingSplitReader
> > > >>
> > > >> which
> > > >>
> > > >> will
> > > >>
> > > >> form the basis of most source implementations.
> > > >>
> > > >> BlockingSplitReader
> > > >>
> > > >> lets
> > > >>
> > > >> users do blocking simple poll() calls.
> > > >> - The BlockingSplitReader would spawn a thread (or
> > > >>
> > > >> more)
> > > >>
> > > >> and
> > > >>
> > > >> the
> > > >>
> > > >> thread(s) can make blocking calls and hand over data
> > > >>
> > > >> buffers
> > > >>
> > > >> via
> > > >>
> > > >> a
> > > >>
> > > >> blocking
> > > >>
> > > >> queue
> > > >> - This should allow us to cover both, a fully async
> > > >>
> > > >> runtime,
> > > >>
> > > >> and a
> > > >>
> > > >> simple
> > > >>
> > > >> blocking interface for users.
> > > >> - This is actually very similar to how the Kafka
> > > >>
> > > >> connectors
> > > >>
> > > >> work.
> > > >>
> > > >> Kafka
> > > >>
> > > >> 9+ with one thread, Kafka 8 with multiple threads
> > > >>
> > > >> - On the base SplitReader (the async one), the
> > > >>
> > > >> non-blocking
> > > >>
> > > >> method
> > > >>
> > > >> that
> > > >>
> > > >> gets the next chunk of data would signal data
> > > >>
> > > >> availability
> > > >>
> > > >> via
> > > >>
> > > >> a
> > > >>
> > > >> CompletableFuture, because that gives the best
> > > >>
> > > >> flexibility
> > > >>
> > > >> (can
> > > >>
> > > >> await
> > > >>
> > > >> completion or register notification handlers).
> > > >> - The source task would register a "thenHandle()" (or
> > > >>
> > > >> similar)
> > > >>
> > > >> on the
> > > >>
> > > >> future to put a "take next data" task into the
> > > >>
> > > >> actor-style
> > > >>
> > > >> mailbox
> > > >>
> > > >> *(4) Split Enumeration and Assignment*
> > > >>
> > > >> - Splits may be generated lazily, both in cases where
> > > >>
> > > >> there
> > > >>
> > > >> is a
> > > >>
> > > >> limited
> > > >>
> > > >> number of splits (but very many), or splits are
> > > >>
> > > >> discovered
> > > >>
> > > >> over
> > > >>
> > > >> time
> > > >>
> > > >> - Assignment should also be lazy, to get better load
> > > >>
> > > >> balancing
> > > >>
> > > >> - Assignment needs support locality preferences
> > > >>
> > > >> - Possible design based on discussion so far:
> > > >>
> > > >>   --> SplitReader has a method "addSplits(SplitT...)"
> > > >>
> > > >> to
> > > >>
> > > >> add
> > > >>
> > > >> one or
> > > >>
> > > >> more
> > > >>
> > > >> splits. Some split readers might assume they have only
> > > >>
> > > >> one
> > > >>
> > > >> split
> > > >>
> > > >> ever,
> > > >>
> > > >> concurrently, others assume multiple splits. (Note:
> > > >>
> > > >> idea
> > > >>
> > > >> behind
> > > >>
> > > >> being
> > > >>
> > > >> able
> > > >>
> > > >> to add multiple splits at the same time is to ease
> > > >>
> > > >> startup
> > > >>
> > > >> where
> > > >>
> > > >> multiple
> > > >>
> > > >> splits may be assigned instantly.)
> > > >>   --> SplitReader has a context object on which it can
> > > >>
> > > >> call
> > > >>
> > > >> indicate
> > > >>
> > > >> when
> > > >>
> > > >> splits are completed. The enumerator gets that
> > > >>
> > > >> notification and
> > > >>
> > > >> can
> > > >>
> > > >> use
> > > >>
> > > >> to
> > > >>
> > > >> decide when to assign new splits. This should help both
> > > >>
> > > >> in
> > > >>
> > > >> cases
> > > >>
> > > >> of
> > > >>
> > > >> sources
> > > >>
> > > >> that take splits lazily (file readers) and in case the
> > > >>
> > > >> source
> > > >>
> > > >> needs to
> > > >>
> > > >> preserve a partial order between splits (Kinesis,
> > > >>
> > > >> Pravega,
> > > >>
> > > >> Pulsar may
> > > >>
> > > >> need
> > > >>
> > > >> that).
> > > >>   --> SplitEnumerator gets notification when
> > > >>
> > > >> SplitReaders
> > > >>
> > > >> start
> > > >>
> > > >> and
> > > >>
> > > >> when
> > > >>
> > > >> they finish splits. They can decide at that moment to
> > > >>
> > > >> push
> > > >>
> > > >> more
> > > >>
> > > >> splits
> > > >>
> > > >> to
> > > >>
> > > >> that reader
> > > >>   --> The SplitEnumerator should probably be aware of
> > > >>
> > > >> the
> > > >>
> > > >> source
> > > >>
> > > >> parallelism, to build its initial distribution.
> > > >>
> > > >> - Open question: Should the source expose something
> > > >>
> > > >> like
> > > >>
> > > >> "host
> > > >>
> > > >> preferences", so that yarn/mesos/k8s can take this into
> > > >>
> > > >> account
> > > >>
> > > >> when
> > > >>
> > > >> selecting a node to start a TM on?
> > > >>
> > > >> *(5) Watermarks and event time alignment*
> > > >>
> > > >> - Watermark generation, as well as idleness, needs to
> > > >>
> > > >> be
> > > >>
> > > >> per
> > > >>
> > > >> split
> > > >>
> > > >> (like
> > > >>
> > > >> currently in the Kafka Source, per partition)
> > > >> - It is desirable to support optional
> > > >>
> > > >> event-time-alignment,
> > > >>
> > > >> meaning
> > > >>
> > > >> that
> > > >>
> > > >> splits that are ahead are back-pressured or temporarily
> > > >>
> > > >> unsubscribed
> > > >>
> > > >> - I think i would be desirable to encapsulate
> > > >>
> > > >> watermark
> > > >>
> > > >> generation
> > > >>
> > > >> logic
> > > >>
> > > >> in watermark generators, for a separation of concerns.
> > > >>
> > > >> The
> > > >>
> > > >> watermark
> > > >>
> > > >> generators should run per split.
> > > >> - Using watermark generators would also help with
> > > >>
> > > >> another
> > > >>
> > > >> problem of
> > > >>
> > > >> the
> > > >>
> > > >> suggested interface, namely supporting non-periodic
> > > >>
> > > >> watermarks
> > > >>
> > > >> efficiently.
> > > >>
> > > >> - Need a way to "dispatch" next record to different
> > > >>
> > > >> watermark
> > > >>
> > > >> generators
> > > >>
> > > >> - Need a way to tell SplitReader to "suspend" a split
> > > >>
> > > >> until a
> > > >>
> > > >> certain
> > > >>
> > > >> watermark is reached (event time backpressure)
> > > >> - This would in fact be not needed (and thus simpler)
> > > >>
> > > >> if
> > > >>
> > > >> we
> > > >>
> > > >> had
> > > >>
> > > >> a
> > > >>
> > > >> SplitReader per split and may be a reason to re-open
> > > >>
> > > >> that
> > > >>
> > > >> discussion
> > > >>
> > > >> *(6) Watermarks across splits and in the Split
> > > >>
> > > >> Enumerator*
> > > >>
> > > >> - The split enumerator may need some watermark
> > > >>
> > > >> awareness,
> > > >>
> > > >> which
> > > >>
> > > >> should
> > > >>
> > > >> be
> > > >>
> > > >> purely based on split metadata (like create timestamp
> > > >>
> > > >> of
> > > >>
> > > >> file
> > > >>
> > > >> splits)
> > > >>
> > > >> - If there are still more splits with overlapping
> > > >>
> > > >> event
> > > >>
> > > >> time
> > > >>
> > > >> range
> > > >>
> > > >> for
> > > >>
> > > >> a
> > > >>
> > > >> split reader, then that split reader should not advance
> > > >>
> > > >> the
> > > >>
> > > >> watermark
> > > >>
> > > >> within the split beyond the overlap boundary. Otherwise
> > > >>
> > > >> future
> > > >>
> > > >> splits
> > > >>
> > > >> will
> > > >>
> > > >> produce late data.
> > > >>
> > > >> - One way to approach this could be that the split
> > > >>
> > > >> enumerator
> > > >>
> > > >> may
> > > >>
> > > >> send
> > > >>
> > > >> watermarks to the readers, and the readers cannot emit
> > > >>
> > > >> watermarks
> > > >>
> > > >> beyond
> > > >>
> > > >> that received watermark.
> > > >> - Many split enumerators would simply immediately send
> > > >>
> > > >> Long.MAX
> > > >>
> > > >> out
> > > >>
> > > >> and
> > > >>
> > > >> leave the progress purely to the split readers.
> > > >>
> > > >> - For event-time alignment / split back pressure, this
> > > >>
> > > >> begs
> > > >>
> > > >> the
> > > >>
> > > >> question
> > > >>
> > > >> how we can avoid deadlocks that may arise when splits
> > > >>
> > > >> are
> > > >>
> > > >> suspended
> > > >>
> > > >> for
> > > >>
> > > >> event time back pressure,
> > > >>
> > > >> *(7) Batch and streaming Unification*
> > > >>
> > > >> - Functionality wise, the above design should support
> > > >>
> > > >> both
> > > >>
> > > >> - Batch often (mostly) does not care about reading "in
> > > >>
> > > >> order"
> > > >>
> > > >> and
> > > >>
> > > >> generating watermarks
> > > >>   --> Might use different enumerator logic that is
> > > >>
> > > >> more
> > > >>
> > > >> locality
> > > >>
> > > >> aware
> > > >>
> > > >> and ignores event time order
> > > >>   --> Does not generate watermarks
> > > >> - Would be great if bounded sources could be
> > > >>
> > > >> identified
> > > >>
> > > >> at
> > > >>
> > > >> compile
> > > >>
> > > >> time,
> > > >>
> > > >> so that "env.addBoundedSource(...)" is type safe and
> > > >>
> > > >> can
> > > >>
> > > >> return a
> > > >>
> > > >> "BoundedDataStream".
> > > >> - Possible to defer this discussion until later
> > > >>
> > > >> *Miscellaneous Comments*
> > > >>
> > > >> - Should the source have a TypeInformation for the
> > > >>
> > > >> produced
> > > >>
> > > >> type,
> > > >>
> > > >> instead
> > > >>
> > > >> of a serializer? We need a type information in the
> > > >>
> > > >> stream
> > > >>
> > > >> anyways, and
> > > >>
> > > >> can
> > > >>
> > > >> derive the serializer from that. Plus, creating the
> > > >>
> > > >> serializer
> > > >>
> > > >> should
> > > >>
> > > >> respect the ExecutionConfig.
> > > >>
> > > >> - The TypeSerializer interface is very powerful but
> > > >>
> > > >> also
> > > >>
> > > >> not
> > > >>
> > > >> easy to
> > > >>
> > > >> implement. Its purpose is to handle data super
> > > >>
> > > >> efficiently,
> > > >>
> > > >> support
> > > >>
> > > >> flexible ways of evolution, etc.
> > > >> For metadata I would suggest to look at the
> > > >>
> > > >> SimpleVersionedSerializer
> > > >>
> > > >> instead, which is used for example for checkpoint
> > > >>
> > > >> master
> > > >>
> > > >> hooks,
> > > >>
> > > >> or for
> > > >>
> > > >> the
> > > >>
> > > >> streaming file sink. I think that is is a good match
> > > >>
> > > >> for
> > > >>
> > > >> cases
> > > >>
> > > >> where
> > > >>
> > > >> we
> > > >>
> > > >> do
> > > >>
> > > >> not need more than ser/deser (no copy, etc.) and don't
> > > >>
> > > >> need to
> > > >>
> > > >> push
> > > >>
> > > >> versioning out of the serialization paths for best
> > > >>
> > > >> performance
> > > >>
> > > >> (as in
> > > >>
> > > >> the
> > > >>
> > > >> TypeSerializer)
> > > >>
> > > >>
> > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > > >>
> > > >> k.kloudas@data-artisans.com>
> > > >>
> > > >> wrote:
> > > >>
> > > >>
> > > >> Hi Biao,
> > > >>
> > > >> Thanks for the answer!
> > > >>
> > > >> So given the multi-threaded readers, now we have as
> > > >>
> > > >> open
> > > >>
> > > >> questions:
> > > >>
> > > >> 1) How do we let the checkpoints pass through our
> > > >>
> > > >> multi-threaded
> > > >>
> > > >> reader
> > > >>
> > > >> operator?
> > > >>
> > > >> 2) Do we have separate reader and source operators or
> > > >>
> > > >> not? In
> > > >>
> > > >> the
> > > >>
> > > >> strategy
> > > >>
> > > >> that has a separate source, the source operator has a
> > > >>
> > > >> parallelism of
> > > >>
> > > >> 1
> > > >>
> > > >> and
> > > >>
> > > >> is responsible for split recovery only.
> > > >>
> > > >> For the first one, given also the constraints
> > > >>
> > > >> (blocking,
> > > >>
> > > >> finite
> > > >>
> > > >> queues,
> > > >>
> > > >> etc), I do not have an answer yet.
> > > >>
> > > >> For the 2nd, I think that we should go with separate
> > > >>
> > > >> operators
> > > >>
> > > >> for
> > > >>
> > > >> the
> > > >>
> > > >> source and the readers, for the following reasons:
> > > >>
> > > >> 1) This is more aligned with a potential future
> > > >>
> > > >> improvement
> > > >>
> > > >> where the
> > > >>
> > > >> split
> > > >>
> > > >> discovery becomes a responsibility of the JobManager
> > > >>
> > > >> and
> > > >>
> > > >> readers are
> > > >>
> > > >> pooling more work from the JM.
> > > >>
> > > >> 2) The source is going to be the "single point of
> > > >>
> > > >> truth".
> > > >>
> > > >> It
> > > >>
> > > >> will
> > > >>
> > > >> know
> > > >>
> > > >> what
> > > >>
> > > >> has been processed and what not. If the source and the
> > > >>
> > > >> readers
> > > >>
> > > >> are a
> > > >>
> > > >> single
> > > >>
> > > >> operator with parallelism > 1, or in general, if the
> > > >>
> > > >> split
> > > >>
> > > >> discovery
> > > >>
> > > >> is
> > > >>
> > > >> done by each task individually, then:
> > > >>  i) we have to have a deterministic scheme for each
> > > >>
> > > >> reader to
> > > >>
> > > >> assign
> > > >>
> > > >> splits to itself (e.g. mod subtaskId). This is not
> > > >>
> > > >> necessarily
> > > >>
> > > >> trivial
> > > >>
> > > >> for
> > > >>
> > > >> all sources.
> > > >>  ii) each reader would have to keep a copy of all its
> > > >>
> > > >> processed
> > > >>
> > > >> slpits
> > > >>
> > > >>  iii) the state has to be a union state with a
> > > >>
> > > >> non-trivial
> > > >>
> > > >> merging
> > > >>
> > > >> logic
> > > >>
> > > >> in order to support rescaling.
> > > >>
> > > >> Two additional points that you raised above:
> > > >>
> > > >> i) The point that you raised that we need to keep all
> > > >>
> > > >> splits
> > > >>
> > > >> (processed
> > > >>
> > > >> and
> > > >>
> > > >> not-processed) I think is a bit of a strong
> > > >>
> > > >> requirement.
> > > >>
> > > >> This
> > > >>
> > > >> would
> > > >>
> > > >> imply
> > > >>
> > > >> that for infinite sources the state will grow
> > > >>
> > > >> indefinitely.
> > > >>
> > > >> This is
> > > >>
> > > >> problem
> > > >>
> > > >> is even more pronounced if we do not have a single
> > > >>
> > > >> source
> > > >>
> > > >> that
> > > >>
> > > >> assigns
> > > >>
> > > >> splits to readers, as each reader will have its own
> > > >>
> > > >> copy
> > > >>
> > > >> of
> > > >>
> > > >> the
> > > >>
> > > >> state.
> > > >>
> > > >> ii) it is true that for finite sources we need to
> > > >>
> > > >> somehow
> > > >>
> > > >> not
> > > >>
> > > >> close
> > > >>
> > > >> the
> > > >>
> > > >> readers when the source/split discoverer finishes. The
> > > >> ContinuousFileReaderOperator has a work-around for
> > > >>
> > > >> that.
> > > >>
> > > >> It is
> > > >>
> > > >> not
> > > >>
> > > >> elegant,
> > > >>
> > > >> and checkpoints are not emitted after closing the
> > > >>
> > > >> source,
> > > >>
> > > >> but
> > > >>
> > > >> this, I
> > > >>
> > > >> believe, is a bigger problem which requires more
> > > >>
> > > >> changes
> > > >>
> > > >> than
> > > >>
> > > >> just
> > > >>
> > > >> refactoring the source interface.
> > > >>
> > > >> Cheers,
> > > >> Kostas
> > > >>
> > > >>
> > > >>
> > > >>
> > > >> --
> > > >> Best, Jingsong Lee
> > >
> > >
> >
>
>
> --
> Best, Jingsong Lee
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Jingsong Li <ji...@gmail.com>.
Hi Becket,

I also have some performance concerns too.

If I understand correctly, SourceOutput will emit data per record into the
queue? I'm worried about the multithreading performance of this queue.

> One example is some batched messaging systems which only have an offset
for the entire batch instead of individual messages in the batch.

As you said, there are some batched system source, like parquet/orc source.
Could we have the batch emit interface to improve performance? The queue of
per record may cause performance degradation.

Best,
Jingsong Lee

On Thu, Dec 12, 2019 at 9:15 AM Jark Wu <im...@gmail.com> wrote:

> Hi Becket,
>
> I think Dawid explained things clearly and makes a lot of sense.
> I'm also in favor of #2, because #1 doesn't work for our future unified
> envrionment.
>
> You can see the vision in this documentation [1]. In the future, we would
> like to
> drop the global streaming/batch mode in SQL (i.e.
> EnvironmentSettings#inStreamingMode/inBatchMode).
> A source is bounded or unbounded once defined, so queries can be inferred
> from source to run
> in streaming or batch or hybrid mode. However, in #1, we will lose this
> ability because the framework
> doesn't know whether the source is bounded or unbounded.
>
> Best,
> Jark
>
>
> [1]:
>
> https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p
>
> On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <pi...@ververica.com> wrote:
>
> > Hi,
> >
> > Regarding the:
> >
> > Collection<E> getNextRecords()
> >
> > I’m pretty sure such design would unfortunately impact the performance
> > (accessing and potentially creating the collection on the hot path).
> >
> > Also the
> >
> > InputStatus emitNext(DataOutput<T> output) throws Exception;
> > or
> > Status pollNext(SourceOutput<T> sourceOutput) throws Exception;
> >
> > Gives us some opportunities in the future, to allow Source hot looping
> > inside, until it receives some signal “please exit because of some
> reasons”
> > (output collector could return such hint upon collecting the result). But
> > that’s another topic outside of this FLIP’s scope.
> >
> > Piotrek
> >
> > > On 11 Dec 2019, at 10:41, Till Rohrmann <tr...@apache.org> wrote:
> > >
> > > Hi Becket,
> > >
> > > quick clarification from my side because I think you misunderstood my
> > > question. I did not suggest to let the SourceReader return only a
> single
> > > record at a time when calling getNextRecords. As the return type
> > indicates,
> > > the method can return an arbitrary number of records.
> > >
> > > Cheers,
> > > Till
> > >
> > > On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> > dwysakowicz@apache.org <ma...@apache.org>>
> > > wrote:
> > >
> > >> Hi Becket,
> > >>
> > >> Issue #1 - Design of Source interface
> > >>
> > >> I mentioned the lack of a method like
> > Source#createEnumerator(Boundedness
> > >> boundedness, SplitEnumeratorContext context), because without the
> > current
> > >> proposal is not complete/does not work.
> > >>
> > >> If we say that boundedness is an intrinsic property of a source imo we
> > >> don't need the Source#createEnumerator(Boundedness boundedness,
> > >> SplitEnumeratorContext context) method.
> > >>
> > >> Assuming a source from my previous example:
> > >>
> > >> Source source = KafkaSource.builder()
> > >>  ...
> > >>  .untilTimestamp(...)
> > >>  .build()
> > >>
> > >> Would the enumerator differ if created like
> > >> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
> > >> .createEnumerator(BOUNDED, ...)? I know I am repeating myself, but
> this
> > is
> > >> the part that my opinion differ the most from the current proposal. I
> > >> really think it should always be the source that tells if it is
> bounded
> > or
> > >> not. In the current proposal methods continousSource/boundedSource
> > somewhat
> > >> reconfigure the source, which I think is misleading.
> > >>
> > >> I think a call like:
> > >>
> > >> Source source = KafkaSource.builder()
> > >>  ...
> > >>  .readContinously() / readUntilLatestOffset() / readUntilTimestamp /
> > readUntilOffsets / ...
> > >>  .build()
> > >>
> > >> is way cleaner (and expressive) than
> > >>
> > >> Source source = KafkaSource.builder()
> > >>  ...
> > >>  .build()
> > >>
> > >>
> > >> env.continousSource(source) // which actually underneath would call
> > createEnumerator(CONTINUOUS, ctx) which would be equivalent to
> > source.readContinously().createEnumerator(ctx)
> > >> // or
> > >> env.boundedSource(source) // which actually underneath would call
> > createEnumerator(BOUNDED, ctx) which would be equivalent to
> > source.readUntilLatestOffset().createEnumerator(ctx)
> > >>
> > >>
> > >> Sorry for the comparison, but to me it seems there is too much magic
> > >> happening underneath those two calls.
> > >>
> > >> I really believe the Source interface should have getBoundedness
> method
> > >> instead of (supportBoundedness) + createEnumerator(Boundedness, ...)
> > >>
> > >>
> > >> Issue #2 - Design of
> > >> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> > >>
> > >> As you might have guessed I am slightly in favor of option #2
> modified.
> > >> Yes I am aware every step of the dag would have to be able to say if
> it
> > is
> > >> bounded or not. I have a feeling it would be easier to express cross
> > >> bounded/unbounded operations, but I must admit I have not thought it
> > >> through thoroughly, In the spirit of batch is just a special case of
> > >> streaming I thought BoundedStream would extend from DataStream.
> Correct
> > me
> > >> if I am wrong. In such a setup the cross bounded/unbounded operation
> > could
> > >> be expressed quite easily I think:
> > >>
> > >> DataStream {
> > >>  DataStream join(DataStream, ...); // we could not really tell if the
> > result is bounded or not, but because bounded stream is a special case of
> > unbounded the API object is correct, irrespective if the left or right
> side
> > of the join is bounded
> > >> }
> > >>
> > >> BoundedStream extends DataStream {
> > >>  BoundedStream join(BoundedStream, ...); // only if both sides are
> > bounded the result can be bounded as well. However we do have access to
> the
> > DataStream#join here, so you can still join with a DataStream
> > >> }
> > >>
> > >>
> > >> On the other hand I also see benefits of two completely disjointed
> APIs,
> > >> as we could prohibit some streaming calls in the bounded API. I can't
> > think
> > >> of any unbounded operators that could not be implemented for bounded
> > stream.
> > >>
> > >> Besides I think we both agree we don't like the method:
> > >>
> > >> DataStream boundedStream(Source)
> > >>
> > >> suggested in the current state of the FLIP. Do we ? :)
> > >>
> > >> Best,
> > >>
> > >> Dawid
> > >>
> > >> On 10/12/2019 18:57, Becket Qin wrote:
> > >>
> > >> Hi folks,
> > >>
> > >> Thanks for the discussion, great feedback. Also thanks Dawid for the
> > >> explanation, it is much clearer now.
> > >>
> > >> One thing that is indeed missing from the FLIP is how the boundedness
> is
> > >> passed to the Source implementation. So the API should be
> > >> Source#createEnumerator(Boundedness boundedness,
> SplitEnumeratorContext
> > >> context)
> > >> And we can probably remove the Source#supportBoundedness(Boundedness
> > >> boundedness) method.
> > >>
> > >> Assuming we have that, we are essentially choosing from one of the
> > >> following two options:
> > >>
> > >> Option 1:
> > >> // The source is continuous source, and only unbounded operations can
> be
> > >> performed.
> > >> DataStream<Type> datastream = env.continuousSource(someSource);
> > >>
> > >> // The source is bounded source, both bounded and unbounded operations
> > can
> > >> be performed.
> > >> BoundedDataStream<Type> boundedDataStream =
> > env.boundedSource(someSource);
> > >>
> > >>  - Pros:
> > >>       a) explicit boundary between bounded / unbounded streams, it is
> > >> quite simple and clear to the users.
> > >>  - Cons:
> > >>       a) For applications that do not involve bounded operations, they
> > >> still have to call different API to distinguish bounded / unbounded
> > streams.
> > >>       b) No support for bounded stream to run in a streaming runtime
> > >> setting, i.e. scheduling and operators behaviors.
> > >>
> > >>
> > >> Option 2:
> > >> // The source is either bounded or unbounded, but only unbounded
> > operations
> > >> could be performed on the returned DataStream.
> > >> DataStream<Type> dataStream = env.source(someSource);
> > >>
> > >> // The source must be a bounded source, otherwise exception is thrown.
> > >> BoundedDataStream<Type> boundedDataStream =
> > >> env.boundedSource(boundedSource);
> > >>
> > >> The pros and cons are exactly the opposite of option 1.
> > >>  - Pros:
> > >>       a) For applications that do not involve bounded operations, they
> > >> still have to call different API to distinguish bounded / unbounded
> > streams.
> > >>       b) Support for bounded stream to run in a streaming runtime
> > setting,
> > >> i.e. scheduling and operators behaviors.
> > >>  - Cons:
> > >>       a) Bounded / unbounded streams are kind of mixed, i.e. given a
> > >> DataStream, it is not clear whether it is bounded or not, unless you
> > have
> > >> the access to its source.
> > >>
> > >>
> > >> If we only think from the Source API perspective, option 2 seems a
> > better
> > >> choice because functionality wise it is a superset of option 1, at the
> > cost
> > >> of some seemingly acceptable ambiguity in the DataStream API.
> > >> But if we look at the DataStream API as a whole, option 1 seems a
> > clearer
> > >> choice. For example, some times a library may have to know whether a
> > >> certain task will finish or not. And it would be difficult to tell if
> > the
> > >> input is a DataStream, unless additional information is provided all
> the
> > >> way from the Source. One possible solution is to have a *modified
> > option 2*
> > >> which adds a method to the DataStream API to indicate boundedness,
> such
> > as
> > >> getBoundedness(). It would solve the problem with a potential
> confusion
> > of
> > >> what is difference between a DataStream with getBoundedness()=true
> and a
> > >> BoundedDataStream. But that seems not super difficult to explain.
> > >>
> > >> So from API's perspective, I don't have a strong opinion between
> > *option 1*
> > >> and *modified option 2. *I like the cleanness of option 1, but
> modified
> > >> option 2 would be more attractive if we have concrete use case for the
> > >> "Bounded stream with unbounded streaming runtime settings".
> > >>
> > >> Re: Till
> > >>
> > >>
> > >> Maybe this has already been asked before but I was wondering why the
> > >> SourceReader interface has the method pollNext which hands the
> > >> responsibility of outputting elements to the SourceReader
> > implementation?
> > >> Has this been done for backwards compatibility reasons with the old
> > source
> > >> interface? If not, then one could define a Collection<E>
> > getNextRecords()
> > >> method which returns the currently retrieved records and then the
> caller
> > >> emits them outside of the SourceReader. That way the interface would
> not
> > >> allow to implement an outputting loop where we never hand back control
> > to
> > >> the caller. At the moment, this contract can be easily broken and is
> > only
> > >> mentioned loosely in the JavaDocs.
> > >>
> > >>
> > >> The primary reason we handover the SourceOutput to the SourceReader is
> > >> because sometimes it is difficult for a SourceReader to emit one
> record
> > at
> > >> a time. One example is some batched messaging systems which only have
> an
> > >> offset for the entire batch instead of individual messages in the
> > batch. In
> > >> that case, returning one record at a time would leave the SourceReader
> > in
> > >> an uncheckpointable state because they can only checkpoint at the
> batch
> > >> boundaries.
> > >>
> > >> Thanks,
> > >>
> > >> Jiangjie (Becket) Qin
> > >>
> > >> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <trohrmann@apache.org
> > <ma...@apache.org>> <trohrmann@apache.org <mailto:
> > trohrmann@apache.org>> wrote:
> > >>
> > >>
> > >> Hi everyone,
> > >>
> > >> thanks for drafting this FLIP. It reads very well.
> > >>
> > >> Concerning Dawid's proposal, I tend to agree. The boundedness could
> come
> > >> from the source and tell the system how to treat the operator
> > (scheduling
> > >> wise). From a user's perspective it should be fine to get back a
> > DataStream
> > >> when calling env.source(boundedSource) if he does not need special
> > >> operations defined on a BoundedDataStream. If he needs this, then one
> > could
> > >> use the method BoundedDataStream env.boundedSource(boundedSource).
> > >>
> > >> If possible, we could enforce the proper usage of env.boundedSource()
> by
> > >> introducing a BoundedSource type so that one cannot pass an
> > >> unbounded source to it. That way users would not be able to shoot
> > >> themselves in the foot.
> > >>
> > >> Maybe this has already been asked before but I was wondering why the
> > >> SourceReader interface has the method pollNext which hands the
> > >> responsibility of outputting elements to the SourceReader
> > implementation?
> > >> Has this been done for backwards compatibility reasons with the old
> > source
> > >> interface? If not, then one could define a Collection<E>
> > getNextRecords()
> > >> method which returns the currently retrieved records and then the
> caller
> > >> emits them outside of the SourceReader. That way the interface would
> not
> > >> allow to implement an outputting loop where we never hand back control
> > to
> > >> the caller. At the moment, this contract can be easily broken and is
> > only
> > >> mentioned loosely in the JavaDocs.
> > >>
> > >> Cheers,
> > >> Till
> > >>
> > >> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <jingsonglee0@gmail.com
> > <ma...@gmail.com>> <jingsonglee0@gmail.com <mailto:
> > jingsonglee0@gmail.com>>
> > >> wrote:
> > >>
> > >>
> > >> Hi all,
> > >>
> > >> I think current design is good.
> > >>
> > >> My understanding is:
> > >>
> > >> For execution mode: bounded mode and continuous mode, It's totally
> > >> different. I don't think we have the ability to integrate the two
> models
> > >>
> > >> at
> > >>
> > >> present. It's about scheduling, memory, algorithms, States, etc. we
> > >> shouldn't confuse them.
> > >>
> > >> For source capabilities: only bounded, only continuous, both bounded
> and
> > >> continuous.
> > >> I think Kafka is a source that can be ran both bounded
> > >> and continuous execution mode.
> > >> And Kafka with end offset should be ran both bounded
> > >> and continuous execution mode.  Using apache Beam with Flink runner, I
> > >>
> > >> used
> > >>
> > >> to run a "bounded" Kafka in streaming mode. For our previous
> DataStream,
> > >>
> > >> it
> > >>
> > >> is not necessarily required that the source cannot be bounded.
> > >>
> > >> So it is my thought for Dawid's question:
> > >> 1.pass a bounded source to continuousSource() +1
> > >> 2.pass a continuous source to boundedSource() -1, should throw
> > exception.
> > >>
> > >> In StreamExecutionEnvironment, continuousSource and boundedSource
> define
> > >> the execution mode. It defines a clear boundary of execution mode.
> > >>
> > >> Best,
> > >> Jingsong Lee
> > >>
> > >> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com <mailto:
> > imjark@gmail.com>> <imjark@gmail.com <ma...@gmail.com>> wrote:
> > >>
> > >>
> > >> I agree with Dawid's point that the boundedness information should
> come
> > >> from the source itself (e.g. the end timestamp), not through
> > >> env.boundedSouce()/continuousSource().
> > >> I think if we want to support something like `env.source()` that
> derive
> > >>
> > >> the
> > >>
> > >> execution mode from source, `supportsBoundedness(Boundedness)`
> > >> method is not enough, because we don't know whether it is bounded or
> > >>
> > >> not.
> > >>
> > >> Best,
> > >> Jark
> > >>
> > >>
> > >> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <dwysakowicz@apache.org
> > <ma...@apache.org>> <dwysakowicz@apache.org <mailto:
> > dwysakowicz@apache.org>>
> > >> wrote:
> > >>
> > >>
> > >> One more thing. In the current proposal, with the
> > >> supportsBoundedness(Boundedness) method and the boundedness coming
> > >>
> > >> from
> > >>
> > >> either continuousSource or boundedSource I could not find how this
> > >> information is fed back to the SplitEnumerator.
> > >>
> > >> Best,
> > >>
> > >> Dawid
> > >>
> > >> On 09/12/2019 13:52, Becket Qin wrote:
> > >>
> > >> Hi Dawid,
> > >>
> > >> Thanks for the comments. This actually brings another relevant
> > >>
> > >> question
> > >>
> > >> about what does a "bounded source" imply. I actually had the same
> > >> impression when I look at the Source API. Here is what I understand
> > >>
> > >> after
> > >>
> > >> some discussion with Stephan. The bounded source has the following
> > >>
> > >> impacts.
> > >>
> > >> 1. API validity.
> > >> - A bounded source generates a bounded stream so some operations
> > >>
> > >> that
> > >>
> > >> only
> > >>
> > >> works for bounded records would be performed, e.g. sort.
> > >> - To expose these bounded stream only APIs, there are two options:
> > >>     a. Add them to the DataStream API and throw exception if a
> > >>
> > >> method
> > >>
> > >> is
> > >>
> > >> called on an unbounded stream.
> > >>     b. Create a BoundedDataStream class which is returned from
> > >> env.boundedSource(), while DataStream is returned from
> > >>
> > >> env.continousSource().
> > >>
> > >> Note that this cannot be done by having single
> > >>
> > >> env.source(theSource)
> > >>
> > >> even
> > >>
> > >> the Source has a getBoundedness() method.
> > >>
> > >> 2. Scheduling
> > >> - A bounded source could be computed stage by stage without
> > >>
> > >> bringing
> > >>
> > >> up
> > >>
> > >> all
> > >>
> > >> the tasks at the same time.
> > >>
> > >> 3. Operator behaviors
> > >> - A bounded source indicates the records are finite so some
> > >>
> > >> operators
> > >>
> > >> can
> > >>
> > >> wait until it receives all the records before it starts the
> > >>
> > >> processing.
> > >>
> > >> In the above impact, only 1 is relevant to the API design. And the
> > >>
> > >> current
> > >>
> > >> proposal in FLIP-27 is following 1.b.
> > >>
> > >> // boundedness depends of source property, imo this should always
> > >>
> > >> be
> > >>
> > >> preferred
> > >>
> > >>
> > >> DataStream<MyType> stream = env.source(theSource);
> > >>
> > >>
> > >> In your proposal, does DataStream have bounded stream only methods?
> > >>
> > >> It
> > >>
> > >> looks it should have, otherwise passing a bounded Source to
> > >>
> > >> env.source()
> > >>
> > >> would be confusing. In that case, we will essentially do 1.a if an
> > >> unbounded Source is created from env.source(unboundedSource).
> > >>
> > >> If we have the methods only supported for bounded streams in
> > >>
> > >> DataStream,
> > >>
> > >> it
> > >>
> > >> seems a little weird to have a separate BoundedDataStream
> > >>
> > >> interface.
> > >>
> > >> Am I understand it correctly?
> > >>
> > >> Thanks,
> > >>
> > >> Jiangjie (Becket) Qin
> > >>
> > >>
> > >>
> > >> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> > >>
> > >> dwysakowicz@apache.org <ma...@apache.org>>
> > >>
> > >> wrote:
> > >>
> > >>
> > >> Hi all,
> > >>
> > >> Really well written proposal and very important one. I must admit
> > >>
> > >> I
> > >>
> > >> have
> > >>
> > >> not understood all the intricacies of it yet.
> > >>
> > >> One question I have though is about where does the information
> > >>
> > >> about
> > >>
> > >> boundedness come from. I think in most cases it is a property of
> > >>
> > >> the
> > >>
> > >> source. As you described it might be e.g. end offset, a flag
> > >>
> > >> should
> > >>
> > >> it
> > >>
> > >> monitor new splits etc. I think it would be a really nice use case
> > >>
> > >> to
> > >>
> > >> be
> > >>
> > >> able to say:
> > >>
> > >> new KafkaSource().readUntil(long timestamp),
> > >>
> > >> which could work as an "end offset". Moreover I think all Bounded
> > >>
> > >> sources
> > >>
> > >> support continuous mode, but no intrinsically continuous source
> > >>
> > >> support
> > >>
> > >> the
> > >>
> > >> Bounded mode. If I understood the proposal correctly it suggest
> > >>
> > >> the
> > >>
> > >> boundedness sort of "comes" from the outside of the source, from
> > >>
> > >> the
> > >>
> > >> invokation of either boundedStream or continousSource.
> > >>
> > >> I am wondering if it would make sense to actually change the
> > >>
> > >> method
> > >>
> > >> boolean Source#supportsBoundedness(Boundedness)
> > >>
> > >> to
> > >>
> > >> Boundedness Source#getBoundedness().
> > >>
> > >> As for the methods #boundedSource, #continousSource, assuming the
> > >> boundedness is property of the source they do not affect how the
> > >>
> > >> enumerator
> > >>
> > >> works, but mostly how the dag is scheduled, right? I am not
> > >>
> > >> against
> > >>
> > >> those
> > >>
> > >> methods, but I think it is a very specific use case to actually
> > >>
> > >> override
> > >>
> > >> the property of the source. In general I would expect users to
> > >>
> > >> only
> > >>
> > >> call
> > >>
> > >> env.source(theSource), where the source tells if it is bounded or
> > >>
> > >> not. I
> > >>
> > >> would suggest considering following set of methods:
> > >>
> > >> // boundedness depends of source property, imo this should always
> > >>
> > >> be
> > >>
> > >> preferred
> > >>
> > >> DataStream<MyType> stream = env.source(theSource);
> > >>
> > >>
> > >> // always continous execution, whether bounded or unbounded source
> > >>
> > >> DataStream<MyType> boundedStream = env.continousSource(theSource);
> > >>
> > >> // imo this would make sense if the BoundedDataStream provides
> > >>
> > >> additional features unavailable for continous mode
> > >>
> > >> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
> > >>
> > >>
> > >> Best,
> > >>
> > >> Dawid
> > >>
> > >>
> > >> On 04/12/2019 11:25, Stephan Ewen wrote:
> > >>
> > >> Thanks, Becket, for updating this.
> > >>
> > >> I agree with moving the aspects you mentioned into separate FLIPs
> > >>
> > >> -
> > >>
> > >> this
> > >>
> > >> one way becoming unwieldy in size.
> > >>
> > >> +1 to the FLIP in its current state. Its a very detailed write-up,
> > >>
> > >> nicely
> > >>
> > >> done!
> > >>
> > >> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <becket.qin@gmail.com
> > <ma...@gmail.com>> <becket.qin@gmail.com <mailto:
> > becket.qin@gmail.com>>
> > >>
> > >> <
> > >>
> > >> becket.qin@gmail.com <ma...@gmail.com>> wrote:
> > >>
> > >> Hi all,
> > >>
> > >> Sorry for the long belated update. I have updated FLIP-27 wiki
> > >>
> > >> page
> > >>
> > >> with
> > >>
> > >> the latest proposals. Some noticeable changes include:
> > >> 1. A new generic communication mechanism between SplitEnumerator
> > >>
> > >> and
> > >>
> > >> SourceReader.
> > >> 2. Some detail API method signature changes.
> > >>
> > >> We left a few things out of this FLIP and will address them in
> > >>
> > >> separate
> > >>
> > >> FLIPs. Including:
> > >> 1. Per split event time.
> > >> 2. Event time alignment.
> > >> 3. Fine grained failover for SplitEnumerator failure.
> > >>
> > >> Please let us know if you have any question.
> > >>
> > >> Thanks,
> > >>
> > >> Jiangjie (Becket) Qin
> > >>
> > >> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <sewen@apache.org
> <mailto:
> > sewen@apache.org>> <sewen@apache.org <ma...@apache.org>> <
> > >>
> > >> sewen@apache.org <ma...@apache.org>> wrote:
> > >>
> > >> Hi  Łukasz!
> > >>
> > >> Becket and me are working hard on figuring out the last details
> > >>
> > >> and
> > >>
> > >> implementing the first PoC. We would update the FLIP hopefully
> > >>
> > >> next
> > >>
> > >> week.
> > >>
> > >> There is a fair chance that a first version of this will be in
> > >>
> > >> 1.10,
> > >>
> > >> but
> > >>
> > >> I
> > >>
> > >> think it will take another release to battle test it and migrate
> > >>
> > >> the
> > >>
> > >> connectors.
> > >>
> > >> Best,
> > >> Stephan
> > >>
> > >>
> > >>
> > >>
> > >> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <ljd@touk.pl
> > <ma...@touk.pl>
> > >>
> > >> <
> > >>
> > >> ljd@touk.pl <ma...@touk.pl>>
> > >>
> > >> wrote:
> > >>
> > >> Hi,
> > >>
> > >> This proposal looks very promising for us. Do you have any plans
> > >>
> > >> in
> > >>
> > >> which
> > >>
> > >> Flink release it is going to be released? We are thinking on
> > >>
> > >> using a
> > >>
> > >> Data
> > >>
> > >> Set API for our future use cases but on the other hand Data Set
> > >>
> > >> API
> > >>
> > >> is
> > >>
> > >> going to be deprecated so using proposed bounded data streams
> > >>
> > >> solution
> > >>
> > >> could be more viable in the long term.
> > >>
> > >> Thanks,
> > >> Łukasz
> > >>
> > >> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com <mailto:
> > thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
> > thomas.weise@gmail.com>> <
> > >>
> > >> thomas.weise@gmail.com <ma...@gmail.com>> wrote:
> > >>
> > >> Thanks for putting together this proposal!
> > >>
> > >> I see that the "Per Split Event Time" and "Event Time Alignment"
> > >>
> > >> sections
> > >>
> > >> are still TBD.
> > >>
> > >> It would probably be good to flesh those out a bit before
> > >>
> > >> proceeding
> > >>
> > >> too
> > >>
> > >> far
> > >>
> > >> as the event time alignment will probably influence the
> > >>
> > >> interaction
> > >>
> > >> with
> > >>
> > >> the split reader, specifically ReaderStatus
> > >>
> > >> emitNext(SourceOutput<E>
> > >>
> > >> output).
> > >>
> > >> We currently have only one implementation for event time alignment
> > >>
> > >> in
> > >>
> > >> the
> > >>
> > >> Kinesis consumer. The synchronization in that case takes place as
> > >>
> > >> the
> > >>
> > >> last
> > >>
> > >> step before records are emitted downstream (RecordEmitter). With
> > >>
> > >> the
> > >>
> > >> currently proposed interfaces, the equivalent can be implemented
> > >>
> > >> in
> > >>
> > >> the
> > >>
> > >> reader loop, although note that in the Kinesis consumer the per
> > >>
> > >> shard
> > >>
> > >> threads push records.
> > >>
> > >> Synchronization has not been implemented for the Kafka consumer
> > >>
> > >> yet.
> > >>
> > >> https://issues.apache.org/jira/browse/FLINK-12675 <
> > https://issues.apache.org/jira/browse/FLINK-12675>
> > >>
> > >> When I looked at it, I realized that the implementation will look
> > >>
> > >> quite
> > >>
> > >> different
> > >> from Kinesis because it needs to take place in the pull part,
> > >>
> > >> where
> > >>
> > >> records
> > >>
> > >> are taken from the Kafka client. Due to the multiplexing it cannot
> > >>
> > >> be
> > >>
> > >> done
> > >>
> > >> by blocking the split thread like it currently works for Kinesis.
> > >>
> > >> Reading
> > >>
> > >> from individual Kafka partitions needs to be controlled via
> > >>
> > >> pause/resume
> > >>
> > >> on the Kafka client.
> > >>
> > >> To take on that responsibility the split thread would need to be
> > >>
> > >> aware
> > >>
> > >> of
> > >>
> > >> the
> > >> watermarks or at least whether it should or should not continue to
> > >>
> > >> consume
> > >>
> > >> a given split and this may require a different SourceReader or
> > >>
> > >> SourceOutput
> > >>
> > >> interface.
> > >>
> > >> Thanks,
> > >> Thomas
> > >>
> > >>
> > >> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com <mailto:
> > mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
> > >>
> > >> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
> > >>
> > >> Hi Stephan,
> > >>
> > >> Thank you for feedback!
> > >> Will take a look at your branch before public discussing.
> > >>
> > >>
> > >> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <sewen@apache.org
> > <ma...@apache.org>> <sewen@apache.org <ma...@apache.org>>
> > >>
> > >> <
> > >>
> > >> sewen@apache.org <ma...@apache.org>>
> > >>
> > >> wrote:
> > >>
> > >> Hi Biao!
> > >>
> > >> Thanks for reviving this. I would like to join this discussion,
> > >>
> > >> but
> > >>
> > >> am
> > >>
> > >> quite occupied with the 1.9 release, so can we maybe pause this
> > >>
> > >> discussion
> > >>
> > >> for a week or so?
> > >>
> > >> In the meantime I can share some suggestion based on prior
> > >>
> > >> experiments:
> > >>
> > >> How to do watermarks / timestamp extractors in a simpler and more
> > >>
> > >> flexible
> > >>
> > >> way. I think that part is quite promising should be part of the
> > >>
> > >> new
> > >>
> > >> source
> > >>
> > >> interface.
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > <
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > >
> > >>
> > >>
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > <
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > >
> > >>
> > >> Some experiments on how to build the source reader and its
> > >>
> > >> library
> > >>
> > >> for
> > >>
> > >> common threading/split patterns:
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > <
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > >
> > >>
> > >> Best,
> > >> Stephan
> > >>
> > >>
> > >> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mmyy1110@gmail.com
> <mailto:
> > mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
> > >>
> > >> mmyy1110@gmail.com <ma...@gmail.com>>
> > >>
> > >> wrote:
> > >>
> > >> Hi devs,
> > >>
> > >> Since 1.9 is nearly released, I think we could get back to
> > >>
> > >> FLIP-27.
> > >>
> > >> I
> > >>
> > >> believe it should be included in 1.10.
> > >>
> > >> There are so many things mentioned in document of FLIP-27. [1] I
> > >>
> > >> think
> > >>
> > >> we'd better discuss them separately. However the wiki is not a
> > >>
> > >> good
> > >>
> > >> place
> > >>
> > >> to discuss. I wrote google doc about SplitReader API which
> > >>
> > >> misses
> > >>
> > >> some
> > >>
> > >> details in the document. [2]
> > >>
> > >> 1.
> > >>
> > >>
> > >>
> > >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > <
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > >
> > >>
> > >> 2.
> > >>
> > >>
> > >>
> > >>
> > >>
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > <
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > >
> > >>
> > >> CC Stephan, Aljoscha, Piotrek, Becket
> > >>
> > >>
> > >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com <mailto:
> > mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
> > >>
> > >> mmyy1110@gmail.com <ma...@gmail.com>>
> > >>
> > >> wrote:
> > >>
> > >> Hi Steven,
> > >> Thank you for the feedback. Please take a look at the document
> > >>
> > >> FLIP-27
> > >>
> > >> <
> > >>
> > >>
> > >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > <
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >
> > >>
> > >> which
> > >>
> > >> is updated recently. A lot of details of enumerator were added
> > >>
> > >> in
> > >>
> > >> this
> > >>
> > >> document. I think it would help.
> > >>
> > >> Steven Wu <stevenz3wu@gmail.com <ma...@gmail.com>> <
> > stevenz3wu@gmail.com <ma...@gmail.com>> <
> stevenz3wu@gmail.com
> > <ma...@gmail.com>> <stevenz3wu@gmail.com <mailto:
> > stevenz3wu@gmail.com>>
> > >>
> > >> 于2019年3月28日周四
> > >>
> > >> 下午12:52写道:
> > >>
> > >> This proposal mentioned that SplitEnumerator might run on the
> > >> JobManager or
> > >> in a single task on a TaskManager.
> > >>
> > >> if enumerator is a single task on a taskmanager, then the job
> > >>
> > >> DAG
> > >>
> > >> can
> > >>
> > >> never
> > >> been embarrassingly parallel anymore. That will nullify the
> > >>
> > >> leverage
> > >>
> > >> of
> > >>
> > >> fine-grained recovery for embarrassingly parallel jobs.
> > >>
> > >> It's not clear to me what's the implication of running
> > >>
> > >> enumerator
> > >>
> > >> on
> > >>
> > >> the
> > >>
> > >> jobmanager. So I will leave that out for now.
> > >>
> > >> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com <mailto:
> > mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
> > >>
> > >> mmyy1110@gmail.com <ma...@gmail.com>>
> > >>
> > >> wrote:
> > >>
> > >> Hi Stephan & Piotrek,
> > >>
> > >> Thank you for feedback.
> > >>
> > >> It seems that there are a lot of things to do in community.
> > >>
> > >> I
> > >>
> > >> am
> > >>
> > >> just
> > >>
> > >> afraid that this discussion may be forgotten since there so
> > >>
> > >> many
> > >>
> > >> proposals
> > >>
> > >> recently.
> > >> Anyway, wish to see the split topics soon :)
> > >>
> > >> Piotr Nowojski <piotr@da-platform.com <ma...@da-platform.com>>
> <
> > piotr@da-platform.com <ma...@da-platform.com>> <
> > piotr@da-platform.com <ma...@da-platform.com>> <
> > piotr@da-platform.com <ma...@da-platform.com>>
> > >>
> > >> 于2019年1月24日周四
> > >>
> > >> 下午8:21写道:
> > >>
> > >> Hi Biao!
> > >>
> > >> This discussion was stalled because of preparations for
> > >>
> > >> the
> > >>
> > >> open
> > >>
> > >> sourcing
> > >>
> > >> & merging Blink. I think before creating the tickets we
> > >>
> > >> should
> > >>
> > >> split this
> > >>
> > >> discussion into topics/areas outlined by Stephan and
> > >>
> > >> create
> > >>
> > >> Flips
> > >>
> > >> for
> > >>
> > >> that.
> > >>
> > >> I think there is no chance for this to be completed in
> > >>
> > >> couple
> > >>
> > >> of
> > >>
> > >> remaining
> > >>
> > >> weeks/1 month before 1.8 feature freeze, however it would
> > >>
> > >> be
> > >>
> > >> good
> > >>
> > >> to aim
> > >>
> > >> with those changes for 1.9.
> > >>
> > >> Piotrek
> > >>
> > >>
> > >> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com <mailto:
> > mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
> > >>
> > >> mmyy1110@gmail.com <ma...@gmail.com>>
> > >>
> > >> wrote:
> > >>
> > >> Hi community,
> > >> The summary of Stephan makes a lot sense to me. It is
> > >>
> > >> much
> > >>
> > >> clearer
> > >>
> > >> indeed
> > >>
> > >> after splitting the complex topic into small ones.
> > >> I was wondering is there any detail plan for next step?
> > >>
> > >> If
> > >>
> > >> not,
> > >>
> > >> I
> > >>
> > >> would
> > >>
> > >> like to push this thing forward by creating some JIRA
> > >>
> > >> issues.
> > >>
> > >> Another question is that should version 1.8 include
> > >>
> > >> these
> > >>
> > >> features?
> > >>
> > >> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <
> > sewen@apache.org <ma...@apache.org>> <sewen@apache.org <mailto:
> > sewen@apache.org>> <sewen@apache.org <ma...@apache.org>>
> > 于2018年12月1日周六
> > >>
> > >> 上午4:20写道:
> > >>
> > >> Thanks everyone for the lively discussion. Let me try
> > >>
> > >> to
> > >>
> > >> summarize
> > >>
> > >> where I
> > >>
> > >> see convergence in the discussion and open issues.
> > >> I'll try to group this by design aspect of the source.
> > >>
> > >> Please
> > >>
> > >> let me
> > >>
> > >> know
> > >>
> > >> if I got things wrong or missed something crucial here.
> > >>
> > >> For issues 1-3, if the below reflects the state of the
> > >>
> > >> discussion, I
> > >>
> > >> would
> > >>
> > >> try and update the FLIP in the next days.
> > >> For the remaining ones we need more discussion.
> > >>
> > >> I would suggest to fork each of these aspects into a
> > >>
> > >> separate
> > >>
> > >> mail
> > >>
> > >> thread,
> > >>
> > >> or will loose sight of the individual aspects.
> > >>
> > >> *(1) Separation of Split Enumerator and Split Reader*
> > >>
> > >> - All seem to agree this is a good thing
> > >> - Split Enumerator could in the end live on JobManager
> > >>
> > >> (and
> > >>
> > >> assign
> > >>
> > >> splits
> > >>
> > >> via RPC) or in a task (and assign splits via data
> > >>
> > >> streams)
> > >>
> > >> - this discussion is orthogonal and should come later,
> > >>
> > >> when
> > >>
> > >> the
> > >>
> > >> interface
> > >>
> > >> is agreed upon.
> > >>
> > >> *(2) Split Readers for one or more splits*
> > >>
> > >> - Discussion seems to agree that we need to support
> > >>
> > >> one
> > >>
> > >> reader
> > >>
> > >> that
> > >>
> > >> possibly handles multiple splits concurrently.
> > >> - The requirement comes from sources where one
> > >>
> > >> poll()-style
> > >>
> > >> call
> > >>
> > >> fetches
> > >>
> > >> data from different splits / partitions
> > >>   --> example sources that require that would be for
> > >>
> > >> example
> > >>
> > >> Kafka,
> > >>
> > >> Pravega, Pulsar
> > >>
> > >> - Could have one split reader per source, or multiple
> > >>
> > >> split
> > >>
> > >> readers
> > >>
> > >> that
> > >>
> > >> share the "poll()" function
> > >> - To not make it too complicated, we can start with
> > >>
> > >> thinking
> > >>
> > >> about
> > >>
> > >> one
> > >>
> > >> split reader for all splits initially and see if that
> > >>
> > >> covers
> > >>
> > >> all
> > >>
> > >> requirements
> > >>
> > >> *(3) Threading model of the Split Reader*
> > >>
> > >> - Most active part of the discussion ;-)
> > >>
> > >> - A non-blocking way for Flink's task code to interact
> > >>
> > >> with
> > >>
> > >> the
> > >>
> > >> source
> > >>
> > >> is
> > >>
> > >> needed in order to a task runtime code based on a
> > >> single-threaded/actor-style task design
> > >>   --> I personally am a big proponent of that, it will
> > >>
> > >> help
> > >>
> > >> with
> > >>
> > >> well-behaved checkpoints, efficiency, and simpler yet
> > >>
> > >> more
> > >>
> > >> robust
> > >>
> > >> runtime
> > >>
> > >> code
> > >>
> > >> - Users care about simple abstraction, so as a
> > >>
> > >> subclass
> > >>
> > >> of
> > >>
> > >> SplitReader
> > >>
> > >> (non-blocking / async) we need to have a
> > >>
> > >> BlockingSplitReader
> > >>
> > >> which
> > >>
> > >> will
> > >>
> > >> form the basis of most source implementations.
> > >>
> > >> BlockingSplitReader
> > >>
> > >> lets
> > >>
> > >> users do blocking simple poll() calls.
> > >> - The BlockingSplitReader would spawn a thread (or
> > >>
> > >> more)
> > >>
> > >> and
> > >>
> > >> the
> > >>
> > >> thread(s) can make blocking calls and hand over data
> > >>
> > >> buffers
> > >>
> > >> via
> > >>
> > >> a
> > >>
> > >> blocking
> > >>
> > >> queue
> > >> - This should allow us to cover both, a fully async
> > >>
> > >> runtime,
> > >>
> > >> and a
> > >>
> > >> simple
> > >>
> > >> blocking interface for users.
> > >> - This is actually very similar to how the Kafka
> > >>
> > >> connectors
> > >>
> > >> work.
> > >>
> > >> Kafka
> > >>
> > >> 9+ with one thread, Kafka 8 with multiple threads
> > >>
> > >> - On the base SplitReader (the async one), the
> > >>
> > >> non-blocking
> > >>
> > >> method
> > >>
> > >> that
> > >>
> > >> gets the next chunk of data would signal data
> > >>
> > >> availability
> > >>
> > >> via
> > >>
> > >> a
> > >>
> > >> CompletableFuture, because that gives the best
> > >>
> > >> flexibility
> > >>
> > >> (can
> > >>
> > >> await
> > >>
> > >> completion or register notification handlers).
> > >> - The source task would register a "thenHandle()" (or
> > >>
> > >> similar)
> > >>
> > >> on the
> > >>
> > >> future to put a "take next data" task into the
> > >>
> > >> actor-style
> > >>
> > >> mailbox
> > >>
> > >> *(4) Split Enumeration and Assignment*
> > >>
> > >> - Splits may be generated lazily, both in cases where
> > >>
> > >> there
> > >>
> > >> is a
> > >>
> > >> limited
> > >>
> > >> number of splits (but very many), or splits are
> > >>
> > >> discovered
> > >>
> > >> over
> > >>
> > >> time
> > >>
> > >> - Assignment should also be lazy, to get better load
> > >>
> > >> balancing
> > >>
> > >> - Assignment needs support locality preferences
> > >>
> > >> - Possible design based on discussion so far:
> > >>
> > >>   --> SplitReader has a method "addSplits(SplitT...)"
> > >>
> > >> to
> > >>
> > >> add
> > >>
> > >> one or
> > >>
> > >> more
> > >>
> > >> splits. Some split readers might assume they have only
> > >>
> > >> one
> > >>
> > >> split
> > >>
> > >> ever,
> > >>
> > >> concurrently, others assume multiple splits. (Note:
> > >>
> > >> idea
> > >>
> > >> behind
> > >>
> > >> being
> > >>
> > >> able
> > >>
> > >> to add multiple splits at the same time is to ease
> > >>
> > >> startup
> > >>
> > >> where
> > >>
> > >> multiple
> > >>
> > >> splits may be assigned instantly.)
> > >>   --> SplitReader has a context object on which it can
> > >>
> > >> call
> > >>
> > >> indicate
> > >>
> > >> when
> > >>
> > >> splits are completed. The enumerator gets that
> > >>
> > >> notification and
> > >>
> > >> can
> > >>
> > >> use
> > >>
> > >> to
> > >>
> > >> decide when to assign new splits. This should help both
> > >>
> > >> in
> > >>
> > >> cases
> > >>
> > >> of
> > >>
> > >> sources
> > >>
> > >> that take splits lazily (file readers) and in case the
> > >>
> > >> source
> > >>
> > >> needs to
> > >>
> > >> preserve a partial order between splits (Kinesis,
> > >>
> > >> Pravega,
> > >>
> > >> Pulsar may
> > >>
> > >> need
> > >>
> > >> that).
> > >>   --> SplitEnumerator gets notification when
> > >>
> > >> SplitReaders
> > >>
> > >> start
> > >>
> > >> and
> > >>
> > >> when
> > >>
> > >> they finish splits. They can decide at that moment to
> > >>
> > >> push
> > >>
> > >> more
> > >>
> > >> splits
> > >>
> > >> to
> > >>
> > >> that reader
> > >>   --> The SplitEnumerator should probably be aware of
> > >>
> > >> the
> > >>
> > >> source
> > >>
> > >> parallelism, to build its initial distribution.
> > >>
> > >> - Open question: Should the source expose something
> > >>
> > >> like
> > >>
> > >> "host
> > >>
> > >> preferences", so that yarn/mesos/k8s can take this into
> > >>
> > >> account
> > >>
> > >> when
> > >>
> > >> selecting a node to start a TM on?
> > >>
> > >> *(5) Watermarks and event time alignment*
> > >>
> > >> - Watermark generation, as well as idleness, needs to
> > >>
> > >> be
> > >>
> > >> per
> > >>
> > >> split
> > >>
> > >> (like
> > >>
> > >> currently in the Kafka Source, per partition)
> > >> - It is desirable to support optional
> > >>
> > >> event-time-alignment,
> > >>
> > >> meaning
> > >>
> > >> that
> > >>
> > >> splits that are ahead are back-pressured or temporarily
> > >>
> > >> unsubscribed
> > >>
> > >> - I think i would be desirable to encapsulate
> > >>
> > >> watermark
> > >>
> > >> generation
> > >>
> > >> logic
> > >>
> > >> in watermark generators, for a separation of concerns.
> > >>
> > >> The
> > >>
> > >> watermark
> > >>
> > >> generators should run per split.
> > >> - Using watermark generators would also help with
> > >>
> > >> another
> > >>
> > >> problem of
> > >>
> > >> the
> > >>
> > >> suggested interface, namely supporting non-periodic
> > >>
> > >> watermarks
> > >>
> > >> efficiently.
> > >>
> > >> - Need a way to "dispatch" next record to different
> > >>
> > >> watermark
> > >>
> > >> generators
> > >>
> > >> - Need a way to tell SplitReader to "suspend" a split
> > >>
> > >> until a
> > >>
> > >> certain
> > >>
> > >> watermark is reached (event time backpressure)
> > >> - This would in fact be not needed (and thus simpler)
> > >>
> > >> if
> > >>
> > >> we
> > >>
> > >> had
> > >>
> > >> a
> > >>
> > >> SplitReader per split and may be a reason to re-open
> > >>
> > >> that
> > >>
> > >> discussion
> > >>
> > >> *(6) Watermarks across splits and in the Split
> > >>
> > >> Enumerator*
> > >>
> > >> - The split enumerator may need some watermark
> > >>
> > >> awareness,
> > >>
> > >> which
> > >>
> > >> should
> > >>
> > >> be
> > >>
> > >> purely based on split metadata (like create timestamp
> > >>
> > >> of
> > >>
> > >> file
> > >>
> > >> splits)
> > >>
> > >> - If there are still more splits with overlapping
> > >>
> > >> event
> > >>
> > >> time
> > >>
> > >> range
> > >>
> > >> for
> > >>
> > >> a
> > >>
> > >> split reader, then that split reader should not advance
> > >>
> > >> the
> > >>
> > >> watermark
> > >>
> > >> within the split beyond the overlap boundary. Otherwise
> > >>
> > >> future
> > >>
> > >> splits
> > >>
> > >> will
> > >>
> > >> produce late data.
> > >>
> > >> - One way to approach this could be that the split
> > >>
> > >> enumerator
> > >>
> > >> may
> > >>
> > >> send
> > >>
> > >> watermarks to the readers, and the readers cannot emit
> > >>
> > >> watermarks
> > >>
> > >> beyond
> > >>
> > >> that received watermark.
> > >> - Many split enumerators would simply immediately send
> > >>
> > >> Long.MAX
> > >>
> > >> out
> > >>
> > >> and
> > >>
> > >> leave the progress purely to the split readers.
> > >>
> > >> - For event-time alignment / split back pressure, this
> > >>
> > >> begs
> > >>
> > >> the
> > >>
> > >> question
> > >>
> > >> how we can avoid deadlocks that may arise when splits
> > >>
> > >> are
> > >>
> > >> suspended
> > >>
> > >> for
> > >>
> > >> event time back pressure,
> > >>
> > >> *(7) Batch and streaming Unification*
> > >>
> > >> - Functionality wise, the above design should support
> > >>
> > >> both
> > >>
> > >> - Batch often (mostly) does not care about reading "in
> > >>
> > >> order"
> > >>
> > >> and
> > >>
> > >> generating watermarks
> > >>   --> Might use different enumerator logic that is
> > >>
> > >> more
> > >>
> > >> locality
> > >>
> > >> aware
> > >>
> > >> and ignores event time order
> > >>   --> Does not generate watermarks
> > >> - Would be great if bounded sources could be
> > >>
> > >> identified
> > >>
> > >> at
> > >>
> > >> compile
> > >>
> > >> time,
> > >>
> > >> so that "env.addBoundedSource(...)" is type safe and
> > >>
> > >> can
> > >>
> > >> return a
> > >>
> > >> "BoundedDataStream".
> > >> - Possible to defer this discussion until later
> > >>
> > >> *Miscellaneous Comments*
> > >>
> > >> - Should the source have a TypeInformation for the
> > >>
> > >> produced
> > >>
> > >> type,
> > >>
> > >> instead
> > >>
> > >> of a serializer? We need a type information in the
> > >>
> > >> stream
> > >>
> > >> anyways, and
> > >>
> > >> can
> > >>
> > >> derive the serializer from that. Plus, creating the
> > >>
> > >> serializer
> > >>
> > >> should
> > >>
> > >> respect the ExecutionConfig.
> > >>
> > >> - The TypeSerializer interface is very powerful but
> > >>
> > >> also
> > >>
> > >> not
> > >>
> > >> easy to
> > >>
> > >> implement. Its purpose is to handle data super
> > >>
> > >> efficiently,
> > >>
> > >> support
> > >>
> > >> flexible ways of evolution, etc.
> > >> For metadata I would suggest to look at the
> > >>
> > >> SimpleVersionedSerializer
> > >>
> > >> instead, which is used for example for checkpoint
> > >>
> > >> master
> > >>
> > >> hooks,
> > >>
> > >> or for
> > >>
> > >> the
> > >>
> > >> streaming file sink. I think that is is a good match
> > >>
> > >> for
> > >>
> > >> cases
> > >>
> > >> where
> > >>
> > >> we
> > >>
> > >> do
> > >>
> > >> not need more than ser/deser (no copy, etc.) and don't
> > >>
> > >> need to
> > >>
> > >> push
> > >>
> > >> versioning out of the serialization paths for best
> > >>
> > >> performance
> > >>
> > >> (as in
> > >>
> > >> the
> > >>
> > >> TypeSerializer)
> > >>
> > >>
> > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > >>
> > >> k.kloudas@data-artisans.com>
> > >>
> > >> wrote:
> > >>
> > >>
> > >> Hi Biao,
> > >>
> > >> Thanks for the answer!
> > >>
> > >> So given the multi-threaded readers, now we have as
> > >>
> > >> open
> > >>
> > >> questions:
> > >>
> > >> 1) How do we let the checkpoints pass through our
> > >>
> > >> multi-threaded
> > >>
> > >> reader
> > >>
> > >> operator?
> > >>
> > >> 2) Do we have separate reader and source operators or
> > >>
> > >> not? In
> > >>
> > >> the
> > >>
> > >> strategy
> > >>
> > >> that has a separate source, the source operator has a
> > >>
> > >> parallelism of
> > >>
> > >> 1
> > >>
> > >> and
> > >>
> > >> is responsible for split recovery only.
> > >>
> > >> For the first one, given also the constraints
> > >>
> > >> (blocking,
> > >>
> > >> finite
> > >>
> > >> queues,
> > >>
> > >> etc), I do not have an answer yet.
> > >>
> > >> For the 2nd, I think that we should go with separate
> > >>
> > >> operators
> > >>
> > >> for
> > >>
> > >> the
> > >>
> > >> source and the readers, for the following reasons:
> > >>
> > >> 1) This is more aligned with a potential future
> > >>
> > >> improvement
> > >>
> > >> where the
> > >>
> > >> split
> > >>
> > >> discovery becomes a responsibility of the JobManager
> > >>
> > >> and
> > >>
> > >> readers are
> > >>
> > >> pooling more work from the JM.
> > >>
> > >> 2) The source is going to be the "single point of
> > >>
> > >> truth".
> > >>
> > >> It
> > >>
> > >> will
> > >>
> > >> know
> > >>
> > >> what
> > >>
> > >> has been processed and what not. If the source and the
> > >>
> > >> readers
> > >>
> > >> are a
> > >>
> > >> single
> > >>
> > >> operator with parallelism > 1, or in general, if the
> > >>
> > >> split
> > >>
> > >> discovery
> > >>
> > >> is
> > >>
> > >> done by each task individually, then:
> > >>  i) we have to have a deterministic scheme for each
> > >>
> > >> reader to
> > >>
> > >> assign
> > >>
> > >> splits to itself (e.g. mod subtaskId). This is not
> > >>
> > >> necessarily
> > >>
> > >> trivial
> > >>
> > >> for
> > >>
> > >> all sources.
> > >>  ii) each reader would have to keep a copy of all its
> > >>
> > >> processed
> > >>
> > >> slpits
> > >>
> > >>  iii) the state has to be a union state with a
> > >>
> > >> non-trivial
> > >>
> > >> merging
> > >>
> > >> logic
> > >>
> > >> in order to support rescaling.
> > >>
> > >> Two additional points that you raised above:
> > >>
> > >> i) The point that you raised that we need to keep all
> > >>
> > >> splits
> > >>
> > >> (processed
> > >>
> > >> and
> > >>
> > >> not-processed) I think is a bit of a strong
> > >>
> > >> requirement.
> > >>
> > >> This
> > >>
> > >> would
> > >>
> > >> imply
> > >>
> > >> that for infinite sources the state will grow
> > >>
> > >> indefinitely.
> > >>
> > >> This is
> > >>
> > >> problem
> > >>
> > >> is even more pronounced if we do not have a single
> > >>
> > >> source
> > >>
> > >> that
> > >>
> > >> assigns
> > >>
> > >> splits to readers, as each reader will have its own
> > >>
> > >> copy
> > >>
> > >> of
> > >>
> > >> the
> > >>
> > >> state.
> > >>
> > >> ii) it is true that for finite sources we need to
> > >>
> > >> somehow
> > >>
> > >> not
> > >>
> > >> close
> > >>
> > >> the
> > >>
> > >> readers when the source/split discoverer finishes. The
> > >> ContinuousFileReaderOperator has a work-around for
> > >>
> > >> that.
> > >>
> > >> It is
> > >>
> > >> not
> > >>
> > >> elegant,
> > >>
> > >> and checkpoints are not emitted after closing the
> > >>
> > >> source,
> > >>
> > >> but
> > >>
> > >> this, I
> > >>
> > >> believe, is a bigger problem which requires more
> > >>
> > >> changes
> > >>
> > >> than
> > >>
> > >> just
> > >>
> > >> refactoring the source interface.
> > >>
> > >> Cheers,
> > >> Kostas
> > >>
> > >>
> > >>
> > >>
> > >> --
> > >> Best, Jingsong Lee
> >
> >
>


-- 
Best, Jingsong Lee

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Jark Wu <im...@gmail.com>.
Hi Becket,

I think Dawid explained things clearly and makes a lot of sense.
I'm also in favor of #2, because #1 doesn't work for our future unified
envrionment.

You can see the vision in this documentation [1]. In the future, we would
like to
drop the global streaming/batch mode in SQL (i.e.
EnvironmentSettings#inStreamingMode/inBatchMode).
A source is bounded or unbounded once defined, so queries can be inferred
from source to run
in streaming or batch or hybrid mode. However, in #1, we will lose this
ability because the framework
doesn't know whether the source is bounded or unbounded.

Best,
Jark


[1]:
https://docs.google.com/document/d/1yrKXEIRATfxHJJ0K3t6wUgXAtZq8D-XgvEnvl2uUcr0/edit#heading=h.v4ib17buma1p

On Wed, 11 Dec 2019 at 20:52, Piotr Nowojski <pi...@ververica.com> wrote:

> Hi,
>
> Regarding the:
>
> Collection<E> getNextRecords()
>
> I’m pretty sure such design would unfortunately impact the performance
> (accessing and potentially creating the collection on the hot path).
>
> Also the
>
> InputStatus emitNext(DataOutput<T> output) throws Exception;
> or
> Status pollNext(SourceOutput<T> sourceOutput) throws Exception;
>
> Gives us some opportunities in the future, to allow Source hot looping
> inside, until it receives some signal “please exit because of some reasons”
> (output collector could return such hint upon collecting the result). But
> that’s another topic outside of this FLIP’s scope.
>
> Piotrek
>
> > On 11 Dec 2019, at 10:41, Till Rohrmann <tr...@apache.org> wrote:
> >
> > Hi Becket,
> >
> > quick clarification from my side because I think you misunderstood my
> > question. I did not suggest to let the SourceReader return only a single
> > record at a time when calling getNextRecords. As the return type
> indicates,
> > the method can return an arbitrary number of records.
> >
> > Cheers,
> > Till
> >
> > On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <
> dwysakowicz@apache.org <ma...@apache.org>>
> > wrote:
> >
> >> Hi Becket,
> >>
> >> Issue #1 - Design of Source interface
> >>
> >> I mentioned the lack of a method like
> Source#createEnumerator(Boundedness
> >> boundedness, SplitEnumeratorContext context), because without the
> current
> >> proposal is not complete/does not work.
> >>
> >> If we say that boundedness is an intrinsic property of a source imo we
> >> don't need the Source#createEnumerator(Boundedness boundedness,
> >> SplitEnumeratorContext context) method.
> >>
> >> Assuming a source from my previous example:
> >>
> >> Source source = KafkaSource.builder()
> >>  ...
> >>  .untilTimestamp(...)
> >>  .build()
> >>
> >> Would the enumerator differ if created like
> >> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
> >> .createEnumerator(BOUNDED, ...)? I know I am repeating myself, but this
> is
> >> the part that my opinion differ the most from the current proposal. I
> >> really think it should always be the source that tells if it is bounded
> or
> >> not. In the current proposal methods continousSource/boundedSource
> somewhat
> >> reconfigure the source, which I think is misleading.
> >>
> >> I think a call like:
> >>
> >> Source source = KafkaSource.builder()
> >>  ...
> >>  .readContinously() / readUntilLatestOffset() / readUntilTimestamp /
> readUntilOffsets / ...
> >>  .build()
> >>
> >> is way cleaner (and expressive) than
> >>
> >> Source source = KafkaSource.builder()
> >>  ...
> >>  .build()
> >>
> >>
> >> env.continousSource(source) // which actually underneath would call
> createEnumerator(CONTINUOUS, ctx) which would be equivalent to
> source.readContinously().createEnumerator(ctx)
> >> // or
> >> env.boundedSource(source) // which actually underneath would call
> createEnumerator(BOUNDED, ctx) which would be equivalent to
> source.readUntilLatestOffset().createEnumerator(ctx)
> >>
> >>
> >> Sorry for the comparison, but to me it seems there is too much magic
> >> happening underneath those two calls.
> >>
> >> I really believe the Source interface should have getBoundedness method
> >> instead of (supportBoundedness) + createEnumerator(Boundedness, ...)
> >>
> >>
> >> Issue #2 - Design of
> >> ExecutionEnvironment#source()/continuousSource()/boundedSource()
> >>
> >> As you might have guessed I am slightly in favor of option #2 modified.
> >> Yes I am aware every step of the dag would have to be able to say if it
> is
> >> bounded or not. I have a feeling it would be easier to express cross
> >> bounded/unbounded operations, but I must admit I have not thought it
> >> through thoroughly, In the spirit of batch is just a special case of
> >> streaming I thought BoundedStream would extend from DataStream. Correct
> me
> >> if I am wrong. In such a setup the cross bounded/unbounded operation
> could
> >> be expressed quite easily I think:
> >>
> >> DataStream {
> >>  DataStream join(DataStream, ...); // we could not really tell if the
> result is bounded or not, but because bounded stream is a special case of
> unbounded the API object is correct, irrespective if the left or right side
> of the join is bounded
> >> }
> >>
> >> BoundedStream extends DataStream {
> >>  BoundedStream join(BoundedStream, ...); // only if both sides are
> bounded the result can be bounded as well. However we do have access to the
> DataStream#join here, so you can still join with a DataStream
> >> }
> >>
> >>
> >> On the other hand I also see benefits of two completely disjointed APIs,
> >> as we could prohibit some streaming calls in the bounded API. I can't
> think
> >> of any unbounded operators that could not be implemented for bounded
> stream.
> >>
> >> Besides I think we both agree we don't like the method:
> >>
> >> DataStream boundedStream(Source)
> >>
> >> suggested in the current state of the FLIP. Do we ? :)
> >>
> >> Best,
> >>
> >> Dawid
> >>
> >> On 10/12/2019 18:57, Becket Qin wrote:
> >>
> >> Hi folks,
> >>
> >> Thanks for the discussion, great feedback. Also thanks Dawid for the
> >> explanation, it is much clearer now.
> >>
> >> One thing that is indeed missing from the FLIP is how the boundedness is
> >> passed to the Source implementation. So the API should be
> >> Source#createEnumerator(Boundedness boundedness, SplitEnumeratorContext
> >> context)
> >> And we can probably remove the Source#supportBoundedness(Boundedness
> >> boundedness) method.
> >>
> >> Assuming we have that, we are essentially choosing from one of the
> >> following two options:
> >>
> >> Option 1:
> >> // The source is continuous source, and only unbounded operations can be
> >> performed.
> >> DataStream<Type> datastream = env.continuousSource(someSource);
> >>
> >> // The source is bounded source, both bounded and unbounded operations
> can
> >> be performed.
> >> BoundedDataStream<Type> boundedDataStream =
> env.boundedSource(someSource);
> >>
> >>  - Pros:
> >>       a) explicit boundary between bounded / unbounded streams, it is
> >> quite simple and clear to the users.
> >>  - Cons:
> >>       a) For applications that do not involve bounded operations, they
> >> still have to call different API to distinguish bounded / unbounded
> streams.
> >>       b) No support for bounded stream to run in a streaming runtime
> >> setting, i.e. scheduling and operators behaviors.
> >>
> >>
> >> Option 2:
> >> // The source is either bounded or unbounded, but only unbounded
> operations
> >> could be performed on the returned DataStream.
> >> DataStream<Type> dataStream = env.source(someSource);
> >>
> >> // The source must be a bounded source, otherwise exception is thrown.
> >> BoundedDataStream<Type> boundedDataStream =
> >> env.boundedSource(boundedSource);
> >>
> >> The pros and cons are exactly the opposite of option 1.
> >>  - Pros:
> >>       a) For applications that do not involve bounded operations, they
> >> still have to call different API to distinguish bounded / unbounded
> streams.
> >>       b) Support for bounded stream to run in a streaming runtime
> setting,
> >> i.e. scheduling and operators behaviors.
> >>  - Cons:
> >>       a) Bounded / unbounded streams are kind of mixed, i.e. given a
> >> DataStream, it is not clear whether it is bounded or not, unless you
> have
> >> the access to its source.
> >>
> >>
> >> If we only think from the Source API perspective, option 2 seems a
> better
> >> choice because functionality wise it is a superset of option 1, at the
> cost
> >> of some seemingly acceptable ambiguity in the DataStream API.
> >> But if we look at the DataStream API as a whole, option 1 seems a
> clearer
> >> choice. For example, some times a library may have to know whether a
> >> certain task will finish or not. And it would be difficult to tell if
> the
> >> input is a DataStream, unless additional information is provided all the
> >> way from the Source. One possible solution is to have a *modified
> option 2*
> >> which adds a method to the DataStream API to indicate boundedness, such
> as
> >> getBoundedness(). It would solve the problem with a potential confusion
> of
> >> what is difference between a DataStream with getBoundedness()=true and a
> >> BoundedDataStream. But that seems not super difficult to explain.
> >>
> >> So from API's perspective, I don't have a strong opinion between
> *option 1*
> >> and *modified option 2. *I like the cleanness of option 1, but modified
> >> option 2 would be more attractive if we have concrete use case for the
> >> "Bounded stream with unbounded streaming runtime settings".
> >>
> >> Re: Till
> >>
> >>
> >> Maybe this has already been asked before but I was wondering why the
> >> SourceReader interface has the method pollNext which hands the
> >> responsibility of outputting elements to the SourceReader
> implementation?
> >> Has this been done for backwards compatibility reasons with the old
> source
> >> interface? If not, then one could define a Collection<E>
> getNextRecords()
> >> method which returns the currently retrieved records and then the caller
> >> emits them outside of the SourceReader. That way the interface would not
> >> allow to implement an outputting loop where we never hand back control
> to
> >> the caller. At the moment, this contract can be easily broken and is
> only
> >> mentioned loosely in the JavaDocs.
> >>
> >>
> >> The primary reason we handover the SourceOutput to the SourceReader is
> >> because sometimes it is difficult for a SourceReader to emit one record
> at
> >> a time. One example is some batched messaging systems which only have an
> >> offset for the entire batch instead of individual messages in the
> batch. In
> >> that case, returning one record at a time would leave the SourceReader
> in
> >> an uncheckpointable state because they can only checkpoint at the batch
> >> boundaries.
> >>
> >> Thanks,
> >>
> >> Jiangjie (Becket) Qin
> >>
> >> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <trohrmann@apache.org
> <ma...@apache.org>> <trohrmann@apache.org <mailto:
> trohrmann@apache.org>> wrote:
> >>
> >>
> >> Hi everyone,
> >>
> >> thanks for drafting this FLIP. It reads very well.
> >>
> >> Concerning Dawid's proposal, I tend to agree. The boundedness could come
> >> from the source and tell the system how to treat the operator
> (scheduling
> >> wise). From a user's perspective it should be fine to get back a
> DataStream
> >> when calling env.source(boundedSource) if he does not need special
> >> operations defined on a BoundedDataStream. If he needs this, then one
> could
> >> use the method BoundedDataStream env.boundedSource(boundedSource).
> >>
> >> If possible, we could enforce the proper usage of env.boundedSource() by
> >> introducing a BoundedSource type so that one cannot pass an
> >> unbounded source to it. That way users would not be able to shoot
> >> themselves in the foot.
> >>
> >> Maybe this has already been asked before but I was wondering why the
> >> SourceReader interface has the method pollNext which hands the
> >> responsibility of outputting elements to the SourceReader
> implementation?
> >> Has this been done for backwards compatibility reasons with the old
> source
> >> interface? If not, then one could define a Collection<E>
> getNextRecords()
> >> method which returns the currently retrieved records and then the caller
> >> emits them outside of the SourceReader. That way the interface would not
> >> allow to implement an outputting loop where we never hand back control
> to
> >> the caller. At the moment, this contract can be easily broken and is
> only
> >> mentioned loosely in the JavaDocs.
> >>
> >> Cheers,
> >> Till
> >>
> >> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <jingsonglee0@gmail.com
> <ma...@gmail.com>> <jingsonglee0@gmail.com <mailto:
> jingsonglee0@gmail.com>>
> >> wrote:
> >>
> >>
> >> Hi all,
> >>
> >> I think current design is good.
> >>
> >> My understanding is:
> >>
> >> For execution mode: bounded mode and continuous mode, It's totally
> >> different. I don't think we have the ability to integrate the two models
> >>
> >> at
> >>
> >> present. It's about scheduling, memory, algorithms, States, etc. we
> >> shouldn't confuse them.
> >>
> >> For source capabilities: only bounded, only continuous, both bounded and
> >> continuous.
> >> I think Kafka is a source that can be ran both bounded
> >> and continuous execution mode.
> >> And Kafka with end offset should be ran both bounded
> >> and continuous execution mode.  Using apache Beam with Flink runner, I
> >>
> >> used
> >>
> >> to run a "bounded" Kafka in streaming mode. For our previous DataStream,
> >>
> >> it
> >>
> >> is not necessarily required that the source cannot be bounded.
> >>
> >> So it is my thought for Dawid's question:
> >> 1.pass a bounded source to continuousSource() +1
> >> 2.pass a continuous source to boundedSource() -1, should throw
> exception.
> >>
> >> In StreamExecutionEnvironment, continuousSource and boundedSource define
> >> the execution mode. It defines a clear boundary of execution mode.
> >>
> >> Best,
> >> Jingsong Lee
> >>
> >> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com <mailto:
> imjark@gmail.com>> <imjark@gmail.com <ma...@gmail.com>> wrote:
> >>
> >>
> >> I agree with Dawid's point that the boundedness information should come
> >> from the source itself (e.g. the end timestamp), not through
> >> env.boundedSouce()/continuousSource().
> >> I think if we want to support something like `env.source()` that derive
> >>
> >> the
> >>
> >> execution mode from source, `supportsBoundedness(Boundedness)`
> >> method is not enough, because we don't know whether it is bounded or
> >>
> >> not.
> >>
> >> Best,
> >> Jark
> >>
> >>
> >> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <dwysakowicz@apache.org
> <ma...@apache.org>> <dwysakowicz@apache.org <mailto:
> dwysakowicz@apache.org>>
> >> wrote:
> >>
> >>
> >> One more thing. In the current proposal, with the
> >> supportsBoundedness(Boundedness) method and the boundedness coming
> >>
> >> from
> >>
> >> either continuousSource or boundedSource I could not find how this
> >> information is fed back to the SplitEnumerator.
> >>
> >> Best,
> >>
> >> Dawid
> >>
> >> On 09/12/2019 13:52, Becket Qin wrote:
> >>
> >> Hi Dawid,
> >>
> >> Thanks for the comments. This actually brings another relevant
> >>
> >> question
> >>
> >> about what does a "bounded source" imply. I actually had the same
> >> impression when I look at the Source API. Here is what I understand
> >>
> >> after
> >>
> >> some discussion with Stephan. The bounded source has the following
> >>
> >> impacts.
> >>
> >> 1. API validity.
> >> - A bounded source generates a bounded stream so some operations
> >>
> >> that
> >>
> >> only
> >>
> >> works for bounded records would be performed, e.g. sort.
> >> - To expose these bounded stream only APIs, there are two options:
> >>     a. Add them to the DataStream API and throw exception if a
> >>
> >> method
> >>
> >> is
> >>
> >> called on an unbounded stream.
> >>     b. Create a BoundedDataStream class which is returned from
> >> env.boundedSource(), while DataStream is returned from
> >>
> >> env.continousSource().
> >>
> >> Note that this cannot be done by having single
> >>
> >> env.source(theSource)
> >>
> >> even
> >>
> >> the Source has a getBoundedness() method.
> >>
> >> 2. Scheduling
> >> - A bounded source could be computed stage by stage without
> >>
> >> bringing
> >>
> >> up
> >>
> >> all
> >>
> >> the tasks at the same time.
> >>
> >> 3. Operator behaviors
> >> - A bounded source indicates the records are finite so some
> >>
> >> operators
> >>
> >> can
> >>
> >> wait until it receives all the records before it starts the
> >>
> >> processing.
> >>
> >> In the above impact, only 1 is relevant to the API design. And the
> >>
> >> current
> >>
> >> proposal in FLIP-27 is following 1.b.
> >>
> >> // boundedness depends of source property, imo this should always
> >>
> >> be
> >>
> >> preferred
> >>
> >>
> >> DataStream<MyType> stream = env.source(theSource);
> >>
> >>
> >> In your proposal, does DataStream have bounded stream only methods?
> >>
> >> It
> >>
> >> looks it should have, otherwise passing a bounded Source to
> >>
> >> env.source()
> >>
> >> would be confusing. In that case, we will essentially do 1.a if an
> >> unbounded Source is created from env.source(unboundedSource).
> >>
> >> If we have the methods only supported for bounded streams in
> >>
> >> DataStream,
> >>
> >> it
> >>
> >> seems a little weird to have a separate BoundedDataStream
> >>
> >> interface.
> >>
> >> Am I understand it correctly?
> >>
> >> Thanks,
> >>
> >> Jiangjie (Becket) Qin
> >>
> >>
> >>
> >> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> >>
> >> dwysakowicz@apache.org <ma...@apache.org>>
> >>
> >> wrote:
> >>
> >>
> >> Hi all,
> >>
> >> Really well written proposal and very important one. I must admit
> >>
> >> I
> >>
> >> have
> >>
> >> not understood all the intricacies of it yet.
> >>
> >> One question I have though is about where does the information
> >>
> >> about
> >>
> >> boundedness come from. I think in most cases it is a property of
> >>
> >> the
> >>
> >> source. As you described it might be e.g. end offset, a flag
> >>
> >> should
> >>
> >> it
> >>
> >> monitor new splits etc. I think it would be a really nice use case
> >>
> >> to
> >>
> >> be
> >>
> >> able to say:
> >>
> >> new KafkaSource().readUntil(long timestamp),
> >>
> >> which could work as an "end offset". Moreover I think all Bounded
> >>
> >> sources
> >>
> >> support continuous mode, but no intrinsically continuous source
> >>
> >> support
> >>
> >> the
> >>
> >> Bounded mode. If I understood the proposal correctly it suggest
> >>
> >> the
> >>
> >> boundedness sort of "comes" from the outside of the source, from
> >>
> >> the
> >>
> >> invokation of either boundedStream or continousSource.
> >>
> >> I am wondering if it would make sense to actually change the
> >>
> >> method
> >>
> >> boolean Source#supportsBoundedness(Boundedness)
> >>
> >> to
> >>
> >> Boundedness Source#getBoundedness().
> >>
> >> As for the methods #boundedSource, #continousSource, assuming the
> >> boundedness is property of the source they do not affect how the
> >>
> >> enumerator
> >>
> >> works, but mostly how the dag is scheduled, right? I am not
> >>
> >> against
> >>
> >> those
> >>
> >> methods, but I think it is a very specific use case to actually
> >>
> >> override
> >>
> >> the property of the source. In general I would expect users to
> >>
> >> only
> >>
> >> call
> >>
> >> env.source(theSource), where the source tells if it is bounded or
> >>
> >> not. I
> >>
> >> would suggest considering following set of methods:
> >>
> >> // boundedness depends of source property, imo this should always
> >>
> >> be
> >>
> >> preferred
> >>
> >> DataStream<MyType> stream = env.source(theSource);
> >>
> >>
> >> // always continous execution, whether bounded or unbounded source
> >>
> >> DataStream<MyType> boundedStream = env.continousSource(theSource);
> >>
> >> // imo this would make sense if the BoundedDataStream provides
> >>
> >> additional features unavailable for continous mode
> >>
> >> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
> >>
> >>
> >> Best,
> >>
> >> Dawid
> >>
> >>
> >> On 04/12/2019 11:25, Stephan Ewen wrote:
> >>
> >> Thanks, Becket, for updating this.
> >>
> >> I agree with moving the aspects you mentioned into separate FLIPs
> >>
> >> -
> >>
> >> this
> >>
> >> one way becoming unwieldy in size.
> >>
> >> +1 to the FLIP in its current state. Its a very detailed write-up,
> >>
> >> nicely
> >>
> >> done!
> >>
> >> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <becket.qin@gmail.com
> <ma...@gmail.com>> <becket.qin@gmail.com <mailto:
> becket.qin@gmail.com>>
> >>
> >> <
> >>
> >> becket.qin@gmail.com <ma...@gmail.com>> wrote:
> >>
> >> Hi all,
> >>
> >> Sorry for the long belated update. I have updated FLIP-27 wiki
> >>
> >> page
> >>
> >> with
> >>
> >> the latest proposals. Some noticeable changes include:
> >> 1. A new generic communication mechanism between SplitEnumerator
> >>
> >> and
> >>
> >> SourceReader.
> >> 2. Some detail API method signature changes.
> >>
> >> We left a few things out of this FLIP and will address them in
> >>
> >> separate
> >>
> >> FLIPs. Including:
> >> 1. Per split event time.
> >> 2. Event time alignment.
> >> 3. Fine grained failover for SplitEnumerator failure.
> >>
> >> Please let us know if you have any question.
> >>
> >> Thanks,
> >>
> >> Jiangjie (Becket) Qin
> >>
> >> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <sewen@apache.org <mailto:
> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>> <
> >>
> >> sewen@apache.org <ma...@apache.org>> wrote:
> >>
> >> Hi  Łukasz!
> >>
> >> Becket and me are working hard on figuring out the last details
> >>
> >> and
> >>
> >> implementing the first PoC. We would update the FLIP hopefully
> >>
> >> next
> >>
> >> week.
> >>
> >> There is a fair chance that a first version of this will be in
> >>
> >> 1.10,
> >>
> >> but
> >>
> >> I
> >>
> >> think it will take another release to battle test it and migrate
> >>
> >> the
> >>
> >> connectors.
> >>
> >> Best,
> >> Stephan
> >>
> >>
> >>
> >>
> >> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <ljd@touk.pl
> <ma...@touk.pl>
> >>
> >> <
> >>
> >> ljd@touk.pl <ma...@touk.pl>>
> >>
> >> wrote:
> >>
> >> Hi,
> >>
> >> This proposal looks very promising for us. Do you have any plans
> >>
> >> in
> >>
> >> which
> >>
> >> Flink release it is going to be released? We are thinking on
> >>
> >> using a
> >>
> >> Data
> >>
> >> Set API for our future use cases but on the other hand Data Set
> >>
> >> API
> >>
> >> is
> >>
> >> going to be deprecated so using proposed bounded data streams
> >>
> >> solution
> >>
> >> could be more viable in the long term.
> >>
> >> Thanks,
> >> Łukasz
> >>
> >> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com <mailto:
> thomas.weise@gmail.com>> <thomas.weise@gmail.com <mailto:
> thomas.weise@gmail.com>> <
> >>
> >> thomas.weise@gmail.com <ma...@gmail.com>> wrote:
> >>
> >> Thanks for putting together this proposal!
> >>
> >> I see that the "Per Split Event Time" and "Event Time Alignment"
> >>
> >> sections
> >>
> >> are still TBD.
> >>
> >> It would probably be good to flesh those out a bit before
> >>
> >> proceeding
> >>
> >> too
> >>
> >> far
> >>
> >> as the event time alignment will probably influence the
> >>
> >> interaction
> >>
> >> with
> >>
> >> the split reader, specifically ReaderStatus
> >>
> >> emitNext(SourceOutput<E>
> >>
> >> output).
> >>
> >> We currently have only one implementation for event time alignment
> >>
> >> in
> >>
> >> the
> >>
> >> Kinesis consumer. The synchronization in that case takes place as
> >>
> >> the
> >>
> >> last
> >>
> >> step before records are emitted downstream (RecordEmitter). With
> >>
> >> the
> >>
> >> currently proposed interfaces, the equivalent can be implemented
> >>
> >> in
> >>
> >> the
> >>
> >> reader loop, although note that in the Kinesis consumer the per
> >>
> >> shard
> >>
> >> threads push records.
> >>
> >> Synchronization has not been implemented for the Kafka consumer
> >>
> >> yet.
> >>
> >> https://issues.apache.org/jira/browse/FLINK-12675 <
> https://issues.apache.org/jira/browse/FLINK-12675>
> >>
> >> When I looked at it, I realized that the implementation will look
> >>
> >> quite
> >>
> >> different
> >> from Kinesis because it needs to take place in the pull part,
> >>
> >> where
> >>
> >> records
> >>
> >> are taken from the Kafka client. Due to the multiplexing it cannot
> >>
> >> be
> >>
> >> done
> >>
> >> by blocking the split thread like it currently works for Kinesis.
> >>
> >> Reading
> >>
> >> from individual Kafka partitions needs to be controlled via
> >>
> >> pause/resume
> >>
> >> on the Kafka client.
> >>
> >> To take on that responsibility the split thread would need to be
> >>
> >> aware
> >>
> >> of
> >>
> >> the
> >> watermarks or at least whether it should or should not continue to
> >>
> >> consume
> >>
> >> a given split and this may require a different SourceReader or
> >>
> >> SourceOutput
> >>
> >> interface.
> >>
> >> Thanks,
> >> Thomas
> >>
> >>
> >> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com <mailto:
> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
> >>
> >> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
> >>
> >> Hi Stephan,
> >>
> >> Thank you for feedback!
> >> Will take a look at your branch before public discussing.
> >>
> >>
> >> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <sewen@apache.org
> <ma...@apache.org>> <sewen@apache.org <ma...@apache.org>>
> >>
> >> <
> >>
> >> sewen@apache.org <ma...@apache.org>>
> >>
> >> wrote:
> >>
> >> Hi Biao!
> >>
> >> Thanks for reviving this. I would like to join this discussion,
> >>
> >> but
> >>
> >> am
> >>
> >> quite occupied with the 1.9 release, so can we maybe pause this
> >>
> >> discussion
> >>
> >> for a week or so?
> >>
> >> In the meantime I can share some suggestion based on prior
> >>
> >> experiments:
> >>
> >> How to do watermarks / timestamp extractors in a simpler and more
> >>
> >> flexible
> >>
> >> way. I think that part is quite promising should be part of the
> >>
> >> new
> >>
> >> source
> >>
> >> interface.
> >>
> >>
> >>
> >>
> >>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> <
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> >
> >>
> >>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> <
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> >
> >>
> >> Some experiments on how to build the source reader and its
> >>
> >> library
> >>
> >> for
> >>
> >> common threading/split patterns:
> >>
> >>
> >>
> >>
> >>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> <
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> >
> >>
> >> Best,
> >> Stephan
> >>
> >>
> >> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mmyy1110@gmail.com <mailto:
> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
> >>
> >> mmyy1110@gmail.com <ma...@gmail.com>>
> >>
> >> wrote:
> >>
> >> Hi devs,
> >>
> >> Since 1.9 is nearly released, I think we could get back to
> >>
> >> FLIP-27.
> >>
> >> I
> >>
> >> believe it should be included in 1.10.
> >>
> >> There are so many things mentioned in document of FLIP-27. [1] I
> >>
> >> think
> >>
> >> we'd better discuss them separately. However the wiki is not a
> >>
> >> good
> >>
> >> place
> >>
> >> to discuss. I wrote google doc about SplitReader API which
> >>
> >> misses
> >>
> >> some
> >>
> >> details in the document. [2]
> >>
> >> 1.
> >>
> >>
> >>
> >>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> <
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >
> >>
> >> 2.
> >>
> >>
> >>
> >>
> >>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> <
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> >
> >>
> >> CC Stephan, Aljoscha, Piotrek, Becket
> >>
> >>
> >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com <mailto:
> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
> >>
> >> mmyy1110@gmail.com <ma...@gmail.com>>
> >>
> >> wrote:
> >>
> >> Hi Steven,
> >> Thank you for the feedback. Please take a look at the document
> >>
> >> FLIP-27
> >>
> >> <
> >>
> >>
> >>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> <
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >
> >>
> >> which
> >>
> >> is updated recently. A lot of details of enumerator were added
> >>
> >> in
> >>
> >> this
> >>
> >> document. I think it would help.
> >>
> >> Steven Wu <stevenz3wu@gmail.com <ma...@gmail.com>> <
> stevenz3wu@gmail.com <ma...@gmail.com>> <stevenz3wu@gmail.com
> <ma...@gmail.com>> <stevenz3wu@gmail.com <mailto:
> stevenz3wu@gmail.com>>
> >>
> >> 于2019年3月28日周四
> >>
> >> 下午12:52写道:
> >>
> >> This proposal mentioned that SplitEnumerator might run on the
> >> JobManager or
> >> in a single task on a TaskManager.
> >>
> >> if enumerator is a single task on a taskmanager, then the job
> >>
> >> DAG
> >>
> >> can
> >>
> >> never
> >> been embarrassingly parallel anymore. That will nullify the
> >>
> >> leverage
> >>
> >> of
> >>
> >> fine-grained recovery for embarrassingly parallel jobs.
> >>
> >> It's not clear to me what's the implication of running
> >>
> >> enumerator
> >>
> >> on
> >>
> >> the
> >>
> >> jobmanager. So I will leave that out for now.
> >>
> >> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com <mailto:
> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
> >>
> >> mmyy1110@gmail.com <ma...@gmail.com>>
> >>
> >> wrote:
> >>
> >> Hi Stephan & Piotrek,
> >>
> >> Thank you for feedback.
> >>
> >> It seems that there are a lot of things to do in community.
> >>
> >> I
> >>
> >> am
> >>
> >> just
> >>
> >> afraid that this discussion may be forgotten since there so
> >>
> >> many
> >>
> >> proposals
> >>
> >> recently.
> >> Anyway, wish to see the split topics soon :)
> >>
> >> Piotr Nowojski <piotr@da-platform.com <ma...@da-platform.com>> <
> piotr@da-platform.com <ma...@da-platform.com>> <
> piotr@da-platform.com <ma...@da-platform.com>> <
> piotr@da-platform.com <ma...@da-platform.com>>
> >>
> >> 于2019年1月24日周四
> >>
> >> 下午8:21写道:
> >>
> >> Hi Biao!
> >>
> >> This discussion was stalled because of preparations for
> >>
> >> the
> >>
> >> open
> >>
> >> sourcing
> >>
> >> & merging Blink. I think before creating the tickets we
> >>
> >> should
> >>
> >> split this
> >>
> >> discussion into topics/areas outlined by Stephan and
> >>
> >> create
> >>
> >> Flips
> >>
> >> for
> >>
> >> that.
> >>
> >> I think there is no chance for this to be completed in
> >>
> >> couple
> >>
> >> of
> >>
> >> remaining
> >>
> >> weeks/1 month before 1.8 feature freeze, however it would
> >>
> >> be
> >>
> >> good
> >>
> >> to aim
> >>
> >> with those changes for 1.9.
> >>
> >> Piotrek
> >>
> >>
> >> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com <mailto:
> mmyy1110@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
> >>
> >> mmyy1110@gmail.com <ma...@gmail.com>>
> >>
> >> wrote:
> >>
> >> Hi community,
> >> The summary of Stephan makes a lot sense to me. It is
> >>
> >> much
> >>
> >> clearer
> >>
> >> indeed
> >>
> >> after splitting the complex topic into small ones.
> >> I was wondering is there any detail plan for next step?
> >>
> >> If
> >>
> >> not,
> >>
> >> I
> >>
> >> would
> >>
> >> like to push this thing forward by creating some JIRA
> >>
> >> issues.
> >>
> >> Another question is that should version 1.8 include
> >>
> >> these
> >>
> >> features?
> >>
> >> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <
> sewen@apache.org <ma...@apache.org>> <sewen@apache.org <mailto:
> sewen@apache.org>> <sewen@apache.org <ma...@apache.org>>
> 于2018年12月1日周六
> >>
> >> 上午4:20写道:
> >>
> >> Thanks everyone for the lively discussion. Let me try
> >>
> >> to
> >>
> >> summarize
> >>
> >> where I
> >>
> >> see convergence in the discussion and open issues.
> >> I'll try to group this by design aspect of the source.
> >>
> >> Please
> >>
> >> let me
> >>
> >> know
> >>
> >> if I got things wrong or missed something crucial here.
> >>
> >> For issues 1-3, if the below reflects the state of the
> >>
> >> discussion, I
> >>
> >> would
> >>
> >> try and update the FLIP in the next days.
> >> For the remaining ones we need more discussion.
> >>
> >> I would suggest to fork each of these aspects into a
> >>
> >> separate
> >>
> >> mail
> >>
> >> thread,
> >>
> >> or will loose sight of the individual aspects.
> >>
> >> *(1) Separation of Split Enumerator and Split Reader*
> >>
> >> - All seem to agree this is a good thing
> >> - Split Enumerator could in the end live on JobManager
> >>
> >> (and
> >>
> >> assign
> >>
> >> splits
> >>
> >> via RPC) or in a task (and assign splits via data
> >>
> >> streams)
> >>
> >> - this discussion is orthogonal and should come later,
> >>
> >> when
> >>
> >> the
> >>
> >> interface
> >>
> >> is agreed upon.
> >>
> >> *(2) Split Readers for one or more splits*
> >>
> >> - Discussion seems to agree that we need to support
> >>
> >> one
> >>
> >> reader
> >>
> >> that
> >>
> >> possibly handles multiple splits concurrently.
> >> - The requirement comes from sources where one
> >>
> >> poll()-style
> >>
> >> call
> >>
> >> fetches
> >>
> >> data from different splits / partitions
> >>   --> example sources that require that would be for
> >>
> >> example
> >>
> >> Kafka,
> >>
> >> Pravega, Pulsar
> >>
> >> - Could have one split reader per source, or multiple
> >>
> >> split
> >>
> >> readers
> >>
> >> that
> >>
> >> share the "poll()" function
> >> - To not make it too complicated, we can start with
> >>
> >> thinking
> >>
> >> about
> >>
> >> one
> >>
> >> split reader for all splits initially and see if that
> >>
> >> covers
> >>
> >> all
> >>
> >> requirements
> >>
> >> *(3) Threading model of the Split Reader*
> >>
> >> - Most active part of the discussion ;-)
> >>
> >> - A non-blocking way for Flink's task code to interact
> >>
> >> with
> >>
> >> the
> >>
> >> source
> >>
> >> is
> >>
> >> needed in order to a task runtime code based on a
> >> single-threaded/actor-style task design
> >>   --> I personally am a big proponent of that, it will
> >>
> >> help
> >>
> >> with
> >>
> >> well-behaved checkpoints, efficiency, and simpler yet
> >>
> >> more
> >>
> >> robust
> >>
> >> runtime
> >>
> >> code
> >>
> >> - Users care about simple abstraction, so as a
> >>
> >> subclass
> >>
> >> of
> >>
> >> SplitReader
> >>
> >> (non-blocking / async) we need to have a
> >>
> >> BlockingSplitReader
> >>
> >> which
> >>
> >> will
> >>
> >> form the basis of most source implementations.
> >>
> >> BlockingSplitReader
> >>
> >> lets
> >>
> >> users do blocking simple poll() calls.
> >> - The BlockingSplitReader would spawn a thread (or
> >>
> >> more)
> >>
> >> and
> >>
> >> the
> >>
> >> thread(s) can make blocking calls and hand over data
> >>
> >> buffers
> >>
> >> via
> >>
> >> a
> >>
> >> blocking
> >>
> >> queue
> >> - This should allow us to cover both, a fully async
> >>
> >> runtime,
> >>
> >> and a
> >>
> >> simple
> >>
> >> blocking interface for users.
> >> - This is actually very similar to how the Kafka
> >>
> >> connectors
> >>
> >> work.
> >>
> >> Kafka
> >>
> >> 9+ with one thread, Kafka 8 with multiple threads
> >>
> >> - On the base SplitReader (the async one), the
> >>
> >> non-blocking
> >>
> >> method
> >>
> >> that
> >>
> >> gets the next chunk of data would signal data
> >>
> >> availability
> >>
> >> via
> >>
> >> a
> >>
> >> CompletableFuture, because that gives the best
> >>
> >> flexibility
> >>
> >> (can
> >>
> >> await
> >>
> >> completion or register notification handlers).
> >> - The source task would register a "thenHandle()" (or
> >>
> >> similar)
> >>
> >> on the
> >>
> >> future to put a "take next data" task into the
> >>
> >> actor-style
> >>
> >> mailbox
> >>
> >> *(4) Split Enumeration and Assignment*
> >>
> >> - Splits may be generated lazily, both in cases where
> >>
> >> there
> >>
> >> is a
> >>
> >> limited
> >>
> >> number of splits (but very many), or splits are
> >>
> >> discovered
> >>
> >> over
> >>
> >> time
> >>
> >> - Assignment should also be lazy, to get better load
> >>
> >> balancing
> >>
> >> - Assignment needs support locality preferences
> >>
> >> - Possible design based on discussion so far:
> >>
> >>   --> SplitReader has a method "addSplits(SplitT...)"
> >>
> >> to
> >>
> >> add
> >>
> >> one or
> >>
> >> more
> >>
> >> splits. Some split readers might assume they have only
> >>
> >> one
> >>
> >> split
> >>
> >> ever,
> >>
> >> concurrently, others assume multiple splits. (Note:
> >>
> >> idea
> >>
> >> behind
> >>
> >> being
> >>
> >> able
> >>
> >> to add multiple splits at the same time is to ease
> >>
> >> startup
> >>
> >> where
> >>
> >> multiple
> >>
> >> splits may be assigned instantly.)
> >>   --> SplitReader has a context object on which it can
> >>
> >> call
> >>
> >> indicate
> >>
> >> when
> >>
> >> splits are completed. The enumerator gets that
> >>
> >> notification and
> >>
> >> can
> >>
> >> use
> >>
> >> to
> >>
> >> decide when to assign new splits. This should help both
> >>
> >> in
> >>
> >> cases
> >>
> >> of
> >>
> >> sources
> >>
> >> that take splits lazily (file readers) and in case the
> >>
> >> source
> >>
> >> needs to
> >>
> >> preserve a partial order between splits (Kinesis,
> >>
> >> Pravega,
> >>
> >> Pulsar may
> >>
> >> need
> >>
> >> that).
> >>   --> SplitEnumerator gets notification when
> >>
> >> SplitReaders
> >>
> >> start
> >>
> >> and
> >>
> >> when
> >>
> >> they finish splits. They can decide at that moment to
> >>
> >> push
> >>
> >> more
> >>
> >> splits
> >>
> >> to
> >>
> >> that reader
> >>   --> The SplitEnumerator should probably be aware of
> >>
> >> the
> >>
> >> source
> >>
> >> parallelism, to build its initial distribution.
> >>
> >> - Open question: Should the source expose something
> >>
> >> like
> >>
> >> "host
> >>
> >> preferences", so that yarn/mesos/k8s can take this into
> >>
> >> account
> >>
> >> when
> >>
> >> selecting a node to start a TM on?
> >>
> >> *(5) Watermarks and event time alignment*
> >>
> >> - Watermark generation, as well as idleness, needs to
> >>
> >> be
> >>
> >> per
> >>
> >> split
> >>
> >> (like
> >>
> >> currently in the Kafka Source, per partition)
> >> - It is desirable to support optional
> >>
> >> event-time-alignment,
> >>
> >> meaning
> >>
> >> that
> >>
> >> splits that are ahead are back-pressured or temporarily
> >>
> >> unsubscribed
> >>
> >> - I think i would be desirable to encapsulate
> >>
> >> watermark
> >>
> >> generation
> >>
> >> logic
> >>
> >> in watermark generators, for a separation of concerns.
> >>
> >> The
> >>
> >> watermark
> >>
> >> generators should run per split.
> >> - Using watermark generators would also help with
> >>
> >> another
> >>
> >> problem of
> >>
> >> the
> >>
> >> suggested interface, namely supporting non-periodic
> >>
> >> watermarks
> >>
> >> efficiently.
> >>
> >> - Need a way to "dispatch" next record to different
> >>
> >> watermark
> >>
> >> generators
> >>
> >> - Need a way to tell SplitReader to "suspend" a split
> >>
> >> until a
> >>
> >> certain
> >>
> >> watermark is reached (event time backpressure)
> >> - This would in fact be not needed (and thus simpler)
> >>
> >> if
> >>
> >> we
> >>
> >> had
> >>
> >> a
> >>
> >> SplitReader per split and may be a reason to re-open
> >>
> >> that
> >>
> >> discussion
> >>
> >> *(6) Watermarks across splits and in the Split
> >>
> >> Enumerator*
> >>
> >> - The split enumerator may need some watermark
> >>
> >> awareness,
> >>
> >> which
> >>
> >> should
> >>
> >> be
> >>
> >> purely based on split metadata (like create timestamp
> >>
> >> of
> >>
> >> file
> >>
> >> splits)
> >>
> >> - If there are still more splits with overlapping
> >>
> >> event
> >>
> >> time
> >>
> >> range
> >>
> >> for
> >>
> >> a
> >>
> >> split reader, then that split reader should not advance
> >>
> >> the
> >>
> >> watermark
> >>
> >> within the split beyond the overlap boundary. Otherwise
> >>
> >> future
> >>
> >> splits
> >>
> >> will
> >>
> >> produce late data.
> >>
> >> - One way to approach this could be that the split
> >>
> >> enumerator
> >>
> >> may
> >>
> >> send
> >>
> >> watermarks to the readers, and the readers cannot emit
> >>
> >> watermarks
> >>
> >> beyond
> >>
> >> that received watermark.
> >> - Many split enumerators would simply immediately send
> >>
> >> Long.MAX
> >>
> >> out
> >>
> >> and
> >>
> >> leave the progress purely to the split readers.
> >>
> >> - For event-time alignment / split back pressure, this
> >>
> >> begs
> >>
> >> the
> >>
> >> question
> >>
> >> how we can avoid deadlocks that may arise when splits
> >>
> >> are
> >>
> >> suspended
> >>
> >> for
> >>
> >> event time back pressure,
> >>
> >> *(7) Batch and streaming Unification*
> >>
> >> - Functionality wise, the above design should support
> >>
> >> both
> >>
> >> - Batch often (mostly) does not care about reading "in
> >>
> >> order"
> >>
> >> and
> >>
> >> generating watermarks
> >>   --> Might use different enumerator logic that is
> >>
> >> more
> >>
> >> locality
> >>
> >> aware
> >>
> >> and ignores event time order
> >>   --> Does not generate watermarks
> >> - Would be great if bounded sources could be
> >>
> >> identified
> >>
> >> at
> >>
> >> compile
> >>
> >> time,
> >>
> >> so that "env.addBoundedSource(...)" is type safe and
> >>
> >> can
> >>
> >> return a
> >>
> >> "BoundedDataStream".
> >> - Possible to defer this discussion until later
> >>
> >> *Miscellaneous Comments*
> >>
> >> - Should the source have a TypeInformation for the
> >>
> >> produced
> >>
> >> type,
> >>
> >> instead
> >>
> >> of a serializer? We need a type information in the
> >>
> >> stream
> >>
> >> anyways, and
> >>
> >> can
> >>
> >> derive the serializer from that. Plus, creating the
> >>
> >> serializer
> >>
> >> should
> >>
> >> respect the ExecutionConfig.
> >>
> >> - The TypeSerializer interface is very powerful but
> >>
> >> also
> >>
> >> not
> >>
> >> easy to
> >>
> >> implement. Its purpose is to handle data super
> >>
> >> efficiently,
> >>
> >> support
> >>
> >> flexible ways of evolution, etc.
> >> For metadata I would suggest to look at the
> >>
> >> SimpleVersionedSerializer
> >>
> >> instead, which is used for example for checkpoint
> >>
> >> master
> >>
> >> hooks,
> >>
> >> or for
> >>
> >> the
> >>
> >> streaming file sink. I think that is is a good match
> >>
> >> for
> >>
> >> cases
> >>
> >> where
> >>
> >> we
> >>
> >> do
> >>
> >> not need more than ser/deser (no copy, etc.) and don't
> >>
> >> need to
> >>
> >> push
> >>
> >> versioning out of the serialization paths for best
> >>
> >> performance
> >>
> >> (as in
> >>
> >> the
> >>
> >> TypeSerializer)
> >>
> >>
> >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> >>
> >> k.kloudas@data-artisans.com>
> >>
> >> wrote:
> >>
> >>
> >> Hi Biao,
> >>
> >> Thanks for the answer!
> >>
> >> So given the multi-threaded readers, now we have as
> >>
> >> open
> >>
> >> questions:
> >>
> >> 1) How do we let the checkpoints pass through our
> >>
> >> multi-threaded
> >>
> >> reader
> >>
> >> operator?
> >>
> >> 2) Do we have separate reader and source operators or
> >>
> >> not? In
> >>
> >> the
> >>
> >> strategy
> >>
> >> that has a separate source, the source operator has a
> >>
> >> parallelism of
> >>
> >> 1
> >>
> >> and
> >>
> >> is responsible for split recovery only.
> >>
> >> For the first one, given also the constraints
> >>
> >> (blocking,
> >>
> >> finite
> >>
> >> queues,
> >>
> >> etc), I do not have an answer yet.
> >>
> >> For the 2nd, I think that we should go with separate
> >>
> >> operators
> >>
> >> for
> >>
> >> the
> >>
> >> source and the readers, for the following reasons:
> >>
> >> 1) This is more aligned with a potential future
> >>
> >> improvement
> >>
> >> where the
> >>
> >> split
> >>
> >> discovery becomes a responsibility of the JobManager
> >>
> >> and
> >>
> >> readers are
> >>
> >> pooling more work from the JM.
> >>
> >> 2) The source is going to be the "single point of
> >>
> >> truth".
> >>
> >> It
> >>
> >> will
> >>
> >> know
> >>
> >> what
> >>
> >> has been processed and what not. If the source and the
> >>
> >> readers
> >>
> >> are a
> >>
> >> single
> >>
> >> operator with parallelism > 1, or in general, if the
> >>
> >> split
> >>
> >> discovery
> >>
> >> is
> >>
> >> done by each task individually, then:
> >>  i) we have to have a deterministic scheme for each
> >>
> >> reader to
> >>
> >> assign
> >>
> >> splits to itself (e.g. mod subtaskId). This is not
> >>
> >> necessarily
> >>
> >> trivial
> >>
> >> for
> >>
> >> all sources.
> >>  ii) each reader would have to keep a copy of all its
> >>
> >> processed
> >>
> >> slpits
> >>
> >>  iii) the state has to be a union state with a
> >>
> >> non-trivial
> >>
> >> merging
> >>
> >> logic
> >>
> >> in order to support rescaling.
> >>
> >> Two additional points that you raised above:
> >>
> >> i) The point that you raised that we need to keep all
> >>
> >> splits
> >>
> >> (processed
> >>
> >> and
> >>
> >> not-processed) I think is a bit of a strong
> >>
> >> requirement.
> >>
> >> This
> >>
> >> would
> >>
> >> imply
> >>
> >> that for infinite sources the state will grow
> >>
> >> indefinitely.
> >>
> >> This is
> >>
> >> problem
> >>
> >> is even more pronounced if we do not have a single
> >>
> >> source
> >>
> >> that
> >>
> >> assigns
> >>
> >> splits to readers, as each reader will have its own
> >>
> >> copy
> >>
> >> of
> >>
> >> the
> >>
> >> state.
> >>
> >> ii) it is true that for finite sources we need to
> >>
> >> somehow
> >>
> >> not
> >>
> >> close
> >>
> >> the
> >>
> >> readers when the source/split discoverer finishes. The
> >> ContinuousFileReaderOperator has a work-around for
> >>
> >> that.
> >>
> >> It is
> >>
> >> not
> >>
> >> elegant,
> >>
> >> and checkpoints are not emitted after closing the
> >>
> >> source,
> >>
> >> but
> >>
> >> this, I
> >>
> >> believe, is a bigger problem which requires more
> >>
> >> changes
> >>
> >> than
> >>
> >> just
> >>
> >> refactoring the source interface.
> >>
> >> Cheers,
> >> Kostas
> >>
> >>
> >>
> >>
> >> --
> >> Best, Jingsong Lee
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@ververica.com>.
Hi,

Regarding the:

Collection<E> getNextRecords()

I’m pretty sure such design would unfortunately impact the performance (accessing and potentially creating the collection on the hot path).

Also the

InputStatus emitNext(DataOutput<T> output) throws Exception;
or
Status pollNext(SourceOutput<T> sourceOutput) throws Exception;

Gives us some opportunities in the future, to allow Source hot looping inside, until it receives some signal “please exit because of some reasons” (output collector could return such hint upon collecting the result). But that’s another topic outside of this FLIP’s scope.

Piotrek

> On 11 Dec 2019, at 10:41, Till Rohrmann <tr...@apache.org> wrote:
> 
> Hi Becket,
> 
> quick clarification from my side because I think you misunderstood my
> question. I did not suggest to let the SourceReader return only a single
> record at a time when calling getNextRecords. As the return type indicates,
> the method can return an arbitrary number of records.
> 
> Cheers,
> Till
> 
> On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <dwysakowicz@apache.org <ma...@apache.org>>
> wrote:
> 
>> Hi Becket,
>> 
>> Issue #1 - Design of Source interface
>> 
>> I mentioned the lack of a method like Source#createEnumerator(Boundedness
>> boundedness, SplitEnumeratorContext context), because without the current
>> proposal is not complete/does not work.
>> 
>> If we say that boundedness is an intrinsic property of a source imo we
>> don't need the Source#createEnumerator(Boundedness boundedness,
>> SplitEnumeratorContext context) method.
>> 
>> Assuming a source from my previous example:
>> 
>> Source source = KafkaSource.builder()
>>  ...
>>  .untilTimestamp(...)
>>  .build()
>> 
>> Would the enumerator differ if created like
>> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
>> .createEnumerator(BOUNDED, ...)? I know I am repeating myself, but this is
>> the part that my opinion differ the most from the current proposal. I
>> really think it should always be the source that tells if it is bounded or
>> not. In the current proposal methods continousSource/boundedSource somewhat
>> reconfigure the source, which I think is misleading.
>> 
>> I think a call like:
>> 
>> Source source = KafkaSource.builder()
>>  ...
>>  .readContinously() / readUntilLatestOffset() / readUntilTimestamp / readUntilOffsets / ...
>>  .build()
>> 
>> is way cleaner (and expressive) than
>> 
>> Source source = KafkaSource.builder()
>>  ...
>>  .build()
>> 
>> 
>> env.continousSource(source) // which actually underneath would call createEnumerator(CONTINUOUS, ctx) which would be equivalent to source.readContinously().createEnumerator(ctx)
>> // or
>> env.boundedSource(source) // which actually underneath would call createEnumerator(BOUNDED, ctx) which would be equivalent to source.readUntilLatestOffset().createEnumerator(ctx)
>> 
>> 
>> Sorry for the comparison, but to me it seems there is too much magic
>> happening underneath those two calls.
>> 
>> I really believe the Source interface should have getBoundedness method
>> instead of (supportBoundedness) + createEnumerator(Boundedness, ...)
>> 
>> 
>> Issue #2 - Design of
>> ExecutionEnvironment#source()/continuousSource()/boundedSource()
>> 
>> As you might have guessed I am slightly in favor of option #2 modified.
>> Yes I am aware every step of the dag would have to be able to say if it is
>> bounded or not. I have a feeling it would be easier to express cross
>> bounded/unbounded operations, but I must admit I have not thought it
>> through thoroughly, In the spirit of batch is just a special case of
>> streaming I thought BoundedStream would extend from DataStream. Correct me
>> if I am wrong. In such a setup the cross bounded/unbounded operation could
>> be expressed quite easily I think:
>> 
>> DataStream {
>>  DataStream join(DataStream, ...); // we could not really tell if the result is bounded or not, but because bounded stream is a special case of unbounded the API object is correct, irrespective if the left or right side of the join is bounded
>> }
>> 
>> BoundedStream extends DataStream {
>>  BoundedStream join(BoundedStream, ...); // only if both sides are bounded the result can be bounded as well. However we do have access to the DataStream#join here, so you can still join with a DataStream
>> }
>> 
>> 
>> On the other hand I also see benefits of two completely disjointed APIs,
>> as we could prohibit some streaming calls in the bounded API. I can't think
>> of any unbounded operators that could not be implemented for bounded stream.
>> 
>> Besides I think we both agree we don't like the method:
>> 
>> DataStream boundedStream(Source)
>> 
>> suggested in the current state of the FLIP. Do we ? :)
>> 
>> Best,
>> 
>> Dawid
>> 
>> On 10/12/2019 18:57, Becket Qin wrote:
>> 
>> Hi folks,
>> 
>> Thanks for the discussion, great feedback. Also thanks Dawid for the
>> explanation, it is much clearer now.
>> 
>> One thing that is indeed missing from the FLIP is how the boundedness is
>> passed to the Source implementation. So the API should be
>> Source#createEnumerator(Boundedness boundedness, SplitEnumeratorContext
>> context)
>> And we can probably remove the Source#supportBoundedness(Boundedness
>> boundedness) method.
>> 
>> Assuming we have that, we are essentially choosing from one of the
>> following two options:
>> 
>> Option 1:
>> // The source is continuous source, and only unbounded operations can be
>> performed.
>> DataStream<Type> datastream = env.continuousSource(someSource);
>> 
>> // The source is bounded source, both bounded and unbounded operations can
>> be performed.
>> BoundedDataStream<Type> boundedDataStream = env.boundedSource(someSource);
>> 
>>  - Pros:
>>       a) explicit boundary between bounded / unbounded streams, it is
>> quite simple and clear to the users.
>>  - Cons:
>>       a) For applications that do not involve bounded operations, they
>> still have to call different API to distinguish bounded / unbounded streams.
>>       b) No support for bounded stream to run in a streaming runtime
>> setting, i.e. scheduling and operators behaviors.
>> 
>> 
>> Option 2:
>> // The source is either bounded or unbounded, but only unbounded operations
>> could be performed on the returned DataStream.
>> DataStream<Type> dataStream = env.source(someSource);
>> 
>> // The source must be a bounded source, otherwise exception is thrown.
>> BoundedDataStream<Type> boundedDataStream =
>> env.boundedSource(boundedSource);
>> 
>> The pros and cons are exactly the opposite of option 1.
>>  - Pros:
>>       a) For applications that do not involve bounded operations, they
>> still have to call different API to distinguish bounded / unbounded streams.
>>       b) Support for bounded stream to run in a streaming runtime setting,
>> i.e. scheduling and operators behaviors.
>>  - Cons:
>>       a) Bounded / unbounded streams are kind of mixed, i.e. given a
>> DataStream, it is not clear whether it is bounded or not, unless you have
>> the access to its source.
>> 
>> 
>> If we only think from the Source API perspective, option 2 seems a better
>> choice because functionality wise it is a superset of option 1, at the cost
>> of some seemingly acceptable ambiguity in the DataStream API.
>> But if we look at the DataStream API as a whole, option 1 seems a clearer
>> choice. For example, some times a library may have to know whether a
>> certain task will finish or not. And it would be difficult to tell if the
>> input is a DataStream, unless additional information is provided all the
>> way from the Source. One possible solution is to have a *modified option 2*
>> which adds a method to the DataStream API to indicate boundedness, such as
>> getBoundedness(). It would solve the problem with a potential confusion of
>> what is difference between a DataStream with getBoundedness()=true and a
>> BoundedDataStream. But that seems not super difficult to explain.
>> 
>> So from API's perspective, I don't have a strong opinion between *option 1*
>> and *modified option 2. *I like the cleanness of option 1, but modified
>> option 2 would be more attractive if we have concrete use case for the
>> "Bounded stream with unbounded streaming runtime settings".
>> 
>> Re: Till
>> 
>> 
>> Maybe this has already been asked before but I was wondering why the
>> SourceReader interface has the method pollNext which hands the
>> responsibility of outputting elements to the SourceReader implementation?
>> Has this been done for backwards compatibility reasons with the old source
>> interface? If not, then one could define a Collection<E> getNextRecords()
>> method which returns the currently retrieved records and then the caller
>> emits them outside of the SourceReader. That way the interface would not
>> allow to implement an outputting loop where we never hand back control to
>> the caller. At the moment, this contract can be easily broken and is only
>> mentioned loosely in the JavaDocs.
>> 
>> 
>> The primary reason we handover the SourceOutput to the SourceReader is
>> because sometimes it is difficult for a SourceReader to emit one record at
>> a time. One example is some batched messaging systems which only have an
>> offset for the entire batch instead of individual messages in the batch. In
>> that case, returning one record at a time would leave the SourceReader in
>> an uncheckpointable state because they can only checkpoint at the batch
>> boundaries.
>> 
>> Thanks,
>> 
>> Jiangjie (Becket) Qin
>> 
>> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <trohrmann@apache.org <ma...@apache.org>> <trohrmann@apache.org <ma...@apache.org>> wrote:
>> 
>> 
>> Hi everyone,
>> 
>> thanks for drafting this FLIP. It reads very well.
>> 
>> Concerning Dawid's proposal, I tend to agree. The boundedness could come
>> from the source and tell the system how to treat the operator (scheduling
>> wise). From a user's perspective it should be fine to get back a DataStream
>> when calling env.source(boundedSource) if he does not need special
>> operations defined on a BoundedDataStream. If he needs this, then one could
>> use the method BoundedDataStream env.boundedSource(boundedSource).
>> 
>> If possible, we could enforce the proper usage of env.boundedSource() by
>> introducing a BoundedSource type so that one cannot pass an
>> unbounded source to it. That way users would not be able to shoot
>> themselves in the foot.
>> 
>> Maybe this has already been asked before but I was wondering why the
>> SourceReader interface has the method pollNext which hands the
>> responsibility of outputting elements to the SourceReader implementation?
>> Has this been done for backwards compatibility reasons with the old source
>> interface? If not, then one could define a Collection<E> getNextRecords()
>> method which returns the currently retrieved records and then the caller
>> emits them outside of the SourceReader. That way the interface would not
>> allow to implement an outputting loop where we never hand back control to
>> the caller. At the moment, this contract can be easily broken and is only
>> mentioned loosely in the JavaDocs.
>> 
>> Cheers,
>> Till
>> 
>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <jingsonglee0@gmail.com <ma...@gmail.com>> <jingsonglee0@gmail.com <ma...@gmail.com>>
>> wrote:
>> 
>> 
>> Hi all,
>> 
>> I think current design is good.
>> 
>> My understanding is:
>> 
>> For execution mode: bounded mode and continuous mode, It's totally
>> different. I don't think we have the ability to integrate the two models
>> 
>> at
>> 
>> present. It's about scheduling, memory, algorithms, States, etc. we
>> shouldn't confuse them.
>> 
>> For source capabilities: only bounded, only continuous, both bounded and
>> continuous.
>> I think Kafka is a source that can be ran both bounded
>> and continuous execution mode.
>> And Kafka with end offset should be ran both bounded
>> and continuous execution mode.  Using apache Beam with Flink runner, I
>> 
>> used
>> 
>> to run a "bounded" Kafka in streaming mode. For our previous DataStream,
>> 
>> it
>> 
>> is not necessarily required that the source cannot be bounded.
>> 
>> So it is my thought for Dawid's question:
>> 1.pass a bounded source to continuousSource() +1
>> 2.pass a continuous source to boundedSource() -1, should throw exception.
>> 
>> In StreamExecutionEnvironment, continuousSource and boundedSource define
>> the execution mode. It defines a clear boundary of execution mode.
>> 
>> Best,
>> Jingsong Lee
>> 
>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <imjark@gmail.com <ma...@gmail.com>> <imjark@gmail.com <ma...@gmail.com>> wrote:
>> 
>> 
>> I agree with Dawid's point that the boundedness information should come
>> from the source itself (e.g. the end timestamp), not through
>> env.boundedSouce()/continuousSource().
>> I think if we want to support something like `env.source()` that derive
>> 
>> the
>> 
>> execution mode from source, `supportsBoundedness(Boundedness)`
>> method is not enough, because we don't know whether it is bounded or
>> 
>> not.
>> 
>> Best,
>> Jark
>> 
>> 
>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <dwysakowicz@apache.org <ma...@apache.org>> <dwysakowicz@apache.org <ma...@apache.org>>
>> wrote:
>> 
>> 
>> One more thing. In the current proposal, with the
>> supportsBoundedness(Boundedness) method and the boundedness coming
>> 
>> from
>> 
>> either continuousSource or boundedSource I could not find how this
>> information is fed back to the SplitEnumerator.
>> 
>> Best,
>> 
>> Dawid
>> 
>> On 09/12/2019 13:52, Becket Qin wrote:
>> 
>> Hi Dawid,
>> 
>> Thanks for the comments. This actually brings another relevant
>> 
>> question
>> 
>> about what does a "bounded source" imply. I actually had the same
>> impression when I look at the Source API. Here is what I understand
>> 
>> after
>> 
>> some discussion with Stephan. The bounded source has the following
>> 
>> impacts.
>> 
>> 1. API validity.
>> - A bounded source generates a bounded stream so some operations
>> 
>> that
>> 
>> only
>> 
>> works for bounded records would be performed, e.g. sort.
>> - To expose these bounded stream only APIs, there are two options:
>>     a. Add them to the DataStream API and throw exception if a
>> 
>> method
>> 
>> is
>> 
>> called on an unbounded stream.
>>     b. Create a BoundedDataStream class which is returned from
>> env.boundedSource(), while DataStream is returned from
>> 
>> env.continousSource().
>> 
>> Note that this cannot be done by having single
>> 
>> env.source(theSource)
>> 
>> even
>> 
>> the Source has a getBoundedness() method.
>> 
>> 2. Scheduling
>> - A bounded source could be computed stage by stage without
>> 
>> bringing
>> 
>> up
>> 
>> all
>> 
>> the tasks at the same time.
>> 
>> 3. Operator behaviors
>> - A bounded source indicates the records are finite so some
>> 
>> operators
>> 
>> can
>> 
>> wait until it receives all the records before it starts the
>> 
>> processing.
>> 
>> In the above impact, only 1 is relevant to the API design. And the
>> 
>> current
>> 
>> proposal in FLIP-27 is following 1.b.
>> 
>> // boundedness depends of source property, imo this should always
>> 
>> be
>> 
>> preferred
>> 
>> 
>> DataStream<MyType> stream = env.source(theSource);
>> 
>> 
>> In your proposal, does DataStream have bounded stream only methods?
>> 
>> It
>> 
>> looks it should have, otherwise passing a bounded Source to
>> 
>> env.source()
>> 
>> would be confusing. In that case, we will essentially do 1.a if an
>> unbounded Source is created from env.source(unboundedSource).
>> 
>> If we have the methods only supported for bounded streams in
>> 
>> DataStream,
>> 
>> it
>> 
>> seems a little weird to have a separate BoundedDataStream
>> 
>> interface.
>> 
>> Am I understand it correctly?
>> 
>> Thanks,
>> 
>> Jiangjie (Becket) Qin
>> 
>> 
>> 
>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
>> 
>> dwysakowicz@apache.org <ma...@apache.org>>
>> 
>> wrote:
>> 
>> 
>> Hi all,
>> 
>> Really well written proposal and very important one. I must admit
>> 
>> I
>> 
>> have
>> 
>> not understood all the intricacies of it yet.
>> 
>> One question I have though is about where does the information
>> 
>> about
>> 
>> boundedness come from. I think in most cases it is a property of
>> 
>> the
>> 
>> source. As you described it might be e.g. end offset, a flag
>> 
>> should
>> 
>> it
>> 
>> monitor new splits etc. I think it would be a really nice use case
>> 
>> to
>> 
>> be
>> 
>> able to say:
>> 
>> new KafkaSource().readUntil(long timestamp),
>> 
>> which could work as an "end offset". Moreover I think all Bounded
>> 
>> sources
>> 
>> support continuous mode, but no intrinsically continuous source
>> 
>> support
>> 
>> the
>> 
>> Bounded mode. If I understood the proposal correctly it suggest
>> 
>> the
>> 
>> boundedness sort of "comes" from the outside of the source, from
>> 
>> the
>> 
>> invokation of either boundedStream or continousSource.
>> 
>> I am wondering if it would make sense to actually change the
>> 
>> method
>> 
>> boolean Source#supportsBoundedness(Boundedness)
>> 
>> to
>> 
>> Boundedness Source#getBoundedness().
>> 
>> As for the methods #boundedSource, #continousSource, assuming the
>> boundedness is property of the source they do not affect how the
>> 
>> enumerator
>> 
>> works, but mostly how the dag is scheduled, right? I am not
>> 
>> against
>> 
>> those
>> 
>> methods, but I think it is a very specific use case to actually
>> 
>> override
>> 
>> the property of the source. In general I would expect users to
>> 
>> only
>> 
>> call
>> 
>> env.source(theSource), where the source tells if it is bounded or
>> 
>> not. I
>> 
>> would suggest considering following set of methods:
>> 
>> // boundedness depends of source property, imo this should always
>> 
>> be
>> 
>> preferred
>> 
>> DataStream<MyType> stream = env.source(theSource);
>> 
>> 
>> // always continous execution, whether bounded or unbounded source
>> 
>> DataStream<MyType> boundedStream = env.continousSource(theSource);
>> 
>> // imo this would make sense if the BoundedDataStream provides
>> 
>> additional features unavailable for continous mode
>> 
>> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
>> 
>> 
>> Best,
>> 
>> Dawid
>> 
>> 
>> On 04/12/2019 11:25, Stephan Ewen wrote:
>> 
>> Thanks, Becket, for updating this.
>> 
>> I agree with moving the aspects you mentioned into separate FLIPs
>> 
>> -
>> 
>> this
>> 
>> one way becoming unwieldy in size.
>> 
>> +1 to the FLIP in its current state. Its a very detailed write-up,
>> 
>> nicely
>> 
>> done!
>> 
>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <becket.qin@gmail.com <ma...@gmail.com>> <becket.qin@gmail.com <ma...@gmail.com>>
>> 
>> <
>> 
>> becket.qin@gmail.com <ma...@gmail.com>> wrote:
>> 
>> Hi all,
>> 
>> Sorry for the long belated update. I have updated FLIP-27 wiki
>> 
>> page
>> 
>> with
>> 
>> the latest proposals. Some noticeable changes include:
>> 1. A new generic communication mechanism between SplitEnumerator
>> 
>> and
>> 
>> SourceReader.
>> 2. Some detail API method signature changes.
>> 
>> We left a few things out of this FLIP and will address them in
>> 
>> separate
>> 
>> FLIPs. Including:
>> 1. Per split event time.
>> 2. Event time alignment.
>> 3. Fine grained failover for SplitEnumerator failure.
>> 
>> Please let us know if you have any question.
>> 
>> Thanks,
>> 
>> Jiangjie (Becket) Qin
>> 
>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <sewen@apache.org <ma...@apache.org>> <sewen@apache.org <ma...@apache.org>> <
>> 
>> sewen@apache.org <ma...@apache.org>> wrote:
>> 
>> Hi  Łukasz!
>> 
>> Becket and me are working hard on figuring out the last details
>> 
>> and
>> 
>> implementing the first PoC. We would update the FLIP hopefully
>> 
>> next
>> 
>> week.
>> 
>> There is a fair chance that a first version of this will be in
>> 
>> 1.10,
>> 
>> but
>> 
>> I
>> 
>> think it will take another release to battle test it and migrate
>> 
>> the
>> 
>> connectors.
>> 
>> Best,
>> Stephan
>> 
>> 
>> 
>> 
>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <ljd@touk.pl <ma...@touk.pl>
>> 
>> <
>> 
>> ljd@touk.pl <ma...@touk.pl>>
>> 
>> wrote:
>> 
>> Hi,
>> 
>> This proposal looks very promising for us. Do you have any plans
>> 
>> in
>> 
>> which
>> 
>> Flink release it is going to be released? We are thinking on
>> 
>> using a
>> 
>> Data
>> 
>> Set API for our future use cases but on the other hand Data Set
>> 
>> API
>> 
>> is
>> 
>> going to be deprecated so using proposed bounded data streams
>> 
>> solution
>> 
>> could be more viable in the long term.
>> 
>> Thanks,
>> Łukasz
>> 
>> On 2019/10/01 15:48:03, Thomas Weise <thomas.weise@gmail.com <ma...@gmail.com>> <thomas.weise@gmail.com <ma...@gmail.com>> <
>> 
>> thomas.weise@gmail.com <ma...@gmail.com>> wrote:
>> 
>> Thanks for putting together this proposal!
>> 
>> I see that the "Per Split Event Time" and "Event Time Alignment"
>> 
>> sections
>> 
>> are still TBD.
>> 
>> It would probably be good to flesh those out a bit before
>> 
>> proceeding
>> 
>> too
>> 
>> far
>> 
>> as the event time alignment will probably influence the
>> 
>> interaction
>> 
>> with
>> 
>> the split reader, specifically ReaderStatus
>> 
>> emitNext(SourceOutput<E>
>> 
>> output).
>> 
>> We currently have only one implementation for event time alignment
>> 
>> in
>> 
>> the
>> 
>> Kinesis consumer. The synchronization in that case takes place as
>> 
>> the
>> 
>> last
>> 
>> step before records are emitted downstream (RecordEmitter). With
>> 
>> the
>> 
>> currently proposed interfaces, the equivalent can be implemented
>> 
>> in
>> 
>> the
>> 
>> reader loop, although note that in the Kinesis consumer the per
>> 
>> shard
>> 
>> threads push records.
>> 
>> Synchronization has not been implemented for the Kafka consumer
>> 
>> yet.
>> 
>> https://issues.apache.org/jira/browse/FLINK-12675 <https://issues.apache.org/jira/browse/FLINK-12675>
>> 
>> When I looked at it, I realized that the implementation will look
>> 
>> quite
>> 
>> different
>> from Kinesis because it needs to take place in the pull part,
>> 
>> where
>> 
>> records
>> 
>> are taken from the Kafka client. Due to the multiplexing it cannot
>> 
>> be
>> 
>> done
>> 
>> by blocking the split thread like it currently works for Kinesis.
>> 
>> Reading
>> 
>> from individual Kafka partitions needs to be controlled via
>> 
>> pause/resume
>> 
>> on the Kafka client.
>> 
>> To take on that responsibility the split thread would need to be
>> 
>> aware
>> 
>> of
>> 
>> the
>> watermarks or at least whether it should or should not continue to
>> 
>> consume
>> 
>> a given split and this may require a different SourceReader or
>> 
>> SourceOutput
>> 
>> interface.
>> 
>> Thanks,
>> Thomas
>> 
>> 
>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mmyy1110@gmail.com <ma...@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
>> 
>> mmyy1110@gmail.com <ma...@gmail.com>> wrote:
>> 
>> Hi Stephan,
>> 
>> Thank you for feedback!
>> Will take a look at your branch before public discussing.
>> 
>> 
>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <sewen@apache.org <ma...@apache.org>> <sewen@apache.org <ma...@apache.org>>
>> 
>> <
>> 
>> sewen@apache.org <ma...@apache.org>>
>> 
>> wrote:
>> 
>> Hi Biao!
>> 
>> Thanks for reviving this. I would like to join this discussion,
>> 
>> but
>> 
>> am
>> 
>> quite occupied with the 1.9 release, so can we maybe pause this
>> 
>> discussion
>> 
>> for a week or so?
>> 
>> In the meantime I can share some suggestion based on prior
>> 
>> experiments:
>> 
>> How to do watermarks / timestamp extractors in a simpler and more
>> 
>> flexible
>> 
>> way. I think that part is quite promising should be part of the
>> 
>> new
>> 
>> source
>> 
>> interface.
>> 
>> 
>> 
>> 
>> 
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime <https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime>
>> 
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java <https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java>
>> 
>> Some experiments on how to build the source reader and its
>> 
>> library
>> 
>> for
>> 
>> common threading/split patterns:
>> 
>> 
>> 
>> 
>> 
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src <https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src>
>> 
>> Best,
>> Stephan
>> 
>> 
>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mmyy1110@gmail.com <ma...@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
>> 
>> mmyy1110@gmail.com <ma...@gmail.com>>
>> 
>> wrote:
>> 
>> Hi devs,
>> 
>> Since 1.9 is nearly released, I think we could get back to
>> 
>> FLIP-27.
>> 
>> I
>> 
>> believe it should be included in 1.10.
>> 
>> There are so many things mentioned in document of FLIP-27. [1] I
>> 
>> think
>> 
>> we'd better discuss them separately. However the wiki is not a
>> 
>> good
>> 
>> place
>> 
>> to discuss. I wrote google doc about SplitReader API which
>> 
>> misses
>> 
>> some
>> 
>> details in the document. [2]
>> 
>> 1.
>> 
>> 
>> 
>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface <https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface>
>> 
>> 2.
>> 
>> 
>> 
>> 
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing <https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing>
>> 
>> CC Stephan, Aljoscha, Piotrek, Becket
>> 
>> 
>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mmyy1110@gmail.com <ma...@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
>> 
>> mmyy1110@gmail.com <ma...@gmail.com>>
>> 
>> wrote:
>> 
>> Hi Steven,
>> Thank you for the feedback. Please take a look at the document
>> 
>> FLIP-27
>> 
>> <
>> 
>> 
>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface <https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface>
>> 
>> which
>> 
>> is updated recently. A lot of details of enumerator were added
>> 
>> in
>> 
>> this
>> 
>> document. I think it would help.
>> 
>> Steven Wu <stevenz3wu@gmail.com <ma...@gmail.com>> <stevenz3wu@gmail.com <ma...@gmail.com>> <stevenz3wu@gmail.com <ma...@gmail.com>> <stevenz3wu@gmail.com <ma...@gmail.com>>
>> 
>> 于2019年3月28日周四
>> 
>> 下午12:52写道:
>> 
>> This proposal mentioned that SplitEnumerator might run on the
>> JobManager or
>> in a single task on a TaskManager.
>> 
>> if enumerator is a single task on a taskmanager, then the job
>> 
>> DAG
>> 
>> can
>> 
>> never
>> been embarrassingly parallel anymore. That will nullify the
>> 
>> leverage
>> 
>> of
>> 
>> fine-grained recovery for embarrassingly parallel jobs.
>> 
>> It's not clear to me what's the implication of running
>> 
>> enumerator
>> 
>> on
>> 
>> the
>> 
>> jobmanager. So I will leave that out for now.
>> 
>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mmyy1110@gmail.com <ma...@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
>> 
>> mmyy1110@gmail.com <ma...@gmail.com>>
>> 
>> wrote:
>> 
>> Hi Stephan & Piotrek,
>> 
>> Thank you for feedback.
>> 
>> It seems that there are a lot of things to do in community.
>> 
>> I
>> 
>> am
>> 
>> just
>> 
>> afraid that this discussion may be forgotten since there so
>> 
>> many
>> 
>> proposals
>> 
>> recently.
>> Anyway, wish to see the split topics soon :)
>> 
>> Piotr Nowojski <piotr@da-platform.com <ma...@da-platform.com>> <piotr@da-platform.com <ma...@da-platform.com>> <piotr@da-platform.com <ma...@da-platform.com>> <piotr@da-platform.com <ma...@da-platform.com>>
>> 
>> 于2019年1月24日周四
>> 
>> 下午8:21写道:
>> 
>> Hi Biao!
>> 
>> This discussion was stalled because of preparations for
>> 
>> the
>> 
>> open
>> 
>> sourcing
>> 
>> & merging Blink. I think before creating the tickets we
>> 
>> should
>> 
>> split this
>> 
>> discussion into topics/areas outlined by Stephan and
>> 
>> create
>> 
>> Flips
>> 
>> for
>> 
>> that.
>> 
>> I think there is no chance for this to be completed in
>> 
>> couple
>> 
>> of
>> 
>> remaining
>> 
>> weeks/1 month before 1.8 feature freeze, however it would
>> 
>> be
>> 
>> good
>> 
>> to aim
>> 
>> with those changes for 1.9.
>> 
>> Piotrek
>> 
>> 
>> On 20 Jan 2019, at 16:08, Biao Liu <mmyy1110@gmail.com <ma...@gmail.com>> <mmyy1110@gmail.com <ma...@gmail.com>> <
>> 
>> mmyy1110@gmail.com <ma...@gmail.com>>
>> 
>> wrote:
>> 
>> Hi community,
>> The summary of Stephan makes a lot sense to me. It is
>> 
>> much
>> 
>> clearer
>> 
>> indeed
>> 
>> after splitting the complex topic into small ones.
>> I was wondering is there any detail plan for next step?
>> 
>> If
>> 
>> not,
>> 
>> I
>> 
>> would
>> 
>> like to push this thing forward by creating some JIRA
>> 
>> issues.
>> 
>> Another question is that should version 1.8 include
>> 
>> these
>> 
>> features?
>> 
>> Stephan Ewen <sewen@apache.org <ma...@apache.org>> <sewen@apache.org <ma...@apache.org>> <sewen@apache.org <ma...@apache.org>> <sewen@apache.org <ma...@apache.org>> 于2018年12月1日周六
>> 
>> 上午4:20写道:
>> 
>> Thanks everyone for the lively discussion. Let me try
>> 
>> to
>> 
>> summarize
>> 
>> where I
>> 
>> see convergence in the discussion and open issues.
>> I'll try to group this by design aspect of the source.
>> 
>> Please
>> 
>> let me
>> 
>> know
>> 
>> if I got things wrong or missed something crucial here.
>> 
>> For issues 1-3, if the below reflects the state of the
>> 
>> discussion, I
>> 
>> would
>> 
>> try and update the FLIP in the next days.
>> For the remaining ones we need more discussion.
>> 
>> I would suggest to fork each of these aspects into a
>> 
>> separate
>> 
>> mail
>> 
>> thread,
>> 
>> or will loose sight of the individual aspects.
>> 
>> *(1) Separation of Split Enumerator and Split Reader*
>> 
>> - All seem to agree this is a good thing
>> - Split Enumerator could in the end live on JobManager
>> 
>> (and
>> 
>> assign
>> 
>> splits
>> 
>> via RPC) or in a task (and assign splits via data
>> 
>> streams)
>> 
>> - this discussion is orthogonal and should come later,
>> 
>> when
>> 
>> the
>> 
>> interface
>> 
>> is agreed upon.
>> 
>> *(2) Split Readers for one or more splits*
>> 
>> - Discussion seems to agree that we need to support
>> 
>> one
>> 
>> reader
>> 
>> that
>> 
>> possibly handles multiple splits concurrently.
>> - The requirement comes from sources where one
>> 
>> poll()-style
>> 
>> call
>> 
>> fetches
>> 
>> data from different splits / partitions
>>   --> example sources that require that would be for
>> 
>> example
>> 
>> Kafka,
>> 
>> Pravega, Pulsar
>> 
>> - Could have one split reader per source, or multiple
>> 
>> split
>> 
>> readers
>> 
>> that
>> 
>> share the "poll()" function
>> - To not make it too complicated, we can start with
>> 
>> thinking
>> 
>> about
>> 
>> one
>> 
>> split reader for all splits initially and see if that
>> 
>> covers
>> 
>> all
>> 
>> requirements
>> 
>> *(3) Threading model of the Split Reader*
>> 
>> - Most active part of the discussion ;-)
>> 
>> - A non-blocking way for Flink's task code to interact
>> 
>> with
>> 
>> the
>> 
>> source
>> 
>> is
>> 
>> needed in order to a task runtime code based on a
>> single-threaded/actor-style task design
>>   --> I personally am a big proponent of that, it will
>> 
>> help
>> 
>> with
>> 
>> well-behaved checkpoints, efficiency, and simpler yet
>> 
>> more
>> 
>> robust
>> 
>> runtime
>> 
>> code
>> 
>> - Users care about simple abstraction, so as a
>> 
>> subclass
>> 
>> of
>> 
>> SplitReader
>> 
>> (non-blocking / async) we need to have a
>> 
>> BlockingSplitReader
>> 
>> which
>> 
>> will
>> 
>> form the basis of most source implementations.
>> 
>> BlockingSplitReader
>> 
>> lets
>> 
>> users do blocking simple poll() calls.
>> - The BlockingSplitReader would spawn a thread (or
>> 
>> more)
>> 
>> and
>> 
>> the
>> 
>> thread(s) can make blocking calls and hand over data
>> 
>> buffers
>> 
>> via
>> 
>> a
>> 
>> blocking
>> 
>> queue
>> - This should allow us to cover both, a fully async
>> 
>> runtime,
>> 
>> and a
>> 
>> simple
>> 
>> blocking interface for users.
>> - This is actually very similar to how the Kafka
>> 
>> connectors
>> 
>> work.
>> 
>> Kafka
>> 
>> 9+ with one thread, Kafka 8 with multiple threads
>> 
>> - On the base SplitReader (the async one), the
>> 
>> non-blocking
>> 
>> method
>> 
>> that
>> 
>> gets the next chunk of data would signal data
>> 
>> availability
>> 
>> via
>> 
>> a
>> 
>> CompletableFuture, because that gives the best
>> 
>> flexibility
>> 
>> (can
>> 
>> await
>> 
>> completion or register notification handlers).
>> - The source task would register a "thenHandle()" (or
>> 
>> similar)
>> 
>> on the
>> 
>> future to put a "take next data" task into the
>> 
>> actor-style
>> 
>> mailbox
>> 
>> *(4) Split Enumeration and Assignment*
>> 
>> - Splits may be generated lazily, both in cases where
>> 
>> there
>> 
>> is a
>> 
>> limited
>> 
>> number of splits (but very many), or splits are
>> 
>> discovered
>> 
>> over
>> 
>> time
>> 
>> - Assignment should also be lazy, to get better load
>> 
>> balancing
>> 
>> - Assignment needs support locality preferences
>> 
>> - Possible design based on discussion so far:
>> 
>>   --> SplitReader has a method "addSplits(SplitT...)"
>> 
>> to
>> 
>> add
>> 
>> one or
>> 
>> more
>> 
>> splits. Some split readers might assume they have only
>> 
>> one
>> 
>> split
>> 
>> ever,
>> 
>> concurrently, others assume multiple splits. (Note:
>> 
>> idea
>> 
>> behind
>> 
>> being
>> 
>> able
>> 
>> to add multiple splits at the same time is to ease
>> 
>> startup
>> 
>> where
>> 
>> multiple
>> 
>> splits may be assigned instantly.)
>>   --> SplitReader has a context object on which it can
>> 
>> call
>> 
>> indicate
>> 
>> when
>> 
>> splits are completed. The enumerator gets that
>> 
>> notification and
>> 
>> can
>> 
>> use
>> 
>> to
>> 
>> decide when to assign new splits. This should help both
>> 
>> in
>> 
>> cases
>> 
>> of
>> 
>> sources
>> 
>> that take splits lazily (file readers) and in case the
>> 
>> source
>> 
>> needs to
>> 
>> preserve a partial order between splits (Kinesis,
>> 
>> Pravega,
>> 
>> Pulsar may
>> 
>> need
>> 
>> that).
>>   --> SplitEnumerator gets notification when
>> 
>> SplitReaders
>> 
>> start
>> 
>> and
>> 
>> when
>> 
>> they finish splits. They can decide at that moment to
>> 
>> push
>> 
>> more
>> 
>> splits
>> 
>> to
>> 
>> that reader
>>   --> The SplitEnumerator should probably be aware of
>> 
>> the
>> 
>> source
>> 
>> parallelism, to build its initial distribution.
>> 
>> - Open question: Should the source expose something
>> 
>> like
>> 
>> "host
>> 
>> preferences", so that yarn/mesos/k8s can take this into
>> 
>> account
>> 
>> when
>> 
>> selecting a node to start a TM on?
>> 
>> *(5) Watermarks and event time alignment*
>> 
>> - Watermark generation, as well as idleness, needs to
>> 
>> be
>> 
>> per
>> 
>> split
>> 
>> (like
>> 
>> currently in the Kafka Source, per partition)
>> - It is desirable to support optional
>> 
>> event-time-alignment,
>> 
>> meaning
>> 
>> that
>> 
>> splits that are ahead are back-pressured or temporarily
>> 
>> unsubscribed
>> 
>> - I think i would be desirable to encapsulate
>> 
>> watermark
>> 
>> generation
>> 
>> logic
>> 
>> in watermark generators, for a separation of concerns.
>> 
>> The
>> 
>> watermark
>> 
>> generators should run per split.
>> - Using watermark generators would also help with
>> 
>> another
>> 
>> problem of
>> 
>> the
>> 
>> suggested interface, namely supporting non-periodic
>> 
>> watermarks
>> 
>> efficiently.
>> 
>> - Need a way to "dispatch" next record to different
>> 
>> watermark
>> 
>> generators
>> 
>> - Need a way to tell SplitReader to "suspend" a split
>> 
>> until a
>> 
>> certain
>> 
>> watermark is reached (event time backpressure)
>> - This would in fact be not needed (and thus simpler)
>> 
>> if
>> 
>> we
>> 
>> had
>> 
>> a
>> 
>> SplitReader per split and may be a reason to re-open
>> 
>> that
>> 
>> discussion
>> 
>> *(6) Watermarks across splits and in the Split
>> 
>> Enumerator*
>> 
>> - The split enumerator may need some watermark
>> 
>> awareness,
>> 
>> which
>> 
>> should
>> 
>> be
>> 
>> purely based on split metadata (like create timestamp
>> 
>> of
>> 
>> file
>> 
>> splits)
>> 
>> - If there are still more splits with overlapping
>> 
>> event
>> 
>> time
>> 
>> range
>> 
>> for
>> 
>> a
>> 
>> split reader, then that split reader should not advance
>> 
>> the
>> 
>> watermark
>> 
>> within the split beyond the overlap boundary. Otherwise
>> 
>> future
>> 
>> splits
>> 
>> will
>> 
>> produce late data.
>> 
>> - One way to approach this could be that the split
>> 
>> enumerator
>> 
>> may
>> 
>> send
>> 
>> watermarks to the readers, and the readers cannot emit
>> 
>> watermarks
>> 
>> beyond
>> 
>> that received watermark.
>> - Many split enumerators would simply immediately send
>> 
>> Long.MAX
>> 
>> out
>> 
>> and
>> 
>> leave the progress purely to the split readers.
>> 
>> - For event-time alignment / split back pressure, this
>> 
>> begs
>> 
>> the
>> 
>> question
>> 
>> how we can avoid deadlocks that may arise when splits
>> 
>> are
>> 
>> suspended
>> 
>> for
>> 
>> event time back pressure,
>> 
>> *(7) Batch and streaming Unification*
>> 
>> - Functionality wise, the above design should support
>> 
>> both
>> 
>> - Batch often (mostly) does not care about reading "in
>> 
>> order"
>> 
>> and
>> 
>> generating watermarks
>>   --> Might use different enumerator logic that is
>> 
>> more
>> 
>> locality
>> 
>> aware
>> 
>> and ignores event time order
>>   --> Does not generate watermarks
>> - Would be great if bounded sources could be
>> 
>> identified
>> 
>> at
>> 
>> compile
>> 
>> time,
>> 
>> so that "env.addBoundedSource(...)" is type safe and
>> 
>> can
>> 
>> return a
>> 
>> "BoundedDataStream".
>> - Possible to defer this discussion until later
>> 
>> *Miscellaneous Comments*
>> 
>> - Should the source have a TypeInformation for the
>> 
>> produced
>> 
>> type,
>> 
>> instead
>> 
>> of a serializer? We need a type information in the
>> 
>> stream
>> 
>> anyways, and
>> 
>> can
>> 
>> derive the serializer from that. Plus, creating the
>> 
>> serializer
>> 
>> should
>> 
>> respect the ExecutionConfig.
>> 
>> - The TypeSerializer interface is very powerful but
>> 
>> also
>> 
>> not
>> 
>> easy to
>> 
>> implement. Its purpose is to handle data super
>> 
>> efficiently,
>> 
>> support
>> 
>> flexible ways of evolution, etc.
>> For metadata I would suggest to look at the
>> 
>> SimpleVersionedSerializer
>> 
>> instead, which is used for example for checkpoint
>> 
>> master
>> 
>> hooks,
>> 
>> or for
>> 
>> the
>> 
>> streaming file sink. I think that is is a good match
>> 
>> for
>> 
>> cases
>> 
>> where
>> 
>> we
>> 
>> do
>> 
>> not need more than ser/deser (no copy, etc.) and don't
>> 
>> need to
>> 
>> push
>> 
>> versioning out of the serialization paths for best
>> 
>> performance
>> 
>> (as in
>> 
>> the
>> 
>> TypeSerializer)
>> 
>> 
>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>> 
>> k.kloudas@data-artisans.com>
>> 
>> wrote:
>> 
>> 
>> Hi Biao,
>> 
>> Thanks for the answer!
>> 
>> So given the multi-threaded readers, now we have as
>> 
>> open
>> 
>> questions:
>> 
>> 1) How do we let the checkpoints pass through our
>> 
>> multi-threaded
>> 
>> reader
>> 
>> operator?
>> 
>> 2) Do we have separate reader and source operators or
>> 
>> not? In
>> 
>> the
>> 
>> strategy
>> 
>> that has a separate source, the source operator has a
>> 
>> parallelism of
>> 
>> 1
>> 
>> and
>> 
>> is responsible for split recovery only.
>> 
>> For the first one, given also the constraints
>> 
>> (blocking,
>> 
>> finite
>> 
>> queues,
>> 
>> etc), I do not have an answer yet.
>> 
>> For the 2nd, I think that we should go with separate
>> 
>> operators
>> 
>> for
>> 
>> the
>> 
>> source and the readers, for the following reasons:
>> 
>> 1) This is more aligned with a potential future
>> 
>> improvement
>> 
>> where the
>> 
>> split
>> 
>> discovery becomes a responsibility of the JobManager
>> 
>> and
>> 
>> readers are
>> 
>> pooling more work from the JM.
>> 
>> 2) The source is going to be the "single point of
>> 
>> truth".
>> 
>> It
>> 
>> will
>> 
>> know
>> 
>> what
>> 
>> has been processed and what not. If the source and the
>> 
>> readers
>> 
>> are a
>> 
>> single
>> 
>> operator with parallelism > 1, or in general, if the
>> 
>> split
>> 
>> discovery
>> 
>> is
>> 
>> done by each task individually, then:
>>  i) we have to have a deterministic scheme for each
>> 
>> reader to
>> 
>> assign
>> 
>> splits to itself (e.g. mod subtaskId). This is not
>> 
>> necessarily
>> 
>> trivial
>> 
>> for
>> 
>> all sources.
>>  ii) each reader would have to keep a copy of all its
>> 
>> processed
>> 
>> slpits
>> 
>>  iii) the state has to be a union state with a
>> 
>> non-trivial
>> 
>> merging
>> 
>> logic
>> 
>> in order to support rescaling.
>> 
>> Two additional points that you raised above:
>> 
>> i) The point that you raised that we need to keep all
>> 
>> splits
>> 
>> (processed
>> 
>> and
>> 
>> not-processed) I think is a bit of a strong
>> 
>> requirement.
>> 
>> This
>> 
>> would
>> 
>> imply
>> 
>> that for infinite sources the state will grow
>> 
>> indefinitely.
>> 
>> This is
>> 
>> problem
>> 
>> is even more pronounced if we do not have a single
>> 
>> source
>> 
>> that
>> 
>> assigns
>> 
>> splits to readers, as each reader will have its own
>> 
>> copy
>> 
>> of
>> 
>> the
>> 
>> state.
>> 
>> ii) it is true that for finite sources we need to
>> 
>> somehow
>> 
>> not
>> 
>> close
>> 
>> the
>> 
>> readers when the source/split discoverer finishes. The
>> ContinuousFileReaderOperator has a work-around for
>> 
>> that.
>> 
>> It is
>> 
>> not
>> 
>> elegant,
>> 
>> and checkpoints are not emitted after closing the
>> 
>> source,
>> 
>> but
>> 
>> this, I
>> 
>> believe, is a bigger problem which requires more
>> 
>> changes
>> 
>> than
>> 
>> just
>> 
>> refactoring the source interface.
>> 
>> Cheers,
>> Kostas
>> 
>> 
>> 
>> 
>> --
>> Best, Jingsong Lee


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Till Rohrmann <tr...@apache.org>.
Hi Becket,

quick clarification from my side because I think you misunderstood my
question. I did not suggest to let the SourceReader return only a single
record at a time when calling getNextRecords. As the return type indicates,
the method can return an arbitrary number of records.

Cheers,
Till

On Wed, Dec 11, 2019 at 10:13 AM Dawid Wysakowicz <dw...@apache.org>
wrote:

> Hi Becket,
>
> Issue #1 - Design of Source interface
>
> I mentioned the lack of a method like Source#createEnumerator(Boundedness
> boundedness, SplitEnumeratorContext context), because without the current
> proposal is not complete/does not work.
>
> If we say that boundedness is an intrinsic property of a source imo we
> don't need the Source#createEnumerator(Boundedness boundedness,
> SplitEnumeratorContext context) method.
>
> Assuming a source from my previous example:
>
> Source source = KafkaSource.builder()
>   ...
>   .untilTimestamp(...)
>   .build()
>
> Would the enumerator differ if created like
> source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
> .createEnumerator(BOUNDED, ...)? I know I am repeating myself, but this is
> the part that my opinion differ the most from the current proposal. I
> really think it should always be the source that tells if it is bounded or
> not. In the current proposal methods continousSource/boundedSource somewhat
> reconfigure the source, which I think is misleading.
>
> I think a call like:
>
> Source source = KafkaSource.builder()
>   ...
>   .readContinously() / readUntilLatestOffset() / readUntilTimestamp / readUntilOffsets / ...
>   .build()
>
> is way cleaner (and expressive) than
>
> Source source = KafkaSource.builder()
>   ...
>   .build()
>
>
> env.continousSource(source) // which actually underneath would call createEnumerator(CONTINUOUS, ctx) which would be equivalent to source.readContinously().createEnumerator(ctx)
> // or
> env.boundedSource(source) // which actually underneath would call createEnumerator(BOUNDED, ctx) which would be equivalent to source.readUntilLatestOffset().createEnumerator(ctx)
>
>
> Sorry for the comparison, but to me it seems there is too much magic
> happening underneath those two calls.
>
> I really believe the Source interface should have getBoundedness method
> instead of (supportBoundedness) + createEnumerator(Boundedness, ...)
>
>
> Issue #2 - Design of
> ExecutionEnvironment#source()/continuousSource()/boundedSource()
>
> As you might have guessed I am slightly in favor of option #2 modified.
> Yes I am aware every step of the dag would have to be able to say if it is
> bounded or not. I have a feeling it would be easier to express cross
> bounded/unbounded operations, but I must admit I have not thought it
> through thoroughly, In the spirit of batch is just a special case of
> streaming I thought BoundedStream would extend from DataStream. Correct me
> if I am wrong. In such a setup the cross bounded/unbounded operation could
> be expressed quite easily I think:
>
> DataStream {
>   DataStream join(DataStream, ...); // we could not really tell if the result is bounded or not, but because bounded stream is a special case of unbounded the API object is correct, irrespective if the left or right side of the join is bounded
> }
>
> BoundedStream extends DataStream {
>   BoundedStream join(BoundedStream, ...); // only if both sides are bounded the result can be bounded as well. However we do have access to the DataStream#join here, so you can still join with a DataStream
> }
>
>
> On the other hand I also see benefits of two completely disjointed APIs,
> as we could prohibit some streaming calls in the bounded API. I can't think
> of any unbounded operators that could not be implemented for bounded stream.
>
> Besides I think we both agree we don't like the method:
>
> DataStream boundedStream(Source)
>
> suggested in the current state of the FLIP. Do we ? :)
>
> Best,
>
> Dawid
>
> On 10/12/2019 18:57, Becket Qin wrote:
>
> Hi folks,
>
> Thanks for the discussion, great feedback. Also thanks Dawid for the
> explanation, it is much clearer now.
>
> One thing that is indeed missing from the FLIP is how the boundedness is
> passed to the Source implementation. So the API should be
> Source#createEnumerator(Boundedness boundedness, SplitEnumeratorContext
> context)
> And we can probably remove the Source#supportBoundedness(Boundedness
> boundedness) method.
>
> Assuming we have that, we are essentially choosing from one of the
> following two options:
>
> Option 1:
> // The source is continuous source, and only unbounded operations can be
> performed.
> DataStream<Type> datastream = env.continuousSource(someSource);
>
> // The source is bounded source, both bounded and unbounded operations can
> be performed.
> BoundedDataStream<Type> boundedDataStream = env.boundedSource(someSource);
>
>   - Pros:
>        a) explicit boundary between bounded / unbounded streams, it is
> quite simple and clear to the users.
>   - Cons:
>        a) For applications that do not involve bounded operations, they
> still have to call different API to distinguish bounded / unbounded streams.
>        b) No support for bounded stream to run in a streaming runtime
> setting, i.e. scheduling and operators behaviors.
>
>
> Option 2:
> // The source is either bounded or unbounded, but only unbounded operations
> could be performed on the returned DataStream.
> DataStream<Type> dataStream = env.source(someSource);
>
> // The source must be a bounded source, otherwise exception is thrown.
> BoundedDataStream<Type> boundedDataStream =
> env.boundedSource(boundedSource);
>
> The pros and cons are exactly the opposite of option 1.
>   - Pros:
>        a) For applications that do not involve bounded operations, they
> still have to call different API to distinguish bounded / unbounded streams.
>        b) Support for bounded stream to run in a streaming runtime setting,
> i.e. scheduling and operators behaviors.
>   - Cons:
>        a) Bounded / unbounded streams are kind of mixed, i.e. given a
> DataStream, it is not clear whether it is bounded or not, unless you have
> the access to its source.
>
>
> If we only think from the Source API perspective, option 2 seems a better
> choice because functionality wise it is a superset of option 1, at the cost
> of some seemingly acceptable ambiguity in the DataStream API.
> But if we look at the DataStream API as a whole, option 1 seems a clearer
> choice. For example, some times a library may have to know whether a
> certain task will finish or not. And it would be difficult to tell if the
> input is a DataStream, unless additional information is provided all the
> way from the Source. One possible solution is to have a *modified option 2*
> which adds a method to the DataStream API to indicate boundedness, such as
> getBoundedness(). It would solve the problem with a potential confusion of
> what is difference between a DataStream with getBoundedness()=true and a
> BoundedDataStream. But that seems not super difficult to explain.
>
> So from API's perspective, I don't have a strong opinion between *option 1*
> and *modified option 2. *I like the cleanness of option 1, but modified
> option 2 would be more attractive if we have concrete use case for the
> "Bounded stream with unbounded streaming runtime settings".
>
> Re: Till
>
>
> Maybe this has already been asked before but I was wondering why the
> SourceReader interface has the method pollNext which hands the
> responsibility of outputting elements to the SourceReader implementation?
> Has this been done for backwards compatibility reasons with the old source
> interface? If not, then one could define a Collection<E> getNextRecords()
> method which returns the currently retrieved records and then the caller
> emits them outside of the SourceReader. That way the interface would not
> allow to implement an outputting loop where we never hand back control to
> the caller. At the moment, this contract can be easily broken and is only
> mentioned loosely in the JavaDocs.
>
>
> The primary reason we handover the SourceOutput to the SourceReader is
> because sometimes it is difficult for a SourceReader to emit one record at
> a time. One example is some batched messaging systems which only have an
> offset for the entire batch instead of individual messages in the batch. In
> that case, returning one record at a time would leave the SourceReader in
> an uncheckpointable state because they can only checkpoint at the batch
> boundaries.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <tr...@apache.org> <tr...@apache.org> wrote:
>
>
> Hi everyone,
>
> thanks for drafting this FLIP. It reads very well.
>
> Concerning Dawid's proposal, I tend to agree. The boundedness could come
> from the source and tell the system how to treat the operator (scheduling
> wise). From a user's perspective it should be fine to get back a DataStream
> when calling env.source(boundedSource) if he does not need special
> operations defined on a BoundedDataStream. If he needs this, then one could
> use the method BoundedDataStream env.boundedSource(boundedSource).
>
> If possible, we could enforce the proper usage of env.boundedSource() by
> introducing a BoundedSource type so that one cannot pass an
> unbounded source to it. That way users would not be able to shoot
> themselves in the foot.
>
> Maybe this has already been asked before but I was wondering why the
> SourceReader interface has the method pollNext which hands the
> responsibility of outputting elements to the SourceReader implementation?
> Has this been done for backwards compatibility reasons with the old source
> interface? If not, then one could define a Collection<E> getNextRecords()
> method which returns the currently retrieved records and then the caller
> emits them outside of the SourceReader. That way the interface would not
> allow to implement an outputting loop where we never hand back control to
> the caller. At the moment, this contract can be easily broken and is only
> mentioned loosely in the JavaDocs.
>
> Cheers,
> Till
>
> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <ji...@gmail.com> <ji...@gmail.com>
> wrote:
>
>
> Hi all,
>
> I think current design is good.
>
> My understanding is:
>
> For execution mode: bounded mode and continuous mode, It's totally
> different. I don't think we have the ability to integrate the two models
>
> at
>
> present. It's about scheduling, memory, algorithms, States, etc. we
> shouldn't confuse them.
>
> For source capabilities: only bounded, only continuous, both bounded and
> continuous.
> I think Kafka is a source that can be ran both bounded
> and continuous execution mode.
> And Kafka with end offset should be ran both bounded
> and continuous execution mode.  Using apache Beam with Flink runner, I
>
> used
>
> to run a "bounded" Kafka in streaming mode. For our previous DataStream,
>
> it
>
> is not necessarily required that the source cannot be bounded.
>
> So it is my thought for Dawid's question:
> 1.pass a bounded source to continuousSource() +1
> 2.pass a continuous source to boundedSource() -1, should throw exception.
>
> In StreamExecutionEnvironment, continuousSource and boundedSource define
> the execution mode. It defines a clear boundary of execution mode.
>
> Best,
> Jingsong Lee
>
> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <im...@gmail.com> <im...@gmail.com> wrote:
>
>
> I agree with Dawid's point that the boundedness information should come
> from the source itself (e.g. the end timestamp), not through
> env.boundedSouce()/continuousSource().
> I think if we want to support something like `env.source()` that derive
>
> the
>
> execution mode from source, `supportsBoundedness(Boundedness)`
> method is not enough, because we don't know whether it is bounded or
>
> not.
>
> Best,
> Jark
>
>
> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <dw...@apache.org> <dw...@apache.org>
> wrote:
>
>
> One more thing. In the current proposal, with the
> supportsBoundedness(Boundedness) method and the boundedness coming
>
> from
>
> either continuousSource or boundedSource I could not find how this
> information is fed back to the SplitEnumerator.
>
> Best,
>
> Dawid
>
> On 09/12/2019 13:52, Becket Qin wrote:
>
> Hi Dawid,
>
> Thanks for the comments. This actually brings another relevant
>
> question
>
> about what does a "bounded source" imply. I actually had the same
> impression when I look at the Source API. Here is what I understand
>
> after
>
> some discussion with Stephan. The bounded source has the following
>
> impacts.
>
> 1. API validity.
> - A bounded source generates a bounded stream so some operations
>
> that
>
> only
>
> works for bounded records would be performed, e.g. sort.
> - To expose these bounded stream only APIs, there are two options:
>      a. Add them to the DataStream API and throw exception if a
>
> method
>
> is
>
> called on an unbounded stream.
>      b. Create a BoundedDataStream class which is returned from
> env.boundedSource(), while DataStream is returned from
>
> env.continousSource().
>
> Note that this cannot be done by having single
>
> env.source(theSource)
>
> even
>
> the Source has a getBoundedness() method.
>
> 2. Scheduling
> - A bounded source could be computed stage by stage without
>
> bringing
>
> up
>
> all
>
> the tasks at the same time.
>
> 3. Operator behaviors
> - A bounded source indicates the records are finite so some
>
> operators
>
> can
>
> wait until it receives all the records before it starts the
>
> processing.
>
> In the above impact, only 1 is relevant to the API design. And the
>
> current
>
> proposal in FLIP-27 is following 1.b.
>
> // boundedness depends of source property, imo this should always
>
> be
>
> preferred
>
>
> DataStream<MyType> stream = env.source(theSource);
>
>
> In your proposal, does DataStream have bounded stream only methods?
>
> It
>
> looks it should have, otherwise passing a bounded Source to
>
> env.source()
>
> would be confusing. In that case, we will essentially do 1.a if an
> unbounded Source is created from env.source(unboundedSource).
>
> If we have the methods only supported for bounded streams in
>
> DataStream,
>
> it
>
> seems a little weird to have a separate BoundedDataStream
>
> interface.
>
> Am I understand it correctly?
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
>
> dwysakowicz@apache.org>
>
> wrote:
>
>
> Hi all,
>
> Really well written proposal and very important one. I must admit
>
> I
>
> have
>
> not understood all the intricacies of it yet.
>
> One question I have though is about where does the information
>
> about
>
> boundedness come from. I think in most cases it is a property of
>
> the
>
> source. As you described it might be e.g. end offset, a flag
>
> should
>
> it
>
> monitor new splits etc. I think it would be a really nice use case
>
> to
>
> be
>
> able to say:
>
> new KafkaSource().readUntil(long timestamp),
>
> which could work as an "end offset". Moreover I think all Bounded
>
> sources
>
> support continuous mode, but no intrinsically continuous source
>
> support
>
> the
>
> Bounded mode. If I understood the proposal correctly it suggest
>
> the
>
> boundedness sort of "comes" from the outside of the source, from
>
> the
>
> invokation of either boundedStream or continousSource.
>
> I am wondering if it would make sense to actually change the
>
> method
>
> boolean Source#supportsBoundedness(Boundedness)
>
> to
>
> Boundedness Source#getBoundedness().
>
> As for the methods #boundedSource, #continousSource, assuming the
> boundedness is property of the source they do not affect how the
>
> enumerator
>
> works, but mostly how the dag is scheduled, right? I am not
>
> against
>
> those
>
> methods, but I think it is a very specific use case to actually
>
> override
>
> the property of the source. In general I would expect users to
>
> only
>
> call
>
> env.source(theSource), where the source tells if it is bounded or
>
> not. I
>
> would suggest considering following set of methods:
>
> // boundedness depends of source property, imo this should always
>
> be
>
> preferred
>
> DataStream<MyType> stream = env.source(theSource);
>
>
> // always continous execution, whether bounded or unbounded source
>
> DataStream<MyType> boundedStream = env.continousSource(theSource);
>
> // imo this would make sense if the BoundedDataStream provides
>
> additional features unavailable for continous mode
>
> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
>
>
> Best,
>
> Dawid
>
>
> On 04/12/2019 11:25, Stephan Ewen wrote:
>
> Thanks, Becket, for updating this.
>
> I agree with moving the aspects you mentioned into separate FLIPs
>
> -
>
> this
>
> one way becoming unwieldy in size.
>
> +1 to the FLIP in its current state. Its a very detailed write-up,
>
> nicely
>
> done!
>
> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <be...@gmail.com> <be...@gmail.com>
>
> <
>
> becket.qin@gmail.com> wrote:
>
> Hi all,
>
> Sorry for the long belated update. I have updated FLIP-27 wiki
>
> page
>
> with
>
> the latest proposals. Some noticeable changes include:
> 1. A new generic communication mechanism between SplitEnumerator
>
> and
>
> SourceReader.
> 2. Some detail API method signature changes.
>
> We left a few things out of this FLIP and will address them in
>
> separate
>
> FLIPs. Including:
> 1. Per split event time.
> 2. Event time alignment.
> 3. Fine grained failover for SplitEnumerator failure.
>
> Please let us know if you have any question.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> <se...@apache.org> <
>
> sewen@apache.org> wrote:
>
> Hi  Łukasz!
>
> Becket and me are working hard on figuring out the last details
>
> and
>
> implementing the first PoC. We would update the FLIP hopefully
>
> next
>
> week.
>
> There is a fair chance that a first version of this will be in
>
> 1.10,
>
> but
>
> I
>
> think it will take another release to battle test it and migrate
>
> the
>
> connectors.
>
> Best,
> Stephan
>
>
>
>
> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <ljd@touk.pl
>
> <
>
> ljd@touk.pl>
>
> wrote:
>
> Hi,
>
> This proposal looks very promising for us. Do you have any plans
>
> in
>
> which
>
> Flink release it is going to be released? We are thinking on
>
> using a
>
> Data
>
> Set API for our future use cases but on the other hand Data Set
>
> API
>
> is
>
> going to be deprecated so using proposed bounded data streams
>
> solution
>
> could be more viable in the long term.
>
> Thanks,
> Łukasz
>
> On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> <th...@gmail.com> <
>
> thomas.weise@gmail.com> wrote:
>
> Thanks for putting together this proposal!
>
> I see that the "Per Split Event Time" and "Event Time Alignment"
>
> sections
>
> are still TBD.
>
> It would probably be good to flesh those out a bit before
>
> proceeding
>
> too
>
> far
>
> as the event time alignment will probably influence the
>
> interaction
>
> with
>
> the split reader, specifically ReaderStatus
>
> emitNext(SourceOutput<E>
>
> output).
>
> We currently have only one implementation for event time alignment
>
> in
>
> the
>
> Kinesis consumer. The synchronization in that case takes place as
>
> the
>
> last
>
> step before records are emitted downstream (RecordEmitter). With
>
> the
>
> currently proposed interfaces, the equivalent can be implemented
>
> in
>
> the
>
> reader loop, although note that in the Kinesis consumer the per
>
> shard
>
> threads push records.
>
> Synchronization has not been implemented for the Kafka consumer
>
> yet.
>
> https://issues.apache.org/jira/browse/FLINK-12675
>
> When I looked at it, I realized that the implementation will look
>
> quite
>
> different
> from Kinesis because it needs to take place in the pull part,
>
> where
>
> records
>
> are taken from the Kafka client. Due to the multiplexing it cannot
>
> be
>
> done
>
> by blocking the split thread like it currently works for Kinesis.
>
> Reading
>
> from individual Kafka partitions needs to be controlled via
>
> pause/resume
>
> on the Kafka client.
>
> To take on that responsibility the split thread would need to be
>
> aware
>
> of
>
> the
> watermarks or at least whether it should or should not continue to
>
> consume
>
> a given split and this may require a different SourceReader or
>
> SourceOutput
>
> interface.
>
> Thanks,
> Thomas
>
>
> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com> <
>
> mmyy1110@gmail.com> wrote:
>
> Hi Stephan,
>
> Thank you for feedback!
> Will take a look at your branch before public discussing.
>
>
> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org> <se...@apache.org>
>
> <
>
> sewen@apache.org>
>
> wrote:
>
> Hi Biao!
>
> Thanks for reviving this. I would like to join this discussion,
>
> but
>
> am
>
> quite occupied with the 1.9 release, so can we maybe pause this
>
> discussion
>
> for a week or so?
>
> In the meantime I can share some suggestion based on prior
>
> experiments:
>
> How to do watermarks / timestamp extractors in a simpler and more
>
> flexible
>
> way. I think that part is quite promising should be part of the
>
> new
>
> source
>
> interface.
>
>
>
>
>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>
> Some experiments on how to build the source reader and its
>
> library
>
> for
>
> common threading/split patterns:
>
>
>
>
>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>
> Best,
> Stephan
>
>
> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com> <
>
> mmyy1110@gmail.com>
>
> wrote:
>
> Hi devs,
>
> Since 1.9 is nearly released, I think we could get back to
>
> FLIP-27.
>
> I
>
> believe it should be included in 1.10.
>
> There are so many things mentioned in document of FLIP-27. [1] I
>
> think
>
> we'd better discuss them separately. However the wiki is not a
>
> good
>
> place
>
> to discuss. I wrote google doc about SplitReader API which
>
> misses
>
> some
>
> details in the document. [2]
>
> 1.
>
>
>
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>
> 2.
>
>
>
>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>
> CC Stephan, Aljoscha, Piotrek, Becket
>
>
> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> <mm...@gmail.com> <
>
> mmyy1110@gmail.com>
>
> wrote:
>
> Hi Steven,
> Thank you for the feedback. Please take a look at the document
>
> FLIP-27
>
> <
>
>
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>
> which
>
> is updated recently. A lot of details of enumerator were added
>
> in
>
> this
>
> document. I think it would help.
>
> Steven Wu <st...@gmail.com> <st...@gmail.com> <st...@gmail.com> <st...@gmail.com>
>
> 于2019年3月28日周四
>
> 下午12:52写道:
>
> This proposal mentioned that SplitEnumerator might run on the
> JobManager or
> in a single task on a TaskManager.
>
> if enumerator is a single task on a taskmanager, then the job
>
> DAG
>
> can
>
> never
> been embarrassingly parallel anymore. That will nullify the
>
> leverage
>
> of
>
> fine-grained recovery for embarrassingly parallel jobs.
>
> It's not clear to me what's the implication of running
>
> enumerator
>
> on
>
> the
>
> jobmanager. So I will leave that out for now.
>
> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com> <
>
> mmyy1110@gmail.com>
>
> wrote:
>
> Hi Stephan & Piotrek,
>
> Thank you for feedback.
>
> It seems that there are a lot of things to do in community.
>
> I
>
> am
>
> just
>
> afraid that this discussion may be forgotten since there so
>
> many
>
> proposals
>
> recently.
> Anyway, wish to see the split topics soon :)
>
> Piotr Nowojski <pi...@da-platform.com> <pi...@da-platform.com> <pi...@da-platform.com> <pi...@da-platform.com>
>
> 于2019年1月24日周四
>
> 下午8:21写道:
>
> Hi Biao!
>
> This discussion was stalled because of preparations for
>
> the
>
> open
>
> sourcing
>
> & merging Blink. I think before creating the tickets we
>
> should
>
> split this
>
> discussion into topics/areas outlined by Stephan and
>
> create
>
> Flips
>
> for
>
> that.
>
> I think there is no chance for this to be completed in
>
> couple
>
> of
>
> remaining
>
> weeks/1 month before 1.8 feature freeze, however it would
>
> be
>
> good
>
> to aim
>
> with those changes for 1.9.
>
> Piotrek
>
>
> On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> <mm...@gmail.com> <
>
> mmyy1110@gmail.com>
>
> wrote:
>
> Hi community,
> The summary of Stephan makes a lot sense to me. It is
>
> much
>
> clearer
>
> indeed
>
> after splitting the complex topic into small ones.
> I was wondering is there any detail plan for next step?
>
> If
>
> not,
>
> I
>
> would
>
> like to push this thing forward by creating some JIRA
>
> issues.
>
> Another question is that should version 1.8 include
>
> these
>
> features?
>
> Stephan Ewen <se...@apache.org> <se...@apache.org> <se...@apache.org> <se...@apache.org> 于2018年12月1日周六
>
> 上午4:20写道:
>
> Thanks everyone for the lively discussion. Let me try
>
> to
>
> summarize
>
> where I
>
> see convergence in the discussion and open issues.
> I'll try to group this by design aspect of the source.
>
> Please
>
> let me
>
> know
>
> if I got things wrong or missed something crucial here.
>
> For issues 1-3, if the below reflects the state of the
>
> discussion, I
>
> would
>
> try and update the FLIP in the next days.
> For the remaining ones we need more discussion.
>
> I would suggest to fork each of these aspects into a
>
> separate
>
> mail
>
> thread,
>
> or will loose sight of the individual aspects.
>
> *(1) Separation of Split Enumerator and Split Reader*
>
>  - All seem to agree this is a good thing
>  - Split Enumerator could in the end live on JobManager
>
> (and
>
> assign
>
> splits
>
> via RPC) or in a task (and assign splits via data
>
> streams)
>
>  - this discussion is orthogonal and should come later,
>
> when
>
> the
>
> interface
>
> is agreed upon.
>
> *(2) Split Readers for one or more splits*
>
>  - Discussion seems to agree that we need to support
>
> one
>
> reader
>
> that
>
> possibly handles multiple splits concurrently.
>  - The requirement comes from sources where one
>
> poll()-style
>
> call
>
> fetches
>
> data from different splits / partitions
>    --> example sources that require that would be for
>
> example
>
> Kafka,
>
> Pravega, Pulsar
>
>  - Could have one split reader per source, or multiple
>
> split
>
> readers
>
> that
>
> share the "poll()" function
>  - To not make it too complicated, we can start with
>
> thinking
>
> about
>
> one
>
> split reader for all splits initially and see if that
>
> covers
>
> all
>
> requirements
>
> *(3) Threading model of the Split Reader*
>
>  - Most active part of the discussion ;-)
>
>  - A non-blocking way for Flink's task code to interact
>
> with
>
> the
>
> source
>
> is
>
> needed in order to a task runtime code based on a
> single-threaded/actor-style task design
>    --> I personally am a big proponent of that, it will
>
> help
>
> with
>
> well-behaved checkpoints, efficiency, and simpler yet
>
> more
>
> robust
>
> runtime
>
> code
>
>  - Users care about simple abstraction, so as a
>
> subclass
>
> of
>
> SplitReader
>
> (non-blocking / async) we need to have a
>
> BlockingSplitReader
>
> which
>
> will
>
> form the basis of most source implementations.
>
> BlockingSplitReader
>
> lets
>
> users do blocking simple poll() calls.
>  - The BlockingSplitReader would spawn a thread (or
>
> more)
>
> and
>
> the
>
> thread(s) can make blocking calls and hand over data
>
> buffers
>
> via
>
> a
>
> blocking
>
> queue
>  - This should allow us to cover both, a fully async
>
> runtime,
>
> and a
>
> simple
>
> blocking interface for users.
>  - This is actually very similar to how the Kafka
>
> connectors
>
> work.
>
> Kafka
>
> 9+ with one thread, Kafka 8 with multiple threads
>
>  - On the base SplitReader (the async one), the
>
> non-blocking
>
> method
>
> that
>
> gets the next chunk of data would signal data
>
> availability
>
> via
>
> a
>
> CompletableFuture, because that gives the best
>
> flexibility
>
> (can
>
> await
>
> completion or register notification handlers).
>  - The source task would register a "thenHandle()" (or
>
> similar)
>
> on the
>
> future to put a "take next data" task into the
>
> actor-style
>
> mailbox
>
> *(4) Split Enumeration and Assignment*
>
>  - Splits may be generated lazily, both in cases where
>
> there
>
> is a
>
> limited
>
> number of splits (but very many), or splits are
>
> discovered
>
> over
>
> time
>
>  - Assignment should also be lazy, to get better load
>
> balancing
>
>  - Assignment needs support locality preferences
>
>  - Possible design based on discussion so far:
>
>    --> SplitReader has a method "addSplits(SplitT...)"
>
> to
>
> add
>
> one or
>
> more
>
> splits. Some split readers might assume they have only
>
> one
>
> split
>
> ever,
>
> concurrently, others assume multiple splits. (Note:
>
> idea
>
> behind
>
> being
>
> able
>
> to add multiple splits at the same time is to ease
>
> startup
>
> where
>
> multiple
>
> splits may be assigned instantly.)
>    --> SplitReader has a context object on which it can
>
> call
>
> indicate
>
> when
>
> splits are completed. The enumerator gets that
>
> notification and
>
> can
>
> use
>
> to
>
> decide when to assign new splits. This should help both
>
> in
>
> cases
>
> of
>
> sources
>
> that take splits lazily (file readers) and in case the
>
> source
>
> needs to
>
> preserve a partial order between splits (Kinesis,
>
> Pravega,
>
> Pulsar may
>
> need
>
> that).
>    --> SplitEnumerator gets notification when
>
> SplitReaders
>
> start
>
> and
>
> when
>
> they finish splits. They can decide at that moment to
>
> push
>
> more
>
> splits
>
> to
>
> that reader
>    --> The SplitEnumerator should probably be aware of
>
> the
>
> source
>
> parallelism, to build its initial distribution.
>
>  - Open question: Should the source expose something
>
> like
>
> "host
>
> preferences", so that yarn/mesos/k8s can take this into
>
> account
>
> when
>
> selecting a node to start a TM on?
>
> *(5) Watermarks and event time alignment*
>
>  - Watermark generation, as well as idleness, needs to
>
> be
>
> per
>
> split
>
> (like
>
> currently in the Kafka Source, per partition)
>  - It is desirable to support optional
>
> event-time-alignment,
>
> meaning
>
> that
>
> splits that are ahead are back-pressured or temporarily
>
> unsubscribed
>
>  - I think i would be desirable to encapsulate
>
> watermark
>
> generation
>
> logic
>
> in watermark generators, for a separation of concerns.
>
> The
>
> watermark
>
> generators should run per split.
>  - Using watermark generators would also help with
>
> another
>
> problem of
>
> the
>
> suggested interface, namely supporting non-periodic
>
> watermarks
>
> efficiently.
>
>  - Need a way to "dispatch" next record to different
>
> watermark
>
> generators
>
>  - Need a way to tell SplitReader to "suspend" a split
>
> until a
>
> certain
>
> watermark is reached (event time backpressure)
>  - This would in fact be not needed (and thus simpler)
>
> if
>
> we
>
> had
>
> a
>
> SplitReader per split and may be a reason to re-open
>
> that
>
> discussion
>
> *(6) Watermarks across splits and in the Split
>
> Enumerator*
>
>  - The split enumerator may need some watermark
>
> awareness,
>
> which
>
> should
>
> be
>
> purely based on split metadata (like create timestamp
>
> of
>
> file
>
> splits)
>
>  - If there are still more splits with overlapping
>
> event
>
> time
>
> range
>
> for
>
> a
>
> split reader, then that split reader should not advance
>
> the
>
> watermark
>
> within the split beyond the overlap boundary. Otherwise
>
> future
>
> splits
>
> will
>
> produce late data.
>
>  - One way to approach this could be that the split
>
> enumerator
>
> may
>
> send
>
> watermarks to the readers, and the readers cannot emit
>
> watermarks
>
> beyond
>
> that received watermark.
>  - Many split enumerators would simply immediately send
>
> Long.MAX
>
> out
>
> and
>
> leave the progress purely to the split readers.
>
>  - For event-time alignment / split back pressure, this
>
> begs
>
> the
>
> question
>
> how we can avoid deadlocks that may arise when splits
>
> are
>
> suspended
>
> for
>
> event time back pressure,
>
> *(7) Batch and streaming Unification*
>
>  - Functionality wise, the above design should support
>
> both
>
>  - Batch often (mostly) does not care about reading "in
>
> order"
>
> and
>
> generating watermarks
>    --> Might use different enumerator logic that is
>
> more
>
> locality
>
> aware
>
> and ignores event time order
>    --> Does not generate watermarks
>  - Would be great if bounded sources could be
>
> identified
>
> at
>
> compile
>
> time,
>
> so that "env.addBoundedSource(...)" is type safe and
>
> can
>
> return a
>
> "BoundedDataStream".
>  - Possible to defer this discussion until later
>
> *Miscellaneous Comments*
>
>  - Should the source have a TypeInformation for the
>
> produced
>
> type,
>
> instead
>
> of a serializer? We need a type information in the
>
> stream
>
> anyways, and
>
> can
>
> derive the serializer from that. Plus, creating the
>
> serializer
>
> should
>
> respect the ExecutionConfig.
>
>  - The TypeSerializer interface is very powerful but
>
> also
>
> not
>
> easy to
>
> implement. Its purpose is to handle data super
>
> efficiently,
>
> support
>
> flexible ways of evolution, etc.
>  For metadata I would suggest to look at the
>
> SimpleVersionedSerializer
>
> instead, which is used for example for checkpoint
>
> master
>
> hooks,
>
> or for
>
> the
>
> streaming file sink. I think that is is a good match
>
> for
>
> cases
>
> where
>
> we
>
> do
>
> not need more than ser/deser (no copy, etc.) and don't
>
> need to
>
> push
>
> versioning out of the serialization paths for best
>
> performance
>
> (as in
>
> the
>
> TypeSerializer)
>
>
> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>
> k.kloudas@data-artisans.com>
>
> wrote:
>
>
> Hi Biao,
>
> Thanks for the answer!
>
> So given the multi-threaded readers, now we have as
>
> open
>
> questions:
>
> 1) How do we let the checkpoints pass through our
>
> multi-threaded
>
> reader
>
> operator?
>
> 2) Do we have separate reader and source operators or
>
> not? In
>
> the
>
> strategy
>
> that has a separate source, the source operator has a
>
> parallelism of
>
> 1
>
> and
>
> is responsible for split recovery only.
>
> For the first one, given also the constraints
>
> (blocking,
>
> finite
>
> queues,
>
> etc), I do not have an answer yet.
>
> For the 2nd, I think that we should go with separate
>
> operators
>
> for
>
> the
>
> source and the readers, for the following reasons:
>
> 1) This is more aligned with a potential future
>
> improvement
>
> where the
>
> split
>
> discovery becomes a responsibility of the JobManager
>
> and
>
> readers are
>
> pooling more work from the JM.
>
> 2) The source is going to be the "single point of
>
> truth".
>
> It
>
> will
>
> know
>
> what
>
> has been processed and what not. If the source and the
>
> readers
>
> are a
>
> single
>
> operator with parallelism > 1, or in general, if the
>
> split
>
> discovery
>
> is
>
> done by each task individually, then:
>   i) we have to have a deterministic scheme for each
>
> reader to
>
> assign
>
> splits to itself (e.g. mod subtaskId). This is not
>
> necessarily
>
> trivial
>
> for
>
> all sources.
>   ii) each reader would have to keep a copy of all its
>
> processed
>
> slpits
>
>   iii) the state has to be a union state with a
>
> non-trivial
>
> merging
>
> logic
>
> in order to support rescaling.
>
> Two additional points that you raised above:
>
> i) The point that you raised that we need to keep all
>
> splits
>
> (processed
>
> and
>
> not-processed) I think is a bit of a strong
>
> requirement.
>
> This
>
> would
>
> imply
>
> that for infinite sources the state will grow
>
> indefinitely.
>
> This is
>
> problem
>
> is even more pronounced if we do not have a single
>
> source
>
> that
>
> assigns
>
> splits to readers, as each reader will have its own
>
> copy
>
> of
>
> the
>
> state.
>
> ii) it is true that for finite sources we need to
>
> somehow
>
> not
>
> close
>
> the
>
> readers when the source/split discoverer finishes. The
> ContinuousFileReaderOperator has a work-around for
>
> that.
>
> It is
>
> not
>
> elegant,
>
> and checkpoints are not emitted after closing the
>
> source,
>
> but
>
> this, I
>
> believe, is a bigger problem which requires more
>
> changes
>
> than
>
> just
>
> refactoring the source interface.
>
> Cheers,
> Kostas
>
>
>
>
> --
> Best, Jingsong Lee
>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Dawid Wysakowicz <dw...@apache.org>.
Hi Becket,

Issue #1 - Design of Source interface

I mentioned the lack of a method like
Source#createEnumerator(Boundedness boundedness, SplitEnumeratorContext
context), because without the current proposal is not complete/does not
work.

If we say that boundedness is an intrinsic property of a source imo we
don't need the Source#createEnumerator(Boundedness boundedness,
SplitEnumeratorContext context) method.

Assuming a source from my previous example:

Source source = KafkaSource.builder()
  ...
  .untilTimestamp(...)
  .build()

Would the enumerator differ if created like
source.createEnumerator(CONTINUOUS_UNBOUNDED, ...) vs source
.createEnumerator(BOUNDED, ...)? I know I am repeating myself, but this
is the part that my opinion differ the most from the current proposal. I
really think it should always be the source that tells if it is bounded
or not. In the current proposal methods continousSource/boundedSource
somewhat reconfigure the source, which I think is misleading.

I think a call like:

Source source = KafkaSource.builder()
  ...
  .readContinously() / readUntilLatestOffset() / readUntilTimestamp / readUntilOffsets / ...
  .build()

is way cleaner (and expressive) than

Source source = KafkaSource.builder()
  ...
  .build()

env.continousSource(source) // which actually underneath would call createEnumerator(CONTINUOUS, ctx) which would be equivalent to source.readContinously().createEnumerator(ctx)
// or
env.boundedSource(source) // which actually underneath would call createEnumerator(BOUNDED, ctx) which would be equivalent to source.readUntilLatestOffset().createEnumerator(ctx)

Sorry for the comparison, but to me it seems there is too much magic
happening underneath those two calls.

I really believe the Source interface should have getBoundedness method
instead of (supportBoundedness) + createEnumerator(Boundedness, ...)


Issue #2 - Design of
ExecutionEnvironment#source()/continuousSource()/boundedSource()

As you might have guessed I am slightly in favor of option #2 modified.
Yes I am aware every step of the dag would have to be able to say if it
is bounded or not. I have a feeling it would be easier to express cross
bounded/unbounded operations, but I must admit I have not thought it
through thoroughly, In the spirit of batch is just a special case of
streaming I thought BoundedStream would extend from DataStream. Correct
me if I am wrong. In such a setup the cross bounded/unbounded operation
could be expressed quite easily I think:

DataStream {
  DataStream join(DataStream, ...); // we could not really tell if the result is bounded or not, but because bounded stream is a special case of unbounded the API object is correct, irrespective if the left or right side of the join is bounded
}

BoundedStream extends DataStream {
  BoundedStream join(BoundedStream, ...); // only if both sides are bounded the result can be bounded as well. However we do have access to the DataStream#join here, so you can still join with a DataStream
}

On the other hand I also see benefits of two completely disjointed APIs,
as we could prohibit some streaming calls in the bounded API. I can't
think of any unbounded operators that could not be implemented for
bounded stream.

Besides I think we both agree we don't like the method:

DataStream boundedStream(Source)

suggested in the current state of the FLIP. Do we ? :)

Best,

Dawid

On 10/12/2019 18:57, Becket Qin wrote:

> Hi folks,
>
> Thanks for the discussion, great feedback. Also thanks Dawid for the
> explanation, it is much clearer now.
>
> One thing that is indeed missing from the FLIP is how the boundedness is
> passed to the Source implementation. So the API should be
> Source#createEnumerator(Boundedness boundedness, SplitEnumeratorContext
> context)
> And we can probably remove the Source#supportBoundedness(Boundedness
> boundedness) method.
>
> Assuming we have that, we are essentially choosing from one of the
> following two options:
>
> Option 1:
> // The source is continuous source, and only unbounded operations can be
> performed.
> DataStream<Type> datastream = env.continuousSource(someSource);
>
> // The source is bounded source, both bounded and unbounded operations can
> be performed.
> BoundedDataStream<Type> boundedDataStream = env.boundedSource(someSource);
>
>   - Pros:
>        a) explicit boundary between bounded / unbounded streams, it is
> quite simple and clear to the users.
>   - Cons:
>        a) For applications that do not involve bounded operations, they
> still have to call different API to distinguish bounded / unbounded streams.
>        b) No support for bounded stream to run in a streaming runtime
> setting, i.e. scheduling and operators behaviors.
>
>
> Option 2:
> // The source is either bounded or unbounded, but only unbounded operations
> could be performed on the returned DataStream.
> DataStream<Type> dataStream = env.source(someSource);
>
> // The source must be a bounded source, otherwise exception is thrown.
> BoundedDataStream<Type> boundedDataStream =
> env.boundedSource(boundedSource);
>
> The pros and cons are exactly the opposite of option 1.
>   - Pros:
>        a) For applications that do not involve bounded operations, they
> still have to call different API to distinguish bounded / unbounded streams.
>        b) Support for bounded stream to run in a streaming runtime setting,
> i.e. scheduling and operators behaviors.
>   - Cons:
>        a) Bounded / unbounded streams are kind of mixed, i.e. given a
> DataStream, it is not clear whether it is bounded or not, unless you have
> the access to its source.
>
>
> If we only think from the Source API perspective, option 2 seems a better
> choice because functionality wise it is a superset of option 1, at the cost
> of some seemingly acceptable ambiguity in the DataStream API.
> But if we look at the DataStream API as a whole, option 1 seems a clearer
> choice. For example, some times a library may have to know whether a
> certain task will finish or not. And it would be difficult to tell if the
> input is a DataStream, unless additional information is provided all the
> way from the Source. One possible solution is to have a *modified option 2*
> which adds a method to the DataStream API to indicate boundedness, such as
> getBoundedness(). It would solve the problem with a potential confusion of
> what is difference between a DataStream with getBoundedness()=true and a
> BoundedDataStream. But that seems not super difficult to explain.
>
> So from API's perspective, I don't have a strong opinion between *option 1*
> and *modified option 2. *I like the cleanness of option 1, but modified
> option 2 would be more attractive if we have concrete use case for the
> "Bounded stream with unbounded streaming runtime settings".
>
> Re: Till
>
>> Maybe this has already been asked before but I was wondering why the
>> SourceReader interface has the method pollNext which hands the
>> responsibility of outputting elements to the SourceReader implementation?
>> Has this been done for backwards compatibility reasons with the old source
>> interface? If not, then one could define a Collection<E> getNextRecords()
>> method which returns the currently retrieved records and then the caller
>> emits them outside of the SourceReader. That way the interface would not
>> allow to implement an outputting loop where we never hand back control to
>> the caller. At the moment, this contract can be easily broken and is only
>> mentioned loosely in the JavaDocs.
>>
> The primary reason we handover the SourceOutput to the SourceReader is
> because sometimes it is difficult for a SourceReader to emit one record at
> a time. One example is some batched messaging systems which only have an
> offset for the entire batch instead of individual messages in the batch. In
> that case, returning one record at a time would leave the SourceReader in
> an uncheckpointable state because they can only checkpoint at the batch
> boundaries.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <tr...@apache.org> wrote:
>
>> Hi everyone,
>>
>> thanks for drafting this FLIP. It reads very well.
>>
>> Concerning Dawid's proposal, I tend to agree. The boundedness could come
>> from the source and tell the system how to treat the operator (scheduling
>> wise). From a user's perspective it should be fine to get back a DataStream
>> when calling env.source(boundedSource) if he does not need special
>> operations defined on a BoundedDataStream. If he needs this, then one could
>> use the method BoundedDataStream env.boundedSource(boundedSource).
>>
>> If possible, we could enforce the proper usage of env.boundedSource() by
>> introducing a BoundedSource type so that one cannot pass an
>> unbounded source to it. That way users would not be able to shoot
>> themselves in the foot.
>>
>> Maybe this has already been asked before but I was wondering why the
>> SourceReader interface has the method pollNext which hands the
>> responsibility of outputting elements to the SourceReader implementation?
>> Has this been done for backwards compatibility reasons with the old source
>> interface? If not, then one could define a Collection<E> getNextRecords()
>> method which returns the currently retrieved records and then the caller
>> emits them outside of the SourceReader. That way the interface would not
>> allow to implement an outputting loop where we never hand back control to
>> the caller. At the moment, this contract can be easily broken and is only
>> mentioned loosely in the JavaDocs.
>>
>> Cheers,
>> Till
>>
>> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <ji...@gmail.com>
>> wrote:
>>
>>> Hi all,
>>>
>>> I think current design is good.
>>>
>>> My understanding is:
>>>
>>> For execution mode: bounded mode and continuous mode, It's totally
>>> different. I don't think we have the ability to integrate the two models
>> at
>>> present. It's about scheduling, memory, algorithms, States, etc. we
>>> shouldn't confuse them.
>>>
>>> For source capabilities: only bounded, only continuous, both bounded and
>>> continuous.
>>> I think Kafka is a source that can be ran both bounded
>>> and continuous execution mode.
>>> And Kafka with end offset should be ran both bounded
>>> and continuous execution mode.  Using apache Beam with Flink runner, I
>> used
>>> to run a "bounded" Kafka in streaming mode. For our previous DataStream,
>> it
>>> is not necessarily required that the source cannot be bounded.
>>>
>>> So it is my thought for Dawid's question:
>>> 1.pass a bounded source to continuousSource() +1
>>> 2.pass a continuous source to boundedSource() -1, should throw exception.
>>>
>>> In StreamExecutionEnvironment, continuousSource and boundedSource define
>>> the execution mode. It defines a clear boundary of execution mode.
>>>
>>> Best,
>>> Jingsong Lee
>>>
>>> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <im...@gmail.com> wrote:
>>>
>>>> I agree with Dawid's point that the boundedness information should come
>>>> from the source itself (e.g. the end timestamp), not through
>>>> env.boundedSouce()/continuousSource().
>>>> I think if we want to support something like `env.source()` that derive
>>> the
>>>> execution mode from source, `supportsBoundedness(Boundedness)`
>>>> method is not enough, because we don't know whether it is bounded or
>> not.
>>>> Best,
>>>> Jark
>>>>
>>>>
>>>> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <dw...@apache.org>
>>>> wrote:
>>>>
>>>>> One more thing. In the current proposal, with the
>>>>> supportsBoundedness(Boundedness) method and the boundedness coming
>> from
>>>>> either continuousSource or boundedSource I could not find how this
>>>>> information is fed back to the SplitEnumerator.
>>>>>
>>>>> Best,
>>>>>
>>>>> Dawid
>>>>>
>>>>> On 09/12/2019 13:52, Becket Qin wrote:
>>>>>> Hi Dawid,
>>>>>>
>>>>>> Thanks for the comments. This actually brings another relevant
>>> question
>>>>>> about what does a "bounded source" imply. I actually had the same
>>>>>> impression when I look at the Source API. Here is what I understand
>>>> after
>>>>>> some discussion with Stephan. The bounded source has the following
>>>>> impacts.
>>>>>> 1. API validity.
>>>>>> - A bounded source generates a bounded stream so some operations
>> that
>>>>> only
>>>>>> works for bounded records would be performed, e.g. sort.
>>>>>> - To expose these bounded stream only APIs, there are two options:
>>>>>>      a. Add them to the DataStream API and throw exception if a
>>> method
>>>> is
>>>>>> called on an unbounded stream.
>>>>>>      b. Create a BoundedDataStream class which is returned from
>>>>>> env.boundedSource(), while DataStream is returned from
>>>>> env.continousSource().
>>>>>> Note that this cannot be done by having single
>> env.source(theSource)
>>>> even
>>>>>> the Source has a getBoundedness() method.
>>>>>>
>>>>>> 2. Scheduling
>>>>>> - A bounded source could be computed stage by stage without
>> bringing
>>> up
>>>>> all
>>>>>> the tasks at the same time.
>>>>>>
>>>>>> 3. Operator behaviors
>>>>>> - A bounded source indicates the records are finite so some
>> operators
>>>> can
>>>>>> wait until it receives all the records before it starts the
>>> processing.
>>>>>> In the above impact, only 1 is relevant to the API design. And the
>>>>> current
>>>>>> proposal in FLIP-27 is following 1.b.
>>>>>>
>>>>>> // boundedness depends of source property, imo this should always
>> be
>>>>>>> preferred
>>>>>>>
>>>>>> DataStream<MyType> stream = env.source(theSource);
>>>>>>
>>>>>>
>>>>>> In your proposal, does DataStream have bounded stream only methods?
>>> It
>>>>>> looks it should have, otherwise passing a bounded Source to
>>>> env.source()
>>>>>> would be confusing. In that case, we will essentially do 1.a if an
>>>>>> unbounded Source is created from env.source(unboundedSource).
>>>>>>
>>>>>> If we have the methods only supported for bounded streams in
>>>> DataStream,
>>>>> it
>>>>>> seems a little weird to have a separate BoundedDataStream
>> interface.
>>>>>> Am I understand it correctly?
>>>>>>
>>>>>> Thanks,
>>>>>>
>>>>>> Jiangjie (Becket) Qin
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
>>>> dwysakowicz@apache.org>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi all,
>>>>>>>
>>>>>>> Really well written proposal and very important one. I must admit
>> I
>>>> have
>>>>>>> not understood all the intricacies of it yet.
>>>>>>>
>>>>>>> One question I have though is about where does the information
>> about
>>>>>>> boundedness come from. I think in most cases it is a property of
>> the
>>>>>>> source. As you described it might be e.g. end offset, a flag
>> should
>>> it
>>>>>>> monitor new splits etc. I think it would be a really nice use case
>>> to
>>>> be
>>>>>>> able to say:
>>>>>>>
>>>>>>> new KafkaSource().readUntil(long timestamp),
>>>>>>>
>>>>>>> which could work as an "end offset". Moreover I think all Bounded
>>>>> sources
>>>>>>> support continuous mode, but no intrinsically continuous source
>>>> support
>>>>> the
>>>>>>> Bounded mode. If I understood the proposal correctly it suggest
>> the
>>>>>>> boundedness sort of "comes" from the outside of the source, from
>> the
>>>>>>> invokation of either boundedStream or continousSource.
>>>>>>>
>>>>>>> I am wondering if it would make sense to actually change the
>> method
>>>>>>> boolean Source#supportsBoundedness(Boundedness)
>>>>>>>
>>>>>>> to
>>>>>>>
>>>>>>> Boundedness Source#getBoundedness().
>>>>>>>
>>>>>>> As for the methods #boundedSource, #continousSource, assuming the
>>>>>>> boundedness is property of the source they do not affect how the
>>>>> enumerator
>>>>>>> works, but mostly how the dag is scheduled, right? I am not
>> against
>>>>> those
>>>>>>> methods, but I think it is a very specific use case to actually
>>>> override
>>>>>>> the property of the source. In general I would expect users to
>> only
>>>> call
>>>>>>> env.source(theSource), where the source tells if it is bounded or
>>>> not. I
>>>>>>> would suggest considering following set of methods:
>>>>>>>
>>>>>>> // boundedness depends of source property, imo this should always
>> be
>>>>> preferred
>>>>>>> DataStream<MyType> stream = env.source(theSource);
>>>>>>>
>>>>>>>
>>>>>>> // always continous execution, whether bounded or unbounded source
>>>>>>>
>>>>>>> DataStream<MyType> boundedStream = env.continousSource(theSource);
>>>>>>>
>>>>>>> // imo this would make sense if the BoundedDataStream provides
>>>>> additional features unavailable for continous mode
>>>>>>> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
>>>>>>>
>>>>>>>
>>>>>>> Best,
>>>>>>>
>>>>>>> Dawid
>>>>>>>
>>>>>>>
>>>>>>> On 04/12/2019 11:25, Stephan Ewen wrote:
>>>>>>>
>>>>>>> Thanks, Becket, for updating this.
>>>>>>>
>>>>>>> I agree with moving the aspects you mentioned into separate FLIPs
>> -
>>>> this
>>>>>>> one way becoming unwieldy in size.
>>>>>>>
>>>>>>> +1 to the FLIP in its current state. Its a very detailed write-up,
>>>>> nicely
>>>>>>> done!
>>>>>>>
>>>>>>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <be...@gmail.com>
>> <
>>>>> becket.qin@gmail.com> wrote:
>>>>>>>
>>>>>>> Hi all,
>>>>>>>
>>>>>>> Sorry for the long belated update. I have updated FLIP-27 wiki
>> page
>>>> with
>>>>>>> the latest proposals. Some noticeable changes include:
>>>>>>> 1. A new generic communication mechanism between SplitEnumerator
>> and
>>>>>>> SourceReader.
>>>>>>> 2. Some detail API method signature changes.
>>>>>>>
>>>>>>> We left a few things out of this FLIP and will address them in
>>>> separate
>>>>>>> FLIPs. Including:
>>>>>>> 1. Per split event time.
>>>>>>> 2. Event time alignment.
>>>>>>> 3. Fine grained failover for SplitEnumerator failure.
>>>>>>>
>>>>>>> Please let us know if you have any question.
>>>>>>>
>>>>>>> Thanks,
>>>>>>>
>>>>>>> Jiangjie (Becket) Qin
>>>>>>>
>>>>>>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> <
>>>>> sewen@apache.org> wrote:
>>>>>>>
>>>>>>> Hi  Łukasz!
>>>>>>>
>>>>>>> Becket and me are working hard on figuring out the last details
>> and
>>>>>>> implementing the first PoC. We would update the FLIP hopefully
>> next
>>>>> week.
>>>>>>> There is a fair chance that a first version of this will be in
>> 1.10,
>>>> but
>>>>>>> I
>>>>>>>
>>>>>>> think it will take another release to battle test it and migrate
>> the
>>>>>>> connectors.
>>>>>>>
>>>>>>> Best,
>>>>>>> Stephan
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <ljd@touk.pl
>>> <
>>>>> ljd@touk.pl>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hi,
>>>>>>>
>>>>>>> This proposal looks very promising for us. Do you have any plans
>> in
>>>>>>> which
>>>>>>>
>>>>>>> Flink release it is going to be released? We are thinking on
>> using a
>>>>>>> Data
>>>>>>>
>>>>>>> Set API for our future use cases but on the other hand Data Set
>> API
>>> is
>>>>>>> going to be deprecated so using proposed bounded data streams
>>> solution
>>>>>>> could be more viable in the long term.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Łukasz
>>>>>>>
>>>>>>> On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> <
>>>>> thomas.weise@gmail.com> wrote:
>>>>>>> Thanks for putting together this proposal!
>>>>>>>
>>>>>>> I see that the "Per Split Event Time" and "Event Time Alignment"
>>>>>>>
>>>>>>> sections
>>>>>>>
>>>>>>> are still TBD.
>>>>>>>
>>>>>>> It would probably be good to flesh those out a bit before
>> proceeding
>>>>>>> too
>>>>>>>
>>>>>>> far
>>>>>>>
>>>>>>> as the event time alignment will probably influence the
>> interaction
>>>>>>> with
>>>>>>>
>>>>>>> the split reader, specifically ReaderStatus
>> emitNext(SourceOutput<E>
>>>>>>> output).
>>>>>>>
>>>>>>> We currently have only one implementation for event time alignment
>>> in
>>>>>>> the
>>>>>>>
>>>>>>> Kinesis consumer. The synchronization in that case takes place as
>>> the
>>>>>>> last
>>>>>>>
>>>>>>> step before records are emitted downstream (RecordEmitter). With
>> the
>>>>>>> currently proposed interfaces, the equivalent can be implemented
>> in
>>>>>>> the
>>>>>>>
>>>>>>> reader loop, although note that in the Kinesis consumer the per
>>> shard
>>>>>>> threads push records.
>>>>>>>
>>>>>>> Synchronization has not been implemented for the Kafka consumer
>> yet.
>>>>>>> https://issues.apache.org/jira/browse/FLINK-12675
>>>>>>>
>>>>>>> When I looked at it, I realized that the implementation will look
>>>>>>>
>>>>>>> quite
>>>>>>>
>>>>>>> different
>>>>>>> from Kinesis because it needs to take place in the pull part,
>> where
>>>>>>> records
>>>>>>>
>>>>>>> are taken from the Kafka client. Due to the multiplexing it cannot
>>> be
>>>>>>> done
>>>>>>>
>>>>>>> by blocking the split thread like it currently works for Kinesis.
>>>>>>>
>>>>>>> Reading
>>>>>>>
>>>>>>> from individual Kafka partitions needs to be controlled via
>>>>>>>
>>>>>>> pause/resume
>>>>>>>
>>>>>>> on the Kafka client.
>>>>>>>
>>>>>>> To take on that responsibility the split thread would need to be
>>>>>>>
>>>>>>> aware
>>>>>>>
>>>>>>> of
>>>>>>>
>>>>>>> the
>>>>>>> watermarks or at least whether it should or should not continue to
>>>>>>>
>>>>>>> consume
>>>>>>>
>>>>>>> a given split and this may require a different SourceReader or
>>>>>>>
>>>>>>> SourceOutput
>>>>>>>
>>>>>>> interface.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Thomas
>>>>>>>
>>>>>>>
>>>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> <
>>>>> mmyy1110@gmail.com> wrote:
>>>>>>>
>>>>>>> Hi Stephan,
>>>>>>>
>>>>>>> Thank you for feedback!
>>>>>>> Will take a look at your branch before public discussing.
>>>>>>>
>>>>>>>
>>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org>
>> <
>>>>> sewen@apache.org>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hi Biao!
>>>>>>>
>>>>>>> Thanks for reviving this. I would like to join this discussion,
>>>>>>>
>>>>>>> but
>>>>>>>
>>>>>>> am
>>>>>>>
>>>>>>> quite occupied with the 1.9 release, so can we maybe pause this
>>>>>>>
>>>>>>> discussion
>>>>>>>
>>>>>>> for a week or so?
>>>>>>>
>>>>>>> In the meantime I can share some suggestion based on prior
>>>>>>>
>>>>>>> experiments:
>>>>>>>
>>>>>>> How to do watermarks / timestamp extractors in a simpler and more
>>>>>>>
>>>>>>> flexible
>>>>>>>
>>>>>>> way. I think that part is quite promising should be part of the
>>>>>>>
>>>>>>> new
>>>>>>>
>>>>>>> source
>>>>>>>
>>>>>>> interface.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>>>>>>>
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>>>>>>> Some experiments on how to build the source reader and its
>>>>>>>
>>>>>>> library
>>>>>>>
>>>>>>> for
>>>>>>>
>>>>>>> common threading/split patterns:
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>>>>>>> Best,
>>>>>>> Stephan
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> <
>>>>> mmyy1110@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hi devs,
>>>>>>>
>>>>>>> Since 1.9 is nearly released, I think we could get back to
>>>>>>>
>>>>>>> FLIP-27.
>>>>>>>
>>>>>>> I
>>>>>>>
>>>>>>> believe it should be included in 1.10.
>>>>>>>
>>>>>>> There are so many things mentioned in document of FLIP-27. [1] I
>>>>>>>
>>>>>>> think
>>>>>>>
>>>>>>> we'd better discuss them separately. However the wiki is not a
>>>>>>>
>>>>>>> good
>>>>>>>
>>>>>>> place
>>>>>>>
>>>>>>> to discuss. I wrote google doc about SplitReader API which
>>>>>>>
>>>>>>> misses
>>>>>>>
>>>>>>> some
>>>>>>>
>>>>>>> details in the document. [2]
>>>>>>>
>>>>>>> 1.
>>>>>>>
>>>>>>>
>>>>>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>>>>>>> 2.
>>>>>>>
>>>>>>>
>>>>>>>
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> <
>>>>> mmyy1110@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hi Steven,
>>>>>>> Thank you for the feedback. Please take a look at the document
>>>>>>>
>>>>>>> FLIP-27
>>>>>>>
>>>>>>> <
>>>>>>>
>>>>>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>> which
>>>>>>>
>>>>>>> is updated recently. A lot of details of enumerator were added
>>>>>>>
>>>>>>> in
>>>>>>>
>>>>>>> this
>>>>>>>
>>>>>>> document. I think it would help.
>>>>>>>
>>>>>>> Steven Wu <st...@gmail.com> <st...@gmail.com>
>>> 于2019年3月28日周四
>>>>> 下午12:52写道:
>>>>>>>
>>>>>>> This proposal mentioned that SplitEnumerator might run on the
>>>>>>> JobManager or
>>>>>>> in a single task on a TaskManager.
>>>>>>>
>>>>>>> if enumerator is a single task on a taskmanager, then the job
>>>>>>>
>>>>>>> DAG
>>>>>>>
>>>>>>> can
>>>>>>>
>>>>>>> never
>>>>>>> been embarrassingly parallel anymore. That will nullify the
>>>>>>>
>>>>>>> leverage
>>>>>>>
>>>>>>> of
>>>>>>>
>>>>>>> fine-grained recovery for embarrassingly parallel jobs.
>>>>>>>
>>>>>>> It's not clear to me what's the implication of running
>>>>>>>
>>>>>>> enumerator
>>>>>>>
>>>>>>> on
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> jobmanager. So I will leave that out for now.
>>>>>>>
>>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> <
>>>>> mmyy1110@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hi Stephan & Piotrek,
>>>>>>>
>>>>>>> Thank you for feedback.
>>>>>>>
>>>>>>> It seems that there are a lot of things to do in community.
>>>>>>>
>>>>>>> I
>>>>>>>
>>>>>>> am
>>>>>>>
>>>>>>> just
>>>>>>>
>>>>>>> afraid that this discussion may be forgotten since there so
>>>>>>>
>>>>>>> many
>>>>>>>
>>>>>>> proposals
>>>>>>>
>>>>>>> recently.
>>>>>>> Anyway, wish to see the split topics soon :)
>>>>>>>
>>>>>>> Piotr Nowojski <pi...@da-platform.com> <pi...@da-platform.com>
>>>>> 于2019年1月24日周四
>>>>>>> 下午8:21写道:
>>>>>>>
>>>>>>> Hi Biao!
>>>>>>>
>>>>>>> This discussion was stalled because of preparations for
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> open
>>>>>>>
>>>>>>> sourcing
>>>>>>>
>>>>>>> & merging Blink. I think before creating the tickets we
>>>>>>>
>>>>>>> should
>>>>>>>
>>>>>>> split this
>>>>>>>
>>>>>>> discussion into topics/areas outlined by Stephan and
>>>>>>>
>>>>>>> create
>>>>>>>
>>>>>>> Flips
>>>>>>>
>>>>>>> for
>>>>>>>
>>>>>>> that.
>>>>>>>
>>>>>>> I think there is no chance for this to be completed in
>>>>>>>
>>>>>>> couple
>>>>>>>
>>>>>>> of
>>>>>>>
>>>>>>> remaining
>>>>>>>
>>>>>>> weeks/1 month before 1.8 feature freeze, however it would
>>>>>>>
>>>>>>> be
>>>>>>>
>>>>>>> good
>>>>>>>
>>>>>>> to aim
>>>>>>>
>>>>>>> with those changes for 1.9.
>>>>>>>
>>>>>>> Piotrek
>>>>>>>
>>>>>>>
>>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> <
>>>>> mmyy1110@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hi community,
>>>>>>> The summary of Stephan makes a lot sense to me. It is
>>>>>>>
>>>>>>> much
>>>>>>>
>>>>>>> clearer
>>>>>>>
>>>>>>> indeed
>>>>>>>
>>>>>>> after splitting the complex topic into small ones.
>>>>>>> I was wondering is there any detail plan for next step?
>>>>>>>
>>>>>>> If
>>>>>>>
>>>>>>> not,
>>>>>>>
>>>>>>> I
>>>>>>>
>>>>>>> would
>>>>>>>
>>>>>>> like to push this thing forward by creating some JIRA
>>>>>>>
>>>>>>> issues.
>>>>>>>
>>>>>>> Another question is that should version 1.8 include
>>>>>>>
>>>>>>> these
>>>>>>>
>>>>>>> features?
>>>>>>>
>>>>>>> Stephan Ewen <se...@apache.org> <se...@apache.org> 于2018年12月1日周六
>>>>> 上午4:20写道:
>>>>>>>
>>>>>>> Thanks everyone for the lively discussion. Let me try
>>>>>>>
>>>>>>> to
>>>>>>>
>>>>>>> summarize
>>>>>>>
>>>>>>> where I
>>>>>>>
>>>>>>> see convergence in the discussion and open issues.
>>>>>>> I'll try to group this by design aspect of the source.
>>>>>>>
>>>>>>> Please
>>>>>>>
>>>>>>> let me
>>>>>>>
>>>>>>> know
>>>>>>>
>>>>>>> if I got things wrong or missed something crucial here.
>>>>>>>
>>>>>>> For issues 1-3, if the below reflects the state of the
>>>>>>>
>>>>>>> discussion, I
>>>>>>>
>>>>>>> would
>>>>>>>
>>>>>>> try and update the FLIP in the next days.
>>>>>>> For the remaining ones we need more discussion.
>>>>>>>
>>>>>>> I would suggest to fork each of these aspects into a
>>>>>>>
>>>>>>> separate
>>>>>>>
>>>>>>> mail
>>>>>>>
>>>>>>> thread,
>>>>>>>
>>>>>>> or will loose sight of the individual aspects.
>>>>>>>
>>>>>>> *(1) Separation of Split Enumerator and Split Reader*
>>>>>>>
>>>>>>>  - All seem to agree this is a good thing
>>>>>>>  - Split Enumerator could in the end live on JobManager
>>>>>>>
>>>>>>> (and
>>>>>>>
>>>>>>> assign
>>>>>>>
>>>>>>> splits
>>>>>>>
>>>>>>> via RPC) or in a task (and assign splits via data
>>>>>>>
>>>>>>> streams)
>>>>>>>
>>>>>>>  - this discussion is orthogonal and should come later,
>>>>>>>
>>>>>>> when
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> interface
>>>>>>>
>>>>>>> is agreed upon.
>>>>>>>
>>>>>>> *(2) Split Readers for one or more splits*
>>>>>>>
>>>>>>>  - Discussion seems to agree that we need to support
>>>>>>>
>>>>>>> one
>>>>>>>
>>>>>>> reader
>>>>>>>
>>>>>>> that
>>>>>>>
>>>>>>> possibly handles multiple splits concurrently.
>>>>>>>  - The requirement comes from sources where one
>>>>>>>
>>>>>>> poll()-style
>>>>>>>
>>>>>>> call
>>>>>>>
>>>>>>> fetches
>>>>>>>
>>>>>>> data from different splits / partitions
>>>>>>>    --> example sources that require that would be for
>>>>>>>
>>>>>>> example
>>>>>>>
>>>>>>> Kafka,
>>>>>>>
>>>>>>> Pravega, Pulsar
>>>>>>>
>>>>>>>  - Could have one split reader per source, or multiple
>>>>>>>
>>>>>>> split
>>>>>>>
>>>>>>> readers
>>>>>>>
>>>>>>> that
>>>>>>>
>>>>>>> share the "poll()" function
>>>>>>>  - To not make it too complicated, we can start with
>>>>>>>
>>>>>>> thinking
>>>>>>>
>>>>>>> about
>>>>>>>
>>>>>>> one
>>>>>>>
>>>>>>> split reader for all splits initially and see if that
>>>>>>>
>>>>>>> covers
>>>>>>>
>>>>>>> all
>>>>>>>
>>>>>>> requirements
>>>>>>>
>>>>>>> *(3) Threading model of the Split Reader*
>>>>>>>
>>>>>>>  - Most active part of the discussion ;-)
>>>>>>>
>>>>>>>  - A non-blocking way for Flink's task code to interact
>>>>>>>
>>>>>>> with
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> source
>>>>>>>
>>>>>>> is
>>>>>>>
>>>>>>> needed in order to a task runtime code based on a
>>>>>>> single-threaded/actor-style task design
>>>>>>>    --> I personally am a big proponent of that, it will
>>>>>>>
>>>>>>> help
>>>>>>>
>>>>>>> with
>>>>>>>
>>>>>>> well-behaved checkpoints, efficiency, and simpler yet
>>>>>>>
>>>>>>> more
>>>>>>>
>>>>>>> robust
>>>>>>>
>>>>>>> runtime
>>>>>>>
>>>>>>> code
>>>>>>>
>>>>>>>  - Users care about simple abstraction, so as a
>>>>>>>
>>>>>>> subclass
>>>>>>>
>>>>>>> of
>>>>>>>
>>>>>>> SplitReader
>>>>>>>
>>>>>>> (non-blocking / async) we need to have a
>>>>>>>
>>>>>>> BlockingSplitReader
>>>>>>>
>>>>>>> which
>>>>>>>
>>>>>>> will
>>>>>>>
>>>>>>> form the basis of most source implementations.
>>>>>>>
>>>>>>> BlockingSplitReader
>>>>>>>
>>>>>>> lets
>>>>>>>
>>>>>>> users do blocking simple poll() calls.
>>>>>>>  - The BlockingSplitReader would spawn a thread (or
>>>>>>>
>>>>>>> more)
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> thread(s) can make blocking calls and hand over data
>>>>>>>
>>>>>>> buffers
>>>>>>>
>>>>>>> via
>>>>>>>
>>>>>>> a
>>>>>>>
>>>>>>> blocking
>>>>>>>
>>>>>>> queue
>>>>>>>  - This should allow us to cover both, a fully async
>>>>>>>
>>>>>>> runtime,
>>>>>>>
>>>>>>> and a
>>>>>>>
>>>>>>> simple
>>>>>>>
>>>>>>> blocking interface for users.
>>>>>>>  - This is actually very similar to how the Kafka
>>>>>>>
>>>>>>> connectors
>>>>>>>
>>>>>>> work.
>>>>>>>
>>>>>>> Kafka
>>>>>>>
>>>>>>> 9+ with one thread, Kafka 8 with multiple threads
>>>>>>>
>>>>>>>  - On the base SplitReader (the async one), the
>>>>>>>
>>>>>>> non-blocking
>>>>>>>
>>>>>>> method
>>>>>>>
>>>>>>> that
>>>>>>>
>>>>>>> gets the next chunk of data would signal data
>>>>>>>
>>>>>>> availability
>>>>>>>
>>>>>>> via
>>>>>>>
>>>>>>> a
>>>>>>>
>>>>>>> CompletableFuture, because that gives the best
>>>>>>>
>>>>>>> flexibility
>>>>>>>
>>>>>>> (can
>>>>>>>
>>>>>>> await
>>>>>>>
>>>>>>> completion or register notification handlers).
>>>>>>>  - The source task would register a "thenHandle()" (or
>>>>>>>
>>>>>>> similar)
>>>>>>>
>>>>>>> on the
>>>>>>>
>>>>>>> future to put a "take next data" task into the
>>>>>>>
>>>>>>> actor-style
>>>>>>>
>>>>>>> mailbox
>>>>>>>
>>>>>>> *(4) Split Enumeration and Assignment*
>>>>>>>
>>>>>>>  - Splits may be generated lazily, both in cases where
>>>>>>>
>>>>>>> there
>>>>>>>
>>>>>>> is a
>>>>>>>
>>>>>>> limited
>>>>>>>
>>>>>>> number of splits (but very many), or splits are
>>>>>>>
>>>>>>> discovered
>>>>>>>
>>>>>>> over
>>>>>>>
>>>>>>> time
>>>>>>>
>>>>>>>  - Assignment should also be lazy, to get better load
>>>>>>>
>>>>>>> balancing
>>>>>>>
>>>>>>>  - Assignment needs support locality preferences
>>>>>>>
>>>>>>>  - Possible design based on discussion so far:
>>>>>>>
>>>>>>>    --> SplitReader has a method "addSplits(SplitT...)"
>>>>>>>
>>>>>>> to
>>>>>>>
>>>>>>> add
>>>>>>>
>>>>>>> one or
>>>>>>>
>>>>>>> more
>>>>>>>
>>>>>>> splits. Some split readers might assume they have only
>>>>>>>
>>>>>>> one
>>>>>>>
>>>>>>> split
>>>>>>>
>>>>>>> ever,
>>>>>>>
>>>>>>> concurrently, others assume multiple splits. (Note:
>>>>>>>
>>>>>>> idea
>>>>>>>
>>>>>>> behind
>>>>>>>
>>>>>>> being
>>>>>>>
>>>>>>> able
>>>>>>>
>>>>>>> to add multiple splits at the same time is to ease
>>>>>>>
>>>>>>> startup
>>>>>>>
>>>>>>> where
>>>>>>>
>>>>>>> multiple
>>>>>>>
>>>>>>> splits may be assigned instantly.)
>>>>>>>    --> SplitReader has a context object on which it can
>>>>>>>
>>>>>>> call
>>>>>>>
>>>>>>> indicate
>>>>>>>
>>>>>>> when
>>>>>>>
>>>>>>> splits are completed. The enumerator gets that
>>>>>>>
>>>>>>> notification and
>>>>>>>
>>>>>>> can
>>>>>>>
>>>>>>> use
>>>>>>>
>>>>>>> to
>>>>>>>
>>>>>>> decide when to assign new splits. This should help both
>>>>>>>
>>>>>>> in
>>>>>>>
>>>>>>> cases
>>>>>>>
>>>>>>> of
>>>>>>>
>>>>>>> sources
>>>>>>>
>>>>>>> that take splits lazily (file readers) and in case the
>>>>>>>
>>>>>>> source
>>>>>>>
>>>>>>> needs to
>>>>>>>
>>>>>>> preserve a partial order between splits (Kinesis,
>>>>>>>
>>>>>>> Pravega,
>>>>>>>
>>>>>>> Pulsar may
>>>>>>>
>>>>>>> need
>>>>>>>
>>>>>>> that).
>>>>>>>    --> SplitEnumerator gets notification when
>>>>>>>
>>>>>>> SplitReaders
>>>>>>>
>>>>>>> start
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>> when
>>>>>>>
>>>>>>> they finish splits. They can decide at that moment to
>>>>>>>
>>>>>>> push
>>>>>>>
>>>>>>> more
>>>>>>>
>>>>>>> splits
>>>>>>>
>>>>>>> to
>>>>>>>
>>>>>>> that reader
>>>>>>>    --> The SplitEnumerator should probably be aware of
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> source
>>>>>>>
>>>>>>> parallelism, to build its initial distribution.
>>>>>>>
>>>>>>>  - Open question: Should the source expose something
>>>>>>>
>>>>>>> like
>>>>>>>
>>>>>>> "host
>>>>>>>
>>>>>>> preferences", so that yarn/mesos/k8s can take this into
>>>>>>>
>>>>>>> account
>>>>>>>
>>>>>>> when
>>>>>>>
>>>>>>> selecting a node to start a TM on?
>>>>>>>
>>>>>>> *(5) Watermarks and event time alignment*
>>>>>>>
>>>>>>>  - Watermark generation, as well as idleness, needs to
>>>>>>>
>>>>>>> be
>>>>>>>
>>>>>>> per
>>>>>>>
>>>>>>> split
>>>>>>>
>>>>>>> (like
>>>>>>>
>>>>>>> currently in the Kafka Source, per partition)
>>>>>>>  - It is desirable to support optional
>>>>>>>
>>>>>>> event-time-alignment,
>>>>>>>
>>>>>>> meaning
>>>>>>>
>>>>>>> that
>>>>>>>
>>>>>>> splits that are ahead are back-pressured or temporarily
>>>>>>>
>>>>>>> unsubscribed
>>>>>>>
>>>>>>>  - I think i would be desirable to encapsulate
>>>>>>>
>>>>>>> watermark
>>>>>>>
>>>>>>> generation
>>>>>>>
>>>>>>> logic
>>>>>>>
>>>>>>> in watermark generators, for a separation of concerns.
>>>>>>>
>>>>>>> The
>>>>>>>
>>>>>>> watermark
>>>>>>>
>>>>>>> generators should run per split.
>>>>>>>  - Using watermark generators would also help with
>>>>>>>
>>>>>>> another
>>>>>>>
>>>>>>> problem of
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> suggested interface, namely supporting non-periodic
>>>>>>>
>>>>>>> watermarks
>>>>>>>
>>>>>>> efficiently.
>>>>>>>
>>>>>>>  - Need a way to "dispatch" next record to different
>>>>>>>
>>>>>>> watermark
>>>>>>>
>>>>>>> generators
>>>>>>>
>>>>>>>  - Need a way to tell SplitReader to "suspend" a split
>>>>>>>
>>>>>>> until a
>>>>>>>
>>>>>>> certain
>>>>>>>
>>>>>>> watermark is reached (event time backpressure)
>>>>>>>  - This would in fact be not needed (and thus simpler)
>>>>>>>
>>>>>>> if
>>>>>>>
>>>>>>> we
>>>>>>>
>>>>>>> had
>>>>>>>
>>>>>>> a
>>>>>>>
>>>>>>> SplitReader per split and may be a reason to re-open
>>>>>>>
>>>>>>> that
>>>>>>>
>>>>>>> discussion
>>>>>>>
>>>>>>> *(6) Watermarks across splits and in the Split
>>>>>>>
>>>>>>> Enumerator*
>>>>>>>
>>>>>>>  - The split enumerator may need some watermark
>>>>>>>
>>>>>>> awareness,
>>>>>>>
>>>>>>> which
>>>>>>>
>>>>>>> should
>>>>>>>
>>>>>>> be
>>>>>>>
>>>>>>> purely based on split metadata (like create timestamp
>>>>>>>
>>>>>>> of
>>>>>>>
>>>>>>> file
>>>>>>>
>>>>>>> splits)
>>>>>>>
>>>>>>>  - If there are still more splits with overlapping
>>>>>>>
>>>>>>> event
>>>>>>>
>>>>>>> time
>>>>>>>
>>>>>>> range
>>>>>>>
>>>>>>> for
>>>>>>>
>>>>>>> a
>>>>>>>
>>>>>>> split reader, then that split reader should not advance
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> watermark
>>>>>>>
>>>>>>> within the split beyond the overlap boundary. Otherwise
>>>>>>>
>>>>>>> future
>>>>>>>
>>>>>>> splits
>>>>>>>
>>>>>>> will
>>>>>>>
>>>>>>> produce late data.
>>>>>>>
>>>>>>>  - One way to approach this could be that the split
>>>>>>>
>>>>>>> enumerator
>>>>>>>
>>>>>>> may
>>>>>>>
>>>>>>> send
>>>>>>>
>>>>>>> watermarks to the readers, and the readers cannot emit
>>>>>>>
>>>>>>> watermarks
>>>>>>>
>>>>>>> beyond
>>>>>>>
>>>>>>> that received watermark.
>>>>>>>  - Many split enumerators would simply immediately send
>>>>>>>
>>>>>>> Long.MAX
>>>>>>>
>>>>>>> out
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>> leave the progress purely to the split readers.
>>>>>>>
>>>>>>>  - For event-time alignment / split back pressure, this
>>>>>>>
>>>>>>> begs
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> question
>>>>>>>
>>>>>>> how we can avoid deadlocks that may arise when splits
>>>>>>>
>>>>>>> are
>>>>>>>
>>>>>>> suspended
>>>>>>>
>>>>>>> for
>>>>>>>
>>>>>>> event time back pressure,
>>>>>>>
>>>>>>> *(7) Batch and streaming Unification*
>>>>>>>
>>>>>>>  - Functionality wise, the above design should support
>>>>>>>
>>>>>>> both
>>>>>>>
>>>>>>>  - Batch often (mostly) does not care about reading "in
>>>>>>>
>>>>>>> order"
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>> generating watermarks
>>>>>>>    --> Might use different enumerator logic that is
>>>>>>>
>>>>>>> more
>>>>>>>
>>>>>>> locality
>>>>>>>
>>>>>>> aware
>>>>>>>
>>>>>>> and ignores event time order
>>>>>>>    --> Does not generate watermarks
>>>>>>>  - Would be great if bounded sources could be
>>>>>>>
>>>>>>> identified
>>>>>>>
>>>>>>> at
>>>>>>>
>>>>>>> compile
>>>>>>>
>>>>>>> time,
>>>>>>>
>>>>>>> so that "env.addBoundedSource(...)" is type safe and
>>>>>>>
>>>>>>> can
>>>>>>>
>>>>>>> return a
>>>>>>>
>>>>>>> "BoundedDataStream".
>>>>>>>  - Possible to defer this discussion until later
>>>>>>>
>>>>>>> *Miscellaneous Comments*
>>>>>>>
>>>>>>>  - Should the source have a TypeInformation for the
>>>>>>>
>>>>>>> produced
>>>>>>>
>>>>>>> type,
>>>>>>>
>>>>>>> instead
>>>>>>>
>>>>>>> of a serializer? We need a type information in the
>>>>>>>
>>>>>>> stream
>>>>>>>
>>>>>>> anyways, and
>>>>>>>
>>>>>>> can
>>>>>>>
>>>>>>> derive the serializer from that. Plus, creating the
>>>>>>>
>>>>>>> serializer
>>>>>>>
>>>>>>> should
>>>>>>>
>>>>>>> respect the ExecutionConfig.
>>>>>>>
>>>>>>>  - The TypeSerializer interface is very powerful but
>>>>>>>
>>>>>>> also
>>>>>>>
>>>>>>> not
>>>>>>>
>>>>>>> easy to
>>>>>>>
>>>>>>> implement. Its purpose is to handle data super
>>>>>>>
>>>>>>> efficiently,
>>>>>>>
>>>>>>> support
>>>>>>>
>>>>>>> flexible ways of evolution, etc.
>>>>>>>  For metadata I would suggest to look at the
>>>>>>>
>>>>>>> SimpleVersionedSerializer
>>>>>>>
>>>>>>> instead, which is used for example for checkpoint
>>>>>>>
>>>>>>> master
>>>>>>>
>>>>>>> hooks,
>>>>>>>
>>>>>>> or for
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> streaming file sink. I think that is is a good match
>>>>>>>
>>>>>>> for
>>>>>>>
>>>>>>> cases
>>>>>>>
>>>>>>> where
>>>>>>>
>>>>>>> we
>>>>>>>
>>>>>>> do
>>>>>>>
>>>>>>> not need more than ser/deser (no copy, etc.) and don't
>>>>>>>
>>>>>>> need to
>>>>>>>
>>>>>>> push
>>>>>>>
>>>>>>> versioning out of the serialization paths for best
>>>>>>>
>>>>>>> performance
>>>>>>>
>>>>>>> (as in
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> TypeSerializer)
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>>>>> k.kloudas@data-artisans.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>
>>>>>>> Hi Biao,
>>>>>>>
>>>>>>> Thanks for the answer!
>>>>>>>
>>>>>>> So given the multi-threaded readers, now we have as
>>>>>>>
>>>>>>> open
>>>>>>>
>>>>>>> questions:
>>>>>>>
>>>>>>> 1) How do we let the checkpoints pass through our
>>>>>>>
>>>>>>> multi-threaded
>>>>>>>
>>>>>>> reader
>>>>>>>
>>>>>>> operator?
>>>>>>>
>>>>>>> 2) Do we have separate reader and source operators or
>>>>>>>
>>>>>>> not? In
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> strategy
>>>>>>>
>>>>>>> that has a separate source, the source operator has a
>>>>>>>
>>>>>>> parallelism of
>>>>>>>
>>>>>>> 1
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>> is responsible for split recovery only.
>>>>>>>
>>>>>>> For the first one, given also the constraints
>>>>>>>
>>>>>>> (blocking,
>>>>>>>
>>>>>>> finite
>>>>>>>
>>>>>>> queues,
>>>>>>>
>>>>>>> etc), I do not have an answer yet.
>>>>>>>
>>>>>>> For the 2nd, I think that we should go with separate
>>>>>>>
>>>>>>> operators
>>>>>>>
>>>>>>> for
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> source and the readers, for the following reasons:
>>>>>>>
>>>>>>> 1) This is more aligned with a potential future
>>>>>>>
>>>>>>> improvement
>>>>>>>
>>>>>>> where the
>>>>>>>
>>>>>>> split
>>>>>>>
>>>>>>> discovery becomes a responsibility of the JobManager
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>> readers are
>>>>>>>
>>>>>>> pooling more work from the JM.
>>>>>>>
>>>>>>> 2) The source is going to be the "single point of
>>>>>>>
>>>>>>> truth".
>>>>>>>
>>>>>>> It
>>>>>>>
>>>>>>> will
>>>>>>>
>>>>>>> know
>>>>>>>
>>>>>>> what
>>>>>>>
>>>>>>> has been processed and what not. If the source and the
>>>>>>>
>>>>>>> readers
>>>>>>>
>>>>>>> are a
>>>>>>>
>>>>>>> single
>>>>>>>
>>>>>>> operator with parallelism > 1, or in general, if the
>>>>>>>
>>>>>>> split
>>>>>>>
>>>>>>> discovery
>>>>>>>
>>>>>>> is
>>>>>>>
>>>>>>> done by each task individually, then:
>>>>>>>   i) we have to have a deterministic scheme for each
>>>>>>>
>>>>>>> reader to
>>>>>>>
>>>>>>> assign
>>>>>>>
>>>>>>> splits to itself (e.g. mod subtaskId). This is not
>>>>>>>
>>>>>>> necessarily
>>>>>>>
>>>>>>> trivial
>>>>>>>
>>>>>>> for
>>>>>>>
>>>>>>> all sources.
>>>>>>>   ii) each reader would have to keep a copy of all its
>>>>>>>
>>>>>>> processed
>>>>>>>
>>>>>>> slpits
>>>>>>>
>>>>>>>   iii) the state has to be a union state with a
>>>>>>>
>>>>>>> non-trivial
>>>>>>>
>>>>>>> merging
>>>>>>>
>>>>>>> logic
>>>>>>>
>>>>>>> in order to support rescaling.
>>>>>>>
>>>>>>> Two additional points that you raised above:
>>>>>>>
>>>>>>> i) The point that you raised that we need to keep all
>>>>>>>
>>>>>>> splits
>>>>>>>
>>>>>>> (processed
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>> not-processed) I think is a bit of a strong
>>>>>>>
>>>>>>> requirement.
>>>>>>>
>>>>>>> This
>>>>>>>
>>>>>>> would
>>>>>>>
>>>>>>> imply
>>>>>>>
>>>>>>> that for infinite sources the state will grow
>>>>>>>
>>>>>>> indefinitely.
>>>>>>>
>>>>>>> This is
>>>>>>>
>>>>>>> problem
>>>>>>>
>>>>>>> is even more pronounced if we do not have a single
>>>>>>>
>>>>>>> source
>>>>>>>
>>>>>>> that
>>>>>>>
>>>>>>> assigns
>>>>>>>
>>>>>>> splits to readers, as each reader will have its own
>>>>>>>
>>>>>>> copy
>>>>>>>
>>>>>>> of
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> state.
>>>>>>>
>>>>>>> ii) it is true that for finite sources we need to
>>>>>>>
>>>>>>> somehow
>>>>>>>
>>>>>>> not
>>>>>>>
>>>>>>> close
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> readers when the source/split discoverer finishes. The
>>>>>>> ContinuousFileReaderOperator has a work-around for
>>>>>>>
>>>>>>> that.
>>>>>>>
>>>>>>> It is
>>>>>>>
>>>>>>> not
>>>>>>>
>>>>>>> elegant,
>>>>>>>
>>>>>>> and checkpoints are not emitted after closing the
>>>>>>>
>>>>>>> source,
>>>>>>>
>>>>>>> but
>>>>>>>
>>>>>>> this, I
>>>>>>>
>>>>>>> believe, is a bigger problem which requires more
>>>>>>>
>>>>>>> changes
>>>>>>>
>>>>>>> than
>>>>>>>
>>>>>>> just
>>>>>>>
>>>>>>> refactoring the source interface.
>>>>>>>
>>>>>>> Cheers,
>>>>>>> Kostas
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>
>>>
>>> --
>>> Best, Jingsong Lee
>>>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi folks,

Thanks for the discussion, great feedback. Also thanks Dawid for the
explanation, it is much clearer now.

One thing that is indeed missing from the FLIP is how the boundedness is
passed to the Source implementation. So the API should be
Source#createEnumerator(Boundedness boundedness, SplitEnumeratorContext
context)
And we can probably remove the Source#supportBoundedness(Boundedness
boundedness) method.

Assuming we have that, we are essentially choosing from one of the
following two options:

Option 1:
// The source is continuous source, and only unbounded operations can be
performed.
DataStream<Type> datastream = env.continuousSource(someSource);

// The source is bounded source, both bounded and unbounded operations can
be performed.
BoundedDataStream<Type> boundedDataStream = env.boundedSource(someSource);

  - Pros:
       a) explicit boundary between bounded / unbounded streams, it is
quite simple and clear to the users.
  - Cons:
       a) For applications that do not involve bounded operations, they
still have to call different API to distinguish bounded / unbounded streams.
       b) No support for bounded stream to run in a streaming runtime
setting, i.e. scheduling and operators behaviors.


Option 2:
// The source is either bounded or unbounded, but only unbounded operations
could be performed on the returned DataStream.
DataStream<Type> dataStream = env.source(someSource);

// The source must be a bounded source, otherwise exception is thrown.
BoundedDataStream<Type> boundedDataStream =
env.boundedSource(boundedSource);

The pros and cons are exactly the opposite of option 1.
  - Pros:
       a) For applications that do not involve bounded operations, they
still have to call different API to distinguish bounded / unbounded streams.
       b) Support for bounded stream to run in a streaming runtime setting,
i.e. scheduling and operators behaviors.
  - Cons:
       a) Bounded / unbounded streams are kind of mixed, i.e. given a
DataStream, it is not clear whether it is bounded or not, unless you have
the access to its source.


If we only think from the Source API perspective, option 2 seems a better
choice because functionality wise it is a superset of option 1, at the cost
of some seemingly acceptable ambiguity in the DataStream API.
But if we look at the DataStream API as a whole, option 1 seems a clearer
choice. For example, some times a library may have to know whether a
certain task will finish or not. And it would be difficult to tell if the
input is a DataStream, unless additional information is provided all the
way from the Source. One possible solution is to have a *modified option 2*
which adds a method to the DataStream API to indicate boundedness, such as
getBoundedness(). It would solve the problem with a potential confusion of
what is difference between a DataStream with getBoundedness()=true and a
BoundedDataStream. But that seems not super difficult to explain.

So from API's perspective, I don't have a strong opinion between *option 1*
and *modified option 2. *I like the cleanness of option 1, but modified
option 2 would be more attractive if we have concrete use case for the
"Bounded stream with unbounded streaming runtime settings".

Re: Till

> Maybe this has already been asked before but I was wondering why the
> SourceReader interface has the method pollNext which hands the
> responsibility of outputting elements to the SourceReader implementation?
> Has this been done for backwards compatibility reasons with the old source
> interface? If not, then one could define a Collection<E> getNextRecords()
> method which returns the currently retrieved records and then the caller
> emits them outside of the SourceReader. That way the interface would not
> allow to implement an outputting loop where we never hand back control to
> the caller. At the moment, this contract can be easily broken and is only
> mentioned loosely in the JavaDocs.
>

The primary reason we handover the SourceOutput to the SourceReader is
because sometimes it is difficult for a SourceReader to emit one record at
a time. One example is some batched messaging systems which only have an
offset for the entire batch instead of individual messages in the batch. In
that case, returning one record at a time would leave the SourceReader in
an uncheckpointable state because they can only checkpoint at the batch
boundaries.

Thanks,

Jiangjie (Becket) Qin

On Tue, Dec 10, 2019 at 5:33 PM Till Rohrmann <tr...@apache.org> wrote:

> Hi everyone,
>
> thanks for drafting this FLIP. It reads very well.
>
> Concerning Dawid's proposal, I tend to agree. The boundedness could come
> from the source and tell the system how to treat the operator (scheduling
> wise). From a user's perspective it should be fine to get back a DataStream
> when calling env.source(boundedSource) if he does not need special
> operations defined on a BoundedDataStream. If he needs this, then one could
> use the method BoundedDataStream env.boundedSource(boundedSource).
>
> If possible, we could enforce the proper usage of env.boundedSource() by
> introducing a BoundedSource type so that one cannot pass an
> unbounded source to it. That way users would not be able to shoot
> themselves in the foot.
>
> Maybe this has already been asked before but I was wondering why the
> SourceReader interface has the method pollNext which hands the
> responsibility of outputting elements to the SourceReader implementation?
> Has this been done for backwards compatibility reasons with the old source
> interface? If not, then one could define a Collection<E> getNextRecords()
> method which returns the currently retrieved records and then the caller
> emits them outside of the SourceReader. That way the interface would not
> allow to implement an outputting loop where we never hand back control to
> the caller. At the moment, this contract can be easily broken and is only
> mentioned loosely in the JavaDocs.
>
> Cheers,
> Till
>
> On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <ji...@gmail.com>
> wrote:
>
> > Hi all,
> >
> > I think current design is good.
> >
> > My understanding is:
> >
> > For execution mode: bounded mode and continuous mode, It's totally
> > different. I don't think we have the ability to integrate the two models
> at
> > present. It's about scheduling, memory, algorithms, States, etc. we
> > shouldn't confuse them.
> >
> > For source capabilities: only bounded, only continuous, both bounded and
> > continuous.
> > I think Kafka is a source that can be ran both bounded
> > and continuous execution mode.
> > And Kafka with end offset should be ran both bounded
> > and continuous execution mode.  Using apache Beam with Flink runner, I
> used
> > to run a "bounded" Kafka in streaming mode. For our previous DataStream,
> it
> > is not necessarily required that the source cannot be bounded.
> >
> > So it is my thought for Dawid's question:
> > 1.pass a bounded source to continuousSource() +1
> > 2.pass a continuous source to boundedSource() -1, should throw exception.
> >
> > In StreamExecutionEnvironment, continuousSource and boundedSource define
> > the execution mode. It defines a clear boundary of execution mode.
> >
> > Best,
> > Jingsong Lee
> >
> > On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <im...@gmail.com> wrote:
> >
> > > I agree with Dawid's point that the boundedness information should come
> > > from the source itself (e.g. the end timestamp), not through
> > > env.boundedSouce()/continuousSource().
> > > I think if we want to support something like `env.source()` that derive
> > the
> > > execution mode from source, `supportsBoundedness(Boundedness)`
> > > method is not enough, because we don't know whether it is bounded or
> not.
> > >
> > > Best,
> > > Jark
> > >
> > >
> > > On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <dw...@apache.org>
> > > wrote:
> > >
> > > > One more thing. In the current proposal, with the
> > > > supportsBoundedness(Boundedness) method and the boundedness coming
> from
> > > > either continuousSource or boundedSource I could not find how this
> > > > information is fed back to the SplitEnumerator.
> > > >
> > > > Best,
> > > >
> > > > Dawid
> > > >
> > > > On 09/12/2019 13:52, Becket Qin wrote:
> > > > > Hi Dawid,
> > > > >
> > > > > Thanks for the comments. This actually brings another relevant
> > question
> > > > > about what does a "bounded source" imply. I actually had the same
> > > > > impression when I look at the Source API. Here is what I understand
> > > after
> > > > > some discussion with Stephan. The bounded source has the following
> > > > impacts.
> > > > >
> > > > > 1. API validity.
> > > > > - A bounded source generates a bounded stream so some operations
> that
> > > > only
> > > > > works for bounded records would be performed, e.g. sort.
> > > > > - To expose these bounded stream only APIs, there are two options:
> > > > >      a. Add them to the DataStream API and throw exception if a
> > method
> > > is
> > > > > called on an unbounded stream.
> > > > >      b. Create a BoundedDataStream class which is returned from
> > > > > env.boundedSource(), while DataStream is returned from
> > > > env.continousSource().
> > > > > Note that this cannot be done by having single
> env.source(theSource)
> > > even
> > > > > the Source has a getBoundedness() method.
> > > > >
> > > > > 2. Scheduling
> > > > > - A bounded source could be computed stage by stage without
> bringing
> > up
> > > > all
> > > > > the tasks at the same time.
> > > > >
> > > > > 3. Operator behaviors
> > > > > - A bounded source indicates the records are finite so some
> operators
> > > can
> > > > > wait until it receives all the records before it starts the
> > processing.
> > > > >
> > > > > In the above impact, only 1 is relevant to the API design. And the
> > > > current
> > > > > proposal in FLIP-27 is following 1.b.
> > > > >
> > > > > // boundedness depends of source property, imo this should always
> be
> > > > >> preferred
> > > > >>
> > > > >
> > > > > DataStream<MyType> stream = env.source(theSource);
> > > > >
> > > > >
> > > > > In your proposal, does DataStream have bounded stream only methods?
> > It
> > > > > looks it should have, otherwise passing a bounded Source to
> > > env.source()
> > > > > would be confusing. In that case, we will essentially do 1.a if an
> > > > > unbounded Source is created from env.source(unboundedSource).
> > > > >
> > > > > If we have the methods only supported for bounded streams in
> > > DataStream,
> > > > it
> > > > > seems a little weird to have a separate BoundedDataStream
> interface.
> > > > >
> > > > > Am I understand it correctly?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jiangjie (Becket) Qin
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> > > dwysakowicz@apache.org>
> > > > > wrote:
> > > > >
> > > > >> Hi all,
> > > > >>
> > > > >> Really well written proposal and very important one. I must admit
> I
> > > have
> > > > >> not understood all the intricacies of it yet.
> > > > >>
> > > > >> One question I have though is about where does the information
> about
> > > > >> boundedness come from. I think in most cases it is a property of
> the
> > > > >> source. As you described it might be e.g. end offset, a flag
> should
> > it
> > > > >> monitor new splits etc. I think it would be a really nice use case
> > to
> > > be
> > > > >> able to say:
> > > > >>
> > > > >> new KafkaSource().readUntil(long timestamp),
> > > > >>
> > > > >> which could work as an "end offset". Moreover I think all Bounded
> > > > sources
> > > > >> support continuous mode, but no intrinsically continuous source
> > > support
> > > > the
> > > > >> Bounded mode. If I understood the proposal correctly it suggest
> the
> > > > >> boundedness sort of "comes" from the outside of the source, from
> the
> > > > >> invokation of either boundedStream or continousSource.
> > > > >>
> > > > >> I am wondering if it would make sense to actually change the
> method
> > > > >>
> > > > >> boolean Source#supportsBoundedness(Boundedness)
> > > > >>
> > > > >> to
> > > > >>
> > > > >> Boundedness Source#getBoundedness().
> > > > >>
> > > > >> As for the methods #boundedSource, #continousSource, assuming the
> > > > >> boundedness is property of the source they do not affect how the
> > > > enumerator
> > > > >> works, but mostly how the dag is scheduled, right? I am not
> against
> > > > those
> > > > >> methods, but I think it is a very specific use case to actually
> > > override
> > > > >> the property of the source. In general I would expect users to
> only
> > > call
> > > > >> env.source(theSource), where the source tells if it is bounded or
> > > not. I
> > > > >> would suggest considering following set of methods:
> > > > >>
> > > > >> // boundedness depends of source property, imo this should always
> be
> > > > preferred
> > > > >>
> > > > >> DataStream<MyType> stream = env.source(theSource);
> > > > >>
> > > > >>
> > > > >> // always continous execution, whether bounded or unbounded source
> > > > >>
> > > > >> DataStream<MyType> boundedStream = env.continousSource(theSource);
> > > > >>
> > > > >> // imo this would make sense if the BoundedDataStream provides
> > > > additional features unavailable for continous mode
> > > > >> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
> > > > >>
> > > > >>
> > > > >> Best,
> > > > >>
> > > > >> Dawid
> > > > >>
> > > > >>
> > > > >> On 04/12/2019 11:25, Stephan Ewen wrote:
> > > > >>
> > > > >> Thanks, Becket, for updating this.
> > > > >>
> > > > >> I agree with moving the aspects you mentioned into separate FLIPs
> -
> > > this
> > > > >> one way becoming unwieldy in size.
> > > > >>
> > > > >> +1 to the FLIP in its current state. Its a very detailed write-up,
> > > > nicely
> > > > >> done!
> > > > >>
> > > > >> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <be...@gmail.com>
> <
> > > > becket.qin@gmail.com> wrote:
> > > > >>
> > > > >>
> > > > >> Hi all,
> > > > >>
> > > > >> Sorry for the long belated update. I have updated FLIP-27 wiki
> page
> > > with
> > > > >> the latest proposals. Some noticeable changes include:
> > > > >> 1. A new generic communication mechanism between SplitEnumerator
> and
> > > > >> SourceReader.
> > > > >> 2. Some detail API method signature changes.
> > > > >>
> > > > >> We left a few things out of this FLIP and will address them in
> > > separate
> > > > >> FLIPs. Including:
> > > > >> 1. Per split event time.
> > > > >> 2. Event time alignment.
> > > > >> 3. Fine grained failover for SplitEnumerator failure.
> > > > >>
> > > > >> Please let us know if you have any question.
> > > > >>
> > > > >> Thanks,
> > > > >>
> > > > >> Jiangjie (Becket) Qin
> > > > >>
> > > > >> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> <
> > > > sewen@apache.org> wrote:
> > > > >>
> > > > >>
> > > > >> Hi  Łukasz!
> > > > >>
> > > > >> Becket and me are working hard on figuring out the last details
> and
> > > > >> implementing the first PoC. We would update the FLIP hopefully
> next
> > > > week.
> > > > >>
> > > > >> There is a fair chance that a first version of this will be in
> 1.10,
> > > but
> > > > >>
> > > > >> I
> > > > >>
> > > > >> think it will take another release to battle test it and migrate
> the
> > > > >> connectors.
> > > > >>
> > > > >> Best,
> > > > >> Stephan
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > > >> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <ljd@touk.pl
> >
> > <
> > > > ljd@touk.pl>
> > > > >>
> > > > >> wrote:
> > > > >>
> > > > >> Hi,
> > > > >>
> > > > >> This proposal looks very promising for us. Do you have any plans
> in
> > > > >>
> > > > >> which
> > > > >>
> > > > >> Flink release it is going to be released? We are thinking on
> using a
> > > > >>
> > > > >> Data
> > > > >>
> > > > >> Set API for our future use cases but on the other hand Data Set
> API
> > is
> > > > >> going to be deprecated so using proposed bounded data streams
> > solution
> > > > >> could be more viable in the long term.
> > > > >>
> > > > >> Thanks,
> > > > >> Łukasz
> > > > >>
> > > > >> On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> <
> > > > thomas.weise@gmail.com> wrote:
> > > > >>
> > > > >> Thanks for putting together this proposal!
> > > > >>
> > > > >> I see that the "Per Split Event Time" and "Event Time Alignment"
> > > > >>
> > > > >> sections
> > > > >>
> > > > >> are still TBD.
> > > > >>
> > > > >> It would probably be good to flesh those out a bit before
> proceeding
> > > > >>
> > > > >> too
> > > > >>
> > > > >> far
> > > > >>
> > > > >> as the event time alignment will probably influence the
> interaction
> > > > >>
> > > > >> with
> > > > >>
> > > > >> the split reader, specifically ReaderStatus
> emitNext(SourceOutput<E>
> > > > >> output).
> > > > >>
> > > > >> We currently have only one implementation for event time alignment
> > in
> > > > >>
> > > > >> the
> > > > >>
> > > > >> Kinesis consumer. The synchronization in that case takes place as
> > the
> > > > >>
> > > > >> last
> > > > >>
> > > > >> step before records are emitted downstream (RecordEmitter). With
> the
> > > > >> currently proposed interfaces, the equivalent can be implemented
> in
> > > > >>
> > > > >> the
> > > > >>
> > > > >> reader loop, although note that in the Kinesis consumer the per
> > shard
> > > > >> threads push records.
> > > > >>
> > > > >> Synchronization has not been implemented for the Kafka consumer
> yet.
> > > > >> https://issues.apache.org/jira/browse/FLINK-12675
> > > > >>
> > > > >> When I looked at it, I realized that the implementation will look
> > > > >>
> > > > >> quite
> > > > >>
> > > > >> different
> > > > >> from Kinesis because it needs to take place in the pull part,
> where
> > > > >>
> > > > >> records
> > > > >>
> > > > >> are taken from the Kafka client. Due to the multiplexing it cannot
> > be
> > > > >>
> > > > >> done
> > > > >>
> > > > >> by blocking the split thread like it currently works for Kinesis.
> > > > >>
> > > > >> Reading
> > > > >>
> > > > >> from individual Kafka partitions needs to be controlled via
> > > > >>
> > > > >> pause/resume
> > > > >>
> > > > >> on the Kafka client.
> > > > >>
> > > > >> To take on that responsibility the split thread would need to be
> > > > >>
> > > > >> aware
> > > > >>
> > > > >> of
> > > > >>
> > > > >> the
> > > > >> watermarks or at least whether it should or should not continue to
> > > > >>
> > > > >> consume
> > > > >>
> > > > >> a given split and this may require a different SourceReader or
> > > > >>
> > > > >> SourceOutput
> > > > >>
> > > > >> interface.
> > > > >>
> > > > >> Thanks,
> > > > >> Thomas
> > > > >>
> > > > >>
> > > > >> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> <
> > > > mmyy1110@gmail.com> wrote:
> > > > >>
> > > > >>
> > > > >> Hi Stephan,
> > > > >>
> > > > >> Thank you for feedback!
> > > > >> Will take a look at your branch before public discussing.
> > > > >>
> > > > >>
> > > > >> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org>
> <
> > > > sewen@apache.org>
> > > > >>
> > > > >> wrote:
> > > > >>
> > > > >> Hi Biao!
> > > > >>
> > > > >> Thanks for reviving this. I would like to join this discussion,
> > > > >>
> > > > >> but
> > > > >>
> > > > >> am
> > > > >>
> > > > >> quite occupied with the 1.9 release, so can we maybe pause this
> > > > >>
> > > > >> discussion
> > > > >>
> > > > >> for a week or so?
> > > > >>
> > > > >> In the meantime I can share some suggestion based on prior
> > > > >>
> > > > >> experiments:
> > > > >>
> > > > >> How to do watermarks / timestamp extractors in a simpler and more
> > > > >>
> > > > >> flexible
> > > > >>
> > > > >> way. I think that part is quite promising should be part of the
> > > > >>
> > > > >> new
> > > > >>
> > > > >> source
> > > > >>
> > > > >> interface.
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > > > >>
> > > > >> Some experiments on how to build the source reader and its
> > > > >>
> > > > >> library
> > > > >>
> > > > >> for
> > > > >>
> > > > >> common threading/split patterns:
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > > > >>
> > > > >> Best,
> > > > >> Stephan
> > > > >>
> > > > >>
> > > > >> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> <
> > > > mmyy1110@gmail.com>
> > > > >>
> > > > >> wrote:
> > > > >>
> > > > >> Hi devs,
> > > > >>
> > > > >> Since 1.9 is nearly released, I think we could get back to
> > > > >>
> > > > >> FLIP-27.
> > > > >>
> > > > >> I
> > > > >>
> > > > >> believe it should be included in 1.10.
> > > > >>
> > > > >> There are so many things mentioned in document of FLIP-27. [1] I
> > > > >>
> > > > >> think
> > > > >>
> > > > >> we'd better discuss them separately. However the wiki is not a
> > > > >>
> > > > >> good
> > > > >>
> > > > >> place
> > > > >>
> > > > >> to discuss. I wrote google doc about SplitReader API which
> > > > >>
> > > > >> misses
> > > > >>
> > > > >> some
> > > > >>
> > > > >> details in the document. [2]
> > > > >>
> > > > >> 1.
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > > >>
> > > > >> 2.
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > > > >>
> > > > >> CC Stephan, Aljoscha, Piotrek, Becket
> > > > >>
> > > > >>
> > > > >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> <
> > > > mmyy1110@gmail.com>
> > > > >>
> > > > >> wrote:
> > > > >>
> > > > >> Hi Steven,
> > > > >> Thank you for the feedback. Please take a look at the document
> > > > >>
> > > > >> FLIP-27
> > > > >>
> > > > >> <
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > > >>
> > > > >> which
> > > > >>
> > > > >> is updated recently. A lot of details of enumerator were added
> > > > >>
> > > > >> in
> > > > >>
> > > > >> this
> > > > >>
> > > > >> document. I think it would help.
> > > > >>
> > > > >> Steven Wu <st...@gmail.com> <st...@gmail.com>
> > 于2019年3月28日周四
> > > > 下午12:52写道:
> > > > >>
> > > > >>
> > > > >> This proposal mentioned that SplitEnumerator might run on the
> > > > >> JobManager or
> > > > >> in a single task on a TaskManager.
> > > > >>
> > > > >> if enumerator is a single task on a taskmanager, then the job
> > > > >>
> > > > >> DAG
> > > > >>
> > > > >> can
> > > > >>
> > > > >> never
> > > > >> been embarrassingly parallel anymore. That will nullify the
> > > > >>
> > > > >> leverage
> > > > >>
> > > > >> of
> > > > >>
> > > > >> fine-grained recovery for embarrassingly parallel jobs.
> > > > >>
> > > > >> It's not clear to me what's the implication of running
> > > > >>
> > > > >> enumerator
> > > > >>
> > > > >> on
> > > > >>
> > > > >> the
> > > > >>
> > > > >> jobmanager. So I will leave that out for now.
> > > > >>
> > > > >> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> <
> > > > mmyy1110@gmail.com>
> > > > >>
> > > > >> wrote:
> > > > >>
> > > > >> Hi Stephan & Piotrek,
> > > > >>
> > > > >> Thank you for feedback.
> > > > >>
> > > > >> It seems that there are a lot of things to do in community.
> > > > >>
> > > > >> I
> > > > >>
> > > > >> am
> > > > >>
> > > > >> just
> > > > >>
> > > > >> afraid that this discussion may be forgotten since there so
> > > > >>
> > > > >> many
> > > > >>
> > > > >> proposals
> > > > >>
> > > > >> recently.
> > > > >> Anyway, wish to see the split topics soon :)
> > > > >>
> > > > >> Piotr Nowojski <pi...@da-platform.com> <pi...@da-platform.com>
> > > > 于2019年1月24日周四
> > > > >>
> > > > >> 下午8:21写道:
> > > > >>
> > > > >> Hi Biao!
> > > > >>
> > > > >> This discussion was stalled because of preparations for
> > > > >>
> > > > >> the
> > > > >>
> > > > >> open
> > > > >>
> > > > >> sourcing
> > > > >>
> > > > >> & merging Blink. I think before creating the tickets we
> > > > >>
> > > > >> should
> > > > >>
> > > > >> split this
> > > > >>
> > > > >> discussion into topics/areas outlined by Stephan and
> > > > >>
> > > > >> create
> > > > >>
> > > > >> Flips
> > > > >>
> > > > >> for
> > > > >>
> > > > >> that.
> > > > >>
> > > > >> I think there is no chance for this to be completed in
> > > > >>
> > > > >> couple
> > > > >>
> > > > >> of
> > > > >>
> > > > >> remaining
> > > > >>
> > > > >> weeks/1 month before 1.8 feature freeze, however it would
> > > > >>
> > > > >> be
> > > > >>
> > > > >> good
> > > > >>
> > > > >> to aim
> > > > >>
> > > > >> with those changes for 1.9.
> > > > >>
> > > > >> Piotrek
> > > > >>
> > > > >>
> > > > >> On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> <
> > > > mmyy1110@gmail.com>
> > > > >>
> > > > >> wrote:
> > > > >>
> > > > >> Hi community,
> > > > >> The summary of Stephan makes a lot sense to me. It is
> > > > >>
> > > > >> much
> > > > >>
> > > > >> clearer
> > > > >>
> > > > >> indeed
> > > > >>
> > > > >> after splitting the complex topic into small ones.
> > > > >> I was wondering is there any detail plan for next step?
> > > > >>
> > > > >> If
> > > > >>
> > > > >> not,
> > > > >>
> > > > >> I
> > > > >>
> > > > >> would
> > > > >>
> > > > >> like to push this thing forward by creating some JIRA
> > > > >>
> > > > >> issues.
> > > > >>
> > > > >> Another question is that should version 1.8 include
> > > > >>
> > > > >> these
> > > > >>
> > > > >> features?
> > > > >>
> > > > >> Stephan Ewen <se...@apache.org> <se...@apache.org> 于2018年12月1日周六
> > > > 上午4:20写道:
> > > > >>
> > > > >>
> > > > >> Thanks everyone for the lively discussion. Let me try
> > > > >>
> > > > >> to
> > > > >>
> > > > >> summarize
> > > > >>
> > > > >> where I
> > > > >>
> > > > >> see convergence in the discussion and open issues.
> > > > >> I'll try to group this by design aspect of the source.
> > > > >>
> > > > >> Please
> > > > >>
> > > > >> let me
> > > > >>
> > > > >> know
> > > > >>
> > > > >> if I got things wrong or missed something crucial here.
> > > > >>
> > > > >> For issues 1-3, if the below reflects the state of the
> > > > >>
> > > > >> discussion, I
> > > > >>
> > > > >> would
> > > > >>
> > > > >> try and update the FLIP in the next days.
> > > > >> For the remaining ones we need more discussion.
> > > > >>
> > > > >> I would suggest to fork each of these aspects into a
> > > > >>
> > > > >> separate
> > > > >>
> > > > >> mail
> > > > >>
> > > > >> thread,
> > > > >>
> > > > >> or will loose sight of the individual aspects.
> > > > >>
> > > > >> *(1) Separation of Split Enumerator and Split Reader*
> > > > >>
> > > > >>  - All seem to agree this is a good thing
> > > > >>  - Split Enumerator could in the end live on JobManager
> > > > >>
> > > > >> (and
> > > > >>
> > > > >> assign
> > > > >>
> > > > >> splits
> > > > >>
> > > > >> via RPC) or in a task (and assign splits via data
> > > > >>
> > > > >> streams)
> > > > >>
> > > > >>  - this discussion is orthogonal and should come later,
> > > > >>
> > > > >> when
> > > > >>
> > > > >> the
> > > > >>
> > > > >> interface
> > > > >>
> > > > >> is agreed upon.
> > > > >>
> > > > >> *(2) Split Readers for one or more splits*
> > > > >>
> > > > >>  - Discussion seems to agree that we need to support
> > > > >>
> > > > >> one
> > > > >>
> > > > >> reader
> > > > >>
> > > > >> that
> > > > >>
> > > > >> possibly handles multiple splits concurrently.
> > > > >>  - The requirement comes from sources where one
> > > > >>
> > > > >> poll()-style
> > > > >>
> > > > >> call
> > > > >>
> > > > >> fetches
> > > > >>
> > > > >> data from different splits / partitions
> > > > >>    --> example sources that require that would be for
> > > > >>
> > > > >> example
> > > > >>
> > > > >> Kafka,
> > > > >>
> > > > >> Pravega, Pulsar
> > > > >>
> > > > >>  - Could have one split reader per source, or multiple
> > > > >>
> > > > >> split
> > > > >>
> > > > >> readers
> > > > >>
> > > > >> that
> > > > >>
> > > > >> share the "poll()" function
> > > > >>  - To not make it too complicated, we can start with
> > > > >>
> > > > >> thinking
> > > > >>
> > > > >> about
> > > > >>
> > > > >> one
> > > > >>
> > > > >> split reader for all splits initially and see if that
> > > > >>
> > > > >> covers
> > > > >>
> > > > >> all
> > > > >>
> > > > >> requirements
> > > > >>
> > > > >> *(3) Threading model of the Split Reader*
> > > > >>
> > > > >>  - Most active part of the discussion ;-)
> > > > >>
> > > > >>  - A non-blocking way for Flink's task code to interact
> > > > >>
> > > > >> with
> > > > >>
> > > > >> the
> > > > >>
> > > > >> source
> > > > >>
> > > > >> is
> > > > >>
> > > > >> needed in order to a task runtime code based on a
> > > > >> single-threaded/actor-style task design
> > > > >>    --> I personally am a big proponent of that, it will
> > > > >>
> > > > >> help
> > > > >>
> > > > >> with
> > > > >>
> > > > >> well-behaved checkpoints, efficiency, and simpler yet
> > > > >>
> > > > >> more
> > > > >>
> > > > >> robust
> > > > >>
> > > > >> runtime
> > > > >>
> > > > >> code
> > > > >>
> > > > >>  - Users care about simple abstraction, so as a
> > > > >>
> > > > >> subclass
> > > > >>
> > > > >> of
> > > > >>
> > > > >> SplitReader
> > > > >>
> > > > >> (non-blocking / async) we need to have a
> > > > >>
> > > > >> BlockingSplitReader
> > > > >>
> > > > >> which
> > > > >>
> > > > >> will
> > > > >>
> > > > >> form the basis of most source implementations.
> > > > >>
> > > > >> BlockingSplitReader
> > > > >>
> > > > >> lets
> > > > >>
> > > > >> users do blocking simple poll() calls.
> > > > >>  - The BlockingSplitReader would spawn a thread (or
> > > > >>
> > > > >> more)
> > > > >>
> > > > >> and
> > > > >>
> > > > >> the
> > > > >>
> > > > >> thread(s) can make blocking calls and hand over data
> > > > >>
> > > > >> buffers
> > > > >>
> > > > >> via
> > > > >>
> > > > >> a
> > > > >>
> > > > >> blocking
> > > > >>
> > > > >> queue
> > > > >>  - This should allow us to cover both, a fully async
> > > > >>
> > > > >> runtime,
> > > > >>
> > > > >> and a
> > > > >>
> > > > >> simple
> > > > >>
> > > > >> blocking interface for users.
> > > > >>  - This is actually very similar to how the Kafka
> > > > >>
> > > > >> connectors
> > > > >>
> > > > >> work.
> > > > >>
> > > > >> Kafka
> > > > >>
> > > > >> 9+ with one thread, Kafka 8 with multiple threads
> > > > >>
> > > > >>  - On the base SplitReader (the async one), the
> > > > >>
> > > > >> non-blocking
> > > > >>
> > > > >> method
> > > > >>
> > > > >> that
> > > > >>
> > > > >> gets the next chunk of data would signal data
> > > > >>
> > > > >> availability
> > > > >>
> > > > >> via
> > > > >>
> > > > >> a
> > > > >>
> > > > >> CompletableFuture, because that gives the best
> > > > >>
> > > > >> flexibility
> > > > >>
> > > > >> (can
> > > > >>
> > > > >> await
> > > > >>
> > > > >> completion or register notification handlers).
> > > > >>  - The source task would register a "thenHandle()" (or
> > > > >>
> > > > >> similar)
> > > > >>
> > > > >> on the
> > > > >>
> > > > >> future to put a "take next data" task into the
> > > > >>
> > > > >> actor-style
> > > > >>
> > > > >> mailbox
> > > > >>
> > > > >> *(4) Split Enumeration and Assignment*
> > > > >>
> > > > >>  - Splits may be generated lazily, both in cases where
> > > > >>
> > > > >> there
> > > > >>
> > > > >> is a
> > > > >>
> > > > >> limited
> > > > >>
> > > > >> number of splits (but very many), or splits are
> > > > >>
> > > > >> discovered
> > > > >>
> > > > >> over
> > > > >>
> > > > >> time
> > > > >>
> > > > >>  - Assignment should also be lazy, to get better load
> > > > >>
> > > > >> balancing
> > > > >>
> > > > >>  - Assignment needs support locality preferences
> > > > >>
> > > > >>  - Possible design based on discussion so far:
> > > > >>
> > > > >>    --> SplitReader has a method "addSplits(SplitT...)"
> > > > >>
> > > > >> to
> > > > >>
> > > > >> add
> > > > >>
> > > > >> one or
> > > > >>
> > > > >> more
> > > > >>
> > > > >> splits. Some split readers might assume they have only
> > > > >>
> > > > >> one
> > > > >>
> > > > >> split
> > > > >>
> > > > >> ever,
> > > > >>
> > > > >> concurrently, others assume multiple splits. (Note:
> > > > >>
> > > > >> idea
> > > > >>
> > > > >> behind
> > > > >>
> > > > >> being
> > > > >>
> > > > >> able
> > > > >>
> > > > >> to add multiple splits at the same time is to ease
> > > > >>
> > > > >> startup
> > > > >>
> > > > >> where
> > > > >>
> > > > >> multiple
> > > > >>
> > > > >> splits may be assigned instantly.)
> > > > >>    --> SplitReader has a context object on which it can
> > > > >>
> > > > >> call
> > > > >>
> > > > >> indicate
> > > > >>
> > > > >> when
> > > > >>
> > > > >> splits are completed. The enumerator gets that
> > > > >>
> > > > >> notification and
> > > > >>
> > > > >> can
> > > > >>
> > > > >> use
> > > > >>
> > > > >> to
> > > > >>
> > > > >> decide when to assign new splits. This should help both
> > > > >>
> > > > >> in
> > > > >>
> > > > >> cases
> > > > >>
> > > > >> of
> > > > >>
> > > > >> sources
> > > > >>
> > > > >> that take splits lazily (file readers) and in case the
> > > > >>
> > > > >> source
> > > > >>
> > > > >> needs to
> > > > >>
> > > > >> preserve a partial order between splits (Kinesis,
> > > > >>
> > > > >> Pravega,
> > > > >>
> > > > >> Pulsar may
> > > > >>
> > > > >> need
> > > > >>
> > > > >> that).
> > > > >>    --> SplitEnumerator gets notification when
> > > > >>
> > > > >> SplitReaders
> > > > >>
> > > > >> start
> > > > >>
> > > > >> and
> > > > >>
> > > > >> when
> > > > >>
> > > > >> they finish splits. They can decide at that moment to
> > > > >>
> > > > >> push
> > > > >>
> > > > >> more
> > > > >>
> > > > >> splits
> > > > >>
> > > > >> to
> > > > >>
> > > > >> that reader
> > > > >>    --> The SplitEnumerator should probably be aware of
> > > > >>
> > > > >> the
> > > > >>
> > > > >> source
> > > > >>
> > > > >> parallelism, to build its initial distribution.
> > > > >>
> > > > >>  - Open question: Should the source expose something
> > > > >>
> > > > >> like
> > > > >>
> > > > >> "host
> > > > >>
> > > > >> preferences", so that yarn/mesos/k8s can take this into
> > > > >>
> > > > >> account
> > > > >>
> > > > >> when
> > > > >>
> > > > >> selecting a node to start a TM on?
> > > > >>
> > > > >> *(5) Watermarks and event time alignment*
> > > > >>
> > > > >>  - Watermark generation, as well as idleness, needs to
> > > > >>
> > > > >> be
> > > > >>
> > > > >> per
> > > > >>
> > > > >> split
> > > > >>
> > > > >> (like
> > > > >>
> > > > >> currently in the Kafka Source, per partition)
> > > > >>  - It is desirable to support optional
> > > > >>
> > > > >> event-time-alignment,
> > > > >>
> > > > >> meaning
> > > > >>
> > > > >> that
> > > > >>
> > > > >> splits that are ahead are back-pressured or temporarily
> > > > >>
> > > > >> unsubscribed
> > > > >>
> > > > >>  - I think i would be desirable to encapsulate
> > > > >>
> > > > >> watermark
> > > > >>
> > > > >> generation
> > > > >>
> > > > >> logic
> > > > >>
> > > > >> in watermark generators, for a separation of concerns.
> > > > >>
> > > > >> The
> > > > >>
> > > > >> watermark
> > > > >>
> > > > >> generators should run per split.
> > > > >>  - Using watermark generators would also help with
> > > > >>
> > > > >> another
> > > > >>
> > > > >> problem of
> > > > >>
> > > > >> the
> > > > >>
> > > > >> suggested interface, namely supporting non-periodic
> > > > >>
> > > > >> watermarks
> > > > >>
> > > > >> efficiently.
> > > > >>
> > > > >>  - Need a way to "dispatch" next record to different
> > > > >>
> > > > >> watermark
> > > > >>
> > > > >> generators
> > > > >>
> > > > >>  - Need a way to tell SplitReader to "suspend" a split
> > > > >>
> > > > >> until a
> > > > >>
> > > > >> certain
> > > > >>
> > > > >> watermark is reached (event time backpressure)
> > > > >>  - This would in fact be not needed (and thus simpler)
> > > > >>
> > > > >> if
> > > > >>
> > > > >> we
> > > > >>
> > > > >> had
> > > > >>
> > > > >> a
> > > > >>
> > > > >> SplitReader per split and may be a reason to re-open
> > > > >>
> > > > >> that
> > > > >>
> > > > >> discussion
> > > > >>
> > > > >> *(6) Watermarks across splits and in the Split
> > > > >>
> > > > >> Enumerator*
> > > > >>
> > > > >>  - The split enumerator may need some watermark
> > > > >>
> > > > >> awareness,
> > > > >>
> > > > >> which
> > > > >>
> > > > >> should
> > > > >>
> > > > >> be
> > > > >>
> > > > >> purely based on split metadata (like create timestamp
> > > > >>
> > > > >> of
> > > > >>
> > > > >> file
> > > > >>
> > > > >> splits)
> > > > >>
> > > > >>  - If there are still more splits with overlapping
> > > > >>
> > > > >> event
> > > > >>
> > > > >> time
> > > > >>
> > > > >> range
> > > > >>
> > > > >> for
> > > > >>
> > > > >> a
> > > > >>
> > > > >> split reader, then that split reader should not advance
> > > > >>
> > > > >> the
> > > > >>
> > > > >> watermark
> > > > >>
> > > > >> within the split beyond the overlap boundary. Otherwise
> > > > >>
> > > > >> future
> > > > >>
> > > > >> splits
> > > > >>
> > > > >> will
> > > > >>
> > > > >> produce late data.
> > > > >>
> > > > >>  - One way to approach this could be that the split
> > > > >>
> > > > >> enumerator
> > > > >>
> > > > >> may
> > > > >>
> > > > >> send
> > > > >>
> > > > >> watermarks to the readers, and the readers cannot emit
> > > > >>
> > > > >> watermarks
> > > > >>
> > > > >> beyond
> > > > >>
> > > > >> that received watermark.
> > > > >>  - Many split enumerators would simply immediately send
> > > > >>
> > > > >> Long.MAX
> > > > >>
> > > > >> out
> > > > >>
> > > > >> and
> > > > >>
> > > > >> leave the progress purely to the split readers.
> > > > >>
> > > > >>  - For event-time alignment / split back pressure, this
> > > > >>
> > > > >> begs
> > > > >>
> > > > >> the
> > > > >>
> > > > >> question
> > > > >>
> > > > >> how we can avoid deadlocks that may arise when splits
> > > > >>
> > > > >> are
> > > > >>
> > > > >> suspended
> > > > >>
> > > > >> for
> > > > >>
> > > > >> event time back pressure,
> > > > >>
> > > > >> *(7) Batch and streaming Unification*
> > > > >>
> > > > >>  - Functionality wise, the above design should support
> > > > >>
> > > > >> both
> > > > >>
> > > > >>  - Batch often (mostly) does not care about reading "in
> > > > >>
> > > > >> order"
> > > > >>
> > > > >> and
> > > > >>
> > > > >> generating watermarks
> > > > >>    --> Might use different enumerator logic that is
> > > > >>
> > > > >> more
> > > > >>
> > > > >> locality
> > > > >>
> > > > >> aware
> > > > >>
> > > > >> and ignores event time order
> > > > >>    --> Does not generate watermarks
> > > > >>  - Would be great if bounded sources could be
> > > > >>
> > > > >> identified
> > > > >>
> > > > >> at
> > > > >>
> > > > >> compile
> > > > >>
> > > > >> time,
> > > > >>
> > > > >> so that "env.addBoundedSource(...)" is type safe and
> > > > >>
> > > > >> can
> > > > >>
> > > > >> return a
> > > > >>
> > > > >> "BoundedDataStream".
> > > > >>  - Possible to defer this discussion until later
> > > > >>
> > > > >> *Miscellaneous Comments*
> > > > >>
> > > > >>  - Should the source have a TypeInformation for the
> > > > >>
> > > > >> produced
> > > > >>
> > > > >> type,
> > > > >>
> > > > >> instead
> > > > >>
> > > > >> of a serializer? We need a type information in the
> > > > >>
> > > > >> stream
> > > > >>
> > > > >> anyways, and
> > > > >>
> > > > >> can
> > > > >>
> > > > >> derive the serializer from that. Plus, creating the
> > > > >>
> > > > >> serializer
> > > > >>
> > > > >> should
> > > > >>
> > > > >> respect the ExecutionConfig.
> > > > >>
> > > > >>  - The TypeSerializer interface is very powerful but
> > > > >>
> > > > >> also
> > > > >>
> > > > >> not
> > > > >>
> > > > >> easy to
> > > > >>
> > > > >> implement. Its purpose is to handle data super
> > > > >>
> > > > >> efficiently,
> > > > >>
> > > > >> support
> > > > >>
> > > > >> flexible ways of evolution, etc.
> > > > >>  For metadata I would suggest to look at the
> > > > >>
> > > > >> SimpleVersionedSerializer
> > > > >>
> > > > >> instead, which is used for example for checkpoint
> > > > >>
> > > > >> master
> > > > >>
> > > > >> hooks,
> > > > >>
> > > > >> or for
> > > > >>
> > > > >> the
> > > > >>
> > > > >> streaming file sink. I think that is is a good match
> > > > >>
> > > > >> for
> > > > >>
> > > > >> cases
> > > > >>
> > > > >> where
> > > > >>
> > > > >> we
> > > > >>
> > > > >> do
> > > > >>
> > > > >> not need more than ser/deser (no copy, etc.) and don't
> > > > >>
> > > > >> need to
> > > > >>
> > > > >> push
> > > > >>
> > > > >> versioning out of the serialization paths for best
> > > > >>
> > > > >> performance
> > > > >>
> > > > >> (as in
> > > > >>
> > > > >> the
> > > > >>
> > > > >> TypeSerializer)
> > > > >>
> > > > >>
> > > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > > > k.kloudas@data-artisans.com>
> > > > >> wrote:
> > > > >>
> > > > >>
> > > > >> Hi Biao,
> > > > >>
> > > > >> Thanks for the answer!
> > > > >>
> > > > >> So given the multi-threaded readers, now we have as
> > > > >>
> > > > >> open
> > > > >>
> > > > >> questions:
> > > > >>
> > > > >> 1) How do we let the checkpoints pass through our
> > > > >>
> > > > >> multi-threaded
> > > > >>
> > > > >> reader
> > > > >>
> > > > >> operator?
> > > > >>
> > > > >> 2) Do we have separate reader and source operators or
> > > > >>
> > > > >> not? In
> > > > >>
> > > > >> the
> > > > >>
> > > > >> strategy
> > > > >>
> > > > >> that has a separate source, the source operator has a
> > > > >>
> > > > >> parallelism of
> > > > >>
> > > > >> 1
> > > > >>
> > > > >> and
> > > > >>
> > > > >> is responsible for split recovery only.
> > > > >>
> > > > >> For the first one, given also the constraints
> > > > >>
> > > > >> (blocking,
> > > > >>
> > > > >> finite
> > > > >>
> > > > >> queues,
> > > > >>
> > > > >> etc), I do not have an answer yet.
> > > > >>
> > > > >> For the 2nd, I think that we should go with separate
> > > > >>
> > > > >> operators
> > > > >>
> > > > >> for
> > > > >>
> > > > >> the
> > > > >>
> > > > >> source and the readers, for the following reasons:
> > > > >>
> > > > >> 1) This is more aligned with a potential future
> > > > >>
> > > > >> improvement
> > > > >>
> > > > >> where the
> > > > >>
> > > > >> split
> > > > >>
> > > > >> discovery becomes a responsibility of the JobManager
> > > > >>
> > > > >> and
> > > > >>
> > > > >> readers are
> > > > >>
> > > > >> pooling more work from the JM.
> > > > >>
> > > > >> 2) The source is going to be the "single point of
> > > > >>
> > > > >> truth".
> > > > >>
> > > > >> It
> > > > >>
> > > > >> will
> > > > >>
> > > > >> know
> > > > >>
> > > > >> what
> > > > >>
> > > > >> has been processed and what not. If the source and the
> > > > >>
> > > > >> readers
> > > > >>
> > > > >> are a
> > > > >>
> > > > >> single
> > > > >>
> > > > >> operator with parallelism > 1, or in general, if the
> > > > >>
> > > > >> split
> > > > >>
> > > > >> discovery
> > > > >>
> > > > >> is
> > > > >>
> > > > >> done by each task individually, then:
> > > > >>   i) we have to have a deterministic scheme for each
> > > > >>
> > > > >> reader to
> > > > >>
> > > > >> assign
> > > > >>
> > > > >> splits to itself (e.g. mod subtaskId). This is not
> > > > >>
> > > > >> necessarily
> > > > >>
> > > > >> trivial
> > > > >>
> > > > >> for
> > > > >>
> > > > >> all sources.
> > > > >>   ii) each reader would have to keep a copy of all its
> > > > >>
> > > > >> processed
> > > > >>
> > > > >> slpits
> > > > >>
> > > > >>   iii) the state has to be a union state with a
> > > > >>
> > > > >> non-trivial
> > > > >>
> > > > >> merging
> > > > >>
> > > > >> logic
> > > > >>
> > > > >> in order to support rescaling.
> > > > >>
> > > > >> Two additional points that you raised above:
> > > > >>
> > > > >> i) The point that you raised that we need to keep all
> > > > >>
> > > > >> splits
> > > > >>
> > > > >> (processed
> > > > >>
> > > > >> and
> > > > >>
> > > > >> not-processed) I think is a bit of a strong
> > > > >>
> > > > >> requirement.
> > > > >>
> > > > >> This
> > > > >>
> > > > >> would
> > > > >>
> > > > >> imply
> > > > >>
> > > > >> that for infinite sources the state will grow
> > > > >>
> > > > >> indefinitely.
> > > > >>
> > > > >> This is
> > > > >>
> > > > >> problem
> > > > >>
> > > > >> is even more pronounced if we do not have a single
> > > > >>
> > > > >> source
> > > > >>
> > > > >> that
> > > > >>
> > > > >> assigns
> > > > >>
> > > > >> splits to readers, as each reader will have its own
> > > > >>
> > > > >> copy
> > > > >>
> > > > >> of
> > > > >>
> > > > >> the
> > > > >>
> > > > >> state.
> > > > >>
> > > > >> ii) it is true that for finite sources we need to
> > > > >>
> > > > >> somehow
> > > > >>
> > > > >> not
> > > > >>
> > > > >> close
> > > > >>
> > > > >> the
> > > > >>
> > > > >> readers when the source/split discoverer finishes. The
> > > > >> ContinuousFileReaderOperator has a work-around for
> > > > >>
> > > > >> that.
> > > > >>
> > > > >> It is
> > > > >>
> > > > >> not
> > > > >>
> > > > >> elegant,
> > > > >>
> > > > >> and checkpoints are not emitted after closing the
> > > > >>
> > > > >> source,
> > > > >>
> > > > >> but
> > > > >>
> > > > >> this, I
> > > > >>
> > > > >> believe, is a bigger problem which requires more
> > > > >>
> > > > >> changes
> > > > >>
> > > > >> than
> > > > >>
> > > > >> just
> > > > >>
> > > > >> refactoring the source interface.
> > > > >>
> > > > >> Cheers,
> > > > >> Kostas
> > > > >>
> > > > >>
> > > > >>
> > > >
> > > >
> > >
> >
> >
> > --
> > Best, Jingsong Lee
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Till Rohrmann <tr...@apache.org>.
Hi everyone,

thanks for drafting this FLIP. It reads very well.

Concerning Dawid's proposal, I tend to agree. The boundedness could come
from the source and tell the system how to treat the operator (scheduling
wise). From a user's perspective it should be fine to get back a DataStream
when calling env.source(boundedSource) if he does not need special
operations defined on a BoundedDataStream. If he needs this, then one could
use the method BoundedDataStream env.boundedSource(boundedSource).

If possible, we could enforce the proper usage of env.boundedSource() by
introducing a BoundedSource type so that one cannot pass an
unbounded source to it. That way users would not be able to shoot
themselves in the foot.

Maybe this has already been asked before but I was wondering why the
SourceReader interface has the method pollNext which hands the
responsibility of outputting elements to the SourceReader implementation?
Has this been done for backwards compatibility reasons with the old source
interface? If not, then one could define a Collection<E> getNextRecords()
method which returns the currently retrieved records and then the caller
emits them outside of the SourceReader. That way the interface would not
allow to implement an outputting loop where we never hand back control to
the caller. At the moment, this contract can be easily broken and is only
mentioned loosely in the JavaDocs.

Cheers,
Till

On Tue, Dec 10, 2019 at 7:49 AM Jingsong Li <ji...@gmail.com> wrote:

> Hi all,
>
> I think current design is good.
>
> My understanding is:
>
> For execution mode: bounded mode and continuous mode, It's totally
> different. I don't think we have the ability to integrate the two models at
> present. It's about scheduling, memory, algorithms, States, etc. we
> shouldn't confuse them.
>
> For source capabilities: only bounded, only continuous, both bounded and
> continuous.
> I think Kafka is a source that can be ran both bounded
> and continuous execution mode.
> And Kafka with end offset should be ran both bounded
> and continuous execution mode.  Using apache Beam with Flink runner, I used
> to run a "bounded" Kafka in streaming mode. For our previous DataStream, it
> is not necessarily required that the source cannot be bounded.
>
> So it is my thought for Dawid's question:
> 1.pass a bounded source to continuousSource() +1
> 2.pass a continuous source to boundedSource() -1, should throw exception.
>
> In StreamExecutionEnvironment, continuousSource and boundedSource define
> the execution mode. It defines a clear boundary of execution mode.
>
> Best,
> Jingsong Lee
>
> On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <im...@gmail.com> wrote:
>
> > I agree with Dawid's point that the boundedness information should come
> > from the source itself (e.g. the end timestamp), not through
> > env.boundedSouce()/continuousSource().
> > I think if we want to support something like `env.source()` that derive
> the
> > execution mode from source, `supportsBoundedness(Boundedness)`
> > method is not enough, because we don't know whether it is bounded or not.
> >
> > Best,
> > Jark
> >
> >
> > On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <dw...@apache.org>
> > wrote:
> >
> > > One more thing. In the current proposal, with the
> > > supportsBoundedness(Boundedness) method and the boundedness coming from
> > > either continuousSource or boundedSource I could not find how this
> > > information is fed back to the SplitEnumerator.
> > >
> > > Best,
> > >
> > > Dawid
> > >
> > > On 09/12/2019 13:52, Becket Qin wrote:
> > > > Hi Dawid,
> > > >
> > > > Thanks for the comments. This actually brings another relevant
> question
> > > > about what does a "bounded source" imply. I actually had the same
> > > > impression when I look at the Source API. Here is what I understand
> > after
> > > > some discussion with Stephan. The bounded source has the following
> > > impacts.
> > > >
> > > > 1. API validity.
> > > > - A bounded source generates a bounded stream so some operations that
> > > only
> > > > works for bounded records would be performed, e.g. sort.
> > > > - To expose these bounded stream only APIs, there are two options:
> > > >      a. Add them to the DataStream API and throw exception if a
> method
> > is
> > > > called on an unbounded stream.
> > > >      b. Create a BoundedDataStream class which is returned from
> > > > env.boundedSource(), while DataStream is returned from
> > > env.continousSource().
> > > > Note that this cannot be done by having single env.source(theSource)
> > even
> > > > the Source has a getBoundedness() method.
> > > >
> > > > 2. Scheduling
> > > > - A bounded source could be computed stage by stage without bringing
> up
> > > all
> > > > the tasks at the same time.
> > > >
> > > > 3. Operator behaviors
> > > > - A bounded source indicates the records are finite so some operators
> > can
> > > > wait until it receives all the records before it starts the
> processing.
> > > >
> > > > In the above impact, only 1 is relevant to the API design. And the
> > > current
> > > > proposal in FLIP-27 is following 1.b.
> > > >
> > > > // boundedness depends of source property, imo this should always be
> > > >> preferred
> > > >>
> > > >
> > > > DataStream<MyType> stream = env.source(theSource);
> > > >
> > > >
> > > > In your proposal, does DataStream have bounded stream only methods?
> It
> > > > looks it should have, otherwise passing a bounded Source to
> > env.source()
> > > > would be confusing. In that case, we will essentially do 1.a if an
> > > > unbounded Source is created from env.source(unboundedSource).
> > > >
> > > > If we have the methods only supported for bounded streams in
> > DataStream,
> > > it
> > > > seems a little weird to have a separate BoundedDataStream interface.
> > > >
> > > > Am I understand it correctly?
> > > >
> > > > Thanks,
> > > >
> > > > Jiangjie (Becket) Qin
> > > >
> > > >
> > > >
> > > > On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> > dwysakowicz@apache.org>
> > > > wrote:
> > > >
> > > >> Hi all,
> > > >>
> > > >> Really well written proposal and very important one. I must admit I
> > have
> > > >> not understood all the intricacies of it yet.
> > > >>
> > > >> One question I have though is about where does the information about
> > > >> boundedness come from. I think in most cases it is a property of the
> > > >> source. As you described it might be e.g. end offset, a flag should
> it
> > > >> monitor new splits etc. I think it would be a really nice use case
> to
> > be
> > > >> able to say:
> > > >>
> > > >> new KafkaSource().readUntil(long timestamp),
> > > >>
> > > >> which could work as an "end offset". Moreover I think all Bounded
> > > sources
> > > >> support continuous mode, but no intrinsically continuous source
> > support
> > > the
> > > >> Bounded mode. If I understood the proposal correctly it suggest the
> > > >> boundedness sort of "comes" from the outside of the source, from the
> > > >> invokation of either boundedStream or continousSource.
> > > >>
> > > >> I am wondering if it would make sense to actually change the method
> > > >>
> > > >> boolean Source#supportsBoundedness(Boundedness)
> > > >>
> > > >> to
> > > >>
> > > >> Boundedness Source#getBoundedness().
> > > >>
> > > >> As for the methods #boundedSource, #continousSource, assuming the
> > > >> boundedness is property of the source they do not affect how the
> > > enumerator
> > > >> works, but mostly how the dag is scheduled, right? I am not against
> > > those
> > > >> methods, but I think it is a very specific use case to actually
> > override
> > > >> the property of the source. In general I would expect users to only
> > call
> > > >> env.source(theSource), where the source tells if it is bounded or
> > not. I
> > > >> would suggest considering following set of methods:
> > > >>
> > > >> // boundedness depends of source property, imo this should always be
> > > preferred
> > > >>
> > > >> DataStream<MyType> stream = env.source(theSource);
> > > >>
> > > >>
> > > >> // always continous execution, whether bounded or unbounded source
> > > >>
> > > >> DataStream<MyType> boundedStream = env.continousSource(theSource);
> > > >>
> > > >> // imo this would make sense if the BoundedDataStream provides
> > > additional features unavailable for continous mode
> > > >> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
> > > >>
> > > >>
> > > >> Best,
> > > >>
> > > >> Dawid
> > > >>
> > > >>
> > > >> On 04/12/2019 11:25, Stephan Ewen wrote:
> > > >>
> > > >> Thanks, Becket, for updating this.
> > > >>
> > > >> I agree with moving the aspects you mentioned into separate FLIPs -
> > this
> > > >> one way becoming unwieldy in size.
> > > >>
> > > >> +1 to the FLIP in its current state. Its a very detailed write-up,
> > > nicely
> > > >> done!
> > > >>
> > > >> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <be...@gmail.com> <
> > > becket.qin@gmail.com> wrote:
> > > >>
> > > >>
> > > >> Hi all,
> > > >>
> > > >> Sorry for the long belated update. I have updated FLIP-27 wiki page
> > with
> > > >> the latest proposals. Some noticeable changes include:
> > > >> 1. A new generic communication mechanism between SplitEnumerator and
> > > >> SourceReader.
> > > >> 2. Some detail API method signature changes.
> > > >>
> > > >> We left a few things out of this FLIP and will address them in
> > separate
> > > >> FLIPs. Including:
> > > >> 1. Per split event time.
> > > >> 2. Event time alignment.
> > > >> 3. Fine grained failover for SplitEnumerator failure.
> > > >>
> > > >> Please let us know if you have any question.
> > > >>
> > > >> Thanks,
> > > >>
> > > >> Jiangjie (Becket) Qin
> > > >>
> > > >> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> <
> > > sewen@apache.org> wrote:
> > > >>
> > > >>
> > > >> Hi  Łukasz!
> > > >>
> > > >> Becket and me are working hard on figuring out the last details and
> > > >> implementing the first PoC. We would update the FLIP hopefully next
> > > week.
> > > >>
> > > >> There is a fair chance that a first version of this will be in 1.10,
> > but
> > > >>
> > > >> I
> > > >>
> > > >> think it will take another release to battle test it and migrate the
> > > >> connectors.
> > > >>
> > > >> Best,
> > > >> Stephan
> > > >>
> > > >>
> > > >>
> > > >>
> > > >> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <lj...@touk.pl>
> <
> > > ljd@touk.pl>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi,
> > > >>
> > > >> This proposal looks very promising for us. Do you have any plans in
> > > >>
> > > >> which
> > > >>
> > > >> Flink release it is going to be released? We are thinking on using a
> > > >>
> > > >> Data
> > > >>
> > > >> Set API for our future use cases but on the other hand Data Set API
> is
> > > >> going to be deprecated so using proposed bounded data streams
> solution
> > > >> could be more viable in the long term.
> > > >>
> > > >> Thanks,
> > > >> Łukasz
> > > >>
> > > >> On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> <
> > > thomas.weise@gmail.com> wrote:
> > > >>
> > > >> Thanks for putting together this proposal!
> > > >>
> > > >> I see that the "Per Split Event Time" and "Event Time Alignment"
> > > >>
> > > >> sections
> > > >>
> > > >> are still TBD.
> > > >>
> > > >> It would probably be good to flesh those out a bit before proceeding
> > > >>
> > > >> too
> > > >>
> > > >> far
> > > >>
> > > >> as the event time alignment will probably influence the interaction
> > > >>
> > > >> with
> > > >>
> > > >> the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
> > > >> output).
> > > >>
> > > >> We currently have only one implementation for event time alignment
> in
> > > >>
> > > >> the
> > > >>
> > > >> Kinesis consumer. The synchronization in that case takes place as
> the
> > > >>
> > > >> last
> > > >>
> > > >> step before records are emitted downstream (RecordEmitter). With the
> > > >> currently proposed interfaces, the equivalent can be implemented in
> > > >>
> > > >> the
> > > >>
> > > >> reader loop, although note that in the Kinesis consumer the per
> shard
> > > >> threads push records.
> > > >>
> > > >> Synchronization has not been implemented for the Kafka consumer yet.
> > > >> https://issues.apache.org/jira/browse/FLINK-12675
> > > >>
> > > >> When I looked at it, I realized that the implementation will look
> > > >>
> > > >> quite
> > > >>
> > > >> different
> > > >> from Kinesis because it needs to take place in the pull part, where
> > > >>
> > > >> records
> > > >>
> > > >> are taken from the Kafka client. Due to the multiplexing it cannot
> be
> > > >>
> > > >> done
> > > >>
> > > >> by blocking the split thread like it currently works for Kinesis.
> > > >>
> > > >> Reading
> > > >>
> > > >> from individual Kafka partitions needs to be controlled via
> > > >>
> > > >> pause/resume
> > > >>
> > > >> on the Kafka client.
> > > >>
> > > >> To take on that responsibility the split thread would need to be
> > > >>
> > > >> aware
> > > >>
> > > >> of
> > > >>
> > > >> the
> > > >> watermarks or at least whether it should or should not continue to
> > > >>
> > > >> consume
> > > >>
> > > >> a given split and this may require a different SourceReader or
> > > >>
> > > >> SourceOutput
> > > >>
> > > >> interface.
> > > >>
> > > >> Thanks,
> > > >> Thomas
> > > >>
> > > >>
> > > >> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> <
> > > mmyy1110@gmail.com> wrote:
> > > >>
> > > >>
> > > >> Hi Stephan,
> > > >>
> > > >> Thank you for feedback!
> > > >> Will take a look at your branch before public discussing.
> > > >>
> > > >>
> > > >> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org> <
> > > sewen@apache.org>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi Biao!
> > > >>
> > > >> Thanks for reviving this. I would like to join this discussion,
> > > >>
> > > >> but
> > > >>
> > > >> am
> > > >>
> > > >> quite occupied with the 1.9 release, so can we maybe pause this
> > > >>
> > > >> discussion
> > > >>
> > > >> for a week or so?
> > > >>
> > > >> In the meantime I can share some suggestion based on prior
> > > >>
> > > >> experiments:
> > > >>
> > > >> How to do watermarks / timestamp extractors in a simpler and more
> > > >>
> > > >> flexible
> > > >>
> > > >> way. I think that part is quite promising should be part of the
> > > >>
> > > >> new
> > > >>
> > > >> source
> > > >>
> > > >> interface.
> > > >>
> > > >>
> > > >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > > >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > > >>
> > > >> Some experiments on how to build the source reader and its
> > > >>
> > > >> library
> > > >>
> > > >> for
> > > >>
> > > >> common threading/split patterns:
> > > >>
> > > >>
> > > >>
> > > >>
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > > >>
> > > >> Best,
> > > >> Stephan
> > > >>
> > > >>
> > > >> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> <
> > > mmyy1110@gmail.com>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi devs,
> > > >>
> > > >> Since 1.9 is nearly released, I think we could get back to
> > > >>
> > > >> FLIP-27.
> > > >>
> > > >> I
> > > >>
> > > >> believe it should be included in 1.10.
> > > >>
> > > >> There are so many things mentioned in document of FLIP-27. [1] I
> > > >>
> > > >> think
> > > >>
> > > >> we'd better discuss them separately. However the wiki is not a
> > > >>
> > > >> good
> > > >>
> > > >> place
> > > >>
> > > >> to discuss. I wrote google doc about SplitReader API which
> > > >>
> > > >> misses
> > > >>
> > > >> some
> > > >>
> > > >> details in the document. [2]
> > > >>
> > > >> 1.
> > > >>
> > > >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > >>
> > > >> 2.
> > > >>
> > > >>
> > > >>
> > >
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > > >>
> > > >> CC Stephan, Aljoscha, Piotrek, Becket
> > > >>
> > > >>
> > > >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> <
> > > mmyy1110@gmail.com>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi Steven,
> > > >> Thank you for the feedback. Please take a look at the document
> > > >>
> > > >> FLIP-27
> > > >>
> > > >> <
> > > >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > >>
> > > >> which
> > > >>
> > > >> is updated recently. A lot of details of enumerator were added
> > > >>
> > > >> in
> > > >>
> > > >> this
> > > >>
> > > >> document. I think it would help.
> > > >>
> > > >> Steven Wu <st...@gmail.com> <st...@gmail.com>
> 于2019年3月28日周四
> > > 下午12:52写道:
> > > >>
> > > >>
> > > >> This proposal mentioned that SplitEnumerator might run on the
> > > >> JobManager or
> > > >> in a single task on a TaskManager.
> > > >>
> > > >> if enumerator is a single task on a taskmanager, then the job
> > > >>
> > > >> DAG
> > > >>
> > > >> can
> > > >>
> > > >> never
> > > >> been embarrassingly parallel anymore. That will nullify the
> > > >>
> > > >> leverage
> > > >>
> > > >> of
> > > >>
> > > >> fine-grained recovery for embarrassingly parallel jobs.
> > > >>
> > > >> It's not clear to me what's the implication of running
> > > >>
> > > >> enumerator
> > > >>
> > > >> on
> > > >>
> > > >> the
> > > >>
> > > >> jobmanager. So I will leave that out for now.
> > > >>
> > > >> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> <
> > > mmyy1110@gmail.com>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi Stephan & Piotrek,
> > > >>
> > > >> Thank you for feedback.
> > > >>
> > > >> It seems that there are a lot of things to do in community.
> > > >>
> > > >> I
> > > >>
> > > >> am
> > > >>
> > > >> just
> > > >>
> > > >> afraid that this discussion may be forgotten since there so
> > > >>
> > > >> many
> > > >>
> > > >> proposals
> > > >>
> > > >> recently.
> > > >> Anyway, wish to see the split topics soon :)
> > > >>
> > > >> Piotr Nowojski <pi...@da-platform.com> <pi...@da-platform.com>
> > > 于2019年1月24日周四
> > > >>
> > > >> 下午8:21写道:
> > > >>
> > > >> Hi Biao!
> > > >>
> > > >> This discussion was stalled because of preparations for
> > > >>
> > > >> the
> > > >>
> > > >> open
> > > >>
> > > >> sourcing
> > > >>
> > > >> & merging Blink. I think before creating the tickets we
> > > >>
> > > >> should
> > > >>
> > > >> split this
> > > >>
> > > >> discussion into topics/areas outlined by Stephan and
> > > >>
> > > >> create
> > > >>
> > > >> Flips
> > > >>
> > > >> for
> > > >>
> > > >> that.
> > > >>
> > > >> I think there is no chance for this to be completed in
> > > >>
> > > >> couple
> > > >>
> > > >> of
> > > >>
> > > >> remaining
> > > >>
> > > >> weeks/1 month before 1.8 feature freeze, however it would
> > > >>
> > > >> be
> > > >>
> > > >> good
> > > >>
> > > >> to aim
> > > >>
> > > >> with those changes for 1.9.
> > > >>
> > > >> Piotrek
> > > >>
> > > >>
> > > >> On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> <
> > > mmyy1110@gmail.com>
> > > >>
> > > >> wrote:
> > > >>
> > > >> Hi community,
> > > >> The summary of Stephan makes a lot sense to me. It is
> > > >>
> > > >> much
> > > >>
> > > >> clearer
> > > >>
> > > >> indeed
> > > >>
> > > >> after splitting the complex topic into small ones.
> > > >> I was wondering is there any detail plan for next step?
> > > >>
> > > >> If
> > > >>
> > > >> not,
> > > >>
> > > >> I
> > > >>
> > > >> would
> > > >>
> > > >> like to push this thing forward by creating some JIRA
> > > >>
> > > >> issues.
> > > >>
> > > >> Another question is that should version 1.8 include
> > > >>
> > > >> these
> > > >>
> > > >> features?
> > > >>
> > > >> Stephan Ewen <se...@apache.org> <se...@apache.org> 于2018年12月1日周六
> > > 上午4:20写道:
> > > >>
> > > >>
> > > >> Thanks everyone for the lively discussion. Let me try
> > > >>
> > > >> to
> > > >>
> > > >> summarize
> > > >>
> > > >> where I
> > > >>
> > > >> see convergence in the discussion and open issues.
> > > >> I'll try to group this by design aspect of the source.
> > > >>
> > > >> Please
> > > >>
> > > >> let me
> > > >>
> > > >> know
> > > >>
> > > >> if I got things wrong or missed something crucial here.
> > > >>
> > > >> For issues 1-3, if the below reflects the state of the
> > > >>
> > > >> discussion, I
> > > >>
> > > >> would
> > > >>
> > > >> try and update the FLIP in the next days.
> > > >> For the remaining ones we need more discussion.
> > > >>
> > > >> I would suggest to fork each of these aspects into a
> > > >>
> > > >> separate
> > > >>
> > > >> mail
> > > >>
> > > >> thread,
> > > >>
> > > >> or will loose sight of the individual aspects.
> > > >>
> > > >> *(1) Separation of Split Enumerator and Split Reader*
> > > >>
> > > >>  - All seem to agree this is a good thing
> > > >>  - Split Enumerator could in the end live on JobManager
> > > >>
> > > >> (and
> > > >>
> > > >> assign
> > > >>
> > > >> splits
> > > >>
> > > >> via RPC) or in a task (and assign splits via data
> > > >>
> > > >> streams)
> > > >>
> > > >>  - this discussion is orthogonal and should come later,
> > > >>
> > > >> when
> > > >>
> > > >> the
> > > >>
> > > >> interface
> > > >>
> > > >> is agreed upon.
> > > >>
> > > >> *(2) Split Readers for one or more splits*
> > > >>
> > > >>  - Discussion seems to agree that we need to support
> > > >>
> > > >> one
> > > >>
> > > >> reader
> > > >>
> > > >> that
> > > >>
> > > >> possibly handles multiple splits concurrently.
> > > >>  - The requirement comes from sources where one
> > > >>
> > > >> poll()-style
> > > >>
> > > >> call
> > > >>
> > > >> fetches
> > > >>
> > > >> data from different splits / partitions
> > > >>    --> example sources that require that would be for
> > > >>
> > > >> example
> > > >>
> > > >> Kafka,
> > > >>
> > > >> Pravega, Pulsar
> > > >>
> > > >>  - Could have one split reader per source, or multiple
> > > >>
> > > >> split
> > > >>
> > > >> readers
> > > >>
> > > >> that
> > > >>
> > > >> share the "poll()" function
> > > >>  - To not make it too complicated, we can start with
> > > >>
> > > >> thinking
> > > >>
> > > >> about
> > > >>
> > > >> one
> > > >>
> > > >> split reader for all splits initially and see if that
> > > >>
> > > >> covers
> > > >>
> > > >> all
> > > >>
> > > >> requirements
> > > >>
> > > >> *(3) Threading model of the Split Reader*
> > > >>
> > > >>  - Most active part of the discussion ;-)
> > > >>
> > > >>  - A non-blocking way for Flink's task code to interact
> > > >>
> > > >> with
> > > >>
> > > >> the
> > > >>
> > > >> source
> > > >>
> > > >> is
> > > >>
> > > >> needed in order to a task runtime code based on a
> > > >> single-threaded/actor-style task design
> > > >>    --> I personally am a big proponent of that, it will
> > > >>
> > > >> help
> > > >>
> > > >> with
> > > >>
> > > >> well-behaved checkpoints, efficiency, and simpler yet
> > > >>
> > > >> more
> > > >>
> > > >> robust
> > > >>
> > > >> runtime
> > > >>
> > > >> code
> > > >>
> > > >>  - Users care about simple abstraction, so as a
> > > >>
> > > >> subclass
> > > >>
> > > >> of
> > > >>
> > > >> SplitReader
> > > >>
> > > >> (non-blocking / async) we need to have a
> > > >>
> > > >> BlockingSplitReader
> > > >>
> > > >> which
> > > >>
> > > >> will
> > > >>
> > > >> form the basis of most source implementations.
> > > >>
> > > >> BlockingSplitReader
> > > >>
> > > >> lets
> > > >>
> > > >> users do blocking simple poll() calls.
> > > >>  - The BlockingSplitReader would spawn a thread (or
> > > >>
> > > >> more)
> > > >>
> > > >> and
> > > >>
> > > >> the
> > > >>
> > > >> thread(s) can make blocking calls and hand over data
> > > >>
> > > >> buffers
> > > >>
> > > >> via
> > > >>
> > > >> a
> > > >>
> > > >> blocking
> > > >>
> > > >> queue
> > > >>  - This should allow us to cover both, a fully async
> > > >>
> > > >> runtime,
> > > >>
> > > >> and a
> > > >>
> > > >> simple
> > > >>
> > > >> blocking interface for users.
> > > >>  - This is actually very similar to how the Kafka
> > > >>
> > > >> connectors
> > > >>
> > > >> work.
> > > >>
> > > >> Kafka
> > > >>
> > > >> 9+ with one thread, Kafka 8 with multiple threads
> > > >>
> > > >>  - On the base SplitReader (the async one), the
> > > >>
> > > >> non-blocking
> > > >>
> > > >> method
> > > >>
> > > >> that
> > > >>
> > > >> gets the next chunk of data would signal data
> > > >>
> > > >> availability
> > > >>
> > > >> via
> > > >>
> > > >> a
> > > >>
> > > >> CompletableFuture, because that gives the best
> > > >>
> > > >> flexibility
> > > >>
> > > >> (can
> > > >>
> > > >> await
> > > >>
> > > >> completion or register notification handlers).
> > > >>  - The source task would register a "thenHandle()" (or
> > > >>
> > > >> similar)
> > > >>
> > > >> on the
> > > >>
> > > >> future to put a "take next data" task into the
> > > >>
> > > >> actor-style
> > > >>
> > > >> mailbox
> > > >>
> > > >> *(4) Split Enumeration and Assignment*
> > > >>
> > > >>  - Splits may be generated lazily, both in cases where
> > > >>
> > > >> there
> > > >>
> > > >> is a
> > > >>
> > > >> limited
> > > >>
> > > >> number of splits (but very many), or splits are
> > > >>
> > > >> discovered
> > > >>
> > > >> over
> > > >>
> > > >> time
> > > >>
> > > >>  - Assignment should also be lazy, to get better load
> > > >>
> > > >> balancing
> > > >>
> > > >>  - Assignment needs support locality preferences
> > > >>
> > > >>  - Possible design based on discussion so far:
> > > >>
> > > >>    --> SplitReader has a method "addSplits(SplitT...)"
> > > >>
> > > >> to
> > > >>
> > > >> add
> > > >>
> > > >> one or
> > > >>
> > > >> more
> > > >>
> > > >> splits. Some split readers might assume they have only
> > > >>
> > > >> one
> > > >>
> > > >> split
> > > >>
> > > >> ever,
> > > >>
> > > >> concurrently, others assume multiple splits. (Note:
> > > >>
> > > >> idea
> > > >>
> > > >> behind
> > > >>
> > > >> being
> > > >>
> > > >> able
> > > >>
> > > >> to add multiple splits at the same time is to ease
> > > >>
> > > >> startup
> > > >>
> > > >> where
> > > >>
> > > >> multiple
> > > >>
> > > >> splits may be assigned instantly.)
> > > >>    --> SplitReader has a context object on which it can
> > > >>
> > > >> call
> > > >>
> > > >> indicate
> > > >>
> > > >> when
> > > >>
> > > >> splits are completed. The enumerator gets that
> > > >>
> > > >> notification and
> > > >>
> > > >> can
> > > >>
> > > >> use
> > > >>
> > > >> to
> > > >>
> > > >> decide when to assign new splits. This should help both
> > > >>
> > > >> in
> > > >>
> > > >> cases
> > > >>
> > > >> of
> > > >>
> > > >> sources
> > > >>
> > > >> that take splits lazily (file readers) and in case the
> > > >>
> > > >> source
> > > >>
> > > >> needs to
> > > >>
> > > >> preserve a partial order between splits (Kinesis,
> > > >>
> > > >> Pravega,
> > > >>
> > > >> Pulsar may
> > > >>
> > > >> need
> > > >>
> > > >> that).
> > > >>    --> SplitEnumerator gets notification when
> > > >>
> > > >> SplitReaders
> > > >>
> > > >> start
> > > >>
> > > >> and
> > > >>
> > > >> when
> > > >>
> > > >> they finish splits. They can decide at that moment to
> > > >>
> > > >> push
> > > >>
> > > >> more
> > > >>
> > > >> splits
> > > >>
> > > >> to
> > > >>
> > > >> that reader
> > > >>    --> The SplitEnumerator should probably be aware of
> > > >>
> > > >> the
> > > >>
> > > >> source
> > > >>
> > > >> parallelism, to build its initial distribution.
> > > >>
> > > >>  - Open question: Should the source expose something
> > > >>
> > > >> like
> > > >>
> > > >> "host
> > > >>
> > > >> preferences", so that yarn/mesos/k8s can take this into
> > > >>
> > > >> account
> > > >>
> > > >> when
> > > >>
> > > >> selecting a node to start a TM on?
> > > >>
> > > >> *(5) Watermarks and event time alignment*
> > > >>
> > > >>  - Watermark generation, as well as idleness, needs to
> > > >>
> > > >> be
> > > >>
> > > >> per
> > > >>
> > > >> split
> > > >>
> > > >> (like
> > > >>
> > > >> currently in the Kafka Source, per partition)
> > > >>  - It is desirable to support optional
> > > >>
> > > >> event-time-alignment,
> > > >>
> > > >> meaning
> > > >>
> > > >> that
> > > >>
> > > >> splits that are ahead are back-pressured or temporarily
> > > >>
> > > >> unsubscribed
> > > >>
> > > >>  - I think i would be desirable to encapsulate
> > > >>
> > > >> watermark
> > > >>
> > > >> generation
> > > >>
> > > >> logic
> > > >>
> > > >> in watermark generators, for a separation of concerns.
> > > >>
> > > >> The
> > > >>
> > > >> watermark
> > > >>
> > > >> generators should run per split.
> > > >>  - Using watermark generators would also help with
> > > >>
> > > >> another
> > > >>
> > > >> problem of
> > > >>
> > > >> the
> > > >>
> > > >> suggested interface, namely supporting non-periodic
> > > >>
> > > >> watermarks
> > > >>
> > > >> efficiently.
> > > >>
> > > >>  - Need a way to "dispatch" next record to different
> > > >>
> > > >> watermark
> > > >>
> > > >> generators
> > > >>
> > > >>  - Need a way to tell SplitReader to "suspend" a split
> > > >>
> > > >> until a
> > > >>
> > > >> certain
> > > >>
> > > >> watermark is reached (event time backpressure)
> > > >>  - This would in fact be not needed (and thus simpler)
> > > >>
> > > >> if
> > > >>
> > > >> we
> > > >>
> > > >> had
> > > >>
> > > >> a
> > > >>
> > > >> SplitReader per split and may be a reason to re-open
> > > >>
> > > >> that
> > > >>
> > > >> discussion
> > > >>
> > > >> *(6) Watermarks across splits and in the Split
> > > >>
> > > >> Enumerator*
> > > >>
> > > >>  - The split enumerator may need some watermark
> > > >>
> > > >> awareness,
> > > >>
> > > >> which
> > > >>
> > > >> should
> > > >>
> > > >> be
> > > >>
> > > >> purely based on split metadata (like create timestamp
> > > >>
> > > >> of
> > > >>
> > > >> file
> > > >>
> > > >> splits)
> > > >>
> > > >>  - If there are still more splits with overlapping
> > > >>
> > > >> event
> > > >>
> > > >> time
> > > >>
> > > >> range
> > > >>
> > > >> for
> > > >>
> > > >> a
> > > >>
> > > >> split reader, then that split reader should not advance
> > > >>
> > > >> the
> > > >>
> > > >> watermark
> > > >>
> > > >> within the split beyond the overlap boundary. Otherwise
> > > >>
> > > >> future
> > > >>
> > > >> splits
> > > >>
> > > >> will
> > > >>
> > > >> produce late data.
> > > >>
> > > >>  - One way to approach this could be that the split
> > > >>
> > > >> enumerator
> > > >>
> > > >> may
> > > >>
> > > >> send
> > > >>
> > > >> watermarks to the readers, and the readers cannot emit
> > > >>
> > > >> watermarks
> > > >>
> > > >> beyond
> > > >>
> > > >> that received watermark.
> > > >>  - Many split enumerators would simply immediately send
> > > >>
> > > >> Long.MAX
> > > >>
> > > >> out
> > > >>
> > > >> and
> > > >>
> > > >> leave the progress purely to the split readers.
> > > >>
> > > >>  - For event-time alignment / split back pressure, this
> > > >>
> > > >> begs
> > > >>
> > > >> the
> > > >>
> > > >> question
> > > >>
> > > >> how we can avoid deadlocks that may arise when splits
> > > >>
> > > >> are
> > > >>
> > > >> suspended
> > > >>
> > > >> for
> > > >>
> > > >> event time back pressure,
> > > >>
> > > >> *(7) Batch and streaming Unification*
> > > >>
> > > >>  - Functionality wise, the above design should support
> > > >>
> > > >> both
> > > >>
> > > >>  - Batch often (mostly) does not care about reading "in
> > > >>
> > > >> order"
> > > >>
> > > >> and
> > > >>
> > > >> generating watermarks
> > > >>    --> Might use different enumerator logic that is
> > > >>
> > > >> more
> > > >>
> > > >> locality
> > > >>
> > > >> aware
> > > >>
> > > >> and ignores event time order
> > > >>    --> Does not generate watermarks
> > > >>  - Would be great if bounded sources could be
> > > >>
> > > >> identified
> > > >>
> > > >> at
> > > >>
> > > >> compile
> > > >>
> > > >> time,
> > > >>
> > > >> so that "env.addBoundedSource(...)" is type safe and
> > > >>
> > > >> can
> > > >>
> > > >> return a
> > > >>
> > > >> "BoundedDataStream".
> > > >>  - Possible to defer this discussion until later
> > > >>
> > > >> *Miscellaneous Comments*
> > > >>
> > > >>  - Should the source have a TypeInformation for the
> > > >>
> > > >> produced
> > > >>
> > > >> type,
> > > >>
> > > >> instead
> > > >>
> > > >> of a serializer? We need a type information in the
> > > >>
> > > >> stream
> > > >>
> > > >> anyways, and
> > > >>
> > > >> can
> > > >>
> > > >> derive the serializer from that. Plus, creating the
> > > >>
> > > >> serializer
> > > >>
> > > >> should
> > > >>
> > > >> respect the ExecutionConfig.
> > > >>
> > > >>  - The TypeSerializer interface is very powerful but
> > > >>
> > > >> also
> > > >>
> > > >> not
> > > >>
> > > >> easy to
> > > >>
> > > >> implement. Its purpose is to handle data super
> > > >>
> > > >> efficiently,
> > > >>
> > > >> support
> > > >>
> > > >> flexible ways of evolution, etc.
> > > >>  For metadata I would suggest to look at the
> > > >>
> > > >> SimpleVersionedSerializer
> > > >>
> > > >> instead, which is used for example for checkpoint
> > > >>
> > > >> master
> > > >>
> > > >> hooks,
> > > >>
> > > >> or for
> > > >>
> > > >> the
> > > >>
> > > >> streaming file sink. I think that is is a good match
> > > >>
> > > >> for
> > > >>
> > > >> cases
> > > >>
> > > >> where
> > > >>
> > > >> we
> > > >>
> > > >> do
> > > >>
> > > >> not need more than ser/deser (no copy, etc.) and don't
> > > >>
> > > >> need to
> > > >>
> > > >> push
> > > >>
> > > >> versioning out of the serialization paths for best
> > > >>
> > > >> performance
> > > >>
> > > >> (as in
> > > >>
> > > >> the
> > > >>
> > > >> TypeSerializer)
> > > >>
> > > >>
> > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > > k.kloudas@data-artisans.com>
> > > >> wrote:
> > > >>
> > > >>
> > > >> Hi Biao,
> > > >>
> > > >> Thanks for the answer!
> > > >>
> > > >> So given the multi-threaded readers, now we have as
> > > >>
> > > >> open
> > > >>
> > > >> questions:
> > > >>
> > > >> 1) How do we let the checkpoints pass through our
> > > >>
> > > >> multi-threaded
> > > >>
> > > >> reader
> > > >>
> > > >> operator?
> > > >>
> > > >> 2) Do we have separate reader and source operators or
> > > >>
> > > >> not? In
> > > >>
> > > >> the
> > > >>
> > > >> strategy
> > > >>
> > > >> that has a separate source, the source operator has a
> > > >>
> > > >> parallelism of
> > > >>
> > > >> 1
> > > >>
> > > >> and
> > > >>
> > > >> is responsible for split recovery only.
> > > >>
> > > >> For the first one, given also the constraints
> > > >>
> > > >> (blocking,
> > > >>
> > > >> finite
> > > >>
> > > >> queues,
> > > >>
> > > >> etc), I do not have an answer yet.
> > > >>
> > > >> For the 2nd, I think that we should go with separate
> > > >>
> > > >> operators
> > > >>
> > > >> for
> > > >>
> > > >> the
> > > >>
> > > >> source and the readers, for the following reasons:
> > > >>
> > > >> 1) This is more aligned with a potential future
> > > >>
> > > >> improvement
> > > >>
> > > >> where the
> > > >>
> > > >> split
> > > >>
> > > >> discovery becomes a responsibility of the JobManager
> > > >>
> > > >> and
> > > >>
> > > >> readers are
> > > >>
> > > >> pooling more work from the JM.
> > > >>
> > > >> 2) The source is going to be the "single point of
> > > >>
> > > >> truth".
> > > >>
> > > >> It
> > > >>
> > > >> will
> > > >>
> > > >> know
> > > >>
> > > >> what
> > > >>
> > > >> has been processed and what not. If the source and the
> > > >>
> > > >> readers
> > > >>
> > > >> are a
> > > >>
> > > >> single
> > > >>
> > > >> operator with parallelism > 1, or in general, if the
> > > >>
> > > >> split
> > > >>
> > > >> discovery
> > > >>
> > > >> is
> > > >>
> > > >> done by each task individually, then:
> > > >>   i) we have to have a deterministic scheme for each
> > > >>
> > > >> reader to
> > > >>
> > > >> assign
> > > >>
> > > >> splits to itself (e.g. mod subtaskId). This is not
> > > >>
> > > >> necessarily
> > > >>
> > > >> trivial
> > > >>
> > > >> for
> > > >>
> > > >> all sources.
> > > >>   ii) each reader would have to keep a copy of all its
> > > >>
> > > >> processed
> > > >>
> > > >> slpits
> > > >>
> > > >>   iii) the state has to be a union state with a
> > > >>
> > > >> non-trivial
> > > >>
> > > >> merging
> > > >>
> > > >> logic
> > > >>
> > > >> in order to support rescaling.
> > > >>
> > > >> Two additional points that you raised above:
> > > >>
> > > >> i) The point that you raised that we need to keep all
> > > >>
> > > >> splits
> > > >>
> > > >> (processed
> > > >>
> > > >> and
> > > >>
> > > >> not-processed) I think is a bit of a strong
> > > >>
> > > >> requirement.
> > > >>
> > > >> This
> > > >>
> > > >> would
> > > >>
> > > >> imply
> > > >>
> > > >> that for infinite sources the state will grow
> > > >>
> > > >> indefinitely.
> > > >>
> > > >> This is
> > > >>
> > > >> problem
> > > >>
> > > >> is even more pronounced if we do not have a single
> > > >>
> > > >> source
> > > >>
> > > >> that
> > > >>
> > > >> assigns
> > > >>
> > > >> splits to readers, as each reader will have its own
> > > >>
> > > >> copy
> > > >>
> > > >> of
> > > >>
> > > >> the
> > > >>
> > > >> state.
> > > >>
> > > >> ii) it is true that for finite sources we need to
> > > >>
> > > >> somehow
> > > >>
> > > >> not
> > > >>
> > > >> close
> > > >>
> > > >> the
> > > >>
> > > >> readers when the source/split discoverer finishes. The
> > > >> ContinuousFileReaderOperator has a work-around for
> > > >>
> > > >> that.
> > > >>
> > > >> It is
> > > >>
> > > >> not
> > > >>
> > > >> elegant,
> > > >>
> > > >> and checkpoints are not emitted after closing the
> > > >>
> > > >> source,
> > > >>
> > > >> but
> > > >>
> > > >> this, I
> > > >>
> > > >> believe, is a bigger problem which requires more
> > > >>
> > > >> changes
> > > >>
> > > >> than
> > > >>
> > > >> just
> > > >>
> > > >> refactoring the source interface.
> > > >>
> > > >> Cheers,
> > > >> Kostas
> > > >>
> > > >>
> > > >>
> > >
> > >
> >
>
>
> --
> Best, Jingsong Lee
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Jingsong Li <ji...@gmail.com>.
Hi all,

I think current design is good.

My understanding is:

For execution mode: bounded mode and continuous mode, It's totally
different. I don't think we have the ability to integrate the two models at
present. It's about scheduling, memory, algorithms, States, etc. we
shouldn't confuse them.

For source capabilities: only bounded, only continuous, both bounded and
continuous.
I think Kafka is a source that can be ran both bounded
and continuous execution mode.
And Kafka with end offset should be ran both bounded
and continuous execution mode.  Using apache Beam with Flink runner, I used
to run a "bounded" Kafka in streaming mode. For our previous DataStream, it
is not necessarily required that the source cannot be bounded.

So it is my thought for Dawid's question:
1.pass a bounded source to continuousSource() +1
2.pass a continuous source to boundedSource() -1, should throw exception.

In StreamExecutionEnvironment, continuousSource and boundedSource define
the execution mode. It defines a clear boundary of execution mode.

Best,
Jingsong Lee

On Tue, Dec 10, 2019 at 10:37 AM Jark Wu <im...@gmail.com> wrote:

> I agree with Dawid's point that the boundedness information should come
> from the source itself (e.g. the end timestamp), not through
> env.boundedSouce()/continuousSource().
> I think if we want to support something like `env.source()` that derive the
> execution mode from source, `supportsBoundedness(Boundedness)`
> method is not enough, because we don't know whether it is bounded or not.
>
> Best,
> Jark
>
>
> On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <dw...@apache.org>
> wrote:
>
> > One more thing. In the current proposal, with the
> > supportsBoundedness(Boundedness) method and the boundedness coming from
> > either continuousSource or boundedSource I could not find how this
> > information is fed back to the SplitEnumerator.
> >
> > Best,
> >
> > Dawid
> >
> > On 09/12/2019 13:52, Becket Qin wrote:
> > > Hi Dawid,
> > >
> > > Thanks for the comments. This actually brings another relevant question
> > > about what does a "bounded source" imply. I actually had the same
> > > impression when I look at the Source API. Here is what I understand
> after
> > > some discussion with Stephan. The bounded source has the following
> > impacts.
> > >
> > > 1. API validity.
> > > - A bounded source generates a bounded stream so some operations that
> > only
> > > works for bounded records would be performed, e.g. sort.
> > > - To expose these bounded stream only APIs, there are two options:
> > >      a. Add them to the DataStream API and throw exception if a method
> is
> > > called on an unbounded stream.
> > >      b. Create a BoundedDataStream class which is returned from
> > > env.boundedSource(), while DataStream is returned from
> > env.continousSource().
> > > Note that this cannot be done by having single env.source(theSource)
> even
> > > the Source has a getBoundedness() method.
> > >
> > > 2. Scheduling
> > > - A bounded source could be computed stage by stage without bringing up
> > all
> > > the tasks at the same time.
> > >
> > > 3. Operator behaviors
> > > - A bounded source indicates the records are finite so some operators
> can
> > > wait until it receives all the records before it starts the processing.
> > >
> > > In the above impact, only 1 is relevant to the API design. And the
> > current
> > > proposal in FLIP-27 is following 1.b.
> > >
> > > // boundedness depends of source property, imo this should always be
> > >> preferred
> > >>
> > >
> > > DataStream<MyType> stream = env.source(theSource);
> > >
> > >
> > > In your proposal, does DataStream have bounded stream only methods? It
> > > looks it should have, otherwise passing a bounded Source to
> env.source()
> > > would be confusing. In that case, we will essentially do 1.a if an
> > > unbounded Source is created from env.source(unboundedSource).
> > >
> > > If we have the methods only supported for bounded streams in
> DataStream,
> > it
> > > seems a little weird to have a separate BoundedDataStream interface.
> > >
> > > Am I understand it correctly?
> > >
> > > Thanks,
> > >
> > > Jiangjie (Becket) Qin
> > >
> > >
> > >
> > > On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <
> dwysakowicz@apache.org>
> > > wrote:
> > >
> > >> Hi all,
> > >>
> > >> Really well written proposal and very important one. I must admit I
> have
> > >> not understood all the intricacies of it yet.
> > >>
> > >> One question I have though is about where does the information about
> > >> boundedness come from. I think in most cases it is a property of the
> > >> source. As you described it might be e.g. end offset, a flag should it
> > >> monitor new splits etc. I think it would be a really nice use case to
> be
> > >> able to say:
> > >>
> > >> new KafkaSource().readUntil(long timestamp),
> > >>
> > >> which could work as an "end offset". Moreover I think all Bounded
> > sources
> > >> support continuous mode, but no intrinsically continuous source
> support
> > the
> > >> Bounded mode. If I understood the proposal correctly it suggest the
> > >> boundedness sort of "comes" from the outside of the source, from the
> > >> invokation of either boundedStream or continousSource.
> > >>
> > >> I am wondering if it would make sense to actually change the method
> > >>
> > >> boolean Source#supportsBoundedness(Boundedness)
> > >>
> > >> to
> > >>
> > >> Boundedness Source#getBoundedness().
> > >>
> > >> As for the methods #boundedSource, #continousSource, assuming the
> > >> boundedness is property of the source they do not affect how the
> > enumerator
> > >> works, but mostly how the dag is scheduled, right? I am not against
> > those
> > >> methods, but I think it is a very specific use case to actually
> override
> > >> the property of the source. In general I would expect users to only
> call
> > >> env.source(theSource), where the source tells if it is bounded or
> not. I
> > >> would suggest considering following set of methods:
> > >>
> > >> // boundedness depends of source property, imo this should always be
> > preferred
> > >>
> > >> DataStream<MyType> stream = env.source(theSource);
> > >>
> > >>
> > >> // always continous execution, whether bounded or unbounded source
> > >>
> > >> DataStream<MyType> boundedStream = env.continousSource(theSource);
> > >>
> > >> // imo this would make sense if the BoundedDataStream provides
> > additional features unavailable for continous mode
> > >> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
> > >>
> > >>
> > >> Best,
> > >>
> > >> Dawid
> > >>
> > >>
> > >> On 04/12/2019 11:25, Stephan Ewen wrote:
> > >>
> > >> Thanks, Becket, for updating this.
> > >>
> > >> I agree with moving the aspects you mentioned into separate FLIPs -
> this
> > >> one way becoming unwieldy in size.
> > >>
> > >> +1 to the FLIP in its current state. Its a very detailed write-up,
> > nicely
> > >> done!
> > >>
> > >> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <be...@gmail.com> <
> > becket.qin@gmail.com> wrote:
> > >>
> > >>
> > >> Hi all,
> > >>
> > >> Sorry for the long belated update. I have updated FLIP-27 wiki page
> with
> > >> the latest proposals. Some noticeable changes include:
> > >> 1. A new generic communication mechanism between SplitEnumerator and
> > >> SourceReader.
> > >> 2. Some detail API method signature changes.
> > >>
> > >> We left a few things out of this FLIP and will address them in
> separate
> > >> FLIPs. Including:
> > >> 1. Per split event time.
> > >> 2. Event time alignment.
> > >> 3. Fine grained failover for SplitEnumerator failure.
> > >>
> > >> Please let us know if you have any question.
> > >>
> > >> Thanks,
> > >>
> > >> Jiangjie (Becket) Qin
> > >>
> > >> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> <
> > sewen@apache.org> wrote:
> > >>
> > >>
> > >> Hi  Łukasz!
> > >>
> > >> Becket and me are working hard on figuring out the last details and
> > >> implementing the first PoC. We would update the FLIP hopefully next
> > week.
> > >>
> > >> There is a fair chance that a first version of this will be in 1.10,
> but
> > >>
> > >> I
> > >>
> > >> think it will take another release to battle test it and migrate the
> > >> connectors.
> > >>
> > >> Best,
> > >> Stephan
> > >>
> > >>
> > >>
> > >>
> > >> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <lj...@touk.pl> <
> > ljd@touk.pl>
> > >>
> > >> wrote:
> > >>
> > >> Hi,
> > >>
> > >> This proposal looks very promising for us. Do you have any plans in
> > >>
> > >> which
> > >>
> > >> Flink release it is going to be released? We are thinking on using a
> > >>
> > >> Data
> > >>
> > >> Set API for our future use cases but on the other hand Data Set API is
> > >> going to be deprecated so using proposed bounded data streams solution
> > >> could be more viable in the long term.
> > >>
> > >> Thanks,
> > >> Łukasz
> > >>
> > >> On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> <
> > thomas.weise@gmail.com> wrote:
> > >>
> > >> Thanks for putting together this proposal!
> > >>
> > >> I see that the "Per Split Event Time" and "Event Time Alignment"
> > >>
> > >> sections
> > >>
> > >> are still TBD.
> > >>
> > >> It would probably be good to flesh those out a bit before proceeding
> > >>
> > >> too
> > >>
> > >> far
> > >>
> > >> as the event time alignment will probably influence the interaction
> > >>
> > >> with
> > >>
> > >> the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
> > >> output).
> > >>
> > >> We currently have only one implementation for event time alignment in
> > >>
> > >> the
> > >>
> > >> Kinesis consumer. The synchronization in that case takes place as the
> > >>
> > >> last
> > >>
> > >> step before records are emitted downstream (RecordEmitter). With the
> > >> currently proposed interfaces, the equivalent can be implemented in
> > >>
> > >> the
> > >>
> > >> reader loop, although note that in the Kinesis consumer the per shard
> > >> threads push records.
> > >>
> > >> Synchronization has not been implemented for the Kafka consumer yet.
> > >> https://issues.apache.org/jira/browse/FLINK-12675
> > >>
> > >> When I looked at it, I realized that the implementation will look
> > >>
> > >> quite
> > >>
> > >> different
> > >> from Kinesis because it needs to take place in the pull part, where
> > >>
> > >> records
> > >>
> > >> are taken from the Kafka client. Due to the multiplexing it cannot be
> > >>
> > >> done
> > >>
> > >> by blocking the split thread like it currently works for Kinesis.
> > >>
> > >> Reading
> > >>
> > >> from individual Kafka partitions needs to be controlled via
> > >>
> > >> pause/resume
> > >>
> > >> on the Kafka client.
> > >>
> > >> To take on that responsibility the split thread would need to be
> > >>
> > >> aware
> > >>
> > >> of
> > >>
> > >> the
> > >> watermarks or at least whether it should or should not continue to
> > >>
> > >> consume
> > >>
> > >> a given split and this may require a different SourceReader or
> > >>
> > >> SourceOutput
> > >>
> > >> interface.
> > >>
> > >> Thanks,
> > >> Thomas
> > >>
> > >>
> > >> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> <
> > mmyy1110@gmail.com> wrote:
> > >>
> > >>
> > >> Hi Stephan,
> > >>
> > >> Thank you for feedback!
> > >> Will take a look at your branch before public discussing.
> > >>
> > >>
> > >> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org> <
> > sewen@apache.org>
> > >>
> > >> wrote:
> > >>
> > >> Hi Biao!
> > >>
> > >> Thanks for reviving this. I would like to join this discussion,
> > >>
> > >> but
> > >>
> > >> am
> > >>
> > >> quite occupied with the 1.9 release, so can we maybe pause this
> > >>
> > >> discussion
> > >>
> > >> for a week or so?
> > >>
> > >> In the meantime I can share some suggestion based on prior
> > >>
> > >> experiments:
> > >>
> > >> How to do watermarks / timestamp extractors in a simpler and more
> > >>
> > >> flexible
> > >>
> > >> way. I think that part is quite promising should be part of the
> > >>
> > >> new
> > >>
> > >> source
> > >>
> > >> interface.
> > >>
> > >>
> > >>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > >>
> > >>
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > >>
> > >> Some experiments on how to build the source reader and its
> > >>
> > >> library
> > >>
> > >> for
> > >>
> > >> common threading/split patterns:
> > >>
> > >>
> > >>
> > >>
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > >>
> > >> Best,
> > >> Stephan
> > >>
> > >>
> > >> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> <
> > mmyy1110@gmail.com>
> > >>
> > >> wrote:
> > >>
> > >> Hi devs,
> > >>
> > >> Since 1.9 is nearly released, I think we could get back to
> > >>
> > >> FLIP-27.
> > >>
> > >> I
> > >>
> > >> believe it should be included in 1.10.
> > >>
> > >> There are so many things mentioned in document of FLIP-27. [1] I
> > >>
> > >> think
> > >>
> > >> we'd better discuss them separately. However the wiki is not a
> > >>
> > >> good
> > >>
> > >> place
> > >>
> > >> to discuss. I wrote google doc about SplitReader API which
> > >>
> > >> misses
> > >>
> > >> some
> > >>
> > >> details in the document. [2]
> > >>
> > >> 1.
> > >>
> > >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > >>
> > >> 2.
> > >>
> > >>
> > >>
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > >>
> > >> CC Stephan, Aljoscha, Piotrek, Becket
> > >>
> > >>
> > >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> <
> > mmyy1110@gmail.com>
> > >>
> > >> wrote:
> > >>
> > >> Hi Steven,
> > >> Thank you for the feedback. Please take a look at the document
> > >>
> > >> FLIP-27
> > >>
> > >> <
> > >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >>
> > >> which
> > >>
> > >> is updated recently. A lot of details of enumerator were added
> > >>
> > >> in
> > >>
> > >> this
> > >>
> > >> document. I think it would help.
> > >>
> > >> Steven Wu <st...@gmail.com> <st...@gmail.com> 于2019年3月28日周四
> > 下午12:52写道:
> > >>
> > >>
> > >> This proposal mentioned that SplitEnumerator might run on the
> > >> JobManager or
> > >> in a single task on a TaskManager.
> > >>
> > >> if enumerator is a single task on a taskmanager, then the job
> > >>
> > >> DAG
> > >>
> > >> can
> > >>
> > >> never
> > >> been embarrassingly parallel anymore. That will nullify the
> > >>
> > >> leverage
> > >>
> > >> of
> > >>
> > >> fine-grained recovery for embarrassingly parallel jobs.
> > >>
> > >> It's not clear to me what's the implication of running
> > >>
> > >> enumerator
> > >>
> > >> on
> > >>
> > >> the
> > >>
> > >> jobmanager. So I will leave that out for now.
> > >>
> > >> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> <
> > mmyy1110@gmail.com>
> > >>
> > >> wrote:
> > >>
> > >> Hi Stephan & Piotrek,
> > >>
> > >> Thank you for feedback.
> > >>
> > >> It seems that there are a lot of things to do in community.
> > >>
> > >> I
> > >>
> > >> am
> > >>
> > >> just
> > >>
> > >> afraid that this discussion may be forgotten since there so
> > >>
> > >> many
> > >>
> > >> proposals
> > >>
> > >> recently.
> > >> Anyway, wish to see the split topics soon :)
> > >>
> > >> Piotr Nowojski <pi...@da-platform.com> <pi...@da-platform.com>
> > 于2019年1月24日周四
> > >>
> > >> 下午8:21写道:
> > >>
> > >> Hi Biao!
> > >>
> > >> This discussion was stalled because of preparations for
> > >>
> > >> the
> > >>
> > >> open
> > >>
> > >> sourcing
> > >>
> > >> & merging Blink. I think before creating the tickets we
> > >>
> > >> should
> > >>
> > >> split this
> > >>
> > >> discussion into topics/areas outlined by Stephan and
> > >>
> > >> create
> > >>
> > >> Flips
> > >>
> > >> for
> > >>
> > >> that.
> > >>
> > >> I think there is no chance for this to be completed in
> > >>
> > >> couple
> > >>
> > >> of
> > >>
> > >> remaining
> > >>
> > >> weeks/1 month before 1.8 feature freeze, however it would
> > >>
> > >> be
> > >>
> > >> good
> > >>
> > >> to aim
> > >>
> > >> with those changes for 1.9.
> > >>
> > >> Piotrek
> > >>
> > >>
> > >> On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> <
> > mmyy1110@gmail.com>
> > >>
> > >> wrote:
> > >>
> > >> Hi community,
> > >> The summary of Stephan makes a lot sense to me. It is
> > >>
> > >> much
> > >>
> > >> clearer
> > >>
> > >> indeed
> > >>
> > >> after splitting the complex topic into small ones.
> > >> I was wondering is there any detail plan for next step?
> > >>
> > >> If
> > >>
> > >> not,
> > >>
> > >> I
> > >>
> > >> would
> > >>
> > >> like to push this thing forward by creating some JIRA
> > >>
> > >> issues.
> > >>
> > >> Another question is that should version 1.8 include
> > >>
> > >> these
> > >>
> > >> features?
> > >>
> > >> Stephan Ewen <se...@apache.org> <se...@apache.org> 于2018年12月1日周六
> > 上午4:20写道:
> > >>
> > >>
> > >> Thanks everyone for the lively discussion. Let me try
> > >>
> > >> to
> > >>
> > >> summarize
> > >>
> > >> where I
> > >>
> > >> see convergence in the discussion and open issues.
> > >> I'll try to group this by design aspect of the source.
> > >>
> > >> Please
> > >>
> > >> let me
> > >>
> > >> know
> > >>
> > >> if I got things wrong or missed something crucial here.
> > >>
> > >> For issues 1-3, if the below reflects the state of the
> > >>
> > >> discussion, I
> > >>
> > >> would
> > >>
> > >> try and update the FLIP in the next days.
> > >> For the remaining ones we need more discussion.
> > >>
> > >> I would suggest to fork each of these aspects into a
> > >>
> > >> separate
> > >>
> > >> mail
> > >>
> > >> thread,
> > >>
> > >> or will loose sight of the individual aspects.
> > >>
> > >> *(1) Separation of Split Enumerator and Split Reader*
> > >>
> > >>  - All seem to agree this is a good thing
> > >>  - Split Enumerator could in the end live on JobManager
> > >>
> > >> (and
> > >>
> > >> assign
> > >>
> > >> splits
> > >>
> > >> via RPC) or in a task (and assign splits via data
> > >>
> > >> streams)
> > >>
> > >>  - this discussion is orthogonal and should come later,
> > >>
> > >> when
> > >>
> > >> the
> > >>
> > >> interface
> > >>
> > >> is agreed upon.
> > >>
> > >> *(2) Split Readers for one or more splits*
> > >>
> > >>  - Discussion seems to agree that we need to support
> > >>
> > >> one
> > >>
> > >> reader
> > >>
> > >> that
> > >>
> > >> possibly handles multiple splits concurrently.
> > >>  - The requirement comes from sources where one
> > >>
> > >> poll()-style
> > >>
> > >> call
> > >>
> > >> fetches
> > >>
> > >> data from different splits / partitions
> > >>    --> example sources that require that would be for
> > >>
> > >> example
> > >>
> > >> Kafka,
> > >>
> > >> Pravega, Pulsar
> > >>
> > >>  - Could have one split reader per source, or multiple
> > >>
> > >> split
> > >>
> > >> readers
> > >>
> > >> that
> > >>
> > >> share the "poll()" function
> > >>  - To not make it too complicated, we can start with
> > >>
> > >> thinking
> > >>
> > >> about
> > >>
> > >> one
> > >>
> > >> split reader for all splits initially and see if that
> > >>
> > >> covers
> > >>
> > >> all
> > >>
> > >> requirements
> > >>
> > >> *(3) Threading model of the Split Reader*
> > >>
> > >>  - Most active part of the discussion ;-)
> > >>
> > >>  - A non-blocking way for Flink's task code to interact
> > >>
> > >> with
> > >>
> > >> the
> > >>
> > >> source
> > >>
> > >> is
> > >>
> > >> needed in order to a task runtime code based on a
> > >> single-threaded/actor-style task design
> > >>    --> I personally am a big proponent of that, it will
> > >>
> > >> help
> > >>
> > >> with
> > >>
> > >> well-behaved checkpoints, efficiency, and simpler yet
> > >>
> > >> more
> > >>
> > >> robust
> > >>
> > >> runtime
> > >>
> > >> code
> > >>
> > >>  - Users care about simple abstraction, so as a
> > >>
> > >> subclass
> > >>
> > >> of
> > >>
> > >> SplitReader
> > >>
> > >> (non-blocking / async) we need to have a
> > >>
> > >> BlockingSplitReader
> > >>
> > >> which
> > >>
> > >> will
> > >>
> > >> form the basis of most source implementations.
> > >>
> > >> BlockingSplitReader
> > >>
> > >> lets
> > >>
> > >> users do blocking simple poll() calls.
> > >>  - The BlockingSplitReader would spawn a thread (or
> > >>
> > >> more)
> > >>
> > >> and
> > >>
> > >> the
> > >>
> > >> thread(s) can make blocking calls and hand over data
> > >>
> > >> buffers
> > >>
> > >> via
> > >>
> > >> a
> > >>
> > >> blocking
> > >>
> > >> queue
> > >>  - This should allow us to cover both, a fully async
> > >>
> > >> runtime,
> > >>
> > >> and a
> > >>
> > >> simple
> > >>
> > >> blocking interface for users.
> > >>  - This is actually very similar to how the Kafka
> > >>
> > >> connectors
> > >>
> > >> work.
> > >>
> > >> Kafka
> > >>
> > >> 9+ with one thread, Kafka 8 with multiple threads
> > >>
> > >>  - On the base SplitReader (the async one), the
> > >>
> > >> non-blocking
> > >>
> > >> method
> > >>
> > >> that
> > >>
> > >> gets the next chunk of data would signal data
> > >>
> > >> availability
> > >>
> > >> via
> > >>
> > >> a
> > >>
> > >> CompletableFuture, because that gives the best
> > >>
> > >> flexibility
> > >>
> > >> (can
> > >>
> > >> await
> > >>
> > >> completion or register notification handlers).
> > >>  - The source task would register a "thenHandle()" (or
> > >>
> > >> similar)
> > >>
> > >> on the
> > >>
> > >> future to put a "take next data" task into the
> > >>
> > >> actor-style
> > >>
> > >> mailbox
> > >>
> > >> *(4) Split Enumeration and Assignment*
> > >>
> > >>  - Splits may be generated lazily, both in cases where
> > >>
> > >> there
> > >>
> > >> is a
> > >>
> > >> limited
> > >>
> > >> number of splits (but very many), or splits are
> > >>
> > >> discovered
> > >>
> > >> over
> > >>
> > >> time
> > >>
> > >>  - Assignment should also be lazy, to get better load
> > >>
> > >> balancing
> > >>
> > >>  - Assignment needs support locality preferences
> > >>
> > >>  - Possible design based on discussion so far:
> > >>
> > >>    --> SplitReader has a method "addSplits(SplitT...)"
> > >>
> > >> to
> > >>
> > >> add
> > >>
> > >> one or
> > >>
> > >> more
> > >>
> > >> splits. Some split readers might assume they have only
> > >>
> > >> one
> > >>
> > >> split
> > >>
> > >> ever,
> > >>
> > >> concurrently, others assume multiple splits. (Note:
> > >>
> > >> idea
> > >>
> > >> behind
> > >>
> > >> being
> > >>
> > >> able
> > >>
> > >> to add multiple splits at the same time is to ease
> > >>
> > >> startup
> > >>
> > >> where
> > >>
> > >> multiple
> > >>
> > >> splits may be assigned instantly.)
> > >>    --> SplitReader has a context object on which it can
> > >>
> > >> call
> > >>
> > >> indicate
> > >>
> > >> when
> > >>
> > >> splits are completed. The enumerator gets that
> > >>
> > >> notification and
> > >>
> > >> can
> > >>
> > >> use
> > >>
> > >> to
> > >>
> > >> decide when to assign new splits. This should help both
> > >>
> > >> in
> > >>
> > >> cases
> > >>
> > >> of
> > >>
> > >> sources
> > >>
> > >> that take splits lazily (file readers) and in case the
> > >>
> > >> source
> > >>
> > >> needs to
> > >>
> > >> preserve a partial order between splits (Kinesis,
> > >>
> > >> Pravega,
> > >>
> > >> Pulsar may
> > >>
> > >> need
> > >>
> > >> that).
> > >>    --> SplitEnumerator gets notification when
> > >>
> > >> SplitReaders
> > >>
> > >> start
> > >>
> > >> and
> > >>
> > >> when
> > >>
> > >> they finish splits. They can decide at that moment to
> > >>
> > >> push
> > >>
> > >> more
> > >>
> > >> splits
> > >>
> > >> to
> > >>
> > >> that reader
> > >>    --> The SplitEnumerator should probably be aware of
> > >>
> > >> the
> > >>
> > >> source
> > >>
> > >> parallelism, to build its initial distribution.
> > >>
> > >>  - Open question: Should the source expose something
> > >>
> > >> like
> > >>
> > >> "host
> > >>
> > >> preferences", so that yarn/mesos/k8s can take this into
> > >>
> > >> account
> > >>
> > >> when
> > >>
> > >> selecting a node to start a TM on?
> > >>
> > >> *(5) Watermarks and event time alignment*
> > >>
> > >>  - Watermark generation, as well as idleness, needs to
> > >>
> > >> be
> > >>
> > >> per
> > >>
> > >> split
> > >>
> > >> (like
> > >>
> > >> currently in the Kafka Source, per partition)
> > >>  - It is desirable to support optional
> > >>
> > >> event-time-alignment,
> > >>
> > >> meaning
> > >>
> > >> that
> > >>
> > >> splits that are ahead are back-pressured or temporarily
> > >>
> > >> unsubscribed
> > >>
> > >>  - I think i would be desirable to encapsulate
> > >>
> > >> watermark
> > >>
> > >> generation
> > >>
> > >> logic
> > >>
> > >> in watermark generators, for a separation of concerns.
> > >>
> > >> The
> > >>
> > >> watermark
> > >>
> > >> generators should run per split.
> > >>  - Using watermark generators would also help with
> > >>
> > >> another
> > >>
> > >> problem of
> > >>
> > >> the
> > >>
> > >> suggested interface, namely supporting non-periodic
> > >>
> > >> watermarks
> > >>
> > >> efficiently.
> > >>
> > >>  - Need a way to "dispatch" next record to different
> > >>
> > >> watermark
> > >>
> > >> generators
> > >>
> > >>  - Need a way to tell SplitReader to "suspend" a split
> > >>
> > >> until a
> > >>
> > >> certain
> > >>
> > >> watermark is reached (event time backpressure)
> > >>  - This would in fact be not needed (and thus simpler)
> > >>
> > >> if
> > >>
> > >> we
> > >>
> > >> had
> > >>
> > >> a
> > >>
> > >> SplitReader per split and may be a reason to re-open
> > >>
> > >> that
> > >>
> > >> discussion
> > >>
> > >> *(6) Watermarks across splits and in the Split
> > >>
> > >> Enumerator*
> > >>
> > >>  - The split enumerator may need some watermark
> > >>
> > >> awareness,
> > >>
> > >> which
> > >>
> > >> should
> > >>
> > >> be
> > >>
> > >> purely based on split metadata (like create timestamp
> > >>
> > >> of
> > >>
> > >> file
> > >>
> > >> splits)
> > >>
> > >>  - If there are still more splits with overlapping
> > >>
> > >> event
> > >>
> > >> time
> > >>
> > >> range
> > >>
> > >> for
> > >>
> > >> a
> > >>
> > >> split reader, then that split reader should not advance
> > >>
> > >> the
> > >>
> > >> watermark
> > >>
> > >> within the split beyond the overlap boundary. Otherwise
> > >>
> > >> future
> > >>
> > >> splits
> > >>
> > >> will
> > >>
> > >> produce late data.
> > >>
> > >>  - One way to approach this could be that the split
> > >>
> > >> enumerator
> > >>
> > >> may
> > >>
> > >> send
> > >>
> > >> watermarks to the readers, and the readers cannot emit
> > >>
> > >> watermarks
> > >>
> > >> beyond
> > >>
> > >> that received watermark.
> > >>  - Many split enumerators would simply immediately send
> > >>
> > >> Long.MAX
> > >>
> > >> out
> > >>
> > >> and
> > >>
> > >> leave the progress purely to the split readers.
> > >>
> > >>  - For event-time alignment / split back pressure, this
> > >>
> > >> begs
> > >>
> > >> the
> > >>
> > >> question
> > >>
> > >> how we can avoid deadlocks that may arise when splits
> > >>
> > >> are
> > >>
> > >> suspended
> > >>
> > >> for
> > >>
> > >> event time back pressure,
> > >>
> > >> *(7) Batch and streaming Unification*
> > >>
> > >>  - Functionality wise, the above design should support
> > >>
> > >> both
> > >>
> > >>  - Batch often (mostly) does not care about reading "in
> > >>
> > >> order"
> > >>
> > >> and
> > >>
> > >> generating watermarks
> > >>    --> Might use different enumerator logic that is
> > >>
> > >> more
> > >>
> > >> locality
> > >>
> > >> aware
> > >>
> > >> and ignores event time order
> > >>    --> Does not generate watermarks
> > >>  - Would be great if bounded sources could be
> > >>
> > >> identified
> > >>
> > >> at
> > >>
> > >> compile
> > >>
> > >> time,
> > >>
> > >> so that "env.addBoundedSource(...)" is type safe and
> > >>
> > >> can
> > >>
> > >> return a
> > >>
> > >> "BoundedDataStream".
> > >>  - Possible to defer this discussion until later
> > >>
> > >> *Miscellaneous Comments*
> > >>
> > >>  - Should the source have a TypeInformation for the
> > >>
> > >> produced
> > >>
> > >> type,
> > >>
> > >> instead
> > >>
> > >> of a serializer? We need a type information in the
> > >>
> > >> stream
> > >>
> > >> anyways, and
> > >>
> > >> can
> > >>
> > >> derive the serializer from that. Plus, creating the
> > >>
> > >> serializer
> > >>
> > >> should
> > >>
> > >> respect the ExecutionConfig.
> > >>
> > >>  - The TypeSerializer interface is very powerful but
> > >>
> > >> also
> > >>
> > >> not
> > >>
> > >> easy to
> > >>
> > >> implement. Its purpose is to handle data super
> > >>
> > >> efficiently,
> > >>
> > >> support
> > >>
> > >> flexible ways of evolution, etc.
> > >>  For metadata I would suggest to look at the
> > >>
> > >> SimpleVersionedSerializer
> > >>
> > >> instead, which is used for example for checkpoint
> > >>
> > >> master
> > >>
> > >> hooks,
> > >>
> > >> or for
> > >>
> > >> the
> > >>
> > >> streaming file sink. I think that is is a good match
> > >>
> > >> for
> > >>
> > >> cases
> > >>
> > >> where
> > >>
> > >> we
> > >>
> > >> do
> > >>
> > >> not need more than ser/deser (no copy, etc.) and don't
> > >>
> > >> need to
> > >>
> > >> push
> > >>
> > >> versioning out of the serialization paths for best
> > >>
> > >> performance
> > >>
> > >> (as in
> > >>
> > >> the
> > >>
> > >> TypeSerializer)
> > >>
> > >>
> > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > k.kloudas@data-artisans.com>
> > >> wrote:
> > >>
> > >>
> > >> Hi Biao,
> > >>
> > >> Thanks for the answer!
> > >>
> > >> So given the multi-threaded readers, now we have as
> > >>
> > >> open
> > >>
> > >> questions:
> > >>
> > >> 1) How do we let the checkpoints pass through our
> > >>
> > >> multi-threaded
> > >>
> > >> reader
> > >>
> > >> operator?
> > >>
> > >> 2) Do we have separate reader and source operators or
> > >>
> > >> not? In
> > >>
> > >> the
> > >>
> > >> strategy
> > >>
> > >> that has a separate source, the source operator has a
> > >>
> > >> parallelism of
> > >>
> > >> 1
> > >>
> > >> and
> > >>
> > >> is responsible for split recovery only.
> > >>
> > >> For the first one, given also the constraints
> > >>
> > >> (blocking,
> > >>
> > >> finite
> > >>
> > >> queues,
> > >>
> > >> etc), I do not have an answer yet.
> > >>
> > >> For the 2nd, I think that we should go with separate
> > >>
> > >> operators
> > >>
> > >> for
> > >>
> > >> the
> > >>
> > >> source and the readers, for the following reasons:
> > >>
> > >> 1) This is more aligned with a potential future
> > >>
> > >> improvement
> > >>
> > >> where the
> > >>
> > >> split
> > >>
> > >> discovery becomes a responsibility of the JobManager
> > >>
> > >> and
> > >>
> > >> readers are
> > >>
> > >> pooling more work from the JM.
> > >>
> > >> 2) The source is going to be the "single point of
> > >>
> > >> truth".
> > >>
> > >> It
> > >>
> > >> will
> > >>
> > >> know
> > >>
> > >> what
> > >>
> > >> has been processed and what not. If the source and the
> > >>
> > >> readers
> > >>
> > >> are a
> > >>
> > >> single
> > >>
> > >> operator with parallelism > 1, or in general, if the
> > >>
> > >> split
> > >>
> > >> discovery
> > >>
> > >> is
> > >>
> > >> done by each task individually, then:
> > >>   i) we have to have a deterministic scheme for each
> > >>
> > >> reader to
> > >>
> > >> assign
> > >>
> > >> splits to itself (e.g. mod subtaskId). This is not
> > >>
> > >> necessarily
> > >>
> > >> trivial
> > >>
> > >> for
> > >>
> > >> all sources.
> > >>   ii) each reader would have to keep a copy of all its
> > >>
> > >> processed
> > >>
> > >> slpits
> > >>
> > >>   iii) the state has to be a union state with a
> > >>
> > >> non-trivial
> > >>
> > >> merging
> > >>
> > >> logic
> > >>
> > >> in order to support rescaling.
> > >>
> > >> Two additional points that you raised above:
> > >>
> > >> i) The point that you raised that we need to keep all
> > >>
> > >> splits
> > >>
> > >> (processed
> > >>
> > >> and
> > >>
> > >> not-processed) I think is a bit of a strong
> > >>
> > >> requirement.
> > >>
> > >> This
> > >>
> > >> would
> > >>
> > >> imply
> > >>
> > >> that for infinite sources the state will grow
> > >>
> > >> indefinitely.
> > >>
> > >> This is
> > >>
> > >> problem
> > >>
> > >> is even more pronounced if we do not have a single
> > >>
> > >> source
> > >>
> > >> that
> > >>
> > >> assigns
> > >>
> > >> splits to readers, as each reader will have its own
> > >>
> > >> copy
> > >>
> > >> of
> > >>
> > >> the
> > >>
> > >> state.
> > >>
> > >> ii) it is true that for finite sources we need to
> > >>
> > >> somehow
> > >>
> > >> not
> > >>
> > >> close
> > >>
> > >> the
> > >>
> > >> readers when the source/split discoverer finishes. The
> > >> ContinuousFileReaderOperator has a work-around for
> > >>
> > >> that.
> > >>
> > >> It is
> > >>
> > >> not
> > >>
> > >> elegant,
> > >>
> > >> and checkpoints are not emitted after closing the
> > >>
> > >> source,
> > >>
> > >> but
> > >>
> > >> this, I
> > >>
> > >> believe, is a bigger problem which requires more
> > >>
> > >> changes
> > >>
> > >> than
> > >>
> > >> just
> > >>
> > >> refactoring the source interface.
> > >>
> > >> Cheers,
> > >> Kostas
> > >>
> > >>
> > >>
> >
> >
>


-- 
Best, Jingsong Lee

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Jark Wu <im...@gmail.com>.
I agree with Dawid's point that the boundedness information should come
from the source itself (e.g. the end timestamp), not through
env.boundedSouce()/continuousSource().
I think if we want to support something like `env.source()` that derive the
execution mode from source, `supportsBoundedness(Boundedness)`
method is not enough, because we don't know whether it is bounded or not.

Best,
Jark


On Mon, 9 Dec 2019 at 22:21, Dawid Wysakowicz <dw...@apache.org>
wrote:

> One more thing. In the current proposal, with the
> supportsBoundedness(Boundedness) method and the boundedness coming from
> either continuousSource or boundedSource I could not find how this
> information is fed back to the SplitEnumerator.
>
> Best,
>
> Dawid
>
> On 09/12/2019 13:52, Becket Qin wrote:
> > Hi Dawid,
> >
> > Thanks for the comments. This actually brings another relevant question
> > about what does a "bounded source" imply. I actually had the same
> > impression when I look at the Source API. Here is what I understand after
> > some discussion with Stephan. The bounded source has the following
> impacts.
> >
> > 1. API validity.
> > - A bounded source generates a bounded stream so some operations that
> only
> > works for bounded records would be performed, e.g. sort.
> > - To expose these bounded stream only APIs, there are two options:
> >      a. Add them to the DataStream API and throw exception if a method is
> > called on an unbounded stream.
> >      b. Create a BoundedDataStream class which is returned from
> > env.boundedSource(), while DataStream is returned from
> env.continousSource().
> > Note that this cannot be done by having single env.source(theSource) even
> > the Source has a getBoundedness() method.
> >
> > 2. Scheduling
> > - A bounded source could be computed stage by stage without bringing up
> all
> > the tasks at the same time.
> >
> > 3. Operator behaviors
> > - A bounded source indicates the records are finite so some operators can
> > wait until it receives all the records before it starts the processing.
> >
> > In the above impact, only 1 is relevant to the API design. And the
> current
> > proposal in FLIP-27 is following 1.b.
> >
> > // boundedness depends of source property, imo this should always be
> >> preferred
> >>
> >
> > DataStream<MyType> stream = env.source(theSource);
> >
> >
> > In your proposal, does DataStream have bounded stream only methods? It
> > looks it should have, otherwise passing a bounded Source to env.source()
> > would be confusing. In that case, we will essentially do 1.a if an
> > unbounded Source is created from env.source(unboundedSource).
> >
> > If we have the methods only supported for bounded streams in DataStream,
> it
> > seems a little weird to have a separate BoundedDataStream interface.
> >
> > Am I understand it correctly?
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> >
> >
> > On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <dw...@apache.org>
> > wrote:
> >
> >> Hi all,
> >>
> >> Really well written proposal and very important one. I must admit I have
> >> not understood all the intricacies of it yet.
> >>
> >> One question I have though is about where does the information about
> >> boundedness come from. I think in most cases it is a property of the
> >> source. As you described it might be e.g. end offset, a flag should it
> >> monitor new splits etc. I think it would be a really nice use case to be
> >> able to say:
> >>
> >> new KafkaSource().readUntil(long timestamp),
> >>
> >> which could work as an "end offset". Moreover I think all Bounded
> sources
> >> support continuous mode, but no intrinsically continuous source support
> the
> >> Bounded mode. If I understood the proposal correctly it suggest the
> >> boundedness sort of "comes" from the outside of the source, from the
> >> invokation of either boundedStream or continousSource.
> >>
> >> I am wondering if it would make sense to actually change the method
> >>
> >> boolean Source#supportsBoundedness(Boundedness)
> >>
> >> to
> >>
> >> Boundedness Source#getBoundedness().
> >>
> >> As for the methods #boundedSource, #continousSource, assuming the
> >> boundedness is property of the source they do not affect how the
> enumerator
> >> works, but mostly how the dag is scheduled, right? I am not against
> those
> >> methods, but I think it is a very specific use case to actually override
> >> the property of the source. In general I would expect users to only call
> >> env.source(theSource), where the source tells if it is bounded or not. I
> >> would suggest considering following set of methods:
> >>
> >> // boundedness depends of source property, imo this should always be
> preferred
> >>
> >> DataStream<MyType> stream = env.source(theSource);
> >>
> >>
> >> // always continous execution, whether bounded or unbounded source
> >>
> >> DataStream<MyType> boundedStream = env.continousSource(theSource);
> >>
> >> // imo this would make sense if the BoundedDataStream provides
> additional features unavailable for continous mode
> >> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
> >>
> >>
> >> Best,
> >>
> >> Dawid
> >>
> >>
> >> On 04/12/2019 11:25, Stephan Ewen wrote:
> >>
> >> Thanks, Becket, for updating this.
> >>
> >> I agree with moving the aspects you mentioned into separate FLIPs - this
> >> one way becoming unwieldy in size.
> >>
> >> +1 to the FLIP in its current state. Its a very detailed write-up,
> nicely
> >> done!
> >>
> >> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <be...@gmail.com> <
> becket.qin@gmail.com> wrote:
> >>
> >>
> >> Hi all,
> >>
> >> Sorry for the long belated update. I have updated FLIP-27 wiki page with
> >> the latest proposals. Some noticeable changes include:
> >> 1. A new generic communication mechanism between SplitEnumerator and
> >> SourceReader.
> >> 2. Some detail API method signature changes.
> >>
> >> We left a few things out of this FLIP and will address them in separate
> >> FLIPs. Including:
> >> 1. Per split event time.
> >> 2. Event time alignment.
> >> 3. Fine grained failover for SplitEnumerator failure.
> >>
> >> Please let us know if you have any question.
> >>
> >> Thanks,
> >>
> >> Jiangjie (Becket) Qin
> >>
> >> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> <
> sewen@apache.org> wrote:
> >>
> >>
> >> Hi  Łukasz!
> >>
> >> Becket and me are working hard on figuring out the last details and
> >> implementing the first PoC. We would update the FLIP hopefully next
> week.
> >>
> >> There is a fair chance that a first version of this will be in 1.10, but
> >>
> >> I
> >>
> >> think it will take another release to battle test it and migrate the
> >> connectors.
> >>
> >> Best,
> >> Stephan
> >>
> >>
> >>
> >>
> >> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <lj...@touk.pl> <
> ljd@touk.pl>
> >>
> >> wrote:
> >>
> >> Hi,
> >>
> >> This proposal looks very promising for us. Do you have any plans in
> >>
> >> which
> >>
> >> Flink release it is going to be released? We are thinking on using a
> >>
> >> Data
> >>
> >> Set API for our future use cases but on the other hand Data Set API is
> >> going to be deprecated so using proposed bounded data streams solution
> >> could be more viable in the long term.
> >>
> >> Thanks,
> >> Łukasz
> >>
> >> On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> <
> thomas.weise@gmail.com> wrote:
> >>
> >> Thanks for putting together this proposal!
> >>
> >> I see that the "Per Split Event Time" and "Event Time Alignment"
> >>
> >> sections
> >>
> >> are still TBD.
> >>
> >> It would probably be good to flesh those out a bit before proceeding
> >>
> >> too
> >>
> >> far
> >>
> >> as the event time alignment will probably influence the interaction
> >>
> >> with
> >>
> >> the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
> >> output).
> >>
> >> We currently have only one implementation for event time alignment in
> >>
> >> the
> >>
> >> Kinesis consumer. The synchronization in that case takes place as the
> >>
> >> last
> >>
> >> step before records are emitted downstream (RecordEmitter). With the
> >> currently proposed interfaces, the equivalent can be implemented in
> >>
> >> the
> >>
> >> reader loop, although note that in the Kinesis consumer the per shard
> >> threads push records.
> >>
> >> Synchronization has not been implemented for the Kafka consumer yet.
> >> https://issues.apache.org/jira/browse/FLINK-12675
> >>
> >> When I looked at it, I realized that the implementation will look
> >>
> >> quite
> >>
> >> different
> >> from Kinesis because it needs to take place in the pull part, where
> >>
> >> records
> >>
> >> are taken from the Kafka client. Due to the multiplexing it cannot be
> >>
> >> done
> >>
> >> by blocking the split thread like it currently works for Kinesis.
> >>
> >> Reading
> >>
> >> from individual Kafka partitions needs to be controlled via
> >>
> >> pause/resume
> >>
> >> on the Kafka client.
> >>
> >> To take on that responsibility the split thread would need to be
> >>
> >> aware
> >>
> >> of
> >>
> >> the
> >> watermarks or at least whether it should or should not continue to
> >>
> >> consume
> >>
> >> a given split and this may require a different SourceReader or
> >>
> >> SourceOutput
> >>
> >> interface.
> >>
> >> Thanks,
> >> Thomas
> >>
> >>
> >> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> <
> mmyy1110@gmail.com> wrote:
> >>
> >>
> >> Hi Stephan,
> >>
> >> Thank you for feedback!
> >> Will take a look at your branch before public discussing.
> >>
> >>
> >> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org> <
> sewen@apache.org>
> >>
> >> wrote:
> >>
> >> Hi Biao!
> >>
> >> Thanks for reviving this. I would like to join this discussion,
> >>
> >> but
> >>
> >> am
> >>
> >> quite occupied with the 1.9 release, so can we maybe pause this
> >>
> >> discussion
> >>
> >> for a week or so?
> >>
> >> In the meantime I can share some suggestion based on prior
> >>
> >> experiments:
> >>
> >> How to do watermarks / timestamp extractors in a simpler and more
> >>
> >> flexible
> >>
> >> way. I think that part is quite promising should be part of the
> >>
> >> new
> >>
> >> source
> >>
> >> interface.
> >>
> >>
> >>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> >>
> >>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> >>
> >> Some experiments on how to build the source reader and its
> >>
> >> library
> >>
> >> for
> >>
> >> common threading/split patterns:
> >>
> >>
> >>
> >>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> >>
> >> Best,
> >> Stephan
> >>
> >>
> >> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> <
> mmyy1110@gmail.com>
> >>
> >> wrote:
> >>
> >> Hi devs,
> >>
> >> Since 1.9 is nearly released, I think we could get back to
> >>
> >> FLIP-27.
> >>
> >> I
> >>
> >> believe it should be included in 1.10.
> >>
> >> There are so many things mentioned in document of FLIP-27. [1] I
> >>
> >> think
> >>
> >> we'd better discuss them separately. However the wiki is not a
> >>
> >> good
> >>
> >> place
> >>
> >> to discuss. I wrote google doc about SplitReader API which
> >>
> >> misses
> >>
> >> some
> >>
> >> details in the document. [2]
> >>
> >> 1.
> >>
> >>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >>
> >> 2.
> >>
> >>
> >>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> >>
> >> CC Stephan, Aljoscha, Piotrek, Becket
> >>
> >>
> >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> <
> mmyy1110@gmail.com>
> >>
> >> wrote:
> >>
> >> Hi Steven,
> >> Thank you for the feedback. Please take a look at the document
> >>
> >> FLIP-27
> >>
> >> <
> >>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>
> >> which
> >>
> >> is updated recently. A lot of details of enumerator were added
> >>
> >> in
> >>
> >> this
> >>
> >> document. I think it would help.
> >>
> >> Steven Wu <st...@gmail.com> <st...@gmail.com> 于2019年3月28日周四
> 下午12:52写道:
> >>
> >>
> >> This proposal mentioned that SplitEnumerator might run on the
> >> JobManager or
> >> in a single task on a TaskManager.
> >>
> >> if enumerator is a single task on a taskmanager, then the job
> >>
> >> DAG
> >>
> >> can
> >>
> >> never
> >> been embarrassingly parallel anymore. That will nullify the
> >>
> >> leverage
> >>
> >> of
> >>
> >> fine-grained recovery for embarrassingly parallel jobs.
> >>
> >> It's not clear to me what's the implication of running
> >>
> >> enumerator
> >>
> >> on
> >>
> >> the
> >>
> >> jobmanager. So I will leave that out for now.
> >>
> >> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> <
> mmyy1110@gmail.com>
> >>
> >> wrote:
> >>
> >> Hi Stephan & Piotrek,
> >>
> >> Thank you for feedback.
> >>
> >> It seems that there are a lot of things to do in community.
> >>
> >> I
> >>
> >> am
> >>
> >> just
> >>
> >> afraid that this discussion may be forgotten since there so
> >>
> >> many
> >>
> >> proposals
> >>
> >> recently.
> >> Anyway, wish to see the split topics soon :)
> >>
> >> Piotr Nowojski <pi...@da-platform.com> <pi...@da-platform.com>
> 于2019年1月24日周四
> >>
> >> 下午8:21写道:
> >>
> >> Hi Biao!
> >>
> >> This discussion was stalled because of preparations for
> >>
> >> the
> >>
> >> open
> >>
> >> sourcing
> >>
> >> & merging Blink. I think before creating the tickets we
> >>
> >> should
> >>
> >> split this
> >>
> >> discussion into topics/areas outlined by Stephan and
> >>
> >> create
> >>
> >> Flips
> >>
> >> for
> >>
> >> that.
> >>
> >> I think there is no chance for this to be completed in
> >>
> >> couple
> >>
> >> of
> >>
> >> remaining
> >>
> >> weeks/1 month before 1.8 feature freeze, however it would
> >>
> >> be
> >>
> >> good
> >>
> >> to aim
> >>
> >> with those changes for 1.9.
> >>
> >> Piotrek
> >>
> >>
> >> On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> <
> mmyy1110@gmail.com>
> >>
> >> wrote:
> >>
> >> Hi community,
> >> The summary of Stephan makes a lot sense to me. It is
> >>
> >> much
> >>
> >> clearer
> >>
> >> indeed
> >>
> >> after splitting the complex topic into small ones.
> >> I was wondering is there any detail plan for next step?
> >>
> >> If
> >>
> >> not,
> >>
> >> I
> >>
> >> would
> >>
> >> like to push this thing forward by creating some JIRA
> >>
> >> issues.
> >>
> >> Another question is that should version 1.8 include
> >>
> >> these
> >>
> >> features?
> >>
> >> Stephan Ewen <se...@apache.org> <se...@apache.org> 于2018年12月1日周六
> 上午4:20写道:
> >>
> >>
> >> Thanks everyone for the lively discussion. Let me try
> >>
> >> to
> >>
> >> summarize
> >>
> >> where I
> >>
> >> see convergence in the discussion and open issues.
> >> I'll try to group this by design aspect of the source.
> >>
> >> Please
> >>
> >> let me
> >>
> >> know
> >>
> >> if I got things wrong or missed something crucial here.
> >>
> >> For issues 1-3, if the below reflects the state of the
> >>
> >> discussion, I
> >>
> >> would
> >>
> >> try and update the FLIP in the next days.
> >> For the remaining ones we need more discussion.
> >>
> >> I would suggest to fork each of these aspects into a
> >>
> >> separate
> >>
> >> mail
> >>
> >> thread,
> >>
> >> or will loose sight of the individual aspects.
> >>
> >> *(1) Separation of Split Enumerator and Split Reader*
> >>
> >>  - All seem to agree this is a good thing
> >>  - Split Enumerator could in the end live on JobManager
> >>
> >> (and
> >>
> >> assign
> >>
> >> splits
> >>
> >> via RPC) or in a task (and assign splits via data
> >>
> >> streams)
> >>
> >>  - this discussion is orthogonal and should come later,
> >>
> >> when
> >>
> >> the
> >>
> >> interface
> >>
> >> is agreed upon.
> >>
> >> *(2) Split Readers for one or more splits*
> >>
> >>  - Discussion seems to agree that we need to support
> >>
> >> one
> >>
> >> reader
> >>
> >> that
> >>
> >> possibly handles multiple splits concurrently.
> >>  - The requirement comes from sources where one
> >>
> >> poll()-style
> >>
> >> call
> >>
> >> fetches
> >>
> >> data from different splits / partitions
> >>    --> example sources that require that would be for
> >>
> >> example
> >>
> >> Kafka,
> >>
> >> Pravega, Pulsar
> >>
> >>  - Could have one split reader per source, or multiple
> >>
> >> split
> >>
> >> readers
> >>
> >> that
> >>
> >> share the "poll()" function
> >>  - To not make it too complicated, we can start with
> >>
> >> thinking
> >>
> >> about
> >>
> >> one
> >>
> >> split reader for all splits initially and see if that
> >>
> >> covers
> >>
> >> all
> >>
> >> requirements
> >>
> >> *(3) Threading model of the Split Reader*
> >>
> >>  - Most active part of the discussion ;-)
> >>
> >>  - A non-blocking way for Flink's task code to interact
> >>
> >> with
> >>
> >> the
> >>
> >> source
> >>
> >> is
> >>
> >> needed in order to a task runtime code based on a
> >> single-threaded/actor-style task design
> >>    --> I personally am a big proponent of that, it will
> >>
> >> help
> >>
> >> with
> >>
> >> well-behaved checkpoints, efficiency, and simpler yet
> >>
> >> more
> >>
> >> robust
> >>
> >> runtime
> >>
> >> code
> >>
> >>  - Users care about simple abstraction, so as a
> >>
> >> subclass
> >>
> >> of
> >>
> >> SplitReader
> >>
> >> (non-blocking / async) we need to have a
> >>
> >> BlockingSplitReader
> >>
> >> which
> >>
> >> will
> >>
> >> form the basis of most source implementations.
> >>
> >> BlockingSplitReader
> >>
> >> lets
> >>
> >> users do blocking simple poll() calls.
> >>  - The BlockingSplitReader would spawn a thread (or
> >>
> >> more)
> >>
> >> and
> >>
> >> the
> >>
> >> thread(s) can make blocking calls and hand over data
> >>
> >> buffers
> >>
> >> via
> >>
> >> a
> >>
> >> blocking
> >>
> >> queue
> >>  - This should allow us to cover both, a fully async
> >>
> >> runtime,
> >>
> >> and a
> >>
> >> simple
> >>
> >> blocking interface for users.
> >>  - This is actually very similar to how the Kafka
> >>
> >> connectors
> >>
> >> work.
> >>
> >> Kafka
> >>
> >> 9+ with one thread, Kafka 8 with multiple threads
> >>
> >>  - On the base SplitReader (the async one), the
> >>
> >> non-blocking
> >>
> >> method
> >>
> >> that
> >>
> >> gets the next chunk of data would signal data
> >>
> >> availability
> >>
> >> via
> >>
> >> a
> >>
> >> CompletableFuture, because that gives the best
> >>
> >> flexibility
> >>
> >> (can
> >>
> >> await
> >>
> >> completion or register notification handlers).
> >>  - The source task would register a "thenHandle()" (or
> >>
> >> similar)
> >>
> >> on the
> >>
> >> future to put a "take next data" task into the
> >>
> >> actor-style
> >>
> >> mailbox
> >>
> >> *(4) Split Enumeration and Assignment*
> >>
> >>  - Splits may be generated lazily, both in cases where
> >>
> >> there
> >>
> >> is a
> >>
> >> limited
> >>
> >> number of splits (but very many), or splits are
> >>
> >> discovered
> >>
> >> over
> >>
> >> time
> >>
> >>  - Assignment should also be lazy, to get better load
> >>
> >> balancing
> >>
> >>  - Assignment needs support locality preferences
> >>
> >>  - Possible design based on discussion so far:
> >>
> >>    --> SplitReader has a method "addSplits(SplitT...)"
> >>
> >> to
> >>
> >> add
> >>
> >> one or
> >>
> >> more
> >>
> >> splits. Some split readers might assume they have only
> >>
> >> one
> >>
> >> split
> >>
> >> ever,
> >>
> >> concurrently, others assume multiple splits. (Note:
> >>
> >> idea
> >>
> >> behind
> >>
> >> being
> >>
> >> able
> >>
> >> to add multiple splits at the same time is to ease
> >>
> >> startup
> >>
> >> where
> >>
> >> multiple
> >>
> >> splits may be assigned instantly.)
> >>    --> SplitReader has a context object on which it can
> >>
> >> call
> >>
> >> indicate
> >>
> >> when
> >>
> >> splits are completed. The enumerator gets that
> >>
> >> notification and
> >>
> >> can
> >>
> >> use
> >>
> >> to
> >>
> >> decide when to assign new splits. This should help both
> >>
> >> in
> >>
> >> cases
> >>
> >> of
> >>
> >> sources
> >>
> >> that take splits lazily (file readers) and in case the
> >>
> >> source
> >>
> >> needs to
> >>
> >> preserve a partial order between splits (Kinesis,
> >>
> >> Pravega,
> >>
> >> Pulsar may
> >>
> >> need
> >>
> >> that).
> >>    --> SplitEnumerator gets notification when
> >>
> >> SplitReaders
> >>
> >> start
> >>
> >> and
> >>
> >> when
> >>
> >> they finish splits. They can decide at that moment to
> >>
> >> push
> >>
> >> more
> >>
> >> splits
> >>
> >> to
> >>
> >> that reader
> >>    --> The SplitEnumerator should probably be aware of
> >>
> >> the
> >>
> >> source
> >>
> >> parallelism, to build its initial distribution.
> >>
> >>  - Open question: Should the source expose something
> >>
> >> like
> >>
> >> "host
> >>
> >> preferences", so that yarn/mesos/k8s can take this into
> >>
> >> account
> >>
> >> when
> >>
> >> selecting a node to start a TM on?
> >>
> >> *(5) Watermarks and event time alignment*
> >>
> >>  - Watermark generation, as well as idleness, needs to
> >>
> >> be
> >>
> >> per
> >>
> >> split
> >>
> >> (like
> >>
> >> currently in the Kafka Source, per partition)
> >>  - It is desirable to support optional
> >>
> >> event-time-alignment,
> >>
> >> meaning
> >>
> >> that
> >>
> >> splits that are ahead are back-pressured or temporarily
> >>
> >> unsubscribed
> >>
> >>  - I think i would be desirable to encapsulate
> >>
> >> watermark
> >>
> >> generation
> >>
> >> logic
> >>
> >> in watermark generators, for a separation of concerns.
> >>
> >> The
> >>
> >> watermark
> >>
> >> generators should run per split.
> >>  - Using watermark generators would also help with
> >>
> >> another
> >>
> >> problem of
> >>
> >> the
> >>
> >> suggested interface, namely supporting non-periodic
> >>
> >> watermarks
> >>
> >> efficiently.
> >>
> >>  - Need a way to "dispatch" next record to different
> >>
> >> watermark
> >>
> >> generators
> >>
> >>  - Need a way to tell SplitReader to "suspend" a split
> >>
> >> until a
> >>
> >> certain
> >>
> >> watermark is reached (event time backpressure)
> >>  - This would in fact be not needed (and thus simpler)
> >>
> >> if
> >>
> >> we
> >>
> >> had
> >>
> >> a
> >>
> >> SplitReader per split and may be a reason to re-open
> >>
> >> that
> >>
> >> discussion
> >>
> >> *(6) Watermarks across splits and in the Split
> >>
> >> Enumerator*
> >>
> >>  - The split enumerator may need some watermark
> >>
> >> awareness,
> >>
> >> which
> >>
> >> should
> >>
> >> be
> >>
> >> purely based on split metadata (like create timestamp
> >>
> >> of
> >>
> >> file
> >>
> >> splits)
> >>
> >>  - If there are still more splits with overlapping
> >>
> >> event
> >>
> >> time
> >>
> >> range
> >>
> >> for
> >>
> >> a
> >>
> >> split reader, then that split reader should not advance
> >>
> >> the
> >>
> >> watermark
> >>
> >> within the split beyond the overlap boundary. Otherwise
> >>
> >> future
> >>
> >> splits
> >>
> >> will
> >>
> >> produce late data.
> >>
> >>  - One way to approach this could be that the split
> >>
> >> enumerator
> >>
> >> may
> >>
> >> send
> >>
> >> watermarks to the readers, and the readers cannot emit
> >>
> >> watermarks
> >>
> >> beyond
> >>
> >> that received watermark.
> >>  - Many split enumerators would simply immediately send
> >>
> >> Long.MAX
> >>
> >> out
> >>
> >> and
> >>
> >> leave the progress purely to the split readers.
> >>
> >>  - For event-time alignment / split back pressure, this
> >>
> >> begs
> >>
> >> the
> >>
> >> question
> >>
> >> how we can avoid deadlocks that may arise when splits
> >>
> >> are
> >>
> >> suspended
> >>
> >> for
> >>
> >> event time back pressure,
> >>
> >> *(7) Batch and streaming Unification*
> >>
> >>  - Functionality wise, the above design should support
> >>
> >> both
> >>
> >>  - Batch often (mostly) does not care about reading "in
> >>
> >> order"
> >>
> >> and
> >>
> >> generating watermarks
> >>    --> Might use different enumerator logic that is
> >>
> >> more
> >>
> >> locality
> >>
> >> aware
> >>
> >> and ignores event time order
> >>    --> Does not generate watermarks
> >>  - Would be great if bounded sources could be
> >>
> >> identified
> >>
> >> at
> >>
> >> compile
> >>
> >> time,
> >>
> >> so that "env.addBoundedSource(...)" is type safe and
> >>
> >> can
> >>
> >> return a
> >>
> >> "BoundedDataStream".
> >>  - Possible to defer this discussion until later
> >>
> >> *Miscellaneous Comments*
> >>
> >>  - Should the source have a TypeInformation for the
> >>
> >> produced
> >>
> >> type,
> >>
> >> instead
> >>
> >> of a serializer? We need a type information in the
> >>
> >> stream
> >>
> >> anyways, and
> >>
> >> can
> >>
> >> derive the serializer from that. Plus, creating the
> >>
> >> serializer
> >>
> >> should
> >>
> >> respect the ExecutionConfig.
> >>
> >>  - The TypeSerializer interface is very powerful but
> >>
> >> also
> >>
> >> not
> >>
> >> easy to
> >>
> >> implement. Its purpose is to handle data super
> >>
> >> efficiently,
> >>
> >> support
> >>
> >> flexible ways of evolution, etc.
> >>  For metadata I would suggest to look at the
> >>
> >> SimpleVersionedSerializer
> >>
> >> instead, which is used for example for checkpoint
> >>
> >> master
> >>
> >> hooks,
> >>
> >> or for
> >>
> >> the
> >>
> >> streaming file sink. I think that is is a good match
> >>
> >> for
> >>
> >> cases
> >>
> >> where
> >>
> >> we
> >>
> >> do
> >>
> >> not need more than ser/deser (no copy, etc.) and don't
> >>
> >> need to
> >>
> >> push
> >>
> >> versioning out of the serialization paths for best
> >>
> >> performance
> >>
> >> (as in
> >>
> >> the
> >>
> >> TypeSerializer)
> >>
> >>
> >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> k.kloudas@data-artisans.com>
> >> wrote:
> >>
> >>
> >> Hi Biao,
> >>
> >> Thanks for the answer!
> >>
> >> So given the multi-threaded readers, now we have as
> >>
> >> open
> >>
> >> questions:
> >>
> >> 1) How do we let the checkpoints pass through our
> >>
> >> multi-threaded
> >>
> >> reader
> >>
> >> operator?
> >>
> >> 2) Do we have separate reader and source operators or
> >>
> >> not? In
> >>
> >> the
> >>
> >> strategy
> >>
> >> that has a separate source, the source operator has a
> >>
> >> parallelism of
> >>
> >> 1
> >>
> >> and
> >>
> >> is responsible for split recovery only.
> >>
> >> For the first one, given also the constraints
> >>
> >> (blocking,
> >>
> >> finite
> >>
> >> queues,
> >>
> >> etc), I do not have an answer yet.
> >>
> >> For the 2nd, I think that we should go with separate
> >>
> >> operators
> >>
> >> for
> >>
> >> the
> >>
> >> source and the readers, for the following reasons:
> >>
> >> 1) This is more aligned with a potential future
> >>
> >> improvement
> >>
> >> where the
> >>
> >> split
> >>
> >> discovery becomes a responsibility of the JobManager
> >>
> >> and
> >>
> >> readers are
> >>
> >> pooling more work from the JM.
> >>
> >> 2) The source is going to be the "single point of
> >>
> >> truth".
> >>
> >> It
> >>
> >> will
> >>
> >> know
> >>
> >> what
> >>
> >> has been processed and what not. If the source and the
> >>
> >> readers
> >>
> >> are a
> >>
> >> single
> >>
> >> operator with parallelism > 1, or in general, if the
> >>
> >> split
> >>
> >> discovery
> >>
> >> is
> >>
> >> done by each task individually, then:
> >>   i) we have to have a deterministic scheme for each
> >>
> >> reader to
> >>
> >> assign
> >>
> >> splits to itself (e.g. mod subtaskId). This is not
> >>
> >> necessarily
> >>
> >> trivial
> >>
> >> for
> >>
> >> all sources.
> >>   ii) each reader would have to keep a copy of all its
> >>
> >> processed
> >>
> >> slpits
> >>
> >>   iii) the state has to be a union state with a
> >>
> >> non-trivial
> >>
> >> merging
> >>
> >> logic
> >>
> >> in order to support rescaling.
> >>
> >> Two additional points that you raised above:
> >>
> >> i) The point that you raised that we need to keep all
> >>
> >> splits
> >>
> >> (processed
> >>
> >> and
> >>
> >> not-processed) I think is a bit of a strong
> >>
> >> requirement.
> >>
> >> This
> >>
> >> would
> >>
> >> imply
> >>
> >> that for infinite sources the state will grow
> >>
> >> indefinitely.
> >>
> >> This is
> >>
> >> problem
> >>
> >> is even more pronounced if we do not have a single
> >>
> >> source
> >>
> >> that
> >>
> >> assigns
> >>
> >> splits to readers, as each reader will have its own
> >>
> >> copy
> >>
> >> of
> >>
> >> the
> >>
> >> state.
> >>
> >> ii) it is true that for finite sources we need to
> >>
> >> somehow
> >>
> >> not
> >>
> >> close
> >>
> >> the
> >>
> >> readers when the source/split discoverer finishes. The
> >> ContinuousFileReaderOperator has a work-around for
> >>
> >> that.
> >>
> >> It is
> >>
> >> not
> >>
> >> elegant,
> >>
> >> and checkpoints are not emitted after closing the
> >>
> >> source,
> >>
> >> but
> >>
> >> this, I
> >>
> >> believe, is a bigger problem which requires more
> >>
> >> changes
> >>
> >> than
> >>
> >> just
> >>
> >> refactoring the source interface.
> >>
> >> Cheers,
> >> Kostas
> >>
> >>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Dawid Wysakowicz <dw...@apache.org>.
One more thing. In the current proposal, with the
supportsBoundedness(Boundedness) method and the boundedness coming from
either continuousSource or boundedSource I could not find how this
information is fed back to the SplitEnumerator.

Best,

Dawid

On 09/12/2019 13:52, Becket Qin wrote:
> Hi Dawid,
>
> Thanks for the comments. This actually brings another relevant question
> about what does a "bounded source" imply. I actually had the same
> impression when I look at the Source API. Here is what I understand after
> some discussion with Stephan. The bounded source has the following impacts.
>
> 1. API validity.
> - A bounded source generates a bounded stream so some operations that only
> works for bounded records would be performed, e.g. sort.
> - To expose these bounded stream only APIs, there are two options:
>      a. Add them to the DataStream API and throw exception if a method is
> called on an unbounded stream.
>      b. Create a BoundedDataStream class which is returned from
> env.boundedSource(), while DataStream is returned from env.continousSource().
> Note that this cannot be done by having single env.source(theSource) even
> the Source has a getBoundedness() method.
>
> 2. Scheduling
> - A bounded source could be computed stage by stage without bringing up all
> the tasks at the same time.
>
> 3. Operator behaviors
> - A bounded source indicates the records are finite so some operators can
> wait until it receives all the records before it starts the processing.
>
> In the above impact, only 1 is relevant to the API design. And the current
> proposal in FLIP-27 is following 1.b.
>
> // boundedness depends of source property, imo this should always be
>> preferred
>>
>
> DataStream<MyType> stream = env.source(theSource);
>
>
> In your proposal, does DataStream have bounded stream only methods? It
> looks it should have, otherwise passing a bounded Source to env.source()
> would be confusing. In that case, we will essentially do 1.a if an
> unbounded Source is created from env.source(unboundedSource).
>
> If we have the methods only supported for bounded streams in DataStream, it
> seems a little weird to have a separate BoundedDataStream interface.
>
> Am I understand it correctly?
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
> On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <dw...@apache.org>
> wrote:
>
>> Hi all,
>>
>> Really well written proposal and very important one. I must admit I have
>> not understood all the intricacies of it yet.
>>
>> One question I have though is about where does the information about
>> boundedness come from. I think in most cases it is a property of the
>> source. As you described it might be e.g. end offset, a flag should it
>> monitor new splits etc. I think it would be a really nice use case to be
>> able to say:
>>
>> new KafkaSource().readUntil(long timestamp),
>>
>> which could work as an "end offset". Moreover I think all Bounded sources
>> support continuous mode, but no intrinsically continuous source support the
>> Bounded mode. If I understood the proposal correctly it suggest the
>> boundedness sort of "comes" from the outside of the source, from the
>> invokation of either boundedStream or continousSource.
>>
>> I am wondering if it would make sense to actually change the method
>>
>> boolean Source#supportsBoundedness(Boundedness)
>>
>> to
>>
>> Boundedness Source#getBoundedness().
>>
>> As for the methods #boundedSource, #continousSource, assuming the
>> boundedness is property of the source they do not affect how the enumerator
>> works, but mostly how the dag is scheduled, right? I am not against those
>> methods, but I think it is a very specific use case to actually override
>> the property of the source. In general I would expect users to only call
>> env.source(theSource), where the source tells if it is bounded or not. I
>> would suggest considering following set of methods:
>>
>> // boundedness depends of source property, imo this should always be preferred
>>
>> DataStream<MyType> stream = env.source(theSource);
>>
>>
>> // always continous execution, whether bounded or unbounded source
>>
>> DataStream<MyType> boundedStream = env.continousSource(theSource);
>>
>> // imo this would make sense if the BoundedDataStream provides additional features unavailable for continous mode
>> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
>>
>>
>> Best,
>>
>> Dawid
>>
>>
>> On 04/12/2019 11:25, Stephan Ewen wrote:
>>
>> Thanks, Becket, for updating this.
>>
>> I agree with moving the aspects you mentioned into separate FLIPs - this
>> one way becoming unwieldy in size.
>>
>> +1 to the FLIP in its current state. Its a very detailed write-up, nicely
>> done!
>>
>> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <be...@gmail.com> <be...@gmail.com> wrote:
>>
>>
>> Hi all,
>>
>> Sorry for the long belated update. I have updated FLIP-27 wiki page with
>> the latest proposals. Some noticeable changes include:
>> 1. A new generic communication mechanism between SplitEnumerator and
>> SourceReader.
>> 2. Some detail API method signature changes.
>>
>> We left a few things out of this FLIP and will address them in separate
>> FLIPs. Including:
>> 1. Per split event time.
>> 2. Event time alignment.
>> 3. Fine grained failover for SplitEnumerator failure.
>>
>> Please let us know if you have any question.
>>
>> Thanks,
>>
>> Jiangjie (Becket) Qin
>>
>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> <se...@apache.org> wrote:
>>
>>
>> Hi  Łukasz!
>>
>> Becket and me are working hard on figuring out the last details and
>> implementing the first PoC. We would update the FLIP hopefully next week.
>>
>> There is a fair chance that a first version of this will be in 1.10, but
>>
>> I
>>
>> think it will take another release to battle test it and migrate the
>> connectors.
>>
>> Best,
>> Stephan
>>
>>
>>
>>
>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <lj...@touk.pl> <lj...@touk.pl>
>>
>> wrote:
>>
>> Hi,
>>
>> This proposal looks very promising for us. Do you have any plans in
>>
>> which
>>
>> Flink release it is going to be released? We are thinking on using a
>>
>> Data
>>
>> Set API for our future use cases but on the other hand Data Set API is
>> going to be deprecated so using proposed bounded data streams solution
>> could be more viable in the long term.
>>
>> Thanks,
>> Łukasz
>>
>> On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> <th...@gmail.com> wrote:
>>
>> Thanks for putting together this proposal!
>>
>> I see that the "Per Split Event Time" and "Event Time Alignment"
>>
>> sections
>>
>> are still TBD.
>>
>> It would probably be good to flesh those out a bit before proceeding
>>
>> too
>>
>> far
>>
>> as the event time alignment will probably influence the interaction
>>
>> with
>>
>> the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
>> output).
>>
>> We currently have only one implementation for event time alignment in
>>
>> the
>>
>> Kinesis consumer. The synchronization in that case takes place as the
>>
>> last
>>
>> step before records are emitted downstream (RecordEmitter). With the
>> currently proposed interfaces, the equivalent can be implemented in
>>
>> the
>>
>> reader loop, although note that in the Kinesis consumer the per shard
>> threads push records.
>>
>> Synchronization has not been implemented for the Kafka consumer yet.
>> https://issues.apache.org/jira/browse/FLINK-12675
>>
>> When I looked at it, I realized that the implementation will look
>>
>> quite
>>
>> different
>> from Kinesis because it needs to take place in the pull part, where
>>
>> records
>>
>> are taken from the Kafka client. Due to the multiplexing it cannot be
>>
>> done
>>
>> by blocking the split thread like it currently works for Kinesis.
>>
>> Reading
>>
>> from individual Kafka partitions needs to be controlled via
>>
>> pause/resume
>>
>> on the Kafka client.
>>
>> To take on that responsibility the split thread would need to be
>>
>> aware
>>
>> of
>>
>> the
>> watermarks or at least whether it should or should not continue to
>>
>> consume
>>
>> a given split and this may require a different SourceReader or
>>
>> SourceOutput
>>
>> interface.
>>
>> Thanks,
>> Thomas
>>
>>
>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com> wrote:
>>
>>
>> Hi Stephan,
>>
>> Thank you for feedback!
>> Will take a look at your branch before public discussing.
>>
>>
>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org> <se...@apache.org>
>>
>> wrote:
>>
>> Hi Biao!
>>
>> Thanks for reviving this. I would like to join this discussion,
>>
>> but
>>
>> am
>>
>> quite occupied with the 1.9 release, so can we maybe pause this
>>
>> discussion
>>
>> for a week or so?
>>
>> In the meantime I can share some suggestion based on prior
>>
>> experiments:
>>
>> How to do watermarks / timestamp extractors in a simpler and more
>>
>> flexible
>>
>> way. I think that part is quite promising should be part of the
>>
>> new
>>
>> source
>>
>> interface.
>>
>>
>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>>
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>>
>> Some experiments on how to build the source reader and its
>>
>> library
>>
>> for
>>
>> common threading/split patterns:
>>
>>
>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>>
>> Best,
>> Stephan
>>
>>
>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>>
>> wrote:
>>
>> Hi devs,
>>
>> Since 1.9 is nearly released, I think we could get back to
>>
>> FLIP-27.
>>
>> I
>>
>> believe it should be included in 1.10.
>>
>> There are so many things mentioned in document of FLIP-27. [1] I
>>
>> think
>>
>> we'd better discuss them separately. However the wiki is not a
>>
>> good
>>
>> place
>>
>> to discuss. I wrote google doc about SplitReader API which
>>
>> misses
>>
>> some
>>
>> details in the document. [2]
>>
>> 1.
>>
>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>>
>> 2.
>>
>>
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>>
>> CC Stephan, Aljoscha, Piotrek, Becket
>>
>>
>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>>
>> wrote:
>>
>> Hi Steven,
>> Thank you for the feedback. Please take a look at the document
>>
>> FLIP-27
>>
>> <
>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>
>> which
>>
>> is updated recently. A lot of details of enumerator were added
>>
>> in
>>
>> this
>>
>> document. I think it would help.
>>
>> Steven Wu <st...@gmail.com> <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
>>
>>
>> This proposal mentioned that SplitEnumerator might run on the
>> JobManager or
>> in a single task on a TaskManager.
>>
>> if enumerator is a single task on a taskmanager, then the job
>>
>> DAG
>>
>> can
>>
>> never
>> been embarrassingly parallel anymore. That will nullify the
>>
>> leverage
>>
>> of
>>
>> fine-grained recovery for embarrassingly parallel jobs.
>>
>> It's not clear to me what's the implication of running
>>
>> enumerator
>>
>> on
>>
>> the
>>
>> jobmanager. So I will leave that out for now.
>>
>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>>
>> wrote:
>>
>> Hi Stephan & Piotrek,
>>
>> Thank you for feedback.
>>
>> It seems that there are a lot of things to do in community.
>>
>> I
>>
>> am
>>
>> just
>>
>> afraid that this discussion may be forgotten since there so
>>
>> many
>>
>> proposals
>>
>> recently.
>> Anyway, wish to see the split topics soon :)
>>
>> Piotr Nowojski <pi...@da-platform.com> <pi...@da-platform.com> 于2019年1月24日周四
>>
>> 下午8:21写道:
>>
>> Hi Biao!
>>
>> This discussion was stalled because of preparations for
>>
>> the
>>
>> open
>>
>> sourcing
>>
>> & merging Blink. I think before creating the tickets we
>>
>> should
>>
>> split this
>>
>> discussion into topics/areas outlined by Stephan and
>>
>> create
>>
>> Flips
>>
>> for
>>
>> that.
>>
>> I think there is no chance for this to be completed in
>>
>> couple
>>
>> of
>>
>> remaining
>>
>> weeks/1 month before 1.8 feature freeze, however it would
>>
>> be
>>
>> good
>>
>> to aim
>>
>> with those changes for 1.9.
>>
>> Piotrek
>>
>>
>> On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>>
>> wrote:
>>
>> Hi community,
>> The summary of Stephan makes a lot sense to me. It is
>>
>> much
>>
>> clearer
>>
>> indeed
>>
>> after splitting the complex topic into small ones.
>> I was wondering is there any detail plan for next step?
>>
>> If
>>
>> not,
>>
>> I
>>
>> would
>>
>> like to push this thing forward by creating some JIRA
>>
>> issues.
>>
>> Another question is that should version 1.8 include
>>
>> these
>>
>> features?
>>
>> Stephan Ewen <se...@apache.org> <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
>>
>>
>> Thanks everyone for the lively discussion. Let me try
>>
>> to
>>
>> summarize
>>
>> where I
>>
>> see convergence in the discussion and open issues.
>> I'll try to group this by design aspect of the source.
>>
>> Please
>>
>> let me
>>
>> know
>>
>> if I got things wrong or missed something crucial here.
>>
>> For issues 1-3, if the below reflects the state of the
>>
>> discussion, I
>>
>> would
>>
>> try and update the FLIP in the next days.
>> For the remaining ones we need more discussion.
>>
>> I would suggest to fork each of these aspects into a
>>
>> separate
>>
>> mail
>>
>> thread,
>>
>> or will loose sight of the individual aspects.
>>
>> *(1) Separation of Split Enumerator and Split Reader*
>>
>>  - All seem to agree this is a good thing
>>  - Split Enumerator could in the end live on JobManager
>>
>> (and
>>
>> assign
>>
>> splits
>>
>> via RPC) or in a task (and assign splits via data
>>
>> streams)
>>
>>  - this discussion is orthogonal and should come later,
>>
>> when
>>
>> the
>>
>> interface
>>
>> is agreed upon.
>>
>> *(2) Split Readers for one or more splits*
>>
>>  - Discussion seems to agree that we need to support
>>
>> one
>>
>> reader
>>
>> that
>>
>> possibly handles multiple splits concurrently.
>>  - The requirement comes from sources where one
>>
>> poll()-style
>>
>> call
>>
>> fetches
>>
>> data from different splits / partitions
>>    --> example sources that require that would be for
>>
>> example
>>
>> Kafka,
>>
>> Pravega, Pulsar
>>
>>  - Could have one split reader per source, or multiple
>>
>> split
>>
>> readers
>>
>> that
>>
>> share the "poll()" function
>>  - To not make it too complicated, we can start with
>>
>> thinking
>>
>> about
>>
>> one
>>
>> split reader for all splits initially and see if that
>>
>> covers
>>
>> all
>>
>> requirements
>>
>> *(3) Threading model of the Split Reader*
>>
>>  - Most active part of the discussion ;-)
>>
>>  - A non-blocking way for Flink's task code to interact
>>
>> with
>>
>> the
>>
>> source
>>
>> is
>>
>> needed in order to a task runtime code based on a
>> single-threaded/actor-style task design
>>    --> I personally am a big proponent of that, it will
>>
>> help
>>
>> with
>>
>> well-behaved checkpoints, efficiency, and simpler yet
>>
>> more
>>
>> robust
>>
>> runtime
>>
>> code
>>
>>  - Users care about simple abstraction, so as a
>>
>> subclass
>>
>> of
>>
>> SplitReader
>>
>> (non-blocking / async) we need to have a
>>
>> BlockingSplitReader
>>
>> which
>>
>> will
>>
>> form the basis of most source implementations.
>>
>> BlockingSplitReader
>>
>> lets
>>
>> users do blocking simple poll() calls.
>>  - The BlockingSplitReader would spawn a thread (or
>>
>> more)
>>
>> and
>>
>> the
>>
>> thread(s) can make blocking calls and hand over data
>>
>> buffers
>>
>> via
>>
>> a
>>
>> blocking
>>
>> queue
>>  - This should allow us to cover both, a fully async
>>
>> runtime,
>>
>> and a
>>
>> simple
>>
>> blocking interface for users.
>>  - This is actually very similar to how the Kafka
>>
>> connectors
>>
>> work.
>>
>> Kafka
>>
>> 9+ with one thread, Kafka 8 with multiple threads
>>
>>  - On the base SplitReader (the async one), the
>>
>> non-blocking
>>
>> method
>>
>> that
>>
>> gets the next chunk of data would signal data
>>
>> availability
>>
>> via
>>
>> a
>>
>> CompletableFuture, because that gives the best
>>
>> flexibility
>>
>> (can
>>
>> await
>>
>> completion or register notification handlers).
>>  - The source task would register a "thenHandle()" (or
>>
>> similar)
>>
>> on the
>>
>> future to put a "take next data" task into the
>>
>> actor-style
>>
>> mailbox
>>
>> *(4) Split Enumeration and Assignment*
>>
>>  - Splits may be generated lazily, both in cases where
>>
>> there
>>
>> is a
>>
>> limited
>>
>> number of splits (but very many), or splits are
>>
>> discovered
>>
>> over
>>
>> time
>>
>>  - Assignment should also be lazy, to get better load
>>
>> balancing
>>
>>  - Assignment needs support locality preferences
>>
>>  - Possible design based on discussion so far:
>>
>>    --> SplitReader has a method "addSplits(SplitT...)"
>>
>> to
>>
>> add
>>
>> one or
>>
>> more
>>
>> splits. Some split readers might assume they have only
>>
>> one
>>
>> split
>>
>> ever,
>>
>> concurrently, others assume multiple splits. (Note:
>>
>> idea
>>
>> behind
>>
>> being
>>
>> able
>>
>> to add multiple splits at the same time is to ease
>>
>> startup
>>
>> where
>>
>> multiple
>>
>> splits may be assigned instantly.)
>>    --> SplitReader has a context object on which it can
>>
>> call
>>
>> indicate
>>
>> when
>>
>> splits are completed. The enumerator gets that
>>
>> notification and
>>
>> can
>>
>> use
>>
>> to
>>
>> decide when to assign new splits. This should help both
>>
>> in
>>
>> cases
>>
>> of
>>
>> sources
>>
>> that take splits lazily (file readers) and in case the
>>
>> source
>>
>> needs to
>>
>> preserve a partial order between splits (Kinesis,
>>
>> Pravega,
>>
>> Pulsar may
>>
>> need
>>
>> that).
>>    --> SplitEnumerator gets notification when
>>
>> SplitReaders
>>
>> start
>>
>> and
>>
>> when
>>
>> they finish splits. They can decide at that moment to
>>
>> push
>>
>> more
>>
>> splits
>>
>> to
>>
>> that reader
>>    --> The SplitEnumerator should probably be aware of
>>
>> the
>>
>> source
>>
>> parallelism, to build its initial distribution.
>>
>>  - Open question: Should the source expose something
>>
>> like
>>
>> "host
>>
>> preferences", so that yarn/mesos/k8s can take this into
>>
>> account
>>
>> when
>>
>> selecting a node to start a TM on?
>>
>> *(5) Watermarks and event time alignment*
>>
>>  - Watermark generation, as well as idleness, needs to
>>
>> be
>>
>> per
>>
>> split
>>
>> (like
>>
>> currently in the Kafka Source, per partition)
>>  - It is desirable to support optional
>>
>> event-time-alignment,
>>
>> meaning
>>
>> that
>>
>> splits that are ahead are back-pressured or temporarily
>>
>> unsubscribed
>>
>>  - I think i would be desirable to encapsulate
>>
>> watermark
>>
>> generation
>>
>> logic
>>
>> in watermark generators, for a separation of concerns.
>>
>> The
>>
>> watermark
>>
>> generators should run per split.
>>  - Using watermark generators would also help with
>>
>> another
>>
>> problem of
>>
>> the
>>
>> suggested interface, namely supporting non-periodic
>>
>> watermarks
>>
>> efficiently.
>>
>>  - Need a way to "dispatch" next record to different
>>
>> watermark
>>
>> generators
>>
>>  - Need a way to tell SplitReader to "suspend" a split
>>
>> until a
>>
>> certain
>>
>> watermark is reached (event time backpressure)
>>  - This would in fact be not needed (and thus simpler)
>>
>> if
>>
>> we
>>
>> had
>>
>> a
>>
>> SplitReader per split and may be a reason to re-open
>>
>> that
>>
>> discussion
>>
>> *(6) Watermarks across splits and in the Split
>>
>> Enumerator*
>>
>>  - The split enumerator may need some watermark
>>
>> awareness,
>>
>> which
>>
>> should
>>
>> be
>>
>> purely based on split metadata (like create timestamp
>>
>> of
>>
>> file
>>
>> splits)
>>
>>  - If there are still more splits with overlapping
>>
>> event
>>
>> time
>>
>> range
>>
>> for
>>
>> a
>>
>> split reader, then that split reader should not advance
>>
>> the
>>
>> watermark
>>
>> within the split beyond the overlap boundary. Otherwise
>>
>> future
>>
>> splits
>>
>> will
>>
>> produce late data.
>>
>>  - One way to approach this could be that the split
>>
>> enumerator
>>
>> may
>>
>> send
>>
>> watermarks to the readers, and the readers cannot emit
>>
>> watermarks
>>
>> beyond
>>
>> that received watermark.
>>  - Many split enumerators would simply immediately send
>>
>> Long.MAX
>>
>> out
>>
>> and
>>
>> leave the progress purely to the split readers.
>>
>>  - For event-time alignment / split back pressure, this
>>
>> begs
>>
>> the
>>
>> question
>>
>> how we can avoid deadlocks that may arise when splits
>>
>> are
>>
>> suspended
>>
>> for
>>
>> event time back pressure,
>>
>> *(7) Batch and streaming Unification*
>>
>>  - Functionality wise, the above design should support
>>
>> both
>>
>>  - Batch often (mostly) does not care about reading "in
>>
>> order"
>>
>> and
>>
>> generating watermarks
>>    --> Might use different enumerator logic that is
>>
>> more
>>
>> locality
>>
>> aware
>>
>> and ignores event time order
>>    --> Does not generate watermarks
>>  - Would be great if bounded sources could be
>>
>> identified
>>
>> at
>>
>> compile
>>
>> time,
>>
>> so that "env.addBoundedSource(...)" is type safe and
>>
>> can
>>
>> return a
>>
>> "BoundedDataStream".
>>  - Possible to defer this discussion until later
>>
>> *Miscellaneous Comments*
>>
>>  - Should the source have a TypeInformation for the
>>
>> produced
>>
>> type,
>>
>> instead
>>
>> of a serializer? We need a type information in the
>>
>> stream
>>
>> anyways, and
>>
>> can
>>
>> derive the serializer from that. Plus, creating the
>>
>> serializer
>>
>> should
>>
>> respect the ExecutionConfig.
>>
>>  - The TypeSerializer interface is very powerful but
>>
>> also
>>
>> not
>>
>> easy to
>>
>> implement. Its purpose is to handle data super
>>
>> efficiently,
>>
>> support
>>
>> flexible ways of evolution, etc.
>>  For metadata I would suggest to look at the
>>
>> SimpleVersionedSerializer
>>
>> instead, which is used for example for checkpoint
>>
>> master
>>
>> hooks,
>>
>> or for
>>
>> the
>>
>> streaming file sink. I think that is is a good match
>>
>> for
>>
>> cases
>>
>> where
>>
>> we
>>
>> do
>>
>> not need more than ser/deser (no copy, etc.) and don't
>>
>> need to
>>
>> push
>>
>> versioning out of the serialization paths for best
>>
>> performance
>>
>> (as in
>>
>> the
>>
>> TypeSerializer)
>>
>>
>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <k....@data-artisans.com>
>> wrote:
>>
>>
>> Hi Biao,
>>
>> Thanks for the answer!
>>
>> So given the multi-threaded readers, now we have as
>>
>> open
>>
>> questions:
>>
>> 1) How do we let the checkpoints pass through our
>>
>> multi-threaded
>>
>> reader
>>
>> operator?
>>
>> 2) Do we have separate reader and source operators or
>>
>> not? In
>>
>> the
>>
>> strategy
>>
>> that has a separate source, the source operator has a
>>
>> parallelism of
>>
>> 1
>>
>> and
>>
>> is responsible for split recovery only.
>>
>> For the first one, given also the constraints
>>
>> (blocking,
>>
>> finite
>>
>> queues,
>>
>> etc), I do not have an answer yet.
>>
>> For the 2nd, I think that we should go with separate
>>
>> operators
>>
>> for
>>
>> the
>>
>> source and the readers, for the following reasons:
>>
>> 1) This is more aligned with a potential future
>>
>> improvement
>>
>> where the
>>
>> split
>>
>> discovery becomes a responsibility of the JobManager
>>
>> and
>>
>> readers are
>>
>> pooling more work from the JM.
>>
>> 2) The source is going to be the "single point of
>>
>> truth".
>>
>> It
>>
>> will
>>
>> know
>>
>> what
>>
>> has been processed and what not. If the source and the
>>
>> readers
>>
>> are a
>>
>> single
>>
>> operator with parallelism > 1, or in general, if the
>>
>> split
>>
>> discovery
>>
>> is
>>
>> done by each task individually, then:
>>   i) we have to have a deterministic scheme for each
>>
>> reader to
>>
>> assign
>>
>> splits to itself (e.g. mod subtaskId). This is not
>>
>> necessarily
>>
>> trivial
>>
>> for
>>
>> all sources.
>>   ii) each reader would have to keep a copy of all its
>>
>> processed
>>
>> slpits
>>
>>   iii) the state has to be a union state with a
>>
>> non-trivial
>>
>> merging
>>
>> logic
>>
>> in order to support rescaling.
>>
>> Two additional points that you raised above:
>>
>> i) The point that you raised that we need to keep all
>>
>> splits
>>
>> (processed
>>
>> and
>>
>> not-processed) I think is a bit of a strong
>>
>> requirement.
>>
>> This
>>
>> would
>>
>> imply
>>
>> that for infinite sources the state will grow
>>
>> indefinitely.
>>
>> This is
>>
>> problem
>>
>> is even more pronounced if we do not have a single
>>
>> source
>>
>> that
>>
>> assigns
>>
>> splits to readers, as each reader will have its own
>>
>> copy
>>
>> of
>>
>> the
>>
>> state.
>>
>> ii) it is true that for finite sources we need to
>>
>> somehow
>>
>> not
>>
>> close
>>
>> the
>>
>> readers when the source/split discoverer finishes. The
>> ContinuousFileReaderOperator has a work-around for
>>
>> that.
>>
>> It is
>>
>> not
>>
>> elegant,
>>
>> and checkpoints are not emitted after closing the
>>
>> source,
>>
>> but
>>
>> this, I
>>
>> believe, is a bigger problem which requires more
>>
>> changes
>>
>> than
>>
>> just
>>
>> refactoring the source interface.
>>
>> Cheers,
>> Kostas
>>
>>
>>


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Dawid,

Thanks for the comments. This actually brings another relevant question
about what does a "bounded source" imply. I actually had the same
impression when I look at the Source API. Here is what I understand after
some discussion with Stephan. The bounded source has the following impacts.

1. API validity.
- A bounded source generates a bounded stream so some operations that only
works for bounded records would be performed, e.g. sort.
- To expose these bounded stream only APIs, there are two options:
     a. Add them to the DataStream API and throw exception if a method is
called on an unbounded stream.
     b. Create a BoundedDataStream class which is returned from
env.boundedSource(), while DataStream is returned from env.continousSource().
Note that this cannot be done by having single env.source(theSource) even
the Source has a getBoundedness() method.

2. Scheduling
- A bounded source could be computed stage by stage without bringing up all
the tasks at the same time.

3. Operator behaviors
- A bounded source indicates the records are finite so some operators can
wait until it receives all the records before it starts the processing.

In the above impact, only 1 is relevant to the API design. And the current
proposal in FLIP-27 is following 1.b.

// boundedness depends of source property, imo this should always be
> preferred
>


DataStream<MyType> stream = env.source(theSource);


In your proposal, does DataStream have bounded stream only methods? It
looks it should have, otherwise passing a bounded Source to env.source()
would be confusing. In that case, we will essentially do 1.a if an
unbounded Source is created from env.source(unboundedSource).

If we have the methods only supported for bounded streams in DataStream, it
seems a little weird to have a separate BoundedDataStream interface.

Am I understand it correctly?

Thanks,

Jiangjie (Becket) Qin



On Mon, Dec 9, 2019 at 6:40 PM Dawid Wysakowicz <dw...@apache.org>
wrote:

> Hi all,
>
> Really well written proposal and very important one. I must admit I have
> not understood all the intricacies of it yet.
>
> One question I have though is about where does the information about
> boundedness come from. I think in most cases it is a property of the
> source. As you described it might be e.g. end offset, a flag should it
> monitor new splits etc. I think it would be a really nice use case to be
> able to say:
>
> new KafkaSource().readUntil(long timestamp),
>
> which could work as an "end offset". Moreover I think all Bounded sources
> support continuous mode, but no intrinsically continuous source support the
> Bounded mode. If I understood the proposal correctly it suggest the
> boundedness sort of "comes" from the outside of the source, from the
> invokation of either boundedStream or continousSource.
>
> I am wondering if it would make sense to actually change the method
>
> boolean Source#supportsBoundedness(Boundedness)
>
> to
>
> Boundedness Source#getBoundedness().
>
> As for the methods #boundedSource, #continousSource, assuming the
> boundedness is property of the source they do not affect how the enumerator
> works, but mostly how the dag is scheduled, right? I am not against those
> methods, but I think it is a very specific use case to actually override
> the property of the source. In general I would expect users to only call
> env.source(theSource), where the source tells if it is bounded or not. I
> would suggest considering following set of methods:
>
> // boundedness depends of source property, imo this should always be preferred
>
> DataStream<MyType> stream = env.source(theSource);
>
>
> // always continous execution, whether bounded or unbounded source
>
> DataStream<MyType> boundedStream = env.continousSource(theSource);
>
> // imo this would make sense if the BoundedDataStream provides additional features unavailable for continous mode
> BoundedDataStream<MyType> batch = env.boundedSource(theSource);
>
>
> Best,
>
> Dawid
>
>
> On 04/12/2019 11:25, Stephan Ewen wrote:
>
> Thanks, Becket, for updating this.
>
> I agree with moving the aspects you mentioned into separate FLIPs - this
> one way becoming unwieldy in size.
>
> +1 to the FLIP in its current state. Its a very detailed write-up, nicely
> done!
>
> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <be...@gmail.com> <be...@gmail.com> wrote:
>
>
> Hi all,
>
> Sorry for the long belated update. I have updated FLIP-27 wiki page with
> the latest proposals. Some noticeable changes include:
> 1. A new generic communication mechanism between SplitEnumerator and
> SourceReader.
> 2. Some detail API method signature changes.
>
> We left a few things out of this FLIP and will address them in separate
> FLIPs. Including:
> 1. Per split event time.
> 2. Event time alignment.
> 3. Fine grained failover for SplitEnumerator failure.
>
> Please let us know if you have any question.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> <se...@apache.org> wrote:
>
>
> Hi  Łukasz!
>
> Becket and me are working hard on figuring out the last details and
> implementing the first PoC. We would update the FLIP hopefully next week.
>
> There is a fair chance that a first version of this will be in 1.10, but
>
> I
>
> think it will take another release to battle test it and migrate the
> connectors.
>
> Best,
> Stephan
>
>
>
>
> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <lj...@touk.pl> <lj...@touk.pl>
>
> wrote:
>
> Hi,
>
> This proposal looks very promising for us. Do you have any plans in
>
> which
>
> Flink release it is going to be released? We are thinking on using a
>
> Data
>
> Set API for our future use cases but on the other hand Data Set API is
> going to be deprecated so using proposed bounded data streams solution
> could be more viable in the long term.
>
> Thanks,
> Łukasz
>
> On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> <th...@gmail.com> wrote:
>
> Thanks for putting together this proposal!
>
> I see that the "Per Split Event Time" and "Event Time Alignment"
>
> sections
>
> are still TBD.
>
> It would probably be good to flesh those out a bit before proceeding
>
> too
>
> far
>
> as the event time alignment will probably influence the interaction
>
> with
>
> the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
> output).
>
> We currently have only one implementation for event time alignment in
>
> the
>
> Kinesis consumer. The synchronization in that case takes place as the
>
> last
>
> step before records are emitted downstream (RecordEmitter). With the
> currently proposed interfaces, the equivalent can be implemented in
>
> the
>
> reader loop, although note that in the Kinesis consumer the per shard
> threads push records.
>
> Synchronization has not been implemented for the Kafka consumer yet.
> https://issues.apache.org/jira/browse/FLINK-12675
>
> When I looked at it, I realized that the implementation will look
>
> quite
>
> different
> from Kinesis because it needs to take place in the pull part, where
>
> records
>
> are taken from the Kafka client. Due to the multiplexing it cannot be
>
> done
>
> by blocking the split thread like it currently works for Kinesis.
>
> Reading
>
> from individual Kafka partitions needs to be controlled via
>
> pause/resume
>
> on the Kafka client.
>
> To take on that responsibility the split thread would need to be
>
> aware
>
> of
>
> the
> watermarks or at least whether it should or should not continue to
>
> consume
>
> a given split and this may require a different SourceReader or
>
> SourceOutput
>
> interface.
>
> Thanks,
> Thomas
>
>
> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com> wrote:
>
>
> Hi Stephan,
>
> Thank you for feedback!
> Will take a look at your branch before public discussing.
>
>
> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org> <se...@apache.org>
>
> wrote:
>
> Hi Biao!
>
> Thanks for reviving this. I would like to join this discussion,
>
> but
>
> am
>
> quite occupied with the 1.9 release, so can we maybe pause this
>
> discussion
>
> for a week or so?
>
> In the meantime I can share some suggestion based on prior
>
> experiments:
>
> How to do watermarks / timestamp extractors in a simpler and more
>
> flexible
>
> way. I think that part is quite promising should be part of the
>
> new
>
> source
>
> interface.
>
>
>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>
> Some experiments on how to build the source reader and its
>
> library
>
> for
>
> common threading/split patterns:
>
>
>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>
> Best,
> Stephan
>
>
> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>
> wrote:
>
> Hi devs,
>
> Since 1.9 is nearly released, I think we could get back to
>
> FLIP-27.
>
> I
>
> believe it should be included in 1.10.
>
> There are so many things mentioned in document of FLIP-27. [1] I
>
> think
>
> we'd better discuss them separately. However the wiki is not a
>
> good
>
> place
>
> to discuss. I wrote google doc about SplitReader API which
>
> misses
>
> some
>
> details in the document. [2]
>
> 1.
>
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>
> 2.
>
>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>
> CC Stephan, Aljoscha, Piotrek, Becket
>
>
> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>
> wrote:
>
> Hi Steven,
> Thank you for the feedback. Please take a look at the document
>
> FLIP-27
>
> <
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>
> which
>
> is updated recently. A lot of details of enumerator were added
>
> in
>
> this
>
> document. I think it would help.
>
> Steven Wu <st...@gmail.com> <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
>
>
> This proposal mentioned that SplitEnumerator might run on the
> JobManager or
> in a single task on a TaskManager.
>
> if enumerator is a single task on a taskmanager, then the job
>
> DAG
>
> can
>
> never
> been embarrassingly parallel anymore. That will nullify the
>
> leverage
>
> of
>
> fine-grained recovery for embarrassingly parallel jobs.
>
> It's not clear to me what's the implication of running
>
> enumerator
>
> on
>
> the
>
> jobmanager. So I will leave that out for now.
>
> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>
> wrote:
>
> Hi Stephan & Piotrek,
>
> Thank you for feedback.
>
> It seems that there are a lot of things to do in community.
>
> I
>
> am
>
> just
>
> afraid that this discussion may be forgotten since there so
>
> many
>
> proposals
>
> recently.
> Anyway, wish to see the split topics soon :)
>
> Piotr Nowojski <pi...@da-platform.com> <pi...@da-platform.com> 于2019年1月24日周四
>
> 下午8:21写道:
>
> Hi Biao!
>
> This discussion was stalled because of preparations for
>
> the
>
> open
>
> sourcing
>
> & merging Blink. I think before creating the tickets we
>
> should
>
> split this
>
> discussion into topics/areas outlined by Stephan and
>
> create
>
> Flips
>
> for
>
> that.
>
> I think there is no chance for this to be completed in
>
> couple
>
> of
>
> remaining
>
> weeks/1 month before 1.8 feature freeze, however it would
>
> be
>
> good
>
> to aim
>
> with those changes for 1.9.
>
> Piotrek
>
>
> On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> <mm...@gmail.com>
>
> wrote:
>
> Hi community,
> The summary of Stephan makes a lot sense to me. It is
>
> much
>
> clearer
>
> indeed
>
> after splitting the complex topic into small ones.
> I was wondering is there any detail plan for next step?
>
> If
>
> not,
>
> I
>
> would
>
> like to push this thing forward by creating some JIRA
>
> issues.
>
> Another question is that should version 1.8 include
>
> these
>
> features?
>
> Stephan Ewen <se...@apache.org> <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
>
>
> Thanks everyone for the lively discussion. Let me try
>
> to
>
> summarize
>
> where I
>
> see convergence in the discussion and open issues.
> I'll try to group this by design aspect of the source.
>
> Please
>
> let me
>
> know
>
> if I got things wrong or missed something crucial here.
>
> For issues 1-3, if the below reflects the state of the
>
> discussion, I
>
> would
>
> try and update the FLIP in the next days.
> For the remaining ones we need more discussion.
>
> I would suggest to fork each of these aspects into a
>
> separate
>
> mail
>
> thread,
>
> or will loose sight of the individual aspects.
>
> *(1) Separation of Split Enumerator and Split Reader*
>
>  - All seem to agree this is a good thing
>  - Split Enumerator could in the end live on JobManager
>
> (and
>
> assign
>
> splits
>
> via RPC) or in a task (and assign splits via data
>
> streams)
>
>  - this discussion is orthogonal and should come later,
>
> when
>
> the
>
> interface
>
> is agreed upon.
>
> *(2) Split Readers for one or more splits*
>
>  - Discussion seems to agree that we need to support
>
> one
>
> reader
>
> that
>
> possibly handles multiple splits concurrently.
>  - The requirement comes from sources where one
>
> poll()-style
>
> call
>
> fetches
>
> data from different splits / partitions
>    --> example sources that require that would be for
>
> example
>
> Kafka,
>
> Pravega, Pulsar
>
>  - Could have one split reader per source, or multiple
>
> split
>
> readers
>
> that
>
> share the "poll()" function
>  - To not make it too complicated, we can start with
>
> thinking
>
> about
>
> one
>
> split reader for all splits initially and see if that
>
> covers
>
> all
>
> requirements
>
> *(3) Threading model of the Split Reader*
>
>  - Most active part of the discussion ;-)
>
>  - A non-blocking way for Flink's task code to interact
>
> with
>
> the
>
> source
>
> is
>
> needed in order to a task runtime code based on a
> single-threaded/actor-style task design
>    --> I personally am a big proponent of that, it will
>
> help
>
> with
>
> well-behaved checkpoints, efficiency, and simpler yet
>
> more
>
> robust
>
> runtime
>
> code
>
>  - Users care about simple abstraction, so as a
>
> subclass
>
> of
>
> SplitReader
>
> (non-blocking / async) we need to have a
>
> BlockingSplitReader
>
> which
>
> will
>
> form the basis of most source implementations.
>
> BlockingSplitReader
>
> lets
>
> users do blocking simple poll() calls.
>  - The BlockingSplitReader would spawn a thread (or
>
> more)
>
> and
>
> the
>
> thread(s) can make blocking calls and hand over data
>
> buffers
>
> via
>
> a
>
> blocking
>
> queue
>  - This should allow us to cover both, a fully async
>
> runtime,
>
> and a
>
> simple
>
> blocking interface for users.
>  - This is actually very similar to how the Kafka
>
> connectors
>
> work.
>
> Kafka
>
> 9+ with one thread, Kafka 8 with multiple threads
>
>  - On the base SplitReader (the async one), the
>
> non-blocking
>
> method
>
> that
>
> gets the next chunk of data would signal data
>
> availability
>
> via
>
> a
>
> CompletableFuture, because that gives the best
>
> flexibility
>
> (can
>
> await
>
> completion or register notification handlers).
>  - The source task would register a "thenHandle()" (or
>
> similar)
>
> on the
>
> future to put a "take next data" task into the
>
> actor-style
>
> mailbox
>
> *(4) Split Enumeration and Assignment*
>
>  - Splits may be generated lazily, both in cases where
>
> there
>
> is a
>
> limited
>
> number of splits (but very many), or splits are
>
> discovered
>
> over
>
> time
>
>  - Assignment should also be lazy, to get better load
>
> balancing
>
>  - Assignment needs support locality preferences
>
>  - Possible design based on discussion so far:
>
>    --> SplitReader has a method "addSplits(SplitT...)"
>
> to
>
> add
>
> one or
>
> more
>
> splits. Some split readers might assume they have only
>
> one
>
> split
>
> ever,
>
> concurrently, others assume multiple splits. (Note:
>
> idea
>
> behind
>
> being
>
> able
>
> to add multiple splits at the same time is to ease
>
> startup
>
> where
>
> multiple
>
> splits may be assigned instantly.)
>    --> SplitReader has a context object on which it can
>
> call
>
> indicate
>
> when
>
> splits are completed. The enumerator gets that
>
> notification and
>
> can
>
> use
>
> to
>
> decide when to assign new splits. This should help both
>
> in
>
> cases
>
> of
>
> sources
>
> that take splits lazily (file readers) and in case the
>
> source
>
> needs to
>
> preserve a partial order between splits (Kinesis,
>
> Pravega,
>
> Pulsar may
>
> need
>
> that).
>    --> SplitEnumerator gets notification when
>
> SplitReaders
>
> start
>
> and
>
> when
>
> they finish splits. They can decide at that moment to
>
> push
>
> more
>
> splits
>
> to
>
> that reader
>    --> The SplitEnumerator should probably be aware of
>
> the
>
> source
>
> parallelism, to build its initial distribution.
>
>  - Open question: Should the source expose something
>
> like
>
> "host
>
> preferences", so that yarn/mesos/k8s can take this into
>
> account
>
> when
>
> selecting a node to start a TM on?
>
> *(5) Watermarks and event time alignment*
>
>  - Watermark generation, as well as idleness, needs to
>
> be
>
> per
>
> split
>
> (like
>
> currently in the Kafka Source, per partition)
>  - It is desirable to support optional
>
> event-time-alignment,
>
> meaning
>
> that
>
> splits that are ahead are back-pressured or temporarily
>
> unsubscribed
>
>  - I think i would be desirable to encapsulate
>
> watermark
>
> generation
>
> logic
>
> in watermark generators, for a separation of concerns.
>
> The
>
> watermark
>
> generators should run per split.
>  - Using watermark generators would also help with
>
> another
>
> problem of
>
> the
>
> suggested interface, namely supporting non-periodic
>
> watermarks
>
> efficiently.
>
>  - Need a way to "dispatch" next record to different
>
> watermark
>
> generators
>
>  - Need a way to tell SplitReader to "suspend" a split
>
> until a
>
> certain
>
> watermark is reached (event time backpressure)
>  - This would in fact be not needed (and thus simpler)
>
> if
>
> we
>
> had
>
> a
>
> SplitReader per split and may be a reason to re-open
>
> that
>
> discussion
>
> *(6) Watermarks across splits and in the Split
>
> Enumerator*
>
>  - The split enumerator may need some watermark
>
> awareness,
>
> which
>
> should
>
> be
>
> purely based on split metadata (like create timestamp
>
> of
>
> file
>
> splits)
>
>  - If there are still more splits with overlapping
>
> event
>
> time
>
> range
>
> for
>
> a
>
> split reader, then that split reader should not advance
>
> the
>
> watermark
>
> within the split beyond the overlap boundary. Otherwise
>
> future
>
> splits
>
> will
>
> produce late data.
>
>  - One way to approach this could be that the split
>
> enumerator
>
> may
>
> send
>
> watermarks to the readers, and the readers cannot emit
>
> watermarks
>
> beyond
>
> that received watermark.
>  - Many split enumerators would simply immediately send
>
> Long.MAX
>
> out
>
> and
>
> leave the progress purely to the split readers.
>
>  - For event-time alignment / split back pressure, this
>
> begs
>
> the
>
> question
>
> how we can avoid deadlocks that may arise when splits
>
> are
>
> suspended
>
> for
>
> event time back pressure,
>
> *(7) Batch and streaming Unification*
>
>  - Functionality wise, the above design should support
>
> both
>
>  - Batch often (mostly) does not care about reading "in
>
> order"
>
> and
>
> generating watermarks
>    --> Might use different enumerator logic that is
>
> more
>
> locality
>
> aware
>
> and ignores event time order
>    --> Does not generate watermarks
>  - Would be great if bounded sources could be
>
> identified
>
> at
>
> compile
>
> time,
>
> so that "env.addBoundedSource(...)" is type safe and
>
> can
>
> return a
>
> "BoundedDataStream".
>  - Possible to defer this discussion until later
>
> *Miscellaneous Comments*
>
>  - Should the source have a TypeInformation for the
>
> produced
>
> type,
>
> instead
>
> of a serializer? We need a type information in the
>
> stream
>
> anyways, and
>
> can
>
> derive the serializer from that. Plus, creating the
>
> serializer
>
> should
>
> respect the ExecutionConfig.
>
>  - The TypeSerializer interface is very powerful but
>
> also
>
> not
>
> easy to
>
> implement. Its purpose is to handle data super
>
> efficiently,
>
> support
>
> flexible ways of evolution, etc.
>  For metadata I would suggest to look at the
>
> SimpleVersionedSerializer
>
> instead, which is used for example for checkpoint
>
> master
>
> hooks,
>
> or for
>
> the
>
> streaming file sink. I think that is is a good match
>
> for
>
> cases
>
> where
>
> we
>
> do
>
> not need more than ser/deser (no copy, etc.) and don't
>
> need to
>
> push
>
> versioning out of the serialization paths for best
>
> performance
>
> (as in
>
> the
>
> TypeSerializer)
>
>
> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <k....@data-artisans.com>
> wrote:
>
>
> Hi Biao,
>
> Thanks for the answer!
>
> So given the multi-threaded readers, now we have as
>
> open
>
> questions:
>
> 1) How do we let the checkpoints pass through our
>
> multi-threaded
>
> reader
>
> operator?
>
> 2) Do we have separate reader and source operators or
>
> not? In
>
> the
>
> strategy
>
> that has a separate source, the source operator has a
>
> parallelism of
>
> 1
>
> and
>
> is responsible for split recovery only.
>
> For the first one, given also the constraints
>
> (blocking,
>
> finite
>
> queues,
>
> etc), I do not have an answer yet.
>
> For the 2nd, I think that we should go with separate
>
> operators
>
> for
>
> the
>
> source and the readers, for the following reasons:
>
> 1) This is more aligned with a potential future
>
> improvement
>
> where the
>
> split
>
> discovery becomes a responsibility of the JobManager
>
> and
>
> readers are
>
> pooling more work from the JM.
>
> 2) The source is going to be the "single point of
>
> truth".
>
> It
>
> will
>
> know
>
> what
>
> has been processed and what not. If the source and the
>
> readers
>
> are a
>
> single
>
> operator with parallelism > 1, or in general, if the
>
> split
>
> discovery
>
> is
>
> done by each task individually, then:
>   i) we have to have a deterministic scheme for each
>
> reader to
>
> assign
>
> splits to itself (e.g. mod subtaskId). This is not
>
> necessarily
>
> trivial
>
> for
>
> all sources.
>   ii) each reader would have to keep a copy of all its
>
> processed
>
> slpits
>
>   iii) the state has to be a union state with a
>
> non-trivial
>
> merging
>
> logic
>
> in order to support rescaling.
>
> Two additional points that you raised above:
>
> i) The point that you raised that we need to keep all
>
> splits
>
> (processed
>
> and
>
> not-processed) I think is a bit of a strong
>
> requirement.
>
> This
>
> would
>
> imply
>
> that for infinite sources the state will grow
>
> indefinitely.
>
> This is
>
> problem
>
> is even more pronounced if we do not have a single
>
> source
>
> that
>
> assigns
>
> splits to readers, as each reader will have its own
>
> copy
>
> of
>
> the
>
> state.
>
> ii) it is true that for finite sources we need to
>
> somehow
>
> not
>
> close
>
> the
>
> readers when the source/split discoverer finishes. The
> ContinuousFileReaderOperator has a work-around for
>
> that.
>
> It is
>
> not
>
> elegant,
>
> and checkpoints are not emitted after closing the
>
> source,
>
> but
>
> this, I
>
> believe, is a bigger problem which requires more
>
> changes
>
> than
>
> just
>
> refactoring the source interface.
>
> Cheers,
> Kostas
>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Dawid Wysakowicz <dw...@apache.org>.
Hi all,

Really well written proposal and very important one. I must admit I have
not understood all the intricacies of it yet.

One question I have though is about where does the information about
boundedness come from. I think in most cases it is a property of the
source. As you described it might be e.g. end offset, a flag should it
monitor new splits etc. I think it would be a really nice use case to be
able to say:

new KafkaSource().readUntil(long timestamp), 

which could work as an "end offset". Moreover I think all Bounded
sources support continuous mode, but no intrinsically continuous source
support the Bounded mode. If I understood the proposal correctly it
suggest the boundedness sort of "comes" from the outside of the source,
from the invokation of either boundedStream or continousSource.

I am wondering if it would make sense to actually change the method

boolean Source#supportsBoundedness(Boundedness) 

to

Boundedness Source#getBoundedness(). 

As for the methods #boundedSource, #continousSource, assuming the
boundedness is property of the source they do not affect how the
enumerator works, but mostly how the dag is scheduled, right? I am not
against those methods, but I think it is a very specific use case to
actually override the property of the source. In general I would expect
users to only call env.source(theSource), where the source tells if it
is bounded or not. I would suggest considering following set of methods:

// boundedness depends of source property, imo this should always be preferred

DataStream<MyType> stream = env.source(theSource);
 

// always continous execution, whether bounded or unbounded source

DataStream<MyType> boundedStream = env.continousSource(theSource);
 
// imo this would make sense if the BoundedDataStream provides additional features unavailable for continous mode
BoundedDataStream<MyType> batch = env.boundedSource(theSource);

Best,

Dawid


On 04/12/2019 11:25, Stephan Ewen wrote:
> Thanks, Becket, for updating this.
>
> I agree with moving the aspects you mentioned into separate FLIPs - this
> one way becoming unwieldy in size.
>
> +1 to the FLIP in its current state. Its a very detailed write-up, nicely
> done!
>
> On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <be...@gmail.com> wrote:
>
>> Hi all,
>>
>> Sorry for the long belated update. I have updated FLIP-27 wiki page with
>> the latest proposals. Some noticeable changes include:
>> 1. A new generic communication mechanism between SplitEnumerator and
>> SourceReader.
>> 2. Some detail API method signature changes.
>>
>> We left a few things out of this FLIP and will address them in separate
>> FLIPs. Including:
>> 1. Per split event time.
>> 2. Event time alignment.
>> 3. Fine grained failover for SplitEnumerator failure.
>>
>> Please let us know if you have any question.
>>
>> Thanks,
>>
>> Jiangjie (Becket) Qin
>>
>> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> wrote:
>>
>>> Hi  Łukasz!
>>>
>>> Becket and me are working hard on figuring out the last details and
>>> implementing the first PoC. We would update the FLIP hopefully next week.
>>>
>>> There is a fair chance that a first version of this will be in 1.10, but
>> I
>>> think it will take another release to battle test it and migrate the
>>> connectors.
>>>
>>> Best,
>>> Stephan
>>>
>>>
>>>
>>>
>>> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <lj...@touk.pl>
>> wrote:
>>>> Hi,
>>>>
>>>> This proposal looks very promising for us. Do you have any plans in
>> which
>>>> Flink release it is going to be released? We are thinking on using a
>> Data
>>>> Set API for our future use cases but on the other hand Data Set API is
>>>> going to be deprecated so using proposed bounded data streams solution
>>>> could be more viable in the long term.
>>>>
>>>> Thanks,
>>>> Łukasz
>>>>
>>>> On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> wrote:
>>>>> Thanks for putting together this proposal!
>>>>>
>>>>> I see that the "Per Split Event Time" and "Event Time Alignment"
>>> sections
>>>>> are still TBD.
>>>>>
>>>>> It would probably be good to flesh those out a bit before proceeding
>>> too
>>>> far
>>>>> as the event time alignment will probably influence the interaction
>>> with
>>>>> the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
>>>>> output).
>>>>>
>>>>> We currently have only one implementation for event time alignment in
>>> the
>>>>> Kinesis consumer. The synchronization in that case takes place as the
>>>> last
>>>>> step before records are emitted downstream (RecordEmitter). With the
>>>>> currently proposed interfaces, the equivalent can be implemented in
>> the
>>>>> reader loop, although note that in the Kinesis consumer the per shard
>>>>> threads push records.
>>>>>
>>>>> Synchronization has not been implemented for the Kafka consumer yet.
>>>>>
>>>>> https://issues.apache.org/jira/browse/FLINK-12675
>>>>>
>>>>> When I looked at it, I realized that the implementation will look
>> quite
>>>>> different
>>>>> from Kinesis because it needs to take place in the pull part, where
>>>> records
>>>>> are taken from the Kafka client. Due to the multiplexing it cannot be
>>>> done
>>>>> by blocking the split thread like it currently works for Kinesis.
>>> Reading
>>>>> from individual Kafka partitions needs to be controlled via
>>> pause/resume
>>>>> on the Kafka client.
>>>>>
>>>>> To take on that responsibility the split thread would need to be
>> aware
>>> of
>>>>> the
>>>>> watermarks or at least whether it should or should not continue to
>>>> consume
>>>>> a given split and this may require a different SourceReader or
>>>> SourceOutput
>>>>> interface.
>>>>>
>>>>> Thanks,
>>>>> Thomas
>>>>>
>>>>>
>>>>> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> wrote:
>>>>>
>>>>>> Hi Stephan,
>>>>>>
>>>>>> Thank you for feedback!
>>>>>> Will take a look at your branch before public discussing.
>>>>>>
>>>>>>
>>>>>> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org>
>>>> wrote:
>>>>>>> Hi Biao!
>>>>>>>
>>>>>>> Thanks for reviving this. I would like to join this discussion,
>> but
>>>> am
>>>>>>> quite occupied with the 1.9 release, so can we maybe pause this
>>>>>> discussion
>>>>>>> for a week or so?
>>>>>>>
>>>>>>> In the meantime I can share some suggestion based on prior
>>>> experiments:
>>>>>>> How to do watermarks / timestamp extractors in a simpler and more
>>>>>> flexible
>>>>>>> way. I think that part is quite promising should be part of the
>> new
>>>>>> source
>>>>>>> interface.
>>>>>>>
>>>>>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>>>>>>>
>>>>>>>
>> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>>>>>>>
>>>>>>>
>>>>>>> Some experiments on how to build the source reader and its
>> library
>>>> for
>>>>>>> common threading/split patterns:
>>>>>>>
>>>>>>>
>> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>>>>>>>
>>>>>>> Best,
>>>>>>> Stephan
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com>
>>>> wrote:
>>>>>>>> Hi devs,
>>>>>>>>
>>>>>>>> Since 1.9 is nearly released, I think we could get back to
>>> FLIP-27.
>>>> I
>>>>>>>> believe it should be included in 1.10.
>>>>>>>>
>>>>>>>> There are so many things mentioned in document of FLIP-27. [1] I
>>>> think
>>>>>>>> we'd better discuss them separately. However the wiki is not a
>>> good
>>>>>> place
>>>>>>>> to discuss. I wrote google doc about SplitReader API which
>> misses
>>>> some
>>>>>>>> details in the document. [2]
>>>>>>>>
>>>>>>>> 1.
>>>>>>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>>>>>>>> 2.
>>>>>>>>
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>>>>>>>> CC Stephan, Aljoscha, Piotrek, Becket
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com>
>>>> wrote:
>>>>>>>>> Hi Steven,
>>>>>>>>> Thank you for the feedback. Please take a look at the document
>>>> FLIP-27
>>>>>>>>> <
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>> which
>>>>>>>>> is updated recently. A lot of details of enumerator were added
>> in
>>>> this
>>>>>>>>> document. I think it would help.
>>>>>>>>>
>>>>>>>>> Steven Wu <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
>>>>>>>>>
>>>>>>>>>> This proposal mentioned that SplitEnumerator might run on the
>>>>>>>>>> JobManager or
>>>>>>>>>> in a single task on a TaskManager.
>>>>>>>>>>
>>>>>>>>>> if enumerator is a single task on a taskmanager, then the job
>>> DAG
>>>> can
>>>>>>>>>> never
>>>>>>>>>> been embarrassingly parallel anymore. That will nullify the
>>>> leverage
>>>>>> of
>>>>>>>>>> fine-grained recovery for embarrassingly parallel jobs.
>>>>>>>>>>
>>>>>>>>>> It's not clear to me what's the implication of running
>>> enumerator
>>>> on
>>>>>> the
>>>>>>>>>> jobmanager. So I will leave that out for now.
>>>>>>>>>>
>>>>>>>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com>
>>>> wrote:
>>>>>>>>>>> Hi Stephan & Piotrek,
>>>>>>>>>>>
>>>>>>>>>>> Thank you for feedback.
>>>>>>>>>>>
>>>>>>>>>>> It seems that there are a lot of things to do in community.
>> I
>>> am
>>>>>> just
>>>>>>>>>>> afraid that this discussion may be forgotten since there so
>>> many
>>>>>>>>>> proposals
>>>>>>>>>>> recently.
>>>>>>>>>>> Anyway, wish to see the split topics soon :)
>>>>>>>>>>>
>>>>>>>>>>> Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四
>>> 下午8:21写道:
>>>>>>>>>>>> Hi Biao!
>>>>>>>>>>>>
>>>>>>>>>>>> This discussion was stalled because of preparations for
>> the
>>>> open
>>>>>>>>>> sourcing
>>>>>>>>>>>> & merging Blink. I think before creating the tickets we
>>> should
>>>>>>>>>> split this
>>>>>>>>>>>> discussion into topics/areas outlined by Stephan and
>> create
>>>> Flips
>>>>>>>>>> for
>>>>>>>>>>> that.
>>>>>>>>>>>> I think there is no chance for this to be completed in
>>> couple
>>>> of
>>>>>>>>>>> remaining
>>>>>>>>>>>> weeks/1 month before 1.8 feature freeze, however it would
>> be
>>>> good
>>>>>>>>>> to aim
>>>>>>>>>>>> with those changes for 1.9.
>>>>>>>>>>>>
>>>>>>>>>>>> Piotrek
>>>>>>>>>>>>
>>>>>>>>>>>>> On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com>
>>>> wrote:
>>>>>>>>>>>>> Hi community,
>>>>>>>>>>>>> The summary of Stephan makes a lot sense to me. It is
>> much
>>>>>> clearer
>>>>>>>>>>> indeed
>>>>>>>>>>>>> after splitting the complex topic into small ones.
>>>>>>>>>>>>> I was wondering is there any detail plan for next step?
>> If
>>>> not,
>>>>>> I
>>>>>>>>>> would
>>>>>>>>>>>>> like to push this thing forward by creating some JIRA
>>>> issues.
>>>>>>>>>>>>> Another question is that should version 1.8 include
>> these
>>>>>>>>>> features?
>>>>>>>>>>>>> Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks everyone for the lively discussion. Let me try
>> to
>>>>>>>>>> summarize
>>>>>>>>>>>> where I
>>>>>>>>>>>>>> see convergence in the discussion and open issues.
>>>>>>>>>>>>>> I'll try to group this by design aspect of the source.
>>>> Please
>>>>>>>>>> let me
>>>>>>>>>>>> know
>>>>>>>>>>>>>> if I got things wrong or missed something crucial here.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> For issues 1-3, if the below reflects the state of the
>>>>>>>>>> discussion, I
>>>>>>>>>>>> would
>>>>>>>>>>>>>> try and update the FLIP in the next days.
>>>>>>>>>>>>>> For the remaining ones we need more discussion.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I would suggest to fork each of these aspects into a
>>>> separate
>>>>>>>>>> mail
>>>>>>>>>>>> thread,
>>>>>>>>>>>>>> or will loose sight of the individual aspects.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> *(1) Separation of Split Enumerator and Split Reader*
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - All seem to agree this is a good thing
>>>>>>>>>>>>>>  - Split Enumerator could in the end live on JobManager
>>>> (and
>>>>>>>>>> assign
>>>>>>>>>>>> splits
>>>>>>>>>>>>>> via RPC) or in a task (and assign splits via data
>>> streams)
>>>>>>>>>>>>>>  - this discussion is orthogonal and should come later,
>>>> when
>>>>>> the
>>>>>>>>>>>> interface
>>>>>>>>>>>>>> is agreed upon.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> *(2) Split Readers for one or more splits*
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - Discussion seems to agree that we need to support
>> one
>>>> reader
>>>>>>>>>> that
>>>>>>>>>>>>>> possibly handles multiple splits concurrently.
>>>>>>>>>>>>>>  - The requirement comes from sources where one
>>>> poll()-style
>>>>>> call
>>>>>>>>>>>> fetches
>>>>>>>>>>>>>> data from different splits / partitions
>>>>>>>>>>>>>>    --> example sources that require that would be for
>>>> example
>>>>>>>>>> Kafka,
>>>>>>>>>>>>>> Pravega, Pulsar
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - Could have one split reader per source, or multiple
>>>> split
>>>>>>>>>> readers
>>>>>>>>>>>> that
>>>>>>>>>>>>>> share the "poll()" function
>>>>>>>>>>>>>>  - To not make it too complicated, we can start with
>>>> thinking
>>>>>>>>>> about
>>>>>>>>>>> one
>>>>>>>>>>>>>> split reader for all splits initially and see if that
>>>> covers
>>>>>> all
>>>>>>>>>>>>>> requirements
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> *(3) Threading model of the Split Reader*
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - Most active part of the discussion ;-)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - A non-blocking way for Flink's task code to interact
>>>> with
>>>>>> the
>>>>>>>>>>> source
>>>>>>>>>>>> is
>>>>>>>>>>>>>> needed in order to a task runtime code based on a
>>>>>>>>>>>>>> single-threaded/actor-style task design
>>>>>>>>>>>>>>    --> I personally am a big proponent of that, it will
>>>> help
>>>>>> with
>>>>>>>>>>>>>> well-behaved checkpoints, efficiency, and simpler yet
>>> more
>>>>>> robust
>>>>>>>>>>>> runtime
>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - Users care about simple abstraction, so as a
>> subclass
>>> of
>>>>>>>>>>> SplitReader
>>>>>>>>>>>>>> (non-blocking / async) we need to have a
>>>> BlockingSplitReader
>>>>>>>>>> which
>>>>>>>>>>> will
>>>>>>>>>>>>>> form the basis of most source implementations.
>>>>>>>>>> BlockingSplitReader
>>>>>>>>>>> lets
>>>>>>>>>>>>>> users do blocking simple poll() calls.
>>>>>>>>>>>>>>  - The BlockingSplitReader would spawn a thread (or
>> more)
>>>> and
>>>>>> the
>>>>>>>>>>>>>> thread(s) can make blocking calls and hand over data
>>>> buffers
>>>>>> via
>>>>>>>>>> a
>>>>>>>>>>>> blocking
>>>>>>>>>>>>>> queue
>>>>>>>>>>>>>>  - This should allow us to cover both, a fully async
>>>> runtime,
>>>>>>>>>> and a
>>>>>>>>>>>> simple
>>>>>>>>>>>>>> blocking interface for users.
>>>>>>>>>>>>>>  - This is actually very similar to how the Kafka
>>>> connectors
>>>>>>>>>> work.
>>>>>>>>>>> Kafka
>>>>>>>>>>>>>> 9+ with one thread, Kafka 8 with multiple threads
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - On the base SplitReader (the async one), the
>>>> non-blocking
>>>>>>>>>> method
>>>>>>>>>>> that
>>>>>>>>>>>>>> gets the next chunk of data would signal data
>>> availability
>>>> via
>>>>>> a
>>>>>>>>>>>>>> CompletableFuture, because that gives the best
>>> flexibility
>>>> (can
>>>>>>>>>> await
>>>>>>>>>>>>>> completion or register notification handlers).
>>>>>>>>>>>>>>  - The source task would register a "thenHandle()" (or
>>>> similar)
>>>>>>>>>> on the
>>>>>>>>>>>>>> future to put a "take next data" task into the
>>> actor-style
>>>>>>>>>> mailbox
>>>>>>>>>>>>>> *(4) Split Enumeration and Assignment*
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - Splits may be generated lazily, both in cases where
>>>> there
>>>>>> is a
>>>>>>>>>>>> limited
>>>>>>>>>>>>>> number of splits (but very many), or splits are
>>> discovered
>>>> over
>>>>>>>>>> time
>>>>>>>>>>>>>>  - Assignment should also be lazy, to get better load
>>>> balancing
>>>>>>>>>>>>>>  - Assignment needs support locality preferences
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - Possible design based on discussion so far:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>    --> SplitReader has a method "addSplits(SplitT...)"
>> to
>>>> add
>>>>>>>>>> one or
>>>>>>>>>>>> more
>>>>>>>>>>>>>> splits. Some split readers might assume they have only
>>> one
>>>>>> split
>>>>>>>>>> ever,
>>>>>>>>>>>>>> concurrently, others assume multiple splits. (Note:
>> idea
>>>> behind
>>>>>>>>>> being
>>>>>>>>>>>> able
>>>>>>>>>>>>>> to add multiple splits at the same time is to ease
>>> startup
>>>>>> where
>>>>>>>>>>>> multiple
>>>>>>>>>>>>>> splits may be assigned instantly.)
>>>>>>>>>>>>>>    --> SplitReader has a context object on which it can
>>>> call
>>>>>>>>>> indicate
>>>>>>>>>>>> when
>>>>>>>>>>>>>> splits are completed. The enumerator gets that
>>>> notification and
>>>>>>>>>> can
>>>>>>>>>>> use
>>>>>>>>>>>> to
>>>>>>>>>>>>>> decide when to assign new splits. This should help both
>>> in
>>>>>> cases
>>>>>>>>>> of
>>>>>>>>>>>> sources
>>>>>>>>>>>>>> that take splits lazily (file readers) and in case the
>>>> source
>>>>>>>>>> needs to
>>>>>>>>>>>>>> preserve a partial order between splits (Kinesis,
>>> Pravega,
>>>>>>>>>> Pulsar may
>>>>>>>>>>>> need
>>>>>>>>>>>>>> that).
>>>>>>>>>>>>>>    --> SplitEnumerator gets notification when
>>> SplitReaders
>>>>>> start
>>>>>>>>>> and
>>>>>>>>>>>> when
>>>>>>>>>>>>>> they finish splits. They can decide at that moment to
>>> push
>>>> more
>>>>>>>>>> splits
>>>>>>>>>>>> to
>>>>>>>>>>>>>> that reader
>>>>>>>>>>>>>>    --> The SplitEnumerator should probably be aware of
>>> the
>>>>>> source
>>>>>>>>>>>>>> parallelism, to build its initial distribution.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - Open question: Should the source expose something
>> like
>>>> "host
>>>>>>>>>>>>>> preferences", so that yarn/mesos/k8s can take this into
>>>> account
>>>>>>>>>> when
>>>>>>>>>>>>>> selecting a node to start a TM on?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> *(5) Watermarks and event time alignment*
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - Watermark generation, as well as idleness, needs to
>> be
>>>> per
>>>>>>>>>> split
>>>>>>>>>>>> (like
>>>>>>>>>>>>>> currently in the Kafka Source, per partition)
>>>>>>>>>>>>>>  - It is desirable to support optional
>>>> event-time-alignment,
>>>>>>>>>> meaning
>>>>>>>>>>>> that
>>>>>>>>>>>>>> splits that are ahead are back-pressured or temporarily
>>>>>>>>>> unsubscribed
>>>>>>>>>>>>>>  - I think i would be desirable to encapsulate
>> watermark
>>>>>>>>>> generation
>>>>>>>>>>>> logic
>>>>>>>>>>>>>> in watermark generators, for a separation of concerns.
>>> The
>>>>>>>>>> watermark
>>>>>>>>>>>>>> generators should run per split.
>>>>>>>>>>>>>>  - Using watermark generators would also help with
>>> another
>>>>>>>>>> problem of
>>>>>>>>>>>> the
>>>>>>>>>>>>>> suggested interface, namely supporting non-periodic
>>>> watermarks
>>>>>>>>>>>> efficiently.
>>>>>>>>>>>>>>  - Need a way to "dispatch" next record to different
>>>> watermark
>>>>>>>>>>>> generators
>>>>>>>>>>>>>>  - Need a way to tell SplitReader to "suspend" a split
>>>> until a
>>>>>>>>>> certain
>>>>>>>>>>>>>> watermark is reached (event time backpressure)
>>>>>>>>>>>>>>  - This would in fact be not needed (and thus simpler)
>> if
>>>> we
>>>>>> had
>>>>>>>>>> a
>>>>>>>>>>>>>> SplitReader per split and may be a reason to re-open
>> that
>>>>>>>>>> discussion
>>>>>>>>>>>>>> *(6) Watermarks across splits and in the Split
>>> Enumerator*
>>>>>>>>>>>>>>  - The split enumerator may need some watermark
>>> awareness,
>>>>>> which
>>>>>>>>>>> should
>>>>>>>>>>>> be
>>>>>>>>>>>>>> purely based on split metadata (like create timestamp
>> of
>>>> file
>>>>>>>>>> splits)
>>>>>>>>>>>>>>  - If there are still more splits with overlapping
>> event
>>>> time
>>>>>>>>>> range
>>>>>>>>>>> for
>>>>>>>>>>>> a
>>>>>>>>>>>>>> split reader, then that split reader should not advance
>>> the
>>>>>>>>>> watermark
>>>>>>>>>>>>>> within the split beyond the overlap boundary. Otherwise
>>>> future
>>>>>>>>>> splits
>>>>>>>>>>>> will
>>>>>>>>>>>>>> produce late data.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - One way to approach this could be that the split
>>>> enumerator
>>>>>>>>>> may
>>>>>>>>>>> send
>>>>>>>>>>>>>> watermarks to the readers, and the readers cannot emit
>>>>>> watermarks
>>>>>>>>>>> beyond
>>>>>>>>>>>>>> that received watermark.
>>>>>>>>>>>>>>  - Many split enumerators would simply immediately send
>>>>>> Long.MAX
>>>>>>>>>> out
>>>>>>>>>>> and
>>>>>>>>>>>>>> leave the progress purely to the split readers.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - For event-time alignment / split back pressure, this
>>>> begs
>>>>>> the
>>>>>>>>>>>> question
>>>>>>>>>>>>>> how we can avoid deadlocks that may arise when splits
>> are
>>>>>>>>>> suspended
>>>>>>>>>>> for
>>>>>>>>>>>>>> event time back pressure,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> *(7) Batch and streaming Unification*
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - Functionality wise, the above design should support
>>> both
>>>>>>>>>>>>>>  - Batch often (mostly) does not care about reading "in
>>>> order"
>>>>>>>>>> and
>>>>>>>>>>>>>> generating watermarks
>>>>>>>>>>>>>>    --> Might use different enumerator logic that is
>> more
>>>>>> locality
>>>>>>>>>>> aware
>>>>>>>>>>>>>> and ignores event time order
>>>>>>>>>>>>>>    --> Does not generate watermarks
>>>>>>>>>>>>>>  - Would be great if bounded sources could be
>> identified
>>> at
>>>>>>>>>> compile
>>>>>>>>>>>> time,
>>>>>>>>>>>>>> so that "env.addBoundedSource(...)" is type safe and
>> can
>>>>>> return a
>>>>>>>>>>>>>> "BoundedDataStream".
>>>>>>>>>>>>>>  - Possible to defer this discussion until later
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> *Miscellaneous Comments*
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - Should the source have a TypeInformation for the
>>>> produced
>>>>>>>>>> type,
>>>>>>>>>>>> instead
>>>>>>>>>>>>>> of a serializer? We need a type information in the
>> stream
>>>>>>>>>> anyways, and
>>>>>>>>>>>> can
>>>>>>>>>>>>>> derive the serializer from that. Plus, creating the
>>>> serializer
>>>>>>>>>> should
>>>>>>>>>>>>>> respect the ExecutionConfig.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>  - The TypeSerializer interface is very powerful but
>> also
>>>> not
>>>>>>>>>> easy to
>>>>>>>>>>>>>> implement. Its purpose is to handle data super
>>> efficiently,
>>>>>>>>>> support
>>>>>>>>>>>>>> flexible ways of evolution, etc.
>>>>>>>>>>>>>>  For metadata I would suggest to look at the
>>>>>>>>>> SimpleVersionedSerializer
>>>>>>>>>>>>>> instead, which is used for example for checkpoint
>> master
>>>> hooks,
>>>>>>>>>> or for
>>>>>>>>>>>> the
>>>>>>>>>>>>>> streaming file sink. I think that is is a good match
>> for
>>>> cases
>>>>>>>>>> where
>>>>>>>>>>> we
>>>>>>>>>>>> do
>>>>>>>>>>>>>> not need more than ser/deser (no copy, etc.) and don't
>>>> need to
>>>>>>>>>> push
>>>>>>>>>>>>>> versioning out of the serialization paths for best
>>>> performance
>>>>>>>>>> (as in
>>>>>>>>>>>> the
>>>>>>>>>>>>>> TypeSerializer)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>>>>>>>>>>>>>> k.kloudas@data-artisans.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Biao,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks for the answer!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> So given the multi-threaded readers, now we have as
>> open
>>>>>>>>>> questions:
>>>>>>>>>>>>>>> 1) How do we let the checkpoints pass through our
>>>>>> multi-threaded
>>>>>>>>>>> reader
>>>>>>>>>>>>>>> operator?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2) Do we have separate reader and source operators or
>>>> not? In
>>>>>>>>>> the
>>>>>>>>>>>>>> strategy
>>>>>>>>>>>>>>> that has a separate source, the source operator has a
>>>>>>>>>> parallelism of
>>>>>>>>>>> 1
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> is responsible for split recovery only.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> For the first one, given also the constraints
>> (blocking,
>>>>>> finite
>>>>>>>>>>> queues,
>>>>>>>>>>>>>>> etc), I do not have an answer yet.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> For the 2nd, I think that we should go with separate
>>>> operators
>>>>>>>>>> for
>>>>>>>>>>> the
>>>>>>>>>>>>>>> source and the readers, for the following reasons:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 1) This is more aligned with a potential future
>>>> improvement
>>>>>>>>>> where the
>>>>>>>>>>>>>> split
>>>>>>>>>>>>>>> discovery becomes a responsibility of the JobManager
>> and
>>>>>>>>>> readers are
>>>>>>>>>>>>>>> pooling more work from the JM.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2) The source is going to be the "single point of
>>> truth".
>>>> It
>>>>>>>>>> will
>>>>>>>>>>> know
>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>> has been processed and what not. If the source and the
>>>> readers
>>>>>>>>>> are a
>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>> operator with parallelism > 1, or in general, if the
>>> split
>>>>>>>>>> discovery
>>>>>>>>>>> is
>>>>>>>>>>>>>>> done by each task individually, then:
>>>>>>>>>>>>>>>   i) we have to have a deterministic scheme for each
>>>> reader to
>>>>>>>>>> assign
>>>>>>>>>>>>>>> splits to itself (e.g. mod subtaskId). This is not
>>>> necessarily
>>>>>>>>>>> trivial
>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>> all sources.
>>>>>>>>>>>>>>>   ii) each reader would have to keep a copy of all its
>>>>>> processed
>>>>>>>>>>> slpits
>>>>>>>>>>>>>>>   iii) the state has to be a union state with a
>>>> non-trivial
>>>>>>>>>> merging
>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>> in order to support rescaling.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Two additional points that you raised above:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> i) The point that you raised that we need to keep all
>>>> splits
>>>>>>>>>>> (processed
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> not-processed) I think is a bit of a strong
>> requirement.
>>>> This
>>>>>>>>>> would
>>>>>>>>>>>> imply
>>>>>>>>>>>>>>> that for infinite sources the state will grow
>>>> indefinitely.
>>>>>>>>>> This is
>>>>>>>>>>>>>> problem
>>>>>>>>>>>>>>> is even more pronounced if we do not have a single
>>> source
>>>> that
>>>>>>>>>>> assigns
>>>>>>>>>>>>>>> splits to readers, as each reader will have its own
>> copy
>>>> of
>>>>>> the
>>>>>>>>>>> state.
>>>>>>>>>>>>>>> ii) it is true that for finite sources we need to
>>> somehow
>>>> not
>>>>>>>>>> close
>>>>>>>>>>> the
>>>>>>>>>>>>>>> readers when the source/split discoverer finishes. The
>>>>>>>>>>>>>>> ContinuousFileReaderOperator has a work-around for
>> that.
>>>> It is
>>>>>>>>>> not
>>>>>>>>>>>>>> elegant,
>>>>>>>>>>>>>>> and checkpoints are not emitted after closing the
>>> source,
>>>> but
>>>>>>>>>> this, I
>>>>>>>>>>>>>>> believe, is a bigger problem which requires more
>> changes
>>>> than
>>>>>>>>>> just
>>>>>>>>>>>>>>> refactoring the source interface.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>> Kostas
>>>>>>>>>>>>>>>
>>>>>>>>>>>>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Stephan Ewen <se...@apache.org>.
Thanks, Becket, for updating this.

I agree with moving the aspects you mentioned into separate FLIPs - this
one way becoming unwieldy in size.

+1 to the FLIP in its current state. Its a very detailed write-up, nicely
done!

On Wed, Dec 4, 2019 at 7:38 AM Becket Qin <be...@gmail.com> wrote:

> Hi all,
>
> Sorry for the long belated update. I have updated FLIP-27 wiki page with
> the latest proposals. Some noticeable changes include:
> 1. A new generic communication mechanism between SplitEnumerator and
> SourceReader.
> 2. Some detail API method signature changes.
>
> We left a few things out of this FLIP and will address them in separate
> FLIPs. Including:
> 1. Per split event time.
> 2. Event time alignment.
> 3. Fine grained failover for SplitEnumerator failure.
>
> Please let us know if you have any question.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> wrote:
>
> > Hi  Łukasz!
> >
> > Becket and me are working hard on figuring out the last details and
> > implementing the first PoC. We would update the FLIP hopefully next week.
> >
> > There is a fair chance that a first version of this will be in 1.10, but
> I
> > think it will take another release to battle test it and migrate the
> > connectors.
> >
> > Best,
> > Stephan
> >
> >
> >
> >
> > On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <lj...@touk.pl>
> wrote:
> >
> > > Hi,
> > >
> > > This proposal looks very promising for us. Do you have any plans in
> which
> > > Flink release it is going to be released? We are thinking on using a
> Data
> > > Set API for our future use cases but on the other hand Data Set API is
> > > going to be deprecated so using proposed bounded data streams solution
> > > could be more viable in the long term.
> > >
> > > Thanks,
> > > Łukasz
> > >
> > > On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> wrote:
> > > > Thanks for putting together this proposal!
> > > >
> > > > I see that the "Per Split Event Time" and "Event Time Alignment"
> > sections
> > > > are still TBD.
> > > >
> > > > It would probably be good to flesh those out a bit before proceeding
> > too
> > > far
> > > > as the event time alignment will probably influence the interaction
> > with
> > > > the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
> > > > output).
> > > >
> > > > We currently have only one implementation for event time alignment in
> > the
> > > > Kinesis consumer. The synchronization in that case takes place as the
> > > last
> > > > step before records are emitted downstream (RecordEmitter). With the
> > > > currently proposed interfaces, the equivalent can be implemented in
> the
> > > > reader loop, although note that in the Kinesis consumer the per shard
> > > > threads push records.
> > > >
> > > > Synchronization has not been implemented for the Kafka consumer yet.
> > > >
> > > > https://issues.apache.org/jira/browse/FLINK-12675
> > > >
> > > > When I looked at it, I realized that the implementation will look
> quite
> > > > different
> > > > from Kinesis because it needs to take place in the pull part, where
> > > records
> > > > are taken from the Kafka client. Due to the multiplexing it cannot be
> > > done
> > > > by blocking the split thread like it currently works for Kinesis.
> > Reading
> > > > from individual Kafka partitions needs to be controlled via
> > pause/resume
> > > > on the Kafka client.
> > > >
> > > > To take on that responsibility the split thread would need to be
> aware
> > of
> > > > the
> > > > watermarks or at least whether it should or should not continue to
> > > consume
> > > > a given split and this may require a different SourceReader or
> > > SourceOutput
> > > > interface.
> > > >
> > > > Thanks,
> > > > Thomas
> > > >
> > > >
> > > > On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> wrote:
> > > >
> > > > > Hi Stephan,
> > > > >
> > > > > Thank you for feedback!
> > > > > Will take a look at your branch before public discussing.
> > > > >
> > > > >
> > > > > On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org>
> > > wrote:
> > > > >
> > > > > > Hi Biao!
> > > > > >
> > > > > > Thanks for reviving this. I would like to join this discussion,
> but
> > > am
> > > > > > quite occupied with the 1.9 release, so can we maybe pause this
> > > > > discussion
> > > > > > for a week or so?
> > > > > >
> > > > > > In the meantime I can share some suggestion based on prior
> > > experiments:
> > > > > >
> > > > > > How to do watermarks / timestamp extractors in a simpler and more
> > > > > flexible
> > > > > > way. I think that part is quite promising should be part of the
> new
> > > > > source
> > > > > > interface.
> > > > > >
> > > > > >
> > > > >
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > >
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > > > > >
> > > > > >
> > > > > >
> > > > > > Some experiments on how to build the source reader and its
> library
> > > for
> > > > > > common threading/split patterns:
> > > > > >
> > > > > >
> > > > >
> > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > > > > >
> > > > > >
> > > > > > Best,
> > > > > > Stephan
> > > > > >
> > > > > >
> > > > > > On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com>
> > > wrote:
> > > > > >
> > > > > >> Hi devs,
> > > > > >>
> > > > > >> Since 1.9 is nearly released, I think we could get back to
> > FLIP-27.
> > > I
> > > > > >> believe it should be included in 1.10.
> > > > > >>
> > > > > >> There are so many things mentioned in document of FLIP-27. [1] I
> > > think
> > > > > >> we'd better discuss them separately. However the wiki is not a
> > good
> > > > > place
> > > > > >> to discuss. I wrote google doc about SplitReader API which
> misses
> > > some
> > > > > >> details in the document. [2]
> > > > > >>
> > > > > >> 1.
> > > > > >>
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > > > >> 2.
> > > > > >>
> > > > >
> > >
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > > > > >>
> > > > > >> CC Stephan, Aljoscha, Piotrek, Becket
> > > > > >>
> > > > > >>
> > > > > >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com>
> > > wrote:
> > > > > >>
> > > > > >>> Hi Steven,
> > > > > >>> Thank you for the feedback. Please take a look at the document
> > > FLIP-27
> > > > > >>> <
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > >
> > > > > which
> > > > > >>> is updated recently. A lot of details of enumerator were added
> in
> > > this
> > > > > >>> document. I think it would help.
> > > > > >>>
> > > > > >>> Steven Wu <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
> > > > > >>>
> > > > > >>>> This proposal mentioned that SplitEnumerator might run on the
> > > > > >>>> JobManager or
> > > > > >>>> in a single task on a TaskManager.
> > > > > >>>>
> > > > > >>>> if enumerator is a single task on a taskmanager, then the job
> > DAG
> > > can
> > > > > >>>> never
> > > > > >>>> been embarrassingly parallel anymore. That will nullify the
> > > leverage
> > > > > of
> > > > > >>>> fine-grained recovery for embarrassingly parallel jobs.
> > > > > >>>>
> > > > > >>>> It's not clear to me what's the implication of running
> > enumerator
> > > on
> > > > > the
> > > > > >>>> jobmanager. So I will leave that out for now.
> > > > > >>>>
> > > > > >>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com>
> > > wrote:
> > > > > >>>>
> > > > > >>>> > Hi Stephan & Piotrek,
> > > > > >>>> >
> > > > > >>>> > Thank you for feedback.
> > > > > >>>> >
> > > > > >>>> > It seems that there are a lot of things to do in community.
> I
> > am
> > > > > just
> > > > > >>>> > afraid that this discussion may be forgotten since there so
> > many
> > > > > >>>> proposals
> > > > > >>>> > recently.
> > > > > >>>> > Anyway, wish to see the split topics soon :)
> > > > > >>>> >
> > > > > >>>> > Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四
> > 下午8:21写道:
> > > > > >>>> >
> > > > > >>>> > > Hi Biao!
> > > > > >>>> > >
> > > > > >>>> > > This discussion was stalled because of preparations for
> the
> > > open
> > > > > >>>> sourcing
> > > > > >>>> > > & merging Blink. I think before creating the tickets we
> > should
> > > > > >>>> split this
> > > > > >>>> > > discussion into topics/areas outlined by Stephan and
> create
> > > Flips
> > > > > >>>> for
> > > > > >>>> > that.
> > > > > >>>> > >
> > > > > >>>> > > I think there is no chance for this to be completed in
> > couple
> > > of
> > > > > >>>> > remaining
> > > > > >>>> > > weeks/1 month before 1.8 feature freeze, however it would
> be
> > > good
> > > > > >>>> to aim
> > > > > >>>> > > with those changes for 1.9.
> > > > > >>>> > >
> > > > > >>>> > > Piotrek
> > > > > >>>> > >
> > > > > >>>> > > > On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com>
> > > wrote:
> > > > > >>>> > > >
> > > > > >>>> > > > Hi community,
> > > > > >>>> > > > The summary of Stephan makes a lot sense to me. It is
> much
> > > > > clearer
> > > > > >>>> > indeed
> > > > > >>>> > > > after splitting the complex topic into small ones.
> > > > > >>>> > > > I was wondering is there any detail plan for next step?
> If
> > > not,
> > > > > I
> > > > > >>>> would
> > > > > >>>> > > > like to push this thing forward by creating some JIRA
> > > issues.
> > > > > >>>> > > > Another question is that should version 1.8 include
> these
> > > > > >>>> features?
> > > > > >>>> > > >
> > > > > >>>> > > > Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
> > > > > >>>> > > >
> > > > > >>>> > > >> Thanks everyone for the lively discussion. Let me try
> to
> > > > > >>>> summarize
> > > > > >>>> > > where I
> > > > > >>>> > > >> see convergence in the discussion and open issues.
> > > > > >>>> > > >> I'll try to group this by design aspect of the source.
> > > Please
> > > > > >>>> let me
> > > > > >>>> > > know
> > > > > >>>> > > >> if I got things wrong or missed something crucial here.
> > > > > >>>> > > >>
> > > > > >>>> > > >> For issues 1-3, if the below reflects the state of the
> > > > > >>>> discussion, I
> > > > > >>>> > > would
> > > > > >>>> > > >> try and update the FLIP in the next days.
> > > > > >>>> > > >> For the remaining ones we need more discussion.
> > > > > >>>> > > >>
> > > > > >>>> > > >> I would suggest to fork each of these aspects into a
> > > separate
> > > > > >>>> mail
> > > > > >>>> > > thread,
> > > > > >>>> > > >> or will loose sight of the individual aspects.
> > > > > >>>> > > >>
> > > > > >>>> > > >> *(1) Separation of Split Enumerator and Split Reader*
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - All seem to agree this is a good thing
> > > > > >>>> > > >>  - Split Enumerator could in the end live on JobManager
> > > (and
> > > > > >>>> assign
> > > > > >>>> > > splits
> > > > > >>>> > > >> via RPC) or in a task (and assign splits via data
> > streams)
> > > > > >>>> > > >>  - this discussion is orthogonal and should come later,
> > > when
> > > > > the
> > > > > >>>> > > interface
> > > > > >>>> > > >> is agreed upon.
> > > > > >>>> > > >>
> > > > > >>>> > > >> *(2) Split Readers for one or more splits*
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - Discussion seems to agree that we need to support
> one
> > > reader
> > > > > >>>> that
> > > > > >>>> > > >> possibly handles multiple splits concurrently.
> > > > > >>>> > > >>  - The requirement comes from sources where one
> > > poll()-style
> > > > > call
> > > > > >>>> > > fetches
> > > > > >>>> > > >> data from different splits / partitions
> > > > > >>>> > > >>    --> example sources that require that would be for
> > > example
> > > > > >>>> Kafka,
> > > > > >>>> > > >> Pravega, Pulsar
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - Could have one split reader per source, or multiple
> > > split
> > > > > >>>> readers
> > > > > >>>> > > that
> > > > > >>>> > > >> share the "poll()" function
> > > > > >>>> > > >>  - To not make it too complicated, we can start with
> > > thinking
> > > > > >>>> about
> > > > > >>>> > one
> > > > > >>>> > > >> split reader for all splits initially and see if that
> > > covers
> > > > > all
> > > > > >>>> > > >> requirements
> > > > > >>>> > > >>
> > > > > >>>> > > >> *(3) Threading model of the Split Reader*
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - Most active part of the discussion ;-)
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - A non-blocking way for Flink's task code to interact
> > > with
> > > > > the
> > > > > >>>> > source
> > > > > >>>> > > is
> > > > > >>>> > > >> needed in order to a task runtime code based on a
> > > > > >>>> > > >> single-threaded/actor-style task design
> > > > > >>>> > > >>    --> I personally am a big proponent of that, it will
> > > help
> > > > > with
> > > > > >>>> > > >> well-behaved checkpoints, efficiency, and simpler yet
> > more
> > > > > robust
> > > > > >>>> > > runtime
> > > > > >>>> > > >> code
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - Users care about simple abstraction, so as a
> subclass
> > of
> > > > > >>>> > SplitReader
> > > > > >>>> > > >> (non-blocking / async) we need to have a
> > > BlockingSplitReader
> > > > > >>>> which
> > > > > >>>> > will
> > > > > >>>> > > >> form the basis of most source implementations.
> > > > > >>>> BlockingSplitReader
> > > > > >>>> > lets
> > > > > >>>> > > >> users do blocking simple poll() calls.
> > > > > >>>> > > >>  - The BlockingSplitReader would spawn a thread (or
> more)
> > > and
> > > > > the
> > > > > >>>> > > >> thread(s) can make blocking calls and hand over data
> > > buffers
> > > > > via
> > > > > >>>> a
> > > > > >>>> > > blocking
> > > > > >>>> > > >> queue
> > > > > >>>> > > >>  - This should allow us to cover both, a fully async
> > > runtime,
> > > > > >>>> and a
> > > > > >>>> > > simple
> > > > > >>>> > > >> blocking interface for users.
> > > > > >>>> > > >>  - This is actually very similar to how the Kafka
> > > connectors
> > > > > >>>> work.
> > > > > >>>> > Kafka
> > > > > >>>> > > >> 9+ with one thread, Kafka 8 with multiple threads
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - On the base SplitReader (the async one), the
> > > non-blocking
> > > > > >>>> method
> > > > > >>>> > that
> > > > > >>>> > > >> gets the next chunk of data would signal data
> > availability
> > > via
> > > > > a
> > > > > >>>> > > >> CompletableFuture, because that gives the best
> > flexibility
> > > (can
> > > > > >>>> await
> > > > > >>>> > > >> completion or register notification handlers).
> > > > > >>>> > > >>  - The source task would register a "thenHandle()" (or
> > > similar)
> > > > > >>>> on the
> > > > > >>>> > > >> future to put a "take next data" task into the
> > actor-style
> > > > > >>>> mailbox
> > > > > >>>> > > >>
> > > > > >>>> > > >> *(4) Split Enumeration and Assignment*
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - Splits may be generated lazily, both in cases where
> > > there
> > > > > is a
> > > > > >>>> > > limited
> > > > > >>>> > > >> number of splits (but very many), or splits are
> > discovered
> > > over
> > > > > >>>> time
> > > > > >>>> > > >>  - Assignment should also be lazy, to get better load
> > > balancing
> > > > > >>>> > > >>  - Assignment needs support locality preferences
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - Possible design based on discussion so far:
> > > > > >>>> > > >>
> > > > > >>>> > > >>    --> SplitReader has a method "addSplits(SplitT...)"
> to
> > > add
> > > > > >>>> one or
> > > > > >>>> > > more
> > > > > >>>> > > >> splits. Some split readers might assume they have only
> > one
> > > > > split
> > > > > >>>> ever,
> > > > > >>>> > > >> concurrently, others assume multiple splits. (Note:
> idea
> > > behind
> > > > > >>>> being
> > > > > >>>> > > able
> > > > > >>>> > > >> to add multiple splits at the same time is to ease
> > startup
> > > > > where
> > > > > >>>> > > multiple
> > > > > >>>> > > >> splits may be assigned instantly.)
> > > > > >>>> > > >>    --> SplitReader has a context object on which it can
> > > call
> > > > > >>>> indicate
> > > > > >>>> > > when
> > > > > >>>> > > >> splits are completed. The enumerator gets that
> > > notification and
> > > > > >>>> can
> > > > > >>>> > use
> > > > > >>>> > > to
> > > > > >>>> > > >> decide when to assign new splits. This should help both
> > in
> > > > > cases
> > > > > >>>> of
> > > > > >>>> > > sources
> > > > > >>>> > > >> that take splits lazily (file readers) and in case the
> > > source
> > > > > >>>> needs to
> > > > > >>>> > > >> preserve a partial order between splits (Kinesis,
> > Pravega,
> > > > > >>>> Pulsar may
> > > > > >>>> > > need
> > > > > >>>> > > >> that).
> > > > > >>>> > > >>    --> SplitEnumerator gets notification when
> > SplitReaders
> > > > > start
> > > > > >>>> and
> > > > > >>>> > > when
> > > > > >>>> > > >> they finish splits. They can decide at that moment to
> > push
> > > more
> > > > > >>>> splits
> > > > > >>>> > > to
> > > > > >>>> > > >> that reader
> > > > > >>>> > > >>    --> The SplitEnumerator should probably be aware of
> > the
> > > > > source
> > > > > >>>> > > >> parallelism, to build its initial distribution.
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - Open question: Should the source expose something
> like
> > > "host
> > > > > >>>> > > >> preferences", so that yarn/mesos/k8s can take this into
> > > account
> > > > > >>>> when
> > > > > >>>> > > >> selecting a node to start a TM on?
> > > > > >>>> > > >>
> > > > > >>>> > > >> *(5) Watermarks and event time alignment*
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - Watermark generation, as well as idleness, needs to
> be
> > > per
> > > > > >>>> split
> > > > > >>>> > > (like
> > > > > >>>> > > >> currently in the Kafka Source, per partition)
> > > > > >>>> > > >>  - It is desirable to support optional
> > > event-time-alignment,
> > > > > >>>> meaning
> > > > > >>>> > > that
> > > > > >>>> > > >> splits that are ahead are back-pressured or temporarily
> > > > > >>>> unsubscribed
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - I think i would be desirable to encapsulate
> watermark
> > > > > >>>> generation
> > > > > >>>> > > logic
> > > > > >>>> > > >> in watermark generators, for a separation of concerns.
> > The
> > > > > >>>> watermark
> > > > > >>>> > > >> generators should run per split.
> > > > > >>>> > > >>  - Using watermark generators would also help with
> > another
> > > > > >>>> problem of
> > > > > >>>> > > the
> > > > > >>>> > > >> suggested interface, namely supporting non-periodic
> > > watermarks
> > > > > >>>> > > efficiently.
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - Need a way to "dispatch" next record to different
> > > watermark
> > > > > >>>> > > generators
> > > > > >>>> > > >>  - Need a way to tell SplitReader to "suspend" a split
> > > until a
> > > > > >>>> certain
> > > > > >>>> > > >> watermark is reached (event time backpressure)
> > > > > >>>> > > >>  - This would in fact be not needed (and thus simpler)
> if
> > > we
> > > > > had
> > > > > >>>> a
> > > > > >>>> > > >> SplitReader per split and may be a reason to re-open
> that
> > > > > >>>> discussion
> > > > > >>>> > > >>
> > > > > >>>> > > >> *(6) Watermarks across splits and in the Split
> > Enumerator*
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - The split enumerator may need some watermark
> > awareness,
> > > > > which
> > > > > >>>> > should
> > > > > >>>> > > be
> > > > > >>>> > > >> purely based on split metadata (like create timestamp
> of
> > > file
> > > > > >>>> splits)
> > > > > >>>> > > >>  - If there are still more splits with overlapping
> event
> > > time
> > > > > >>>> range
> > > > > >>>> > for
> > > > > >>>> > > a
> > > > > >>>> > > >> split reader, then that split reader should not advance
> > the
> > > > > >>>> watermark
> > > > > >>>> > > >> within the split beyond the overlap boundary. Otherwise
> > > future
> > > > > >>>> splits
> > > > > >>>> > > will
> > > > > >>>> > > >> produce late data.
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - One way to approach this could be that the split
> > > enumerator
> > > > > >>>> may
> > > > > >>>> > send
> > > > > >>>> > > >> watermarks to the readers, and the readers cannot emit
> > > > > watermarks
> > > > > >>>> > beyond
> > > > > >>>> > > >> that received watermark.
> > > > > >>>> > > >>  - Many split enumerators would simply immediately send
> > > > > Long.MAX
> > > > > >>>> out
> > > > > >>>> > and
> > > > > >>>> > > >> leave the progress purely to the split readers.
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - For event-time alignment / split back pressure, this
> > > begs
> > > > > the
> > > > > >>>> > > question
> > > > > >>>> > > >> how we can avoid deadlocks that may arise when splits
> are
> > > > > >>>> suspended
> > > > > >>>> > for
> > > > > >>>> > > >> event time back pressure,
> > > > > >>>> > > >>
> > > > > >>>> > > >> *(7) Batch and streaming Unification*
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - Functionality wise, the above design should support
> > both
> > > > > >>>> > > >>  - Batch often (mostly) does not care about reading "in
> > > order"
> > > > > >>>> and
> > > > > >>>> > > >> generating watermarks
> > > > > >>>> > > >>    --> Might use different enumerator logic that is
> more
> > > > > locality
> > > > > >>>> > aware
> > > > > >>>> > > >> and ignores event time order
> > > > > >>>> > > >>    --> Does not generate watermarks
> > > > > >>>> > > >>  - Would be great if bounded sources could be
> identified
> > at
> > > > > >>>> compile
> > > > > >>>> > > time,
> > > > > >>>> > > >> so that "env.addBoundedSource(...)" is type safe and
> can
> > > > > return a
> > > > > >>>> > > >> "BoundedDataStream".
> > > > > >>>> > > >>  - Possible to defer this discussion until later
> > > > > >>>> > > >>
> > > > > >>>> > > >> *Miscellaneous Comments*
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - Should the source have a TypeInformation for the
> > > produced
> > > > > >>>> type,
> > > > > >>>> > > instead
> > > > > >>>> > > >> of a serializer? We need a type information in the
> stream
> > > > > >>>> anyways, and
> > > > > >>>> > > can
> > > > > >>>> > > >> derive the serializer from that. Plus, creating the
> > > serializer
> > > > > >>>> should
> > > > > >>>> > > >> respect the ExecutionConfig.
> > > > > >>>> > > >>
> > > > > >>>> > > >>  - The TypeSerializer interface is very powerful but
> also
> > > not
> > > > > >>>> easy to
> > > > > >>>> > > >> implement. Its purpose is to handle data super
> > efficiently,
> > > > > >>>> support
> > > > > >>>> > > >> flexible ways of evolution, etc.
> > > > > >>>> > > >>  For metadata I would suggest to look at the
> > > > > >>>> SimpleVersionedSerializer
> > > > > >>>> > > >> instead, which is used for example for checkpoint
> master
> > > hooks,
> > > > > >>>> or for
> > > > > >>>> > > the
> > > > > >>>> > > >> streaming file sink. I think that is is a good match
> for
> > > cases
> > > > > >>>> where
> > > > > >>>> > we
> > > > > >>>> > > do
> > > > > >>>> > > >> not need more than ser/deser (no copy, etc.) and don't
> > > need to
> > > > > >>>> push
> > > > > >>>> > > >> versioning out of the serialization paths for best
> > > performance
> > > > > >>>> (as in
> > > > > >>>> > > the
> > > > > >>>> > > >> TypeSerializer)
> > > > > >>>> > > >>
> > > > > >>>> > > >>
> > > > > >>>> > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > > > > >>>> > > >> k.kloudas@data-artisans.com>
> > > > > >>>> > > >> wrote:
> > > > > >>>> > > >>
> > > > > >>>> > > >>> Hi Biao,
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> Thanks for the answer!
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> So given the multi-threaded readers, now we have as
> open
> > > > > >>>> questions:
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> 1) How do we let the checkpoints pass through our
> > > > > multi-threaded
> > > > > >>>> > reader
> > > > > >>>> > > >>> operator?
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> 2) Do we have separate reader and source operators or
> > > not? In
> > > > > >>>> the
> > > > > >>>> > > >> strategy
> > > > > >>>> > > >>> that has a separate source, the source operator has a
> > > > > >>>> parallelism of
> > > > > >>>> > 1
> > > > > >>>> > > >> and
> > > > > >>>> > > >>> is responsible for split recovery only.
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> For the first one, given also the constraints
> (blocking,
> > > > > finite
> > > > > >>>> > queues,
> > > > > >>>> > > >>> etc), I do not have an answer yet.
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> For the 2nd, I think that we should go with separate
> > > operators
> > > > > >>>> for
> > > > > >>>> > the
> > > > > >>>> > > >>> source and the readers, for the following reasons:
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> 1) This is more aligned with a potential future
> > > improvement
> > > > > >>>> where the
> > > > > >>>> > > >> split
> > > > > >>>> > > >>> discovery becomes a responsibility of the JobManager
> and
> > > > > >>>> readers are
> > > > > >>>> > > >>> pooling more work from the JM.
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> 2) The source is going to be the "single point of
> > truth".
> > > It
> > > > > >>>> will
> > > > > >>>> > know
> > > > > >>>> > > >> what
> > > > > >>>> > > >>> has been processed and what not. If the source and the
> > > readers
> > > > > >>>> are a
> > > > > >>>> > > >> single
> > > > > >>>> > > >>> operator with parallelism > 1, or in general, if the
> > split
> > > > > >>>> discovery
> > > > > >>>> > is
> > > > > >>>> > > >>> done by each task individually, then:
> > > > > >>>> > > >>>   i) we have to have a deterministic scheme for each
> > > reader to
> > > > > >>>> assign
> > > > > >>>> > > >>> splits to itself (e.g. mod subtaskId). This is not
> > > necessarily
> > > > > >>>> > trivial
> > > > > >>>> > > >> for
> > > > > >>>> > > >>> all sources.
> > > > > >>>> > > >>>   ii) each reader would have to keep a copy of all its
> > > > > processed
> > > > > >>>> > slpits
> > > > > >>>> > > >>>   iii) the state has to be a union state with a
> > > non-trivial
> > > > > >>>> merging
> > > > > >>>> > > >> logic
> > > > > >>>> > > >>> in order to support rescaling.
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> Two additional points that you raised above:
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> i) The point that you raised that we need to keep all
> > > splits
> > > > > >>>> > (processed
> > > > > >>>> > > >> and
> > > > > >>>> > > >>> not-processed) I think is a bit of a strong
> requirement.
> > > This
> > > > > >>>> would
> > > > > >>>> > > imply
> > > > > >>>> > > >>> that for infinite sources the state will grow
> > > indefinitely.
> > > > > >>>> This is
> > > > > >>>> > > >> problem
> > > > > >>>> > > >>> is even more pronounced if we do not have a single
> > source
> > > that
> > > > > >>>> > assigns
> > > > > >>>> > > >>> splits to readers, as each reader will have its own
> copy
> > > of
> > > > > the
> > > > > >>>> > state.
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> ii) it is true that for finite sources we need to
> > somehow
> > > not
> > > > > >>>> close
> > > > > >>>> > the
> > > > > >>>> > > >>> readers when the source/split discoverer finishes. The
> > > > > >>>> > > >>> ContinuousFileReaderOperator has a work-around for
> that.
> > > It is
> > > > > >>>> not
> > > > > >>>> > > >> elegant,
> > > > > >>>> > > >>> and checkpoints are not emitted after closing the
> > source,
> > > but
> > > > > >>>> this, I
> > > > > >>>> > > >>> believe, is a bigger problem which requires more
> changes
> > > than
> > > > > >>>> just
> > > > > >>>> > > >>> refactoring the source interface.
> > > > > >>>> > > >>>
> > > > > >>>> > > >>> Cheers,
> > > > > >>>> > > >>> Kostas
> > > > > >>>> > > >>>
> > > > > >>>> > > >>
> > > > > >>>> > >
> > > > > >>>> > >
> > > > > >>>> >
> > > > > >>>>
> > > > > >>>
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi all,

Sorry for the long belated update. I have updated FLIP-27 wiki page with
the latest proposals. Some noticeable changes include:
1. A new generic communication mechanism between SplitEnumerator and
SourceReader.
2. Some detail API method signature changes.

We left a few things out of this FLIP and will address them in separate
FLIPs. Including:
1. Per split event time.
2. Event time alignment.
3. Fine grained failover for SplitEnumerator failure.

Please let us know if you have any question.

Thanks,

Jiangjie (Becket) Qin

On Sat, Nov 16, 2019 at 6:10 AM Stephan Ewen <se...@apache.org> wrote:

> Hi  Łukasz!
>
> Becket and me are working hard on figuring out the last details and
> implementing the first PoC. We would update the FLIP hopefully next week.
>
> There is a fair chance that a first version of this will be in 1.10, but I
> think it will take another release to battle test it and migrate the
> connectors.
>
> Best,
> Stephan
>
>
>
>
> On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <lj...@touk.pl> wrote:
>
> > Hi,
> >
> > This proposal looks very promising for us. Do you have any plans in which
> > Flink release it is going to be released? We are thinking on using a Data
> > Set API for our future use cases but on the other hand Data Set API is
> > going to be deprecated so using proposed bounded data streams solution
> > could be more viable in the long term.
> >
> > Thanks,
> > Łukasz
> >
> > On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> wrote:
> > > Thanks for putting together this proposal!
> > >
> > > I see that the "Per Split Event Time" and "Event Time Alignment"
> sections
> > > are still TBD.
> > >
> > > It would probably be good to flesh those out a bit before proceeding
> too
> > far
> > > as the event time alignment will probably influence the interaction
> with
> > > the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
> > > output).
> > >
> > > We currently have only one implementation for event time alignment in
> the
> > > Kinesis consumer. The synchronization in that case takes place as the
> > last
> > > step before records are emitted downstream (RecordEmitter). With the
> > > currently proposed interfaces, the equivalent can be implemented in the
> > > reader loop, although note that in the Kinesis consumer the per shard
> > > threads push records.
> > >
> > > Synchronization has not been implemented for the Kafka consumer yet.
> > >
> > > https://issues.apache.org/jira/browse/FLINK-12675
> > >
> > > When I looked at it, I realized that the implementation will look quite
> > > different
> > > from Kinesis because it needs to take place in the pull part, where
> > records
> > > are taken from the Kafka client. Due to the multiplexing it cannot be
> > done
> > > by blocking the split thread like it currently works for Kinesis.
> Reading
> > > from individual Kafka partitions needs to be controlled via
> pause/resume
> > > on the Kafka client.
> > >
> > > To take on that responsibility the split thread would need to be aware
> of
> > > the
> > > watermarks or at least whether it should or should not continue to
> > consume
> > > a given split and this may require a different SourceReader or
> > SourceOutput
> > > interface.
> > >
> > > Thanks,
> > > Thomas
> > >
> > >
> > > On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> wrote:
> > >
> > > > Hi Stephan,
> > > >
> > > > Thank you for feedback!
> > > > Will take a look at your branch before public discussing.
> > > >
> > > >
> > > > On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org>
> > wrote:
> > > >
> > > > > Hi Biao!
> > > > >
> > > > > Thanks for reviving this. I would like to join this discussion, but
> > am
> > > > > quite occupied with the 1.9 release, so can we maybe pause this
> > > > discussion
> > > > > for a week or so?
> > > > >
> > > > > In the meantime I can share some suggestion based on prior
> > experiments:
> > > > >
> > > > > How to do watermarks / timestamp extractors in a simpler and more
> > > > flexible
> > > > > way. I think that part is quite promising should be part of the new
> > > > source
> > > > > interface.
> > > > >
> > > > >
> > > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > > > >
> > > > >
> > > > >
> > > >
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > > > >
> > > > >
> > > > >
> > > > > Some experiments on how to build the source reader and its library
> > for
> > > > > common threading/split patterns:
> > > > >
> > > > >
> > > >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > > > >
> > > > >
> > > > > Best,
> > > > > Stephan
> > > > >
> > > > >
> > > > > On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com>
> > wrote:
> > > > >
> > > > >> Hi devs,
> > > > >>
> > > > >> Since 1.9 is nearly released, I think we could get back to
> FLIP-27.
> > I
> > > > >> believe it should be included in 1.10.
> > > > >>
> > > > >> There are so many things mentioned in document of FLIP-27. [1] I
> > think
> > > > >> we'd better discuss them separately. However the wiki is not a
> good
> > > > place
> > > > >> to discuss. I wrote google doc about SplitReader API which misses
> > some
> > > > >> details in the document. [2]
> > > > >>
> > > > >> 1.
> > > > >>
> > > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > > >> 2.
> > > > >>
> > > >
> >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > > > >>
> > > > >> CC Stephan, Aljoscha, Piotrek, Becket
> > > > >>
> > > > >>
> > > > >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com>
> > wrote:
> > > > >>
> > > > >>> Hi Steven,
> > > > >>> Thank you for the feedback. Please take a look at the document
> > FLIP-27
> > > > >>> <
> > > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >
> > > > which
> > > > >>> is updated recently. A lot of details of enumerator were added in
> > this
> > > > >>> document. I think it would help.
> > > > >>>
> > > > >>> Steven Wu <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
> > > > >>>
> > > > >>>> This proposal mentioned that SplitEnumerator might run on the
> > > > >>>> JobManager or
> > > > >>>> in a single task on a TaskManager.
> > > > >>>>
> > > > >>>> if enumerator is a single task on a taskmanager, then the job
> DAG
> > can
> > > > >>>> never
> > > > >>>> been embarrassingly parallel anymore. That will nullify the
> > leverage
> > > > of
> > > > >>>> fine-grained recovery for embarrassingly parallel jobs.
> > > > >>>>
> > > > >>>> It's not clear to me what's the implication of running
> enumerator
> > on
> > > > the
> > > > >>>> jobmanager. So I will leave that out for now.
> > > > >>>>
> > > > >>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com>
> > wrote:
> > > > >>>>
> > > > >>>> > Hi Stephan & Piotrek,
> > > > >>>> >
> > > > >>>> > Thank you for feedback.
> > > > >>>> >
> > > > >>>> > It seems that there are a lot of things to do in community. I
> am
> > > > just
> > > > >>>> > afraid that this discussion may be forgotten since there so
> many
> > > > >>>> proposals
> > > > >>>> > recently.
> > > > >>>> > Anyway, wish to see the split topics soon :)
> > > > >>>> >
> > > > >>>> > Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四
> 下午8:21写道:
> > > > >>>> >
> > > > >>>> > > Hi Biao!
> > > > >>>> > >
> > > > >>>> > > This discussion was stalled because of preparations for the
> > open
> > > > >>>> sourcing
> > > > >>>> > > & merging Blink. I think before creating the tickets we
> should
> > > > >>>> split this
> > > > >>>> > > discussion into topics/areas outlined by Stephan and create
> > Flips
> > > > >>>> for
> > > > >>>> > that.
> > > > >>>> > >
> > > > >>>> > > I think there is no chance for this to be completed in
> couple
> > of
> > > > >>>> > remaining
> > > > >>>> > > weeks/1 month before 1.8 feature freeze, however it would be
> > good
> > > > >>>> to aim
> > > > >>>> > > with those changes for 1.9.
> > > > >>>> > >
> > > > >>>> > > Piotrek
> > > > >>>> > >
> > > > >>>> > > > On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com>
> > wrote:
> > > > >>>> > > >
> > > > >>>> > > > Hi community,
> > > > >>>> > > > The summary of Stephan makes a lot sense to me. It is much
> > > > clearer
> > > > >>>> > indeed
> > > > >>>> > > > after splitting the complex topic into small ones.
> > > > >>>> > > > I was wondering is there any detail plan for next step? If
> > not,
> > > > I
> > > > >>>> would
> > > > >>>> > > > like to push this thing forward by creating some JIRA
> > issues.
> > > > >>>> > > > Another question is that should version 1.8 include these
> > > > >>>> features?
> > > > >>>> > > >
> > > > >>>> > > > Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
> > > > >>>> > > >
> > > > >>>> > > >> Thanks everyone for the lively discussion. Let me try to
> > > > >>>> summarize
> > > > >>>> > > where I
> > > > >>>> > > >> see convergence in the discussion and open issues.
> > > > >>>> > > >> I'll try to group this by design aspect of the source.
> > Please
> > > > >>>> let me
> > > > >>>> > > know
> > > > >>>> > > >> if I got things wrong or missed something crucial here.
> > > > >>>> > > >>
> > > > >>>> > > >> For issues 1-3, if the below reflects the state of the
> > > > >>>> discussion, I
> > > > >>>> > > would
> > > > >>>> > > >> try and update the FLIP in the next days.
> > > > >>>> > > >> For the remaining ones we need more discussion.
> > > > >>>> > > >>
> > > > >>>> > > >> I would suggest to fork each of these aspects into a
> > separate
> > > > >>>> mail
> > > > >>>> > > thread,
> > > > >>>> > > >> or will loose sight of the individual aspects.
> > > > >>>> > > >>
> > > > >>>> > > >> *(1) Separation of Split Enumerator and Split Reader*
> > > > >>>> > > >>
> > > > >>>> > > >>  - All seem to agree this is a good thing
> > > > >>>> > > >>  - Split Enumerator could in the end live on JobManager
> > (and
> > > > >>>> assign
> > > > >>>> > > splits
> > > > >>>> > > >> via RPC) or in a task (and assign splits via data
> streams)
> > > > >>>> > > >>  - this discussion is orthogonal and should come later,
> > when
> > > > the
> > > > >>>> > > interface
> > > > >>>> > > >> is agreed upon.
> > > > >>>> > > >>
> > > > >>>> > > >> *(2) Split Readers for one or more splits*
> > > > >>>> > > >>
> > > > >>>> > > >>  - Discussion seems to agree that we need to support one
> > reader
> > > > >>>> that
> > > > >>>> > > >> possibly handles multiple splits concurrently.
> > > > >>>> > > >>  - The requirement comes from sources where one
> > poll()-style
> > > > call
> > > > >>>> > > fetches
> > > > >>>> > > >> data from different splits / partitions
> > > > >>>> > > >>    --> example sources that require that would be for
> > example
> > > > >>>> Kafka,
> > > > >>>> > > >> Pravega, Pulsar
> > > > >>>> > > >>
> > > > >>>> > > >>  - Could have one split reader per source, or multiple
> > split
> > > > >>>> readers
> > > > >>>> > > that
> > > > >>>> > > >> share the "poll()" function
> > > > >>>> > > >>  - To not make it too complicated, we can start with
> > thinking
> > > > >>>> about
> > > > >>>> > one
> > > > >>>> > > >> split reader for all splits initially and see if that
> > covers
> > > > all
> > > > >>>> > > >> requirements
> > > > >>>> > > >>
> > > > >>>> > > >> *(3) Threading model of the Split Reader*
> > > > >>>> > > >>
> > > > >>>> > > >>  - Most active part of the discussion ;-)
> > > > >>>> > > >>
> > > > >>>> > > >>  - A non-blocking way for Flink's task code to interact
> > with
> > > > the
> > > > >>>> > source
> > > > >>>> > > is
> > > > >>>> > > >> needed in order to a task runtime code based on a
> > > > >>>> > > >> single-threaded/actor-style task design
> > > > >>>> > > >>    --> I personally am a big proponent of that, it will
> > help
> > > > with
> > > > >>>> > > >> well-behaved checkpoints, efficiency, and simpler yet
> more
> > > > robust
> > > > >>>> > > runtime
> > > > >>>> > > >> code
> > > > >>>> > > >>
> > > > >>>> > > >>  - Users care about simple abstraction, so as a subclass
> of
> > > > >>>> > SplitReader
> > > > >>>> > > >> (non-blocking / async) we need to have a
> > BlockingSplitReader
> > > > >>>> which
> > > > >>>> > will
> > > > >>>> > > >> form the basis of most source implementations.
> > > > >>>> BlockingSplitReader
> > > > >>>> > lets
> > > > >>>> > > >> users do blocking simple poll() calls.
> > > > >>>> > > >>  - The BlockingSplitReader would spawn a thread (or more)
> > and
> > > > the
> > > > >>>> > > >> thread(s) can make blocking calls and hand over data
> > buffers
> > > > via
> > > > >>>> a
> > > > >>>> > > blocking
> > > > >>>> > > >> queue
> > > > >>>> > > >>  - This should allow us to cover both, a fully async
> > runtime,
> > > > >>>> and a
> > > > >>>> > > simple
> > > > >>>> > > >> blocking interface for users.
> > > > >>>> > > >>  - This is actually very similar to how the Kafka
> > connectors
> > > > >>>> work.
> > > > >>>> > Kafka
> > > > >>>> > > >> 9+ with one thread, Kafka 8 with multiple threads
> > > > >>>> > > >>
> > > > >>>> > > >>  - On the base SplitReader (the async one), the
> > non-blocking
> > > > >>>> method
> > > > >>>> > that
> > > > >>>> > > >> gets the next chunk of data would signal data
> availability
> > via
> > > > a
> > > > >>>> > > >> CompletableFuture, because that gives the best
> flexibility
> > (can
> > > > >>>> await
> > > > >>>> > > >> completion or register notification handlers).
> > > > >>>> > > >>  - The source task would register a "thenHandle()" (or
> > similar)
> > > > >>>> on the
> > > > >>>> > > >> future to put a "take next data" task into the
> actor-style
> > > > >>>> mailbox
> > > > >>>> > > >>
> > > > >>>> > > >> *(4) Split Enumeration and Assignment*
> > > > >>>> > > >>
> > > > >>>> > > >>  - Splits may be generated lazily, both in cases where
> > there
> > > > is a
> > > > >>>> > > limited
> > > > >>>> > > >> number of splits (but very many), or splits are
> discovered
> > over
> > > > >>>> time
> > > > >>>> > > >>  - Assignment should also be lazy, to get better load
> > balancing
> > > > >>>> > > >>  - Assignment needs support locality preferences
> > > > >>>> > > >>
> > > > >>>> > > >>  - Possible design based on discussion so far:
> > > > >>>> > > >>
> > > > >>>> > > >>    --> SplitReader has a method "addSplits(SplitT...)" to
> > add
> > > > >>>> one or
> > > > >>>> > > more
> > > > >>>> > > >> splits. Some split readers might assume they have only
> one
> > > > split
> > > > >>>> ever,
> > > > >>>> > > >> concurrently, others assume multiple splits. (Note: idea
> > behind
> > > > >>>> being
> > > > >>>> > > able
> > > > >>>> > > >> to add multiple splits at the same time is to ease
> startup
> > > > where
> > > > >>>> > > multiple
> > > > >>>> > > >> splits may be assigned instantly.)
> > > > >>>> > > >>    --> SplitReader has a context object on which it can
> > call
> > > > >>>> indicate
> > > > >>>> > > when
> > > > >>>> > > >> splits are completed. The enumerator gets that
> > notification and
> > > > >>>> can
> > > > >>>> > use
> > > > >>>> > > to
> > > > >>>> > > >> decide when to assign new splits. This should help both
> in
> > > > cases
> > > > >>>> of
> > > > >>>> > > sources
> > > > >>>> > > >> that take splits lazily (file readers) and in case the
> > source
> > > > >>>> needs to
> > > > >>>> > > >> preserve a partial order between splits (Kinesis,
> Pravega,
> > > > >>>> Pulsar may
> > > > >>>> > > need
> > > > >>>> > > >> that).
> > > > >>>> > > >>    --> SplitEnumerator gets notification when
> SplitReaders
> > > > start
> > > > >>>> and
> > > > >>>> > > when
> > > > >>>> > > >> they finish splits. They can decide at that moment to
> push
> > more
> > > > >>>> splits
> > > > >>>> > > to
> > > > >>>> > > >> that reader
> > > > >>>> > > >>    --> The SplitEnumerator should probably be aware of
> the
> > > > source
> > > > >>>> > > >> parallelism, to build its initial distribution.
> > > > >>>> > > >>
> > > > >>>> > > >>  - Open question: Should the source expose something like
> > "host
> > > > >>>> > > >> preferences", so that yarn/mesos/k8s can take this into
> > account
> > > > >>>> when
> > > > >>>> > > >> selecting a node to start a TM on?
> > > > >>>> > > >>
> > > > >>>> > > >> *(5) Watermarks and event time alignment*
> > > > >>>> > > >>
> > > > >>>> > > >>  - Watermark generation, as well as idleness, needs to be
> > per
> > > > >>>> split
> > > > >>>> > > (like
> > > > >>>> > > >> currently in the Kafka Source, per partition)
> > > > >>>> > > >>  - It is desirable to support optional
> > event-time-alignment,
> > > > >>>> meaning
> > > > >>>> > > that
> > > > >>>> > > >> splits that are ahead are back-pressured or temporarily
> > > > >>>> unsubscribed
> > > > >>>> > > >>
> > > > >>>> > > >>  - I think i would be desirable to encapsulate watermark
> > > > >>>> generation
> > > > >>>> > > logic
> > > > >>>> > > >> in watermark generators, for a separation of concerns.
> The
> > > > >>>> watermark
> > > > >>>> > > >> generators should run per split.
> > > > >>>> > > >>  - Using watermark generators would also help with
> another
> > > > >>>> problem of
> > > > >>>> > > the
> > > > >>>> > > >> suggested interface, namely supporting non-periodic
> > watermarks
> > > > >>>> > > efficiently.
> > > > >>>> > > >>
> > > > >>>> > > >>  - Need a way to "dispatch" next record to different
> > watermark
> > > > >>>> > > generators
> > > > >>>> > > >>  - Need a way to tell SplitReader to "suspend" a split
> > until a
> > > > >>>> certain
> > > > >>>> > > >> watermark is reached (event time backpressure)
> > > > >>>> > > >>  - This would in fact be not needed (and thus simpler) if
> > we
> > > > had
> > > > >>>> a
> > > > >>>> > > >> SplitReader per split and may be a reason to re-open that
> > > > >>>> discussion
> > > > >>>> > > >>
> > > > >>>> > > >> *(6) Watermarks across splits and in the Split
> Enumerator*
> > > > >>>> > > >>
> > > > >>>> > > >>  - The split enumerator may need some watermark
> awareness,
> > > > which
> > > > >>>> > should
> > > > >>>> > > be
> > > > >>>> > > >> purely based on split metadata (like create timestamp of
> > file
> > > > >>>> splits)
> > > > >>>> > > >>  - If there are still more splits with overlapping event
> > time
> > > > >>>> range
> > > > >>>> > for
> > > > >>>> > > a
> > > > >>>> > > >> split reader, then that split reader should not advance
> the
> > > > >>>> watermark
> > > > >>>> > > >> within the split beyond the overlap boundary. Otherwise
> > future
> > > > >>>> splits
> > > > >>>> > > will
> > > > >>>> > > >> produce late data.
> > > > >>>> > > >>
> > > > >>>> > > >>  - One way to approach this could be that the split
> > enumerator
> > > > >>>> may
> > > > >>>> > send
> > > > >>>> > > >> watermarks to the readers, and the readers cannot emit
> > > > watermarks
> > > > >>>> > beyond
> > > > >>>> > > >> that received watermark.
> > > > >>>> > > >>  - Many split enumerators would simply immediately send
> > > > Long.MAX
> > > > >>>> out
> > > > >>>> > and
> > > > >>>> > > >> leave the progress purely to the split readers.
> > > > >>>> > > >>
> > > > >>>> > > >>  - For event-time alignment / split back pressure, this
> > begs
> > > > the
> > > > >>>> > > question
> > > > >>>> > > >> how we can avoid deadlocks that may arise when splits are
> > > > >>>> suspended
> > > > >>>> > for
> > > > >>>> > > >> event time back pressure,
> > > > >>>> > > >>
> > > > >>>> > > >> *(7) Batch and streaming Unification*
> > > > >>>> > > >>
> > > > >>>> > > >>  - Functionality wise, the above design should support
> both
> > > > >>>> > > >>  - Batch often (mostly) does not care about reading "in
> > order"
> > > > >>>> and
> > > > >>>> > > >> generating watermarks
> > > > >>>> > > >>    --> Might use different enumerator logic that is more
> > > > locality
> > > > >>>> > aware
> > > > >>>> > > >> and ignores event time order
> > > > >>>> > > >>    --> Does not generate watermarks
> > > > >>>> > > >>  - Would be great if bounded sources could be identified
> at
> > > > >>>> compile
> > > > >>>> > > time,
> > > > >>>> > > >> so that "env.addBoundedSource(...)" is type safe and can
> > > > return a
> > > > >>>> > > >> "BoundedDataStream".
> > > > >>>> > > >>  - Possible to defer this discussion until later
> > > > >>>> > > >>
> > > > >>>> > > >> *Miscellaneous Comments*
> > > > >>>> > > >>
> > > > >>>> > > >>  - Should the source have a TypeInformation for the
> > produced
> > > > >>>> type,
> > > > >>>> > > instead
> > > > >>>> > > >> of a serializer? We need a type information in the stream
> > > > >>>> anyways, and
> > > > >>>> > > can
> > > > >>>> > > >> derive the serializer from that. Plus, creating the
> > serializer
> > > > >>>> should
> > > > >>>> > > >> respect the ExecutionConfig.
> > > > >>>> > > >>
> > > > >>>> > > >>  - The TypeSerializer interface is very powerful but also
> > not
> > > > >>>> easy to
> > > > >>>> > > >> implement. Its purpose is to handle data super
> efficiently,
> > > > >>>> support
> > > > >>>> > > >> flexible ways of evolution, etc.
> > > > >>>> > > >>  For metadata I would suggest to look at the
> > > > >>>> SimpleVersionedSerializer
> > > > >>>> > > >> instead, which is used for example for checkpoint master
> > hooks,
> > > > >>>> or for
> > > > >>>> > > the
> > > > >>>> > > >> streaming file sink. I think that is is a good match for
> > cases
> > > > >>>> where
> > > > >>>> > we
> > > > >>>> > > do
> > > > >>>> > > >> not need more than ser/deser (no copy, etc.) and don't
> > need to
> > > > >>>> push
> > > > >>>> > > >> versioning out of the serialization paths for best
> > performance
> > > > >>>> (as in
> > > > >>>> > > the
> > > > >>>> > > >> TypeSerializer)
> > > > >>>> > > >>
> > > > >>>> > > >>
> > > > >>>> > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > > > >>>> > > >> k.kloudas@data-artisans.com>
> > > > >>>> > > >> wrote:
> > > > >>>> > > >>
> > > > >>>> > > >>> Hi Biao,
> > > > >>>> > > >>>
> > > > >>>> > > >>> Thanks for the answer!
> > > > >>>> > > >>>
> > > > >>>> > > >>> So given the multi-threaded readers, now we have as open
> > > > >>>> questions:
> > > > >>>> > > >>>
> > > > >>>> > > >>> 1) How do we let the checkpoints pass through our
> > > > multi-threaded
> > > > >>>> > reader
> > > > >>>> > > >>> operator?
> > > > >>>> > > >>>
> > > > >>>> > > >>> 2) Do we have separate reader and source operators or
> > not? In
> > > > >>>> the
> > > > >>>> > > >> strategy
> > > > >>>> > > >>> that has a separate source, the source operator has a
> > > > >>>> parallelism of
> > > > >>>> > 1
> > > > >>>> > > >> and
> > > > >>>> > > >>> is responsible for split recovery only.
> > > > >>>> > > >>>
> > > > >>>> > > >>> For the first one, given also the constraints (blocking,
> > > > finite
> > > > >>>> > queues,
> > > > >>>> > > >>> etc), I do not have an answer yet.
> > > > >>>> > > >>>
> > > > >>>> > > >>> For the 2nd, I think that we should go with separate
> > operators
> > > > >>>> for
> > > > >>>> > the
> > > > >>>> > > >>> source and the readers, for the following reasons:
> > > > >>>> > > >>>
> > > > >>>> > > >>> 1) This is more aligned with a potential future
> > improvement
> > > > >>>> where the
> > > > >>>> > > >> split
> > > > >>>> > > >>> discovery becomes a responsibility of the JobManager and
> > > > >>>> readers are
> > > > >>>> > > >>> pooling more work from the JM.
> > > > >>>> > > >>>
> > > > >>>> > > >>> 2) The source is going to be the "single point of
> truth".
> > It
> > > > >>>> will
> > > > >>>> > know
> > > > >>>> > > >> what
> > > > >>>> > > >>> has been processed and what not. If the source and the
> > readers
> > > > >>>> are a
> > > > >>>> > > >> single
> > > > >>>> > > >>> operator with parallelism > 1, or in general, if the
> split
> > > > >>>> discovery
> > > > >>>> > is
> > > > >>>> > > >>> done by each task individually, then:
> > > > >>>> > > >>>   i) we have to have a deterministic scheme for each
> > reader to
> > > > >>>> assign
> > > > >>>> > > >>> splits to itself (e.g. mod subtaskId). This is not
> > necessarily
> > > > >>>> > trivial
> > > > >>>> > > >> for
> > > > >>>> > > >>> all sources.
> > > > >>>> > > >>>   ii) each reader would have to keep a copy of all its
> > > > processed
> > > > >>>> > slpits
> > > > >>>> > > >>>   iii) the state has to be a union state with a
> > non-trivial
> > > > >>>> merging
> > > > >>>> > > >> logic
> > > > >>>> > > >>> in order to support rescaling.
> > > > >>>> > > >>>
> > > > >>>> > > >>> Two additional points that you raised above:
> > > > >>>> > > >>>
> > > > >>>> > > >>> i) The point that you raised that we need to keep all
> > splits
> > > > >>>> > (processed
> > > > >>>> > > >> and
> > > > >>>> > > >>> not-processed) I think is a bit of a strong requirement.
> > This
> > > > >>>> would
> > > > >>>> > > imply
> > > > >>>> > > >>> that for infinite sources the state will grow
> > indefinitely.
> > > > >>>> This is
> > > > >>>> > > >> problem
> > > > >>>> > > >>> is even more pronounced if we do not have a single
> source
> > that
> > > > >>>> > assigns
> > > > >>>> > > >>> splits to readers, as each reader will have its own copy
> > of
> > > > the
> > > > >>>> > state.
> > > > >>>> > > >>>
> > > > >>>> > > >>> ii) it is true that for finite sources we need to
> somehow
> > not
> > > > >>>> close
> > > > >>>> > the
> > > > >>>> > > >>> readers when the source/split discoverer finishes. The
> > > > >>>> > > >>> ContinuousFileReaderOperator has a work-around for that.
> > It is
> > > > >>>> not
> > > > >>>> > > >> elegant,
> > > > >>>> > > >>> and checkpoints are not emitted after closing the
> source,
> > but
> > > > >>>> this, I
> > > > >>>> > > >>> believe, is a bigger problem which requires more changes
> > than
> > > > >>>> just
> > > > >>>> > > >>> refactoring the source interface.
> > > > >>>> > > >>>
> > > > >>>> > > >>> Cheers,
> > > > >>>> > > >>> Kostas
> > > > >>>> > > >>>
> > > > >>>> > > >>
> > > > >>>> > >
> > > > >>>> > >
> > > > >>>> >
> > > > >>>>
> > > > >>>
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Stephan Ewen <se...@apache.org>.
Hi  Łukasz!

Becket and me are working hard on figuring out the last details and
implementing the first PoC. We would update the FLIP hopefully next week.

There is a fair chance that a first version of this will be in 1.10, but I
think it will take another release to battle test it and migrate the
connectors.

Best,
Stephan




On Fri, Nov 15, 2019 at 11:14 AM Łukasz Jędrzejewski <lj...@touk.pl> wrote:

> Hi,
>
> This proposal looks very promising for us. Do you have any plans in which
> Flink release it is going to be released? We are thinking on using a Data
> Set API for our future use cases but on the other hand Data Set API is
> going to be deprecated so using proposed bounded data streams solution
> could be more viable in the long term.
>
> Thanks,
> Łukasz
>
> On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> wrote:
> > Thanks for putting together this proposal!
> >
> > I see that the "Per Split Event Time" and "Event Time Alignment" sections
> > are still TBD.
> >
> > It would probably be good to flesh those out a bit before proceeding too
> far
> > as the event time alignment will probably influence the interaction with
> > the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
> > output).
> >
> > We currently have only one implementation for event time alignment in the
> > Kinesis consumer. The synchronization in that case takes place as the
> last
> > step before records are emitted downstream (RecordEmitter). With the
> > currently proposed interfaces, the equivalent can be implemented in the
> > reader loop, although note that in the Kinesis consumer the per shard
> > threads push records.
> >
> > Synchronization has not been implemented for the Kafka consumer yet.
> >
> > https://issues.apache.org/jira/browse/FLINK-12675
> >
> > When I looked at it, I realized that the implementation will look quite
> > different
> > from Kinesis because it needs to take place in the pull part, where
> records
> > are taken from the Kafka client. Due to the multiplexing it cannot be
> done
> > by blocking the split thread like it currently works for Kinesis. Reading
> > from individual Kafka partitions needs to be controlled via pause/resume
> > on the Kafka client.
> >
> > To take on that responsibility the split thread would need to be aware of
> > the
> > watermarks or at least whether it should or should not continue to
> consume
> > a given split and this may require a different SourceReader or
> SourceOutput
> > interface.
> >
> > Thanks,
> > Thomas
> >
> >
> > On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> wrote:
> >
> > > Hi Stephan,
> > >
> > > Thank you for feedback!
> > > Will take a look at your branch before public discussing.
> > >
> > >
> > > On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org>
> wrote:
> > >
> > > > Hi Biao!
> > > >
> > > > Thanks for reviving this. I would like to join this discussion, but
> am
> > > > quite occupied with the 1.9 release, so can we maybe pause this
> > > discussion
> > > > for a week or so?
> > > >
> > > > In the meantime I can share some suggestion based on prior
> experiments:
> > > >
> > > > How to do watermarks / timestamp extractors in a simpler and more
> > > flexible
> > > > way. I think that part is quite promising should be part of the new
> > > source
> > > > interface.
> > > >
> > > >
> > >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > > >
> > > >
> > > >
> > >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > > >
> > > >
> > > >
> > > > Some experiments on how to build the source reader and its library
> for
> > > > common threading/split patterns:
> > > >
> > > >
> > >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > > >
> > > >
> > > > Best,
> > > > Stephan
> > > >
> > > >
> > > > On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com>
> wrote:
> > > >
> > > >> Hi devs,
> > > >>
> > > >> Since 1.9 is nearly released, I think we could get back to FLIP-27.
> I
> > > >> believe it should be included in 1.10.
> > > >>
> > > >> There are so many things mentioned in document of FLIP-27. [1] I
> think
> > > >> we'd better discuss them separately. However the wiki is not a good
> > > place
> > > >> to discuss. I wrote google doc about SplitReader API which misses
> some
> > > >> details in the document. [2]
> > > >>
> > > >> 1.
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > >> 2.
> > > >>
> > >
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > > >>
> > > >> CC Stephan, Aljoscha, Piotrek, Becket
> > > >>
> > > >>
> > > >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com>
> wrote:
> > > >>
> > > >>> Hi Steven,
> > > >>> Thank you for the feedback. Please take a look at the document
> FLIP-27
> > > >>> <
> > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >
> > > which
> > > >>> is updated recently. A lot of details of enumerator were added in
> this
> > > >>> document. I think it would help.
> > > >>>
> > > >>> Steven Wu <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
> > > >>>
> > > >>>> This proposal mentioned that SplitEnumerator might run on the
> > > >>>> JobManager or
> > > >>>> in a single task on a TaskManager.
> > > >>>>
> > > >>>> if enumerator is a single task on a taskmanager, then the job DAG
> can
> > > >>>> never
> > > >>>> been embarrassingly parallel anymore. That will nullify the
> leverage
> > > of
> > > >>>> fine-grained recovery for embarrassingly parallel jobs.
> > > >>>>
> > > >>>> It's not clear to me what's the implication of running enumerator
> on
> > > the
> > > >>>> jobmanager. So I will leave that out for now.
> > > >>>>
> > > >>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com>
> wrote:
> > > >>>>
> > > >>>> > Hi Stephan & Piotrek,
> > > >>>> >
> > > >>>> > Thank you for feedback.
> > > >>>> >
> > > >>>> > It seems that there are a lot of things to do in community. I am
> > > just
> > > >>>> > afraid that this discussion may be forgotten since there so many
> > > >>>> proposals
> > > >>>> > recently.
> > > >>>> > Anyway, wish to see the split topics soon :)
> > > >>>> >
> > > >>>> > Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四 下午8:21写道:
> > > >>>> >
> > > >>>> > > Hi Biao!
> > > >>>> > >
> > > >>>> > > This discussion was stalled because of preparations for the
> open
> > > >>>> sourcing
> > > >>>> > > & merging Blink. I think before creating the tickets we should
> > > >>>> split this
> > > >>>> > > discussion into topics/areas outlined by Stephan and create
> Flips
> > > >>>> for
> > > >>>> > that.
> > > >>>> > >
> > > >>>> > > I think there is no chance for this to be completed in couple
> of
> > > >>>> > remaining
> > > >>>> > > weeks/1 month before 1.8 feature freeze, however it would be
> good
> > > >>>> to aim
> > > >>>> > > with those changes for 1.9.
> > > >>>> > >
> > > >>>> > > Piotrek
> > > >>>> > >
> > > >>>> > > > On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com>
> wrote:
> > > >>>> > > >
> > > >>>> > > > Hi community,
> > > >>>> > > > The summary of Stephan makes a lot sense to me. It is much
> > > clearer
> > > >>>> > indeed
> > > >>>> > > > after splitting the complex topic into small ones.
> > > >>>> > > > I was wondering is there any detail plan for next step? If
> not,
> > > I
> > > >>>> would
> > > >>>> > > > like to push this thing forward by creating some JIRA
> issues.
> > > >>>> > > > Another question is that should version 1.8 include these
> > > >>>> features?
> > > >>>> > > >
> > > >>>> > > > Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
> > > >>>> > > >
> > > >>>> > > >> Thanks everyone for the lively discussion. Let me try to
> > > >>>> summarize
> > > >>>> > > where I
> > > >>>> > > >> see convergence in the discussion and open issues.
> > > >>>> > > >> I'll try to group this by design aspect of the source.
> Please
> > > >>>> let me
> > > >>>> > > know
> > > >>>> > > >> if I got things wrong or missed something crucial here.
> > > >>>> > > >>
> > > >>>> > > >> For issues 1-3, if the below reflects the state of the
> > > >>>> discussion, I
> > > >>>> > > would
> > > >>>> > > >> try and update the FLIP in the next days.
> > > >>>> > > >> For the remaining ones we need more discussion.
> > > >>>> > > >>
> > > >>>> > > >> I would suggest to fork each of these aspects into a
> separate
> > > >>>> mail
> > > >>>> > > thread,
> > > >>>> > > >> or will loose sight of the individual aspects.
> > > >>>> > > >>
> > > >>>> > > >> *(1) Separation of Split Enumerator and Split Reader*
> > > >>>> > > >>
> > > >>>> > > >>  - All seem to agree this is a good thing
> > > >>>> > > >>  - Split Enumerator could in the end live on JobManager
> (and
> > > >>>> assign
> > > >>>> > > splits
> > > >>>> > > >> via RPC) or in a task (and assign splits via data streams)
> > > >>>> > > >>  - this discussion is orthogonal and should come later,
> when
> > > the
> > > >>>> > > interface
> > > >>>> > > >> is agreed upon.
> > > >>>> > > >>
> > > >>>> > > >> *(2) Split Readers for one or more splits*
> > > >>>> > > >>
> > > >>>> > > >>  - Discussion seems to agree that we need to support one
> reader
> > > >>>> that
> > > >>>> > > >> possibly handles multiple splits concurrently.
> > > >>>> > > >>  - The requirement comes from sources where one
> poll()-style
> > > call
> > > >>>> > > fetches
> > > >>>> > > >> data from different splits / partitions
> > > >>>> > > >>    --> example sources that require that would be for
> example
> > > >>>> Kafka,
> > > >>>> > > >> Pravega, Pulsar
> > > >>>> > > >>
> > > >>>> > > >>  - Could have one split reader per source, or multiple
> split
> > > >>>> readers
> > > >>>> > > that
> > > >>>> > > >> share the "poll()" function
> > > >>>> > > >>  - To not make it too complicated, we can start with
> thinking
> > > >>>> about
> > > >>>> > one
> > > >>>> > > >> split reader for all splits initially and see if that
> covers
> > > all
> > > >>>> > > >> requirements
> > > >>>> > > >>
> > > >>>> > > >> *(3) Threading model of the Split Reader*
> > > >>>> > > >>
> > > >>>> > > >>  - Most active part of the discussion ;-)
> > > >>>> > > >>
> > > >>>> > > >>  - A non-blocking way for Flink's task code to interact
> with
> > > the
> > > >>>> > source
> > > >>>> > > is
> > > >>>> > > >> needed in order to a task runtime code based on a
> > > >>>> > > >> single-threaded/actor-style task design
> > > >>>> > > >>    --> I personally am a big proponent of that, it will
> help
> > > with
> > > >>>> > > >> well-behaved checkpoints, efficiency, and simpler yet more
> > > robust
> > > >>>> > > runtime
> > > >>>> > > >> code
> > > >>>> > > >>
> > > >>>> > > >>  - Users care about simple abstraction, so as a subclass of
> > > >>>> > SplitReader
> > > >>>> > > >> (non-blocking / async) we need to have a
> BlockingSplitReader
> > > >>>> which
> > > >>>> > will
> > > >>>> > > >> form the basis of most source implementations.
> > > >>>> BlockingSplitReader
> > > >>>> > lets
> > > >>>> > > >> users do blocking simple poll() calls.
> > > >>>> > > >>  - The BlockingSplitReader would spawn a thread (or more)
> and
> > > the
> > > >>>> > > >> thread(s) can make blocking calls and hand over data
> buffers
> > > via
> > > >>>> a
> > > >>>> > > blocking
> > > >>>> > > >> queue
> > > >>>> > > >>  - This should allow us to cover both, a fully async
> runtime,
> > > >>>> and a
> > > >>>> > > simple
> > > >>>> > > >> blocking interface for users.
> > > >>>> > > >>  - This is actually very similar to how the Kafka
> connectors
> > > >>>> work.
> > > >>>> > Kafka
> > > >>>> > > >> 9+ with one thread, Kafka 8 with multiple threads
> > > >>>> > > >>
> > > >>>> > > >>  - On the base SplitReader (the async one), the
> non-blocking
> > > >>>> method
> > > >>>> > that
> > > >>>> > > >> gets the next chunk of data would signal data availability
> via
> > > a
> > > >>>> > > >> CompletableFuture, because that gives the best flexibility
> (can
> > > >>>> await
> > > >>>> > > >> completion or register notification handlers).
> > > >>>> > > >>  - The source task would register a "thenHandle()" (or
> similar)
> > > >>>> on the
> > > >>>> > > >> future to put a "take next data" task into the actor-style
> > > >>>> mailbox
> > > >>>> > > >>
> > > >>>> > > >> *(4) Split Enumeration and Assignment*
> > > >>>> > > >>
> > > >>>> > > >>  - Splits may be generated lazily, both in cases where
> there
> > > is a
> > > >>>> > > limited
> > > >>>> > > >> number of splits (but very many), or splits are discovered
> over
> > > >>>> time
> > > >>>> > > >>  - Assignment should also be lazy, to get better load
> balancing
> > > >>>> > > >>  - Assignment needs support locality preferences
> > > >>>> > > >>
> > > >>>> > > >>  - Possible design based on discussion so far:
> > > >>>> > > >>
> > > >>>> > > >>    --> SplitReader has a method "addSplits(SplitT...)" to
> add
> > > >>>> one or
> > > >>>> > > more
> > > >>>> > > >> splits. Some split readers might assume they have only one
> > > split
> > > >>>> ever,
> > > >>>> > > >> concurrently, others assume multiple splits. (Note: idea
> behind
> > > >>>> being
> > > >>>> > > able
> > > >>>> > > >> to add multiple splits at the same time is to ease startup
> > > where
> > > >>>> > > multiple
> > > >>>> > > >> splits may be assigned instantly.)
> > > >>>> > > >>    --> SplitReader has a context object on which it can
> call
> > > >>>> indicate
> > > >>>> > > when
> > > >>>> > > >> splits are completed. The enumerator gets that
> notification and
> > > >>>> can
> > > >>>> > use
> > > >>>> > > to
> > > >>>> > > >> decide when to assign new splits. This should help both in
> > > cases
> > > >>>> of
> > > >>>> > > sources
> > > >>>> > > >> that take splits lazily (file readers) and in case the
> source
> > > >>>> needs to
> > > >>>> > > >> preserve a partial order between splits (Kinesis, Pravega,
> > > >>>> Pulsar may
> > > >>>> > > need
> > > >>>> > > >> that).
> > > >>>> > > >>    --> SplitEnumerator gets notification when SplitReaders
> > > start
> > > >>>> and
> > > >>>> > > when
> > > >>>> > > >> they finish splits. They can decide at that moment to push
> more
> > > >>>> splits
> > > >>>> > > to
> > > >>>> > > >> that reader
> > > >>>> > > >>    --> The SplitEnumerator should probably be aware of the
> > > source
> > > >>>> > > >> parallelism, to build its initial distribution.
> > > >>>> > > >>
> > > >>>> > > >>  - Open question: Should the source expose something like
> "host
> > > >>>> > > >> preferences", so that yarn/mesos/k8s can take this into
> account
> > > >>>> when
> > > >>>> > > >> selecting a node to start a TM on?
> > > >>>> > > >>
> > > >>>> > > >> *(5) Watermarks and event time alignment*
> > > >>>> > > >>
> > > >>>> > > >>  - Watermark generation, as well as idleness, needs to be
> per
> > > >>>> split
> > > >>>> > > (like
> > > >>>> > > >> currently in the Kafka Source, per partition)
> > > >>>> > > >>  - It is desirable to support optional
> event-time-alignment,
> > > >>>> meaning
> > > >>>> > > that
> > > >>>> > > >> splits that are ahead are back-pressured or temporarily
> > > >>>> unsubscribed
> > > >>>> > > >>
> > > >>>> > > >>  - I think i would be desirable to encapsulate watermark
> > > >>>> generation
> > > >>>> > > logic
> > > >>>> > > >> in watermark generators, for a separation of concerns. The
> > > >>>> watermark
> > > >>>> > > >> generators should run per split.
> > > >>>> > > >>  - Using watermark generators would also help with another
> > > >>>> problem of
> > > >>>> > > the
> > > >>>> > > >> suggested interface, namely supporting non-periodic
> watermarks
> > > >>>> > > efficiently.
> > > >>>> > > >>
> > > >>>> > > >>  - Need a way to "dispatch" next record to different
> watermark
> > > >>>> > > generators
> > > >>>> > > >>  - Need a way to tell SplitReader to "suspend" a split
> until a
> > > >>>> certain
> > > >>>> > > >> watermark is reached (event time backpressure)
> > > >>>> > > >>  - This would in fact be not needed (and thus simpler) if
> we
> > > had
> > > >>>> a
> > > >>>> > > >> SplitReader per split and may be a reason to re-open that
> > > >>>> discussion
> > > >>>> > > >>
> > > >>>> > > >> *(6) Watermarks across splits and in the Split Enumerator*
> > > >>>> > > >>
> > > >>>> > > >>  - The split enumerator may need some watermark awareness,
> > > which
> > > >>>> > should
> > > >>>> > > be
> > > >>>> > > >> purely based on split metadata (like create timestamp of
> file
> > > >>>> splits)
> > > >>>> > > >>  - If there are still more splits with overlapping event
> time
> > > >>>> range
> > > >>>> > for
> > > >>>> > > a
> > > >>>> > > >> split reader, then that split reader should not advance the
> > > >>>> watermark
> > > >>>> > > >> within the split beyond the overlap boundary. Otherwise
> future
> > > >>>> splits
> > > >>>> > > will
> > > >>>> > > >> produce late data.
> > > >>>> > > >>
> > > >>>> > > >>  - One way to approach this could be that the split
> enumerator
> > > >>>> may
> > > >>>> > send
> > > >>>> > > >> watermarks to the readers, and the readers cannot emit
> > > watermarks
> > > >>>> > beyond
> > > >>>> > > >> that received watermark.
> > > >>>> > > >>  - Many split enumerators would simply immediately send
> > > Long.MAX
> > > >>>> out
> > > >>>> > and
> > > >>>> > > >> leave the progress purely to the split readers.
> > > >>>> > > >>
> > > >>>> > > >>  - For event-time alignment / split back pressure, this
> begs
> > > the
> > > >>>> > > question
> > > >>>> > > >> how we can avoid deadlocks that may arise when splits are
> > > >>>> suspended
> > > >>>> > for
> > > >>>> > > >> event time back pressure,
> > > >>>> > > >>
> > > >>>> > > >> *(7) Batch and streaming Unification*
> > > >>>> > > >>
> > > >>>> > > >>  - Functionality wise, the above design should support both
> > > >>>> > > >>  - Batch often (mostly) does not care about reading "in
> order"
> > > >>>> and
> > > >>>> > > >> generating watermarks
> > > >>>> > > >>    --> Might use different enumerator logic that is more
> > > locality
> > > >>>> > aware
> > > >>>> > > >> and ignores event time order
> > > >>>> > > >>    --> Does not generate watermarks
> > > >>>> > > >>  - Would be great if bounded sources could be identified at
> > > >>>> compile
> > > >>>> > > time,
> > > >>>> > > >> so that "env.addBoundedSource(...)" is type safe and can
> > > return a
> > > >>>> > > >> "BoundedDataStream".
> > > >>>> > > >>  - Possible to defer this discussion until later
> > > >>>> > > >>
> > > >>>> > > >> *Miscellaneous Comments*
> > > >>>> > > >>
> > > >>>> > > >>  - Should the source have a TypeInformation for the
> produced
> > > >>>> type,
> > > >>>> > > instead
> > > >>>> > > >> of a serializer? We need a type information in the stream
> > > >>>> anyways, and
> > > >>>> > > can
> > > >>>> > > >> derive the serializer from that. Plus, creating the
> serializer
> > > >>>> should
> > > >>>> > > >> respect the ExecutionConfig.
> > > >>>> > > >>
> > > >>>> > > >>  - The TypeSerializer interface is very powerful but also
> not
> > > >>>> easy to
> > > >>>> > > >> implement. Its purpose is to handle data super efficiently,
> > > >>>> support
> > > >>>> > > >> flexible ways of evolution, etc.
> > > >>>> > > >>  For metadata I would suggest to look at the
> > > >>>> SimpleVersionedSerializer
> > > >>>> > > >> instead, which is used for example for checkpoint master
> hooks,
> > > >>>> or for
> > > >>>> > > the
> > > >>>> > > >> streaming file sink. I think that is is a good match for
> cases
> > > >>>> where
> > > >>>> > we
> > > >>>> > > do
> > > >>>> > > >> not need more than ser/deser (no copy, etc.) and don't
> need to
> > > >>>> push
> > > >>>> > > >> versioning out of the serialization paths for best
> performance
> > > >>>> (as in
> > > >>>> > > the
> > > >>>> > > >> TypeSerializer)
> > > >>>> > > >>
> > > >>>> > > >>
> > > >>>> > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > > >>>> > > >> k.kloudas@data-artisans.com>
> > > >>>> > > >> wrote:
> > > >>>> > > >>
> > > >>>> > > >>> Hi Biao,
> > > >>>> > > >>>
> > > >>>> > > >>> Thanks for the answer!
> > > >>>> > > >>>
> > > >>>> > > >>> So given the multi-threaded readers, now we have as open
> > > >>>> questions:
> > > >>>> > > >>>
> > > >>>> > > >>> 1) How do we let the checkpoints pass through our
> > > multi-threaded
> > > >>>> > reader
> > > >>>> > > >>> operator?
> > > >>>> > > >>>
> > > >>>> > > >>> 2) Do we have separate reader and source operators or
> not? In
> > > >>>> the
> > > >>>> > > >> strategy
> > > >>>> > > >>> that has a separate source, the source operator has a
> > > >>>> parallelism of
> > > >>>> > 1
> > > >>>> > > >> and
> > > >>>> > > >>> is responsible for split recovery only.
> > > >>>> > > >>>
> > > >>>> > > >>> For the first one, given also the constraints (blocking,
> > > finite
> > > >>>> > queues,
> > > >>>> > > >>> etc), I do not have an answer yet.
> > > >>>> > > >>>
> > > >>>> > > >>> For the 2nd, I think that we should go with separate
> operators
> > > >>>> for
> > > >>>> > the
> > > >>>> > > >>> source and the readers, for the following reasons:
> > > >>>> > > >>>
> > > >>>> > > >>> 1) This is more aligned with a potential future
> improvement
> > > >>>> where the
> > > >>>> > > >> split
> > > >>>> > > >>> discovery becomes a responsibility of the JobManager and
> > > >>>> readers are
> > > >>>> > > >>> pooling more work from the JM.
> > > >>>> > > >>>
> > > >>>> > > >>> 2) The source is going to be the "single point of truth".
> It
> > > >>>> will
> > > >>>> > know
> > > >>>> > > >> what
> > > >>>> > > >>> has been processed and what not. If the source and the
> readers
> > > >>>> are a
> > > >>>> > > >> single
> > > >>>> > > >>> operator with parallelism > 1, or in general, if the split
> > > >>>> discovery
> > > >>>> > is
> > > >>>> > > >>> done by each task individually, then:
> > > >>>> > > >>>   i) we have to have a deterministic scheme for each
> reader to
> > > >>>> assign
> > > >>>> > > >>> splits to itself (e.g. mod subtaskId). This is not
> necessarily
> > > >>>> > trivial
> > > >>>> > > >> for
> > > >>>> > > >>> all sources.
> > > >>>> > > >>>   ii) each reader would have to keep a copy of all its
> > > processed
> > > >>>> > slpits
> > > >>>> > > >>>   iii) the state has to be a union state with a
> non-trivial
> > > >>>> merging
> > > >>>> > > >> logic
> > > >>>> > > >>> in order to support rescaling.
> > > >>>> > > >>>
> > > >>>> > > >>> Two additional points that you raised above:
> > > >>>> > > >>>
> > > >>>> > > >>> i) The point that you raised that we need to keep all
> splits
> > > >>>> > (processed
> > > >>>> > > >> and
> > > >>>> > > >>> not-processed) I think is a bit of a strong requirement.
> This
> > > >>>> would
> > > >>>> > > imply
> > > >>>> > > >>> that for infinite sources the state will grow
> indefinitely.
> > > >>>> This is
> > > >>>> > > >> problem
> > > >>>> > > >>> is even more pronounced if we do not have a single source
> that
> > > >>>> > assigns
> > > >>>> > > >>> splits to readers, as each reader will have its own copy
> of
> > > the
> > > >>>> > state.
> > > >>>> > > >>>
> > > >>>> > > >>> ii) it is true that for finite sources we need to somehow
> not
> > > >>>> close
> > > >>>> > the
> > > >>>> > > >>> readers when the source/split discoverer finishes. The
> > > >>>> > > >>> ContinuousFileReaderOperator has a work-around for that.
> It is
> > > >>>> not
> > > >>>> > > >> elegant,
> > > >>>> > > >>> and checkpoints are not emitted after closing the source,
> but
> > > >>>> this, I
> > > >>>> > > >>> believe, is a bigger problem which requires more changes
> than
> > > >>>> just
> > > >>>> > > >>> refactoring the source interface.
> > > >>>> > > >>>
> > > >>>> > > >>> Cheers,
> > > >>>> > > >>> Kostas
> > > >>>> > > >>>
> > > >>>> > > >>
> > > >>>> > >
> > > >>>> > >
> > > >>>> >
> > > >>>>
> > > >>>
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by ��ukasz J��drzejewski <lj...@touk.pl>.
Hi,

This proposal looks very promising for us. Do you have any plans in which Flink release it is going to be released? We are thinking on using a Data Set API for our future use cases but on the other hand Data Set API is going to be deprecated so using proposed bounded data streams solution could be more viable in the long term. 

Thanks,
Łukasz

On 2019/10/01 15:48:03, Thomas Weise <th...@gmail.com> wrote: 
> Thanks for putting together this proposal!
> 
> I see that the "Per Split Event Time" and "Event Time Alignment" sections
> are still TBD.
> 
> It would probably be good to flesh those out a bit before proceeding too far
> as the event time alignment will probably influence the interaction with
> the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
> output).
> 
> We currently have only one implementation for event time alignment in the
> Kinesis consumer. The synchronization in that case takes place as the last
> step before records are emitted downstream (RecordEmitter). With the
> currently proposed interfaces, the equivalent can be implemented in the
> reader loop, although note that in the Kinesis consumer the per shard
> threads push records.
> 
> Synchronization has not been implemented for the Kafka consumer yet.
> 
> https://issues.apache.org/jira/browse/FLINK-12675
> 
> When I looked at it, I realized that the implementation will look quite
> different
> from Kinesis because it needs to take place in the pull part, where records
> are taken from the Kafka client. Due to the multiplexing it cannot be done
> by blocking the split thread like it currently works for Kinesis. Reading
> from individual Kafka partitions needs to be controlled via pause/resume
> on the Kafka client.
> 
> To take on that responsibility the split thread would need to be aware of
> the
> watermarks or at least whether it should or should not continue to consume
> a given split and this may require a different SourceReader or SourceOutput
> interface.
> 
> Thanks,
> Thomas
> 
> 
> On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> wrote:
> 
> > Hi Stephan,
> >
> > Thank you for feedback!
> > Will take a look at your branch before public discussing.
> >
> >
> > On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org> wrote:
> >
> > > Hi Biao!
> > >
> > > Thanks for reviving this. I would like to join this discussion, but am
> > > quite occupied with the 1.9 release, so can we maybe pause this
> > discussion
> > > for a week or so?
> > >
> > > In the meantime I can share some suggestion based on prior experiments:
> > >
> > > How to do watermarks / timestamp extractors in a simpler and more
> > flexible
> > > way. I think that part is quite promising should be part of the new
> > source
> > > interface.
> > >
> > >
> > https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> > >
> > >
> > >
> > https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> > >
> > >
> > >
> > > Some experiments on how to build the source reader and its library for
> > > common threading/split patterns:
> > >
> > >
> > https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> > >
> > >
> > > Best,
> > > Stephan
> > >
> > >
> > > On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> wrote:
> > >
> > >> Hi devs,
> > >>
> > >> Since 1.9 is nearly released, I think we could get back to FLIP-27. I
> > >> believe it should be included in 1.10.
> > >>
> > >> There are so many things mentioned in document of FLIP-27. [1] I think
> > >> we'd better discuss them separately. However the wiki is not a good
> > place
> > >> to discuss. I wrote google doc about SplitReader API which misses some
> > >> details in the document. [2]
> > >>
> > >> 1.
> > >>
> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > >> 2.
> > >>
> > https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> > >>
> > >> CC Stephan, Aljoscha, Piotrek, Becket
> > >>
> > >>
> > >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> wrote:
> > >>
> > >>> Hi Steven,
> > >>> Thank you for the feedback. Please take a look at the document FLIP-27
> > >>> <
> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface>
> > which
> > >>> is updated recently. A lot of details of enumerator were added in this
> > >>> document. I think it would help.
> > >>>
> > >>> Steven Wu <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
> > >>>
> > >>>> This proposal mentioned that SplitEnumerator might run on the
> > >>>> JobManager or
> > >>>> in a single task on a TaskManager.
> > >>>>
> > >>>> if enumerator is a single task on a taskmanager, then the job DAG can
> > >>>> never
> > >>>> been embarrassingly parallel anymore. That will nullify the leverage
> > of
> > >>>> fine-grained recovery for embarrassingly parallel jobs.
> > >>>>
> > >>>> It's not clear to me what's the implication of running enumerator on
> > the
> > >>>> jobmanager. So I will leave that out for now.
> > >>>>
> > >>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> wrote:
> > >>>>
> > >>>> > Hi Stephan & Piotrek,
> > >>>> >
> > >>>> > Thank you for feedback.
> > >>>> >
> > >>>> > It seems that there are a lot of things to do in community. I am
> > just
> > >>>> > afraid that this discussion may be forgotten since there so many
> > >>>> proposals
> > >>>> > recently.
> > >>>> > Anyway, wish to see the split topics soon :)
> > >>>> >
> > >>>> > Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四 下午8:21写道:
> > >>>> >
> > >>>> > > Hi Biao!
> > >>>> > >
> > >>>> > > This discussion was stalled because of preparations for the open
> > >>>> sourcing
> > >>>> > > & merging Blink. I think before creating the tickets we should
> > >>>> split this
> > >>>> > > discussion into topics/areas outlined by Stephan and create Flips
> > >>>> for
> > >>>> > that.
> > >>>> > >
> > >>>> > > I think there is no chance for this to be completed in couple of
> > >>>> > remaining
> > >>>> > > weeks/1 month before 1.8 feature freeze, however it would be good
> > >>>> to aim
> > >>>> > > with those changes for 1.9.
> > >>>> > >
> > >>>> > > Piotrek
> > >>>> > >
> > >>>> > > > On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> wrote:
> > >>>> > > >
> > >>>> > > > Hi community,
> > >>>> > > > The summary of Stephan makes a lot sense to me. It is much
> > clearer
> > >>>> > indeed
> > >>>> > > > after splitting the complex topic into small ones.
> > >>>> > > > I was wondering is there any detail plan for next step? If not,
> > I
> > >>>> would
> > >>>> > > > like to push this thing forward by creating some JIRA issues.
> > >>>> > > > Another question is that should version 1.8 include these
> > >>>> features?
> > >>>> > > >
> > >>>> > > > Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
> > >>>> > > >
> > >>>> > > >> Thanks everyone for the lively discussion. Let me try to
> > >>>> summarize
> > >>>> > > where I
> > >>>> > > >> see convergence in the discussion and open issues.
> > >>>> > > >> I'll try to group this by design aspect of the source. Please
> > >>>> let me
> > >>>> > > know
> > >>>> > > >> if I got things wrong or missed something crucial here.
> > >>>> > > >>
> > >>>> > > >> For issues 1-3, if the below reflects the state of the
> > >>>> discussion, I
> > >>>> > > would
> > >>>> > > >> try and update the FLIP in the next days.
> > >>>> > > >> For the remaining ones we need more discussion.
> > >>>> > > >>
> > >>>> > > >> I would suggest to fork each of these aspects into a separate
> > >>>> mail
> > >>>> > > thread,
> > >>>> > > >> or will loose sight of the individual aspects.
> > >>>> > > >>
> > >>>> > > >> *(1) Separation of Split Enumerator and Split Reader*
> > >>>> > > >>
> > >>>> > > >>  - All seem to agree this is a good thing
> > >>>> > > >>  - Split Enumerator could in the end live on JobManager (and
> > >>>> assign
> > >>>> > > splits
> > >>>> > > >> via RPC) or in a task (and assign splits via data streams)
> > >>>> > > >>  - this discussion is orthogonal and should come later, when
> > the
> > >>>> > > interface
> > >>>> > > >> is agreed upon.
> > >>>> > > >>
> > >>>> > > >> *(2) Split Readers for one or more splits*
> > >>>> > > >>
> > >>>> > > >>  - Discussion seems to agree that we need to support one reader
> > >>>> that
> > >>>> > > >> possibly handles multiple splits concurrently.
> > >>>> > > >>  - The requirement comes from sources where one poll()-style
> > call
> > >>>> > > fetches
> > >>>> > > >> data from different splits / partitions
> > >>>> > > >>    --> example sources that require that would be for example
> > >>>> Kafka,
> > >>>> > > >> Pravega, Pulsar
> > >>>> > > >>
> > >>>> > > >>  - Could have one split reader per source, or multiple split
> > >>>> readers
> > >>>> > > that
> > >>>> > > >> share the "poll()" function
> > >>>> > > >>  - To not make it too complicated, we can start with thinking
> > >>>> about
> > >>>> > one
> > >>>> > > >> split reader for all splits initially and see if that covers
> > all
> > >>>> > > >> requirements
> > >>>> > > >>
> > >>>> > > >> *(3) Threading model of the Split Reader*
> > >>>> > > >>
> > >>>> > > >>  - Most active part of the discussion ;-)
> > >>>> > > >>
> > >>>> > > >>  - A non-blocking way for Flink's task code to interact with
> > the
> > >>>> > source
> > >>>> > > is
> > >>>> > > >> needed in order to a task runtime code based on a
> > >>>> > > >> single-threaded/actor-style task design
> > >>>> > > >>    --> I personally am a big proponent of that, it will help
> > with
> > >>>> > > >> well-behaved checkpoints, efficiency, and simpler yet more
> > robust
> > >>>> > > runtime
> > >>>> > > >> code
> > >>>> > > >>
> > >>>> > > >>  - Users care about simple abstraction, so as a subclass of
> > >>>> > SplitReader
> > >>>> > > >> (non-blocking / async) we need to have a BlockingSplitReader
> > >>>> which
> > >>>> > will
> > >>>> > > >> form the basis of most source implementations.
> > >>>> BlockingSplitReader
> > >>>> > lets
> > >>>> > > >> users do blocking simple poll() calls.
> > >>>> > > >>  - The BlockingSplitReader would spawn a thread (or more) and
> > the
> > >>>> > > >> thread(s) can make blocking calls and hand over data buffers
> > via
> > >>>> a
> > >>>> > > blocking
> > >>>> > > >> queue
> > >>>> > > >>  - This should allow us to cover both, a fully async runtime,
> > >>>> and a
> > >>>> > > simple
> > >>>> > > >> blocking interface for users.
> > >>>> > > >>  - This is actually very similar to how the Kafka connectors
> > >>>> work.
> > >>>> > Kafka
> > >>>> > > >> 9+ with one thread, Kafka 8 with multiple threads
> > >>>> > > >>
> > >>>> > > >>  - On the base SplitReader (the async one), the non-blocking
> > >>>> method
> > >>>> > that
> > >>>> > > >> gets the next chunk of data would signal data availability via
> > a
> > >>>> > > >> CompletableFuture, because that gives the best flexibility (can
> > >>>> await
> > >>>> > > >> completion or register notification handlers).
> > >>>> > > >>  - The source task would register a "thenHandle()" (or similar)
> > >>>> on the
> > >>>> > > >> future to put a "take next data" task into the actor-style
> > >>>> mailbox
> > >>>> > > >>
> > >>>> > > >> *(4) Split Enumeration and Assignment*
> > >>>> > > >>
> > >>>> > > >>  - Splits may be generated lazily, both in cases where there
> > is a
> > >>>> > > limited
> > >>>> > > >> number of splits (but very many), or splits are discovered over
> > >>>> time
> > >>>> > > >>  - Assignment should also be lazy, to get better load balancing
> > >>>> > > >>  - Assignment needs support locality preferences
> > >>>> > > >>
> > >>>> > > >>  - Possible design based on discussion so far:
> > >>>> > > >>
> > >>>> > > >>    --> SplitReader has a method "addSplits(SplitT...)" to add
> > >>>> one or
> > >>>> > > more
> > >>>> > > >> splits. Some split readers might assume they have only one
> > split
> > >>>> ever,
> > >>>> > > >> concurrently, others assume multiple splits. (Note: idea behind
> > >>>> being
> > >>>> > > able
> > >>>> > > >> to add multiple splits at the same time is to ease startup
> > where
> > >>>> > > multiple
> > >>>> > > >> splits may be assigned instantly.)
> > >>>> > > >>    --> SplitReader has a context object on which it can call
> > >>>> indicate
> > >>>> > > when
> > >>>> > > >> splits are completed. The enumerator gets that notification and
> > >>>> can
> > >>>> > use
> > >>>> > > to
> > >>>> > > >> decide when to assign new splits. This should help both in
> > cases
> > >>>> of
> > >>>> > > sources
> > >>>> > > >> that take splits lazily (file readers) and in case the source
> > >>>> needs to
> > >>>> > > >> preserve a partial order between splits (Kinesis, Pravega,
> > >>>> Pulsar may
> > >>>> > > need
> > >>>> > > >> that).
> > >>>> > > >>    --> SplitEnumerator gets notification when SplitReaders
> > start
> > >>>> and
> > >>>> > > when
> > >>>> > > >> they finish splits. They can decide at that moment to push more
> > >>>> splits
> > >>>> > > to
> > >>>> > > >> that reader
> > >>>> > > >>    --> The SplitEnumerator should probably be aware of the
> > source
> > >>>> > > >> parallelism, to build its initial distribution.
> > >>>> > > >>
> > >>>> > > >>  - Open question: Should the source expose something like "host
> > >>>> > > >> preferences", so that yarn/mesos/k8s can take this into account
> > >>>> when
> > >>>> > > >> selecting a node to start a TM on?
> > >>>> > > >>
> > >>>> > > >> *(5) Watermarks and event time alignment*
> > >>>> > > >>
> > >>>> > > >>  - Watermark generation, as well as idleness, needs to be per
> > >>>> split
> > >>>> > > (like
> > >>>> > > >> currently in the Kafka Source, per partition)
> > >>>> > > >>  - It is desirable to support optional event-time-alignment,
> > >>>> meaning
> > >>>> > > that
> > >>>> > > >> splits that are ahead are back-pressured or temporarily
> > >>>> unsubscribed
> > >>>> > > >>
> > >>>> > > >>  - I think i would be desirable to encapsulate watermark
> > >>>> generation
> > >>>> > > logic
> > >>>> > > >> in watermark generators, for a separation of concerns. The
> > >>>> watermark
> > >>>> > > >> generators should run per split.
> > >>>> > > >>  - Using watermark generators would also help with another
> > >>>> problem of
> > >>>> > > the
> > >>>> > > >> suggested interface, namely supporting non-periodic watermarks
> > >>>> > > efficiently.
> > >>>> > > >>
> > >>>> > > >>  - Need a way to "dispatch" next record to different watermark
> > >>>> > > generators
> > >>>> > > >>  - Need a way to tell SplitReader to "suspend" a split until a
> > >>>> certain
> > >>>> > > >> watermark is reached (event time backpressure)
> > >>>> > > >>  - This would in fact be not needed (and thus simpler) if we
> > had
> > >>>> a
> > >>>> > > >> SplitReader per split and may be a reason to re-open that
> > >>>> discussion
> > >>>> > > >>
> > >>>> > > >> *(6) Watermarks across splits and in the Split Enumerator*
> > >>>> > > >>
> > >>>> > > >>  - The split enumerator may need some watermark awareness,
> > which
> > >>>> > should
> > >>>> > > be
> > >>>> > > >> purely based on split metadata (like create timestamp of file
> > >>>> splits)
> > >>>> > > >>  - If there are still more splits with overlapping event time
> > >>>> range
> > >>>> > for
> > >>>> > > a
> > >>>> > > >> split reader, then that split reader should not advance the
> > >>>> watermark
> > >>>> > > >> within the split beyond the overlap boundary. Otherwise future
> > >>>> splits
> > >>>> > > will
> > >>>> > > >> produce late data.
> > >>>> > > >>
> > >>>> > > >>  - One way to approach this could be that the split enumerator
> > >>>> may
> > >>>> > send
> > >>>> > > >> watermarks to the readers, and the readers cannot emit
> > watermarks
> > >>>> > beyond
> > >>>> > > >> that received watermark.
> > >>>> > > >>  - Many split enumerators would simply immediately send
> > Long.MAX
> > >>>> out
> > >>>> > and
> > >>>> > > >> leave the progress purely to the split readers.
> > >>>> > > >>
> > >>>> > > >>  - For event-time alignment / split back pressure, this begs
> > the
> > >>>> > > question
> > >>>> > > >> how we can avoid deadlocks that may arise when splits are
> > >>>> suspended
> > >>>> > for
> > >>>> > > >> event time back pressure,
> > >>>> > > >>
> > >>>> > > >> *(7) Batch and streaming Unification*
> > >>>> > > >>
> > >>>> > > >>  - Functionality wise, the above design should support both
> > >>>> > > >>  - Batch often (mostly) does not care about reading "in order"
> > >>>> and
> > >>>> > > >> generating watermarks
> > >>>> > > >>    --> Might use different enumerator logic that is more
> > locality
> > >>>> > aware
> > >>>> > > >> and ignores event time order
> > >>>> > > >>    --> Does not generate watermarks
> > >>>> > > >>  - Would be great if bounded sources could be identified at
> > >>>> compile
> > >>>> > > time,
> > >>>> > > >> so that "env.addBoundedSource(...)" is type safe and can
> > return a
> > >>>> > > >> "BoundedDataStream".
> > >>>> > > >>  - Possible to defer this discussion until later
> > >>>> > > >>
> > >>>> > > >> *Miscellaneous Comments*
> > >>>> > > >>
> > >>>> > > >>  - Should the source have a TypeInformation for the produced
> > >>>> type,
> > >>>> > > instead
> > >>>> > > >> of a serializer? We need a type information in the stream
> > >>>> anyways, and
> > >>>> > > can
> > >>>> > > >> derive the serializer from that. Plus, creating the serializer
> > >>>> should
> > >>>> > > >> respect the ExecutionConfig.
> > >>>> > > >>
> > >>>> > > >>  - The TypeSerializer interface is very powerful but also not
> > >>>> easy to
> > >>>> > > >> implement. Its purpose is to handle data super efficiently,
> > >>>> support
> > >>>> > > >> flexible ways of evolution, etc.
> > >>>> > > >>  For metadata I would suggest to look at the
> > >>>> SimpleVersionedSerializer
> > >>>> > > >> instead, which is used for example for checkpoint master hooks,
> > >>>> or for
> > >>>> > > the
> > >>>> > > >> streaming file sink. I think that is is a good match for cases
> > >>>> where
> > >>>> > we
> > >>>> > > do
> > >>>> > > >> not need more than ser/deser (no copy, etc.) and don't need to
> > >>>> push
> > >>>> > > >> versioning out of the serialization paths for best performance
> > >>>> (as in
> > >>>> > > the
> > >>>> > > >> TypeSerializer)
> > >>>> > > >>
> > >>>> > > >>
> > >>>> > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > >>>> > > >> k.kloudas@data-artisans.com>
> > >>>> > > >> wrote:
> > >>>> > > >>
> > >>>> > > >>> Hi Biao,
> > >>>> > > >>>
> > >>>> > > >>> Thanks for the answer!
> > >>>> > > >>>
> > >>>> > > >>> So given the multi-threaded readers, now we have as open
> > >>>> questions:
> > >>>> > > >>>
> > >>>> > > >>> 1) How do we let the checkpoints pass through our
> > multi-threaded
> > >>>> > reader
> > >>>> > > >>> operator?
> > >>>> > > >>>
> > >>>> > > >>> 2) Do we have separate reader and source operators or not? In
> > >>>> the
> > >>>> > > >> strategy
> > >>>> > > >>> that has a separate source, the source operator has a
> > >>>> parallelism of
> > >>>> > 1
> > >>>> > > >> and
> > >>>> > > >>> is responsible for split recovery only.
> > >>>> > > >>>
> > >>>> > > >>> For the first one, given also the constraints (blocking,
> > finite
> > >>>> > queues,
> > >>>> > > >>> etc), I do not have an answer yet.
> > >>>> > > >>>
> > >>>> > > >>> For the 2nd, I think that we should go with separate operators
> > >>>> for
> > >>>> > the
> > >>>> > > >>> source and the readers, for the following reasons:
> > >>>> > > >>>
> > >>>> > > >>> 1) This is more aligned with a potential future improvement
> > >>>> where the
> > >>>> > > >> split
> > >>>> > > >>> discovery becomes a responsibility of the JobManager and
> > >>>> readers are
> > >>>> > > >>> pooling more work from the JM.
> > >>>> > > >>>
> > >>>> > > >>> 2) The source is going to be the "single point of truth". It
> > >>>> will
> > >>>> > know
> > >>>> > > >> what
> > >>>> > > >>> has been processed and what not. If the source and the readers
> > >>>> are a
> > >>>> > > >> single
> > >>>> > > >>> operator with parallelism > 1, or in general, if the split
> > >>>> discovery
> > >>>> > is
> > >>>> > > >>> done by each task individually, then:
> > >>>> > > >>>   i) we have to have a deterministic scheme for each reader to
> > >>>> assign
> > >>>> > > >>> splits to itself (e.g. mod subtaskId). This is not necessarily
> > >>>> > trivial
> > >>>> > > >> for
> > >>>> > > >>> all sources.
> > >>>> > > >>>   ii) each reader would have to keep a copy of all its
> > processed
> > >>>> > slpits
> > >>>> > > >>>   iii) the state has to be a union state with a non-trivial
> > >>>> merging
> > >>>> > > >> logic
> > >>>> > > >>> in order to support rescaling.
> > >>>> > > >>>
> > >>>> > > >>> Two additional points that you raised above:
> > >>>> > > >>>
> > >>>> > > >>> i) The point that you raised that we need to keep all splits
> > >>>> > (processed
> > >>>> > > >> and
> > >>>> > > >>> not-processed) I think is a bit of a strong requirement. This
> > >>>> would
> > >>>> > > imply
> > >>>> > > >>> that for infinite sources the state will grow indefinitely.
> > >>>> This is
> > >>>> > > >> problem
> > >>>> > > >>> is even more pronounced if we do not have a single source that
> > >>>> > assigns
> > >>>> > > >>> splits to readers, as each reader will have its own copy of
> > the
> > >>>> > state.
> > >>>> > > >>>
> > >>>> > > >>> ii) it is true that for finite sources we need to somehow not
> > >>>> close
> > >>>> > the
> > >>>> > > >>> readers when the source/split discoverer finishes. The
> > >>>> > > >>> ContinuousFileReaderOperator has a work-around for that. It is
> > >>>> not
> > >>>> > > >> elegant,
> > >>>> > > >>> and checkpoints are not emitted after closing the source, but
> > >>>> this, I
> > >>>> > > >>> believe, is a bigger problem which requires more changes than
> > >>>> just
> > >>>> > > >>> refactoring the source interface.
> > >>>> > > >>>
> > >>>> > > >>> Cheers,
> > >>>> > > >>> Kostas
> > >>>> > > >>>
> > >>>> > > >>
> > >>>> > >
> > >>>> > >
> > >>>> >
> > >>>>
> > >>>
> >
> 

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Thomas Weise <th...@gmail.com>.
Thanks for putting together this proposal!

I see that the "Per Split Event Time" and "Event Time Alignment" sections
are still TBD.

It would probably be good to flesh those out a bit before proceeding too far
as the event time alignment will probably influence the interaction with
the split reader, specifically ReaderStatus emitNext(SourceOutput<E>
output).

We currently have only one implementation for event time alignment in the
Kinesis consumer. The synchronization in that case takes place as the last
step before records are emitted downstream (RecordEmitter). With the
currently proposed interfaces, the equivalent can be implemented in the
reader loop, although note that in the Kinesis consumer the per shard
threads push records.

Synchronization has not been implemented for the Kafka consumer yet.

https://issues.apache.org/jira/browse/FLINK-12675

When I looked at it, I realized that the implementation will look quite
different
from Kinesis because it needs to take place in the pull part, where records
are taken from the Kafka client. Due to the multiplexing it cannot be done
by blocking the split thread like it currently works for Kinesis. Reading
from individual Kafka partitions needs to be controlled via pause/resume
on the Kafka client.

To take on that responsibility the split thread would need to be aware of
the
watermarks or at least whether it should or should not continue to consume
a given split and this may require a different SourceReader or SourceOutput
interface.

Thanks,
Thomas


On Fri, Jul 26, 2019 at 1:39 AM Biao Liu <mm...@gmail.com> wrote:

> Hi Stephan,
>
> Thank you for feedback!
> Will take a look at your branch before public discussing.
>
>
> On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org> wrote:
>
> > Hi Biao!
> >
> > Thanks for reviving this. I would like to join this discussion, but am
> > quite occupied with the 1.9 release, so can we maybe pause this
> discussion
> > for a week or so?
> >
> > In the meantime I can share some suggestion based on prior experiments:
> >
> > How to do watermarks / timestamp extractors in a simpler and more
> flexible
> > way. I think that part is quite promising should be part of the new
> source
> > interface.
> >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
> >
> >
> >
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
> >
> >
> >
> > Some experiments on how to build the source reader and its library for
> > common threading/split patterns:
> >
> >
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
> >
> >
> > Best,
> > Stephan
> >
> >
> > On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> wrote:
> >
> >> Hi devs,
> >>
> >> Since 1.9 is nearly released, I think we could get back to FLIP-27. I
> >> believe it should be included in 1.10.
> >>
> >> There are so many things mentioned in document of FLIP-27. [1] I think
> >> we'd better discuss them separately. However the wiki is not a good
> place
> >> to discuss. I wrote google doc about SplitReader API which misses some
> >> details in the document. [2]
> >>
> >> 1.
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >> 2.
> >>
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
> >>
> >> CC Stephan, Aljoscha, Piotrek, Becket
> >>
> >>
> >> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> wrote:
> >>
> >>> Hi Steven,
> >>> Thank you for the feedback. Please take a look at the document FLIP-27
> >>> <
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface>
> which
> >>> is updated recently. A lot of details of enumerator were added in this
> >>> document. I think it would help.
> >>>
> >>> Steven Wu <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
> >>>
> >>>> This proposal mentioned that SplitEnumerator might run on the
> >>>> JobManager or
> >>>> in a single task on a TaskManager.
> >>>>
> >>>> if enumerator is a single task on a taskmanager, then the job DAG can
> >>>> never
> >>>> been embarrassingly parallel anymore. That will nullify the leverage
> of
> >>>> fine-grained recovery for embarrassingly parallel jobs.
> >>>>
> >>>> It's not clear to me what's the implication of running enumerator on
> the
> >>>> jobmanager. So I will leave that out for now.
> >>>>
> >>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> wrote:
> >>>>
> >>>> > Hi Stephan & Piotrek,
> >>>> >
> >>>> > Thank you for feedback.
> >>>> >
> >>>> > It seems that there are a lot of things to do in community. I am
> just
> >>>> > afraid that this discussion may be forgotten since there so many
> >>>> proposals
> >>>> > recently.
> >>>> > Anyway, wish to see the split topics soon :)
> >>>> >
> >>>> > Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四 下午8:21写道:
> >>>> >
> >>>> > > Hi Biao!
> >>>> > >
> >>>> > > This discussion was stalled because of preparations for the open
> >>>> sourcing
> >>>> > > & merging Blink. I think before creating the tickets we should
> >>>> split this
> >>>> > > discussion into topics/areas outlined by Stephan and create Flips
> >>>> for
> >>>> > that.
> >>>> > >
> >>>> > > I think there is no chance for this to be completed in couple of
> >>>> > remaining
> >>>> > > weeks/1 month before 1.8 feature freeze, however it would be good
> >>>> to aim
> >>>> > > with those changes for 1.9.
> >>>> > >
> >>>> > > Piotrek
> >>>> > >
> >>>> > > > On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> wrote:
> >>>> > > >
> >>>> > > > Hi community,
> >>>> > > > The summary of Stephan makes a lot sense to me. It is much
> clearer
> >>>> > indeed
> >>>> > > > after splitting the complex topic into small ones.
> >>>> > > > I was wondering is there any detail plan for next step? If not,
> I
> >>>> would
> >>>> > > > like to push this thing forward by creating some JIRA issues.
> >>>> > > > Another question is that should version 1.8 include these
> >>>> features?
> >>>> > > >
> >>>> > > > Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
> >>>> > > >
> >>>> > > >> Thanks everyone for the lively discussion. Let me try to
> >>>> summarize
> >>>> > > where I
> >>>> > > >> see convergence in the discussion and open issues.
> >>>> > > >> I'll try to group this by design aspect of the source. Please
> >>>> let me
> >>>> > > know
> >>>> > > >> if I got things wrong or missed something crucial here.
> >>>> > > >>
> >>>> > > >> For issues 1-3, if the below reflects the state of the
> >>>> discussion, I
> >>>> > > would
> >>>> > > >> try and update the FLIP in the next days.
> >>>> > > >> For the remaining ones we need more discussion.
> >>>> > > >>
> >>>> > > >> I would suggest to fork each of these aspects into a separate
> >>>> mail
> >>>> > > thread,
> >>>> > > >> or will loose sight of the individual aspects.
> >>>> > > >>
> >>>> > > >> *(1) Separation of Split Enumerator and Split Reader*
> >>>> > > >>
> >>>> > > >>  - All seem to agree this is a good thing
> >>>> > > >>  - Split Enumerator could in the end live on JobManager (and
> >>>> assign
> >>>> > > splits
> >>>> > > >> via RPC) or in a task (and assign splits via data streams)
> >>>> > > >>  - this discussion is orthogonal and should come later, when
> the
> >>>> > > interface
> >>>> > > >> is agreed upon.
> >>>> > > >>
> >>>> > > >> *(2) Split Readers for one or more splits*
> >>>> > > >>
> >>>> > > >>  - Discussion seems to agree that we need to support one reader
> >>>> that
> >>>> > > >> possibly handles multiple splits concurrently.
> >>>> > > >>  - The requirement comes from sources where one poll()-style
> call
> >>>> > > fetches
> >>>> > > >> data from different splits / partitions
> >>>> > > >>    --> example sources that require that would be for example
> >>>> Kafka,
> >>>> > > >> Pravega, Pulsar
> >>>> > > >>
> >>>> > > >>  - Could have one split reader per source, or multiple split
> >>>> readers
> >>>> > > that
> >>>> > > >> share the "poll()" function
> >>>> > > >>  - To not make it too complicated, we can start with thinking
> >>>> about
> >>>> > one
> >>>> > > >> split reader for all splits initially and see if that covers
> all
> >>>> > > >> requirements
> >>>> > > >>
> >>>> > > >> *(3) Threading model of the Split Reader*
> >>>> > > >>
> >>>> > > >>  - Most active part of the discussion ;-)
> >>>> > > >>
> >>>> > > >>  - A non-blocking way for Flink's task code to interact with
> the
> >>>> > source
> >>>> > > is
> >>>> > > >> needed in order to a task runtime code based on a
> >>>> > > >> single-threaded/actor-style task design
> >>>> > > >>    --> I personally am a big proponent of that, it will help
> with
> >>>> > > >> well-behaved checkpoints, efficiency, and simpler yet more
> robust
> >>>> > > runtime
> >>>> > > >> code
> >>>> > > >>
> >>>> > > >>  - Users care about simple abstraction, so as a subclass of
> >>>> > SplitReader
> >>>> > > >> (non-blocking / async) we need to have a BlockingSplitReader
> >>>> which
> >>>> > will
> >>>> > > >> form the basis of most source implementations.
> >>>> BlockingSplitReader
> >>>> > lets
> >>>> > > >> users do blocking simple poll() calls.
> >>>> > > >>  - The BlockingSplitReader would spawn a thread (or more) and
> the
> >>>> > > >> thread(s) can make blocking calls and hand over data buffers
> via
> >>>> a
> >>>> > > blocking
> >>>> > > >> queue
> >>>> > > >>  - This should allow us to cover both, a fully async runtime,
> >>>> and a
> >>>> > > simple
> >>>> > > >> blocking interface for users.
> >>>> > > >>  - This is actually very similar to how the Kafka connectors
> >>>> work.
> >>>> > Kafka
> >>>> > > >> 9+ with one thread, Kafka 8 with multiple threads
> >>>> > > >>
> >>>> > > >>  - On the base SplitReader (the async one), the non-blocking
> >>>> method
> >>>> > that
> >>>> > > >> gets the next chunk of data would signal data availability via
> a
> >>>> > > >> CompletableFuture, because that gives the best flexibility (can
> >>>> await
> >>>> > > >> completion or register notification handlers).
> >>>> > > >>  - The source task would register a "thenHandle()" (or similar)
> >>>> on the
> >>>> > > >> future to put a "take next data" task into the actor-style
> >>>> mailbox
> >>>> > > >>
> >>>> > > >> *(4) Split Enumeration and Assignment*
> >>>> > > >>
> >>>> > > >>  - Splits may be generated lazily, both in cases where there
> is a
> >>>> > > limited
> >>>> > > >> number of splits (but very many), or splits are discovered over
> >>>> time
> >>>> > > >>  - Assignment should also be lazy, to get better load balancing
> >>>> > > >>  - Assignment needs support locality preferences
> >>>> > > >>
> >>>> > > >>  - Possible design based on discussion so far:
> >>>> > > >>
> >>>> > > >>    --> SplitReader has a method "addSplits(SplitT...)" to add
> >>>> one or
> >>>> > > more
> >>>> > > >> splits. Some split readers might assume they have only one
> split
> >>>> ever,
> >>>> > > >> concurrently, others assume multiple splits. (Note: idea behind
> >>>> being
> >>>> > > able
> >>>> > > >> to add multiple splits at the same time is to ease startup
> where
> >>>> > > multiple
> >>>> > > >> splits may be assigned instantly.)
> >>>> > > >>    --> SplitReader has a context object on which it can call
> >>>> indicate
> >>>> > > when
> >>>> > > >> splits are completed. The enumerator gets that notification and
> >>>> can
> >>>> > use
> >>>> > > to
> >>>> > > >> decide when to assign new splits. This should help both in
> cases
> >>>> of
> >>>> > > sources
> >>>> > > >> that take splits lazily (file readers) and in case the source
> >>>> needs to
> >>>> > > >> preserve a partial order between splits (Kinesis, Pravega,
> >>>> Pulsar may
> >>>> > > need
> >>>> > > >> that).
> >>>> > > >>    --> SplitEnumerator gets notification when SplitReaders
> start
> >>>> and
> >>>> > > when
> >>>> > > >> they finish splits. They can decide at that moment to push more
> >>>> splits
> >>>> > > to
> >>>> > > >> that reader
> >>>> > > >>    --> The SplitEnumerator should probably be aware of the
> source
> >>>> > > >> parallelism, to build its initial distribution.
> >>>> > > >>
> >>>> > > >>  - Open question: Should the source expose something like "host
> >>>> > > >> preferences", so that yarn/mesos/k8s can take this into account
> >>>> when
> >>>> > > >> selecting a node to start a TM on?
> >>>> > > >>
> >>>> > > >> *(5) Watermarks and event time alignment*
> >>>> > > >>
> >>>> > > >>  - Watermark generation, as well as idleness, needs to be per
> >>>> split
> >>>> > > (like
> >>>> > > >> currently in the Kafka Source, per partition)
> >>>> > > >>  - It is desirable to support optional event-time-alignment,
> >>>> meaning
> >>>> > > that
> >>>> > > >> splits that are ahead are back-pressured or temporarily
> >>>> unsubscribed
> >>>> > > >>
> >>>> > > >>  - I think i would be desirable to encapsulate watermark
> >>>> generation
> >>>> > > logic
> >>>> > > >> in watermark generators, for a separation of concerns. The
> >>>> watermark
> >>>> > > >> generators should run per split.
> >>>> > > >>  - Using watermark generators would also help with another
> >>>> problem of
> >>>> > > the
> >>>> > > >> suggested interface, namely supporting non-periodic watermarks
> >>>> > > efficiently.
> >>>> > > >>
> >>>> > > >>  - Need a way to "dispatch" next record to different watermark
> >>>> > > generators
> >>>> > > >>  - Need a way to tell SplitReader to "suspend" a split until a
> >>>> certain
> >>>> > > >> watermark is reached (event time backpressure)
> >>>> > > >>  - This would in fact be not needed (and thus simpler) if we
> had
> >>>> a
> >>>> > > >> SplitReader per split and may be a reason to re-open that
> >>>> discussion
> >>>> > > >>
> >>>> > > >> *(6) Watermarks across splits and in the Split Enumerator*
> >>>> > > >>
> >>>> > > >>  - The split enumerator may need some watermark awareness,
> which
> >>>> > should
> >>>> > > be
> >>>> > > >> purely based on split metadata (like create timestamp of file
> >>>> splits)
> >>>> > > >>  - If there are still more splits with overlapping event time
> >>>> range
> >>>> > for
> >>>> > > a
> >>>> > > >> split reader, then that split reader should not advance the
> >>>> watermark
> >>>> > > >> within the split beyond the overlap boundary. Otherwise future
> >>>> splits
> >>>> > > will
> >>>> > > >> produce late data.
> >>>> > > >>
> >>>> > > >>  - One way to approach this could be that the split enumerator
> >>>> may
> >>>> > send
> >>>> > > >> watermarks to the readers, and the readers cannot emit
> watermarks
> >>>> > beyond
> >>>> > > >> that received watermark.
> >>>> > > >>  - Many split enumerators would simply immediately send
> Long.MAX
> >>>> out
> >>>> > and
> >>>> > > >> leave the progress purely to the split readers.
> >>>> > > >>
> >>>> > > >>  - For event-time alignment / split back pressure, this begs
> the
> >>>> > > question
> >>>> > > >> how we can avoid deadlocks that may arise when splits are
> >>>> suspended
> >>>> > for
> >>>> > > >> event time back pressure,
> >>>> > > >>
> >>>> > > >> *(7) Batch and streaming Unification*
> >>>> > > >>
> >>>> > > >>  - Functionality wise, the above design should support both
> >>>> > > >>  - Batch often (mostly) does not care about reading "in order"
> >>>> and
> >>>> > > >> generating watermarks
> >>>> > > >>    --> Might use different enumerator logic that is more
> locality
> >>>> > aware
> >>>> > > >> and ignores event time order
> >>>> > > >>    --> Does not generate watermarks
> >>>> > > >>  - Would be great if bounded sources could be identified at
> >>>> compile
> >>>> > > time,
> >>>> > > >> so that "env.addBoundedSource(...)" is type safe and can
> return a
> >>>> > > >> "BoundedDataStream".
> >>>> > > >>  - Possible to defer this discussion until later
> >>>> > > >>
> >>>> > > >> *Miscellaneous Comments*
> >>>> > > >>
> >>>> > > >>  - Should the source have a TypeInformation for the produced
> >>>> type,
> >>>> > > instead
> >>>> > > >> of a serializer? We need a type information in the stream
> >>>> anyways, and
> >>>> > > can
> >>>> > > >> derive the serializer from that. Plus, creating the serializer
> >>>> should
> >>>> > > >> respect the ExecutionConfig.
> >>>> > > >>
> >>>> > > >>  - The TypeSerializer interface is very powerful but also not
> >>>> easy to
> >>>> > > >> implement. Its purpose is to handle data super efficiently,
> >>>> support
> >>>> > > >> flexible ways of evolution, etc.
> >>>> > > >>  For metadata I would suggest to look at the
> >>>> SimpleVersionedSerializer
> >>>> > > >> instead, which is used for example for checkpoint master hooks,
> >>>> or for
> >>>> > > the
> >>>> > > >> streaming file sink. I think that is is a good match for cases
> >>>> where
> >>>> > we
> >>>> > > do
> >>>> > > >> not need more than ser/deser (no copy, etc.) and don't need to
> >>>> push
> >>>> > > >> versioning out of the serialization paths for best performance
> >>>> (as in
> >>>> > > the
> >>>> > > >> TypeSerializer)
> >>>> > > >>
> >>>> > > >>
> >>>> > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> >>>> > > >> k.kloudas@data-artisans.com>
> >>>> > > >> wrote:
> >>>> > > >>
> >>>> > > >>> Hi Biao,
> >>>> > > >>>
> >>>> > > >>> Thanks for the answer!
> >>>> > > >>>
> >>>> > > >>> So given the multi-threaded readers, now we have as open
> >>>> questions:
> >>>> > > >>>
> >>>> > > >>> 1) How do we let the checkpoints pass through our
> multi-threaded
> >>>> > reader
> >>>> > > >>> operator?
> >>>> > > >>>
> >>>> > > >>> 2) Do we have separate reader and source operators or not? In
> >>>> the
> >>>> > > >> strategy
> >>>> > > >>> that has a separate source, the source operator has a
> >>>> parallelism of
> >>>> > 1
> >>>> > > >> and
> >>>> > > >>> is responsible for split recovery only.
> >>>> > > >>>
> >>>> > > >>> For the first one, given also the constraints (blocking,
> finite
> >>>> > queues,
> >>>> > > >>> etc), I do not have an answer yet.
> >>>> > > >>>
> >>>> > > >>> For the 2nd, I think that we should go with separate operators
> >>>> for
> >>>> > the
> >>>> > > >>> source and the readers, for the following reasons:
> >>>> > > >>>
> >>>> > > >>> 1) This is more aligned with a potential future improvement
> >>>> where the
> >>>> > > >> split
> >>>> > > >>> discovery becomes a responsibility of the JobManager and
> >>>> readers are
> >>>> > > >>> pooling more work from the JM.
> >>>> > > >>>
> >>>> > > >>> 2) The source is going to be the "single point of truth". It
> >>>> will
> >>>> > know
> >>>> > > >> what
> >>>> > > >>> has been processed and what not. If the source and the readers
> >>>> are a
> >>>> > > >> single
> >>>> > > >>> operator with parallelism > 1, or in general, if the split
> >>>> discovery
> >>>> > is
> >>>> > > >>> done by each task individually, then:
> >>>> > > >>>   i) we have to have a deterministic scheme for each reader to
> >>>> assign
> >>>> > > >>> splits to itself (e.g. mod subtaskId). This is not necessarily
> >>>> > trivial
> >>>> > > >> for
> >>>> > > >>> all sources.
> >>>> > > >>>   ii) each reader would have to keep a copy of all its
> processed
> >>>> > slpits
> >>>> > > >>>   iii) the state has to be a union state with a non-trivial
> >>>> merging
> >>>> > > >> logic
> >>>> > > >>> in order to support rescaling.
> >>>> > > >>>
> >>>> > > >>> Two additional points that you raised above:
> >>>> > > >>>
> >>>> > > >>> i) The point that you raised that we need to keep all splits
> >>>> > (processed
> >>>> > > >> and
> >>>> > > >>> not-processed) I think is a bit of a strong requirement. This
> >>>> would
> >>>> > > imply
> >>>> > > >>> that for infinite sources the state will grow indefinitely.
> >>>> This is
> >>>> > > >> problem
> >>>> > > >>> is even more pronounced if we do not have a single source that
> >>>> > assigns
> >>>> > > >>> splits to readers, as each reader will have its own copy of
> the
> >>>> > state.
> >>>> > > >>>
> >>>> > > >>> ii) it is true that for finite sources we need to somehow not
> >>>> close
> >>>> > the
> >>>> > > >>> readers when the source/split discoverer finishes. The
> >>>> > > >>> ContinuousFileReaderOperator has a work-around for that. It is
> >>>> not
> >>>> > > >> elegant,
> >>>> > > >>> and checkpoints are not emitted after closing the source, but
> >>>> this, I
> >>>> > > >>> believe, is a bigger problem which requires more changes than
> >>>> just
> >>>> > > >>> refactoring the source interface.
> >>>> > > >>>
> >>>> > > >>> Cheers,
> >>>> > > >>> Kostas
> >>>> > > >>>
> >>>> > > >>
> >>>> > >
> >>>> > >
> >>>> >
> >>>>
> >>>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Biao Liu <mm...@gmail.com>.
Hi Stephan,

Thank you for feedback!
Will take a look at your branch before public discussing.


On Fri, Jul 26, 2019 at 12:01 AM Stephan Ewen <se...@apache.org> wrote:

> Hi Biao!
>
> Thanks for reviving this. I would like to join this discussion, but am
> quite occupied with the 1.9 release, so can we maybe pause this discussion
> for a week or so?
>
> In the meantime I can share some suggestion based on prior experiments:
>
> How to do watermarks / timestamp extractors in a simpler and more flexible
> way. I think that part is quite promising should be part of the new source
> interface.
>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime
>
>
> https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java
>
>
>
> Some experiments on how to build the source reader and its library for
> common threading/split patterns:
>
> https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src
>
>
> Best,
> Stephan
>
>
> On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> wrote:
>
>> Hi devs,
>>
>> Since 1.9 is nearly released, I think we could get back to FLIP-27. I
>> believe it should be included in 1.10.
>>
>> There are so many things mentioned in document of FLIP-27. [1] I think
>> we'd better discuss them separately. However the wiki is not a good place
>> to discuss. I wrote google doc about SplitReader API which misses some
>> details in the document. [2]
>>
>> 1.
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
>> 2.
>> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>>
>> CC Stephan, Aljoscha, Piotrek, Becket
>>
>>
>> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> wrote:
>>
>>> Hi Steven,
>>> Thank you for the feedback. Please take a look at the document FLIP-27
>>> <https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface> which
>>> is updated recently. A lot of details of enumerator were added in this
>>> document. I think it would help.
>>>
>>> Steven Wu <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
>>>
>>>> This proposal mentioned that SplitEnumerator might run on the
>>>> JobManager or
>>>> in a single task on a TaskManager.
>>>>
>>>> if enumerator is a single task on a taskmanager, then the job DAG can
>>>> never
>>>> been embarrassingly parallel anymore. That will nullify the leverage of
>>>> fine-grained recovery for embarrassingly parallel jobs.
>>>>
>>>> It's not clear to me what's the implication of running enumerator on the
>>>> jobmanager. So I will leave that out for now.
>>>>
>>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> wrote:
>>>>
>>>> > Hi Stephan & Piotrek,
>>>> >
>>>> > Thank you for feedback.
>>>> >
>>>> > It seems that there are a lot of things to do in community. I am just
>>>> > afraid that this discussion may be forgotten since there so many
>>>> proposals
>>>> > recently.
>>>> > Anyway, wish to see the split topics soon :)
>>>> >
>>>> > Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四 下午8:21写道:
>>>> >
>>>> > > Hi Biao!
>>>> > >
>>>> > > This discussion was stalled because of preparations for the open
>>>> sourcing
>>>> > > & merging Blink. I think before creating the tickets we should
>>>> split this
>>>> > > discussion into topics/areas outlined by Stephan and create Flips
>>>> for
>>>> > that.
>>>> > >
>>>> > > I think there is no chance for this to be completed in couple of
>>>> > remaining
>>>> > > weeks/1 month before 1.8 feature freeze, however it would be good
>>>> to aim
>>>> > > with those changes for 1.9.
>>>> > >
>>>> > > Piotrek
>>>> > >
>>>> > > > On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> wrote:
>>>> > > >
>>>> > > > Hi community,
>>>> > > > The summary of Stephan makes a lot sense to me. It is much clearer
>>>> > indeed
>>>> > > > after splitting the complex topic into small ones.
>>>> > > > I was wondering is there any detail plan for next step? If not, I
>>>> would
>>>> > > > like to push this thing forward by creating some JIRA issues.
>>>> > > > Another question is that should version 1.8 include these
>>>> features?
>>>> > > >
>>>> > > > Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
>>>> > > >
>>>> > > >> Thanks everyone for the lively discussion. Let me try to
>>>> summarize
>>>> > > where I
>>>> > > >> see convergence in the discussion and open issues.
>>>> > > >> I'll try to group this by design aspect of the source. Please
>>>> let me
>>>> > > know
>>>> > > >> if I got things wrong or missed something crucial here.
>>>> > > >>
>>>> > > >> For issues 1-3, if the below reflects the state of the
>>>> discussion, I
>>>> > > would
>>>> > > >> try and update the FLIP in the next days.
>>>> > > >> For the remaining ones we need more discussion.
>>>> > > >>
>>>> > > >> I would suggest to fork each of these aspects into a separate
>>>> mail
>>>> > > thread,
>>>> > > >> or will loose sight of the individual aspects.
>>>> > > >>
>>>> > > >> *(1) Separation of Split Enumerator and Split Reader*
>>>> > > >>
>>>> > > >>  - All seem to agree this is a good thing
>>>> > > >>  - Split Enumerator could in the end live on JobManager (and
>>>> assign
>>>> > > splits
>>>> > > >> via RPC) or in a task (and assign splits via data streams)
>>>> > > >>  - this discussion is orthogonal and should come later, when the
>>>> > > interface
>>>> > > >> is agreed upon.
>>>> > > >>
>>>> > > >> *(2) Split Readers for one or more splits*
>>>> > > >>
>>>> > > >>  - Discussion seems to agree that we need to support one reader
>>>> that
>>>> > > >> possibly handles multiple splits concurrently.
>>>> > > >>  - The requirement comes from sources where one poll()-style call
>>>> > > fetches
>>>> > > >> data from different splits / partitions
>>>> > > >>    --> example sources that require that would be for example
>>>> Kafka,
>>>> > > >> Pravega, Pulsar
>>>> > > >>
>>>> > > >>  - Could have one split reader per source, or multiple split
>>>> readers
>>>> > > that
>>>> > > >> share the "poll()" function
>>>> > > >>  - To not make it too complicated, we can start with thinking
>>>> about
>>>> > one
>>>> > > >> split reader for all splits initially and see if that covers all
>>>> > > >> requirements
>>>> > > >>
>>>> > > >> *(3) Threading model of the Split Reader*
>>>> > > >>
>>>> > > >>  - Most active part of the discussion ;-)
>>>> > > >>
>>>> > > >>  - A non-blocking way for Flink's task code to interact with the
>>>> > source
>>>> > > is
>>>> > > >> needed in order to a task runtime code based on a
>>>> > > >> single-threaded/actor-style task design
>>>> > > >>    --> I personally am a big proponent of that, it will help with
>>>> > > >> well-behaved checkpoints, efficiency, and simpler yet more robust
>>>> > > runtime
>>>> > > >> code
>>>> > > >>
>>>> > > >>  - Users care about simple abstraction, so as a subclass of
>>>> > SplitReader
>>>> > > >> (non-blocking / async) we need to have a BlockingSplitReader
>>>> which
>>>> > will
>>>> > > >> form the basis of most source implementations.
>>>> BlockingSplitReader
>>>> > lets
>>>> > > >> users do blocking simple poll() calls.
>>>> > > >>  - The BlockingSplitReader would spawn a thread (or more) and the
>>>> > > >> thread(s) can make blocking calls and hand over data buffers via
>>>> a
>>>> > > blocking
>>>> > > >> queue
>>>> > > >>  - This should allow us to cover both, a fully async runtime,
>>>> and a
>>>> > > simple
>>>> > > >> blocking interface for users.
>>>> > > >>  - This is actually very similar to how the Kafka connectors
>>>> work.
>>>> > Kafka
>>>> > > >> 9+ with one thread, Kafka 8 with multiple threads
>>>> > > >>
>>>> > > >>  - On the base SplitReader (the async one), the non-blocking
>>>> method
>>>> > that
>>>> > > >> gets the next chunk of data would signal data availability via a
>>>> > > >> CompletableFuture, because that gives the best flexibility (can
>>>> await
>>>> > > >> completion or register notification handlers).
>>>> > > >>  - The source task would register a "thenHandle()" (or similar)
>>>> on the
>>>> > > >> future to put a "take next data" task into the actor-style
>>>> mailbox
>>>> > > >>
>>>> > > >> *(4) Split Enumeration and Assignment*
>>>> > > >>
>>>> > > >>  - Splits may be generated lazily, both in cases where there is a
>>>> > > limited
>>>> > > >> number of splits (but very many), or splits are discovered over
>>>> time
>>>> > > >>  - Assignment should also be lazy, to get better load balancing
>>>> > > >>  - Assignment needs support locality preferences
>>>> > > >>
>>>> > > >>  - Possible design based on discussion so far:
>>>> > > >>
>>>> > > >>    --> SplitReader has a method "addSplits(SplitT...)" to add
>>>> one or
>>>> > > more
>>>> > > >> splits. Some split readers might assume they have only one split
>>>> ever,
>>>> > > >> concurrently, others assume multiple splits. (Note: idea behind
>>>> being
>>>> > > able
>>>> > > >> to add multiple splits at the same time is to ease startup where
>>>> > > multiple
>>>> > > >> splits may be assigned instantly.)
>>>> > > >>    --> SplitReader has a context object on which it can call
>>>> indicate
>>>> > > when
>>>> > > >> splits are completed. The enumerator gets that notification and
>>>> can
>>>> > use
>>>> > > to
>>>> > > >> decide when to assign new splits. This should help both in cases
>>>> of
>>>> > > sources
>>>> > > >> that take splits lazily (file readers) and in case the source
>>>> needs to
>>>> > > >> preserve a partial order between splits (Kinesis, Pravega,
>>>> Pulsar may
>>>> > > need
>>>> > > >> that).
>>>> > > >>    --> SplitEnumerator gets notification when SplitReaders start
>>>> and
>>>> > > when
>>>> > > >> they finish splits. They can decide at that moment to push more
>>>> splits
>>>> > > to
>>>> > > >> that reader
>>>> > > >>    --> The SplitEnumerator should probably be aware of the source
>>>> > > >> parallelism, to build its initial distribution.
>>>> > > >>
>>>> > > >>  - Open question: Should the source expose something like "host
>>>> > > >> preferences", so that yarn/mesos/k8s can take this into account
>>>> when
>>>> > > >> selecting a node to start a TM on?
>>>> > > >>
>>>> > > >> *(5) Watermarks and event time alignment*
>>>> > > >>
>>>> > > >>  - Watermark generation, as well as idleness, needs to be per
>>>> split
>>>> > > (like
>>>> > > >> currently in the Kafka Source, per partition)
>>>> > > >>  - It is desirable to support optional event-time-alignment,
>>>> meaning
>>>> > > that
>>>> > > >> splits that are ahead are back-pressured or temporarily
>>>> unsubscribed
>>>> > > >>
>>>> > > >>  - I think i would be desirable to encapsulate watermark
>>>> generation
>>>> > > logic
>>>> > > >> in watermark generators, for a separation of concerns. The
>>>> watermark
>>>> > > >> generators should run per split.
>>>> > > >>  - Using watermark generators would also help with another
>>>> problem of
>>>> > > the
>>>> > > >> suggested interface, namely supporting non-periodic watermarks
>>>> > > efficiently.
>>>> > > >>
>>>> > > >>  - Need a way to "dispatch" next record to different watermark
>>>> > > generators
>>>> > > >>  - Need a way to tell SplitReader to "suspend" a split until a
>>>> certain
>>>> > > >> watermark is reached (event time backpressure)
>>>> > > >>  - This would in fact be not needed (and thus simpler) if we had
>>>> a
>>>> > > >> SplitReader per split and may be a reason to re-open that
>>>> discussion
>>>> > > >>
>>>> > > >> *(6) Watermarks across splits and in the Split Enumerator*
>>>> > > >>
>>>> > > >>  - The split enumerator may need some watermark awareness, which
>>>> > should
>>>> > > be
>>>> > > >> purely based on split metadata (like create timestamp of file
>>>> splits)
>>>> > > >>  - If there are still more splits with overlapping event time
>>>> range
>>>> > for
>>>> > > a
>>>> > > >> split reader, then that split reader should not advance the
>>>> watermark
>>>> > > >> within the split beyond the overlap boundary. Otherwise future
>>>> splits
>>>> > > will
>>>> > > >> produce late data.
>>>> > > >>
>>>> > > >>  - One way to approach this could be that the split enumerator
>>>> may
>>>> > send
>>>> > > >> watermarks to the readers, and the readers cannot emit watermarks
>>>> > beyond
>>>> > > >> that received watermark.
>>>> > > >>  - Many split enumerators would simply immediately send Long.MAX
>>>> out
>>>> > and
>>>> > > >> leave the progress purely to the split readers.
>>>> > > >>
>>>> > > >>  - For event-time alignment / split back pressure, this begs the
>>>> > > question
>>>> > > >> how we can avoid deadlocks that may arise when splits are
>>>> suspended
>>>> > for
>>>> > > >> event time back pressure,
>>>> > > >>
>>>> > > >> *(7) Batch and streaming Unification*
>>>> > > >>
>>>> > > >>  - Functionality wise, the above design should support both
>>>> > > >>  - Batch often (mostly) does not care about reading "in order"
>>>> and
>>>> > > >> generating watermarks
>>>> > > >>    --> Might use different enumerator logic that is more locality
>>>> > aware
>>>> > > >> and ignores event time order
>>>> > > >>    --> Does not generate watermarks
>>>> > > >>  - Would be great if bounded sources could be identified at
>>>> compile
>>>> > > time,
>>>> > > >> so that "env.addBoundedSource(...)" is type safe and can return a
>>>> > > >> "BoundedDataStream".
>>>> > > >>  - Possible to defer this discussion until later
>>>> > > >>
>>>> > > >> *Miscellaneous Comments*
>>>> > > >>
>>>> > > >>  - Should the source have a TypeInformation for the produced
>>>> type,
>>>> > > instead
>>>> > > >> of a serializer? We need a type information in the stream
>>>> anyways, and
>>>> > > can
>>>> > > >> derive the serializer from that. Plus, creating the serializer
>>>> should
>>>> > > >> respect the ExecutionConfig.
>>>> > > >>
>>>> > > >>  - The TypeSerializer interface is very powerful but also not
>>>> easy to
>>>> > > >> implement. Its purpose is to handle data super efficiently,
>>>> support
>>>> > > >> flexible ways of evolution, etc.
>>>> > > >>  For metadata I would suggest to look at the
>>>> SimpleVersionedSerializer
>>>> > > >> instead, which is used for example for checkpoint master hooks,
>>>> or for
>>>> > > the
>>>> > > >> streaming file sink. I think that is is a good match for cases
>>>> where
>>>> > we
>>>> > > do
>>>> > > >> not need more than ser/deser (no copy, etc.) and don't need to
>>>> push
>>>> > > >> versioning out of the serialization paths for best performance
>>>> (as in
>>>> > > the
>>>> > > >> TypeSerializer)
>>>> > > >>
>>>> > > >>
>>>> > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>>>> > > >> k.kloudas@data-artisans.com>
>>>> > > >> wrote:
>>>> > > >>
>>>> > > >>> Hi Biao,
>>>> > > >>>
>>>> > > >>> Thanks for the answer!
>>>> > > >>>
>>>> > > >>> So given the multi-threaded readers, now we have as open
>>>> questions:
>>>> > > >>>
>>>> > > >>> 1) How do we let the checkpoints pass through our multi-threaded
>>>> > reader
>>>> > > >>> operator?
>>>> > > >>>
>>>> > > >>> 2) Do we have separate reader and source operators or not? In
>>>> the
>>>> > > >> strategy
>>>> > > >>> that has a separate source, the source operator has a
>>>> parallelism of
>>>> > 1
>>>> > > >> and
>>>> > > >>> is responsible for split recovery only.
>>>> > > >>>
>>>> > > >>> For the first one, given also the constraints (blocking, finite
>>>> > queues,
>>>> > > >>> etc), I do not have an answer yet.
>>>> > > >>>
>>>> > > >>> For the 2nd, I think that we should go with separate operators
>>>> for
>>>> > the
>>>> > > >>> source and the readers, for the following reasons:
>>>> > > >>>
>>>> > > >>> 1) This is more aligned with a potential future improvement
>>>> where the
>>>> > > >> split
>>>> > > >>> discovery becomes a responsibility of the JobManager and
>>>> readers are
>>>> > > >>> pooling more work from the JM.
>>>> > > >>>
>>>> > > >>> 2) The source is going to be the "single point of truth". It
>>>> will
>>>> > know
>>>> > > >> what
>>>> > > >>> has been processed and what not. If the source and the readers
>>>> are a
>>>> > > >> single
>>>> > > >>> operator with parallelism > 1, or in general, if the split
>>>> discovery
>>>> > is
>>>> > > >>> done by each task individually, then:
>>>> > > >>>   i) we have to have a deterministic scheme for each reader to
>>>> assign
>>>> > > >>> splits to itself (e.g. mod subtaskId). This is not necessarily
>>>> > trivial
>>>> > > >> for
>>>> > > >>> all sources.
>>>> > > >>>   ii) each reader would have to keep a copy of all its processed
>>>> > slpits
>>>> > > >>>   iii) the state has to be a union state with a non-trivial
>>>> merging
>>>> > > >> logic
>>>> > > >>> in order to support rescaling.
>>>> > > >>>
>>>> > > >>> Two additional points that you raised above:
>>>> > > >>>
>>>> > > >>> i) The point that you raised that we need to keep all splits
>>>> > (processed
>>>> > > >> and
>>>> > > >>> not-processed) I think is a bit of a strong requirement. This
>>>> would
>>>> > > imply
>>>> > > >>> that for infinite sources the state will grow indefinitely.
>>>> This is
>>>> > > >> problem
>>>> > > >>> is even more pronounced if we do not have a single source that
>>>> > assigns
>>>> > > >>> splits to readers, as each reader will have its own copy of the
>>>> > state.
>>>> > > >>>
>>>> > > >>> ii) it is true that for finite sources we need to somehow not
>>>> close
>>>> > the
>>>> > > >>> readers when the source/split discoverer finishes. The
>>>> > > >>> ContinuousFileReaderOperator has a work-around for that. It is
>>>> not
>>>> > > >> elegant,
>>>> > > >>> and checkpoints are not emitted after closing the source, but
>>>> this, I
>>>> > > >>> believe, is a bigger problem which requires more changes than
>>>> just
>>>> > > >>> refactoring the source interface.
>>>> > > >>>
>>>> > > >>> Cheers,
>>>> > > >>> Kostas
>>>> > > >>>
>>>> > > >>
>>>> > >
>>>> > >
>>>> >
>>>>
>>>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Stephan Ewen <se...@apache.org>.
Hi Biao!

Thanks for reviving this. I would like to join this discussion, but am
quite occupied with the 1.9 release, so can we maybe pause this discussion
for a week or so?

In the meantime I can share some suggestion based on prior experiments:

How to do watermarks / timestamp extractors in a simpler and more flexible
way. I think that part is quite promising should be part of the new source
interface.
https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/eventtime

https://github.com/StephanEwen/flink/blob/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src/SourceOutput.java



Some experiments on how to build the source reader and its library for
common threading/split patterns:
https://github.com/StephanEwen/flink/tree/source_interface/flink-core/src/main/java/org/apache/flink/api/common/src


Best,
Stephan


On Thu, Jul 25, 2019 at 10:03 AM Biao Liu <mm...@gmail.com> wrote:

> Hi devs,
>
> Since 1.9 is nearly released, I think we could get back to FLIP-27. I
> believe it should be included in 1.10.
>
> There are so many things mentioned in document of FLIP-27. [1] I think
> we'd better discuss them separately. However the wiki is not a good place
> to discuss. I wrote google doc about SplitReader API which misses some
> details in the document. [2]
>
> 1.
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> 2.
> https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing
>
> CC Stephan, Aljoscha, Piotrek, Becket
>
>
> On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> wrote:
>
>> Hi Steven,
>> Thank you for the feedback. Please take a look at the document FLIP-27
>> <https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface> which
>> is updated recently. A lot of details of enumerator were added in this
>> document. I think it would help.
>>
>> Steven Wu <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
>>
>>> This proposal mentioned that SplitEnumerator might run on the JobManager
>>> or
>>> in a single task on a TaskManager.
>>>
>>> if enumerator is a single task on a taskmanager, then the job DAG can
>>> never
>>> been embarrassingly parallel anymore. That will nullify the leverage of
>>> fine-grained recovery for embarrassingly parallel jobs.
>>>
>>> It's not clear to me what's the implication of running enumerator on the
>>> jobmanager. So I will leave that out for now.
>>>
>>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> wrote:
>>>
>>> > Hi Stephan & Piotrek,
>>> >
>>> > Thank you for feedback.
>>> >
>>> > It seems that there are a lot of things to do in community. I am just
>>> > afraid that this discussion may be forgotten since there so many
>>> proposals
>>> > recently.
>>> > Anyway, wish to see the split topics soon :)
>>> >
>>> > Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四 下午8:21写道:
>>> >
>>> > > Hi Biao!
>>> > >
>>> > > This discussion was stalled because of preparations for the open
>>> sourcing
>>> > > & merging Blink. I think before creating the tickets we should split
>>> this
>>> > > discussion into topics/areas outlined by Stephan and create Flips for
>>> > that.
>>> > >
>>> > > I think there is no chance for this to be completed in couple of
>>> > remaining
>>> > > weeks/1 month before 1.8 feature freeze, however it would be good to
>>> aim
>>> > > with those changes for 1.9.
>>> > >
>>> > > Piotrek
>>> > >
>>> > > > On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> wrote:
>>> > > >
>>> > > > Hi community,
>>> > > > The summary of Stephan makes a lot sense to me. It is much clearer
>>> > indeed
>>> > > > after splitting the complex topic into small ones.
>>> > > > I was wondering is there any detail plan for next step? If not, I
>>> would
>>> > > > like to push this thing forward by creating some JIRA issues.
>>> > > > Another question is that should version 1.8 include these features?
>>> > > >
>>> > > > Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
>>> > > >
>>> > > >> Thanks everyone for the lively discussion. Let me try to summarize
>>> > > where I
>>> > > >> see convergence in the discussion and open issues.
>>> > > >> I'll try to group this by design aspect of the source. Please let
>>> me
>>> > > know
>>> > > >> if I got things wrong or missed something crucial here.
>>> > > >>
>>> > > >> For issues 1-3, if the below reflects the state of the
>>> discussion, I
>>> > > would
>>> > > >> try and update the FLIP in the next days.
>>> > > >> For the remaining ones we need more discussion.
>>> > > >>
>>> > > >> I would suggest to fork each of these aspects into a separate mail
>>> > > thread,
>>> > > >> or will loose sight of the individual aspects.
>>> > > >>
>>> > > >> *(1) Separation of Split Enumerator and Split Reader*
>>> > > >>
>>> > > >>  - All seem to agree this is a good thing
>>> > > >>  - Split Enumerator could in the end live on JobManager (and
>>> assign
>>> > > splits
>>> > > >> via RPC) or in a task (and assign splits via data streams)
>>> > > >>  - this discussion is orthogonal and should come later, when the
>>> > > interface
>>> > > >> is agreed upon.
>>> > > >>
>>> > > >> *(2) Split Readers for one or more splits*
>>> > > >>
>>> > > >>  - Discussion seems to agree that we need to support one reader
>>> that
>>> > > >> possibly handles multiple splits concurrently.
>>> > > >>  - The requirement comes from sources where one poll()-style call
>>> > > fetches
>>> > > >> data from different splits / partitions
>>> > > >>    --> example sources that require that would be for example
>>> Kafka,
>>> > > >> Pravega, Pulsar
>>> > > >>
>>> > > >>  - Could have one split reader per source, or multiple split
>>> readers
>>> > > that
>>> > > >> share the "poll()" function
>>> > > >>  - To not make it too complicated, we can start with thinking
>>> about
>>> > one
>>> > > >> split reader for all splits initially and see if that covers all
>>> > > >> requirements
>>> > > >>
>>> > > >> *(3) Threading model of the Split Reader*
>>> > > >>
>>> > > >>  - Most active part of the discussion ;-)
>>> > > >>
>>> > > >>  - A non-blocking way for Flink's task code to interact with the
>>> > source
>>> > > is
>>> > > >> needed in order to a task runtime code based on a
>>> > > >> single-threaded/actor-style task design
>>> > > >>    --> I personally am a big proponent of that, it will help with
>>> > > >> well-behaved checkpoints, efficiency, and simpler yet more robust
>>> > > runtime
>>> > > >> code
>>> > > >>
>>> > > >>  - Users care about simple abstraction, so as a subclass of
>>> > SplitReader
>>> > > >> (non-blocking / async) we need to have a BlockingSplitReader which
>>> > will
>>> > > >> form the basis of most source implementations. BlockingSplitReader
>>> > lets
>>> > > >> users do blocking simple poll() calls.
>>> > > >>  - The BlockingSplitReader would spawn a thread (or more) and the
>>> > > >> thread(s) can make blocking calls and hand over data buffers via a
>>> > > blocking
>>> > > >> queue
>>> > > >>  - This should allow us to cover both, a fully async runtime, and
>>> a
>>> > > simple
>>> > > >> blocking interface for users.
>>> > > >>  - This is actually very similar to how the Kafka connectors work.
>>> > Kafka
>>> > > >> 9+ with one thread, Kafka 8 with multiple threads
>>> > > >>
>>> > > >>  - On the base SplitReader (the async one), the non-blocking
>>> method
>>> > that
>>> > > >> gets the next chunk of data would signal data availability via a
>>> > > >> CompletableFuture, because that gives the best flexibility (can
>>> await
>>> > > >> completion or register notification handlers).
>>> > > >>  - The source task would register a "thenHandle()" (or similar)
>>> on the
>>> > > >> future to put a "take next data" task into the actor-style mailbox
>>> > > >>
>>> > > >> *(4) Split Enumeration and Assignment*
>>> > > >>
>>> > > >>  - Splits may be generated lazily, both in cases where there is a
>>> > > limited
>>> > > >> number of splits (but very many), or splits are discovered over
>>> time
>>> > > >>  - Assignment should also be lazy, to get better load balancing
>>> > > >>  - Assignment needs support locality preferences
>>> > > >>
>>> > > >>  - Possible design based on discussion so far:
>>> > > >>
>>> > > >>    --> SplitReader has a method "addSplits(SplitT...)" to add one
>>> or
>>> > > more
>>> > > >> splits. Some split readers might assume they have only one split
>>> ever,
>>> > > >> concurrently, others assume multiple splits. (Note: idea behind
>>> being
>>> > > able
>>> > > >> to add multiple splits at the same time is to ease startup where
>>> > > multiple
>>> > > >> splits may be assigned instantly.)
>>> > > >>    --> SplitReader has a context object on which it can call
>>> indicate
>>> > > when
>>> > > >> splits are completed. The enumerator gets that notification and
>>> can
>>> > use
>>> > > to
>>> > > >> decide when to assign new splits. This should help both in cases
>>> of
>>> > > sources
>>> > > >> that take splits lazily (file readers) and in case the source
>>> needs to
>>> > > >> preserve a partial order between splits (Kinesis, Pravega, Pulsar
>>> may
>>> > > need
>>> > > >> that).
>>> > > >>    --> SplitEnumerator gets notification when SplitReaders start
>>> and
>>> > > when
>>> > > >> they finish splits. They can decide at that moment to push more
>>> splits
>>> > > to
>>> > > >> that reader
>>> > > >>    --> The SplitEnumerator should probably be aware of the source
>>> > > >> parallelism, to build its initial distribution.
>>> > > >>
>>> > > >>  - Open question: Should the source expose something like "host
>>> > > >> preferences", so that yarn/mesos/k8s can take this into account
>>> when
>>> > > >> selecting a node to start a TM on?
>>> > > >>
>>> > > >> *(5) Watermarks and event time alignment*
>>> > > >>
>>> > > >>  - Watermark generation, as well as idleness, needs to be per
>>> split
>>> > > (like
>>> > > >> currently in the Kafka Source, per partition)
>>> > > >>  - It is desirable to support optional event-time-alignment,
>>> meaning
>>> > > that
>>> > > >> splits that are ahead are back-pressured or temporarily
>>> unsubscribed
>>> > > >>
>>> > > >>  - I think i would be desirable to encapsulate watermark
>>> generation
>>> > > logic
>>> > > >> in watermark generators, for a separation of concerns. The
>>> watermark
>>> > > >> generators should run per split.
>>> > > >>  - Using watermark generators would also help with another
>>> problem of
>>> > > the
>>> > > >> suggested interface, namely supporting non-periodic watermarks
>>> > > efficiently.
>>> > > >>
>>> > > >>  - Need a way to "dispatch" next record to different watermark
>>> > > generators
>>> > > >>  - Need a way to tell SplitReader to "suspend" a split until a
>>> certain
>>> > > >> watermark is reached (event time backpressure)
>>> > > >>  - This would in fact be not needed (and thus simpler) if we had a
>>> > > >> SplitReader per split and may be a reason to re-open that
>>> discussion
>>> > > >>
>>> > > >> *(6) Watermarks across splits and in the Split Enumerator*
>>> > > >>
>>> > > >>  - The split enumerator may need some watermark awareness, which
>>> > should
>>> > > be
>>> > > >> purely based on split metadata (like create timestamp of file
>>> splits)
>>> > > >>  - If there are still more splits with overlapping event time
>>> range
>>> > for
>>> > > a
>>> > > >> split reader, then that split reader should not advance the
>>> watermark
>>> > > >> within the split beyond the overlap boundary. Otherwise future
>>> splits
>>> > > will
>>> > > >> produce late data.
>>> > > >>
>>> > > >>  - One way to approach this could be that the split enumerator may
>>> > send
>>> > > >> watermarks to the readers, and the readers cannot emit watermarks
>>> > beyond
>>> > > >> that received watermark.
>>> > > >>  - Many split enumerators would simply immediately send Long.MAX
>>> out
>>> > and
>>> > > >> leave the progress purely to the split readers.
>>> > > >>
>>> > > >>  - For event-time alignment / split back pressure, this begs the
>>> > > question
>>> > > >> how we can avoid deadlocks that may arise when splits are
>>> suspended
>>> > for
>>> > > >> event time back pressure,
>>> > > >>
>>> > > >> *(7) Batch and streaming Unification*
>>> > > >>
>>> > > >>  - Functionality wise, the above design should support both
>>> > > >>  - Batch often (mostly) does not care about reading "in order" and
>>> > > >> generating watermarks
>>> > > >>    --> Might use different enumerator logic that is more locality
>>> > aware
>>> > > >> and ignores event time order
>>> > > >>    --> Does not generate watermarks
>>> > > >>  - Would be great if bounded sources could be identified at
>>> compile
>>> > > time,
>>> > > >> so that "env.addBoundedSource(...)" is type safe and can return a
>>> > > >> "BoundedDataStream".
>>> > > >>  - Possible to defer this discussion until later
>>> > > >>
>>> > > >> *Miscellaneous Comments*
>>> > > >>
>>> > > >>  - Should the source have a TypeInformation for the produced type,
>>> > > instead
>>> > > >> of a serializer? We need a type information in the stream
>>> anyways, and
>>> > > can
>>> > > >> derive the serializer from that. Plus, creating the serializer
>>> should
>>> > > >> respect the ExecutionConfig.
>>> > > >>
>>> > > >>  - The TypeSerializer interface is very powerful but also not
>>> easy to
>>> > > >> implement. Its purpose is to handle data super efficiently,
>>> support
>>> > > >> flexible ways of evolution, etc.
>>> > > >>  For metadata I would suggest to look at the
>>> SimpleVersionedSerializer
>>> > > >> instead, which is used for example for checkpoint master hooks,
>>> or for
>>> > > the
>>> > > >> streaming file sink. I think that is is a good match for cases
>>> where
>>> > we
>>> > > do
>>> > > >> not need more than ser/deser (no copy, etc.) and don't need to
>>> push
>>> > > >> versioning out of the serialization paths for best performance
>>> (as in
>>> > > the
>>> > > >> TypeSerializer)
>>> > > >>
>>> > > >>
>>> > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>>> > > >> k.kloudas@data-artisans.com>
>>> > > >> wrote:
>>> > > >>
>>> > > >>> Hi Biao,
>>> > > >>>
>>> > > >>> Thanks for the answer!
>>> > > >>>
>>> > > >>> So given the multi-threaded readers, now we have as open
>>> questions:
>>> > > >>>
>>> > > >>> 1) How do we let the checkpoints pass through our multi-threaded
>>> > reader
>>> > > >>> operator?
>>> > > >>>
>>> > > >>> 2) Do we have separate reader and source operators or not? In the
>>> > > >> strategy
>>> > > >>> that has a separate source, the source operator has a
>>> parallelism of
>>> > 1
>>> > > >> and
>>> > > >>> is responsible for split recovery only.
>>> > > >>>
>>> > > >>> For the first one, given also the constraints (blocking, finite
>>> > queues,
>>> > > >>> etc), I do not have an answer yet.
>>> > > >>>
>>> > > >>> For the 2nd, I think that we should go with separate operators
>>> for
>>> > the
>>> > > >>> source and the readers, for the following reasons:
>>> > > >>>
>>> > > >>> 1) This is more aligned with a potential future improvement
>>> where the
>>> > > >> split
>>> > > >>> discovery becomes a responsibility of the JobManager and readers
>>> are
>>> > > >>> pooling more work from the JM.
>>> > > >>>
>>> > > >>> 2) The source is going to be the "single point of truth". It will
>>> > know
>>> > > >> what
>>> > > >>> has been processed and what not. If the source and the readers
>>> are a
>>> > > >> single
>>> > > >>> operator with parallelism > 1, or in general, if the split
>>> discovery
>>> > is
>>> > > >>> done by each task individually, then:
>>> > > >>>   i) we have to have a deterministic scheme for each reader to
>>> assign
>>> > > >>> splits to itself (e.g. mod subtaskId). This is not necessarily
>>> > trivial
>>> > > >> for
>>> > > >>> all sources.
>>> > > >>>   ii) each reader would have to keep a copy of all its processed
>>> > slpits
>>> > > >>>   iii) the state has to be a union state with a non-trivial
>>> merging
>>> > > >> logic
>>> > > >>> in order to support rescaling.
>>> > > >>>
>>> > > >>> Two additional points that you raised above:
>>> > > >>>
>>> > > >>> i) The point that you raised that we need to keep all splits
>>> > (processed
>>> > > >> and
>>> > > >>> not-processed) I think is a bit of a strong requirement. This
>>> would
>>> > > imply
>>> > > >>> that for infinite sources the state will grow indefinitely. This
>>> is
>>> > > >> problem
>>> > > >>> is even more pronounced if we do not have a single source that
>>> > assigns
>>> > > >>> splits to readers, as each reader will have its own copy of the
>>> > state.
>>> > > >>>
>>> > > >>> ii) it is true that for finite sources we need to somehow not
>>> close
>>> > the
>>> > > >>> readers when the source/split discoverer finishes. The
>>> > > >>> ContinuousFileReaderOperator has a work-around for that. It is
>>> not
>>> > > >> elegant,
>>> > > >>> and checkpoints are not emitted after closing the source, but
>>> this, I
>>> > > >>> believe, is a bigger problem which requires more changes than
>>> just
>>> > > >>> refactoring the source interface.
>>> > > >>>
>>> > > >>> Cheers,
>>> > > >>> Kostas
>>> > > >>>
>>> > > >>
>>> > >
>>> > >
>>> >
>>>
>>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Biao Liu <mm...@gmail.com>.
Hi devs,

Since 1.9 is nearly released, I think we could get back to FLIP-27. I
believe it should be included in 1.10.

There are so many things mentioned in document of FLIP-27. [1] I think we'd
better discuss them separately. However the wiki is not a good place to
discuss. I wrote google doc about SplitReader API which misses some details
in the document. [2]

1.
https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
2.
https://docs.google.com/document/d/1R1s_89T4S3CZwq7Tf31DciaMCqZwrLHGZFqPASu66oE/edit?usp=sharing

CC Stephan, Aljoscha, Piotrek, Becket


On Thu, Mar 28, 2019 at 4:38 PM Biao Liu <mm...@gmail.com> wrote:

> Hi Steven,
> Thank you for the feedback. Please take a look at the document FLIP-27
> <https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface> which
> is updated recently. A lot of details of enumerator were added in this
> document. I think it would help.
>
> Steven Wu <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:
>
>> This proposal mentioned that SplitEnumerator might run on the JobManager
>> or
>> in a single task on a TaskManager.
>>
>> if enumerator is a single task on a taskmanager, then the job DAG can
>> never
>> been embarrassingly parallel anymore. That will nullify the leverage of
>> fine-grained recovery for embarrassingly parallel jobs.
>>
>> It's not clear to me what's the implication of running enumerator on the
>> jobmanager. So I will leave that out for now.
>>
>> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> wrote:
>>
>> > Hi Stephan & Piotrek,
>> >
>> > Thank you for feedback.
>> >
>> > It seems that there are a lot of things to do in community. I am just
>> > afraid that this discussion may be forgotten since there so many
>> proposals
>> > recently.
>> > Anyway, wish to see the split topics soon :)
>> >
>> > Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四 下午8:21写道:
>> >
>> > > Hi Biao!
>> > >
>> > > This discussion was stalled because of preparations for the open
>> sourcing
>> > > & merging Blink. I think before creating the tickets we should split
>> this
>> > > discussion into topics/areas outlined by Stephan and create Flips for
>> > that.
>> > >
>> > > I think there is no chance for this to be completed in couple of
>> > remaining
>> > > weeks/1 month before 1.8 feature freeze, however it would be good to
>> aim
>> > > with those changes for 1.9.
>> > >
>> > > Piotrek
>> > >
>> > > > On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> wrote:
>> > > >
>> > > > Hi community,
>> > > > The summary of Stephan makes a lot sense to me. It is much clearer
>> > indeed
>> > > > after splitting the complex topic into small ones.
>> > > > I was wondering is there any detail plan for next step? If not, I
>> would
>> > > > like to push this thing forward by creating some JIRA issues.
>> > > > Another question is that should version 1.8 include these features?
>> > > >
>> > > > Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
>> > > >
>> > > >> Thanks everyone for the lively discussion. Let me try to summarize
>> > > where I
>> > > >> see convergence in the discussion and open issues.
>> > > >> I'll try to group this by design aspect of the source. Please let
>> me
>> > > know
>> > > >> if I got things wrong or missed something crucial here.
>> > > >>
>> > > >> For issues 1-3, if the below reflects the state of the discussion,
>> I
>> > > would
>> > > >> try and update the FLIP in the next days.
>> > > >> For the remaining ones we need more discussion.
>> > > >>
>> > > >> I would suggest to fork each of these aspects into a separate mail
>> > > thread,
>> > > >> or will loose sight of the individual aspects.
>> > > >>
>> > > >> *(1) Separation of Split Enumerator and Split Reader*
>> > > >>
>> > > >>  - All seem to agree this is a good thing
>> > > >>  - Split Enumerator could in the end live on JobManager (and assign
>> > > splits
>> > > >> via RPC) or in a task (and assign splits via data streams)
>> > > >>  - this discussion is orthogonal and should come later, when the
>> > > interface
>> > > >> is agreed upon.
>> > > >>
>> > > >> *(2) Split Readers for one or more splits*
>> > > >>
>> > > >>  - Discussion seems to agree that we need to support one reader
>> that
>> > > >> possibly handles multiple splits concurrently.
>> > > >>  - The requirement comes from sources where one poll()-style call
>> > > fetches
>> > > >> data from different splits / partitions
>> > > >>    --> example sources that require that would be for example
>> Kafka,
>> > > >> Pravega, Pulsar
>> > > >>
>> > > >>  - Could have one split reader per source, or multiple split
>> readers
>> > > that
>> > > >> share the "poll()" function
>> > > >>  - To not make it too complicated, we can start with thinking about
>> > one
>> > > >> split reader for all splits initially and see if that covers all
>> > > >> requirements
>> > > >>
>> > > >> *(3) Threading model of the Split Reader*
>> > > >>
>> > > >>  - Most active part of the discussion ;-)
>> > > >>
>> > > >>  - A non-blocking way for Flink's task code to interact with the
>> > source
>> > > is
>> > > >> needed in order to a task runtime code based on a
>> > > >> single-threaded/actor-style task design
>> > > >>    --> I personally am a big proponent of that, it will help with
>> > > >> well-behaved checkpoints, efficiency, and simpler yet more robust
>> > > runtime
>> > > >> code
>> > > >>
>> > > >>  - Users care about simple abstraction, so as a subclass of
>> > SplitReader
>> > > >> (non-blocking / async) we need to have a BlockingSplitReader which
>> > will
>> > > >> form the basis of most source implementations. BlockingSplitReader
>> > lets
>> > > >> users do blocking simple poll() calls.
>> > > >>  - The BlockingSplitReader would spawn a thread (or more) and the
>> > > >> thread(s) can make blocking calls and hand over data buffers via a
>> > > blocking
>> > > >> queue
>> > > >>  - This should allow us to cover both, a fully async runtime, and a
>> > > simple
>> > > >> blocking interface for users.
>> > > >>  - This is actually very similar to how the Kafka connectors work.
>> > Kafka
>> > > >> 9+ with one thread, Kafka 8 with multiple threads
>> > > >>
>> > > >>  - On the base SplitReader (the async one), the non-blocking method
>> > that
>> > > >> gets the next chunk of data would signal data availability via a
>> > > >> CompletableFuture, because that gives the best flexibility (can
>> await
>> > > >> completion or register notification handlers).
>> > > >>  - The source task would register a "thenHandle()" (or similar) on
>> the
>> > > >> future to put a "take next data" task into the actor-style mailbox
>> > > >>
>> > > >> *(4) Split Enumeration and Assignment*
>> > > >>
>> > > >>  - Splits may be generated lazily, both in cases where there is a
>> > > limited
>> > > >> number of splits (but very many), or splits are discovered over
>> time
>> > > >>  - Assignment should also be lazy, to get better load balancing
>> > > >>  - Assignment needs support locality preferences
>> > > >>
>> > > >>  - Possible design based on discussion so far:
>> > > >>
>> > > >>    --> SplitReader has a method "addSplits(SplitT...)" to add one
>> or
>> > > more
>> > > >> splits. Some split readers might assume they have only one split
>> ever,
>> > > >> concurrently, others assume multiple splits. (Note: idea behind
>> being
>> > > able
>> > > >> to add multiple splits at the same time is to ease startup where
>> > > multiple
>> > > >> splits may be assigned instantly.)
>> > > >>    --> SplitReader has a context object on which it can call
>> indicate
>> > > when
>> > > >> splits are completed. The enumerator gets that notification and can
>> > use
>> > > to
>> > > >> decide when to assign new splits. This should help both in cases of
>> > > sources
>> > > >> that take splits lazily (file readers) and in case the source
>> needs to
>> > > >> preserve a partial order between splits (Kinesis, Pravega, Pulsar
>> may
>> > > need
>> > > >> that).
>> > > >>    --> SplitEnumerator gets notification when SplitReaders start
>> and
>> > > when
>> > > >> they finish splits. They can decide at that moment to push more
>> splits
>> > > to
>> > > >> that reader
>> > > >>    --> The SplitEnumerator should probably be aware of the source
>> > > >> parallelism, to build its initial distribution.
>> > > >>
>> > > >>  - Open question: Should the source expose something like "host
>> > > >> preferences", so that yarn/mesos/k8s can take this into account
>> when
>> > > >> selecting a node to start a TM on?
>> > > >>
>> > > >> *(5) Watermarks and event time alignment*
>> > > >>
>> > > >>  - Watermark generation, as well as idleness, needs to be per split
>> > > (like
>> > > >> currently in the Kafka Source, per partition)
>> > > >>  - It is desirable to support optional event-time-alignment,
>> meaning
>> > > that
>> > > >> splits that are ahead are back-pressured or temporarily
>> unsubscribed
>> > > >>
>> > > >>  - I think i would be desirable to encapsulate watermark generation
>> > > logic
>> > > >> in watermark generators, for a separation of concerns. The
>> watermark
>> > > >> generators should run per split.
>> > > >>  - Using watermark generators would also help with another problem
>> of
>> > > the
>> > > >> suggested interface, namely supporting non-periodic watermarks
>> > > efficiently.
>> > > >>
>> > > >>  - Need a way to "dispatch" next record to different watermark
>> > > generators
>> > > >>  - Need a way to tell SplitReader to "suspend" a split until a
>> certain
>> > > >> watermark is reached (event time backpressure)
>> > > >>  - This would in fact be not needed (and thus simpler) if we had a
>> > > >> SplitReader per split and may be a reason to re-open that
>> discussion
>> > > >>
>> > > >> *(6) Watermarks across splits and in the Split Enumerator*
>> > > >>
>> > > >>  - The split enumerator may need some watermark awareness, which
>> > should
>> > > be
>> > > >> purely based on split metadata (like create timestamp of file
>> splits)
>> > > >>  - If there are still more splits with overlapping event time range
>> > for
>> > > a
>> > > >> split reader, then that split reader should not advance the
>> watermark
>> > > >> within the split beyond the overlap boundary. Otherwise future
>> splits
>> > > will
>> > > >> produce late data.
>> > > >>
>> > > >>  - One way to approach this could be that the split enumerator may
>> > send
>> > > >> watermarks to the readers, and the readers cannot emit watermarks
>> > beyond
>> > > >> that received watermark.
>> > > >>  - Many split enumerators would simply immediately send Long.MAX
>> out
>> > and
>> > > >> leave the progress purely to the split readers.
>> > > >>
>> > > >>  - For event-time alignment / split back pressure, this begs the
>> > > question
>> > > >> how we can avoid deadlocks that may arise when splits are suspended
>> > for
>> > > >> event time back pressure,
>> > > >>
>> > > >> *(7) Batch and streaming Unification*
>> > > >>
>> > > >>  - Functionality wise, the above design should support both
>> > > >>  - Batch often (mostly) does not care about reading "in order" and
>> > > >> generating watermarks
>> > > >>    --> Might use different enumerator logic that is more locality
>> > aware
>> > > >> and ignores event time order
>> > > >>    --> Does not generate watermarks
>> > > >>  - Would be great if bounded sources could be identified at compile
>> > > time,
>> > > >> so that "env.addBoundedSource(...)" is type safe and can return a
>> > > >> "BoundedDataStream".
>> > > >>  - Possible to defer this discussion until later
>> > > >>
>> > > >> *Miscellaneous Comments*
>> > > >>
>> > > >>  - Should the source have a TypeInformation for the produced type,
>> > > instead
>> > > >> of a serializer? We need a type information in the stream anyways,
>> and
>> > > can
>> > > >> derive the serializer from that. Plus, creating the serializer
>> should
>> > > >> respect the ExecutionConfig.
>> > > >>
>> > > >>  - The TypeSerializer interface is very powerful but also not easy
>> to
>> > > >> implement. Its purpose is to handle data super efficiently, support
>> > > >> flexible ways of evolution, etc.
>> > > >>  For metadata I would suggest to look at the
>> SimpleVersionedSerializer
>> > > >> instead, which is used for example for checkpoint master hooks, or
>> for
>> > > the
>> > > >> streaming file sink. I think that is is a good match for cases
>> where
>> > we
>> > > do
>> > > >> not need more than ser/deser (no copy, etc.) and don't need to push
>> > > >> versioning out of the serialization paths for best performance (as
>> in
>> > > the
>> > > >> TypeSerializer)
>> > > >>
>> > > >>
>> > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>> > > >> k.kloudas@data-artisans.com>
>> > > >> wrote:
>> > > >>
>> > > >>> Hi Biao,
>> > > >>>
>> > > >>> Thanks for the answer!
>> > > >>>
>> > > >>> So given the multi-threaded readers, now we have as open
>> questions:
>> > > >>>
>> > > >>> 1) How do we let the checkpoints pass through our multi-threaded
>> > reader
>> > > >>> operator?
>> > > >>>
>> > > >>> 2) Do we have separate reader and source operators or not? In the
>> > > >> strategy
>> > > >>> that has a separate source, the source operator has a parallelism
>> of
>> > 1
>> > > >> and
>> > > >>> is responsible for split recovery only.
>> > > >>>
>> > > >>> For the first one, given also the constraints (blocking, finite
>> > queues,
>> > > >>> etc), I do not have an answer yet.
>> > > >>>
>> > > >>> For the 2nd, I think that we should go with separate operators for
>> > the
>> > > >>> source and the readers, for the following reasons:
>> > > >>>
>> > > >>> 1) This is more aligned with a potential future improvement where
>> the
>> > > >> split
>> > > >>> discovery becomes a responsibility of the JobManager and readers
>> are
>> > > >>> pooling more work from the JM.
>> > > >>>
>> > > >>> 2) The source is going to be the "single point of truth". It will
>> > know
>> > > >> what
>> > > >>> has been processed and what not. If the source and the readers
>> are a
>> > > >> single
>> > > >>> operator with parallelism > 1, or in general, if the split
>> discovery
>> > is
>> > > >>> done by each task individually, then:
>> > > >>>   i) we have to have a deterministic scheme for each reader to
>> assign
>> > > >>> splits to itself (e.g. mod subtaskId). This is not necessarily
>> > trivial
>> > > >> for
>> > > >>> all sources.
>> > > >>>   ii) each reader would have to keep a copy of all its processed
>> > slpits
>> > > >>>   iii) the state has to be a union state with a non-trivial
>> merging
>> > > >> logic
>> > > >>> in order to support rescaling.
>> > > >>>
>> > > >>> Two additional points that you raised above:
>> > > >>>
>> > > >>> i) The point that you raised that we need to keep all splits
>> > (processed
>> > > >> and
>> > > >>> not-processed) I think is a bit of a strong requirement. This
>> would
>> > > imply
>> > > >>> that for infinite sources the state will grow indefinitely. This
>> is
>> > > >> problem
>> > > >>> is even more pronounced if we do not have a single source that
>> > assigns
>> > > >>> splits to readers, as each reader will have its own copy of the
>> > state.
>> > > >>>
>> > > >>> ii) it is true that for finite sources we need to somehow not
>> close
>> > the
>> > > >>> readers when the source/split discoverer finishes. The
>> > > >>> ContinuousFileReaderOperator has a work-around for that. It is not
>> > > >> elegant,
>> > > >>> and checkpoints are not emitted after closing the source, but
>> this, I
>> > > >>> believe, is a bigger problem which requires more changes than just
>> > > >>> refactoring the source interface.
>> > > >>>
>> > > >>> Cheers,
>> > > >>> Kostas
>> > > >>>
>> > > >>
>> > >
>> > >
>> >
>>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Biao Liu <mm...@gmail.com>.
Hi Steven,
Thank you for the feedback. Please take a look at the document FLIP-27
<https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface>
which
is updated recently. A lot of details of enumerator were added in this
document. I think it would help.

Steven Wu <st...@gmail.com> 于2019年3月28日周四 下午12:52写道:

> This proposal mentioned that SplitEnumerator might run on the JobManager or
> in a single task on a TaskManager.
>
> if enumerator is a single task on a taskmanager, then the job DAG can never
> been embarrassingly parallel anymore. That will nullify the leverage of
> fine-grained recovery for embarrassingly parallel jobs.
>
> It's not clear to me what's the implication of running enumerator on the
> jobmanager. So I will leave that out for now.
>
> On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> wrote:
>
> > Hi Stephan & Piotrek,
> >
> > Thank you for feedback.
> >
> > It seems that there are a lot of things to do in community. I am just
> > afraid that this discussion may be forgotten since there so many
> proposals
> > recently.
> > Anyway, wish to see the split topics soon :)
> >
> > Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四 下午8:21写道:
> >
> > > Hi Biao!
> > >
> > > This discussion was stalled because of preparations for the open
> sourcing
> > > & merging Blink. I think before creating the tickets we should split
> this
> > > discussion into topics/areas outlined by Stephan and create Flips for
> > that.
> > >
> > > I think there is no chance for this to be completed in couple of
> > remaining
> > > weeks/1 month before 1.8 feature freeze, however it would be good to
> aim
> > > with those changes for 1.9.
> > >
> > > Piotrek
> > >
> > > > On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> wrote:
> > > >
> > > > Hi community,
> > > > The summary of Stephan makes a lot sense to me. It is much clearer
> > indeed
> > > > after splitting the complex topic into small ones.
> > > > I was wondering is there any detail plan for next step? If not, I
> would
> > > > like to push this thing forward by creating some JIRA issues.
> > > > Another question is that should version 1.8 include these features?
> > > >
> > > > Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
> > > >
> > > >> Thanks everyone for the lively discussion. Let me try to summarize
> > > where I
> > > >> see convergence in the discussion and open issues.
> > > >> I'll try to group this by design aspect of the source. Please let me
> > > know
> > > >> if I got things wrong or missed something crucial here.
> > > >>
> > > >> For issues 1-3, if the below reflects the state of the discussion, I
> > > would
> > > >> try and update the FLIP in the next days.
> > > >> For the remaining ones we need more discussion.
> > > >>
> > > >> I would suggest to fork each of these aspects into a separate mail
> > > thread,
> > > >> or will loose sight of the individual aspects.
> > > >>
> > > >> *(1) Separation of Split Enumerator and Split Reader*
> > > >>
> > > >>  - All seem to agree this is a good thing
> > > >>  - Split Enumerator could in the end live on JobManager (and assign
> > > splits
> > > >> via RPC) or in a task (and assign splits via data streams)
> > > >>  - this discussion is orthogonal and should come later, when the
> > > interface
> > > >> is agreed upon.
> > > >>
> > > >> *(2) Split Readers for one or more splits*
> > > >>
> > > >>  - Discussion seems to agree that we need to support one reader that
> > > >> possibly handles multiple splits concurrently.
> > > >>  - The requirement comes from sources where one poll()-style call
> > > fetches
> > > >> data from different splits / partitions
> > > >>    --> example sources that require that would be for example Kafka,
> > > >> Pravega, Pulsar
> > > >>
> > > >>  - Could have one split reader per source, or multiple split readers
> > > that
> > > >> share the "poll()" function
> > > >>  - To not make it too complicated, we can start with thinking about
> > one
> > > >> split reader for all splits initially and see if that covers all
> > > >> requirements
> > > >>
> > > >> *(3) Threading model of the Split Reader*
> > > >>
> > > >>  - Most active part of the discussion ;-)
> > > >>
> > > >>  - A non-blocking way for Flink's task code to interact with the
> > source
> > > is
> > > >> needed in order to a task runtime code based on a
> > > >> single-threaded/actor-style task design
> > > >>    --> I personally am a big proponent of that, it will help with
> > > >> well-behaved checkpoints, efficiency, and simpler yet more robust
> > > runtime
> > > >> code
> > > >>
> > > >>  - Users care about simple abstraction, so as a subclass of
> > SplitReader
> > > >> (non-blocking / async) we need to have a BlockingSplitReader which
> > will
> > > >> form the basis of most source implementations. BlockingSplitReader
> > lets
> > > >> users do blocking simple poll() calls.
> > > >>  - The BlockingSplitReader would spawn a thread (or more) and the
> > > >> thread(s) can make blocking calls and hand over data buffers via a
> > > blocking
> > > >> queue
> > > >>  - This should allow us to cover both, a fully async runtime, and a
> > > simple
> > > >> blocking interface for users.
> > > >>  - This is actually very similar to how the Kafka connectors work.
> > Kafka
> > > >> 9+ with one thread, Kafka 8 with multiple threads
> > > >>
> > > >>  - On the base SplitReader (the async one), the non-blocking method
> > that
> > > >> gets the next chunk of data would signal data availability via a
> > > >> CompletableFuture, because that gives the best flexibility (can
> await
> > > >> completion or register notification handlers).
> > > >>  - The source task would register a "thenHandle()" (or similar) on
> the
> > > >> future to put a "take next data" task into the actor-style mailbox
> > > >>
> > > >> *(4) Split Enumeration and Assignment*
> > > >>
> > > >>  - Splits may be generated lazily, both in cases where there is a
> > > limited
> > > >> number of splits (but very many), or splits are discovered over time
> > > >>  - Assignment should also be lazy, to get better load balancing
> > > >>  - Assignment needs support locality preferences
> > > >>
> > > >>  - Possible design based on discussion so far:
> > > >>
> > > >>    --> SplitReader has a method "addSplits(SplitT...)" to add one or
> > > more
> > > >> splits. Some split readers might assume they have only one split
> ever,
> > > >> concurrently, others assume multiple splits. (Note: idea behind
> being
> > > able
> > > >> to add multiple splits at the same time is to ease startup where
> > > multiple
> > > >> splits may be assigned instantly.)
> > > >>    --> SplitReader has a context object on which it can call
> indicate
> > > when
> > > >> splits are completed. The enumerator gets that notification and can
> > use
> > > to
> > > >> decide when to assign new splits. This should help both in cases of
> > > sources
> > > >> that take splits lazily (file readers) and in case the source needs
> to
> > > >> preserve a partial order between splits (Kinesis, Pravega, Pulsar
> may
> > > need
> > > >> that).
> > > >>    --> SplitEnumerator gets notification when SplitReaders start and
> > > when
> > > >> they finish splits. They can decide at that moment to push more
> splits
> > > to
> > > >> that reader
> > > >>    --> The SplitEnumerator should probably be aware of the source
> > > >> parallelism, to build its initial distribution.
> > > >>
> > > >>  - Open question: Should the source expose something like "host
> > > >> preferences", so that yarn/mesos/k8s can take this into account when
> > > >> selecting a node to start a TM on?
> > > >>
> > > >> *(5) Watermarks and event time alignment*
> > > >>
> > > >>  - Watermark generation, as well as idleness, needs to be per split
> > > (like
> > > >> currently in the Kafka Source, per partition)
> > > >>  - It is desirable to support optional event-time-alignment, meaning
> > > that
> > > >> splits that are ahead are back-pressured or temporarily unsubscribed
> > > >>
> > > >>  - I think i would be desirable to encapsulate watermark generation
> > > logic
> > > >> in watermark generators, for a separation of concerns. The watermark
> > > >> generators should run per split.
> > > >>  - Using watermark generators would also help with another problem
> of
> > > the
> > > >> suggested interface, namely supporting non-periodic watermarks
> > > efficiently.
> > > >>
> > > >>  - Need a way to "dispatch" next record to different watermark
> > > generators
> > > >>  - Need a way to tell SplitReader to "suspend" a split until a
> certain
> > > >> watermark is reached (event time backpressure)
> > > >>  - This would in fact be not needed (and thus simpler) if we had a
> > > >> SplitReader per split and may be a reason to re-open that discussion
> > > >>
> > > >> *(6) Watermarks across splits and in the Split Enumerator*
> > > >>
> > > >>  - The split enumerator may need some watermark awareness, which
> > should
> > > be
> > > >> purely based on split metadata (like create timestamp of file
> splits)
> > > >>  - If there are still more splits with overlapping event time range
> > for
> > > a
> > > >> split reader, then that split reader should not advance the
> watermark
> > > >> within the split beyond the overlap boundary. Otherwise future
> splits
> > > will
> > > >> produce late data.
> > > >>
> > > >>  - One way to approach this could be that the split enumerator may
> > send
> > > >> watermarks to the readers, and the readers cannot emit watermarks
> > beyond
> > > >> that received watermark.
> > > >>  - Many split enumerators would simply immediately send Long.MAX out
> > and
> > > >> leave the progress purely to the split readers.
> > > >>
> > > >>  - For event-time alignment / split back pressure, this begs the
> > > question
> > > >> how we can avoid deadlocks that may arise when splits are suspended
> > for
> > > >> event time back pressure,
> > > >>
> > > >> *(7) Batch and streaming Unification*
> > > >>
> > > >>  - Functionality wise, the above design should support both
> > > >>  - Batch often (mostly) does not care about reading "in order" and
> > > >> generating watermarks
> > > >>    --> Might use different enumerator logic that is more locality
> > aware
> > > >> and ignores event time order
> > > >>    --> Does not generate watermarks
> > > >>  - Would be great if bounded sources could be identified at compile
> > > time,
> > > >> so that "env.addBoundedSource(...)" is type safe and can return a
> > > >> "BoundedDataStream".
> > > >>  - Possible to defer this discussion until later
> > > >>
> > > >> *Miscellaneous Comments*
> > > >>
> > > >>  - Should the source have a TypeInformation for the produced type,
> > > instead
> > > >> of a serializer? We need a type information in the stream anyways,
> and
> > > can
> > > >> derive the serializer from that. Plus, creating the serializer
> should
> > > >> respect the ExecutionConfig.
> > > >>
> > > >>  - The TypeSerializer interface is very powerful but also not easy
> to
> > > >> implement. Its purpose is to handle data super efficiently, support
> > > >> flexible ways of evolution, etc.
> > > >>  For metadata I would suggest to look at the
> SimpleVersionedSerializer
> > > >> instead, which is used for example for checkpoint master hooks, or
> for
> > > the
> > > >> streaming file sink. I think that is is a good match for cases where
> > we
> > > do
> > > >> not need more than ser/deser (no copy, etc.) and don't need to push
> > > >> versioning out of the serialization paths for best performance (as
> in
> > > the
> > > >> TypeSerializer)
> > > >>
> > > >>
> > > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > > >> k.kloudas@data-artisans.com>
> > > >> wrote:
> > > >>
> > > >>> Hi Biao,
> > > >>>
> > > >>> Thanks for the answer!
> > > >>>
> > > >>> So given the multi-threaded readers, now we have as open questions:
> > > >>>
> > > >>> 1) How do we let the checkpoints pass through our multi-threaded
> > reader
> > > >>> operator?
> > > >>>
> > > >>> 2) Do we have separate reader and source operators or not? In the
> > > >> strategy
> > > >>> that has a separate source, the source operator has a parallelism
> of
> > 1
> > > >> and
> > > >>> is responsible for split recovery only.
> > > >>>
> > > >>> For the first one, given also the constraints (blocking, finite
> > queues,
> > > >>> etc), I do not have an answer yet.
> > > >>>
> > > >>> For the 2nd, I think that we should go with separate operators for
> > the
> > > >>> source and the readers, for the following reasons:
> > > >>>
> > > >>> 1) This is more aligned with a potential future improvement where
> the
> > > >> split
> > > >>> discovery becomes a responsibility of the JobManager and readers
> are
> > > >>> pooling more work from the JM.
> > > >>>
> > > >>> 2) The source is going to be the "single point of truth". It will
> > know
> > > >> what
> > > >>> has been processed and what not. If the source and the readers are
> a
> > > >> single
> > > >>> operator with parallelism > 1, or in general, if the split
> discovery
> > is
> > > >>> done by each task individually, then:
> > > >>>   i) we have to have a deterministic scheme for each reader to
> assign
> > > >>> splits to itself (e.g. mod subtaskId). This is not necessarily
> > trivial
> > > >> for
> > > >>> all sources.
> > > >>>   ii) each reader would have to keep a copy of all its processed
> > slpits
> > > >>>   iii) the state has to be a union state with a non-trivial merging
> > > >> logic
> > > >>> in order to support rescaling.
> > > >>>
> > > >>> Two additional points that you raised above:
> > > >>>
> > > >>> i) The point that you raised that we need to keep all splits
> > (processed
> > > >> and
> > > >>> not-processed) I think is a bit of a strong requirement. This would
> > > imply
> > > >>> that for infinite sources the state will grow indefinitely. This is
> > > >> problem
> > > >>> is even more pronounced if we do not have a single source that
> > assigns
> > > >>> splits to readers, as each reader will have its own copy of the
> > state.
> > > >>>
> > > >>> ii) it is true that for finite sources we need to somehow not close
> > the
> > > >>> readers when the source/split discoverer finishes. The
> > > >>> ContinuousFileReaderOperator has a work-around for that. It is not
> > > >> elegant,
> > > >>> and checkpoints are not emitted after closing the source, but
> this, I
> > > >>> believe, is a bigger problem which requires more changes than just
> > > >>> refactoring the source interface.
> > > >>>
> > > >>> Cheers,
> > > >>> Kostas
> > > >>>
> > > >>
> > >
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Steven Wu <st...@gmail.com>.
This proposal mentioned that SplitEnumerator might run on the JobManager or
in a single task on a TaskManager.

if enumerator is a single task on a taskmanager, then the job DAG can never
been embarrassingly parallel anymore. That will nullify the leverage of
fine-grained recovery for embarrassingly parallel jobs.

It's not clear to me what's the implication of running enumerator on the
jobmanager. So I will leave that out for now.

On Mon, Jan 28, 2019 at 3:05 AM Biao Liu <mm...@gmail.com> wrote:

> Hi Stephan & Piotrek,
>
> Thank you for feedback.
>
> It seems that there are a lot of things to do in community. I am just
> afraid that this discussion may be forgotten since there so many proposals
> recently.
> Anyway, wish to see the split topics soon :)
>
> Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四 下午8:21写道:
>
> > Hi Biao!
> >
> > This discussion was stalled because of preparations for the open sourcing
> > & merging Blink. I think before creating the tickets we should split this
> > discussion into topics/areas outlined by Stephan and create Flips for
> that.
> >
> > I think there is no chance for this to be completed in couple of
> remaining
> > weeks/1 month before 1.8 feature freeze, however it would be good to aim
> > with those changes for 1.9.
> >
> > Piotrek
> >
> > > On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> wrote:
> > >
> > > Hi community,
> > > The summary of Stephan makes a lot sense to me. It is much clearer
> indeed
> > > after splitting the complex topic into small ones.
> > > I was wondering is there any detail plan for next step? If not, I would
> > > like to push this thing forward by creating some JIRA issues.
> > > Another question is that should version 1.8 include these features?
> > >
> > > Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
> > >
> > >> Thanks everyone for the lively discussion. Let me try to summarize
> > where I
> > >> see convergence in the discussion and open issues.
> > >> I'll try to group this by design aspect of the source. Please let me
> > know
> > >> if I got things wrong or missed something crucial here.
> > >>
> > >> For issues 1-3, if the below reflects the state of the discussion, I
> > would
> > >> try and update the FLIP in the next days.
> > >> For the remaining ones we need more discussion.
> > >>
> > >> I would suggest to fork each of these aspects into a separate mail
> > thread,
> > >> or will loose sight of the individual aspects.
> > >>
> > >> *(1) Separation of Split Enumerator and Split Reader*
> > >>
> > >>  - All seem to agree this is a good thing
> > >>  - Split Enumerator could in the end live on JobManager (and assign
> > splits
> > >> via RPC) or in a task (and assign splits via data streams)
> > >>  - this discussion is orthogonal and should come later, when the
> > interface
> > >> is agreed upon.
> > >>
> > >> *(2) Split Readers for one or more splits*
> > >>
> > >>  - Discussion seems to agree that we need to support one reader that
> > >> possibly handles multiple splits concurrently.
> > >>  - The requirement comes from sources where one poll()-style call
> > fetches
> > >> data from different splits / partitions
> > >>    --> example sources that require that would be for example Kafka,
> > >> Pravega, Pulsar
> > >>
> > >>  - Could have one split reader per source, or multiple split readers
> > that
> > >> share the "poll()" function
> > >>  - To not make it too complicated, we can start with thinking about
> one
> > >> split reader for all splits initially and see if that covers all
> > >> requirements
> > >>
> > >> *(3) Threading model of the Split Reader*
> > >>
> > >>  - Most active part of the discussion ;-)
> > >>
> > >>  - A non-blocking way for Flink's task code to interact with the
> source
> > is
> > >> needed in order to a task runtime code based on a
> > >> single-threaded/actor-style task design
> > >>    --> I personally am a big proponent of that, it will help with
> > >> well-behaved checkpoints, efficiency, and simpler yet more robust
> > runtime
> > >> code
> > >>
> > >>  - Users care about simple abstraction, so as a subclass of
> SplitReader
> > >> (non-blocking / async) we need to have a BlockingSplitReader which
> will
> > >> form the basis of most source implementations. BlockingSplitReader
> lets
> > >> users do blocking simple poll() calls.
> > >>  - The BlockingSplitReader would spawn a thread (or more) and the
> > >> thread(s) can make blocking calls and hand over data buffers via a
> > blocking
> > >> queue
> > >>  - This should allow us to cover both, a fully async runtime, and a
> > simple
> > >> blocking interface for users.
> > >>  - This is actually very similar to how the Kafka connectors work.
> Kafka
> > >> 9+ with one thread, Kafka 8 with multiple threads
> > >>
> > >>  - On the base SplitReader (the async one), the non-blocking method
> that
> > >> gets the next chunk of data would signal data availability via a
> > >> CompletableFuture, because that gives the best flexibility (can await
> > >> completion or register notification handlers).
> > >>  - The source task would register a "thenHandle()" (or similar) on the
> > >> future to put a "take next data" task into the actor-style mailbox
> > >>
> > >> *(4) Split Enumeration and Assignment*
> > >>
> > >>  - Splits may be generated lazily, both in cases where there is a
> > limited
> > >> number of splits (but very many), or splits are discovered over time
> > >>  - Assignment should also be lazy, to get better load balancing
> > >>  - Assignment needs support locality preferences
> > >>
> > >>  - Possible design based on discussion so far:
> > >>
> > >>    --> SplitReader has a method "addSplits(SplitT...)" to add one or
> > more
> > >> splits. Some split readers might assume they have only one split ever,
> > >> concurrently, others assume multiple splits. (Note: idea behind being
> > able
> > >> to add multiple splits at the same time is to ease startup where
> > multiple
> > >> splits may be assigned instantly.)
> > >>    --> SplitReader has a context object on which it can call indicate
> > when
> > >> splits are completed. The enumerator gets that notification and can
> use
> > to
> > >> decide when to assign new splits. This should help both in cases of
> > sources
> > >> that take splits lazily (file readers) and in case the source needs to
> > >> preserve a partial order between splits (Kinesis, Pravega, Pulsar may
> > need
> > >> that).
> > >>    --> SplitEnumerator gets notification when SplitReaders start and
> > when
> > >> they finish splits. They can decide at that moment to push more splits
> > to
> > >> that reader
> > >>    --> The SplitEnumerator should probably be aware of the source
> > >> parallelism, to build its initial distribution.
> > >>
> > >>  - Open question: Should the source expose something like "host
> > >> preferences", so that yarn/mesos/k8s can take this into account when
> > >> selecting a node to start a TM on?
> > >>
> > >> *(5) Watermarks and event time alignment*
> > >>
> > >>  - Watermark generation, as well as idleness, needs to be per split
> > (like
> > >> currently in the Kafka Source, per partition)
> > >>  - It is desirable to support optional event-time-alignment, meaning
> > that
> > >> splits that are ahead are back-pressured or temporarily unsubscribed
> > >>
> > >>  - I think i would be desirable to encapsulate watermark generation
> > logic
> > >> in watermark generators, for a separation of concerns. The watermark
> > >> generators should run per split.
> > >>  - Using watermark generators would also help with another problem of
> > the
> > >> suggested interface, namely supporting non-periodic watermarks
> > efficiently.
> > >>
> > >>  - Need a way to "dispatch" next record to different watermark
> > generators
> > >>  - Need a way to tell SplitReader to "suspend" a split until a certain
> > >> watermark is reached (event time backpressure)
> > >>  - This would in fact be not needed (and thus simpler) if we had a
> > >> SplitReader per split and may be a reason to re-open that discussion
> > >>
> > >> *(6) Watermarks across splits and in the Split Enumerator*
> > >>
> > >>  - The split enumerator may need some watermark awareness, which
> should
> > be
> > >> purely based on split metadata (like create timestamp of file splits)
> > >>  - If there are still more splits with overlapping event time range
> for
> > a
> > >> split reader, then that split reader should not advance the watermark
> > >> within the split beyond the overlap boundary. Otherwise future splits
> > will
> > >> produce late data.
> > >>
> > >>  - One way to approach this could be that the split enumerator may
> send
> > >> watermarks to the readers, and the readers cannot emit watermarks
> beyond
> > >> that received watermark.
> > >>  - Many split enumerators would simply immediately send Long.MAX out
> and
> > >> leave the progress purely to the split readers.
> > >>
> > >>  - For event-time alignment / split back pressure, this begs the
> > question
> > >> how we can avoid deadlocks that may arise when splits are suspended
> for
> > >> event time back pressure,
> > >>
> > >> *(7) Batch and streaming Unification*
> > >>
> > >>  - Functionality wise, the above design should support both
> > >>  - Batch often (mostly) does not care about reading "in order" and
> > >> generating watermarks
> > >>    --> Might use different enumerator logic that is more locality
> aware
> > >> and ignores event time order
> > >>    --> Does not generate watermarks
> > >>  - Would be great if bounded sources could be identified at compile
> > time,
> > >> so that "env.addBoundedSource(...)" is type safe and can return a
> > >> "BoundedDataStream".
> > >>  - Possible to defer this discussion until later
> > >>
> > >> *Miscellaneous Comments*
> > >>
> > >>  - Should the source have a TypeInformation for the produced type,
> > instead
> > >> of a serializer? We need a type information in the stream anyways, and
> > can
> > >> derive the serializer from that. Plus, creating the serializer should
> > >> respect the ExecutionConfig.
> > >>
> > >>  - The TypeSerializer interface is very powerful but also not easy to
> > >> implement. Its purpose is to handle data super efficiently, support
> > >> flexible ways of evolution, etc.
> > >>  For metadata I would suggest to look at the SimpleVersionedSerializer
> > >> instead, which is used for example for checkpoint master hooks, or for
> > the
> > >> streaming file sink. I think that is is a good match for cases where
> we
> > do
> > >> not need more than ser/deser (no copy, etc.) and don't need to push
> > >> versioning out of the serialization paths for best performance (as in
> > the
> > >> TypeSerializer)
> > >>
> > >>
> > >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > >> k.kloudas@data-artisans.com>
> > >> wrote:
> > >>
> > >>> Hi Biao,
> > >>>
> > >>> Thanks for the answer!
> > >>>
> > >>> So given the multi-threaded readers, now we have as open questions:
> > >>>
> > >>> 1) How do we let the checkpoints pass through our multi-threaded
> reader
> > >>> operator?
> > >>>
> > >>> 2) Do we have separate reader and source operators or not? In the
> > >> strategy
> > >>> that has a separate source, the source operator has a parallelism of
> 1
> > >> and
> > >>> is responsible for split recovery only.
> > >>>
> > >>> For the first one, given also the constraints (blocking, finite
> queues,
> > >>> etc), I do not have an answer yet.
> > >>>
> > >>> For the 2nd, I think that we should go with separate operators for
> the
> > >>> source and the readers, for the following reasons:
> > >>>
> > >>> 1) This is more aligned with a potential future improvement where the
> > >> split
> > >>> discovery becomes a responsibility of the JobManager and readers are
> > >>> pooling more work from the JM.
> > >>>
> > >>> 2) The source is going to be the "single point of truth". It will
> know
> > >> what
> > >>> has been processed and what not. If the source and the readers are a
> > >> single
> > >>> operator with parallelism > 1, or in general, if the split discovery
> is
> > >>> done by each task individually, then:
> > >>>   i) we have to have a deterministic scheme for each reader to assign
> > >>> splits to itself (e.g. mod subtaskId). This is not necessarily
> trivial
> > >> for
> > >>> all sources.
> > >>>   ii) each reader would have to keep a copy of all its processed
> slpits
> > >>>   iii) the state has to be a union state with a non-trivial merging
> > >> logic
> > >>> in order to support rescaling.
> > >>>
> > >>> Two additional points that you raised above:
> > >>>
> > >>> i) The point that you raised that we need to keep all splits
> (processed
> > >> and
> > >>> not-processed) I think is a bit of a strong requirement. This would
> > imply
> > >>> that for infinite sources the state will grow indefinitely. This is
> > >> problem
> > >>> is even more pronounced if we do not have a single source that
> assigns
> > >>> splits to readers, as each reader will have its own copy of the
> state.
> > >>>
> > >>> ii) it is true that for finite sources we need to somehow not close
> the
> > >>> readers when the source/split discoverer finishes. The
> > >>> ContinuousFileReaderOperator has a work-around for that. It is not
> > >> elegant,
> > >>> and checkpoints are not emitted after closing the source, but this, I
> > >>> believe, is a bigger problem which requires more changes than just
> > >>> refactoring the source interface.
> > >>>
> > >>> Cheers,
> > >>> Kostas
> > >>>
> > >>
> >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Biao Liu <mm...@gmail.com>.
Hi Stephan & Piotrek,

Thank you for feedback.

It seems that there are a lot of things to do in community. I am just
afraid that this discussion may be forgotten since there so many proposals
recently.
Anyway, wish to see the split topics soon :)

Piotr Nowojski <pi...@da-platform.com> 于2019年1月24日周四 下午8:21写道:

> Hi Biao!
>
> This discussion was stalled because of preparations for the open sourcing
> & merging Blink. I think before creating the tickets we should split this
> discussion into topics/areas outlined by Stephan and create Flips for that.
>
> I think there is no chance for this to be completed in couple of remaining
> weeks/1 month before 1.8 feature freeze, however it would be good to aim
> with those changes for 1.9.
>
> Piotrek
>
> > On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> wrote:
> >
> > Hi community,
> > The summary of Stephan makes a lot sense to me. It is much clearer indeed
> > after splitting the complex topic into small ones.
> > I was wondering is there any detail plan for next step? If not, I would
> > like to push this thing forward by creating some JIRA issues.
> > Another question is that should version 1.8 include these features?
> >
> > Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
> >
> >> Thanks everyone for the lively discussion. Let me try to summarize
> where I
> >> see convergence in the discussion and open issues.
> >> I'll try to group this by design aspect of the source. Please let me
> know
> >> if I got things wrong or missed something crucial here.
> >>
> >> For issues 1-3, if the below reflects the state of the discussion, I
> would
> >> try and update the FLIP in the next days.
> >> For the remaining ones we need more discussion.
> >>
> >> I would suggest to fork each of these aspects into a separate mail
> thread,
> >> or will loose sight of the individual aspects.
> >>
> >> *(1) Separation of Split Enumerator and Split Reader*
> >>
> >>  - All seem to agree this is a good thing
> >>  - Split Enumerator could in the end live on JobManager (and assign
> splits
> >> via RPC) or in a task (and assign splits via data streams)
> >>  - this discussion is orthogonal and should come later, when the
> interface
> >> is agreed upon.
> >>
> >> *(2) Split Readers for one or more splits*
> >>
> >>  - Discussion seems to agree that we need to support one reader that
> >> possibly handles multiple splits concurrently.
> >>  - The requirement comes from sources where one poll()-style call
> fetches
> >> data from different splits / partitions
> >>    --> example sources that require that would be for example Kafka,
> >> Pravega, Pulsar
> >>
> >>  - Could have one split reader per source, or multiple split readers
> that
> >> share the "poll()" function
> >>  - To not make it too complicated, we can start with thinking about one
> >> split reader for all splits initially and see if that covers all
> >> requirements
> >>
> >> *(3) Threading model of the Split Reader*
> >>
> >>  - Most active part of the discussion ;-)
> >>
> >>  - A non-blocking way for Flink's task code to interact with the source
> is
> >> needed in order to a task runtime code based on a
> >> single-threaded/actor-style task design
> >>    --> I personally am a big proponent of that, it will help with
> >> well-behaved checkpoints, efficiency, and simpler yet more robust
> runtime
> >> code
> >>
> >>  - Users care about simple abstraction, so as a subclass of SplitReader
> >> (non-blocking / async) we need to have a BlockingSplitReader which will
> >> form the basis of most source implementations. BlockingSplitReader lets
> >> users do blocking simple poll() calls.
> >>  - The BlockingSplitReader would spawn a thread (or more) and the
> >> thread(s) can make blocking calls and hand over data buffers via a
> blocking
> >> queue
> >>  - This should allow us to cover both, a fully async runtime, and a
> simple
> >> blocking interface for users.
> >>  - This is actually very similar to how the Kafka connectors work. Kafka
> >> 9+ with one thread, Kafka 8 with multiple threads
> >>
> >>  - On the base SplitReader (the async one), the non-blocking method that
> >> gets the next chunk of data would signal data availability via a
> >> CompletableFuture, because that gives the best flexibility (can await
> >> completion or register notification handlers).
> >>  - The source task would register a "thenHandle()" (or similar) on the
> >> future to put a "take next data" task into the actor-style mailbox
> >>
> >> *(4) Split Enumeration and Assignment*
> >>
> >>  - Splits may be generated lazily, both in cases where there is a
> limited
> >> number of splits (but very many), or splits are discovered over time
> >>  - Assignment should also be lazy, to get better load balancing
> >>  - Assignment needs support locality preferences
> >>
> >>  - Possible design based on discussion so far:
> >>
> >>    --> SplitReader has a method "addSplits(SplitT...)" to add one or
> more
> >> splits. Some split readers might assume they have only one split ever,
> >> concurrently, others assume multiple splits. (Note: idea behind being
> able
> >> to add multiple splits at the same time is to ease startup where
> multiple
> >> splits may be assigned instantly.)
> >>    --> SplitReader has a context object on which it can call indicate
> when
> >> splits are completed. The enumerator gets that notification and can use
> to
> >> decide when to assign new splits. This should help both in cases of
> sources
> >> that take splits lazily (file readers) and in case the source needs to
> >> preserve a partial order between splits (Kinesis, Pravega, Pulsar may
> need
> >> that).
> >>    --> SplitEnumerator gets notification when SplitReaders start and
> when
> >> they finish splits. They can decide at that moment to push more splits
> to
> >> that reader
> >>    --> The SplitEnumerator should probably be aware of the source
> >> parallelism, to build its initial distribution.
> >>
> >>  - Open question: Should the source expose something like "host
> >> preferences", so that yarn/mesos/k8s can take this into account when
> >> selecting a node to start a TM on?
> >>
> >> *(5) Watermarks and event time alignment*
> >>
> >>  - Watermark generation, as well as idleness, needs to be per split
> (like
> >> currently in the Kafka Source, per partition)
> >>  - It is desirable to support optional event-time-alignment, meaning
> that
> >> splits that are ahead are back-pressured or temporarily unsubscribed
> >>
> >>  - I think i would be desirable to encapsulate watermark generation
> logic
> >> in watermark generators, for a separation of concerns. The watermark
> >> generators should run per split.
> >>  - Using watermark generators would also help with another problem of
> the
> >> suggested interface, namely supporting non-periodic watermarks
> efficiently.
> >>
> >>  - Need a way to "dispatch" next record to different watermark
> generators
> >>  - Need a way to tell SplitReader to "suspend" a split until a certain
> >> watermark is reached (event time backpressure)
> >>  - This would in fact be not needed (and thus simpler) if we had a
> >> SplitReader per split and may be a reason to re-open that discussion
> >>
> >> *(6) Watermarks across splits and in the Split Enumerator*
> >>
> >>  - The split enumerator may need some watermark awareness, which should
> be
> >> purely based on split metadata (like create timestamp of file splits)
> >>  - If there are still more splits with overlapping event time range for
> a
> >> split reader, then that split reader should not advance the watermark
> >> within the split beyond the overlap boundary. Otherwise future splits
> will
> >> produce late data.
> >>
> >>  - One way to approach this could be that the split enumerator may send
> >> watermarks to the readers, and the readers cannot emit watermarks beyond
> >> that received watermark.
> >>  - Many split enumerators would simply immediately send Long.MAX out and
> >> leave the progress purely to the split readers.
> >>
> >>  - For event-time alignment / split back pressure, this begs the
> question
> >> how we can avoid deadlocks that may arise when splits are suspended for
> >> event time back pressure,
> >>
> >> *(7) Batch and streaming Unification*
> >>
> >>  - Functionality wise, the above design should support both
> >>  - Batch often (mostly) does not care about reading "in order" and
> >> generating watermarks
> >>    --> Might use different enumerator logic that is more locality aware
> >> and ignores event time order
> >>    --> Does not generate watermarks
> >>  - Would be great if bounded sources could be identified at compile
> time,
> >> so that "env.addBoundedSource(...)" is type safe and can return a
> >> "BoundedDataStream".
> >>  - Possible to defer this discussion until later
> >>
> >> *Miscellaneous Comments*
> >>
> >>  - Should the source have a TypeInformation for the produced type,
> instead
> >> of a serializer? We need a type information in the stream anyways, and
> can
> >> derive the serializer from that. Plus, creating the serializer should
> >> respect the ExecutionConfig.
> >>
> >>  - The TypeSerializer interface is very powerful but also not easy to
> >> implement. Its purpose is to handle data super efficiently, support
> >> flexible ways of evolution, etc.
> >>  For metadata I would suggest to look at the SimpleVersionedSerializer
> >> instead, which is used for example for checkpoint master hooks, or for
> the
> >> streaming file sink. I think that is is a good match for cases where we
> do
> >> not need more than ser/deser (no copy, etc.) and don't need to push
> >> versioning out of the serialization paths for best performance (as in
> the
> >> TypeSerializer)
> >>
> >>
> >> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> >> k.kloudas@data-artisans.com>
> >> wrote:
> >>
> >>> Hi Biao,
> >>>
> >>> Thanks for the answer!
> >>>
> >>> So given the multi-threaded readers, now we have as open questions:
> >>>
> >>> 1) How do we let the checkpoints pass through our multi-threaded reader
> >>> operator?
> >>>
> >>> 2) Do we have separate reader and source operators or not? In the
> >> strategy
> >>> that has a separate source, the source operator has a parallelism of 1
> >> and
> >>> is responsible for split recovery only.
> >>>
> >>> For the first one, given also the constraints (blocking, finite queues,
> >>> etc), I do not have an answer yet.
> >>>
> >>> For the 2nd, I think that we should go with separate operators for the
> >>> source and the readers, for the following reasons:
> >>>
> >>> 1) This is more aligned with a potential future improvement where the
> >> split
> >>> discovery becomes a responsibility of the JobManager and readers are
> >>> pooling more work from the JM.
> >>>
> >>> 2) The source is going to be the "single point of truth". It will know
> >> what
> >>> has been processed and what not. If the source and the readers are a
> >> single
> >>> operator with parallelism > 1, or in general, if the split discovery is
> >>> done by each task individually, then:
> >>>   i) we have to have a deterministic scheme for each reader to assign
> >>> splits to itself (e.g. mod subtaskId). This is not necessarily trivial
> >> for
> >>> all sources.
> >>>   ii) each reader would have to keep a copy of all its processed slpits
> >>>   iii) the state has to be a union state with a non-trivial merging
> >> logic
> >>> in order to support rescaling.
> >>>
> >>> Two additional points that you raised above:
> >>>
> >>> i) The point that you raised that we need to keep all splits (processed
> >> and
> >>> not-processed) I think is a bit of a strong requirement. This would
> imply
> >>> that for infinite sources the state will grow indefinitely. This is
> >> problem
> >>> is even more pronounced if we do not have a single source that assigns
> >>> splits to readers, as each reader will have its own copy of the state.
> >>>
> >>> ii) it is true that for finite sources we need to somehow not close the
> >>> readers when the source/split discoverer finishes. The
> >>> ContinuousFileReaderOperator has a work-around for that. It is not
> >> elegant,
> >>> and checkpoints are not emitted after closing the source, but this, I
> >>> believe, is a bigger problem which requires more changes than just
> >>> refactoring the source interface.
> >>>
> >>> Cheers,
> >>> Kostas
> >>>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@da-platform.com>.
Hi Biao!

This discussion was stalled because of preparations for the open sourcing & merging Blink. I think before creating the tickets we should split this discussion into topics/areas outlined by Stephan and create Flips for that.

I think there is no chance for this to be completed in couple of remaining weeks/1 month before 1.8 feature freeze, however it would be good to aim with those changes for 1.9.

Piotrek 

> On 20 Jan 2019, at 16:08, Biao Liu <mm...@gmail.com> wrote:
> 
> Hi community,
> The summary of Stephan makes a lot sense to me. It is much clearer indeed
> after splitting the complex topic into small ones.
> I was wondering is there any detail plan for next step? If not, I would
> like to push this thing forward by creating some JIRA issues.
> Another question is that should version 1.8 include these features?
> 
> Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
> 
>> Thanks everyone for the lively discussion. Let me try to summarize where I
>> see convergence in the discussion and open issues.
>> I'll try to group this by design aspect of the source. Please let me know
>> if I got things wrong or missed something crucial here.
>> 
>> For issues 1-3, if the below reflects the state of the discussion, I would
>> try and update the FLIP in the next days.
>> For the remaining ones we need more discussion.
>> 
>> I would suggest to fork each of these aspects into a separate mail thread,
>> or will loose sight of the individual aspects.
>> 
>> *(1) Separation of Split Enumerator and Split Reader*
>> 
>>  - All seem to agree this is a good thing
>>  - Split Enumerator could in the end live on JobManager (and assign splits
>> via RPC) or in a task (and assign splits via data streams)
>>  - this discussion is orthogonal and should come later, when the interface
>> is agreed upon.
>> 
>> *(2) Split Readers for one or more splits*
>> 
>>  - Discussion seems to agree that we need to support one reader that
>> possibly handles multiple splits concurrently.
>>  - The requirement comes from sources where one poll()-style call fetches
>> data from different splits / partitions
>>    --> example sources that require that would be for example Kafka,
>> Pravega, Pulsar
>> 
>>  - Could have one split reader per source, or multiple split readers that
>> share the "poll()" function
>>  - To not make it too complicated, we can start with thinking about one
>> split reader for all splits initially and see if that covers all
>> requirements
>> 
>> *(3) Threading model of the Split Reader*
>> 
>>  - Most active part of the discussion ;-)
>> 
>>  - A non-blocking way for Flink's task code to interact with the source is
>> needed in order to a task runtime code based on a
>> single-threaded/actor-style task design
>>    --> I personally am a big proponent of that, it will help with
>> well-behaved checkpoints, efficiency, and simpler yet more robust runtime
>> code
>> 
>>  - Users care about simple abstraction, so as a subclass of SplitReader
>> (non-blocking / async) we need to have a BlockingSplitReader which will
>> form the basis of most source implementations. BlockingSplitReader lets
>> users do blocking simple poll() calls.
>>  - The BlockingSplitReader would spawn a thread (or more) and the
>> thread(s) can make blocking calls and hand over data buffers via a blocking
>> queue
>>  - This should allow us to cover both, a fully async runtime, and a simple
>> blocking interface for users.
>>  - This is actually very similar to how the Kafka connectors work. Kafka
>> 9+ with one thread, Kafka 8 with multiple threads
>> 
>>  - On the base SplitReader (the async one), the non-blocking method that
>> gets the next chunk of data would signal data availability via a
>> CompletableFuture, because that gives the best flexibility (can await
>> completion or register notification handlers).
>>  - The source task would register a "thenHandle()" (or similar) on the
>> future to put a "take next data" task into the actor-style mailbox
>> 
>> *(4) Split Enumeration and Assignment*
>> 
>>  - Splits may be generated lazily, both in cases where there is a limited
>> number of splits (but very many), or splits are discovered over time
>>  - Assignment should also be lazy, to get better load balancing
>>  - Assignment needs support locality preferences
>> 
>>  - Possible design based on discussion so far:
>> 
>>    --> SplitReader has a method "addSplits(SplitT...)" to add one or more
>> splits. Some split readers might assume they have only one split ever,
>> concurrently, others assume multiple splits. (Note: idea behind being able
>> to add multiple splits at the same time is to ease startup where multiple
>> splits may be assigned instantly.)
>>    --> SplitReader has a context object on which it can call indicate when
>> splits are completed. The enumerator gets that notification and can use to
>> decide when to assign new splits. This should help both in cases of sources
>> that take splits lazily (file readers) and in case the source needs to
>> preserve a partial order between splits (Kinesis, Pravega, Pulsar may need
>> that).
>>    --> SplitEnumerator gets notification when SplitReaders start and when
>> they finish splits. They can decide at that moment to push more splits to
>> that reader
>>    --> The SplitEnumerator should probably be aware of the source
>> parallelism, to build its initial distribution.
>> 
>>  - Open question: Should the source expose something like "host
>> preferences", so that yarn/mesos/k8s can take this into account when
>> selecting a node to start a TM on?
>> 
>> *(5) Watermarks and event time alignment*
>> 
>>  - Watermark generation, as well as idleness, needs to be per split (like
>> currently in the Kafka Source, per partition)
>>  - It is desirable to support optional event-time-alignment, meaning that
>> splits that are ahead are back-pressured or temporarily unsubscribed
>> 
>>  - I think i would be desirable to encapsulate watermark generation logic
>> in watermark generators, for a separation of concerns. The watermark
>> generators should run per split.
>>  - Using watermark generators would also help with another problem of the
>> suggested interface, namely supporting non-periodic watermarks efficiently.
>> 
>>  - Need a way to "dispatch" next record to different watermark generators
>>  - Need a way to tell SplitReader to "suspend" a split until a certain
>> watermark is reached (event time backpressure)
>>  - This would in fact be not needed (and thus simpler) if we had a
>> SplitReader per split and may be a reason to re-open that discussion
>> 
>> *(6) Watermarks across splits and in the Split Enumerator*
>> 
>>  - The split enumerator may need some watermark awareness, which should be
>> purely based on split metadata (like create timestamp of file splits)
>>  - If there are still more splits with overlapping event time range for a
>> split reader, then that split reader should not advance the watermark
>> within the split beyond the overlap boundary. Otherwise future splits will
>> produce late data.
>> 
>>  - One way to approach this could be that the split enumerator may send
>> watermarks to the readers, and the readers cannot emit watermarks beyond
>> that received watermark.
>>  - Many split enumerators would simply immediately send Long.MAX out and
>> leave the progress purely to the split readers.
>> 
>>  - For event-time alignment / split back pressure, this begs the question
>> how we can avoid deadlocks that may arise when splits are suspended for
>> event time back pressure,
>> 
>> *(7) Batch and streaming Unification*
>> 
>>  - Functionality wise, the above design should support both
>>  - Batch often (mostly) does not care about reading "in order" and
>> generating watermarks
>>    --> Might use different enumerator logic that is more locality aware
>> and ignores event time order
>>    --> Does not generate watermarks
>>  - Would be great if bounded sources could be identified at compile time,
>> so that "env.addBoundedSource(...)" is type safe and can return a
>> "BoundedDataStream".
>>  - Possible to defer this discussion until later
>> 
>> *Miscellaneous Comments*
>> 
>>  - Should the source have a TypeInformation for the produced type, instead
>> of a serializer? We need a type information in the stream anyways, and can
>> derive the serializer from that. Plus, creating the serializer should
>> respect the ExecutionConfig.
>> 
>>  - The TypeSerializer interface is very powerful but also not easy to
>> implement. Its purpose is to handle data super efficiently, support
>> flexible ways of evolution, etc.
>>  For metadata I would suggest to look at the SimpleVersionedSerializer
>> instead, which is used for example for checkpoint master hooks, or for the
>> streaming file sink. I think that is is a good match for cases where we do
>> not need more than ser/deser (no copy, etc.) and don't need to push
>> versioning out of the serialization paths for best performance (as in the
>> TypeSerializer)
>> 
>> 
>> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
>> k.kloudas@data-artisans.com>
>> wrote:
>> 
>>> Hi Biao,
>>> 
>>> Thanks for the answer!
>>> 
>>> So given the multi-threaded readers, now we have as open questions:
>>> 
>>> 1) How do we let the checkpoints pass through our multi-threaded reader
>>> operator?
>>> 
>>> 2) Do we have separate reader and source operators or not? In the
>> strategy
>>> that has a separate source, the source operator has a parallelism of 1
>> and
>>> is responsible for split recovery only.
>>> 
>>> For the first one, given also the constraints (blocking, finite queues,
>>> etc), I do not have an answer yet.
>>> 
>>> For the 2nd, I think that we should go with separate operators for the
>>> source and the readers, for the following reasons:
>>> 
>>> 1) This is more aligned with a potential future improvement where the
>> split
>>> discovery becomes a responsibility of the JobManager and readers are
>>> pooling more work from the JM.
>>> 
>>> 2) The source is going to be the "single point of truth". It will know
>> what
>>> has been processed and what not. If the source and the readers are a
>> single
>>> operator with parallelism > 1, or in general, if the split discovery is
>>> done by each task individually, then:
>>>   i) we have to have a deterministic scheme for each reader to assign
>>> splits to itself (e.g. mod subtaskId). This is not necessarily trivial
>> for
>>> all sources.
>>>   ii) each reader would have to keep a copy of all its processed slpits
>>>   iii) the state has to be a union state with a non-trivial merging
>> logic
>>> in order to support rescaling.
>>> 
>>> Two additional points that you raised above:
>>> 
>>> i) The point that you raised that we need to keep all splits (processed
>> and
>>> not-processed) I think is a bit of a strong requirement. This would imply
>>> that for infinite sources the state will grow indefinitely. This is
>> problem
>>> is even more pronounced if we do not have a single source that assigns
>>> splits to readers, as each reader will have its own copy of the state.
>>> 
>>> ii) it is true that for finite sources we need to somehow not close the
>>> readers when the source/split discoverer finishes. The
>>> ContinuousFileReaderOperator has a work-around for that. It is not
>> elegant,
>>> and checkpoints are not emitted after closing the source, but this, I
>>> believe, is a bigger problem which requires more changes than just
>>> refactoring the source interface.
>>> 
>>> Cheers,
>>> Kostas
>>> 
>> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Stephan Ewen <se...@apache.org>.
Before creating any JIRA issues, we need to converge a bit further on the
design.
There are too many unsolved questions in the above summary.

I would try and come up with a next version of the interface proposal in
the coming days and
use that as the base to continue the discussion.

Whether this can be part of 1.8 or not depends on how fast we converge. If
the release interval is similar
to the past releases, we would see feature freeze in the mid of next month.

Best,
Stephan


On Sun, Jan 20, 2019 at 4:09 PM Biao Liu <mm...@gmail.com> wrote:

> Hi community,
> The summary of Stephan makes a lot sense to me. It is much clearer indeed
> after splitting the complex topic into small ones.
> I was wondering is there any detail plan for next step? If not, I would
> like to push this thing forward by creating some JIRA issues.
> Another question is that should version 1.8 include these features?
>
> Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:
>
> > Thanks everyone for the lively discussion. Let me try to summarize where
> I
> > see convergence in the discussion and open issues.
> > I'll try to group this by design aspect of the source. Please let me know
> > if I got things wrong or missed something crucial here.
> >
> > For issues 1-3, if the below reflects the state of the discussion, I
> would
> > try and update the FLIP in the next days.
> > For the remaining ones we need more discussion.
> >
> > I would suggest to fork each of these aspects into a separate mail
> thread,
> > or will loose sight of the individual aspects.
> >
> > *(1) Separation of Split Enumerator and Split Reader*
> >
> >   - All seem to agree this is a good thing
> >   - Split Enumerator could in the end live on JobManager (and assign
> splits
> > via RPC) or in a task (and assign splits via data streams)
> >   - this discussion is orthogonal and should come later, when the
> interface
> > is agreed upon.
> >
> > *(2) Split Readers for one or more splits*
> >
> >   - Discussion seems to agree that we need to support one reader that
> > possibly handles multiple splits concurrently.
> >   - The requirement comes from sources where one poll()-style call
> fetches
> > data from different splits / partitions
> >     --> example sources that require that would be for example Kafka,
> > Pravega, Pulsar
> >
> >   - Could have one split reader per source, or multiple split readers
> that
> > share the "poll()" function
> >   - To not make it too complicated, we can start with thinking about one
> > split reader for all splits initially and see if that covers all
> > requirements
> >
> > *(3) Threading model of the Split Reader*
> >
> >   - Most active part of the discussion ;-)
> >
> >   - A non-blocking way for Flink's task code to interact with the source
> is
> > needed in order to a task runtime code based on a
> > single-threaded/actor-style task design
> >     --> I personally am a big proponent of that, it will help with
> > well-behaved checkpoints, efficiency, and simpler yet more robust runtime
> > code
> >
> >   - Users care about simple abstraction, so as a subclass of SplitReader
> > (non-blocking / async) we need to have a BlockingSplitReader which will
> > form the basis of most source implementations. BlockingSplitReader lets
> > users do blocking simple poll() calls.
> >   - The BlockingSplitReader would spawn a thread (or more) and the
> > thread(s) can make blocking calls and hand over data buffers via a
> blocking
> > queue
> >   - This should allow us to cover both, a fully async runtime, and a
> simple
> > blocking interface for users.
> >   - This is actually very similar to how the Kafka connectors work. Kafka
> > 9+ with one thread, Kafka 8 with multiple threads
> >
> >   - On the base SplitReader (the async one), the non-blocking method that
> > gets the next chunk of data would signal data availability via a
> > CompletableFuture, because that gives the best flexibility (can await
> > completion or register notification handlers).
> >   - The source task would register a "thenHandle()" (or similar) on the
> > future to put a "take next data" task into the actor-style mailbox
> >
> > *(4) Split Enumeration and Assignment*
> >
> >   - Splits may be generated lazily, both in cases where there is a
> limited
> > number of splits (but very many), or splits are discovered over time
> >   - Assignment should also be lazy, to get better load balancing
> >   - Assignment needs support locality preferences
> >
> >   - Possible design based on discussion so far:
> >
> >     --> SplitReader has a method "addSplits(SplitT...)" to add one or
> more
> > splits. Some split readers might assume they have only one split ever,
> > concurrently, others assume multiple splits. (Note: idea behind being
> able
> > to add multiple splits at the same time is to ease startup where multiple
> > splits may be assigned instantly.)
> >     --> SplitReader has a context object on which it can call indicate
> when
> > splits are completed. The enumerator gets that notification and can use
> to
> > decide when to assign new splits. This should help both in cases of
> sources
> > that take splits lazily (file readers) and in case the source needs to
> > preserve a partial order between splits (Kinesis, Pravega, Pulsar may
> need
> > that).
> >     --> SplitEnumerator gets notification when SplitReaders start and
> when
> > they finish splits. They can decide at that moment to push more splits to
> > that reader
> >     --> The SplitEnumerator should probably be aware of the source
> > parallelism, to build its initial distribution.
> >
> >   - Open question: Should the source expose something like "host
> > preferences", so that yarn/mesos/k8s can take this into account when
> > selecting a node to start a TM on?
> >
> > *(5) Watermarks and event time alignment*
> >
> >   - Watermark generation, as well as idleness, needs to be per split
> (like
> > currently in the Kafka Source, per partition)
> >   - It is desirable to support optional event-time-alignment, meaning
> that
> > splits that are ahead are back-pressured or temporarily unsubscribed
> >
> >   - I think i would be desirable to encapsulate watermark generation
> logic
> > in watermark generators, for a separation of concerns. The watermark
> > generators should run per split.
> >   - Using watermark generators would also help with another problem of
> the
> > suggested interface, namely supporting non-periodic watermarks
> efficiently.
> >
> >   - Need a way to "dispatch" next record to different watermark
> generators
> >   - Need a way to tell SplitReader to "suspend" a split until a certain
> > watermark is reached (event time backpressure)
> >   - This would in fact be not needed (and thus simpler) if we had a
> > SplitReader per split and may be a reason to re-open that discussion
> >
> > *(6) Watermarks across splits and in the Split Enumerator*
> >
> >   - The split enumerator may need some watermark awareness, which should
> be
> > purely based on split metadata (like create timestamp of file splits)
> >   - If there are still more splits with overlapping event time range for
> a
> > split reader, then that split reader should not advance the watermark
> > within the split beyond the overlap boundary. Otherwise future splits
> will
> > produce late data.
> >
> >   - One way to approach this could be that the split enumerator may send
> > watermarks to the readers, and the readers cannot emit watermarks beyond
> > that received watermark.
> >   - Many split enumerators would simply immediately send Long.MAX out and
> > leave the progress purely to the split readers.
> >
> >   - For event-time alignment / split back pressure, this begs the
> question
> > how we can avoid deadlocks that may arise when splits are suspended for
> > event time back pressure,
> >
> > *(7) Batch and streaming Unification*
> >
> >   - Functionality wise, the above design should support both
> >   - Batch often (mostly) does not care about reading "in order" and
> > generating watermarks
> >     --> Might use different enumerator logic that is more locality aware
> > and ignores event time order
> >     --> Does not generate watermarks
> >   - Would be great if bounded sources could be identified at compile
> time,
> > so that "env.addBoundedSource(...)" is type safe and can return a
> > "BoundedDataStream".
> >   - Possible to defer this discussion until later
> >
> > *Miscellaneous Comments*
> >
> >   - Should the source have a TypeInformation for the produced type,
> instead
> > of a serializer? We need a type information in the stream anyways, and
> can
> > derive the serializer from that. Plus, creating the serializer should
> > respect the ExecutionConfig.
> >
> >   - The TypeSerializer interface is very powerful but also not easy to
> > implement. Its purpose is to handle data super efficiently, support
> > flexible ways of evolution, etc.
> >   For metadata I would suggest to look at the SimpleVersionedSerializer
> > instead, which is used for example for checkpoint master hooks, or for
> the
> > streaming file sink. I think that is is a good match for cases where we
> do
> > not need more than ser/deser (no copy, etc.) and don't need to push
> > versioning out of the serialization paths for best performance (as in the
> > TypeSerializer)
> >
> >
> > On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> > k.kloudas@data-artisans.com>
> > wrote:
> >
> > > Hi Biao,
> > >
> > > Thanks for the answer!
> > >
> > > So given the multi-threaded readers, now we have as open questions:
> > >
> > > 1) How do we let the checkpoints pass through our multi-threaded reader
> > > operator?
> > >
> > > 2) Do we have separate reader and source operators or not? In the
> > strategy
> > > that has a separate source, the source operator has a parallelism of 1
> > and
> > > is responsible for split recovery only.
> > >
> > > For the first one, given also the constraints (blocking, finite queues,
> > > etc), I do not have an answer yet.
> > >
> > > For the 2nd, I think that we should go with separate operators for the
> > > source and the readers, for the following reasons:
> > >
> > > 1) This is more aligned with a potential future improvement where the
> > split
> > > discovery becomes a responsibility of the JobManager and readers are
> > > pooling more work from the JM.
> > >
> > > 2) The source is going to be the "single point of truth". It will know
> > what
> > > has been processed and what not. If the source and the readers are a
> > single
> > > operator with parallelism > 1, or in general, if the split discovery is
> > > done by each task individually, then:
> > >    i) we have to have a deterministic scheme for each reader to assign
> > > splits to itself (e.g. mod subtaskId). This is not necessarily trivial
> > for
> > > all sources.
> > >    ii) each reader would have to keep a copy of all its processed
> slpits
> > >    iii) the state has to be a union state with a non-trivial merging
> > logic
> > > in order to support rescaling.
> > >
> > > Two additional points that you raised above:
> > >
> > > i) The point that you raised that we need to keep all splits (processed
> > and
> > > not-processed) I think is a bit of a strong requirement. This would
> imply
> > > that for infinite sources the state will grow indefinitely. This is
> > problem
> > > is even more pronounced if we do not have a single source that assigns
> > > splits to readers, as each reader will have its own copy of the state.
> > >
> > > ii) it is true that for finite sources we need to somehow not close the
> > > readers when the source/split discoverer finishes. The
> > > ContinuousFileReaderOperator has a work-around for that. It is not
> > elegant,
> > > and checkpoints are not emitted after closing the source, but this, I
> > > believe, is a bigger problem which requires more changes than just
> > > refactoring the source interface.
> > >
> > > Cheers,
> > > Kostas
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Biao Liu <mm...@gmail.com>.
Hi community,
The summary of Stephan makes a lot sense to me. It is much clearer indeed
after splitting the complex topic into small ones.
I was wondering is there any detail plan for next step? If not, I would
like to push this thing forward by creating some JIRA issues.
Another question is that should version 1.8 include these features?

Stephan Ewen <se...@apache.org> 于2018年12月1日周六 上午4:20写道:

> Thanks everyone for the lively discussion. Let me try to summarize where I
> see convergence in the discussion and open issues.
> I'll try to group this by design aspect of the source. Please let me know
> if I got things wrong or missed something crucial here.
>
> For issues 1-3, if the below reflects the state of the discussion, I would
> try and update the FLIP in the next days.
> For the remaining ones we need more discussion.
>
> I would suggest to fork each of these aspects into a separate mail thread,
> or will loose sight of the individual aspects.
>
> *(1) Separation of Split Enumerator and Split Reader*
>
>   - All seem to agree this is a good thing
>   - Split Enumerator could in the end live on JobManager (and assign splits
> via RPC) or in a task (and assign splits via data streams)
>   - this discussion is orthogonal and should come later, when the interface
> is agreed upon.
>
> *(2) Split Readers for one or more splits*
>
>   - Discussion seems to agree that we need to support one reader that
> possibly handles multiple splits concurrently.
>   - The requirement comes from sources where one poll()-style call fetches
> data from different splits / partitions
>     --> example sources that require that would be for example Kafka,
> Pravega, Pulsar
>
>   - Could have one split reader per source, or multiple split readers that
> share the "poll()" function
>   - To not make it too complicated, we can start with thinking about one
> split reader for all splits initially and see if that covers all
> requirements
>
> *(3) Threading model of the Split Reader*
>
>   - Most active part of the discussion ;-)
>
>   - A non-blocking way for Flink's task code to interact with the source is
> needed in order to a task runtime code based on a
> single-threaded/actor-style task design
>     --> I personally am a big proponent of that, it will help with
> well-behaved checkpoints, efficiency, and simpler yet more robust runtime
> code
>
>   - Users care about simple abstraction, so as a subclass of SplitReader
> (non-blocking / async) we need to have a BlockingSplitReader which will
> form the basis of most source implementations. BlockingSplitReader lets
> users do blocking simple poll() calls.
>   - The BlockingSplitReader would spawn a thread (or more) and the
> thread(s) can make blocking calls and hand over data buffers via a blocking
> queue
>   - This should allow us to cover both, a fully async runtime, and a simple
> blocking interface for users.
>   - This is actually very similar to how the Kafka connectors work. Kafka
> 9+ with one thread, Kafka 8 with multiple threads
>
>   - On the base SplitReader (the async one), the non-blocking method that
> gets the next chunk of data would signal data availability via a
> CompletableFuture, because that gives the best flexibility (can await
> completion or register notification handlers).
>   - The source task would register a "thenHandle()" (or similar) on the
> future to put a "take next data" task into the actor-style mailbox
>
> *(4) Split Enumeration and Assignment*
>
>   - Splits may be generated lazily, both in cases where there is a limited
> number of splits (but very many), or splits are discovered over time
>   - Assignment should also be lazy, to get better load balancing
>   - Assignment needs support locality preferences
>
>   - Possible design based on discussion so far:
>
>     --> SplitReader has a method "addSplits(SplitT...)" to add one or more
> splits. Some split readers might assume they have only one split ever,
> concurrently, others assume multiple splits. (Note: idea behind being able
> to add multiple splits at the same time is to ease startup where multiple
> splits may be assigned instantly.)
>     --> SplitReader has a context object on which it can call indicate when
> splits are completed. The enumerator gets that notification and can use to
> decide when to assign new splits. This should help both in cases of sources
> that take splits lazily (file readers) and in case the source needs to
> preserve a partial order between splits (Kinesis, Pravega, Pulsar may need
> that).
>     --> SplitEnumerator gets notification when SplitReaders start and when
> they finish splits. They can decide at that moment to push more splits to
> that reader
>     --> The SplitEnumerator should probably be aware of the source
> parallelism, to build its initial distribution.
>
>   - Open question: Should the source expose something like "host
> preferences", so that yarn/mesos/k8s can take this into account when
> selecting a node to start a TM on?
>
> *(5) Watermarks and event time alignment*
>
>   - Watermark generation, as well as idleness, needs to be per split (like
> currently in the Kafka Source, per partition)
>   - It is desirable to support optional event-time-alignment, meaning that
> splits that are ahead are back-pressured or temporarily unsubscribed
>
>   - I think i would be desirable to encapsulate watermark generation logic
> in watermark generators, for a separation of concerns. The watermark
> generators should run per split.
>   - Using watermark generators would also help with another problem of the
> suggested interface, namely supporting non-periodic watermarks efficiently.
>
>   - Need a way to "dispatch" next record to different watermark generators
>   - Need a way to tell SplitReader to "suspend" a split until a certain
> watermark is reached (event time backpressure)
>   - This would in fact be not needed (and thus simpler) if we had a
> SplitReader per split and may be a reason to re-open that discussion
>
> *(6) Watermarks across splits and in the Split Enumerator*
>
>   - The split enumerator may need some watermark awareness, which should be
> purely based on split metadata (like create timestamp of file splits)
>   - If there are still more splits with overlapping event time range for a
> split reader, then that split reader should not advance the watermark
> within the split beyond the overlap boundary. Otherwise future splits will
> produce late data.
>
>   - One way to approach this could be that the split enumerator may send
> watermarks to the readers, and the readers cannot emit watermarks beyond
> that received watermark.
>   - Many split enumerators would simply immediately send Long.MAX out and
> leave the progress purely to the split readers.
>
>   - For event-time alignment / split back pressure, this begs the question
> how we can avoid deadlocks that may arise when splits are suspended for
> event time back pressure,
>
> *(7) Batch and streaming Unification*
>
>   - Functionality wise, the above design should support both
>   - Batch often (mostly) does not care about reading "in order" and
> generating watermarks
>     --> Might use different enumerator logic that is more locality aware
> and ignores event time order
>     --> Does not generate watermarks
>   - Would be great if bounded sources could be identified at compile time,
> so that "env.addBoundedSource(...)" is type safe and can return a
> "BoundedDataStream".
>   - Possible to defer this discussion until later
>
> *Miscellaneous Comments*
>
>   - Should the source have a TypeInformation for the produced type, instead
> of a serializer? We need a type information in the stream anyways, and can
> derive the serializer from that. Plus, creating the serializer should
> respect the ExecutionConfig.
>
>   - The TypeSerializer interface is very powerful but also not easy to
> implement. Its purpose is to handle data super efficiently, support
> flexible ways of evolution, etc.
>   For metadata I would suggest to look at the SimpleVersionedSerializer
> instead, which is used for example for checkpoint master hooks, or for the
> streaming file sink. I think that is is a good match for cases where we do
> not need more than ser/deser (no copy, etc.) and don't need to push
> versioning out of the serialization paths for best performance (as in the
> TypeSerializer)
>
>
> On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <
> k.kloudas@data-artisans.com>
> wrote:
>
> > Hi Biao,
> >
> > Thanks for the answer!
> >
> > So given the multi-threaded readers, now we have as open questions:
> >
> > 1) How do we let the checkpoints pass through our multi-threaded reader
> > operator?
> >
> > 2) Do we have separate reader and source operators or not? In the
> strategy
> > that has a separate source, the source operator has a parallelism of 1
> and
> > is responsible for split recovery only.
> >
> > For the first one, given also the constraints (blocking, finite queues,
> > etc), I do not have an answer yet.
> >
> > For the 2nd, I think that we should go with separate operators for the
> > source and the readers, for the following reasons:
> >
> > 1) This is more aligned with a potential future improvement where the
> split
> > discovery becomes a responsibility of the JobManager and readers are
> > pooling more work from the JM.
> >
> > 2) The source is going to be the "single point of truth". It will know
> what
> > has been processed and what not. If the source and the readers are a
> single
> > operator with parallelism > 1, or in general, if the split discovery is
> > done by each task individually, then:
> >    i) we have to have a deterministic scheme for each reader to assign
> > splits to itself (e.g. mod subtaskId). This is not necessarily trivial
> for
> > all sources.
> >    ii) each reader would have to keep a copy of all its processed slpits
> >    iii) the state has to be a union state with a non-trivial merging
> logic
> > in order to support rescaling.
> >
> > Two additional points that you raised above:
> >
> > i) The point that you raised that we need to keep all splits (processed
> and
> > not-processed) I think is a bit of a strong requirement. This would imply
> > that for infinite sources the state will grow indefinitely. This is
> problem
> > is even more pronounced if we do not have a single source that assigns
> > splits to readers, as each reader will have its own copy of the state.
> >
> > ii) it is true that for finite sources we need to somehow not close the
> > readers when the source/split discoverer finishes. The
> > ContinuousFileReaderOperator has a work-around for that. It is not
> elegant,
> > and checkpoints are not emitted after closing the source, but this, I
> > believe, is a bigger problem which requires more changes than just
> > refactoring the source interface.
> >
> > Cheers,
> > Kostas
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Stephan Ewen <se...@apache.org>.
Thanks everyone for the lively discussion. Let me try to summarize where I
see convergence in the discussion and open issues.
I'll try to group this by design aspect of the source. Please let me know
if I got things wrong or missed something crucial here.

For issues 1-3, if the below reflects the state of the discussion, I would
try and update the FLIP in the next days.
For the remaining ones we need more discussion.

I would suggest to fork each of these aspects into a separate mail thread,
or will loose sight of the individual aspects.

*(1) Separation of Split Enumerator and Split Reader*

  - All seem to agree this is a good thing
  - Split Enumerator could in the end live on JobManager (and assign splits
via RPC) or in a task (and assign splits via data streams)
  - this discussion is orthogonal and should come later, when the interface
is agreed upon.

*(2) Split Readers for one or more splits*

  - Discussion seems to agree that we need to support one reader that
possibly handles multiple splits concurrently.
  - The requirement comes from sources where one poll()-style call fetches
data from different splits / partitions
    --> example sources that require that would be for example Kafka,
Pravega, Pulsar

  - Could have one split reader per source, or multiple split readers that
share the "poll()" function
  - To not make it too complicated, we can start with thinking about one
split reader for all splits initially and see if that covers all
requirements

*(3) Threading model of the Split Reader*

  - Most active part of the discussion ;-)

  - A non-blocking way for Flink's task code to interact with the source is
needed in order to a task runtime code based on a
single-threaded/actor-style task design
    --> I personally am a big proponent of that, it will help with
well-behaved checkpoints, efficiency, and simpler yet more robust runtime
code

  - Users care about simple abstraction, so as a subclass of SplitReader
(non-blocking / async) we need to have a BlockingSplitReader which will
form the basis of most source implementations. BlockingSplitReader lets
users do blocking simple poll() calls.
  - The BlockingSplitReader would spawn a thread (or more) and the
thread(s) can make blocking calls and hand over data buffers via a blocking
queue
  - This should allow us to cover both, a fully async runtime, and a simple
blocking interface for users.
  - This is actually very similar to how the Kafka connectors work. Kafka
9+ with one thread, Kafka 8 with multiple threads

  - On the base SplitReader (the async one), the non-blocking method that
gets the next chunk of data would signal data availability via a
CompletableFuture, because that gives the best flexibility (can await
completion or register notification handlers).
  - The source task would register a "thenHandle()" (or similar) on the
future to put a "take next data" task into the actor-style mailbox

*(4) Split Enumeration and Assignment*

  - Splits may be generated lazily, both in cases where there is a limited
number of splits (but very many), or splits are discovered over time
  - Assignment should also be lazy, to get better load balancing
  - Assignment needs support locality preferences

  - Possible design based on discussion so far:

    --> SplitReader has a method "addSplits(SplitT...)" to add one or more
splits. Some split readers might assume they have only one split ever,
concurrently, others assume multiple splits. (Note: idea behind being able
to add multiple splits at the same time is to ease startup where multiple
splits may be assigned instantly.)
    --> SplitReader has a context object on which it can call indicate when
splits are completed. The enumerator gets that notification and can use to
decide when to assign new splits. This should help both in cases of sources
that take splits lazily (file readers) and in case the source needs to
preserve a partial order between splits (Kinesis, Pravega, Pulsar may need
that).
    --> SplitEnumerator gets notification when SplitReaders start and when
they finish splits. They can decide at that moment to push more splits to
that reader
    --> The SplitEnumerator should probably be aware of the source
parallelism, to build its initial distribution.

  - Open question: Should the source expose something like "host
preferences", so that yarn/mesos/k8s can take this into account when
selecting a node to start a TM on?

*(5) Watermarks and event time alignment*

  - Watermark generation, as well as idleness, needs to be per split (like
currently in the Kafka Source, per partition)
  - It is desirable to support optional event-time-alignment, meaning that
splits that are ahead are back-pressured or temporarily unsubscribed

  - I think i would be desirable to encapsulate watermark generation logic
in watermark generators, for a separation of concerns. The watermark
generators should run per split.
  - Using watermark generators would also help with another problem of the
suggested interface, namely supporting non-periodic watermarks efficiently.

  - Need a way to "dispatch" next record to different watermark generators
  - Need a way to tell SplitReader to "suspend" a split until a certain
watermark is reached (event time backpressure)
  - This would in fact be not needed (and thus simpler) if we had a
SplitReader per split and may be a reason to re-open that discussion

*(6) Watermarks across splits and in the Split Enumerator*

  - The split enumerator may need some watermark awareness, which should be
purely based on split metadata (like create timestamp of file splits)
  - If there are still more splits with overlapping event time range for a
split reader, then that split reader should not advance the watermark
within the split beyond the overlap boundary. Otherwise future splits will
produce late data.

  - One way to approach this could be that the split enumerator may send
watermarks to the readers, and the readers cannot emit watermarks beyond
that received watermark.
  - Many split enumerators would simply immediately send Long.MAX out and
leave the progress purely to the split readers.

  - For event-time alignment / split back pressure, this begs the question
how we can avoid deadlocks that may arise when splits are suspended for
event time back pressure,

*(7) Batch and streaming Unification*

  - Functionality wise, the above design should support both
  - Batch often (mostly) does not care about reading "in order" and
generating watermarks
    --> Might use different enumerator logic that is more locality aware
and ignores event time order
    --> Does not generate watermarks
  - Would be great if bounded sources could be identified at compile time,
so that "env.addBoundedSource(...)" is type safe and can return a
"BoundedDataStream".
  - Possible to defer this discussion until later

*Miscellaneous Comments*

  - Should the source have a TypeInformation for the produced type, instead
of a serializer? We need a type information in the stream anyways, and can
derive the serializer from that. Plus, creating the serializer should
respect the ExecutionConfig.

  - The TypeSerializer interface is very powerful but also not easy to
implement. Its purpose is to handle data super efficiently, support
flexible ways of evolution, etc.
  For metadata I would suggest to look at the SimpleVersionedSerializer
instead, which is used for example for checkpoint master hooks, or for the
streaming file sink. I think that is is a good match for cases where we do
not need more than ser/deser (no copy, etc.) and don't need to push
versioning out of the serialization paths for best performance (as in the
TypeSerializer)


On Tue, Nov 27, 2018 at 11:45 AM Kostas Kloudas <k....@data-artisans.com>
wrote:

> Hi Biao,
>
> Thanks for the answer!
>
> So given the multi-threaded readers, now we have as open questions:
>
> 1) How do we let the checkpoints pass through our multi-threaded reader
> operator?
>
> 2) Do we have separate reader and source operators or not? In the strategy
> that has a separate source, the source operator has a parallelism of 1 and
> is responsible for split recovery only.
>
> For the first one, given also the constraints (blocking, finite queues,
> etc), I do not have an answer yet.
>
> For the 2nd, I think that we should go with separate operators for the
> source and the readers, for the following reasons:
>
> 1) This is more aligned with a potential future improvement where the split
> discovery becomes a responsibility of the JobManager and readers are
> pooling more work from the JM.
>
> 2) The source is going to be the "single point of truth". It will know what
> has been processed and what not. If the source and the readers are a single
> operator with parallelism > 1, or in general, if the split discovery is
> done by each task individually, then:
>    i) we have to have a deterministic scheme for each reader to assign
> splits to itself (e.g. mod subtaskId). This is not necessarily trivial for
> all sources.
>    ii) each reader would have to keep a copy of all its processed slpits
>    iii) the state has to be a union state with a non-trivial merging logic
> in order to support rescaling.
>
> Two additional points that you raised above:
>
> i) The point that you raised that we need to keep all splits (processed and
> not-processed) I think is a bit of a strong requirement. This would imply
> that for infinite sources the state will grow indefinitely. This is problem
> is even more pronounced if we do not have a single source that assigns
> splits to readers, as each reader will have its own copy of the state.
>
> ii) it is true that for finite sources we need to somehow not close the
> readers when the source/split discoverer finishes. The
> ContinuousFileReaderOperator has a work-around for that. It is not elegant,
> and checkpoints are not emitted after closing the source, but this, I
> believe, is a bigger problem which requires more changes than just
> refactoring the source interface.
>
> Cheers,
> Kostas
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Kostas Kloudas <k....@data-artisans.com>.
Hi Biao,

Thanks for the answer!

So given the multi-threaded readers, now we have as open questions:

1) How do we let the checkpoints pass through our multi-threaded reader
operator?

2) Do we have separate reader and source operators or not? In the strategy
that has a separate source, the source operator has a parallelism of 1 and
is responsible for split recovery only.

For the first one, given also the constraints (blocking, finite queues,
etc), I do not have an answer yet.

For the 2nd, I think that we should go with separate operators for the
source and the readers, for the following reasons:

1) This is more aligned with a potential future improvement where the split
discovery becomes a responsibility of the JobManager and readers are
pooling more work from the JM.

2) The source is going to be the "single point of truth". It will know what
has been processed and what not. If the source and the readers are a single
operator with parallelism > 1, or in general, if the split discovery is
done by each task individually, then:
   i) we have to have a deterministic scheme for each reader to assign
splits to itself (e.g. mod subtaskId). This is not necessarily trivial for
all sources.
   ii) each reader would have to keep a copy of all its processed slpits
   iii) the state has to be a union state with a non-trivial merging logic
in order to support rescaling.

Two additional points that you raised above:

i) The point that you raised that we need to keep all splits (processed and
not-processed) I think is a bit of a strong requirement. This would imply
that for infinite sources the state will grow indefinitely. This is problem
is even more pronounced if we do not have a single source that assigns
splits to readers, as each reader will have its own copy of the state.

ii) it is true that for finite sources we need to somehow not close the
readers when the source/split discoverer finishes. The
ContinuousFileReaderOperator has a work-around for that. It is not elegant,
and checkpoints are not emitted after closing the source, but this, I
believe, is a bigger problem which requires more changes than just
refactoring the source interface.

Cheers,
Kostas

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Biao Liu <mm...@gmail.com>.
Hi Kostas again,

Did I misunderstand you in last response?

If you mean checkpoint in the scenario that the source and the split reader
are in different operators, like Aljoscha's prototype. That's indeed a
problem, so I think that's would not be the final version. Aljoscha also
said in FLIP doc that it is a MVP (minimum viable product? correct me if I
was wrong).

There is some other problem in this scenario. For example, if the split
count is fixed. So the source discovery node will finish soon. If the split
is infinite, such as a message queue. The checkpoint would be never
triggered since source node has already been finished.


Biao Liu <mm...@gmail.com> 于2018年11月27日周二 上午10:37写道:

> Hi Kostas,
>
> Regarding the checkpoint of "per thread for each split mode". IMO, there
> are severals things source operator need to do.
> 1. Source operator need to record all splits in checkpoint. The unfinished
> splits must be recorded. I'm not sure whether we could skip recording the
> finished splits, it depends on split discovery implementation.
> 2. Source operator need to collect the last record polled from each split
> queue. And put them into checkpoint.
> 3. SplitReader can be restored by giving a specific split with a position
> of last record.
>
> And I think you raised another important issue. The queue between task
> thread and split readers.
> 1. I agree that it must be a thread-safe, size limited queue, such as
> ArrayBlockingQueue.
> 2. Also it's hard to decide the size of queue. We have to consider the
> split count, the size of item in queue to make sure the memory of source
> operator will not be out of control. Giving a unified queue size is not
> proper since there may be several different sources in one job. It's better
> that each source can decide the queue size of itself.
>
>
> Kostas Kloudas <k....@data-artisans.com> 于2018年11月26日周一 下午8:42写道:
>
>> Hi all,
>>
>> From the discussion, I understand that we are leaning towards a design
>> where the user writes a single-threaded SplitReader, which Flink executes
>> on another thread (not the main task thread). This way the task can have
>> multiple readers running concurrently, each one reading a different split.
>>
>> Each of these threads writes in its own queue. These queues are then
>> polled
>> by the main thread (based on a potentially user-defined prioritization),
>> which is responsible for emitting data downstream. There were also
>> proposals for a single shared queue, but I believe that 1) the contention
>> for the lock in such a queue can be a limitation and 2) it is not easy to
>> prioritise which elements to consume first (assuming that we want to
>> support different prioritisation strategies).
>>
>> Assuming the above model, I have the following question:
>>
>> We have the split/shard/partition discovery logic outside the "reader"
>> operator. For now it can be a plain old source function with parallelism
>> of
>> 1 that periodically checks for new splits (for an example see the existing
>> ContinuousFileMonitoringFunction).[1]
>>
>> This source sends the split to be read downstream to the multi-threaded
>> readers. In these settings, there must be a "throttling" or
>> "rate-limitting" mechanism that guaranttees that we do not surpass the
>> capabilities of the machines. The first thing that comes to mind is some
>> kind of a fixed size (blocking) queue or a fixed size thread pool. The
>> main
>> thread adds splits to the queue and the readers consume them. When the
>> queue or the pool is full, then we block (backpressure).
>>
>> In the case above, how do we make sure that the checkpoints still go
>> through?
>>
>> Cheers,
>> Kostas
>>
>> PS: I am assuming the current task implementation and not an "actor" based
>> one.
>>
>> *[1] The ContinuousFileReaderOperator has a single thread (different from
>> the main task thread) consuming the splits one by one. Unfortunately,
>> there
>> is no rate-limiting mechanism.
>>
>>
>> On Sun, Nov 25, 2018 at 6:40 PM Biao Liu <mm...@gmail.com> wrote:
>>
>> > Hi community,
>> > Glad to see this topic is still so active.
>> >
>> > Thanks for replying @Piotrek and @Becket.
>> >
>> > Last time, I expressed some rough ideas about the thread model. However
>> I
>> > found that it's hard to describe clearly in mailing list. So I wrote it
>> > down with some graphs, exampled some kinds of models, see Thread Model
>> of
>> > Source
>> > <
>> >
>> https://docs.google.com/document/d/1XpYkkJo97CUw-UMVrKU6b0ZZuJJ2V7mBb__L6UzdWTw/edit?usp=sharing
>> > >.
>> > I wish that can be helpful.
>> >
>> > IMO thread model is an important part. Without thinking of
>> implementation
>> > clearly, it's difficult to decide what the up level interface should
>> look
>> > like.
>> > It would be better if we draw the whole picture first and then fill the
>> > detail parts one by one.
>> >
>> > @Piotrek About adding new splits to existing split reader. It's an
>> > interesting idea. Not only for solving too many threads problem, but
>> also
>> > for supporting some more complicated system. I know in some storage
>> > systems, there is some scenario which the partition is
>> dynamic(dynamically
>> > splitting or merging). Though I have not think of it very clearly now. I
>> > would give you more detailed reply asap :)
>> >
>> >
>> > Guowei Ma <gu...@gmail.com> 于2018年11月23日周五 下午6:37写道:
>> >
>> > > Hi,Piotr
>> > > Sorry  for so late to response.
>> > >
>> > >
>> > > First of all I think Flink runtime can assigned a thread for a
>> > StreamTask,
>> > > which likes  'Actor' model. The number of threads for a StreamTask
>> should
>> > > not be proportional to the operator or other things. This will give
>> Flink
>> > > the ability to scale horizontally. So I think it's not just the
>> > > network(flush),checkpoint and  source, but some operators' threads can
>> > also
>> > > be removed in the future, like AsyncWaitOperator.
>> > >
>> > >
>> > >
>> > > for b)
>> > > When using event time, some sources want to assign a timestamp to each
>> > > element. In current Flink interface, user will write like this
>> > > public class EventTimeSource<Element> implements SourceFunction {
>> > >   public void run() {
>> > >      while(...){
>> > >          Element record = // get from file or some queue;
>> > >          long timestamp = parseTimestampFromElement(record);
>> > >          sourceContext.collectWithTimestamp(record, timestamp);
>> > >      }
>> > >   }
>> > > }
>> > > Using the interfaces from this FLIP, user can write like this
>> > >
>> > > public EventTimeSplitReader<Element,X> implements SplitReader {
>> > >     Element currentRecord = null;
>> > >
>> > >
>> > >     // Please ignoring the handling of boundary conditions
>> > >     public boolean advace(){
>> > >        currentRecord = //move a pointer forward
>> > >        return true;
>> > >      }
>> > >
>> > >     public Element getCurrent(){
>> > >        return currentRecord;
>> > >     }
>> > >     public long getCurrentTimestamp() {
>> > >       return parseTimestampFromElement(currentRecord);
>> > >     }
>> > > }
>> > >
>> > > if merging the advance/getNext to a method like getNext() , the
>> > SplitReader
>> > > interface may need to change a little like this
>> > >
>> > > public interface SplitReader2<T,X> {
>> > > public class ElementWithTimestamp {
>> > >     T element;
>> > >     long timestamp;
>> > > }
>> > >
>> > > public ElementWithTimestamp getNext() ;
>> > >
>> > > }
>> > > Now user may need implement the source like this
>> > > public EventTimeSplitReader<Element,X> implements SplitReader2 {
>> > >     Element currentRecord = null;
>> > >
>> > >     // Please ignoring the handling of boundary conditions
>> > >     public ElementWithTimestamp getCurrent(){
>> > >        return new ElementWithTimestamp(currentRecord,
>> > > parseTimestampFromElement(currentRecord))
>> > >     }
>> > > }
>> > > The user can use a constant ElementWithTimestamp but I think this need
>> > the
>> > > every connector developers to know this trick. The current Flip will
>> not
>> > > have this burden.
>> > > Maybe there has other way like '' void
>> getCurrent(ElementWithTimestamp)"
>> > > to avoid creating a new object.  But my personal preference is
>> > > ‘advance/getCurrent’.
>> > >
>> > >
>> > >
>> > > Piotr Nowojski <pi...@data-artisans.com> 于2018年11月7日周三 下午4:31写道:
>> > >
>> > > > Hi,
>> > > >
>> > > > a)
>> > > >
>> > > > > BTW, regarding the isBlock() method, I have a few more questions.
>> 21,
>> > > Is
>> > > > a method isReady() with boolean as a return value
>> > > > > equivalent? Personally I found it is a little bit confusing in
>> what
>> > is
>> > > > supposed to be returned when the future is completed. 22. if
>> > > > > the implementation of isBlocked() is optional, how do the callers
>> > know
>> > > > whether the method is properly implemented or not?
>> > > > > Does not implemented mean it always return a completed future?
>> > > >
>> > > > `CompletableFuture<?> isBlocked()` is more or less an equivalent to
>> > > > `boolean hasNext()` which in case of “false” provides some kind of a
>> > > > listener/callback that notifies about presence of next element.
>> There
>> > are
>> > > > some minor details, like `CompletableFuture<?>` has a minimal two
>> state
>> > > > logic:
>> > > >
>> > > > 1. Future is completed - we have more data
>> > > > 2. Future not yet completed - we don’t have data now, but we
>> might/we
>> > > will
>> > > > have in the future
>> > > >
>> > > > While `boolean hasNext()` and `notify()` callback are a bit more
>> > > > complicated/dispersed and can lead/encourage `notify()` spam.
>> > > >
>> > > > b)
>> > > >
>> > > > > 3. If merge the `advance` and `getCurrent`  to one method like
>> > > `getNext`
>> > > > the `getNext` would need return a
>> > > > >`ElementWithTimestamp` because some sources want to add timestamp
>> to
>> > > > every element. IMO, this is not so memory friendly
>> > > > > so I prefer this design.
>> > > >
>> > > > Guowei I don’t quite understand this. Could you elaborate why
>> having a
>> > > > separate `advance()` help?
>> > > >
>> > > > c)
>> > > >
>> > > > Regarding advance/poll/take. What’s the value of having two separate
>> > > > methods: poll and take? Which one of them should be called and which
>> > > > implemented? What’s the benefit of having those methods compared to
>> > > having
>> > > > a one single method `getNextElement()` (or `pollElement() or
>> whatever
>> > we
>> > > > name it) with following contract:
>> > > >
>> > > > CompletableFuture<?> isBlocked();
>> > > >
>> > > > /**
>> > > > Return next element - will be called only if `isBlocked()` is
>> > completed.
>> > > > Try to implement it in non blocking fashion, but if that’s
>> impossible
>> > or
>> > > > you just don’t need the effort, you can block in this method.
>> > > > */
>> > > > T getNextElement();
>> > > >
>> > > > I mean, if the connector is implemented non-blockingly, Flink should
>> > use
>> > > > it that way. If it’s not, then `poll()` will `throw new
>> > > > NotImplementedException()`. Implementing both of them and providing
>> > both
>> > > of
>> > > > them to Flink wouldn’t make a sense, thus why not merge them into a
>> > > single
>> > > > method call that should preferably (but not necessarily need to) be
>> > > > non-blocking? It’s not like we are implementing general purpose
>> > `Queue`,
>> > > > which users might want to call either of `poll` or `take`. We would
>> > > always
>> > > > prefer to call `poll`, but if it’s blocking, then still we have no
>> > > choice,
>> > > > but to call it and block on it.
>> > > >
>> > > > d)
>> > > >
>> > > > > 1. I agree with Piotr and Becket that the non-blocking source is
>> very
>> > > > > important. But in addition to `Future/poll`, there may be another
>> way
>> > > to
>> > > > > achieve this. I think it may be not very memory friendly if every
>> > > advance
>> > > > > call return a Future.
>> > > >
>> > > > I didn’t want to mention this, to not clog my initial proposal, but
>> > there
>> > > > is a simple solution for the problem:
>> > > >
>> > > > public interface SplitReader {
>> > > >
>> > > >     (…)
>> > > >
>> > > >     CompletableFuture<?> NOT_BLOCKED =
>> > > > CompletableFuture.completedFuture(null);
>> > > >
>> > > >     /**
>> > > >      * Returns a future that will be completed when the page source
>> > > becomes
>> > > >      * unblocked.  If the page source is not blocked, this method
>> > should
>> > > > return
>> > > >      * {@code NOT_BLOCKED}.
>> > > >      */
>> > > >     default CompletableFuture<?> isBlocked()
>> > > >     {
>> > > >         return NOT_BLOCKED;
>> > > >     }
>> > > >
>> > > > If we are blocked and we are waiting for the IO, then creating a new
>> > > > Future is non-issue. Under full throttle/throughput and not blocked
>> > > sources
>> > > > returning a static `NOT_BLOCKED` constant  should also solve the
>> > problem.
>> > > >
>> > > > One more remark, non-blocking sources might be a necessity in a
>> single
>> > > > threaded model without a checkpointing lock. (Currently when sources
>> > are
>> > > > blocked, they can release checkpointing lock and re-acquire it again
>> > > > later). Non-blocking `poll`/`getNext()` would allow for checkpoints
>> to
>> > > > happen when source is idling. In that case either `notify()` or my
>> > > proposed
>> > > > `isBlocked()` would allow to avoid busy-looping.
>> > > >
>> > > > Piotrek
>> > > >
>> > > > > On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
>> > > > >
>> > > > > Hi Thomas,
>> > > > >
>> > > > > The iterator-like API was also the first thing that came to me.
>> But
>> > it
>> > > > > seems a little confusing that hasNext() does not mean "the stream
>> has
>> > > not
>> > > > > ended", but means "the next record is ready", which is repurposing
>> > the
>> > > > well
>> > > > > known meaning of hasNext(). If we follow the hasNext()/next()
>> > pattern,
>> > > an
>> > > > > additional isNextReady() method to indicate whether the next
>> record
>> > is
>> > > > > ready seems more intuitive to me.
>> > > > >
>> > > > > Similarly, in poll()/take() pattern, another method of isDone() is
>> > > needed
>> > > > > to indicate whether the stream has ended or not.
>> > > > >
>> > > > > Compared with hasNext()/next()/isNextReady() pattern,
>> > > > > isDone()/poll()/take() seems more flexible for the reader
>> > > implementation.
>> > > > > When I am implementing a reader, I could have a couple of choices:
>> > > > >
>> > > > >   - A thread-less reader that does not have any internal thread.
>> > > > >   - When poll() is called, the same calling thread will perform a
>> > bunch
>> > > > of
>> > > > >      IO asynchronously.
>> > > > >      - When take() is called, the same calling thread will
>> perform a
>> > > > bunch
>> > > > >      of IO and wait until the record is ready.
>> > > > >   - A reader with internal threads performing network IO and put
>> > > records
>> > > > >   into a buffer.
>> > > > >      - When poll() is called, the calling thread simply reads from
>> > the
>> > > > >      buffer and return empty result immediately if there is no
>> > record.
>> > > > >      - When take() is called, the calling thread reads from the
>> > buffer
>> > > > and
>> > > > >      block waiting if the buffer is empty.
>> > > > >
>> > > > > On the other hand, with the hasNext()/next()/isNextReady() API,
>> it is
>> > > > less
>> > > > > intuitive for the reader developers to write the thread-less
>> pattern.
>> > > > > Although technically speaking one can still do the asynchronous
>> IO to
>> > > > > prepare the record in isNextReady(). But it is inexplicit and
>> seems
>> > > > > somewhat hacky.
>> > > > >
>> > > > > Thanks,
>> > > > >
>> > > > > Jiangjie (Becket) Qin
>> > > > >
>> > > > >
>> > > > >
>> > > > > On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org>
>> wrote:
>> > > > >
>> > > > >> Couple more points regarding discovery:
>> > > > >>
>> > > > >> The proposal mentions that discovery could be outside the
>> execution
>> > > > graph.
>> > > > >> Today, discovered partitions/shards are checkpointed. I believe
>> that
>> > > > will
>> > > > >> also need to be the case in the future, even when discovery and
>> > > reading
>> > > > are
>> > > > >> split between different tasks.
>> > > > >>
>> > > > >> For cases such as resharding of a Kinesis stream, the
>> relationship
>> > > > between
>> > > > >> splits needs to be considered. Splits cannot be randomly
>> distributed
>> > > > over
>> > > > >> readers in certain situations. An example was mentioned here:
>> > > > >> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>> > > > >>
>> > > > >> Thomas
>> > > > >>
>> > > > >>
>> > > > >> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org>
>> wrote:
>> > > > >>
>> > > > >>> Thanks for getting the ball rolling on this!
>> > > > >>>
>> > > > >>> Can the number of splits decrease? Yes, splits can be closed
>> and go
>> > > > away.
>> > > > >>> An example would be a shard merge in Kinesis (2 existing shards
>> > will
>> > > be
>> > > > >>> closed and replaced with a new shard).
>> > > > >>>
>> > > > >>> Regarding advance/poll/take: IMO the least restrictive approach
>> > would
>> > > > be
>> > > > >>> the thread-less IO model (pull based, non-blocking, caller
>> > retrieves
>> > > > new
>> > > > >>> records when available). The current Kinesis API requires the
>> use
>> > of
>> > > > >>> threads. But that can be internal to the split reader and does
>> not
>> > > need
>> > > > >> to
>> > > > >>> be a source API concern. In fact, that's what we are working on
>> > right
>> > > > now
>> > > > >>> as improvement to the existing consumer: Each shard consumer
>> thread
>> > > > will
>> > > > >>> push to a queue, the consumer main thread will poll the
>> queue(s).
>> > It
>> > > is
>> > > > >>> essentially a mapping from threaded IO to non-blocking.
>> > > > >>>
>> > > > >>> The proposed SplitReader interface would fit the thread-less IO
>> > > model.
>> > > > >>> Similar to an iterator, we find out if there is a new element
>> > > (hasNext)
>> > > > >> and
>> > > > >>> if so, move to it (next()). Separate calls deliver the meta
>> > > information
>> > > > >>> (timestamp, watermark). Perhaps advance call could offer a
>> timeout
>> > > > >> option,
>> > > > >>> so that the caller does not end up in a busy wait. On the other
>> > > hand, a
>> > > > >>> caller processing multiple splits may want to cycle through
>> fast,
>> > to
>> > > > >>> process elements of other splits as soon as they become
>> available.
>> > > The
>> > > > >> nice
>> > > > >>> thing is that this "split merge" logic can now live in Flink
>> and be
>> > > > >>> optimized and shared between different sources.
>> > > > >>>
>> > > > >>> Thanks,
>> > > > >>> Thomas
>> > > > >>>
>> > > > >>>
>> > > > >>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
>> > > wrote:
>> > > > >>>
>> > > > >>>> Hi,
>> > > > >>>> Thanks Aljoscha for this FLIP.
>> > > > >>>>
>> > > > >>>> 1. I agree with Piotr and Becket that the non-blocking source
>> is
>> > > very
>> > > > >>>> important. But in addition to `Future/poll`, there may be
>> another
>> > > way
>> > > > to
>> > > > >>>> achieve this. I think it may be not very memory friendly if
>> every
>> > > > >> advance
>> > > > >>>> call return a Future.
>> > > > >>>>
>> > > > >>>> public interface Listener {
>> > > > >>>>     public void notify();
>> > > > >>>> }
>> > > > >>>>
>> > > > >>>> public interface SplitReader() {
>> > > > >>>>     /**
>> > > > >>>>      * When there is no element temporarily, this will return
>> > false.
>> > > > >>>>      * When elements is available again splitReader can call
>> > > > >>>> listener.notify()
>> > > > >>>>      * In addition the frame would check `advance`
>> periodically .
>> > > > >>>>      * Of course advance can always return true and ignore the
>> > > > listener
>> > > > >>>> argument for simplicity.
>> > > > >>>>      */
>> > > > >>>>     public boolean advance(Listener listener);
>> > > > >>>> }
>> > > > >>>>
>> > > > >>>> 2.  The FLIP tells us very clearly that how to create all
>> Splits
>> > and
>> > > > how
>> > > > >>>> to create a SplitReader from a Split. But there is no strategy
>> for
>> > > the
>> > > > >> user
>> > > > >>>> to choose how to assign the splits to the tasks. I think we
>> could
>> > > add
>> > > > a
>> > > > >>>> Enum to let user to choose.
>> > > > >>>> /**
>> > > > >>>>  public Enum SplitsAssignmentPolicy {
>> > > > >>>>    Location,
>> > > > >>>>    Workload,
>> > > > >>>>    Random,
>> > > > >>>>    Average
>> > > > >>>>  }
>> > > > >>>> */
>> > > > >>>>
>> > > > >>>> 3. If merge the `advance` and `getCurrent`  to one method like
>> > > > `getNext`
>> > > > >>>> the `getNext` would need return a `ElementWithTimestamp`
>> because
>> > > some
>> > > > >>>> sources want to add timestamp to every element. IMO, this is
>> not
>> > so
>> > > > >> memory
>> > > > >>>> friendly so I prefer this design.
>> > > > >>>>
>> > > > >>>>
>> > > > >>>>
>> > > > >>>> Thanks
>> > > > >>>>
>> > > > >>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
>> 下午6:08写道:
>> > > > >>>>
>> > > > >>>>> Hi,
>> > > > >>>>>
>> > > > >>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot
>> of
>> > > other
>> > > > >>>>> possible improvements. I have one proposal. Instead of having
>> a
>> > > > method:
>> > > > >>>>>
>> > > > >>>>> boolean advance() throws IOException;
>> > > > >>>>>
>> > > > >>>>> I would replace it with
>> > > > >>>>>
>> > > > >>>>> /*
>> > > > >>>>> * Return a future, which when completed means that source has
>> > more
>> > > > >> data
>> > > > >>>>> and getNext() will not block.
>> > > > >>>>> * If you wish to use benefits of non blocking connectors,
>> please
>> > > > >>>>> implement this method appropriately.
>> > > > >>>>> */
>> > > > >>>>> default CompletableFuture<?> isBlocked() {
>> > > > >>>>>        return CompletableFuture.completedFuture(null);
>> > > > >>>>> }
>> > > > >>>>>
>> > > > >>>>> And rename `getCurrent()` to `getNext()`.
>> > > > >>>>>
>> > > > >>>>> Couple of arguments:
>> > > > >>>>> 1. I don’t understand the division of work between `advance()`
>> > and
>> > > > >>>>> `getCurrent()`. What should be done in which, especially for
>> > > > connectors
>> > > > >>>>> that handle records in batches (like Kafka) and when should
>> you
>> > > call
>> > > > >>>>> `advance` and when `getCurrent()`.
>> > > > >>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow
>> us
>> > in
>> > > > the
>> > > > >>>>> future to have asynchronous/non blocking connectors and more
>> > > > >> efficiently
>> > > > >>>>> handle large number of blocked threads, without busy waiting.
>> > While
>> > > > at
>> > > > >> the
>> > > > >>>>> same time it doesn’t add much complexity, since naive
>> connector
>> > > > >>>>> implementations can be always blocking.
>> > > > >>>>> 3. This also would allow us to use a fixed size thread pool of
>> > task
>> > > > >>>>> executors, instead of one thread per task.
>> > > > >>>>>
>> > > > >>>>> Piotrek
>> > > > >>>>>
>> > > > >>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
>> aljoscha@apache.org
>> > >
>> > > > >>>>> wrote:
>> > > > >>>>>>
>> > > > >>>>>> Hi All,
>> > > > >>>>>>
>> > > > >>>>>> In order to finally get the ball rolling on the new source
>> > > interface
>> > > > >>>>> that we have discussed for so long I finally created a FLIP:
>> > > > >>>>>
>> > > > >>
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>> > > > >>>>>>
>> > > > >>>>>> I cc'ed Thomas and Jamie because of the ongoing
>> work/discussion
>> > > > about
>> > > > >>>>> adding per-partition watermark support to the Kinesis source
>> and
>> > > > >> because
>> > > > >>>>> this would enable generic implementation of event-time
>> alignment
>> > > for
>> > > > >> all
>> > > > >>>>> sources. Maybe we need another FLIP for the event-time
>> alignment
>> > > > part,
>> > > > >>>>> especially the part about information sharing between
>> operations
>> > > (I'm
>> > > > >> not
>> > > > >>>>> calling it state sharing because state has a special meaning
>> in
>> > > > Flink).
>> > > > >>>>>>
>> > > > >>>>>> Please discuss away!
>> > > > >>>>>>
>> > > > >>>>>> Aljoscha
>> > > > >>>>>>
>> > > > >>>>>>
>> > > > >>>>>
>> > > > >>>>>
>> > > > >>
>> > > >
>> > > >
>> > >
>> >
>>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Biao Liu <mm...@gmail.com>.
Hi Kostas,

Regarding the checkpoint of "per thread for each split mode". IMO, there
are severals things source operator need to do.
1. Source operator need to record all splits in checkpoint. The unfinished
splits must be recorded. I'm not sure whether we could skip recording the
finished splits, it depends on split discovery implementation.
2. Source operator need to collect the last record polled from each split
queue. And put them into checkpoint.
3. SplitReader can be restored by giving a specific split with a position
of last record.

And I think you raised another important issue. The queue between task
thread and split readers.
1. I agree that it must be a thread-safe, size limited queue, such as
ArrayBlockingQueue.
2. Also it's hard to decide the size of queue. We have to consider the
split count, the size of item in queue to make sure the memory of source
operator will not be out of control. Giving a unified queue size is not
proper since there may be several different sources in one job. It's better
that each source can decide the queue size of itself.


Kostas Kloudas <k....@data-artisans.com> 于2018年11月26日周一 下午8:42写道:

> Hi all,
>
> From the discussion, I understand that we are leaning towards a design
> where the user writes a single-threaded SplitReader, which Flink executes
> on another thread (not the main task thread). This way the task can have
> multiple readers running concurrently, each one reading a different split.
>
> Each of these threads writes in its own queue. These queues are then polled
> by the main thread (based on a potentially user-defined prioritization),
> which is responsible for emitting data downstream. There were also
> proposals for a single shared queue, but I believe that 1) the contention
> for the lock in such a queue can be a limitation and 2) it is not easy to
> prioritise which elements to consume first (assuming that we want to
> support different prioritisation strategies).
>
> Assuming the above model, I have the following question:
>
> We have the split/shard/partition discovery logic outside the "reader"
> operator. For now it can be a plain old source function with parallelism of
> 1 that periodically checks for new splits (for an example see the existing
> ContinuousFileMonitoringFunction).[1]
>
> This source sends the split to be read downstream to the multi-threaded
> readers. In these settings, there must be a "throttling" or
> "rate-limitting" mechanism that guaranttees that we do not surpass the
> capabilities of the machines. The first thing that comes to mind is some
> kind of a fixed size (blocking) queue or a fixed size thread pool. The main
> thread adds splits to the queue and the readers consume them. When the
> queue or the pool is full, then we block (backpressure).
>
> In the case above, how do we make sure that the checkpoints still go
> through?
>
> Cheers,
> Kostas
>
> PS: I am assuming the current task implementation and not an "actor" based
> one.
>
> *[1] The ContinuousFileReaderOperator has a single thread (different from
> the main task thread) consuming the splits one by one. Unfortunately, there
> is no rate-limiting mechanism.
>
>
> On Sun, Nov 25, 2018 at 6:40 PM Biao Liu <mm...@gmail.com> wrote:
>
> > Hi community,
> > Glad to see this topic is still so active.
> >
> > Thanks for replying @Piotrek and @Becket.
> >
> > Last time, I expressed some rough ideas about the thread model. However I
> > found that it's hard to describe clearly in mailing list. So I wrote it
> > down with some graphs, exampled some kinds of models, see Thread Model of
> > Source
> > <
> >
> https://docs.google.com/document/d/1XpYkkJo97CUw-UMVrKU6b0ZZuJJ2V7mBb__L6UzdWTw/edit?usp=sharing
> > >.
> > I wish that can be helpful.
> >
> > IMO thread model is an important part. Without thinking of implementation
> > clearly, it's difficult to decide what the up level interface should look
> > like.
> > It would be better if we draw the whole picture first and then fill the
> > detail parts one by one.
> >
> > @Piotrek About adding new splits to existing split reader. It's an
> > interesting idea. Not only for solving too many threads problem, but also
> > for supporting some more complicated system. I know in some storage
> > systems, there is some scenario which the partition is
> dynamic(dynamically
> > splitting or merging). Though I have not think of it very clearly now. I
> > would give you more detailed reply asap :)
> >
> >
> > Guowei Ma <gu...@gmail.com> 于2018年11月23日周五 下午6:37写道:
> >
> > > Hi,Piotr
> > > Sorry  for so late to response.
> > >
> > >
> > > First of all I think Flink runtime can assigned a thread for a
> > StreamTask,
> > > which likes  'Actor' model. The number of threads for a StreamTask
> should
> > > not be proportional to the operator or other things. This will give
> Flink
> > > the ability to scale horizontally. So I think it's not just the
> > > network(flush),checkpoint and  source, but some operators' threads can
> > also
> > > be removed in the future, like AsyncWaitOperator.
> > >
> > >
> > >
> > > for b)
> > > When using event time, some sources want to assign a timestamp to each
> > > element. In current Flink interface, user will write like this
> > > public class EventTimeSource<Element> implements SourceFunction {
> > >   public void run() {
> > >      while(...){
> > >          Element record = // get from file or some queue;
> > >          long timestamp = parseTimestampFromElement(record);
> > >          sourceContext.collectWithTimestamp(record, timestamp);
> > >      }
> > >   }
> > > }
> > > Using the interfaces from this FLIP, user can write like this
> > >
> > > public EventTimeSplitReader<Element,X> implements SplitReader {
> > >     Element currentRecord = null;
> > >
> > >
> > >     // Please ignoring the handling of boundary conditions
> > >     public boolean advace(){
> > >        currentRecord = //move a pointer forward
> > >        return true;
> > >      }
> > >
> > >     public Element getCurrent(){
> > >        return currentRecord;
> > >     }
> > >     public long getCurrentTimestamp() {
> > >       return parseTimestampFromElement(currentRecord);
> > >     }
> > > }
> > >
> > > if merging the advance/getNext to a method like getNext() , the
> > SplitReader
> > > interface may need to change a little like this
> > >
> > > public interface SplitReader2<T,X> {
> > > public class ElementWithTimestamp {
> > >     T element;
> > >     long timestamp;
> > > }
> > >
> > > public ElementWithTimestamp getNext() ;
> > >
> > > }
> > > Now user may need implement the source like this
> > > public EventTimeSplitReader<Element,X> implements SplitReader2 {
> > >     Element currentRecord = null;
> > >
> > >     // Please ignoring the handling of boundary conditions
> > >     public ElementWithTimestamp getCurrent(){
> > >        return new ElementWithTimestamp(currentRecord,
> > > parseTimestampFromElement(currentRecord))
> > >     }
> > > }
> > > The user can use a constant ElementWithTimestamp but I think this need
> > the
> > > every connector developers to know this trick. The current Flip will
> not
> > > have this burden.
> > > Maybe there has other way like '' void
> getCurrent(ElementWithTimestamp)"
> > > to avoid creating a new object.  But my personal preference is
> > > ‘advance/getCurrent’.
> > >
> > >
> > >
> > > Piotr Nowojski <pi...@data-artisans.com> 于2018年11月7日周三 下午4:31写道:
> > >
> > > > Hi,
> > > >
> > > > a)
> > > >
> > > > > BTW, regarding the isBlock() method, I have a few more questions.
> 21,
> > > Is
> > > > a method isReady() with boolean as a return value
> > > > > equivalent? Personally I found it is a little bit confusing in what
> > is
> > > > supposed to be returned when the future is completed. 22. if
> > > > > the implementation of isBlocked() is optional, how do the callers
> > know
> > > > whether the method is properly implemented or not?
> > > > > Does not implemented mean it always return a completed future?
> > > >
> > > > `CompletableFuture<?> isBlocked()` is more or less an equivalent to
> > > > `boolean hasNext()` which in case of “false” provides some kind of a
> > > > listener/callback that notifies about presence of next element. There
> > are
> > > > some minor details, like `CompletableFuture<?>` has a minimal two
> state
> > > > logic:
> > > >
> > > > 1. Future is completed - we have more data
> > > > 2. Future not yet completed - we don’t have data now, but we might/we
> > > will
> > > > have in the future
> > > >
> > > > While `boolean hasNext()` and `notify()` callback are a bit more
> > > > complicated/dispersed and can lead/encourage `notify()` spam.
> > > >
> > > > b)
> > > >
> > > > > 3. If merge the `advance` and `getCurrent`  to one method like
> > > `getNext`
> > > > the `getNext` would need return a
> > > > >`ElementWithTimestamp` because some sources want to add timestamp to
> > > > every element. IMO, this is not so memory friendly
> > > > > so I prefer this design.
> > > >
> > > > Guowei I don’t quite understand this. Could you elaborate why having
> a
> > > > separate `advance()` help?
> > > >
> > > > c)
> > > >
> > > > Regarding advance/poll/take. What’s the value of having two separate
> > > > methods: poll and take? Which one of them should be called and which
> > > > implemented? What’s the benefit of having those methods compared to
> > > having
> > > > a one single method `getNextElement()` (or `pollElement() or whatever
> > we
> > > > name it) with following contract:
> > > >
> > > > CompletableFuture<?> isBlocked();
> > > >
> > > > /**
> > > > Return next element - will be called only if `isBlocked()` is
> > completed.
> > > > Try to implement it in non blocking fashion, but if that’s impossible
> > or
> > > > you just don’t need the effort, you can block in this method.
> > > > */
> > > > T getNextElement();
> > > >
> > > > I mean, if the connector is implemented non-blockingly, Flink should
> > use
> > > > it that way. If it’s not, then `poll()` will `throw new
> > > > NotImplementedException()`. Implementing both of them and providing
> > both
> > > of
> > > > them to Flink wouldn’t make a sense, thus why not merge them into a
> > > single
> > > > method call that should preferably (but not necessarily need to) be
> > > > non-blocking? It’s not like we are implementing general purpose
> > `Queue`,
> > > > which users might want to call either of `poll` or `take`. We would
> > > always
> > > > prefer to call `poll`, but if it’s blocking, then still we have no
> > > choice,
> > > > but to call it and block on it.
> > > >
> > > > d)
> > > >
> > > > > 1. I agree with Piotr and Becket that the non-blocking source is
> very
> > > > > important. But in addition to `Future/poll`, there may be another
> way
> > > to
> > > > > achieve this. I think it may be not very memory friendly if every
> > > advance
> > > > > call return a Future.
> > > >
> > > > I didn’t want to mention this, to not clog my initial proposal, but
> > there
> > > > is a simple solution for the problem:
> > > >
> > > > public interface SplitReader {
> > > >
> > > >     (…)
> > > >
> > > >     CompletableFuture<?> NOT_BLOCKED =
> > > > CompletableFuture.completedFuture(null);
> > > >
> > > >     /**
> > > >      * Returns a future that will be completed when the page source
> > > becomes
> > > >      * unblocked.  If the page source is not blocked, this method
> > should
> > > > return
> > > >      * {@code NOT_BLOCKED}.
> > > >      */
> > > >     default CompletableFuture<?> isBlocked()
> > > >     {
> > > >         return NOT_BLOCKED;
> > > >     }
> > > >
> > > > If we are blocked and we are waiting for the IO, then creating a new
> > > > Future is non-issue. Under full throttle/throughput and not blocked
> > > sources
> > > > returning a static `NOT_BLOCKED` constant  should also solve the
> > problem.
> > > >
> > > > One more remark, non-blocking sources might be a necessity in a
> single
> > > > threaded model without a checkpointing lock. (Currently when sources
> > are
> > > > blocked, they can release checkpointing lock and re-acquire it again
> > > > later). Non-blocking `poll`/`getNext()` would allow for checkpoints
> to
> > > > happen when source is idling. In that case either `notify()` or my
> > > proposed
> > > > `isBlocked()` would allow to avoid busy-looping.
> > > >
> > > > Piotrek
> > > >
> > > > > On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
> > > > >
> > > > > Hi Thomas,
> > > > >
> > > > > The iterator-like API was also the first thing that came to me. But
> > it
> > > > > seems a little confusing that hasNext() does not mean "the stream
> has
> > > not
> > > > > ended", but means "the next record is ready", which is repurposing
> > the
> > > > well
> > > > > known meaning of hasNext(). If we follow the hasNext()/next()
> > pattern,
> > > an
> > > > > additional isNextReady() method to indicate whether the next record
> > is
> > > > > ready seems more intuitive to me.
> > > > >
> > > > > Similarly, in poll()/take() pattern, another method of isDone() is
> > > needed
> > > > > to indicate whether the stream has ended or not.
> > > > >
> > > > > Compared with hasNext()/next()/isNextReady() pattern,
> > > > > isDone()/poll()/take() seems more flexible for the reader
> > > implementation.
> > > > > When I am implementing a reader, I could have a couple of choices:
> > > > >
> > > > >   - A thread-less reader that does not have any internal thread.
> > > > >   - When poll() is called, the same calling thread will perform a
> > bunch
> > > > of
> > > > >      IO asynchronously.
> > > > >      - When take() is called, the same calling thread will perform
> a
> > > > bunch
> > > > >      of IO and wait until the record is ready.
> > > > >   - A reader with internal threads performing network IO and put
> > > records
> > > > >   into a buffer.
> > > > >      - When poll() is called, the calling thread simply reads from
> > the
> > > > >      buffer and return empty result immediately if there is no
> > record.
> > > > >      - When take() is called, the calling thread reads from the
> > buffer
> > > > and
> > > > >      block waiting if the buffer is empty.
> > > > >
> > > > > On the other hand, with the hasNext()/next()/isNextReady() API, it
> is
> > > > less
> > > > > intuitive for the reader developers to write the thread-less
> pattern.
> > > > > Although technically speaking one can still do the asynchronous IO
> to
> > > > > prepare the record in isNextReady(). But it is inexplicit and seems
> > > > > somewhat hacky.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jiangjie (Becket) Qin
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org>
> wrote:
> > > > >
> > > > >> Couple more points regarding discovery:
> > > > >>
> > > > >> The proposal mentions that discovery could be outside the
> execution
> > > > graph.
> > > > >> Today, discovered partitions/shards are checkpointed. I believe
> that
> > > > will
> > > > >> also need to be the case in the future, even when discovery and
> > > reading
> > > > are
> > > > >> split between different tasks.
> > > > >>
> > > > >> For cases such as resharding of a Kinesis stream, the relationship
> > > > between
> > > > >> splits needs to be considered. Splits cannot be randomly
> distributed
> > > > over
> > > > >> readers in certain situations. An example was mentioned here:
> > > > >> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> > > > >>
> > > > >> Thomas
> > > > >>
> > > > >>
> > > > >> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org>
> wrote:
> > > > >>
> > > > >>> Thanks for getting the ball rolling on this!
> > > > >>>
> > > > >>> Can the number of splits decrease? Yes, splits can be closed and
> go
> > > > away.
> > > > >>> An example would be a shard merge in Kinesis (2 existing shards
> > will
> > > be
> > > > >>> closed and replaced with a new shard).
> > > > >>>
> > > > >>> Regarding advance/poll/take: IMO the least restrictive approach
> > would
> > > > be
> > > > >>> the thread-less IO model (pull based, non-blocking, caller
> > retrieves
> > > > new
> > > > >>> records when available). The current Kinesis API requires the use
> > of
> > > > >>> threads. But that can be internal to the split reader and does
> not
> > > need
> > > > >> to
> > > > >>> be a source API concern. In fact, that's what we are working on
> > right
> > > > now
> > > > >>> as improvement to the existing consumer: Each shard consumer
> thread
> > > > will
> > > > >>> push to a queue, the consumer main thread will poll the queue(s).
> > It
> > > is
> > > > >>> essentially a mapping from threaded IO to non-blocking.
> > > > >>>
> > > > >>> The proposed SplitReader interface would fit the thread-less IO
> > > model.
> > > > >>> Similar to an iterator, we find out if there is a new element
> > > (hasNext)
> > > > >> and
> > > > >>> if so, move to it (next()). Separate calls deliver the meta
> > > information
> > > > >>> (timestamp, watermark). Perhaps advance call could offer a
> timeout
> > > > >> option,
> > > > >>> so that the caller does not end up in a busy wait. On the other
> > > hand, a
> > > > >>> caller processing multiple splits may want to cycle through fast,
> > to
> > > > >>> process elements of other splits as soon as they become
> available.
> > > The
> > > > >> nice
> > > > >>> thing is that this "split merge" logic can now live in Flink and
> be
> > > > >>> optimized and shared between different sources.
> > > > >>>
> > > > >>> Thanks,
> > > > >>> Thomas
> > > > >>>
> > > > >>>
> > > > >>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
> > > wrote:
> > > > >>>
> > > > >>>> Hi,
> > > > >>>> Thanks Aljoscha for this FLIP.
> > > > >>>>
> > > > >>>> 1. I agree with Piotr and Becket that the non-blocking source is
> > > very
> > > > >>>> important. But in addition to `Future/poll`, there may be
> another
> > > way
> > > > to
> > > > >>>> achieve this. I think it may be not very memory friendly if
> every
> > > > >> advance
> > > > >>>> call return a Future.
> > > > >>>>
> > > > >>>> public interface Listener {
> > > > >>>>     public void notify();
> > > > >>>> }
> > > > >>>>
> > > > >>>> public interface SplitReader() {
> > > > >>>>     /**
> > > > >>>>      * When there is no element temporarily, this will return
> > false.
> > > > >>>>      * When elements is available again splitReader can call
> > > > >>>> listener.notify()
> > > > >>>>      * In addition the frame would check `advance` periodically
> .
> > > > >>>>      * Of course advance can always return true and ignore the
> > > > listener
> > > > >>>> argument for simplicity.
> > > > >>>>      */
> > > > >>>>     public boolean advance(Listener listener);
> > > > >>>> }
> > > > >>>>
> > > > >>>> 2.  The FLIP tells us very clearly that how to create all Splits
> > and
> > > > how
> > > > >>>> to create a SplitReader from a Split. But there is no strategy
> for
> > > the
> > > > >> user
> > > > >>>> to choose how to assign the splits to the tasks. I think we
> could
> > > add
> > > > a
> > > > >>>> Enum to let user to choose.
> > > > >>>> /**
> > > > >>>>  public Enum SplitsAssignmentPolicy {
> > > > >>>>    Location,
> > > > >>>>    Workload,
> > > > >>>>    Random,
> > > > >>>>    Average
> > > > >>>>  }
> > > > >>>> */
> > > > >>>>
> > > > >>>> 3. If merge the `advance` and `getCurrent`  to one method like
> > > > `getNext`
> > > > >>>> the `getNext` would need return a `ElementWithTimestamp` because
> > > some
> > > > >>>> sources want to add timestamp to every element. IMO, this is not
> > so
> > > > >> memory
> > > > >>>> friendly so I prefer this design.
> > > > >>>>
> > > > >>>>
> > > > >>>>
> > > > >>>> Thanks
> > > > >>>>
> > > > >>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四
> 下午6:08写道:
> > > > >>>>
> > > > >>>>> Hi,
> > > > >>>>>
> > > > >>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
> > > other
> > > > >>>>> possible improvements. I have one proposal. Instead of having a
> > > > method:
> > > > >>>>>
> > > > >>>>> boolean advance() throws IOException;
> > > > >>>>>
> > > > >>>>> I would replace it with
> > > > >>>>>
> > > > >>>>> /*
> > > > >>>>> * Return a future, which when completed means that source has
> > more
> > > > >> data
> > > > >>>>> and getNext() will not block.
> > > > >>>>> * If you wish to use benefits of non blocking connectors,
> please
> > > > >>>>> implement this method appropriately.
> > > > >>>>> */
> > > > >>>>> default CompletableFuture<?> isBlocked() {
> > > > >>>>>        return CompletableFuture.completedFuture(null);
> > > > >>>>> }
> > > > >>>>>
> > > > >>>>> And rename `getCurrent()` to `getNext()`.
> > > > >>>>>
> > > > >>>>> Couple of arguments:
> > > > >>>>> 1. I don’t understand the division of work between `advance()`
> > and
> > > > >>>>> `getCurrent()`. What should be done in which, especially for
> > > > connectors
> > > > >>>>> that handle records in batches (like Kafka) and when should you
> > > call
> > > > >>>>> `advance` and when `getCurrent()`.
> > > > >>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow
> us
> > in
> > > > the
> > > > >>>>> future to have asynchronous/non blocking connectors and more
> > > > >> efficiently
> > > > >>>>> handle large number of blocked threads, without busy waiting.
> > While
> > > > at
> > > > >> the
> > > > >>>>> same time it doesn’t add much complexity, since naive connector
> > > > >>>>> implementations can be always blocking.
> > > > >>>>> 3. This also would allow us to use a fixed size thread pool of
> > task
> > > > >>>>> executors, instead of one thread per task.
> > > > >>>>>
> > > > >>>>> Piotrek
> > > > >>>>>
> > > > >>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
> aljoscha@apache.org
> > >
> > > > >>>>> wrote:
> > > > >>>>>>
> > > > >>>>>> Hi All,
> > > > >>>>>>
> > > > >>>>>> In order to finally get the ball rolling on the new source
> > > interface
> > > > >>>>> that we have discussed for so long I finally created a FLIP:
> > > > >>>>>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > > >>>>>>
> > > > >>>>>> I cc'ed Thomas and Jamie because of the ongoing
> work/discussion
> > > > about
> > > > >>>>> adding per-partition watermark support to the Kinesis source
> and
> > > > >> because
> > > > >>>>> this would enable generic implementation of event-time
> alignment
> > > for
> > > > >> all
> > > > >>>>> sources. Maybe we need another FLIP for the event-time
> alignment
> > > > part,
> > > > >>>>> especially the part about information sharing between
> operations
> > > (I'm
> > > > >> not
> > > > >>>>> calling it state sharing because state has a special meaning in
> > > > Flink).
> > > > >>>>>>
> > > > >>>>>> Please discuss away!
> > > > >>>>>>
> > > > >>>>>> Aljoscha
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>>>>
> > > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Kostas Kloudas <k....@data-artisans.com>.
Hi all,

From the discussion, I understand that we are leaning towards a design
where the user writes a single-threaded SplitReader, which Flink executes
on another thread (not the main task thread). This way the task can have
multiple readers running concurrently, each one reading a different split.

Each of these threads writes in its own queue. These queues are then polled
by the main thread (based on a potentially user-defined prioritization),
which is responsible for emitting data downstream. There were also
proposals for a single shared queue, but I believe that 1) the contention
for the lock in such a queue can be a limitation and 2) it is not easy to
prioritise which elements to consume first (assuming that we want to
support different prioritisation strategies).

Assuming the above model, I have the following question:

We have the split/shard/partition discovery logic outside the "reader"
operator. For now it can be a plain old source function with parallelism of
1 that periodically checks for new splits (for an example see the existing
ContinuousFileMonitoringFunction).[1]

This source sends the split to be read downstream to the multi-threaded
readers. In these settings, there must be a "throttling" or
"rate-limitting" mechanism that guaranttees that we do not surpass the
capabilities of the machines. The first thing that comes to mind is some
kind of a fixed size (blocking) queue or a fixed size thread pool. The main
thread adds splits to the queue and the readers consume them. When the
queue or the pool is full, then we block (backpressure).

In the case above, how do we make sure that the checkpoints still go
through?

Cheers,
Kostas

PS: I am assuming the current task implementation and not an "actor" based
one.

*[1] The ContinuousFileReaderOperator has a single thread (different from
the main task thread) consuming the splits one by one. Unfortunately, there
is no rate-limiting mechanism.


On Sun, Nov 25, 2018 at 6:40 PM Biao Liu <mm...@gmail.com> wrote:

> Hi community,
> Glad to see this topic is still so active.
>
> Thanks for replying @Piotrek and @Becket.
>
> Last time, I expressed some rough ideas about the thread model. However I
> found that it's hard to describe clearly in mailing list. So I wrote it
> down with some graphs, exampled some kinds of models, see Thread Model of
> Source
> <
> https://docs.google.com/document/d/1XpYkkJo97CUw-UMVrKU6b0ZZuJJ2V7mBb__L6UzdWTw/edit?usp=sharing
> >.
> I wish that can be helpful.
>
> IMO thread model is an important part. Without thinking of implementation
> clearly, it's difficult to decide what the up level interface should look
> like.
> It would be better if we draw the whole picture first and then fill the
> detail parts one by one.
>
> @Piotrek About adding new splits to existing split reader. It's an
> interesting idea. Not only for solving too many threads problem, but also
> for supporting some more complicated system. I know in some storage
> systems, there is some scenario which the partition is dynamic(dynamically
> splitting or merging). Though I have not think of it very clearly now. I
> would give you more detailed reply asap :)
>
>
> Guowei Ma <gu...@gmail.com> 于2018年11月23日周五 下午6:37写道:
>
> > Hi,Piotr
> > Sorry  for so late to response.
> >
> >
> > First of all I think Flink runtime can assigned a thread for a
> StreamTask,
> > which likes  'Actor' model. The number of threads for a StreamTask should
> > not be proportional to the operator or other things. This will give Flink
> > the ability to scale horizontally. So I think it's not just the
> > network(flush),checkpoint and  source, but some operators' threads can
> also
> > be removed in the future, like AsyncWaitOperator.
> >
> >
> >
> > for b)
> > When using event time, some sources want to assign a timestamp to each
> > element. In current Flink interface, user will write like this
> > public class EventTimeSource<Element> implements SourceFunction {
> >   public void run() {
> >      while(...){
> >          Element record = // get from file or some queue;
> >          long timestamp = parseTimestampFromElement(record);
> >          sourceContext.collectWithTimestamp(record, timestamp);
> >      }
> >   }
> > }
> > Using the interfaces from this FLIP, user can write like this
> >
> > public EventTimeSplitReader<Element,X> implements SplitReader {
> >     Element currentRecord = null;
> >
> >
> >     // Please ignoring the handling of boundary conditions
> >     public boolean advace(){
> >        currentRecord = //move a pointer forward
> >        return true;
> >      }
> >
> >     public Element getCurrent(){
> >        return currentRecord;
> >     }
> >     public long getCurrentTimestamp() {
> >       return parseTimestampFromElement(currentRecord);
> >     }
> > }
> >
> > if merging the advance/getNext to a method like getNext() , the
> SplitReader
> > interface may need to change a little like this
> >
> > public interface SplitReader2<T,X> {
> > public class ElementWithTimestamp {
> >     T element;
> >     long timestamp;
> > }
> >
> > public ElementWithTimestamp getNext() ;
> >
> > }
> > Now user may need implement the source like this
> > public EventTimeSplitReader<Element,X> implements SplitReader2 {
> >     Element currentRecord = null;
> >
> >     // Please ignoring the handling of boundary conditions
> >     public ElementWithTimestamp getCurrent(){
> >        return new ElementWithTimestamp(currentRecord,
> > parseTimestampFromElement(currentRecord))
> >     }
> > }
> > The user can use a constant ElementWithTimestamp but I think this need
> the
> > every connector developers to know this trick. The current Flip will not
> > have this burden.
> > Maybe there has other way like '' void getCurrent(ElementWithTimestamp)"
> > to avoid creating a new object.  But my personal preference is
> > ‘advance/getCurrent’.
> >
> >
> >
> > Piotr Nowojski <pi...@data-artisans.com> 于2018年11月7日周三 下午4:31写道:
> >
> > > Hi,
> > >
> > > a)
> > >
> > > > BTW, regarding the isBlock() method, I have a few more questions. 21,
> > Is
> > > a method isReady() with boolean as a return value
> > > > equivalent? Personally I found it is a little bit confusing in what
> is
> > > supposed to be returned when the future is completed. 22. if
> > > > the implementation of isBlocked() is optional, how do the callers
> know
> > > whether the method is properly implemented or not?
> > > > Does not implemented mean it always return a completed future?
> > >
> > > `CompletableFuture<?> isBlocked()` is more or less an equivalent to
> > > `boolean hasNext()` which in case of “false” provides some kind of a
> > > listener/callback that notifies about presence of next element. There
> are
> > > some minor details, like `CompletableFuture<?>` has a minimal two state
> > > logic:
> > >
> > > 1. Future is completed - we have more data
> > > 2. Future not yet completed - we don’t have data now, but we might/we
> > will
> > > have in the future
> > >
> > > While `boolean hasNext()` and `notify()` callback are a bit more
> > > complicated/dispersed and can lead/encourage `notify()` spam.
> > >
> > > b)
> > >
> > > > 3. If merge the `advance` and `getCurrent`  to one method like
> > `getNext`
> > > the `getNext` would need return a
> > > >`ElementWithTimestamp` because some sources want to add timestamp to
> > > every element. IMO, this is not so memory friendly
> > > > so I prefer this design.
> > >
> > > Guowei I don’t quite understand this. Could you elaborate why having a
> > > separate `advance()` help?
> > >
> > > c)
> > >
> > > Regarding advance/poll/take. What’s the value of having two separate
> > > methods: poll and take? Which one of them should be called and which
> > > implemented? What’s the benefit of having those methods compared to
> > having
> > > a one single method `getNextElement()` (or `pollElement() or whatever
> we
> > > name it) with following contract:
> > >
> > > CompletableFuture<?> isBlocked();
> > >
> > > /**
> > > Return next element - will be called only if `isBlocked()` is
> completed.
> > > Try to implement it in non blocking fashion, but if that’s impossible
> or
> > > you just don’t need the effort, you can block in this method.
> > > */
> > > T getNextElement();
> > >
> > > I mean, if the connector is implemented non-blockingly, Flink should
> use
> > > it that way. If it’s not, then `poll()` will `throw new
> > > NotImplementedException()`. Implementing both of them and providing
> both
> > of
> > > them to Flink wouldn’t make a sense, thus why not merge them into a
> > single
> > > method call that should preferably (but not necessarily need to) be
> > > non-blocking? It’s not like we are implementing general purpose
> `Queue`,
> > > which users might want to call either of `poll` or `take`. We would
> > always
> > > prefer to call `poll`, but if it’s blocking, then still we have no
> > choice,
> > > but to call it and block on it.
> > >
> > > d)
> > >
> > > > 1. I agree with Piotr and Becket that the non-blocking source is very
> > > > important. But in addition to `Future/poll`, there may be another way
> > to
> > > > achieve this. I think it may be not very memory friendly if every
> > advance
> > > > call return a Future.
> > >
> > > I didn’t want to mention this, to not clog my initial proposal, but
> there
> > > is a simple solution for the problem:
> > >
> > > public interface SplitReader {
> > >
> > >     (…)
> > >
> > >     CompletableFuture<?> NOT_BLOCKED =
> > > CompletableFuture.completedFuture(null);
> > >
> > >     /**
> > >      * Returns a future that will be completed when the page source
> > becomes
> > >      * unblocked.  If the page source is not blocked, this method
> should
> > > return
> > >      * {@code NOT_BLOCKED}.
> > >      */
> > >     default CompletableFuture<?> isBlocked()
> > >     {
> > >         return NOT_BLOCKED;
> > >     }
> > >
> > > If we are blocked and we are waiting for the IO, then creating a new
> > > Future is non-issue. Under full throttle/throughput and not blocked
> > sources
> > > returning a static `NOT_BLOCKED` constant  should also solve the
> problem.
> > >
> > > One more remark, non-blocking sources might be a necessity in a single
> > > threaded model without a checkpointing lock. (Currently when sources
> are
> > > blocked, they can release checkpointing lock and re-acquire it again
> > > later). Non-blocking `poll`/`getNext()` would allow for checkpoints to
> > > happen when source is idling. In that case either `notify()` or my
> > proposed
> > > `isBlocked()` would allow to avoid busy-looping.
> > >
> > > Piotrek
> > >
> > > > On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
> > > >
> > > > Hi Thomas,
> > > >
> > > > The iterator-like API was also the first thing that came to me. But
> it
> > > > seems a little confusing that hasNext() does not mean "the stream has
> > not
> > > > ended", but means "the next record is ready", which is repurposing
> the
> > > well
> > > > known meaning of hasNext(). If we follow the hasNext()/next()
> pattern,
> > an
> > > > additional isNextReady() method to indicate whether the next record
> is
> > > > ready seems more intuitive to me.
> > > >
> > > > Similarly, in poll()/take() pattern, another method of isDone() is
> > needed
> > > > to indicate whether the stream has ended or not.
> > > >
> > > > Compared with hasNext()/next()/isNextReady() pattern,
> > > > isDone()/poll()/take() seems more flexible for the reader
> > implementation.
> > > > When I am implementing a reader, I could have a couple of choices:
> > > >
> > > >   - A thread-less reader that does not have any internal thread.
> > > >   - When poll() is called, the same calling thread will perform a
> bunch
> > > of
> > > >      IO asynchronously.
> > > >      - When take() is called, the same calling thread will perform a
> > > bunch
> > > >      of IO and wait until the record is ready.
> > > >   - A reader with internal threads performing network IO and put
> > records
> > > >   into a buffer.
> > > >      - When poll() is called, the calling thread simply reads from
> the
> > > >      buffer and return empty result immediately if there is no
> record.
> > > >      - When take() is called, the calling thread reads from the
> buffer
> > > and
> > > >      block waiting if the buffer is empty.
> > > >
> > > > On the other hand, with the hasNext()/next()/isNextReady() API, it is
> > > less
> > > > intuitive for the reader developers to write the thread-less pattern.
> > > > Although technically speaking one can still do the asynchronous IO to
> > > > prepare the record in isNextReady(). But it is inexplicit and seems
> > > > somewhat hacky.
> > > >
> > > > Thanks,
> > > >
> > > > Jiangjie (Becket) Qin
> > > >
> > > >
> > > >
> > > > On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
> > > >
> > > >> Couple more points regarding discovery:
> > > >>
> > > >> The proposal mentions that discovery could be outside the execution
> > > graph.
> > > >> Today, discovered partitions/shards are checkpointed. I believe that
> > > will
> > > >> also need to be the case in the future, even when discovery and
> > reading
> > > are
> > > >> split between different tasks.
> > > >>
> > > >> For cases such as resharding of a Kinesis stream, the relationship
> > > between
> > > >> splits needs to be considered. Splits cannot be randomly distributed
> > > over
> > > >> readers in certain situations. An example was mentioned here:
> > > >> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> > > >>
> > > >> Thomas
> > > >>
> > > >>
> > > >> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
> > > >>
> > > >>> Thanks for getting the ball rolling on this!
> > > >>>
> > > >>> Can the number of splits decrease? Yes, splits can be closed and go
> > > away.
> > > >>> An example would be a shard merge in Kinesis (2 existing shards
> will
> > be
> > > >>> closed and replaced with a new shard).
> > > >>>
> > > >>> Regarding advance/poll/take: IMO the least restrictive approach
> would
> > > be
> > > >>> the thread-less IO model (pull based, non-blocking, caller
> retrieves
> > > new
> > > >>> records when available). The current Kinesis API requires the use
> of
> > > >>> threads. But that can be internal to the split reader and does not
> > need
> > > >> to
> > > >>> be a source API concern. In fact, that's what we are working on
> right
> > > now
> > > >>> as improvement to the existing consumer: Each shard consumer thread
> > > will
> > > >>> push to a queue, the consumer main thread will poll the queue(s).
> It
> > is
> > > >>> essentially a mapping from threaded IO to non-blocking.
> > > >>>
> > > >>> The proposed SplitReader interface would fit the thread-less IO
> > model.
> > > >>> Similar to an iterator, we find out if there is a new element
> > (hasNext)
> > > >> and
> > > >>> if so, move to it (next()). Separate calls deliver the meta
> > information
> > > >>> (timestamp, watermark). Perhaps advance call could offer a timeout
> > > >> option,
> > > >>> so that the caller does not end up in a busy wait. On the other
> > hand, a
> > > >>> caller processing multiple splits may want to cycle through fast,
> to
> > > >>> process elements of other splits as soon as they become available.
> > The
> > > >> nice
> > > >>> thing is that this "split merge" logic can now live in Flink and be
> > > >>> optimized and shared between different sources.
> > > >>>
> > > >>> Thanks,
> > > >>> Thomas
> > > >>>
> > > >>>
> > > >>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
> > wrote:
> > > >>>
> > > >>>> Hi,
> > > >>>> Thanks Aljoscha for this FLIP.
> > > >>>>
> > > >>>> 1. I agree with Piotr and Becket that the non-blocking source is
> > very
> > > >>>> important. But in addition to `Future/poll`, there may be another
> > way
> > > to
> > > >>>> achieve this. I think it may be not very memory friendly if every
> > > >> advance
> > > >>>> call return a Future.
> > > >>>>
> > > >>>> public interface Listener {
> > > >>>>     public void notify();
> > > >>>> }
> > > >>>>
> > > >>>> public interface SplitReader() {
> > > >>>>     /**
> > > >>>>      * When there is no element temporarily, this will return
> false.
> > > >>>>      * When elements is available again splitReader can call
> > > >>>> listener.notify()
> > > >>>>      * In addition the frame would check `advance` periodically .
> > > >>>>      * Of course advance can always return true and ignore the
> > > listener
> > > >>>> argument for simplicity.
> > > >>>>      */
> > > >>>>     public boolean advance(Listener listener);
> > > >>>> }
> > > >>>>
> > > >>>> 2.  The FLIP tells us very clearly that how to create all Splits
> and
> > > how
> > > >>>> to create a SplitReader from a Split. But there is no strategy for
> > the
> > > >> user
> > > >>>> to choose how to assign the splits to the tasks. I think we could
> > add
> > > a
> > > >>>> Enum to let user to choose.
> > > >>>> /**
> > > >>>>  public Enum SplitsAssignmentPolicy {
> > > >>>>    Location,
> > > >>>>    Workload,
> > > >>>>    Random,
> > > >>>>    Average
> > > >>>>  }
> > > >>>> */
> > > >>>>
> > > >>>> 3. If merge the `advance` and `getCurrent`  to one method like
> > > `getNext`
> > > >>>> the `getNext` would need return a `ElementWithTimestamp` because
> > some
> > > >>>> sources want to add timestamp to every element. IMO, this is not
> so
> > > >> memory
> > > >>>> friendly so I prefer this design.
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>> Thanks
> > > >>>>
> > > >>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
> > > >>>>
> > > >>>>> Hi,
> > > >>>>>
> > > >>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
> > other
> > > >>>>> possible improvements. I have one proposal. Instead of having a
> > > method:
> > > >>>>>
> > > >>>>> boolean advance() throws IOException;
> > > >>>>>
> > > >>>>> I would replace it with
> > > >>>>>
> > > >>>>> /*
> > > >>>>> * Return a future, which when completed means that source has
> more
> > > >> data
> > > >>>>> and getNext() will not block.
> > > >>>>> * If you wish to use benefits of non blocking connectors, please
> > > >>>>> implement this method appropriately.
> > > >>>>> */
> > > >>>>> default CompletableFuture<?> isBlocked() {
> > > >>>>>        return CompletableFuture.completedFuture(null);
> > > >>>>> }
> > > >>>>>
> > > >>>>> And rename `getCurrent()` to `getNext()`.
> > > >>>>>
> > > >>>>> Couple of arguments:
> > > >>>>> 1. I don’t understand the division of work between `advance()`
> and
> > > >>>>> `getCurrent()`. What should be done in which, especially for
> > > connectors
> > > >>>>> that handle records in batches (like Kafka) and when should you
> > call
> > > >>>>> `advance` and when `getCurrent()`.
> > > >>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us
> in
> > > the
> > > >>>>> future to have asynchronous/non blocking connectors and more
> > > >> efficiently
> > > >>>>> handle large number of blocked threads, without busy waiting.
> While
> > > at
> > > >> the
> > > >>>>> same time it doesn’t add much complexity, since naive connector
> > > >>>>> implementations can be always blocking.
> > > >>>>> 3. This also would allow us to use a fixed size thread pool of
> task
> > > >>>>> executors, instead of one thread per task.
> > > >>>>>
> > > >>>>> Piotrek
> > > >>>>>
> > > >>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <aljoscha@apache.org
> >
> > > >>>>> wrote:
> > > >>>>>>
> > > >>>>>> Hi All,
> > > >>>>>>
> > > >>>>>> In order to finally get the ball rolling on the new source
> > interface
> > > >>>>> that we have discussed for so long I finally created a FLIP:
> > > >>>>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > >>>>>>
> > > >>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
> > > about
> > > >>>>> adding per-partition watermark support to the Kinesis source and
> > > >> because
> > > >>>>> this would enable generic implementation of event-time alignment
> > for
> > > >> all
> > > >>>>> sources. Maybe we need another FLIP for the event-time alignment
> > > part,
> > > >>>>> especially the part about information sharing between operations
> > (I'm
> > > >> not
> > > >>>>> calling it state sharing because state has a special meaning in
> > > Flink).
> > > >>>>>>
> > > >>>>>> Please discuss away!
> > > >>>>>>
> > > >>>>>> Aljoscha
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>
> > >
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Biao Liu <mm...@gmail.com>.
Hi community,
Glad to see this topic is still so active.

Thanks for replying @Piotrek and @Becket.

Last time, I expressed some rough ideas about the thread model. However I
found that it's hard to describe clearly in mailing list. So I wrote it
down with some graphs, exampled some kinds of models, see Thread Model of
Source
<https://docs.google.com/document/d/1XpYkkJo97CUw-UMVrKU6b0ZZuJJ2V7mBb__L6UzdWTw/edit?usp=sharing>.
I wish that can be helpful.

IMO thread model is an important part. Without thinking of implementation
clearly, it's difficult to decide what the up level interface should look
like.
It would be better if we draw the whole picture first and then fill the
detail parts one by one.

@Piotrek About adding new splits to existing split reader. It's an
interesting idea. Not only for solving too many threads problem, but also
for supporting some more complicated system. I know in some storage
systems, there is some scenario which the partition is dynamic(dynamically
splitting or merging). Though I have not think of it very clearly now. I
would give you more detailed reply asap :)


Guowei Ma <gu...@gmail.com> 于2018年11月23日周五 下午6:37写道:

> Hi,Piotr
> Sorry  for so late to response.
>
>
> First of all I think Flink runtime can assigned a thread for a StreamTask,
> which likes  'Actor' model. The number of threads for a StreamTask should
> not be proportional to the operator or other things. This will give Flink
> the ability to scale horizontally. So I think it's not just the
> network(flush),checkpoint and  source, but some operators' threads can also
> be removed in the future, like AsyncWaitOperator.
>
>
>
> for b)
> When using event time, some sources want to assign a timestamp to each
> element. In current Flink interface, user will write like this
> public class EventTimeSource<Element> implements SourceFunction {
>   public void run() {
>      while(...){
>          Element record = // get from file or some queue;
>          long timestamp = parseTimestampFromElement(record);
>          sourceContext.collectWithTimestamp(record, timestamp);
>      }
>   }
> }
> Using the interfaces from this FLIP, user can write like this
>
> public EventTimeSplitReader<Element,X> implements SplitReader {
>     Element currentRecord = null;
>
>
>     // Please ignoring the handling of boundary conditions
>     public boolean advace(){
>        currentRecord = //move a pointer forward
>        return true;
>      }
>
>     public Element getCurrent(){
>        return currentRecord;
>     }
>     public long getCurrentTimestamp() {
>       return parseTimestampFromElement(currentRecord);
>     }
> }
>
> if merging the advance/getNext to a method like getNext() , the SplitReader
> interface may need to change a little like this
>
> public interface SplitReader2<T,X> {
> public class ElementWithTimestamp {
>     T element;
>     long timestamp;
> }
>
> public ElementWithTimestamp getNext() ;
>
> }
> Now user may need implement the source like this
> public EventTimeSplitReader<Element,X> implements SplitReader2 {
>     Element currentRecord = null;
>
>     // Please ignoring the handling of boundary conditions
>     public ElementWithTimestamp getCurrent(){
>        return new ElementWithTimestamp(currentRecord,
> parseTimestampFromElement(currentRecord))
>     }
> }
> The user can use a constant ElementWithTimestamp but I think this need the
> every connector developers to know this trick. The current Flip will not
> have this burden.
> Maybe there has other way like '' void getCurrent(ElementWithTimestamp)"
> to avoid creating a new object.  But my personal preference is
> ‘advance/getCurrent’.
>
>
>
> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月7日周三 下午4:31写道:
>
> > Hi,
> >
> > a)
> >
> > > BTW, regarding the isBlock() method, I have a few more questions. 21,
> Is
> > a method isReady() with boolean as a return value
> > > equivalent? Personally I found it is a little bit confusing in what is
> > supposed to be returned when the future is completed. 22. if
> > > the implementation of isBlocked() is optional, how do the callers know
> > whether the method is properly implemented or not?
> > > Does not implemented mean it always return a completed future?
> >
> > `CompletableFuture<?> isBlocked()` is more or less an equivalent to
> > `boolean hasNext()` which in case of “false” provides some kind of a
> > listener/callback that notifies about presence of next element. There are
> > some minor details, like `CompletableFuture<?>` has a minimal two state
> > logic:
> >
> > 1. Future is completed - we have more data
> > 2. Future not yet completed - we don’t have data now, but we might/we
> will
> > have in the future
> >
> > While `boolean hasNext()` and `notify()` callback are a bit more
> > complicated/dispersed and can lead/encourage `notify()` spam.
> >
> > b)
> >
> > > 3. If merge the `advance` and `getCurrent`  to one method like
> `getNext`
> > the `getNext` would need return a
> > >`ElementWithTimestamp` because some sources want to add timestamp to
> > every element. IMO, this is not so memory friendly
> > > so I prefer this design.
> >
> > Guowei I don’t quite understand this. Could you elaborate why having a
> > separate `advance()` help?
> >
> > c)
> >
> > Regarding advance/poll/take. What’s the value of having two separate
> > methods: poll and take? Which one of them should be called and which
> > implemented? What’s the benefit of having those methods compared to
> having
> > a one single method `getNextElement()` (or `pollElement() or whatever we
> > name it) with following contract:
> >
> > CompletableFuture<?> isBlocked();
> >
> > /**
> > Return next element - will be called only if `isBlocked()` is completed.
> > Try to implement it in non blocking fashion, but if that’s impossible or
> > you just don’t need the effort, you can block in this method.
> > */
> > T getNextElement();
> >
> > I mean, if the connector is implemented non-blockingly, Flink should use
> > it that way. If it’s not, then `poll()` will `throw new
> > NotImplementedException()`. Implementing both of them and providing both
> of
> > them to Flink wouldn’t make a sense, thus why not merge them into a
> single
> > method call that should preferably (but not necessarily need to) be
> > non-blocking? It’s not like we are implementing general purpose `Queue`,
> > which users might want to call either of `poll` or `take`. We would
> always
> > prefer to call `poll`, but if it’s blocking, then still we have no
> choice,
> > but to call it and block on it.
> >
> > d)
> >
> > > 1. I agree with Piotr and Becket that the non-blocking source is very
> > > important. But in addition to `Future/poll`, there may be another way
> to
> > > achieve this. I think it may be not very memory friendly if every
> advance
> > > call return a Future.
> >
> > I didn’t want to mention this, to not clog my initial proposal, but there
> > is a simple solution for the problem:
> >
> > public interface SplitReader {
> >
> >     (…)
> >
> >     CompletableFuture<?> NOT_BLOCKED =
> > CompletableFuture.completedFuture(null);
> >
> >     /**
> >      * Returns a future that will be completed when the page source
> becomes
> >      * unblocked.  If the page source is not blocked, this method should
> > return
> >      * {@code NOT_BLOCKED}.
> >      */
> >     default CompletableFuture<?> isBlocked()
> >     {
> >         return NOT_BLOCKED;
> >     }
> >
> > If we are blocked and we are waiting for the IO, then creating a new
> > Future is non-issue. Under full throttle/throughput and not blocked
> sources
> > returning a static `NOT_BLOCKED` constant  should also solve the problem.
> >
> > One more remark, non-blocking sources might be a necessity in a single
> > threaded model without a checkpointing lock. (Currently when sources are
> > blocked, they can release checkpointing lock and re-acquire it again
> > later). Non-blocking `poll`/`getNext()` would allow for checkpoints to
> > happen when source is idling. In that case either `notify()` or my
> proposed
> > `isBlocked()` would allow to avoid busy-looping.
> >
> > Piotrek
> >
> > > On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
> > >
> > > Hi Thomas,
> > >
> > > The iterator-like API was also the first thing that came to me. But it
> > > seems a little confusing that hasNext() does not mean "the stream has
> not
> > > ended", but means "the next record is ready", which is repurposing the
> > well
> > > known meaning of hasNext(). If we follow the hasNext()/next() pattern,
> an
> > > additional isNextReady() method to indicate whether the next record is
> > > ready seems more intuitive to me.
> > >
> > > Similarly, in poll()/take() pattern, another method of isDone() is
> needed
> > > to indicate whether the stream has ended or not.
> > >
> > > Compared with hasNext()/next()/isNextReady() pattern,
> > > isDone()/poll()/take() seems more flexible for the reader
> implementation.
> > > When I am implementing a reader, I could have a couple of choices:
> > >
> > >   - A thread-less reader that does not have any internal thread.
> > >   - When poll() is called, the same calling thread will perform a bunch
> > of
> > >      IO asynchronously.
> > >      - When take() is called, the same calling thread will perform a
> > bunch
> > >      of IO and wait until the record is ready.
> > >   - A reader with internal threads performing network IO and put
> records
> > >   into a buffer.
> > >      - When poll() is called, the calling thread simply reads from the
> > >      buffer and return empty result immediately if there is no record.
> > >      - When take() is called, the calling thread reads from the buffer
> > and
> > >      block waiting if the buffer is empty.
> > >
> > > On the other hand, with the hasNext()/next()/isNextReady() API, it is
> > less
> > > intuitive for the reader developers to write the thread-less pattern.
> > > Although technically speaking one can still do the asynchronous IO to
> > > prepare the record in isNextReady(). But it is inexplicit and seems
> > > somewhat hacky.
> > >
> > > Thanks,
> > >
> > > Jiangjie (Becket) Qin
> > >
> > >
> > >
> > > On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
> > >
> > >> Couple more points regarding discovery:
> > >>
> > >> The proposal mentions that discovery could be outside the execution
> > graph.
> > >> Today, discovered partitions/shards are checkpointed. I believe that
> > will
> > >> also need to be the case in the future, even when discovery and
> reading
> > are
> > >> split between different tasks.
> > >>
> > >> For cases such as resharding of a Kinesis stream, the relationship
> > between
> > >> splits needs to be considered. Splits cannot be randomly distributed
> > over
> > >> readers in certain situations. An example was mentioned here:
> > >> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> > >>
> > >> Thomas
> > >>
> > >>
> > >> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
> > >>
> > >>> Thanks for getting the ball rolling on this!
> > >>>
> > >>> Can the number of splits decrease? Yes, splits can be closed and go
> > away.
> > >>> An example would be a shard merge in Kinesis (2 existing shards will
> be
> > >>> closed and replaced with a new shard).
> > >>>
> > >>> Regarding advance/poll/take: IMO the least restrictive approach would
> > be
> > >>> the thread-less IO model (pull based, non-blocking, caller retrieves
> > new
> > >>> records when available). The current Kinesis API requires the use of
> > >>> threads. But that can be internal to the split reader and does not
> need
> > >> to
> > >>> be a source API concern. In fact, that's what we are working on right
> > now
> > >>> as improvement to the existing consumer: Each shard consumer thread
> > will
> > >>> push to a queue, the consumer main thread will poll the queue(s). It
> is
> > >>> essentially a mapping from threaded IO to non-blocking.
> > >>>
> > >>> The proposed SplitReader interface would fit the thread-less IO
> model.
> > >>> Similar to an iterator, we find out if there is a new element
> (hasNext)
> > >> and
> > >>> if so, move to it (next()). Separate calls deliver the meta
> information
> > >>> (timestamp, watermark). Perhaps advance call could offer a timeout
> > >> option,
> > >>> so that the caller does not end up in a busy wait. On the other
> hand, a
> > >>> caller processing multiple splits may want to cycle through fast, to
> > >>> process elements of other splits as soon as they become available.
> The
> > >> nice
> > >>> thing is that this "split merge" logic can now live in Flink and be
> > >>> optimized and shared between different sources.
> > >>>
> > >>> Thanks,
> > >>> Thomas
> > >>>
> > >>>
> > >>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
> wrote:
> > >>>
> > >>>> Hi,
> > >>>> Thanks Aljoscha for this FLIP.
> > >>>>
> > >>>> 1. I agree with Piotr and Becket that the non-blocking source is
> very
> > >>>> important. But in addition to `Future/poll`, there may be another
> way
> > to
> > >>>> achieve this. I think it may be not very memory friendly if every
> > >> advance
> > >>>> call return a Future.
> > >>>>
> > >>>> public interface Listener {
> > >>>>     public void notify();
> > >>>> }
> > >>>>
> > >>>> public interface SplitReader() {
> > >>>>     /**
> > >>>>      * When there is no element temporarily, this will return false.
> > >>>>      * When elements is available again splitReader can call
> > >>>> listener.notify()
> > >>>>      * In addition the frame would check `advance` periodically .
> > >>>>      * Of course advance can always return true and ignore the
> > listener
> > >>>> argument for simplicity.
> > >>>>      */
> > >>>>     public boolean advance(Listener listener);
> > >>>> }
> > >>>>
> > >>>> 2.  The FLIP tells us very clearly that how to create all Splits and
> > how
> > >>>> to create a SplitReader from a Split. But there is no strategy for
> the
> > >> user
> > >>>> to choose how to assign the splits to the tasks. I think we could
> add
> > a
> > >>>> Enum to let user to choose.
> > >>>> /**
> > >>>>  public Enum SplitsAssignmentPolicy {
> > >>>>    Location,
> > >>>>    Workload,
> > >>>>    Random,
> > >>>>    Average
> > >>>>  }
> > >>>> */
> > >>>>
> > >>>> 3. If merge the `advance` and `getCurrent`  to one method like
> > `getNext`
> > >>>> the `getNext` would need return a `ElementWithTimestamp` because
> some
> > >>>> sources want to add timestamp to every element. IMO, this is not so
> > >> memory
> > >>>> friendly so I prefer this design.
> > >>>>
> > >>>>
> > >>>>
> > >>>> Thanks
> > >>>>
> > >>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
> > >>>>
> > >>>>> Hi,
> > >>>>>
> > >>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
> other
> > >>>>> possible improvements. I have one proposal. Instead of having a
> > method:
> > >>>>>
> > >>>>> boolean advance() throws IOException;
> > >>>>>
> > >>>>> I would replace it with
> > >>>>>
> > >>>>> /*
> > >>>>> * Return a future, which when completed means that source has more
> > >> data
> > >>>>> and getNext() will not block.
> > >>>>> * If you wish to use benefits of non blocking connectors, please
> > >>>>> implement this method appropriately.
> > >>>>> */
> > >>>>> default CompletableFuture<?> isBlocked() {
> > >>>>>        return CompletableFuture.completedFuture(null);
> > >>>>> }
> > >>>>>
> > >>>>> And rename `getCurrent()` to `getNext()`.
> > >>>>>
> > >>>>> Couple of arguments:
> > >>>>> 1. I don’t understand the division of work between `advance()` and
> > >>>>> `getCurrent()`. What should be done in which, especially for
> > connectors
> > >>>>> that handle records in batches (like Kafka) and when should you
> call
> > >>>>> `advance` and when `getCurrent()`.
> > >>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in
> > the
> > >>>>> future to have asynchronous/non blocking connectors and more
> > >> efficiently
> > >>>>> handle large number of blocked threads, without busy waiting. While
> > at
> > >> the
> > >>>>> same time it doesn’t add much complexity, since naive connector
> > >>>>> implementations can be always blocking.
> > >>>>> 3. This also would allow us to use a fixed size thread pool of task
> > >>>>> executors, instead of one thread per task.
> > >>>>>
> > >>>>> Piotrek
> > >>>>>
> > >>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
> > >>>>> wrote:
> > >>>>>>
> > >>>>>> Hi All,
> > >>>>>>
> > >>>>>> In order to finally get the ball rolling on the new source
> interface
> > >>>>> that we have discussed for so long I finally created a FLIP:
> > >>>>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >>>>>>
> > >>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
> > about
> > >>>>> adding per-partition watermark support to the Kinesis source and
> > >> because
> > >>>>> this would enable generic implementation of event-time alignment
> for
> > >> all
> > >>>>> sources. Maybe we need another FLIP for the event-time alignment
> > part,
> > >>>>> especially the part about information sharing between operations
> (I'm
> > >> not
> > >>>>> calling it state sharing because state has a special meaning in
> > Flink).
> > >>>>>>
> > >>>>>> Please discuss away!
> > >>>>>>
> > >>>>>> Aljoscha
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>
> >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Guowei Ma <gu...@gmail.com>.
Hi,Piotr
Sorry  for so late to response.


First of all I think Flink runtime can assigned a thread for a StreamTask,
which likes  'Actor' model. The number of threads for a StreamTask should
not be proportional to the operator or other things. This will give Flink
the ability to scale horizontally. So I think it's not just the
network(flush),checkpoint and  source, but some operators' threads can also
be removed in the future, like AsyncWaitOperator.



for b)
When using event time, some sources want to assign a timestamp to each
element. In current Flink interface, user will write like this
public class EventTimeSource<Element> implements SourceFunction {
  public void run() {
     while(...){
         Element record = // get from file or some queue;
         long timestamp = parseTimestampFromElement(record);
         sourceContext.collectWithTimestamp(record, timestamp);
     }
  }
}
Using the interfaces from this FLIP, user can write like this

public EventTimeSplitReader<Element,X> implements SplitReader {
    Element currentRecord = null;


    // Please ignoring the handling of boundary conditions
    public boolean advace(){
       currentRecord = //move a pointer forward
       return true;
     }

    public Element getCurrent(){
       return currentRecord;
    }
    public long getCurrentTimestamp() {
      return parseTimestampFromElement(currentRecord);
    }
}

if merging the advance/getNext to a method like getNext() , the SplitReader
interface may need to change a little like this

public interface SplitReader2<T,X> {
public class ElementWithTimestamp {
    T element;
    long timestamp;
}

public ElementWithTimestamp getNext() ;

}
Now user may need implement the source like this
public EventTimeSplitReader<Element,X> implements SplitReader2 {
    Element currentRecord = null;

    // Please ignoring the handling of boundary conditions
    public ElementWithTimestamp getCurrent(){
       return new ElementWithTimestamp(currentRecord,
parseTimestampFromElement(currentRecord))
    }
}
The user can use a constant ElementWithTimestamp but I think this need the
every connector developers to know this trick. The current Flip will not
have this burden.
Maybe there has other way like '' void getCurrent(ElementWithTimestamp)"
to avoid creating a new object.  But my personal preference is
‘advance/getCurrent’.



Piotr Nowojski <pi...@data-artisans.com> 于2018年11月7日周三 下午4:31写道:

> Hi,
>
> a)
>
> > BTW, regarding the isBlock() method, I have a few more questions. 21, Is
> a method isReady() with boolean as a return value
> > equivalent? Personally I found it is a little bit confusing in what is
> supposed to be returned when the future is completed. 22. if
> > the implementation of isBlocked() is optional, how do the callers know
> whether the method is properly implemented or not?
> > Does not implemented mean it always return a completed future?
>
> `CompletableFuture<?> isBlocked()` is more or less an equivalent to
> `boolean hasNext()` which in case of “false” provides some kind of a
> listener/callback that notifies about presence of next element. There are
> some minor details, like `CompletableFuture<?>` has a minimal two state
> logic:
>
> 1. Future is completed - we have more data
> 2. Future not yet completed - we don’t have data now, but we might/we will
> have in the future
>
> While `boolean hasNext()` and `notify()` callback are a bit more
> complicated/dispersed and can lead/encourage `notify()` spam.
>
> b)
>
> > 3. If merge the `advance` and `getCurrent`  to one method like `getNext`
> the `getNext` would need return a
> >`ElementWithTimestamp` because some sources want to add timestamp to
> every element. IMO, this is not so memory friendly
> > so I prefer this design.
>
> Guowei I don’t quite understand this. Could you elaborate why having a
> separate `advance()` help?
>
> c)
>
> Regarding advance/poll/take. What’s the value of having two separate
> methods: poll and take? Which one of them should be called and which
> implemented? What’s the benefit of having those methods compared to having
> a one single method `getNextElement()` (or `pollElement() or whatever we
> name it) with following contract:
>
> CompletableFuture<?> isBlocked();
>
> /**
> Return next element - will be called only if `isBlocked()` is completed.
> Try to implement it in non blocking fashion, but if that’s impossible or
> you just don’t need the effort, you can block in this method.
> */
> T getNextElement();
>
> I mean, if the connector is implemented non-blockingly, Flink should use
> it that way. If it’s not, then `poll()` will `throw new
> NotImplementedException()`. Implementing both of them and providing both of
> them to Flink wouldn’t make a sense, thus why not merge them into a single
> method call that should preferably (but not necessarily need to) be
> non-blocking? It’s not like we are implementing general purpose `Queue`,
> which users might want to call either of `poll` or `take`. We would always
> prefer to call `poll`, but if it’s blocking, then still we have no choice,
> but to call it and block on it.
>
> d)
>
> > 1. I agree with Piotr and Becket that the non-blocking source is very
> > important. But in addition to `Future/poll`, there may be another way to
> > achieve this. I think it may be not very memory friendly if every advance
> > call return a Future.
>
> I didn’t want to mention this, to not clog my initial proposal, but there
> is a simple solution for the problem:
>
> public interface SplitReader {
>
>     (…)
>
>     CompletableFuture<?> NOT_BLOCKED =
> CompletableFuture.completedFuture(null);
>
>     /**
>      * Returns a future that will be completed when the page source becomes
>      * unblocked.  If the page source is not blocked, this method should
> return
>      * {@code NOT_BLOCKED}.
>      */
>     default CompletableFuture<?> isBlocked()
>     {
>         return NOT_BLOCKED;
>     }
>
> If we are blocked and we are waiting for the IO, then creating a new
> Future is non-issue. Under full throttle/throughput and not blocked sources
> returning a static `NOT_BLOCKED` constant  should also solve the problem.
>
> One more remark, non-blocking sources might be a necessity in a single
> threaded model without a checkpointing lock. (Currently when sources are
> blocked, they can release checkpointing lock and re-acquire it again
> later). Non-blocking `poll`/`getNext()` would allow for checkpoints to
> happen when source is idling. In that case either `notify()` or my proposed
> `isBlocked()` would allow to avoid busy-looping.
>
> Piotrek
>
> > On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
> >
> > Hi Thomas,
> >
> > The iterator-like API was also the first thing that came to me. But it
> > seems a little confusing that hasNext() does not mean "the stream has not
> > ended", but means "the next record is ready", which is repurposing the
> well
> > known meaning of hasNext(). If we follow the hasNext()/next() pattern, an
> > additional isNextReady() method to indicate whether the next record is
> > ready seems more intuitive to me.
> >
> > Similarly, in poll()/take() pattern, another method of isDone() is needed
> > to indicate whether the stream has ended or not.
> >
> > Compared with hasNext()/next()/isNextReady() pattern,
> > isDone()/poll()/take() seems more flexible for the reader implementation.
> > When I am implementing a reader, I could have a couple of choices:
> >
> >   - A thread-less reader that does not have any internal thread.
> >   - When poll() is called, the same calling thread will perform a bunch
> of
> >      IO asynchronously.
> >      - When take() is called, the same calling thread will perform a
> bunch
> >      of IO and wait until the record is ready.
> >   - A reader with internal threads performing network IO and put records
> >   into a buffer.
> >      - When poll() is called, the calling thread simply reads from the
> >      buffer and return empty result immediately if there is no record.
> >      - When take() is called, the calling thread reads from the buffer
> and
> >      block waiting if the buffer is empty.
> >
> > On the other hand, with the hasNext()/next()/isNextReady() API, it is
> less
> > intuitive for the reader developers to write the thread-less pattern.
> > Although technically speaking one can still do the asynchronous IO to
> > prepare the record in isNextReady(). But it is inexplicit and seems
> > somewhat hacky.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> >
> >
> > On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
> >
> >> Couple more points regarding discovery:
> >>
> >> The proposal mentions that discovery could be outside the execution
> graph.
> >> Today, discovered partitions/shards are checkpointed. I believe that
> will
> >> also need to be the case in the future, even when discovery and reading
> are
> >> split between different tasks.
> >>
> >> For cases such as resharding of a Kinesis stream, the relationship
> between
> >> splits needs to be considered. Splits cannot be randomly distributed
> over
> >> readers in certain situations. An example was mentioned here:
> >> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >>
> >> Thomas
> >>
> >>
> >> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
> >>
> >>> Thanks for getting the ball rolling on this!
> >>>
> >>> Can the number of splits decrease? Yes, splits can be closed and go
> away.
> >>> An example would be a shard merge in Kinesis (2 existing shards will be
> >>> closed and replaced with a new shard).
> >>>
> >>> Regarding advance/poll/take: IMO the least restrictive approach would
> be
> >>> the thread-less IO model (pull based, non-blocking, caller retrieves
> new
> >>> records when available). The current Kinesis API requires the use of
> >>> threads. But that can be internal to the split reader and does not need
> >> to
> >>> be a source API concern. In fact, that's what we are working on right
> now
> >>> as improvement to the existing consumer: Each shard consumer thread
> will
> >>> push to a queue, the consumer main thread will poll the queue(s). It is
> >>> essentially a mapping from threaded IO to non-blocking.
> >>>
> >>> The proposed SplitReader interface would fit the thread-less IO model.
> >>> Similar to an iterator, we find out if there is a new element (hasNext)
> >> and
> >>> if so, move to it (next()). Separate calls deliver the meta information
> >>> (timestamp, watermark). Perhaps advance call could offer a timeout
> >> option,
> >>> so that the caller does not end up in a busy wait. On the other hand, a
> >>> caller processing multiple splits may want to cycle through fast, to
> >>> process elements of other splits as soon as they become available. The
> >> nice
> >>> thing is that this "split merge" logic can now live in Flink and be
> >>> optimized and shared between different sources.
> >>>
> >>> Thanks,
> >>> Thomas
> >>>
> >>>
> >>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com> wrote:
> >>>
> >>>> Hi,
> >>>> Thanks Aljoscha for this FLIP.
> >>>>
> >>>> 1. I agree with Piotr and Becket that the non-blocking source is very
> >>>> important. But in addition to `Future/poll`, there may be another way
> to
> >>>> achieve this. I think it may be not very memory friendly if every
> >> advance
> >>>> call return a Future.
> >>>>
> >>>> public interface Listener {
> >>>>     public void notify();
> >>>> }
> >>>>
> >>>> public interface SplitReader() {
> >>>>     /**
> >>>>      * When there is no element temporarily, this will return false.
> >>>>      * When elements is available again splitReader can call
> >>>> listener.notify()
> >>>>      * In addition the frame would check `advance` periodically .
> >>>>      * Of course advance can always return true and ignore the
> listener
> >>>> argument for simplicity.
> >>>>      */
> >>>>     public boolean advance(Listener listener);
> >>>> }
> >>>>
> >>>> 2.  The FLIP tells us very clearly that how to create all Splits and
> how
> >>>> to create a SplitReader from a Split. But there is no strategy for the
> >> user
> >>>> to choose how to assign the splits to the tasks. I think we could add
> a
> >>>> Enum to let user to choose.
> >>>> /**
> >>>>  public Enum SplitsAssignmentPolicy {
> >>>>    Location,
> >>>>    Workload,
> >>>>    Random,
> >>>>    Average
> >>>>  }
> >>>> */
> >>>>
> >>>> 3. If merge the `advance` and `getCurrent`  to one method like
> `getNext`
> >>>> the `getNext` would need return a `ElementWithTimestamp` because some
> >>>> sources want to add timestamp to every element. IMO, this is not so
> >> memory
> >>>> friendly so I prefer this design.
> >>>>
> >>>>
> >>>>
> >>>> Thanks
> >>>>
> >>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
> >>>>
> >>>>> Hi,
> >>>>>
> >>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
> >>>>> possible improvements. I have one proposal. Instead of having a
> method:
> >>>>>
> >>>>> boolean advance() throws IOException;
> >>>>>
> >>>>> I would replace it with
> >>>>>
> >>>>> /*
> >>>>> * Return a future, which when completed means that source has more
> >> data
> >>>>> and getNext() will not block.
> >>>>> * If you wish to use benefits of non blocking connectors, please
> >>>>> implement this method appropriately.
> >>>>> */
> >>>>> default CompletableFuture<?> isBlocked() {
> >>>>>        return CompletableFuture.completedFuture(null);
> >>>>> }
> >>>>>
> >>>>> And rename `getCurrent()` to `getNext()`.
> >>>>>
> >>>>> Couple of arguments:
> >>>>> 1. I don’t understand the division of work between `advance()` and
> >>>>> `getCurrent()`. What should be done in which, especially for
> connectors
> >>>>> that handle records in batches (like Kafka) and when should you call
> >>>>> `advance` and when `getCurrent()`.
> >>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in
> the
> >>>>> future to have asynchronous/non blocking connectors and more
> >> efficiently
> >>>>> handle large number of blocked threads, without busy waiting. While
> at
> >> the
> >>>>> same time it doesn’t add much complexity, since naive connector
> >>>>> implementations can be always blocking.
> >>>>> 3. This also would allow us to use a fixed size thread pool of task
> >>>>> executors, instead of one thread per task.
> >>>>>
> >>>>> Piotrek
> >>>>>
> >>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
> >>>>> wrote:
> >>>>>>
> >>>>>> Hi All,
> >>>>>>
> >>>>>> In order to finally get the ball rolling on the new source interface
> >>>>> that we have discussed for so long I finally created a FLIP:
> >>>>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>
> >>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
> about
> >>>>> adding per-partition watermark support to the Kinesis source and
> >> because
> >>>>> this would enable generic implementation of event-time alignment for
> >> all
> >>>>> sources. Maybe we need another FLIP for the event-time alignment
> part,
> >>>>> especially the part about information sharing between operations (I'm
> >> not
> >>>>> calling it state sharing because state has a special meaning in
> Flink).
> >>>>>>
> >>>>>> Please discuss away!
> >>>>>>
> >>>>>> Aljoscha
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@data-artisans.com>.
Hi,

a)

> BTW, regarding the isBlock() method, I have a few more questions. 21, Is a method isReady() with boolean as a return value
> equivalent? Personally I found it is a little bit confusing in what is supposed to be returned when the future is completed. 22. if
> the implementation of isBlocked() is optional, how do the callers know whether the method is properly implemented or not?
> Does not implemented mean it always return a completed future?

`CompletableFuture<?> isBlocked()` is more or less an equivalent to `boolean hasNext()` which in case of “false” provides some kind of a listener/callback that notifies about presence of next element. There are some minor details, like `CompletableFuture<?>` has a minimal two state logic:

1. Future is completed - we have more data
2. Future not yet completed - we don’t have data now, but we might/we will have in the future

While `boolean hasNext()` and `notify()` callback are a bit more complicated/dispersed and can lead/encourage `notify()` spam.

b)

> 3. If merge the `advance` and `getCurrent`  to one method like `getNext` the `getNext` would need return a
>`ElementWithTimestamp` because some sources want to add timestamp to every element. IMO, this is not so memory friendly
> so I prefer this design.

Guowei I don’t quite understand this. Could you elaborate why having a separate `advance()` help?

c)

Regarding advance/poll/take. What’s the value of having two separate methods: poll and take? Which one of them should be called and which implemented? What’s the benefit of having those methods compared to having a one single method `getNextElement()` (or `pollElement() or whatever we name it) with following contract:

CompletableFuture<?> isBlocked();

/**
Return next element - will be called only if `isBlocked()` is completed. Try to implement it in non blocking fashion, but if that’s impossible or you just don’t need the effort, you can block in this method.
*/
T getNextElement();

I mean, if the connector is implemented non-blockingly, Flink should use it that way. If it’s not, then `poll()` will `throw new NotImplementedException()`. Implementing both of them and providing both of them to Flink wouldn’t make a sense, thus why not merge them into a single method call that should preferably (but not necessarily need to) be non-blocking? It’s not like we are implementing general purpose `Queue`, which users might want to call either of `poll` or `take`. We would always prefer to call `poll`, but if it’s blocking, then still we have no choice, but to call it and block on it.

d)

> 1. I agree with Piotr and Becket that the non-blocking source is very
> important. But in addition to `Future/poll`, there may be another way to
> achieve this. I think it may be not very memory friendly if every advance
> call return a Future.

I didn’t want to mention this, to not clog my initial proposal, but there is a simple solution for the problem:

public interface SplitReader {
    
    (…)

    CompletableFuture<?> NOT_BLOCKED = CompletableFuture.completedFuture(null);

    /**
     * Returns a future that will be completed when the page source becomes
     * unblocked.  If the page source is not blocked, this method should return
     * {@code NOT_BLOCKED}.
     */
    default CompletableFuture<?> isBlocked()
    {
        return NOT_BLOCKED;
    }

If we are blocked and we are waiting for the IO, then creating a new Future is non-issue. Under full throttle/throughput and not blocked sources returning a static `NOT_BLOCKED` constant  should also solve the problem.

One more remark, non-blocking sources might be a necessity in a single threaded model without a checkpointing lock. (Currently when sources are blocked, they can release checkpointing lock and re-acquire it again later). Non-blocking `poll`/`getNext()` would allow for checkpoints to happen when source is idling. In that case either `notify()` or my proposed `isBlocked()` would allow to avoid busy-looping.

Piotrek 

> On 5 Nov 2018, at 03:59, Becket Qin <be...@gmail.com> wrote:
> 
> Hi Thomas,
> 
> The iterator-like API was also the first thing that came to me. But it
> seems a little confusing that hasNext() does not mean "the stream has not
> ended", but means "the next record is ready", which is repurposing the well
> known meaning of hasNext(). If we follow the hasNext()/next() pattern, an
> additional isNextReady() method to indicate whether the next record is
> ready seems more intuitive to me.
> 
> Similarly, in poll()/take() pattern, another method of isDone() is needed
> to indicate whether the stream has ended or not.
> 
> Compared with hasNext()/next()/isNextReady() pattern,
> isDone()/poll()/take() seems more flexible for the reader implementation.
> When I am implementing a reader, I could have a couple of choices:
> 
>   - A thread-less reader that does not have any internal thread.
>   - When poll() is called, the same calling thread will perform a bunch of
>      IO asynchronously.
>      - When take() is called, the same calling thread will perform a bunch
>      of IO and wait until the record is ready.
>   - A reader with internal threads performing network IO and put records
>   into a buffer.
>      - When poll() is called, the calling thread simply reads from the
>      buffer and return empty result immediately if there is no record.
>      - When take() is called, the calling thread reads from the buffer and
>      block waiting if the buffer is empty.
> 
> On the other hand, with the hasNext()/next()/isNextReady() API, it is less
> intuitive for the reader developers to write the thread-less pattern.
> Although technically speaking one can still do the asynchronous IO to
> prepare the record in isNextReady(). But it is inexplicit and seems
> somewhat hacky.
> 
> Thanks,
> 
> Jiangjie (Becket) Qin
> 
> 
> 
> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
> 
>> Couple more points regarding discovery:
>> 
>> The proposal mentions that discovery could be outside the execution graph.
>> Today, discovered partitions/shards are checkpointed. I believe that will
>> also need to be the case in the future, even when discovery and reading are
>> split between different tasks.
>> 
>> For cases such as resharding of a Kinesis stream, the relationship between
>> splits needs to be considered. Splits cannot be randomly distributed over
>> readers in certain situations. An example was mentioned here:
>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>> 
>> Thomas
>> 
>> 
>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
>> 
>>> Thanks for getting the ball rolling on this!
>>> 
>>> Can the number of splits decrease? Yes, splits can be closed and go away.
>>> An example would be a shard merge in Kinesis (2 existing shards will be
>>> closed and replaced with a new shard).
>>> 
>>> Regarding advance/poll/take: IMO the least restrictive approach would be
>>> the thread-less IO model (pull based, non-blocking, caller retrieves new
>>> records when available). The current Kinesis API requires the use of
>>> threads. But that can be internal to the split reader and does not need
>> to
>>> be a source API concern. In fact, that's what we are working on right now
>>> as improvement to the existing consumer: Each shard consumer thread will
>>> push to a queue, the consumer main thread will poll the queue(s). It is
>>> essentially a mapping from threaded IO to non-blocking.
>>> 
>>> The proposed SplitReader interface would fit the thread-less IO model.
>>> Similar to an iterator, we find out if there is a new element (hasNext)
>> and
>>> if so, move to it (next()). Separate calls deliver the meta information
>>> (timestamp, watermark). Perhaps advance call could offer a timeout
>> option,
>>> so that the caller does not end up in a busy wait. On the other hand, a
>>> caller processing multiple splits may want to cycle through fast, to
>>> process elements of other splits as soon as they become available. The
>> nice
>>> thing is that this "split merge" logic can now live in Flink and be
>>> optimized and shared between different sources.
>>> 
>>> Thanks,
>>> Thomas
>>> 
>>> 
>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com> wrote:
>>> 
>>>> Hi,
>>>> Thanks Aljoscha for this FLIP.
>>>> 
>>>> 1. I agree with Piotr and Becket that the non-blocking source is very
>>>> important. But in addition to `Future/poll`, there may be another way to
>>>> achieve this. I think it may be not very memory friendly if every
>> advance
>>>> call return a Future.
>>>> 
>>>> public interface Listener {
>>>>     public void notify();
>>>> }
>>>> 
>>>> public interface SplitReader() {
>>>>     /**
>>>>      * When there is no element temporarily, this will return false.
>>>>      * When elements is available again splitReader can call
>>>> listener.notify()
>>>>      * In addition the frame would check `advance` periodically .
>>>>      * Of course advance can always return true and ignore the listener
>>>> argument for simplicity.
>>>>      */
>>>>     public boolean advance(Listener listener);
>>>> }
>>>> 
>>>> 2.  The FLIP tells us very clearly that how to create all Splits and how
>>>> to create a SplitReader from a Split. But there is no strategy for the
>> user
>>>> to choose how to assign the splits to the tasks. I think we could add a
>>>> Enum to let user to choose.
>>>> /**
>>>>  public Enum SplitsAssignmentPolicy {
>>>>    Location,
>>>>    Workload,
>>>>    Random,
>>>>    Average
>>>>  }
>>>> */
>>>> 
>>>> 3. If merge the `advance` and `getCurrent`  to one method like `getNext`
>>>> the `getNext` would need return a `ElementWithTimestamp` because some
>>>> sources want to add timestamp to every element. IMO, this is not so
>> memory
>>>> friendly so I prefer this design.
>>>> 
>>>> 
>>>> 
>>>> Thanks
>>>> 
>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
>>>> 
>>>>> Hi,
>>>>> 
>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
>>>>> possible improvements. I have one proposal. Instead of having a method:
>>>>> 
>>>>> boolean advance() throws IOException;
>>>>> 
>>>>> I would replace it with
>>>>> 
>>>>> /*
>>>>> * Return a future, which when completed means that source has more
>> data
>>>>> and getNext() will not block.
>>>>> * If you wish to use benefits of non blocking connectors, please
>>>>> implement this method appropriately.
>>>>> */
>>>>> default CompletableFuture<?> isBlocked() {
>>>>>        return CompletableFuture.completedFuture(null);
>>>>> }
>>>>> 
>>>>> And rename `getCurrent()` to `getNext()`.
>>>>> 
>>>>> Couple of arguments:
>>>>> 1. I don’t understand the division of work between `advance()` and
>>>>> `getCurrent()`. What should be done in which, especially for connectors
>>>>> that handle records in batches (like Kafka) and when should you call
>>>>> `advance` and when `getCurrent()`.
>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in the
>>>>> future to have asynchronous/non blocking connectors and more
>> efficiently
>>>>> handle large number of blocked threads, without busy waiting. While at
>> the
>>>>> same time it doesn’t add much complexity, since naive connector
>>>>> implementations can be always blocking.
>>>>> 3. This also would allow us to use a fixed size thread pool of task
>>>>> executors, instead of one thread per task.
>>>>> 
>>>>> Piotrek
>>>>> 
>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
>>>>> wrote:
>>>>>> 
>>>>>> Hi All,
>>>>>> 
>>>>>> In order to finally get the ball rolling on the new source interface
>>>>> that we have discussed for so long I finally created a FLIP:
>>>>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>> 
>>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion about
>>>>> adding per-partition watermark support to the Kinesis source and
>> because
>>>>> this would enable generic implementation of event-time alignment for
>> all
>>>>> sources. Maybe we need another FLIP for the event-time alignment part,
>>>>> especially the part about information sharing between operations (I'm
>> not
>>>>> calling it state sharing because state has a special meaning in Flink).
>>>>>> 
>>>>>> Please discuss away!
>>>>>> 
>>>>>> Aljoscha
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Biao,

Thanks for the explanation. The current API makes more sense to me now. It
basically means:
1. Readers should all be non-blocking
2. The offset advancing and the record fetching are two steps.
3. After each advance() call, the currentRecord, currentTimestamp and
watermark will all be updated at the same time. And those values can be
accessed multiple times.

That being said, with the poll()/take() method, we don't have to return
tuple3. poll()/take() will just return the record. It means:
1. Readers could be blocking (take()) or non-blocking(poll())
2. The offset advancing and the record fetching are combined into one step,
i.e. poll()/take()
3. After each poll()/take(), the currentTimestamp and watermark are
updated. That means after poll()/take(), users can call
getCurrentTimestamp() or getWatermark() to get the information at the point
after the previous record was returned.

One concern I have for the completely non-blocking reader is that it would
be difficult to implement a blocking behavior on top of the thread-less
non-blocking reader. If advance() returns false, since the reader is
thread-less, unless the caller thread call something on the reader, no
progress will be made. Hence the caller has to call advance() again to
check, either with a backoff (introducing some latency) or a tight loop.
But neither of them is ideal. From this perspective, I think it is useful
to have a blocking() API in the reader, so the blocking behavior could be
done efficiently, e.g. by using a NIO selector which relies on the OS
signals.

WRT to the SplitEnumerator, I still feel that it would be better for the
SplitEnumerator to not only return new splits but all the splits to cover
the splits shrink case. Also, it took me a while to understand why
*createInitialEnumeratorCheckpoint()
*is needed. I am wondering whether it would be better to replace it with a
parameter-less *createSplitEnumerator(). *

Thanks,

Jiangjie (Becket) Qin



On Tue, Nov 6, 2018 at 11:40 PM Biao Liu <mm...@gmail.com> wrote:

> Regarding the naming style.
>
> The advantage of `poll()` style is that basically the name of `poll` means
> it should be a non-blocking operator, same with `Queue` in Java API. It's
> easy to understand. We don't need to write too much in docs to imply the
> implementation should not do something heavy.
> However `poll` also means it should return the thing we want. In our
> scenario, there are 3 types currently, record, timestamp and watermark. So
> the return type of `poll` should be tuple3 or something like that. It looks
> a little hacky IMO.
>
> The `advance()` style is more like RecordReader
> <
> https://hadoop.apache.org/docs/current/api/org/apache/hadoop/mapreduce/RecordReader.html
> >
> of
> MapReduce, or ISpout
> <
> https://storm.apache.org/releases/1.1.2/javadocs/org/apache/storm/spout/ISpout.html
> >
> of
> Storm. It means moving the offset forward indeed. It makes sense to me.
> To be honest I like `advance()` style more.
>
> And there is also another small point I can't get.
>
> Why use `start()` and `close()` in `SplitReader`? `start()` makes me think
> of "starting a thread" or something like that. We should not assume there
> would be some thread. I prefer `open()`, it also matches the `close()`
> better.
>
>
> Becket Qin <be...@gmail.com> 于2018年11月6日周二 上午11:04写道:
>
> > Thanks for updating the wiki, Aljoscha.
> >
> > The isDone()/advance()/getCurrent() API looks more similar to
> > hasNext()/isNextReady()/getNext(), but implying some different behaviors.
> >
> > If users call getCurrent() twice without calling advance() in between,
> will
> > they get the same record back? From the API itself, users might think
> > advance() is the API that moves the offset forward, and getCurrent() just
> > return the record at the current offset.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > On Mon, Nov 5, 2018 at 10:41 PM Aljoscha Krettek <al...@apache.org>
> > wrote:
> >
> > > I updated the FLIP [1] with some Javadoc for the SplitReader to outline
> > > what I had in mind with the interface. Sorry for not doing that
> earlier,
> > > it's not quite clear how the methods should work from the name alone.
> > >
> > > The gist of it is that advance() should be non-blocking, so
> > > isDone/advance()/getCurrent() are very similar to
> isDone()/poll()/take()
> > > that I have seen mentioned.
> > >
> > > [1]
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > <
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > > >
> > >
> > > > On 5. Nov 2018, at 11:05, Biao Liu <mm...@gmail.com> wrote:
> > > >
> > > > Thanks Aljoscha for bringing us this discussion!
> > > >
> > > > 1. I think one of the reason about separating `advance()` and
> > > > `getCurrent()` is that we have several different types returned by
> > > source.
> > > > Not just the `record`, but also the timestamp of record and the
> > > watermark.
> > > > If we don't separate these into different methods, the source has to
> > > return
> > > > a tuple3 which is not so user friendly. The prototype of Aljoscha is
> > > > acceptable to me. Regarding the specific method name, I'm not sure
> > which
> > > > one is better. Both of them are reasonable for me.
> > > >
> > > > 2. As Thomas and Becket mentioned before, I think a non-blocking API
> is
> > > > necessary. Moreover, IMO we should not offer a blocking API. It
> doesn't
> > > > help but makes things more complicated.
> > > >
> > > > 3. About the thread model.
> > > > I agree with Thomas about the thread-less IO model. A standard
> workflow
> > > > should look like below.
> > > >  - If there is available data, Flink would read it.
> > > >  - If there is no data available temporary, Flink would check again a
> > > > moment later. Maybe waiting on a semaphore until a timer wake it up.
> > > > Furthermore, we can offer an optional optimization for source which
> has
> > > > external thread. Like Guowei mentioned, there can be a listener which
> > the
> > > > reader can wake the framework up as soon as new data comes. This can
> > > solve
> > > > Piotr's concern about efficiency.
> > > >
> > > > 4. One more thing. After taking a look at the prototype codes. Off
> the
> > > top
> > > > of my head, the implementation is more fit for batch job not
> streaming
> > > job.
> > > > There are two types of tasks in prototype. First is a source task
> that
> > > > discovers the splits. The source passes the splits to the second task
> > > which
> > > > process the splits one by one. And then the source keeps watch to
> > > discover
> > > > more splits.
> > > >
> > > > However, I think the more common scenario of streaming job is:
> > > > there are fixed splits, each of the subtasks takes several splits.
> The
> > > > subtasks just keep processing the fixed splits. There would be
> > continuous
> > > > datum in each split. We don't need a source task to discover more
> > splits.
> > > > It can not be finished in streaming job since we don't want the
> > > processing
> > > > task finished even there are no more splits.
> > > >
> > > > So IMO we should offer another source operator for the new interface.
> > It
> > > > would discover all splits when it is opening. Then picks the splits
> > > belong
> > > > to this subtask. Keep processing these splits until all of them are
> > > > finished.
> > > >
> > > >
> > > > Becket Qin <be...@gmail.com> 于2018年11月5日周一 上午11:00写道:
> > > >
> > > >> Hi Thomas,
> > > >>
> > > >> The iterator-like API was also the first thing that came to me. But
> it
> > > >> seems a little confusing that hasNext() does not mean "the stream
> has
> > > not
> > > >> ended", but means "the next record is ready", which is repurposing
> the
> > > well
> > > >> known meaning of hasNext(). If we follow the hasNext()/next()
> pattern,
> > > an
> > > >> additional isNextReady() method to indicate whether the next record
> is
> > > >> ready seems more intuitive to me.
> > > >>
> > > >> Similarly, in poll()/take() pattern, another method of isDone() is
> > > needed
> > > >> to indicate whether the stream has ended or not.
> > > >>
> > > >> Compared with hasNext()/next()/isNextReady() pattern,
> > > >> isDone()/poll()/take() seems more flexible for the reader
> > > implementation.
> > > >> When I am implementing a reader, I could have a couple of choices:
> > > >>
> > > >>   - A thread-less reader that does not have any internal thread.
> > > >>   - When poll() is called, the same calling thread will perform a
> > bunch
> > > of
> > > >>      IO asynchronously.
> > > >>      - When take() is called, the same calling thread will perform a
> > > bunch
> > > >>      of IO and wait until the record is ready.
> > > >>   - A reader with internal threads performing network IO and put
> > records
> > > >>   into a buffer.
> > > >>      - When poll() is called, the calling thread simply reads from
> the
> > > >>      buffer and return empty result immediately if there is no
> record.
> > > >>      - When take() is called, the calling thread reads from the
> buffer
> > > and
> > > >>      block waiting if the buffer is empty.
> > > >>
> > > >> On the other hand, with the hasNext()/next()/isNextReady() API, it
> is
> > > less
> > > >> intuitive for the reader developers to write the thread-less
> pattern.
> > > >> Although technically speaking one can still do the asynchronous IO
> to
> > > >> prepare the record in isNextReady(). But it is inexplicit and seems
> > > >> somewhat hacky.
> > > >>
> > > >> Thanks,
> > > >>
> > > >> Jiangjie (Becket) Qin
> > > >>
> > > >>
> > > >>
> > > >> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
> > > >>
> > > >>> Couple more points regarding discovery:
> > > >>>
> > > >>> The proposal mentions that discovery could be outside the execution
> > > >> graph.
> > > >>> Today, discovered partitions/shards are checkpointed. I believe
> that
> > > will
> > > >>> also need to be the case in the future, even when discovery and
> > reading
> > > >> are
> > > >>> split between different tasks.
> > > >>>
> > > >>> For cases such as resharding of a Kinesis stream, the relationship
> > > >> between
> > > >>> splits needs to be considered. Splits cannot be randomly
> distributed
> > > over
> > > >>> readers in certain situations. An example was mentioned here:
> > > >>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> > > >>>
> > > >>> Thomas
> > > >>>
> > > >>>
> > > >>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org>
> wrote:
> > > >>>
> > > >>>> Thanks for getting the ball rolling on this!
> > > >>>>
> > > >>>> Can the number of splits decrease? Yes, splits can be closed and
> go
> > > >> away.
> > > >>>> An example would be a shard merge in Kinesis (2 existing shards
> will
> > > be
> > > >>>> closed and replaced with a new shard).
> > > >>>>
> > > >>>> Regarding advance/poll/take: IMO the least restrictive approach
> > would
> > > >> be
> > > >>>> the thread-less IO model (pull based, non-blocking, caller
> retrieves
> > > >> new
> > > >>>> records when available). The current Kinesis API requires the use
> of
> > > >>>> threads. But that can be internal to the split reader and does not
> > > need
> > > >>> to
> > > >>>> be a source API concern. In fact, that's what we are working on
> > right
> > > >> now
> > > >>>> as improvement to the existing consumer: Each shard consumer
> thread
> > > >> will
> > > >>>> push to a queue, the consumer main thread will poll the queue(s).
> It
> > > is
> > > >>>> essentially a mapping from threaded IO to non-blocking.
> > > >>>>
> > > >>>> The proposed SplitReader interface would fit the thread-less IO
> > model.
> > > >>>> Similar to an iterator, we find out if there is a new element
> > > (hasNext)
> > > >>> and
> > > >>>> if so, move to it (next()). Separate calls deliver the meta
> > > information
> > > >>>> (timestamp, watermark). Perhaps advance call could offer a timeout
> > > >>> option,
> > > >>>> so that the caller does not end up in a busy wait. On the other
> > hand,
> > > a
> > > >>>> caller processing multiple splits may want to cycle through fast,
> to
> > > >>>> process elements of other splits as soon as they become available.
> > The
> > > >>> nice
> > > >>>> thing is that this "split merge" logic can now live in Flink and
> be
> > > >>>> optimized and shared between different sources.
> > > >>>>
> > > >>>> Thanks,
> > > >>>> Thomas
> > > >>>>
> > > >>>>
> > > >>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
> > > wrote:
> > > >>>>
> > > >>>>> Hi,
> > > >>>>> Thanks Aljoscha for this FLIP.
> > > >>>>>
> > > >>>>> 1. I agree with Piotr and Becket that the non-blocking source is
> > very
> > > >>>>> important. But in addition to `Future/poll`, there may be another
> > way
> > > >> to
> > > >>>>> achieve this. I think it may be not very memory friendly if every
> > > >>> advance
> > > >>>>> call return a Future.
> > > >>>>>
> > > >>>>> public interface Listener {
> > > >>>>>     public void notify();
> > > >>>>> }
> > > >>>>>
> > > >>>>> public interface SplitReader() {
> > > >>>>>     /**
> > > >>>>>      * When there is no element temporarily, this will return
> > false.
> > > >>>>>      * When elements is available again splitReader can call
> > > >>>>> listener.notify()
> > > >>>>>      * In addition the frame would check `advance` periodically .
> > > >>>>>      * Of course advance can always return true and ignore the
> > > >> listener
> > > >>>>> argument for simplicity.
> > > >>>>>      */
> > > >>>>>     public boolean advance(Listener listener);
> > > >>>>> }
> > > >>>>>
> > > >>>>> 2.  The FLIP tells us very clearly that how to create all Splits
> > and
> > > >> how
> > > >>>>> to create a SplitReader from a Split. But there is no strategy
> for
> > > the
> > > >>> user
> > > >>>>> to choose how to assign the splits to the tasks. I think we could
> > add
> > > >> a
> > > >>>>> Enum to let user to choose.
> > > >>>>> /**
> > > >>>>>  public Enum SplitsAssignmentPolicy {
> > > >>>>>    Location,
> > > >>>>>    Workload,
> > > >>>>>    Random,
> > > >>>>>    Average
> > > >>>>>  }
> > > >>>>> */
> > > >>>>>
> > > >>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> > > >> `getNext`
> > > >>>>> the `getNext` would need return a `ElementWithTimestamp` because
> > some
> > > >>>>> sources want to add timestamp to every element. IMO, this is not
> so
> > > >>> memory
> > > >>>>> friendly so I prefer this design.
> > > >>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>> Thanks
> > > >>>>>
> > > >>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
> > > >>>>>
> > > >>>>>> Hi,
> > > >>>>>>
> > > >>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
> > > other
> > > >>>>>> possible improvements. I have one proposal. Instead of having a
> > > >> method:
> > > >>>>>>
> > > >>>>>> boolean advance() throws IOException;
> > > >>>>>>
> > > >>>>>> I would replace it with
> > > >>>>>>
> > > >>>>>> /*
> > > >>>>>> * Return a future, which when completed means that source has
> more
> > > >>> data
> > > >>>>>> and getNext() will not block.
> > > >>>>>> * If you wish to use benefits of non blocking connectors, please
> > > >>>>>> implement this method appropriately.
> > > >>>>>> */
> > > >>>>>> default CompletableFuture<?> isBlocked() {
> > > >>>>>>        return CompletableFuture.completedFuture(null);
> > > >>>>>> }
> > > >>>>>>
> > > >>>>>> And rename `getCurrent()` to `getNext()`.
> > > >>>>>>
> > > >>>>>> Couple of arguments:
> > > >>>>>> 1. I don’t understand the division of work between `advance()`
> and
> > > >>>>>> `getCurrent()`. What should be done in which, especially for
> > > >> connectors
> > > >>>>>> that handle records in batches (like Kafka) and when should you
> > call
> > > >>>>>> `advance` and when `getCurrent()`.
> > > >>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us
> > in
> > > >> the
> > > >>>>>> future to have asynchronous/non blocking connectors and more
> > > >>> efficiently
> > > >>>>>> handle large number of blocked threads, without busy waiting.
> > While
> > > >> at
> > > >>> the
> > > >>>>>> same time it doesn’t add much complexity, since naive connector
> > > >>>>>> implementations can be always blocking.
> > > >>>>>> 3. This also would allow us to use a fixed size thread pool of
> > task
> > > >>>>>> executors, instead of one thread per task.
> > > >>>>>>
> > > >>>>>> Piotrek
> > > >>>>>>
> > > >>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <
> aljoscha@apache.org>
> > > >>>>>> wrote:
> > > >>>>>>>
> > > >>>>>>> Hi All,
> > > >>>>>>>
> > > >>>>>>> In order to finally get the ball rolling on the new source
> > > >> interface
> > > >>>>>> that we have discussed for so long I finally created a FLIP:
> > > >>>>>>
> > > >>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > > >>>>>>>
> > > >>>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
> > > >> about
> > > >>>>>> adding per-partition watermark support to the Kinesis source and
> > > >>> because
> > > >>>>>> this would enable generic implementation of event-time alignment
> > for
> > > >>> all
> > > >>>>>> sources. Maybe we need another FLIP for the event-time alignment
> > > >> part,
> > > >>>>>> especially the part about information sharing between operations
> > > (I'm
> > > >>> not
> > > >>>>>> calling it state sharing because state has a special meaning in
> > > >> Flink).
> > > >>>>>>>
> > > >>>>>>> Please discuss away!
> > > >>>>>>>
> > > >>>>>>> Aljoscha
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>
> > > >>
> > >
> > >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Biao Liu <mm...@gmail.com>.
Regarding the naming style.

The advantage of `poll()` style is that basically the name of `poll` means
it should be a non-blocking operator, same with `Queue` in Java API. It's
easy to understand. We don't need to write too much in docs to imply the
implementation should not do something heavy.
However `poll` also means it should return the thing we want. In our
scenario, there are 3 types currently, record, timestamp and watermark. So
the return type of `poll` should be tuple3 or something like that. It looks
a little hacky IMO.

The `advance()` style is more like RecordReader
<https://hadoop.apache.org/docs/current/api/org/apache/hadoop/mapreduce/RecordReader.html>
of
MapReduce, or ISpout
<https://storm.apache.org/releases/1.1.2/javadocs/org/apache/storm/spout/ISpout.html>
of
Storm. It means moving the offset forward indeed. It makes sense to me.
To be honest I like `advance()` style more.

And there is also another small point I can't get.

Why use `start()` and `close()` in `SplitReader`? `start()` makes me think
of "starting a thread" or something like that. We should not assume there
would be some thread. I prefer `open()`, it also matches the `close()`
better.


Becket Qin <be...@gmail.com> 于2018年11月6日周二 上午11:04写道:

> Thanks for updating the wiki, Aljoscha.
>
> The isDone()/advance()/getCurrent() API looks more similar to
> hasNext()/isNextReady()/getNext(), but implying some different behaviors.
>
> If users call getCurrent() twice without calling advance() in between, will
> they get the same record back? From the API itself, users might think
> advance() is the API that moves the offset forward, and getCurrent() just
> return the record at the current offset.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Mon, Nov 5, 2018 at 10:41 PM Aljoscha Krettek <al...@apache.org>
> wrote:
>
> > I updated the FLIP [1] with some Javadoc for the SplitReader to outline
> > what I had in mind with the interface. Sorry for not doing that earlier,
> > it's not quite clear how the methods should work from the name alone.
> >
> > The gist of it is that advance() should be non-blocking, so
> > isDone/advance()/getCurrent() are very similar to isDone()/poll()/take()
> > that I have seen mentioned.
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > <
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> > >
> >
> > > On 5. Nov 2018, at 11:05, Biao Liu <mm...@gmail.com> wrote:
> > >
> > > Thanks Aljoscha for bringing us this discussion!
> > >
> > > 1. I think one of the reason about separating `advance()` and
> > > `getCurrent()` is that we have several different types returned by
> > source.
> > > Not just the `record`, but also the timestamp of record and the
> > watermark.
> > > If we don't separate these into different methods, the source has to
> > return
> > > a tuple3 which is not so user friendly. The prototype of Aljoscha is
> > > acceptable to me. Regarding the specific method name, I'm not sure
> which
> > > one is better. Both of them are reasonable for me.
> > >
> > > 2. As Thomas and Becket mentioned before, I think a non-blocking API is
> > > necessary. Moreover, IMO we should not offer a blocking API. It doesn't
> > > help but makes things more complicated.
> > >
> > > 3. About the thread model.
> > > I agree with Thomas about the thread-less IO model. A standard workflow
> > > should look like below.
> > >  - If there is available data, Flink would read it.
> > >  - If there is no data available temporary, Flink would check again a
> > > moment later. Maybe waiting on a semaphore until a timer wake it up.
> > > Furthermore, we can offer an optional optimization for source which has
> > > external thread. Like Guowei mentioned, there can be a listener which
> the
> > > reader can wake the framework up as soon as new data comes. This can
> > solve
> > > Piotr's concern about efficiency.
> > >
> > > 4. One more thing. After taking a look at the prototype codes. Off the
> > top
> > > of my head, the implementation is more fit for batch job not streaming
> > job.
> > > There are two types of tasks in prototype. First is a source task that
> > > discovers the splits. The source passes the splits to the second task
> > which
> > > process the splits one by one. And then the source keeps watch to
> > discover
> > > more splits.
> > >
> > > However, I think the more common scenario of streaming job is:
> > > there are fixed splits, each of the subtasks takes several splits. The
> > > subtasks just keep processing the fixed splits. There would be
> continuous
> > > datum in each split. We don't need a source task to discover more
> splits.
> > > It can not be finished in streaming job since we don't want the
> > processing
> > > task finished even there are no more splits.
> > >
> > > So IMO we should offer another source operator for the new interface.
> It
> > > would discover all splits when it is opening. Then picks the splits
> > belong
> > > to this subtask. Keep processing these splits until all of them are
> > > finished.
> > >
> > >
> > > Becket Qin <be...@gmail.com> 于2018年11月5日周一 上午11:00写道:
> > >
> > >> Hi Thomas,
> > >>
> > >> The iterator-like API was also the first thing that came to me. But it
> > >> seems a little confusing that hasNext() does not mean "the stream has
> > not
> > >> ended", but means "the next record is ready", which is repurposing the
> > well
> > >> known meaning of hasNext(). If we follow the hasNext()/next() pattern,
> > an
> > >> additional isNextReady() method to indicate whether the next record is
> > >> ready seems more intuitive to me.
> > >>
> > >> Similarly, in poll()/take() pattern, another method of isDone() is
> > needed
> > >> to indicate whether the stream has ended or not.
> > >>
> > >> Compared with hasNext()/next()/isNextReady() pattern,
> > >> isDone()/poll()/take() seems more flexible for the reader
> > implementation.
> > >> When I am implementing a reader, I could have a couple of choices:
> > >>
> > >>   - A thread-less reader that does not have any internal thread.
> > >>   - When poll() is called, the same calling thread will perform a
> bunch
> > of
> > >>      IO asynchronously.
> > >>      - When take() is called, the same calling thread will perform a
> > bunch
> > >>      of IO and wait until the record is ready.
> > >>   - A reader with internal threads performing network IO and put
> records
> > >>   into a buffer.
> > >>      - When poll() is called, the calling thread simply reads from the
> > >>      buffer and return empty result immediately if there is no record.
> > >>      - When take() is called, the calling thread reads from the buffer
> > and
> > >>      block waiting if the buffer is empty.
> > >>
> > >> On the other hand, with the hasNext()/next()/isNextReady() API, it is
> > less
> > >> intuitive for the reader developers to write the thread-less pattern.
> > >> Although technically speaking one can still do the asynchronous IO to
> > >> prepare the record in isNextReady(). But it is inexplicit and seems
> > >> somewhat hacky.
> > >>
> > >> Thanks,
> > >>
> > >> Jiangjie (Becket) Qin
> > >>
> > >>
> > >>
> > >> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
> > >>
> > >>> Couple more points regarding discovery:
> > >>>
> > >>> The proposal mentions that discovery could be outside the execution
> > >> graph.
> > >>> Today, discovered partitions/shards are checkpointed. I believe that
> > will
> > >>> also need to be the case in the future, even when discovery and
> reading
> > >> are
> > >>> split between different tasks.
> > >>>
> > >>> For cases such as resharding of a Kinesis stream, the relationship
> > >> between
> > >>> splits needs to be considered. Splits cannot be randomly distributed
> > over
> > >>> readers in certain situations. An example was mentioned here:
> > >>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> > >>>
> > >>> Thomas
> > >>>
> > >>>
> > >>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
> > >>>
> > >>>> Thanks for getting the ball rolling on this!
> > >>>>
> > >>>> Can the number of splits decrease? Yes, splits can be closed and go
> > >> away.
> > >>>> An example would be a shard merge in Kinesis (2 existing shards will
> > be
> > >>>> closed and replaced with a new shard).
> > >>>>
> > >>>> Regarding advance/poll/take: IMO the least restrictive approach
> would
> > >> be
> > >>>> the thread-less IO model (pull based, non-blocking, caller retrieves
> > >> new
> > >>>> records when available). The current Kinesis API requires the use of
> > >>>> threads. But that can be internal to the split reader and does not
> > need
> > >>> to
> > >>>> be a source API concern. In fact, that's what we are working on
> right
> > >> now
> > >>>> as improvement to the existing consumer: Each shard consumer thread
> > >> will
> > >>>> push to a queue, the consumer main thread will poll the queue(s). It
> > is
> > >>>> essentially a mapping from threaded IO to non-blocking.
> > >>>>
> > >>>> The proposed SplitReader interface would fit the thread-less IO
> model.
> > >>>> Similar to an iterator, we find out if there is a new element
> > (hasNext)
> > >>> and
> > >>>> if so, move to it (next()). Separate calls deliver the meta
> > information
> > >>>> (timestamp, watermark). Perhaps advance call could offer a timeout
> > >>> option,
> > >>>> so that the caller does not end up in a busy wait. On the other
> hand,
> > a
> > >>>> caller processing multiple splits may want to cycle through fast, to
> > >>>> process elements of other splits as soon as they become available.
> The
> > >>> nice
> > >>>> thing is that this "split merge" logic can now live in Flink and be
> > >>>> optimized and shared between different sources.
> > >>>>
> > >>>> Thanks,
> > >>>> Thomas
> > >>>>
> > >>>>
> > >>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
> > wrote:
> > >>>>
> > >>>>> Hi,
> > >>>>> Thanks Aljoscha for this FLIP.
> > >>>>>
> > >>>>> 1. I agree with Piotr and Becket that the non-blocking source is
> very
> > >>>>> important. But in addition to `Future/poll`, there may be another
> way
> > >> to
> > >>>>> achieve this. I think it may be not very memory friendly if every
> > >>> advance
> > >>>>> call return a Future.
> > >>>>>
> > >>>>> public interface Listener {
> > >>>>>     public void notify();
> > >>>>> }
> > >>>>>
> > >>>>> public interface SplitReader() {
> > >>>>>     /**
> > >>>>>      * When there is no element temporarily, this will return
> false.
> > >>>>>      * When elements is available again splitReader can call
> > >>>>> listener.notify()
> > >>>>>      * In addition the frame would check `advance` periodically .
> > >>>>>      * Of course advance can always return true and ignore the
> > >> listener
> > >>>>> argument for simplicity.
> > >>>>>      */
> > >>>>>     public boolean advance(Listener listener);
> > >>>>> }
> > >>>>>
> > >>>>> 2.  The FLIP tells us very clearly that how to create all Splits
> and
> > >> how
> > >>>>> to create a SplitReader from a Split. But there is no strategy for
> > the
> > >>> user
> > >>>>> to choose how to assign the splits to the tasks. I think we could
> add
> > >> a
> > >>>>> Enum to let user to choose.
> > >>>>> /**
> > >>>>>  public Enum SplitsAssignmentPolicy {
> > >>>>>    Location,
> > >>>>>    Workload,
> > >>>>>    Random,
> > >>>>>    Average
> > >>>>>  }
> > >>>>> */
> > >>>>>
> > >>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> > >> `getNext`
> > >>>>> the `getNext` would need return a `ElementWithTimestamp` because
> some
> > >>>>> sources want to add timestamp to every element. IMO, this is not so
> > >>> memory
> > >>>>> friendly so I prefer this design.
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> Thanks
> > >>>>>
> > >>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
> > >>>>>
> > >>>>>> Hi,
> > >>>>>>
> > >>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
> > other
> > >>>>>> possible improvements. I have one proposal. Instead of having a
> > >> method:
> > >>>>>>
> > >>>>>> boolean advance() throws IOException;
> > >>>>>>
> > >>>>>> I would replace it with
> > >>>>>>
> > >>>>>> /*
> > >>>>>> * Return a future, which when completed means that source has more
> > >>> data
> > >>>>>> and getNext() will not block.
> > >>>>>> * If you wish to use benefits of non blocking connectors, please
> > >>>>>> implement this method appropriately.
> > >>>>>> */
> > >>>>>> default CompletableFuture<?> isBlocked() {
> > >>>>>>        return CompletableFuture.completedFuture(null);
> > >>>>>> }
> > >>>>>>
> > >>>>>> And rename `getCurrent()` to `getNext()`.
> > >>>>>>
> > >>>>>> Couple of arguments:
> > >>>>>> 1. I don’t understand the division of work between `advance()` and
> > >>>>>> `getCurrent()`. What should be done in which, especially for
> > >> connectors
> > >>>>>> that handle records in batches (like Kafka) and when should you
> call
> > >>>>>> `advance` and when `getCurrent()`.
> > >>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us
> in
> > >> the
> > >>>>>> future to have asynchronous/non blocking connectors and more
> > >>> efficiently
> > >>>>>> handle large number of blocked threads, without busy waiting.
> While
> > >> at
> > >>> the
> > >>>>>> same time it doesn’t add much complexity, since naive connector
> > >>>>>> implementations can be always blocking.
> > >>>>>> 3. This also would allow us to use a fixed size thread pool of
> task
> > >>>>>> executors, instead of one thread per task.
> > >>>>>>
> > >>>>>> Piotrek
> > >>>>>>
> > >>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
> > >>>>>> wrote:
> > >>>>>>>
> > >>>>>>> Hi All,
> > >>>>>>>
> > >>>>>>> In order to finally get the ball rolling on the new source
> > >> interface
> > >>>>>> that we have discussed for so long I finally created a FLIP:
> > >>>>>>
> > >>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >>>>>>>
> > >>>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
> > >> about
> > >>>>>> adding per-partition watermark support to the Kinesis source and
> > >>> because
> > >>>>>> this would enable generic implementation of event-time alignment
> for
> > >>> all
> > >>>>>> sources. Maybe we need another FLIP for the event-time alignment
> > >> part,
> > >>>>>> especially the part about information sharing between operations
> > (I'm
> > >>> not
> > >>>>>> calling it state sharing because state has a special meaning in
> > >> Flink).
> > >>>>>>>
> > >>>>>>> Please discuss away!
> > >>>>>>>
> > >>>>>>> Aljoscha
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>
> > >>
> >
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Thanks for updating the wiki, Aljoscha.

The isDone()/advance()/getCurrent() API looks more similar to
hasNext()/isNextReady()/getNext(), but implying some different behaviors.

If users call getCurrent() twice without calling advance() in between, will
they get the same record back? From the API itself, users might think
advance() is the API that moves the offset forward, and getCurrent() just
return the record at the current offset.

Thanks,

Jiangjie (Becket) Qin

On Mon, Nov 5, 2018 at 10:41 PM Aljoscha Krettek <al...@apache.org>
wrote:

> I updated the FLIP [1] with some Javadoc for the SplitReader to outline
> what I had in mind with the interface. Sorry for not doing that earlier,
> it's not quite clear how the methods should work from the name alone.
>
> The gist of it is that advance() should be non-blocking, so
> isDone/advance()/getCurrent() are very similar to isDone()/poll()/take()
> that I have seen mentioned.
>
> [1]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> <
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface
> >
>
> > On 5. Nov 2018, at 11:05, Biao Liu <mm...@gmail.com> wrote:
> >
> > Thanks Aljoscha for bringing us this discussion!
> >
> > 1. I think one of the reason about separating `advance()` and
> > `getCurrent()` is that we have several different types returned by
> source.
> > Not just the `record`, but also the timestamp of record and the
> watermark.
> > If we don't separate these into different methods, the source has to
> return
> > a tuple3 which is not so user friendly. The prototype of Aljoscha is
> > acceptable to me. Regarding the specific method name, I'm not sure which
> > one is better. Both of them are reasonable for me.
> >
> > 2. As Thomas and Becket mentioned before, I think a non-blocking API is
> > necessary. Moreover, IMO we should not offer a blocking API. It doesn't
> > help but makes things more complicated.
> >
> > 3. About the thread model.
> > I agree with Thomas about the thread-less IO model. A standard workflow
> > should look like below.
> >  - If there is available data, Flink would read it.
> >  - If there is no data available temporary, Flink would check again a
> > moment later. Maybe waiting on a semaphore until a timer wake it up.
> > Furthermore, we can offer an optional optimization for source which has
> > external thread. Like Guowei mentioned, there can be a listener which the
> > reader can wake the framework up as soon as new data comes. This can
> solve
> > Piotr's concern about efficiency.
> >
> > 4. One more thing. After taking a look at the prototype codes. Off the
> top
> > of my head, the implementation is more fit for batch job not streaming
> job.
> > There are two types of tasks in prototype. First is a source task that
> > discovers the splits. The source passes the splits to the second task
> which
> > process the splits one by one. And then the source keeps watch to
> discover
> > more splits.
> >
> > However, I think the more common scenario of streaming job is:
> > there are fixed splits, each of the subtasks takes several splits. The
> > subtasks just keep processing the fixed splits. There would be continuous
> > datum in each split. We don't need a source task to discover more splits.
> > It can not be finished in streaming job since we don't want the
> processing
> > task finished even there are no more splits.
> >
> > So IMO we should offer another source operator for the new interface. It
> > would discover all splits when it is opening. Then picks the splits
> belong
> > to this subtask. Keep processing these splits until all of them are
> > finished.
> >
> >
> > Becket Qin <be...@gmail.com> 于2018年11月5日周一 上午11:00写道:
> >
> >> Hi Thomas,
> >>
> >> The iterator-like API was also the first thing that came to me. But it
> >> seems a little confusing that hasNext() does not mean "the stream has
> not
> >> ended", but means "the next record is ready", which is repurposing the
> well
> >> known meaning of hasNext(). If we follow the hasNext()/next() pattern,
> an
> >> additional isNextReady() method to indicate whether the next record is
> >> ready seems more intuitive to me.
> >>
> >> Similarly, in poll()/take() pattern, another method of isDone() is
> needed
> >> to indicate whether the stream has ended or not.
> >>
> >> Compared with hasNext()/next()/isNextReady() pattern,
> >> isDone()/poll()/take() seems more flexible for the reader
> implementation.
> >> When I am implementing a reader, I could have a couple of choices:
> >>
> >>   - A thread-less reader that does not have any internal thread.
> >>   - When poll() is called, the same calling thread will perform a bunch
> of
> >>      IO asynchronously.
> >>      - When take() is called, the same calling thread will perform a
> bunch
> >>      of IO and wait until the record is ready.
> >>   - A reader with internal threads performing network IO and put records
> >>   into a buffer.
> >>      - When poll() is called, the calling thread simply reads from the
> >>      buffer and return empty result immediately if there is no record.
> >>      - When take() is called, the calling thread reads from the buffer
> and
> >>      block waiting if the buffer is empty.
> >>
> >> On the other hand, with the hasNext()/next()/isNextReady() API, it is
> less
> >> intuitive for the reader developers to write the thread-less pattern.
> >> Although technically speaking one can still do the asynchronous IO to
> >> prepare the record in isNextReady(). But it is inexplicit and seems
> >> somewhat hacky.
> >>
> >> Thanks,
> >>
> >> Jiangjie (Becket) Qin
> >>
> >>
> >>
> >> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
> >>
> >>> Couple more points regarding discovery:
> >>>
> >>> The proposal mentions that discovery could be outside the execution
> >> graph.
> >>> Today, discovered partitions/shards are checkpointed. I believe that
> will
> >>> also need to be the case in the future, even when discovery and reading
> >> are
> >>> split between different tasks.
> >>>
> >>> For cases such as resharding of a Kinesis stream, the relationship
> >> between
> >>> splits needs to be considered. Splits cannot be randomly distributed
> over
> >>> readers in certain situations. An example was mentioned here:
> >>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >>>
> >>> Thomas
> >>>
> >>>
> >>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
> >>>
> >>>> Thanks for getting the ball rolling on this!
> >>>>
> >>>> Can the number of splits decrease? Yes, splits can be closed and go
> >> away.
> >>>> An example would be a shard merge in Kinesis (2 existing shards will
> be
> >>>> closed and replaced with a new shard).
> >>>>
> >>>> Regarding advance/poll/take: IMO the least restrictive approach would
> >> be
> >>>> the thread-less IO model (pull based, non-blocking, caller retrieves
> >> new
> >>>> records when available). The current Kinesis API requires the use of
> >>>> threads. But that can be internal to the split reader and does not
> need
> >>> to
> >>>> be a source API concern. In fact, that's what we are working on right
> >> now
> >>>> as improvement to the existing consumer: Each shard consumer thread
> >> will
> >>>> push to a queue, the consumer main thread will poll the queue(s). It
> is
> >>>> essentially a mapping from threaded IO to non-blocking.
> >>>>
> >>>> The proposed SplitReader interface would fit the thread-less IO model.
> >>>> Similar to an iterator, we find out if there is a new element
> (hasNext)
> >>> and
> >>>> if so, move to it (next()). Separate calls deliver the meta
> information
> >>>> (timestamp, watermark). Perhaps advance call could offer a timeout
> >>> option,
> >>>> so that the caller does not end up in a busy wait. On the other hand,
> a
> >>>> caller processing multiple splits may want to cycle through fast, to
> >>>> process elements of other splits as soon as they become available. The
> >>> nice
> >>>> thing is that this "split merge" logic can now live in Flink and be
> >>>> optimized and shared between different sources.
> >>>>
> >>>> Thanks,
> >>>> Thomas
> >>>>
> >>>>
> >>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com>
> wrote:
> >>>>
> >>>>> Hi,
> >>>>> Thanks Aljoscha for this FLIP.
> >>>>>
> >>>>> 1. I agree with Piotr and Becket that the non-blocking source is very
> >>>>> important. But in addition to `Future/poll`, there may be another way
> >> to
> >>>>> achieve this. I think it may be not very memory friendly if every
> >>> advance
> >>>>> call return a Future.
> >>>>>
> >>>>> public interface Listener {
> >>>>>     public void notify();
> >>>>> }
> >>>>>
> >>>>> public interface SplitReader() {
> >>>>>     /**
> >>>>>      * When there is no element temporarily, this will return false.
> >>>>>      * When elements is available again splitReader can call
> >>>>> listener.notify()
> >>>>>      * In addition the frame would check `advance` periodically .
> >>>>>      * Of course advance can always return true and ignore the
> >> listener
> >>>>> argument for simplicity.
> >>>>>      */
> >>>>>     public boolean advance(Listener listener);
> >>>>> }
> >>>>>
> >>>>> 2.  The FLIP tells us very clearly that how to create all Splits and
> >> how
> >>>>> to create a SplitReader from a Split. But there is no strategy for
> the
> >>> user
> >>>>> to choose how to assign the splits to the tasks. I think we could add
> >> a
> >>>>> Enum to let user to choose.
> >>>>> /**
> >>>>>  public Enum SplitsAssignmentPolicy {
> >>>>>    Location,
> >>>>>    Workload,
> >>>>>    Random,
> >>>>>    Average
> >>>>>  }
> >>>>> */
> >>>>>
> >>>>> 3. If merge the `advance` and `getCurrent`  to one method like
> >> `getNext`
> >>>>> the `getNext` would need return a `ElementWithTimestamp` because some
> >>>>> sources want to add timestamp to every element. IMO, this is not so
> >>> memory
> >>>>> friendly so I prefer this design.
> >>>>>
> >>>>>
> >>>>>
> >>>>> Thanks
> >>>>>
> >>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
> >>>>>
> >>>>>> Hi,
> >>>>>>
> >>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of
> other
> >>>>>> possible improvements. I have one proposal. Instead of having a
> >> method:
> >>>>>>
> >>>>>> boolean advance() throws IOException;
> >>>>>>
> >>>>>> I would replace it with
> >>>>>>
> >>>>>> /*
> >>>>>> * Return a future, which when completed means that source has more
> >>> data
> >>>>>> and getNext() will not block.
> >>>>>> * If you wish to use benefits of non blocking connectors, please
> >>>>>> implement this method appropriately.
> >>>>>> */
> >>>>>> default CompletableFuture<?> isBlocked() {
> >>>>>>        return CompletableFuture.completedFuture(null);
> >>>>>> }
> >>>>>>
> >>>>>> And rename `getCurrent()` to `getNext()`.
> >>>>>>
> >>>>>> Couple of arguments:
> >>>>>> 1. I don’t understand the division of work between `advance()` and
> >>>>>> `getCurrent()`. What should be done in which, especially for
> >> connectors
> >>>>>> that handle records in batches (like Kafka) and when should you call
> >>>>>> `advance` and when `getCurrent()`.
> >>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in
> >> the
> >>>>>> future to have asynchronous/non blocking connectors and more
> >>> efficiently
> >>>>>> handle large number of blocked threads, without busy waiting. While
> >> at
> >>> the
> >>>>>> same time it doesn’t add much complexity, since naive connector
> >>>>>> implementations can be always blocking.
> >>>>>> 3. This also would allow us to use a fixed size thread pool of task
> >>>>>> executors, instead of one thread per task.
> >>>>>>
> >>>>>> Piotrek
> >>>>>>
> >>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
> >>>>>> wrote:
> >>>>>>>
> >>>>>>> Hi All,
> >>>>>>>
> >>>>>>> In order to finally get the ball rolling on the new source
> >> interface
> >>>>>> that we have discussed for so long I finally created a FLIP:
> >>>>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>>>>>>
> >>>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
> >> about
> >>>>>> adding per-partition watermark support to the Kinesis source and
> >>> because
> >>>>>> this would enable generic implementation of event-time alignment for
> >>> all
> >>>>>> sources. Maybe we need another FLIP for the event-time alignment
> >> part,
> >>>>>> especially the part about information sharing between operations
> (I'm
> >>> not
> >>>>>> calling it state sharing because state has a special meaning in
> >> Flink).
> >>>>>>>
> >>>>>>> Please discuss away!
> >>>>>>>
> >>>>>>> Aljoscha
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>
> >>
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Aljoscha Krettek <al...@apache.org>.
I updated the FLIP [1] with some Javadoc for the SplitReader to outline what I had in mind with the interface. Sorry for not doing that earlier, it's not quite clear how the methods should work from the name alone.

The gist of it is that advance() should be non-blocking, so isDone/advance()/getCurrent() are very similar to isDone()/poll()/take() that I have seen mentioned.

[1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface <https://cwiki.apache.org/confluence/display/FLINK/FLIP-27:+Refactor+Source+Interface>

> On 5. Nov 2018, at 11:05, Biao Liu <mm...@gmail.com> wrote:
> 
> Thanks Aljoscha for bringing us this discussion!
> 
> 1. I think one of the reason about separating `advance()` and
> `getCurrent()` is that we have several different types returned by source.
> Not just the `record`, but also the timestamp of record and the watermark.
> If we don't separate these into different methods, the source has to return
> a tuple3 which is not so user friendly. The prototype of Aljoscha is
> acceptable to me. Regarding the specific method name, I'm not sure which
> one is better. Both of them are reasonable for me.
> 
> 2. As Thomas and Becket mentioned before, I think a non-blocking API is
> necessary. Moreover, IMO we should not offer a blocking API. It doesn't
> help but makes things more complicated.
> 
> 3. About the thread model.
> I agree with Thomas about the thread-less IO model. A standard workflow
> should look like below.
>  - If there is available data, Flink would read it.
>  - If there is no data available temporary, Flink would check again a
> moment later. Maybe waiting on a semaphore until a timer wake it up.
> Furthermore, we can offer an optional optimization for source which has
> external thread. Like Guowei mentioned, there can be a listener which the
> reader can wake the framework up as soon as new data comes. This can solve
> Piotr's concern about efficiency.
> 
> 4. One more thing. After taking a look at the prototype codes. Off the top
> of my head, the implementation is more fit for batch job not streaming job.
> There are two types of tasks in prototype. First is a source task that
> discovers the splits. The source passes the splits to the second task which
> process the splits one by one. And then the source keeps watch to discover
> more splits.
> 
> However, I think the more common scenario of streaming job is:
> there are fixed splits, each of the subtasks takes several splits. The
> subtasks just keep processing the fixed splits. There would be continuous
> datum in each split. We don't need a source task to discover more splits.
> It can not be finished in streaming job since we don't want the processing
> task finished even there are no more splits.
> 
> So IMO we should offer another source operator for the new interface. It
> would discover all splits when it is opening. Then picks the splits belong
> to this subtask. Keep processing these splits until all of them are
> finished.
> 
> 
> Becket Qin <be...@gmail.com> 于2018年11月5日周一 上午11:00写道:
> 
>> Hi Thomas,
>> 
>> The iterator-like API was also the first thing that came to me. But it
>> seems a little confusing that hasNext() does not mean "the stream has not
>> ended", but means "the next record is ready", which is repurposing the well
>> known meaning of hasNext(). If we follow the hasNext()/next() pattern, an
>> additional isNextReady() method to indicate whether the next record is
>> ready seems more intuitive to me.
>> 
>> Similarly, in poll()/take() pattern, another method of isDone() is needed
>> to indicate whether the stream has ended or not.
>> 
>> Compared with hasNext()/next()/isNextReady() pattern,
>> isDone()/poll()/take() seems more flexible for the reader implementation.
>> When I am implementing a reader, I could have a couple of choices:
>> 
>>   - A thread-less reader that does not have any internal thread.
>>   - When poll() is called, the same calling thread will perform a bunch of
>>      IO asynchronously.
>>      - When take() is called, the same calling thread will perform a bunch
>>      of IO and wait until the record is ready.
>>   - A reader with internal threads performing network IO and put records
>>   into a buffer.
>>      - When poll() is called, the calling thread simply reads from the
>>      buffer and return empty result immediately if there is no record.
>>      - When take() is called, the calling thread reads from the buffer and
>>      block waiting if the buffer is empty.
>> 
>> On the other hand, with the hasNext()/next()/isNextReady() API, it is less
>> intuitive for the reader developers to write the thread-less pattern.
>> Although technically speaking one can still do the asynchronous IO to
>> prepare the record in isNextReady(). But it is inexplicit and seems
>> somewhat hacky.
>> 
>> Thanks,
>> 
>> Jiangjie (Becket) Qin
>> 
>> 
>> 
>> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
>> 
>>> Couple more points regarding discovery:
>>> 
>>> The proposal mentions that discovery could be outside the execution
>> graph.
>>> Today, discovered partitions/shards are checkpointed. I believe that will
>>> also need to be the case in the future, even when discovery and reading
>> are
>>> split between different tasks.
>>> 
>>> For cases such as resharding of a Kinesis stream, the relationship
>> between
>>> splits needs to be considered. Splits cannot be randomly distributed over
>>> readers in certain situations. An example was mentioned here:
>>> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>>> 
>>> Thomas
>>> 
>>> 
>>> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
>>> 
>>>> Thanks for getting the ball rolling on this!
>>>> 
>>>> Can the number of splits decrease? Yes, splits can be closed and go
>> away.
>>>> An example would be a shard merge in Kinesis (2 existing shards will be
>>>> closed and replaced with a new shard).
>>>> 
>>>> Regarding advance/poll/take: IMO the least restrictive approach would
>> be
>>>> the thread-less IO model (pull based, non-blocking, caller retrieves
>> new
>>>> records when available). The current Kinesis API requires the use of
>>>> threads. But that can be internal to the split reader and does not need
>>> to
>>>> be a source API concern. In fact, that's what we are working on right
>> now
>>>> as improvement to the existing consumer: Each shard consumer thread
>> will
>>>> push to a queue, the consumer main thread will poll the queue(s). It is
>>>> essentially a mapping from threaded IO to non-blocking.
>>>> 
>>>> The proposed SplitReader interface would fit the thread-less IO model.
>>>> Similar to an iterator, we find out if there is a new element (hasNext)
>>> and
>>>> if so, move to it (next()). Separate calls deliver the meta information
>>>> (timestamp, watermark). Perhaps advance call could offer a timeout
>>> option,
>>>> so that the caller does not end up in a busy wait. On the other hand, a
>>>> caller processing multiple splits may want to cycle through fast, to
>>>> process elements of other splits as soon as they become available. The
>>> nice
>>>> thing is that this "split merge" logic can now live in Flink and be
>>>> optimized and shared between different sources.
>>>> 
>>>> Thanks,
>>>> Thomas
>>>> 
>>>> 
>>>> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com> wrote:
>>>> 
>>>>> Hi,
>>>>> Thanks Aljoscha for this FLIP.
>>>>> 
>>>>> 1. I agree with Piotr and Becket that the non-blocking source is very
>>>>> important. But in addition to `Future/poll`, there may be another way
>> to
>>>>> achieve this. I think it may be not very memory friendly if every
>>> advance
>>>>> call return a Future.
>>>>> 
>>>>> public interface Listener {
>>>>>     public void notify();
>>>>> }
>>>>> 
>>>>> public interface SplitReader() {
>>>>>     /**
>>>>>      * When there is no element temporarily, this will return false.
>>>>>      * When elements is available again splitReader can call
>>>>> listener.notify()
>>>>>      * In addition the frame would check `advance` periodically .
>>>>>      * Of course advance can always return true and ignore the
>> listener
>>>>> argument for simplicity.
>>>>>      */
>>>>>     public boolean advance(Listener listener);
>>>>> }
>>>>> 
>>>>> 2.  The FLIP tells us very clearly that how to create all Splits and
>> how
>>>>> to create a SplitReader from a Split. But there is no strategy for the
>>> user
>>>>> to choose how to assign the splits to the tasks. I think we could add
>> a
>>>>> Enum to let user to choose.
>>>>> /**
>>>>>  public Enum SplitsAssignmentPolicy {
>>>>>    Location,
>>>>>    Workload,
>>>>>    Random,
>>>>>    Average
>>>>>  }
>>>>> */
>>>>> 
>>>>> 3. If merge the `advance` and `getCurrent`  to one method like
>> `getNext`
>>>>> the `getNext` would need return a `ElementWithTimestamp` because some
>>>>> sources want to add timestamp to every element. IMO, this is not so
>>> memory
>>>>> friendly so I prefer this design.
>>>>> 
>>>>> 
>>>>> 
>>>>> Thanks
>>>>> 
>>>>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
>>>>> 
>>>>>> Hi,
>>>>>> 
>>>>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
>>>>>> possible improvements. I have one proposal. Instead of having a
>> method:
>>>>>> 
>>>>>> boolean advance() throws IOException;
>>>>>> 
>>>>>> I would replace it with
>>>>>> 
>>>>>> /*
>>>>>> * Return a future, which when completed means that source has more
>>> data
>>>>>> and getNext() will not block.
>>>>>> * If you wish to use benefits of non blocking connectors, please
>>>>>> implement this method appropriately.
>>>>>> */
>>>>>> default CompletableFuture<?> isBlocked() {
>>>>>>        return CompletableFuture.completedFuture(null);
>>>>>> }
>>>>>> 
>>>>>> And rename `getCurrent()` to `getNext()`.
>>>>>> 
>>>>>> Couple of arguments:
>>>>>> 1. I don’t understand the division of work between `advance()` and
>>>>>> `getCurrent()`. What should be done in which, especially for
>> connectors
>>>>>> that handle records in batches (like Kafka) and when should you call
>>>>>> `advance` and when `getCurrent()`.
>>>>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in
>> the
>>>>>> future to have asynchronous/non blocking connectors and more
>>> efficiently
>>>>>> handle large number of blocked threads, without busy waiting. While
>> at
>>> the
>>>>>> same time it doesn’t add much complexity, since naive connector
>>>>>> implementations can be always blocking.
>>>>>> 3. This also would allow us to use a fixed size thread pool of task
>>>>>> executors, instead of one thread per task.
>>>>>> 
>>>>>> Piotrek
>>>>>> 
>>>>>>> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
>>>>>> wrote:
>>>>>>> 
>>>>>>> Hi All,
>>>>>>> 
>>>>>>> In order to finally get the ball rolling on the new source
>> interface
>>>>>> that we have discussed for so long I finally created a FLIP:
>>>>>> 
>>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>>>>>> 
>>>>>>> I cc'ed Thomas and Jamie because of the ongoing work/discussion
>> about
>>>>>> adding per-partition watermark support to the Kinesis source and
>>> because
>>>>>> this would enable generic implementation of event-time alignment for
>>> all
>>>>>> sources. Maybe we need another FLIP for the event-time alignment
>> part,
>>>>>> especially the part about information sharing between operations (I'm
>>> not
>>>>>> calling it state sharing because state has a special meaning in
>> Flink).
>>>>>>> 
>>>>>>> Please discuss away!
>>>>>>> 
>>>>>>> Aljoscha
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>> 
>> 


Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Biao Liu <mm...@gmail.com>.
Thanks Aljoscha for bringing us this discussion!

1. I think one of the reason about separating `advance()` and
`getCurrent()` is that we have several different types returned by source.
Not just the `record`, but also the timestamp of record and the watermark.
If we don't separate these into different methods, the source has to return
a tuple3 which is not so user friendly. The prototype of Aljoscha is
acceptable to me. Regarding the specific method name, I'm not sure which
one is better. Both of them are reasonable for me.

2. As Thomas and Becket mentioned before, I think a non-blocking API is
necessary. Moreover, IMO we should not offer a blocking API. It doesn't
help but makes things more complicated.

3. About the thread model.
I agree with Thomas about the thread-less IO model. A standard workflow
should look like below.
  - If there is available data, Flink would read it.
  - If there is no data available temporary, Flink would check again a
moment later. Maybe waiting on a semaphore until a timer wake it up.
Furthermore, we can offer an optional optimization for source which has
external thread. Like Guowei mentioned, there can be a listener which the
reader can wake the framework up as soon as new data comes. This can solve
Piotr's concern about efficiency.

4. One more thing. After taking a look at the prototype codes. Off the top
of my head, the implementation is more fit for batch job not streaming job.
There are two types of tasks in prototype. First is a source task that
discovers the splits. The source passes the splits to the second task which
process the splits one by one. And then the source keeps watch to discover
more splits.

However, I think the more common scenario of streaming job is:
there are fixed splits, each of the subtasks takes several splits. The
subtasks just keep processing the fixed splits. There would be continuous
datum in each split. We don't need a source task to discover more splits.
It can not be finished in streaming job since we don't want the processing
task finished even there are no more splits.

So IMO we should offer another source operator for the new interface. It
would discover all splits when it is opening. Then picks the splits belong
to this subtask. Keep processing these splits until all of them are
finished.


Becket Qin <be...@gmail.com> 于2018年11月5日周一 上午11:00写道:

> Hi Thomas,
>
> The iterator-like API was also the first thing that came to me. But it
> seems a little confusing that hasNext() does not mean "the stream has not
> ended", but means "the next record is ready", which is repurposing the well
> known meaning of hasNext(). If we follow the hasNext()/next() pattern, an
> additional isNextReady() method to indicate whether the next record is
> ready seems more intuitive to me.
>
> Similarly, in poll()/take() pattern, another method of isDone() is needed
> to indicate whether the stream has ended or not.
>
> Compared with hasNext()/next()/isNextReady() pattern,
> isDone()/poll()/take() seems more flexible for the reader implementation.
> When I am implementing a reader, I could have a couple of choices:
>
>    - A thread-less reader that does not have any internal thread.
>    - When poll() is called, the same calling thread will perform a bunch of
>       IO asynchronously.
>       - When take() is called, the same calling thread will perform a bunch
>       of IO and wait until the record is ready.
>    - A reader with internal threads performing network IO and put records
>    into a buffer.
>       - When poll() is called, the calling thread simply reads from the
>       buffer and return empty result immediately if there is no record.
>       - When take() is called, the calling thread reads from the buffer and
>       block waiting if the buffer is empty.
>
> On the other hand, with the hasNext()/next()/isNextReady() API, it is less
> intuitive for the reader developers to write the thread-less pattern.
> Although technically speaking one can still do the asynchronous IO to
> prepare the record in isNextReady(). But it is inexplicit and seems
> somewhat hacky.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
> On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:
>
> > Couple more points regarding discovery:
> >
> > The proposal mentions that discovery could be outside the execution
> graph.
> > Today, discovered partitions/shards are checkpointed. I believe that will
> > also need to be the case in the future, even when discovery and reading
> are
> > split between different tasks.
> >
> > For cases such as resharding of a Kinesis stream, the relationship
> between
> > splits needs to be considered. Splits cannot be randomly distributed over
> > readers in certain situations. An example was mentioned here:
> > https://github.com/apache/flink/pull/6980#issuecomment-435202809
> >
> > Thomas
> >
> >
> > On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
> >
> > > Thanks for getting the ball rolling on this!
> > >
> > > Can the number of splits decrease? Yes, splits can be closed and go
> away.
> > > An example would be a shard merge in Kinesis (2 existing shards will be
> > > closed and replaced with a new shard).
> > >
> > > Regarding advance/poll/take: IMO the least restrictive approach would
> be
> > > the thread-less IO model (pull based, non-blocking, caller retrieves
> new
> > > records when available). The current Kinesis API requires the use of
> > > threads. But that can be internal to the split reader and does not need
> > to
> > > be a source API concern. In fact, that's what we are working on right
> now
> > > as improvement to the existing consumer: Each shard consumer thread
> will
> > > push to a queue, the consumer main thread will poll the queue(s). It is
> > > essentially a mapping from threaded IO to non-blocking.
> > >
> > > The proposed SplitReader interface would fit the thread-less IO model.
> > > Similar to an iterator, we find out if there is a new element (hasNext)
> > and
> > > if so, move to it (next()). Separate calls deliver the meta information
> > > (timestamp, watermark). Perhaps advance call could offer a timeout
> > option,
> > > so that the caller does not end up in a busy wait. On the other hand, a
> > > caller processing multiple splits may want to cycle through fast, to
> > > process elements of other splits as soon as they become available. The
> > nice
> > > thing is that this "split merge" logic can now live in Flink and be
> > > optimized and shared between different sources.
> > >
> > > Thanks,
> > > Thomas
> > >
> > >
> > > On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com> wrote:
> > >
> > >> Hi,
> > >> Thanks Aljoscha for this FLIP.
> > >>
> > >> 1. I agree with Piotr and Becket that the non-blocking source is very
> > >> important. But in addition to `Future/poll`, there may be another way
> to
> > >> achieve this. I think it may be not very memory friendly if every
> > advance
> > >> call return a Future.
> > >>
> > >> public interface Listener {
> > >>      public void notify();
> > >> }
> > >>
> > >> public interface SplitReader() {
> > >>      /**
> > >>       * When there is no element temporarily, this will return false.
> > >>       * When elements is available again splitReader can call
> > >> listener.notify()
> > >>       * In addition the frame would check `advance` periodically .
> > >>       * Of course advance can always return true and ignore the
> listener
> > >> argument for simplicity.
> > >>       */
> > >>      public boolean advance(Listener listener);
> > >> }
> > >>
> > >> 2.  The FLIP tells us very clearly that how to create all Splits and
> how
> > >> to create a SplitReader from a Split. But there is no strategy for the
> > user
> > >> to choose how to assign the splits to the tasks. I think we could add
> a
> > >> Enum to let user to choose.
> > >> /**
> > >>   public Enum SplitsAssignmentPolicy {
> > >>     Location,
> > >>     Workload,
> > >>     Random,
> > >>     Average
> > >>   }
> > >> */
> > >>
> > >> 3. If merge the `advance` and `getCurrent`  to one method like
> `getNext`
> > >> the `getNext` would need return a `ElementWithTimestamp` because some
> > >> sources want to add timestamp to every element. IMO, this is not so
> > memory
> > >> friendly so I prefer this design.
> > >>
> > >>
> > >>
> > >> Thanks
> > >>
> > >> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
> > >>
> > >>> Hi,
> > >>>
> > >>> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
> > >>> possible improvements. I have one proposal. Instead of having a
> method:
> > >>>
> > >>> boolean advance() throws IOException;
> > >>>
> > >>> I would replace it with
> > >>>
> > >>> /*
> > >>>  * Return a future, which when completed means that source has more
> > data
> > >>> and getNext() will not block.
> > >>>  * If you wish to use benefits of non blocking connectors, please
> > >>> implement this method appropriately.
> > >>>  */
> > >>> default CompletableFuture<?> isBlocked() {
> > >>>         return CompletableFuture.completedFuture(null);
> > >>> }
> > >>>
> > >>> And rename `getCurrent()` to `getNext()`.
> > >>>
> > >>> Couple of arguments:
> > >>> 1. I don’t understand the division of work between `advance()` and
> > >>> `getCurrent()`. What should be done in which, especially for
> connectors
> > >>> that handle records in batches (like Kafka) and when should you call
> > >>> `advance` and when `getCurrent()`.
> > >>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in
> the
> > >>> future to have asynchronous/non blocking connectors and more
> > efficiently
> > >>> handle large number of blocked threads, without busy waiting. While
> at
> > the
> > >>> same time it doesn’t add much complexity, since naive connector
> > >>> implementations can be always blocking.
> > >>> 3. This also would allow us to use a fixed size thread pool of task
> > >>> executors, instead of one thread per task.
> > >>>
> > >>> Piotrek
> > >>>
> > >>> > On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
> > >>> wrote:
> > >>> >
> > >>> > Hi All,
> > >>> >
> > >>> > In order to finally get the ball rolling on the new source
> interface
> > >>> that we have discussed for so long I finally created a FLIP:
> > >>>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> > >>> >
> > >>> > I cc'ed Thomas and Jamie because of the ongoing work/discussion
> about
> > >>> adding per-partition watermark support to the Kinesis source and
> > because
> > >>> this would enable generic implementation of event-time alignment for
> > all
> > >>> sources. Maybe we need another FLIP for the event-time alignment
> part,
> > >>> especially the part about information sharing between operations (I'm
> > not
> > >>> calling it state sharing because state has a special meaning in
> Flink).
> > >>> >
> > >>> > Please discuss away!
> > >>> >
> > >>> > Aljoscha
> > >>> >
> > >>> >
> > >>>
> > >>>
> >
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Becket Qin <be...@gmail.com>.
Hi Thomas,

The iterator-like API was also the first thing that came to me. But it
seems a little confusing that hasNext() does not mean "the stream has not
ended", but means "the next record is ready", which is repurposing the well
known meaning of hasNext(). If we follow the hasNext()/next() pattern, an
additional isNextReady() method to indicate whether the next record is
ready seems more intuitive to me.

Similarly, in poll()/take() pattern, another method of isDone() is needed
to indicate whether the stream has ended or not.

Compared with hasNext()/next()/isNextReady() pattern,
isDone()/poll()/take() seems more flexible for the reader implementation.
When I am implementing a reader, I could have a couple of choices:

   - A thread-less reader that does not have any internal thread.
   - When poll() is called, the same calling thread will perform a bunch of
      IO asynchronously.
      - When take() is called, the same calling thread will perform a bunch
      of IO and wait until the record is ready.
   - A reader with internal threads performing network IO and put records
   into a buffer.
      - When poll() is called, the calling thread simply reads from the
      buffer and return empty result immediately if there is no record.
      - When take() is called, the calling thread reads from the buffer and
      block waiting if the buffer is empty.

On the other hand, with the hasNext()/next()/isNextReady() API, it is less
intuitive for the reader developers to write the thread-less pattern.
Although technically speaking one can still do the asynchronous IO to
prepare the record in isNextReady(). But it is inexplicit and seems
somewhat hacky.

Thanks,

Jiangjie (Becket) Qin



On Mon, Nov 5, 2018 at 6:55 AM Thomas Weise <th...@apache.org> wrote:

> Couple more points regarding discovery:
>
> The proposal mentions that discovery could be outside the execution graph.
> Today, discovered partitions/shards are checkpointed. I believe that will
> also need to be the case in the future, even when discovery and reading are
> split between different tasks.
>
> For cases such as resharding of a Kinesis stream, the relationship between
> splits needs to be considered. Splits cannot be randomly distributed over
> readers in certain situations. An example was mentioned here:
> https://github.com/apache/flink/pull/6980#issuecomment-435202809
>
> Thomas
>
>
> On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:
>
> > Thanks for getting the ball rolling on this!
> >
> > Can the number of splits decrease? Yes, splits can be closed and go away.
> > An example would be a shard merge in Kinesis (2 existing shards will be
> > closed and replaced with a new shard).
> >
> > Regarding advance/poll/take: IMO the least restrictive approach would be
> > the thread-less IO model (pull based, non-blocking, caller retrieves new
> > records when available). The current Kinesis API requires the use of
> > threads. But that can be internal to the split reader and does not need
> to
> > be a source API concern. In fact, that's what we are working on right now
> > as improvement to the existing consumer: Each shard consumer thread will
> > push to a queue, the consumer main thread will poll the queue(s). It is
> > essentially a mapping from threaded IO to non-blocking.
> >
> > The proposed SplitReader interface would fit the thread-less IO model.
> > Similar to an iterator, we find out if there is a new element (hasNext)
> and
> > if so, move to it (next()). Separate calls deliver the meta information
> > (timestamp, watermark). Perhaps advance call could offer a timeout
> option,
> > so that the caller does not end up in a busy wait. On the other hand, a
> > caller processing multiple splits may want to cycle through fast, to
> > process elements of other splits as soon as they become available. The
> nice
> > thing is that this "split merge" logic can now live in Flink and be
> > optimized and shared between different sources.
> >
> > Thanks,
> > Thomas
> >
> >
> > On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com> wrote:
> >
> >> Hi,
> >> Thanks Aljoscha for this FLIP.
> >>
> >> 1. I agree with Piotr and Becket that the non-blocking source is very
> >> important. But in addition to `Future/poll`, there may be another way to
> >> achieve this. I think it may be not very memory friendly if every
> advance
> >> call return a Future.
> >>
> >> public interface Listener {
> >>      public void notify();
> >> }
> >>
> >> public interface SplitReader() {
> >>      /**
> >>       * When there is no element temporarily, this will return false.
> >>       * When elements is available again splitReader can call
> >> listener.notify()
> >>       * In addition the frame would check `advance` periodically .
> >>       * Of course advance can always return true and ignore the listener
> >> argument for simplicity.
> >>       */
> >>      public boolean advance(Listener listener);
> >> }
> >>
> >> 2.  The FLIP tells us very clearly that how to create all Splits and how
> >> to create a SplitReader from a Split. But there is no strategy for the
> user
> >> to choose how to assign the splits to the tasks. I think we could add a
> >> Enum to let user to choose.
> >> /**
> >>   public Enum SplitsAssignmentPolicy {
> >>     Location,
> >>     Workload,
> >>     Random,
> >>     Average
> >>   }
> >> */
> >>
> >> 3. If merge the `advance` and `getCurrent`  to one method like `getNext`
> >> the `getNext` would need return a `ElementWithTimestamp` because some
> >> sources want to add timestamp to every element. IMO, this is not so
> memory
> >> friendly so I prefer this design.
> >>
> >>
> >>
> >> Thanks
> >>
> >> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
> >>
> >>> Hi,
> >>>
> >>> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
> >>> possible improvements. I have one proposal. Instead of having a method:
> >>>
> >>> boolean advance() throws IOException;
> >>>
> >>> I would replace it with
> >>>
> >>> /*
> >>>  * Return a future, which when completed means that source has more
> data
> >>> and getNext() will not block.
> >>>  * If you wish to use benefits of non blocking connectors, please
> >>> implement this method appropriately.
> >>>  */
> >>> default CompletableFuture<?> isBlocked() {
> >>>         return CompletableFuture.completedFuture(null);
> >>> }
> >>>
> >>> And rename `getCurrent()` to `getNext()`.
> >>>
> >>> Couple of arguments:
> >>> 1. I don’t understand the division of work between `advance()` and
> >>> `getCurrent()`. What should be done in which, especially for connectors
> >>> that handle records in batches (like Kafka) and when should you call
> >>> `advance` and when `getCurrent()`.
> >>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in the
> >>> future to have asynchronous/non blocking connectors and more
> efficiently
> >>> handle large number of blocked threads, without busy waiting. While at
> the
> >>> same time it doesn’t add much complexity, since naive connector
> >>> implementations can be always blocking.
> >>> 3. This also would allow us to use a fixed size thread pool of task
> >>> executors, instead of one thread per task.
> >>>
> >>> Piotrek
> >>>
> >>> > On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
> >>> wrote:
> >>> >
> >>> > Hi All,
> >>> >
> >>> > In order to finally get the ball rolling on the new source interface
> >>> that we have discussed for so long I finally created a FLIP:
> >>>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >>> >
> >>> > I cc'ed Thomas and Jamie because of the ongoing work/discussion about
> >>> adding per-partition watermark support to the Kinesis source and
> because
> >>> this would enable generic implementation of event-time alignment for
> all
> >>> sources. Maybe we need another FLIP for the event-time alignment part,
> >>> especially the part about information sharing between operations (I'm
> not
> >>> calling it state sharing because state has a special meaning in Flink).
> >>> >
> >>> > Please discuss away!
> >>> >
> >>> > Aljoscha
> >>> >
> >>> >
> >>>
> >>>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Thomas Weise <th...@apache.org>.
Couple more points regarding discovery:

The proposal mentions that discovery could be outside the execution graph.
Today, discovered partitions/shards are checkpointed. I believe that will
also need to be the case in the future, even when discovery and reading are
split between different tasks.

For cases such as resharding of a Kinesis stream, the relationship between
splits needs to be considered. Splits cannot be randomly distributed over
readers in certain situations. An example was mentioned here:
https://github.com/apache/flink/pull/6980#issuecomment-435202809

Thomas


On Sun, Nov 4, 2018 at 1:43 PM Thomas Weise <th...@apache.org> wrote:

> Thanks for getting the ball rolling on this!
>
> Can the number of splits decrease? Yes, splits can be closed and go away.
> An example would be a shard merge in Kinesis (2 existing shards will be
> closed and replaced with a new shard).
>
> Regarding advance/poll/take: IMO the least restrictive approach would be
> the thread-less IO model (pull based, non-blocking, caller retrieves new
> records when available). The current Kinesis API requires the use of
> threads. But that can be internal to the split reader and does not need to
> be a source API concern. In fact, that's what we are working on right now
> as improvement to the existing consumer: Each shard consumer thread will
> push to a queue, the consumer main thread will poll the queue(s). It is
> essentially a mapping from threaded IO to non-blocking.
>
> The proposed SplitReader interface would fit the thread-less IO model.
> Similar to an iterator, we find out if there is a new element (hasNext) and
> if so, move to it (next()). Separate calls deliver the meta information
> (timestamp, watermark). Perhaps advance call could offer a timeout option,
> so that the caller does not end up in a busy wait. On the other hand, a
> caller processing multiple splits may want to cycle through fast, to
> process elements of other splits as soon as they become available. The nice
> thing is that this "split merge" logic can now live in Flink and be
> optimized and shared between different sources.
>
> Thanks,
> Thomas
>
>
> On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com> wrote:
>
>> Hi,
>> Thanks Aljoscha for this FLIP.
>>
>> 1. I agree with Piotr and Becket that the non-blocking source is very
>> important. But in addition to `Future/poll`, there may be another way to
>> achieve this. I think it may be not very memory friendly if every advance
>> call return a Future.
>>
>> public interface Listener {
>>      public void notify();
>> }
>>
>> public interface SplitReader() {
>>      /**
>>       * When there is no element temporarily, this will return false.
>>       * When elements is available again splitReader can call
>> listener.notify()
>>       * In addition the frame would check `advance` periodically .
>>       * Of course advance can always return true and ignore the listener
>> argument for simplicity.
>>       */
>>      public boolean advance(Listener listener);
>> }
>>
>> 2.  The FLIP tells us very clearly that how to create all Splits and how
>> to create a SplitReader from a Split. But there is no strategy for the user
>> to choose how to assign the splits to the tasks. I think we could add a
>> Enum to let user to choose.
>> /**
>>   public Enum SplitsAssignmentPolicy {
>>     Location,
>>     Workload,
>>     Random,
>>     Average
>>   }
>> */
>>
>> 3. If merge the `advance` and `getCurrent`  to one method like `getNext`
>> the `getNext` would need return a `ElementWithTimestamp` because some
>> sources want to add timestamp to every element. IMO, this is not so memory
>> friendly so I prefer this design.
>>
>>
>>
>> Thanks
>>
>> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
>>
>>> Hi,
>>>
>>> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
>>> possible improvements. I have one proposal. Instead of having a method:
>>>
>>> boolean advance() throws IOException;
>>>
>>> I would replace it with
>>>
>>> /*
>>>  * Return a future, which when completed means that source has more data
>>> and getNext() will not block.
>>>  * If you wish to use benefits of non blocking connectors, please
>>> implement this method appropriately.
>>>  */
>>> default CompletableFuture<?> isBlocked() {
>>>         return CompletableFuture.completedFuture(null);
>>> }
>>>
>>> And rename `getCurrent()` to `getNext()`.
>>>
>>> Couple of arguments:
>>> 1. I don’t understand the division of work between `advance()` and
>>> `getCurrent()`. What should be done in which, especially for connectors
>>> that handle records in batches (like Kafka) and when should you call
>>> `advance` and when `getCurrent()`.
>>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in the
>>> future to have asynchronous/non blocking connectors and more efficiently
>>> handle large number of blocked threads, without busy waiting. While at the
>>> same time it doesn’t add much complexity, since naive connector
>>> implementations can be always blocking.
>>> 3. This also would allow us to use a fixed size thread pool of task
>>> executors, instead of one thread per task.
>>>
>>> Piotrek
>>>
>>> > On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org>
>>> wrote:
>>> >
>>> > Hi All,
>>> >
>>> > In order to finally get the ball rolling on the new source interface
>>> that we have discussed for so long I finally created a FLIP:
>>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>>> >
>>> > I cc'ed Thomas and Jamie because of the ongoing work/discussion about
>>> adding per-partition watermark support to the Kinesis source and because
>>> this would enable generic implementation of event-time alignment for all
>>> sources. Maybe we need another FLIP for the event-time alignment part,
>>> especially the part about information sharing between operations (I'm not
>>> calling it state sharing because state has a special meaning in Flink).
>>> >
>>> > Please discuss away!
>>> >
>>> > Aljoscha
>>> >
>>> >
>>>
>>>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Thomas Weise <th...@apache.org>.
Thanks for getting the ball rolling on this!

Can the number of splits decrease? Yes, splits can be closed and go away.
An example would be a shard merge in Kinesis (2 existing shards will be
closed and replaced with a new shard).

Regarding advance/poll/take: IMO the least restrictive approach would be
the thread-less IO model (pull based, non-blocking, caller retrieves new
records when available). The current Kinesis API requires the use of
threads. But that can be internal to the split reader and does not need to
be a source API concern. In fact, that's what we are working on right now
as improvement to the existing consumer: Each shard consumer thread will
push to a queue, the consumer main thread will poll the queue(s). It is
essentially a mapping from threaded IO to non-blocking.

The proposed SplitReader interface would fit the thread-less IO model.
Similar to an iterator, we find out if there is a new element (hasNext) and
if so, move to it (next()). Separate calls deliver the meta information
(timestamp, watermark). Perhaps advance call could offer a timeout option,
so that the caller does not end up in a busy wait. On the other hand, a
caller processing multiple splits may want to cycle through fast, to
process elements of other splits as soon as they become available. The nice
thing is that this "split merge" logic can now live in Flink and be
optimized and shared between different sources.

Thanks,
Thomas


On Sun, Nov 4, 2018 at 6:34 AM Guowei Ma <gu...@gmail.com> wrote:

> Hi,
> Thanks Aljoscha for this FLIP.
>
> 1. I agree with Piotr and Becket that the non-blocking source is very
> important. But in addition to `Future/poll`, there may be another way to
> achieve this. I think it may be not very memory friendly if every advance
> call return a Future.
>
> public interface Listener {
>      public void notify();
> }
>
> public interface SplitReader() {
>      /**
>       * When there is no element temporarily, this will return false.
>       * When elements is available again splitReader can call
> listener.notify()
>       * In addition the frame would check `advance` periodically .
>       * Of course advance can always return true and ignore the listener
> argument for simplicity.
>       */
>      public boolean advance(Listener listener);
> }
>
> 2.  The FLIP tells us very clearly that how to create all Splits and how
> to create a SplitReader from a Split. But there is no strategy for the user
> to choose how to assign the splits to the tasks. I think we could add a
> Enum to let user to choose.
> /**
>   public Enum SplitsAssignmentPolicy {
>     Location,
>     Workload,
>     Random,
>     Average
>   }
> */
>
> 3. If merge the `advance` and `getCurrent`  to one method like `getNext`
> the `getNext` would need return a `ElementWithTimestamp` because some
> sources want to add timestamp to every element. IMO, this is not so memory
> friendly so I prefer this design.
>
>
>
> Thanks
>
> Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:
>
>> Hi,
>>
>> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
>> possible improvements. I have one proposal. Instead of having a method:
>>
>> boolean advance() throws IOException;
>>
>> I would replace it with
>>
>> /*
>>  * Return a future, which when completed means that source has more data
>> and getNext() will not block.
>>  * If you wish to use benefits of non blocking connectors, please
>> implement this method appropriately.
>>  */
>> default CompletableFuture<?> isBlocked() {
>>         return CompletableFuture.completedFuture(null);
>> }
>>
>> And rename `getCurrent()` to `getNext()`.
>>
>> Couple of arguments:
>> 1. I don’t understand the division of work between `advance()` and
>> `getCurrent()`. What should be done in which, especially for connectors
>> that handle records in batches (like Kafka) and when should you call
>> `advance` and when `getCurrent()`.
>> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in the
>> future to have asynchronous/non blocking connectors and more efficiently
>> handle large number of blocked threads, without busy waiting. While at the
>> same time it doesn’t add much complexity, since naive connector
>> implementations can be always blocking.
>> 3. This also would allow us to use a fixed size thread pool of task
>> executors, instead of one thread per task.
>>
>> Piotrek
>>
>> > On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org> wrote:
>> >
>> > Hi All,
>> >
>> > In order to finally get the ball rolling on the new source interface
>> that we have discussed for so long I finally created a FLIP:
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>> >
>> > I cc'ed Thomas and Jamie because of the ongoing work/discussion about
>> adding per-partition watermark support to the Kinesis source and because
>> this would enable generic implementation of event-time alignment for all
>> sources. Maybe we need another FLIP for the event-time alignment part,
>> especially the part about information sharing between operations (I'm not
>> calling it state sharing because state has a special meaning in Flink).
>> >
>> > Please discuss away!
>> >
>> > Aljoscha
>> >
>> >
>>
>>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Guowei Ma <gu...@gmail.com>.
Hi,
Thanks Aljoscha for this FLIP.

1. I agree with Piotr and Becket that the non-blocking source is very
important. But in addition to `Future/poll`, there may be another way to
achieve this. I think it may be not very memory friendly if every advance
call return a Future.

public interface Listener {
     public void notify();
}

public interface SplitReader() {
     /**
      * When there is no element temporarily, this will return false.
      * When elements is available again splitReader can call
listener.notify()
      * In addition the frame would check `advance` periodically .
      * Of course advance can always return true and ignore the listener
argument for simplicity.
      */
     public boolean advance(Listener listener);
}

2.  The FLIP tells us very clearly that how to create all Splits and how to
create a SplitReader from a Split. But there is no strategy for the user to
choose how to assign the splits to the tasks. I think we could add a Enum
to let user to choose.
/**
  public Enum SplitsAssignmentPolicy {
    Location,
    Workload,
    Random,
    Average
  }
*/

3. If merge the `advance` and `getCurrent`  to one method like `getNext`
the `getNext` would need return a `ElementWithTimestamp` because some
sources want to add timestamp to every element. IMO, this is not so memory
friendly so I prefer this design.



Thanks

Piotr Nowojski <pi...@data-artisans.com> 于2018年11月1日周四 下午6:08写道:

> Hi,
>
> Thanks Aljoscha for starting this, it’s blocking quite a lot of other
> possible improvements. I have one proposal. Instead of having a method:
>
> boolean advance() throws IOException;
>
> I would replace it with
>
> /*
>  * Return a future, which when completed means that source has more data
> and getNext() will not block.
>  * If you wish to use benefits of non blocking connectors, please
> implement this method appropriately.
>  */
> default CompletableFuture<?> isBlocked() {
>         return CompletableFuture.completedFuture(null);
> }
>
> And rename `getCurrent()` to `getNext()`.
>
> Couple of arguments:
> 1. I don’t understand the division of work between `advance()` and
> `getCurrent()`. What should be done in which, especially for connectors
> that handle records in batches (like Kafka) and when should you call
> `advance` and when `getCurrent()`.
> 2. Replacing `boolean` with `CompletableFuture<?>` will allow us in the
> future to have asynchronous/non blocking connectors and more efficiently
> handle large number of blocked threads, without busy waiting. While at the
> same time it doesn’t add much complexity, since naive connector
> implementations can be always blocking.
> 3. This also would allow us to use a fixed size thread pool of task
> executors, instead of one thread per task.
>
> Piotrek
>
> > On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org> wrote:
> >
> > Hi All,
> >
> > In order to finally get the ball rolling on the new source interface
> that we have discussed for so long I finally created a FLIP:
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> >
> > I cc'ed Thomas and Jamie because of the ongoing work/discussion about
> adding per-partition watermark support to the Kinesis source and because
> this would enable generic implementation of event-time alignment for all
> sources. Maybe we need another FLIP for the event-time alignment part,
> especially the part about information sharing between operations (I'm not
> calling it state sharing because state has a special meaning in Flink).
> >
> > Please discuss away!
> >
> > Aljoscha
> >
> >
>
>

Re: [DISCUSS] FLIP-27: Refactor Source Interface

Posted by Piotr Nowojski <pi...@data-artisans.com>.
Hi,

Thanks Aljoscha for starting this, it’s blocking quite a lot of other possible improvements. I have one proposal. Instead of having a method:

boolean advance() throws IOException;

I would replace it with

/* 
 * Return a future, which when completed means that source has more data and getNext() will not block.
 * If you wish to use benefits of non blocking connectors, please implement this method appropriately.
 */
default CompletableFuture<?> isBlocked() {
	return CompletableFuture.completedFuture(null);
}

And rename `getCurrent()` to `getNext()`.

Couple of arguments:
1. I don’t understand the division of work between `advance()` and `getCurrent()`. What should be done in which, especially for connectors that handle records in batches (like Kafka) and when should you call `advance` and when `getCurrent()`. 
2. Replacing `boolean` with `CompletableFuture<?>` will allow us in the future to have asynchronous/non blocking connectors and more efficiently handle large number of blocked threads, without busy waiting. While at the same time it doesn’t add much complexity, since naive connector implementations can be always blocking.
3. This also would allow us to use a fixed size thread pool of task executors, instead of one thread per task.

Piotrek

> On 31 Oct 2018, at 17:22, Aljoscha Krettek <al...@apache.org> wrote:
> 
> Hi All,
> 
> In order to finally get the ball rolling on the new source interface that we have discussed for so long I finally created a FLIP: https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
> 
> I cc'ed Thomas and Jamie because of the ongoing work/discussion about adding per-partition watermark support to the Kinesis source and because this would enable generic implementation of event-time alignment for all sources. Maybe we need another FLIP for the event-time alignment part, especially the part about information sharing between operations (I'm not calling it state sharing because state has a special meaning in Flink).
> 
> Please discuss away!
> 
> Aljoscha
> 
>