Re: [DISCUSS] FLIP-223: Support HiveServer2 Endpoint

2022-06-01 Thread Jingsong Li
Hi Shengkai, Thanks for driving. I have a few comments: ## Unsupported features I've read the FLIP and it's not quite clear what the specific unsupported items are? - For example, security related, is it not supported. - For example, is there a loss of precision for types - For example, the Fet

Re: [DISCUSS] FLIP-234: Support Retryable Lookup Join To Solve Delayed Updates Issue In External Systems

2022-06-01 Thread Jingsong Li
Hi Lincoln, > It's better making decisions at the query level when a connector has both capabilities. Can you clarify the mechanism? - only sync connector: What connector developers should do - only async connector: What connector developers should do - both async and sync connector: What connect

Re: [DISCUSS] FLIP-221 Abstraction for lookup source cache and metric

2022-06-01 Thread Jingsong Li
Thanks Alexander for your reply. We can discuss the new interface when it comes out. We are more inclined to deprecate the connector `async` option when discussing FLIP-234 [1]. We should use hint to let planner decide. Although the discussion has not yet produced a conclusion, can we remove this

[jira] [Created] (FLINK-27863) Refactor RecordReader and RecordIterator in table store into generic types

2022-06-01 Thread Caizhi Weng (Jira)
Caizhi Weng created FLINK-27863: --- Summary: Refactor RecordReader and RecordIterator in table store into generic types Key: FLINK-27863 URL: https://issues.apache.org/jira/browse/FLINK-27863 Project: Fli

Re: About Native Deployment's Autoscaling implementation

2022-06-01 Thread Yang Wang
Hi Talat, Using sub resources for the auto scaling makes a lot of sense to me. Could you be more specific why you think changing task manager count will > not work for native deployment ? The native K8s integration is using active resourcemanager. It means that the TaskManager count will be cal

Re: [DISCUSS] FLIP-234: Support Retryable Lookup Join To Solve Delayed Updates Issue In External Systems

2022-06-01 Thread Lincoln Lee
Hi Jingsong, There will be no change for connectors with only one capability (sync or async). Query hint works in a best effort manner, so if users specifies a hint with invalid option, the query plan keeps unchanged, e.g., use LOOKUP('table'='customer', 'async'='true'), but backend lookup source

Re: [VOTE] Apache Flink Kubernetes Operator Release 1.0.0, release candidate #3

2022-06-01 Thread Márton Balassi
Thanks, Yang. Looks good after your change. +1 for the 48 hour voting time. On Wed, Jun 1, 2022 at 7:59 AM Gyula Fóra wrote: > I agree to create a new RC based on this find. > And 48h voting time seems reasonable as all functional testing will carry > over to this new candidate. > > Gyula > > O

[jira] [Created] (FLINK-27864) Duplicated classes (jar hell) on flink-table-runtime

2022-06-01 Thread Guillaume Lederrey (Jira)
Guillaume Lederrey created FLINK-27864: -- Summary: Duplicated classes (jar hell) on flink-table-runtime Key: FLINK-27864 URL: https://issues.apache.org/jira/browse/FLINK-27864 Project: Flink

[jira] [Created] (FLINK-27865) Add guide and example for configuring SASL and SSL in Kafka SQL connector document

2022-06-01 Thread Qingsheng Ren (Jira)
Qingsheng Ren created FLINK-27865: - Summary: Add guide and example for configuring SASL and SSL in Kafka SQL connector document Key: FLINK-27865 URL: https://issues.apache.org/jira/browse/FLINK-27865

Re: [DISCUSS] FLIP-221 Abstraction for lookup source cache and metric

2022-06-01 Thread Qingsheng Ren
Hi devs, Thanks for the in-depth discussion! We recently update some designs of FLIP-221 [1]: 1. Introduce a new interface “FullCachingReloadTrigger” for developer to customize the reload strategy. The previous design was only time-based and not flexable enough. Developers can implement any lo

Re: [DISCUSS] FLIP-221 Abstraction for lookup source cache and metric

