Re: [DISCUSS] FLIP-190: Support Version Upgrades for Table API & SQL Programs

2021-12-08 Thread Jing Zhang
Hi Timo, Thanks a lot for driving this discussion. I believe it could solve many problems what we are suffering in upgrading. I only have a little complain on the following point. > For simplification of the design, we assume that upgrades use a step size of a single minor version. We don't

Re: [DISCUSS] Releasing Flink 1.14.1

2021-12-08 Thread Caizhi Weng
Hi devs! Sorry for the interruptions, but I just found an issue [1] (which I think is a blocking one) in every Flink version, including Flink 1.14.1. For Flink < 1.15, this issue will cause incorrect result when user cast two strings to numerics and compare the numerics. I'm planning for a

[jira] [Created] (FLINK-25227) Comparing the equality of the same (boxed) numeric values returns false

2021-12-08 Thread Caizhi Weng (Jira)
Caizhi Weng created FLINK-25227: --- Summary: Comparing the equality of the same (boxed) numeric values returns false Key: FLINK-25227 URL: https://issues.apache.org/jira/browse/FLINK-25227 Project: Flink

[jira] [Created] (FLINK-25226) Add documentation about the AdaptiveBatchScheduler

2021-12-08 Thread Zhu Zhu (Jira)
Zhu Zhu created FLINK-25226: --- Summary: Add documentation about the AdaptiveBatchScheduler Key: FLINK-25226 URL: https://issues.apache.org/jira/browse/FLINK-25226 Project: Flink Issue Type:

[jira] [Created] (FLINK-25225) Add e2e TPCDS tests to run against the AdatpiveBatchScheduler

2021-12-08 Thread Zhu Zhu (Jira)
Zhu Zhu created FLINK-25225: --- Summary: Add e2e TPCDS tests to run against the AdatpiveBatchScheduler Key: FLINK-25225 URL: https://issues.apache.org/jira/browse/FLINK-25225 Project: Flink Issue

Re: [DISCUSS] Releasing Flink 1.14.1

2021-12-08 Thread Zhu Zhu
update: backport of FLINK-19142 is done Thanks, Zhu Zhu Zhu 于2021年12月8日周三 19:35写道: > Hi Martijn, > > I'd like to backport the fix of FLINK-19142 to 1.14.1. > The backport is in progress. > Will update it here when it is done. > > Thanks, > Zhu > > Jingsong Li 于2021年12月8日周三 10:33写道: > >> Hi

[jira] [Created] (FLINK-25224) Bump the hadoop version up

2021-12-08 Thread Jira
刘方奇 created FLINK-25224: --- Summary: Bump the hadoop version up Key: FLINK-25224 URL: https://issues.apache.org/jira/browse/FLINK-25224 Project: Flink Issue Type: Improvement Components:

[DISCUSS][FLINK-24427] Hide Scala from table planner

2021-12-08 Thread Francesco Guardiani
Hi all, In case you haven't seen, last week I published in the issue comments this document to explain how we're proceeding to hide Scala from table planner: https://docs.google.com/document/d/12yDUCnvcwU2mODBKTHQ1xhfOq1ujYUrXltiN_rbhT34/edit?usp=sharing There is a section I've added yesterday

Re: [DISCUSS] Strong read-after-write consistency of Flink FileSystems

2021-12-08 Thread David Morávek
Hi Martijn, I simply wasn't aware of that one :) It seems to be provided the guarantees that we need [1]. > Of course, Azure Storage is built on a platform grounded in strong > consistency guaranteeing that writes are made durable before acknowledging > success to the client. This is critically

Re: [VOTE] Deprecate Java 8 support

2021-12-08 Thread Thomas Weise
+1 (binding) On Wed, Dec 8, 2021 at 2:20 AM Till Rohrmann wrote: > > +1 (binding) > > Cheers, > Till > > On Tue, Dec 7, 2021 at 12:35 PM Matthias Pohl > wrote: > > > Thanks for pushing this Chesnay! > > +1 (binding) > > > > On Mon, Dec 6, 2021 at 9:44 PM Martijn Visser > > wrote: > > > > > +1

[DISCUSS] Deprecate MapR FS

2021-12-08 Thread Martijn Visser
Hi all, Flink supports multiple file systems [1] which includes MapR FS. MapR as a company doesn't exist anymore since 2019, the technology and intellectual property has been sold to Hewlett Packard. I don't think that there's anyone who's using MapR anymore and therefore I think it would be

[jira] [Created] (FLINK-25223) ElasticsearchWriterITCase fails on AZP

2021-12-08 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-25223: - Summary: ElasticsearchWriterITCase fails on AZP Key: FLINK-25223 URL: https://issues.apache.org/jira/browse/FLINK-25223 Project: Flink Issue Type: Bug

Re: [DISCUSS] Strong read-after-write consistency of Flink FileSystems

