Re: [DISCUSS] FLIP-137: Support Pandas UDAF in PyFlink

2020-09-02 Thread jincheng sun
Thanks for the update Xingbo! Pandas UDAF can reuse the `class aggregate function (user defined function)` interface in FLIP-139, and the core logic of Pandas UDAF users is written in the `accumulate` method. In this way, we can unify the interface semantics of all UDAF. What do you think?

[jira] [Created] (FLINK-19130) Expose backpressure metrics / logs for function dispatcher operator

2020-09-02 Thread Tzu-Li (Gordon) Tai (Jira)
Tzu-Li (Gordon) Tai created FLINK-19130: --- Summary: Expose backpressure metrics / logs for function dispatcher operator Key: FLINK-19130 URL: https://issues.apache.org/jira/browse/FLINK-19130

Re: FileSystemHaServices and BlobStore

2020-09-02 Thread Yang Wang
Hi Alexey, Thanks for the feedback. You are right. StatefulSet + PersistentVolume + FileSystemHaService could be another bundle of services for Flink HA support on K8s. The user jars could be built into the image or downloaded by init-container or mount via the PV. So they do not need to be

RE: [DISCUSS] Releasing Flink 1.11.2

2020-09-02 Thread Zhou, Brian
Hi, Thanks Becket for addressing the issue. FLINK-18641 is now a blocker for Pravega connector integration, hope we can have it included in 1.11.2 release. Best Regards, Brian -Original Message- From: Becket Qin Sent: Thursday, September 3, 2020 11:18 To: dev Cc:

Re: [DISCUSS] Releasing Flink 1.11.2

2020-09-02 Thread Becket Qin
Hi Zhuzhu, Thanks for starting the discussion. I'd like to include FLINK-18641 into 1.11.2 as well. It is a regression from previous versions and is currently blocking the development of Pravega source on top of FLIP-27. Thanks, Jiangjie (Becket) Qin On Wed, Sep 2, 2020 at 11:13 PM Zhu Zhu

Re: [DISCUSS] Releasing Flink 1.11.2

2020-09-02 Thread Zhijiang
Thanks for launching this discussion and volunteering as the release manager. +1 on my side and I am willing to provide any help during the release procedure, :) Best, Zhijiang -- From:Konstantin Knauf Send Time:2020年9月2日(星期三)

Re: [DISCUSS] Introduce partitioning strategies to Table/SQL

2020-09-02 Thread Benchao Li
Hi Jingsong, Thanks for the clarification, and sorry to misunderstand your first intention. What I was talking about is indeed another topic, we can leave it to the future, and see if there are any other people who have the same scenarios. Jingsong Li 于2020年9月3日周四 上午10:56写道: > Thanks Timo for

Re: [DISCUSS] Introduce partitioning strategies to Table/SQL

2020-09-02 Thread Jingsong Li
Thanks Timo for working on FLIP-107. Agree, I think it is good. I'll spend more time to form a FLIP in detail later. Best, Jingsong On Wed, Sep 2, 2020 at 7:12 PM Timo Walther wrote: > Hi Jingsong, > > I haven't looked at your proposal but I think it make sense to have a > separate FLIP for

Re: FileSystemHaServices and BlobStore

2020-09-02 Thread Alexey Trenikhun
Judging from FLIP-19 (thank you Roman for the link), of 3 use cases (jars, RPC messages and log files) only jar files need HA guarantee, and in my particular case, job cluster with jar as part of image, it seems doesn't matter, I guess it explains why in my test I was able to recover from

[jira] [Created] (FLINK-19129) Helm charts are missing the latest log4j-console file

2020-09-02 Thread Igal Shilman (Jira)
Igal Shilman created FLINK-19129: Summary: Helm charts are missing the latest log4j-console file Key: FLINK-19129 URL: https://issues.apache.org/jira/browse/FLINK-19129 Project: Flink Issue

[jira] [Created] (FLINK-19128) Remove the runtime execution configuration in sql-client-defaults.yaml

