Jark Wu created FLINK-19824:
---
Summary: Refactor and merge SupportsComputedColumnPushDown and
SupportsWatermarkPushDown interfaces
Key: FLINK-19824
URL: https://issues.apache.org/jira/browse/FLINK-19824
Hi, Timo.
I agree with you that the concepts Watermark and ComputedColumn should be
separated. However, we are merging the interface SupportsCalcPushDown and
SupportsWatermarkPushDown actually. The concept computed column has
disappeared in optimization.
As for the drawback you mentiond, I have alr
Hi Jark, Hi Shengkai,
"shall we push the expressions in the following Projection too?"
This is something that we should at least consider.
I also don't find a strong use case. But what I see is that we are
merging concepts that actually can be separated. And we are executing
the same code twi
Hi Timo and Jark.Thanks for your replies.
Maybe I don't explain clearly in doc. I think the main reason behind is we
have no means to distinguish the calc in LogicalProject. Let me give you an
example to illustrate the reason. Assume we have 2 cases:
case 1:
create table MyTable (
int a,
int
Hi Timo,
Regarding "pushing other computed columns into source, e.g. encrypted
records/columns, performing checksum checks, reading metadata etc.",
I'm not sure about this.
1. the planner don't know which computed column should be pushed into source
2. it seems that we can't improve performances i
Hi Shengkai,
first of I would not consider the section Problems of
SupportsWatermarkPushDown" as a "problem". It was planned to update the
WatermarkProvider once the interfaces are ready. See the comment in
WatermarkProvider:
// marker interface that will be filled after FLIP-126:
// Waterma
Thanks to Shengkai for summarizing the problems on the FLIP-95 interfaces
and solutions.
I think the new proposal, i.e. only pushing the "WatermarkStrategy" is much
cleaner and easier to develop than before.
So I'm +1 to the proposal.
Best,
Jark
On Sat, 5 Sep 2020 at 13:44, Shengkai Fang wrote:
Hi, all. It seems the format is not normal. So I add a google doc in
link[1]. This discussion is about the interfaces in FLIP-95: New Table
Source And Table Sink and propose to merge two interfaces
SupportsWatermarkPushDown and SupportsComputedColumnPushDown.
I am looking forward to any opinions
Hi, all. Currently, we use two seperated interfaces
SupportsComputedColumnPushDown and SupportsWatermarkPushDown in design. The
interface SupportsWatermarkPushDown relies on
SupportsComputedColumnPushDown when
watermark is defined on a computed column. During the implementation, we
find the meth