2022-06-01 Thread Qingsheng Ren
Hi Alexander, Thanks for the update! I made some updates on the FLIP and here’s some diffs compared to your version: 1. ReloadStrategy has been covered by the new interface FullCachingReloadTrigger. This trigger will be taken over and run by the LookupJoinRunner and provides enough flexibilit

Re: [DISCUSS] FLIP-221 Abstraction for lookup source cache and metric

2022-06-01 Thread Qingsheng Ren
Hi Becket, Thanks for your comments! 1. We have removed the LookupCacheFactory in the latest design and we added open/close method to the LookupCache for initialization. 2. Custom reload strategy is a great idea! We added a new interface FullCachingReloadTrigger for developers to implement t

[VOTE] Apache Flink Kubernetes Operator Release 1.0.0, release candidate #4

2022-06-01 Thread Yang Wang
Hi everyone, Please review and vote on the release candidate #4 for the version 1.0.0 of Apache Flink Kubernetes Operator, as follows: [ ] +1, Approve the release [ ] -1, Do not approve the release (please provide specific comments) **Release Overview** As an overview, the release consists of th

Re: [DISCUSS] FLIP-221 Abstraction for lookup source cache and metric

2022-06-01 Thread Qingsheng Ren
Hi Jingsong, Thanks for your comments! > AllCache definition is not flexible, for example, PartialCache can use any > custom storage, while the AllCache can not, AllCache can also be considered > to store memory or disk, also need a flexible strategy. We had an offline discussion with Jark an

[jira] [Created] (FLINK-27866) CassandraOutputFormat and CassandraSink will timeout twice if the number of maximum requests is reached

2022-06-01 Thread Etienne Chauchot (Jira)
Etienne Chauchot created FLINK-27866: Summary: CassandraOutputFormat and CassandraSink will timeout twice if the number of maximum requests is reached Key: FLINK-27866 URL: https://issues.apache.org/jira/brows

Translation in korean?

2022-06-01 Thread Dennis Jung
Hello, have a question about document translation. Are there some korean translating tasks going on for flink documents? https://issues.apache.org/jira/browse/FLINK-27601 I could see tasks for Chinese translation, but not for korean. Regards.

[jira] [Created] (FLINK-27867) SplitAggregateITCase fails on CI

2022-06-01 Thread Alexander Preuss (Jira)
Alexander Preuss created FLINK-27867: Summary: SplitAggregateITCase fails on CI Key: FLINK-27867 URL: https://issues.apache.org/jira/browse/FLINK-27867 Project: Flink Issue Type: Bug

Re: [VOTE] Apache Flink Kubernetes Operator Release 1.0.0, release candidate #4

2022-06-01 Thread Gyula Fóra
Hi Yang! +1 (binding) Thank you for fixing the NOTICE file issue Successfully verified the following: - Signatures, Hashes - No binaries in source release - Helm Repo works, Helm install works, docker image matches release commit tag - Build from source, build container from source - Submit exam

[jira] [Created] (FLINK-27868) Harden running job check before triggering savepoints or savepoint upgrades

2022-06-01 Thread Gyula Fora (Jira)
Gyula Fora created FLINK-27868: -- Summary: Harden running job check before triggering savepoints or savepoint upgrades Key: FLINK-27868 URL: https://issues.apache.org/jira/browse/FLINK-27868 Project: Flin

[jira] [Created] (FLINK-27869) AdaptiveSchedulerITCase. testStopWithSavepointFailOnStop failed with FAIL_ON_CHECKPOINT_COMPLETE

2022-06-01 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-27869: Summary: AdaptiveSchedulerITCase. testStopWithSavepointFailOnStop failed with FAIL_ON_CHECKPOINT_COMPLETE Key: FLINK-27869 URL: https://issues.apache.org/jira/browse/FLINK-27869

[jira] [Created] (FLINK-27870) SemiAntiJoinStreamITCase. testAntiJoinWithTwoSidesRetraction failed with InterruptedException

2022-06-01 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-27870: Summary: SemiAntiJoinStreamITCase. testAntiJoinWithTwoSidesRetraction failed with InterruptedException Key: FLINK-27870 URL: https://issues.apache.org/jira/browse/FLINK-27870