2020-09-02 Thread Jark Wu (Jira)
Jark Wu created FLINK-19128: --- Summary: Remove the runtime execution configuration in sql-client-defaults.yaml Key: FLINK-19128 URL: https://issues.apache.org/jira/browse/FLINK-19128 Project: Flink

[jira] [Created] (FLINK-19127) Provide a replacement of StreamExecutionEnvironment.createRemoteEnvironment for TableEnvironment

2020-09-02 Thread Timo Walther (Jira)
Timo Walther created FLINK-19127: Summary: Provide a replacement of StreamExecutionEnvironment.createRemoteEnvironment for TableEnvironment Key: FLINK-19127 URL: https://issues.apache.org/jira/browse/FLINK-19127

Re: [DISCUSS] Releasing Flink 1.11.2

2020-09-02 Thread Zhu Zhu
Thank you all for the inputs! I agree with Till that we should set a soft deadline first. I'd like to propose next Monday if no new blocker issue pops up. But feel free to raise your concerns if you feel next Monday as a deadline may not work for fixes which should be a blocker for 1.11.2.

[jira] [Created] (FLINK-19126) Failed to run job in yarn-cluster mode due to No Executor found.

2020-09-02 Thread Tang Yan (Jira)
Tang Yan created FLINK-19126: Summary: Failed to run job in yarn-cluster mode due to No Executor found. Key: FLINK-19126 URL: https://issues.apache.org/jira/browse/FLINK-19126 Project: Flink

[jira] [Created] (FLINK-19125) Avoid memory fragmentation when running flink docker image

2020-09-02 Thread Yun Tang (Jira)
Yun Tang created FLINK-19125: Summary: Avoid memory fragmentation when running flink docker image Key: FLINK-19125 URL: https://issues.apache.org/jira/browse/FLINK-19125 Project: Flink Issue

Re: [DISCUSS] Releasing Flink 1.11.2

2020-09-02 Thread Konstantin Knauf
I think it would be nice to include a fix for https://issues.apache.org/jira/browse/FLINK-18934, too, as it affects a highly requested feature of Flink 1.11 quite severely. On Wed, Sep 2, 2020 at 2:51 PM Till Rohrmann wrote: > Thanks a lot for starting this discussion Zhu Zhu and for

Re: [DISCUSS] Releasing Flink 1.11.2

2020-09-02 Thread Till Rohrmann
Thanks a lot for starting this discussion Zhu Zhu and for volunteering as the release manager. Big +1 for creating the next 1.11 bug fix release. I think we already collected quite a bit of fixes which our users will benefit from. For the pending fixes, I would suggest setting a soft deadline

Re: [DISCUSS] Releasing Flink 1.11.2

2020-09-02 Thread David Anderson
I think it's worth considering whether we can get this bugfix included in 1.11.2: - FLINK-19109 Split Reader eats chained periodic watermarks There is a PR, but it's still a work in progress. Cc'ing Roman, who has been working on this. Regards, David On Wed, Sep 2, 2020 at 2:19 PM Zhu Zhu

[DISCUSS] Releasing Flink 1.11.2

2020-09-02 Thread Zhu Zhu
Hi All, It has been about 1 month since we released Flink 1.11.1. It's not too far but we already have more than 80 resolved improvements/bugs in the release-1.11 branch. Some of them are quite critical. Therefore, I propose to create the next bugfix release 1.11.2 for Flink 1.11. Most

[VOTE] FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet API)

