Re: [DISCUSS] Align the semantic of returning null from InputFormat#nextRecord

2020-07-29 Thread Benchao Li
Hi Jingsong, Thanks for the input. Our current implementation is very like the way you posted. Btw, we are doing the unification of streaming/batch work using our Flink SQL engine, hope to contribute more in this realm. Jingsong Li 于2020年7月30日周四 上午11:49写道: > Hi Benchao, > > I'm very glad that

Re: [DISCUSS] Align the semantic of returning null from InputFormat#nextRecord

2020-07-29 Thread Jingsong Li
Hi Benchao, I'm very glad that you are developing the ecology of batch~ More detail for #1 is in `CsvInputFormat`: - There are dirty records (Comments) in `CsvInputFormat.readRecord`, so unfortunately, it must return null. - So go on like this, will return null in `nextRecord` like #2, so `CsvInp

Re: [DISCUSS] Align the semantic of returning null from InputFormat#nextRecord

2020-07-29 Thread Benchao Li
Thanks Jingsong for the response. If we plan to deprecate InputFormat in the near future, I think it's ok to keep current behavior as it is for now. I raised this discussion because I found that InputFormatSourceFunction is not working as expected while I'm developing a batch table source interna

Re: [DISCUSS] Align the semantic of returning null from InputFormat#nextRecord

2020-07-29 Thread Jingsong Li
Hi Benchao, My understanding is that #1 treats null as the end of input. This means we should try our best to avoid returning null before the end of input in the implementation of `InputFormat`. Even if we have to return null, we have to return it at the end of input. I think this is the most safe

[DISCUSS] Align the semantic of returning null from InputFormat#nextRecord

2020-07-23 Thread Benchao Li
Hi all, I'd like to discuss about the semantic of returning null from InputFormat#nextRecord. For now, there is no explicit java doc about this. And there are three ways to handle this in Flink: 1. treat null as the end of input 2. skip null 3. assumes that the value from InputFormat#nextRecord c