2021-12-08 Thread Martijn Visser
Hi David, Just to be sure, since you've already included Azure Blob Storage, but did you deliberately skip Azure Data Lake Store Gen2? That's currently supported and also used by Flink users [1]. There's also MapR FS, but I doubt if that is still used. Best regards, [1]

FileSink in Apache Flink not generating logs in output folder

2021-12-08 Thread Dhingra, Kajal
I am new to Flink and doing a POC on it and using it to read data from kafka topic and to store it in files on server. I am using FileSink to store files, it creates the directory structure date and time wise but no logs files are getting created. When i run the program it creates directory

Re: [ANNOUNCE] New Apache Flink Committer - Ingo Bürk

2021-12-08 Thread godfrey he
Congratulations, Ingo! Best, Godfrey Roman Khachatryan 于2021年12月6日周一 下午6:07写道: > > Congratulations, Ingo! > > Regards, > Roman > > > On Mon, Dec 6, 2021 at 11:05 AM Yang Wang wrote: > > > > Congratulations, Ingo! > > > > Best, > > Yang > > > > Sergey Nuyanzin 于2021年12月6日周一 下午3:35写道: > > > > >

Re: [ANNOUNCE] New Apache Flink Committer - Matthias Pohl

2021-12-08 Thread godfrey he
Congratulations, Matthias! Best, Godfrey Roman Khachatryan 于2021年12月6日周一 下午6:07写道: > > Congratulations, Matthias! > > Regards, > Roman > > > On Mon, Dec 6, 2021 at 11:04 AM Yang Wang wrote: > > > > Congratulations, Matthias! > > > > Best, > > Yang > > > > Sergey Nuyanzin 于2021年12月6日周一

Re: [DISCUSS] FLIP-190: Support Version Upgrades for Table API & SQL Programs

2021-12-08 Thread godfrey he
Hi Timo, Thanks for the explanation, it's much clearer now. One thing I want to confirm about `supportedPlanFormat ` and `supportedSavepointFormat `: `supportedPlanFormat ` supports multiple versions, while `supportedSavepointFormat ` supports only one version ? A json plan can be deserialized

[jira] [Created] (FLINK-25222) Remove NetworkFailureProxy used for Kafka connector tests

2021-12-08 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25222: --- Summary: Remove NetworkFailureProxy used for Kafka connector tests Key: FLINK-25222 URL: https://issues.apache.org/jira/browse/FLINK-25222 Project: Flink

[jira] [Created] (FLINK-25221) Allow global table options for all table connectors

2021-12-08 Thread Timo Walther (Jira)
Timo Walther created FLINK-25221: Summary: Allow global table options for all table connectors Key: FLINK-25221 URL: https://issues.apache.org/jira/browse/FLINK-25221 Project: Flink Issue

Re: [DISCUSS] Releasing Flink 1.14.1

2021-12-08 Thread Zhu Zhu
Hi Martijn, I'd like to backport the fix of FLINK-19142 to 1.14.1. The backport is in progress. Will update it here when it is done. Thanks, Zhu Jingsong Li 于2021年12月8日周三 10:33写道: > Hi Martijn, > > We just created a cherry-pick pull-request for >

[jira] [Created] (FLINK-25220) Writing an architectural rule for all IT cases w.r.t. MiniCluster

2021-12-08 Thread Jing Ge (Jira)
Jing Ge created FLINK-25220: --- Summary: Writing an architectural rule for all IT cases w.r.t. MiniCluster Key: FLINK-25220 URL: https://issues.apache.org/jira/browse/FLINK-25220 Project: Flink

Re: [VOTE] Deprecate Java 8 support

2021-12-08 Thread Till Rohrmann
+1 (binding) Cheers, Till On Tue, Dec 7, 2021 at 12:35 PM Matthias Pohl wrote: > Thanks for pushing this Chesnay! > +1 (binding) > > On Mon, Dec 6, 2021 at 9:44 PM Martijn Visser > wrote: > > > +1 (non-binding) > > > > Op ma 6 dec. 2021 om 19:58 schreef Ingo Bürk > > > > > Before more people

[jira] [Created] (FLINK-25219) load configuration from flink-conf.yaml, some of my parameters were modified

2021-12-08 Thread jackie (Jira)
jackie created FLINK-25219: -- Summary: load configuration from flink-conf.yaml, some of my parameters were modified Key: FLINK-25219 URL: https://issues.apache.org/jira/browse/FLINK-25219 Project: Flink

Re: [DISCUSS] FLIP-190: Support Version Upgrades for Table API & SQL Programs

2021-12-08 Thread Timo Walther
Hi Wenlong, thanks for the feedback. Great that we reached consensus here. I will update the entire document with my previous example shortly. > if we don't update the version when plan format changes, we can't find that the plan can't not be deserialized in 1.15 This should not be a