2020-09-02 Thread Aljoscha Krettek
Hi all, After the discussion in [1], I would like to open a voting thread for FLIP-131 (https://s.apache.org/FLIP-131) [2] which discusses the deprecation of the DataSet API and future work on the DataStream API and Table API for bounded (batch) execution. The vote will be open until

Re: [DISCUSS] Introduce partitioning strategies to Table/SQL

2020-09-02 Thread Timo Walther
Hi Jingsong, I haven't looked at your proposal but I think it make sense to have a separate FLIP for the parititioning topic. I'm currently working on an update to FLIP-107 and would suggest to remove the paritioning topic there. FLIP-107 will only focus on accessing metadata and expressing

Re: [DISCUSS] FLIP-136: Improve interoperability between DataStream and Table API

2020-09-02 Thread Dawid Wysakowicz
Hi all, A comment from my side on the topic of the current, weird renaming/naming/reordering when registering a DataStream. It might be just me, but I find it extremely confusing and I would be really, really happy if we could simplify it. I really don't like that the actual behaviour of this

Re: [DISCUSS] FLIP-136: Improve interoperability between DataStream and Table API

2020-09-02 Thread Danny Chan
Thanks Timo ~ “No this is not possible, because T records have no changeflag. Without a changeflag, a ChangelogMode makes not much sense. “ I agree, but just distinguish the different ChangelogMode with a renamed API still does not resolve the problem either, an API change compared to an

Re: [DISCUSS] FLIP-141: Intra-Slot Managed Memory Sharing

2020-09-02 Thread Xintong Song
Thanks for the input, Yu. I believe the current proposal should work with RocksDB, or any other state backend, using memory at either the slot or the scope. With the proposed approach, all we need is an indicator (e.g., a configuration option) telling us which scope should we calculate the

[jira] [Created] (FLINK-19123) TestStreamEnvironment does not use shared MiniCluster for executeAsync()

2020-09-02 Thread Aljoscha Krettek (Jira)
Aljoscha Krettek created FLINK-19123: Summary: TestStreamEnvironment does not use shared MiniCluster for executeAsync() Key: FLINK-19123 URL: https://issues.apache.org/jira/browse/FLINK-19123

Re: [DISCUSS] FLIP-136: Improve interoperability between DataStream and Table API

2020-09-02 Thread Jark Wu
Hi Timo, 1. "fromDataStream VS fromInsertStream" In terms of naming, personally, I prefer `fromDataStream`, `fromChangelogStream`, `toDataStream`, `toChangelogStream` than `fromInsertStream`, `toInsertStream`. 2. "fromDataStream(DataStream, Expression...) VS

Re: [DISCUSS] FLIP-141: Intra-Slot Managed Memory Sharing

2020-09-02 Thread Yu Li
Thanks for compiling the FLIP Xintong, and +1 for the updated doc. Just one supplement for the RocksDB state backend part: It's true that currently we're using managed memory at the slot scope. However, IMHO, we may support setting weights for different stateful operators (for advanced usage) in

[jira] [Created] (FLINK-19122) Prometheus scrape generates huge scrape target.

2020-09-02 Thread Harold Dost III (Jira)
Harold Dost III created FLINK-19122: --- Summary: Prometheus scrape generates huge scrape target. Key: FLINK-19122 URL: https://issues.apache.org/jira/browse/FLINK-19122 Project: Flink Issue

Re: [DISCUSS] FLIP-136: Improve interoperability between DataStream and Table API

2020-09-02 Thread Timo Walther
Hi everyone thanks for your feedback. It's a lot of content that needs to be digested. I will update the FLIP shortly to incorporate some of the feedback already. But let me respond to some topics first: "not deprecate these API", "the API of the table layer is changing too fast" I agree

Re: [DISCUSS] FLIP-141: Intra-Slot Managed Memory Sharing

2020-09-02 Thread Xintong Song
Thanks all for the feedback and discussion. I have updated the FLIP, with the following changes. - Choose the main proposal over the alternative approach - Combine weights of RocksDB and batch operators - Expose weights through configuration options, rather than via ExecutionConfig.

Re: [DISCUSS] FLIP-141: Intra-Slot Managed Memory Sharing

2020-09-02 Thread Xintong Song
Thanks for the inputs, Aljoscha & Till. # Weight Configuration I think exposing the knobs incrementally is a good idea. However, I'm not sure about non-configurable as the first step. Currently, users can tune memory for rocksdb ('taskmanager.memory.managed.size') and python