Re: [DISCUSS] FLIP-233: Introduce HTTP Connector

2022-06-01 Thread Ber, Jeremy
Hi Maciej, This is great! I would love to test this out myself and review your Async Sink implementation. The best way to get this sink merged in would be to reopen the FLIP, add in your modifications / implementation details, and drive the FLIP process forward. Let's work together to make it

Re: [DISCUSS] FLIP-223: Support HiveServer2 Endpoint

2022-06-01 Thread Shengkai Fang
Hi, Jingsong. Thanks for your feedback. > I've read the FLIP and it's not quite clear what the specific unsupported items are Yes. I have added a section named Difference with HiveServer2 and list the difference between the SQL Gateway with HiveServer2 endpoint and HiveServer2. > Support multip

Re: Translation in korean?

2022-06-01 Thread Daisy Tsang
Not sure whether this helps, but there was a previous discussion regarding adding support for the Japanese language: https://www.mail-archive.com/dev@flink.apache.org/msg39138.html On Wed, Jun 1, 2022 at 12:57 PM Dennis Jung wrote: > Hello, have a question about document translation. > > Are the

Re: [VOTE] Apache Flink Kubernetes Operator Release 1.0.0, release candidate #4

2022-06-01 Thread Márton Balassi
Hi team, +1 (binding) Verified the following: - NOTICE file looks good :-) - Signatures, Hashes - No binaries in source release - Helm Repo works, Helm install works, docker image matches release commit tag - Build from source, build container from source - Submit example application, session an

RE: RE: [DISCUSS] FLIP-233: Introduce HTTP Connector

2022-06-01 Thread Krzysztof Chmielewski
Hello Jeremy, I'm the original author and currently main contributor to our HTTP connector [1] that Maciej wrote about. I'm more than happy to help and get this going. Should you have any questions about our implementation feel free to contact me. [1]: https://github.com/getindata/flink-http-conn

[jira] [Created] (FLINK-27871) Dynamic configuration change is undedected on config removal

2022-06-01 Thread Matyas Orhidi (Jira)
Matyas Orhidi created FLINK-27871: - Summary: Dynamic configuration change is undedected on config removal Key: FLINK-27871 URL: https://issues.apache.org/jira/browse/FLINK-27871 Project: Flink

Re: [VOTE] Apache Flink Kubernetes Operator Release 1.0.0, release candidate #4

2022-06-01 Thread Biao Geng
Hi there, +1 (non-binding) Successfully verified the following: - Checksums and gpg signatures of the tar files. - No binaries in source release - Build from source, build image from source - Helm Repo works, Helm install works, docker image matches release commit tag - Submit example application

[jira] [Created] (FLINK-27872) Allow KafkaBuilder to set arbitrary subscribers

2022-06-01 Thread Salva (Jira)
Salva created FLINK-27872: - Summary: Allow KafkaBuilder to set arbitrary subscribers Key: FLINK-27872 URL: https://issues.apache.org/jira/browse/FLINK-27872 Project: Flink Issue Type: New Feature

[DISCUSS] suggest using granularityNumber in ColumnStats

2022-06-01 Thread Jing Ge
Hi Dev, I am not really sure if it is feasible to start this discussion. According to the contribution guidelines, dev ml is the right place to reach consensus. In ColumnStats, Currently ndv, which stands for "number of distinct values", is used. First of all, it is difficult to understand the me

Re: [VOTE] Apache Flink Kubernetes Operator Release 1.0.0, release candidate #4

2022-06-01 Thread Jim Busche
Hi Yang, +1 (not-binding) * Helm install looks good, logs look normal * Podman build from source looks good * Security scans of a built image and your ghcr.io/apache/flink-kubernetes-operator:fa2cd14 container look great. * UI and basic sample look good. Thank you, Jim

[jira] [Created] (FLINK-27873) Primary key is lost after calling addColumns

2022-06-01 Thread Yaroslav Tkachenko (Jira)
Yaroslav Tkachenko created FLINK-27873: -- Summary: Primary key is lost after calling addColumns Key: FLINK-27873 URL: https://issues.apache.org/jira/browse/FLINK-27873 Project: Flink Issu

[jira] [Created] (FLINK-27874) KeyedStateCheckpointingITCase Timeout

2022-06-01 Thread Shengkai Fang (Jira)
Shengkai Fang created FLINK-27874: - Summary: KeyedStateCheckpointingITCase Timeout Key: FLINK-27874 URL: https://issues.apache.org/jira/browse/FLINK-27874 Project: Flink Issue Type: Bug

Re:RE: RE: [DISCUSS] FLIP-233: Introduce HTTP Connector

2022-06-01 Thread Xuyang
Hi, Jeremy, thanks for driving this work. I have noticed that there have been a lot of users that need a official HTTP connector from user-email. I have some comments about this Flip. * The title is "Introduce HTTP Connector", but it seems that this FLIP only contains the content of HTTP sink.

[jira] [Created] (FLINK-27875) Introduce TableScan and TableRead as an abstraction layer above FileStore for reading RowData

2022-06-01 Thread Caizhi Weng (Jira)
Caizhi Weng created FLINK-27875: --- Summary: Introduce TableScan and TableRead as an abstraction layer above FileStore for reading RowData Key: FLINK-27875 URL: https://issues.apache.org/jira/browse/FLINK-27875

[jira] [Created] (FLINK-27876) Error side is chosen in BatchPhysicalHashJoinRule when the join is Semi or Anti

2022-06-01 Thread xuyang (Jira)
xuyang created FLINK-27876: -- Summary: Error side is chosen in BatchPhysicalHashJoinRule when the join is Semi or Anti Key: FLINK-27876 URL: https://issues.apache.org/jira/browse/FLINK-27876 Project: Flink

Re: [DISCUSS] FLIP-234: Support Retryable Lookup Join To Solve Delayed Updates Issue In External Systems

2022-06-01 Thread Lincoln Lee
Hi everyone, I've updated the FLIP[1] based on this discussion thread that we agree to have a single unified 'LOOKUP' hint and also a related part in FLIP-221[2] which is mainly for the necessity of the common table option 'lookup.async'. The main updates are: 1. the new unified 'LOOKUP' hint, m

Re: [DISCUSS] FLIP-221 Abstraction for lookup source cache and metric

2022-06-01 Thread Jark Wu
Thank Qingsheng for the detailed summary and updates, The changes look good to me in general. I just have one minor improvement comment. Could we add a static util method to the "FullCachingReloadTrigger" interface for quick usage? #periodicReloadAtFixedRate(Duration) #periodicReloadWithFixedDela

Re: [VOTE] Apache Flink Kubernetes Operator Release 1.0.0, release candidate #4

2022-06-01 Thread Yang Wang
+1 (binding) Successfully verified the following: - Verify that the checksums and GPG files - Verify that the source distributions do not contain any binaries - Build binary and image from release source - Verify the NOTICE and licenses in source release and the docker image - Verify the helm char

Re: [DISCUSS] FLIP-221 Abstraction for lookup source cache and metric

2022-06-01 Thread Becket Qin
Thanks for updating the FLIP, Qingsheng. A few more comments: 1. I am still not sure about what is the use case for cacheMissingKey(). More specifically, when would users want to have getCache() return a non-empty value and cacheMissingKey() returns false? 2. The builder pattern. Usually the buil

[jira] [Created] (FLINK-27877) Improve performance of several feature engineering algorithms

2022-06-01 Thread Zhipeng Zhang (Jira)
Zhipeng Zhang created FLINK-27877: - Summary: Improve performance of several feature engineering algorithms Key: FLINK-27877 URL: https://issues.apache.org/jira/browse/FLINK-27877 Project: Flink

Re: [DISCUSS] suggest using granularityNumber in ColumnStats

2022-06-01 Thread Becket Qin
Hi Jing, While I do agree that NDV is a little confusing at first sight, it seems quite concise once I got the meaning. So personally I am OK with keeping it as is, but proper documentation would be helpful. If we really want to replace it with a more professional name, *cardinality* might be a go