Re: [DISCUSS] FLIP-378: Support Avro timestamp with local timezone

2023-11-13 Thread Mingliang Liu
+1 on this effort.

I opened a JIRA sometime ago on this topic [1] and Jagadesh submitted a PR
[2] for review. In the comments, James proposed a different approach from
the PR. I don't think this topic was well-discussed there by the community
and no consensus on how to move forward. Reading the doc you shared and
revisiting the discussion there, the proposed mapping looks good to me.

[1] https://issues.apache.org/jira/browse/FLINK-30483
[2] https://github.com/apache/flink/pull/21594

On Mon, Nov 13, 2023 at 8:10 PM Jark Wu  wrote:

> +1
>
> I think we can mark the temporary config option as deprecated when we
> introduce it.
> So we can remove it after 2 minor releases (1.19, 1.20), i.e., drop in 2.0
> release.
>
> A minor comment about the config option, I would suggest to use
> "avro.timestamp-mapping.legacy"
> instead of "avro.timestamp_mapping.legacy". Because Flink community prefer
> dash than underline[1].
>
> Best.,
> Jark
>
> [1]:
>
> https://flink.apache.org/how-to-contribute/code-style-and-quality-components/#configuration-changes
>
> On Sat, 4 Nov 2023 at 21:40, Gyula Fóra  wrote:
>
> > +1
> >
> > Gyula
> >
> > On Thu, Nov 2, 2023 at 6:18 AM Martijn Visser 
> > wrote:
> >
> > > +1
> > >
> > > On Thu, Nov 2, 2023 at 12:44 PM Leonard Xu  wrote:
> > > >
> > > >
> > > > > Thanks @Leonard Xu . Two minor versions
> > are
> > > definitely needed for flip the configs.
> > > >
> > > > Sorry, Peter. I thought the next minor versions are 1.19、2.0, but
> > > actually it should be 1.19、1.20、2.0 from current community version plan
> > > IIUC, so remove the config in 2.0 should be okay if the 1.20 version
> > exists
> > > .
> > > >
> > > > Best,
> > > > Leonard
> > > >
> > > >
> > > > >
> > > > > On Mon, Oct 30, 2023 at 8:55 PM Leonard Xu  >  > > xbjt...@gmail.com>> wrote:
> > > > > Thanks @Peter for driving this FLIP
> > > > >
> > > > > +1 from my side, the timestamp semantics mapping looks good to me.
> > > > >
> > > > > >  In the end, the legacy behavior will be dropped in
> > > > > > Flink 2.0
> > > > > > I don’t think we can drop this option which introduced in 1.19
> and
> > > drop in 2.0, the API removal requires at least two minor versions.
> > > > >
> > > > >
> > > > > Best,
> > > > > Leonard
> > > > >
> > > > > > 2023年10月31日 上午11:18,Peter Huang  >  > > huangzhenqiu0...@gmail.com>> 写道:
> > > > > >
> > > > > > Hi Devs,
> > > > > >
> > > > > > Currently, Flink Avro Format doesn't support the Avro time
> > > (milli/micros)
> > > > > > with local timezone type.
> > > > > > Although the Avro timestamp (millis/micros) type is supported and
> > is
> > > mapped
> > > > > > to flink timestamp without timezone,
> > > > > > it is not compliant to semantics defined in Consistent timestamp
> > > types in
> > > > > > Hadoop SQL engines
> > > > > > <
> > >
> >
> https://docs.google.com/document/d/1gNRww9mZJcHvUDCXklzjFEQGpefsuR_akCDfWsdE35Q/edit#heading=h.n699ftkvhjlo
> > > <
> > >
> >
> https://docs.google.com/document/d/1gNRww9mZJcHvUDCXklzjFEQGpefsuR_akCDfWsdE35Q/edit#heading=h.n699ftkvhjlo
> > > >>
> > > > > > .
> > > > > >
> > > > > > I propose to support Avro timestamps with the compliance to the
> > > mapping
> > > > > > semantics [1] by using a configuration flag.
> > > > > > To keep back compatibility, current mapping is kept as default
> > > behavior.
> > > > > > Users can explicitly turn on the new mapping
> > > > > > by setting it to false. In the end, the legacy behavior will be
> > > dropped in
> > > > > > Flink 2.0
> > > > > >
> > > > > > Looking forward to your feedback.
> > > > > >
> > > > > >
> > > > > > [1]
> > > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-378%3A+Support+Avro+timestamp+with+local+timezone
> > > <
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-378%3A+Support+Avro+timestamp+with+local+timezone
> > > >
> > > > > >
> > > > > >
> > > > > > Best Regards
> > > > > >
> > > > > > Peter Huang
> > > > >
> > > >
> > >
> >
>


Re: [VOTE] FLIP-377: Support configuration to disable filter push down for Table/SQL Sources

2023-11-13 Thread Jiabao Sun
Thanks Leonard for the feedback.

Opening this voting process because there are no responses have been received 
for over a week.
I will continue to check with the developers in the discussion thread.

Best,
Jiabao


> 2023年11月14日 14:51,Leonard Xu  写道:
> 
> Thanks Jiabao for driving this.
> 
> The discussion thread need a consensus result before we start a VOTE, could 
> you check the discussion status in the discussion thread first and then 
> continue the vote, please correct me I understand the discussion thread state 
> incorrectly.
> 
> Best,
> Leonard
> 
> 
> 
> 
>> 2023年11月7日 上午11:16,Jiabao Sun  写道:
>> 
>> Hi Devs,
>> 
>> I'd like to start a vote on FLIP-377: Support configuration to disable 
>> filter pushdown for Table/SQL Sources[1] which has been discussed in this 
>> thread [2].
>> 
>> The vote will be open for at least 72 hours unless there is an objection or 
>> not enough votes.
>> 
>> [1] 
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=276105768
>> [2] https://lists.apache.org/thread/nvxx8sp9jm009yywm075hoffr632tm7j
>> 
>> Best,
>> Jiabao



Re: [VOTE] FLIP-377: Support configuration to disable filter push down for Table/SQL Sources

2023-11-13 Thread Leonard Xu
Thanks Jiabao for driving this.

The discussion thread need a consensus result before we start a VOTE, could you 
check the discussion status in the discussion thread first and then continue 
the vote, please correct me I understand the discussion thread state 
incorrectly.

Best,
Leonard




> 2023年11月7日 上午11:16,Jiabao Sun  写道:
> 
> Hi Devs,
> 
> I'd like to start a vote on FLIP-377: Support configuration to disable filter 
> pushdown for Table/SQL Sources[1] which has been discussed in this thread [2].
> 
> The vote will be open for at least 72 hours unless there is an objection or 
> not enough votes.
> 
> [1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=276105768
> [2] https://lists.apache.org/thread/nvxx8sp9jm009yywm075hoffr632tm7j
> 
> Best,
> Jiabao



Re: [DISCUSS] FLIP-378: Support Avro timestamp with local timezone

2023-11-13 Thread Jark Wu
+1

I think we can mark the temporary config option as deprecated when we
introduce it.
So we can remove it after 2 minor releases (1.19, 1.20), i.e., drop in 2.0
release.

A minor comment about the config option, I would suggest to use
"avro.timestamp-mapping.legacy"
instead of "avro.timestamp_mapping.legacy". Because Flink community prefer
dash than underline[1].

Best.,
Jark

[1]:
https://flink.apache.org/how-to-contribute/code-style-and-quality-components/#configuration-changes

On Sat, 4 Nov 2023 at 21:40, Gyula Fóra  wrote:

> +1
>
> Gyula
>
> On Thu, Nov 2, 2023 at 6:18 AM Martijn Visser 
> wrote:
>
> > +1
> >
> > On Thu, Nov 2, 2023 at 12:44 PM Leonard Xu  wrote:
> > >
> > >
> > > > Thanks @Leonard Xu . Two minor versions
> are
> > definitely needed for flip the configs.
> > >
> > > Sorry, Peter. I thought the next minor versions are 1.19、2.0, but
> > actually it should be 1.19、1.20、2.0 from current community version plan
> > IIUC, so remove the config in 2.0 should be okay if the 1.20 version
> exists
> > .
> > >
> > > Best,
> > > Leonard
> > >
> > >
> > > >
> > > > On Mon, Oct 30, 2023 at 8:55 PM Leonard Xu   > xbjt...@gmail.com>> wrote:
> > > > Thanks @Peter for driving this FLIP
> > > >
> > > > +1 from my side, the timestamp semantics mapping looks good to me.
> > > >
> > > > >  In the end, the legacy behavior will be dropped in
> > > > > Flink 2.0
> > > > > I don’t think we can drop this option which introduced in 1.19 and
> > drop in 2.0, the API removal requires at least two minor versions.
> > > >
> > > >
> > > > Best,
> > > > Leonard
> > > >
> > > > > 2023年10月31日 上午11:18,Peter Huang   > huangzhenqiu0...@gmail.com>> 写道:
> > > > >
> > > > > Hi Devs,
> > > > >
> > > > > Currently, Flink Avro Format doesn't support the Avro time
> > (milli/micros)
> > > > > with local timezone type.
> > > > > Although the Avro timestamp (millis/micros) type is supported and
> is
> > mapped
> > > > > to flink timestamp without timezone,
> > > > > it is not compliant to semantics defined in Consistent timestamp
> > types in
> > > > > Hadoop SQL engines
> > > > > <
> >
> https://docs.google.com/document/d/1gNRww9mZJcHvUDCXklzjFEQGpefsuR_akCDfWsdE35Q/edit#heading=h.n699ftkvhjlo
> > <
> >
> https://docs.google.com/document/d/1gNRww9mZJcHvUDCXklzjFEQGpefsuR_akCDfWsdE35Q/edit#heading=h.n699ftkvhjlo
> > >>
> > > > > .
> > > > >
> > > > > I propose to support Avro timestamps with the compliance to the
> > mapping
> > > > > semantics [1] by using a configuration flag.
> > > > > To keep back compatibility, current mapping is kept as default
> > behavior.
> > > > > Users can explicitly turn on the new mapping
> > > > > by setting it to false. In the end, the legacy behavior will be
> > dropped in
> > > > > Flink 2.0
> > > > >
> > > > > Looking forward to your feedback.
> > > > >
> > > > >
> > > > > [1]
> > > > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-378%3A+Support+Avro+timestamp+with+local+timezone
> > <
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-378%3A+Support+Avro+timestamp+with+local+timezone
> > >
> > > > >
> > > > >
> > > > > Best Regards
> > > > >
> > > > > Peter Huang
> > > >
> > >
> >
>


[jira] [Created] (FLINK-33539) Potential regression (Nov. 7th): listAddAll on RocksDB with Java8

2023-11-13 Thread Zakelly Lan (Jira)
Zakelly Lan created FLINK-33539:
---

 Summary: Potential regression (Nov. 7th): listAddAll on RocksDB 
with Java8
 Key: FLINK-33539
 URL: https://issues.apache.org/jira/browse/FLINK-33539
 Project: Flink
  Issue Type: Bug
  Components: Runtime / State Backends
Affects Versions: 1.19.0
Reporter: Zakelly Lan
Assignee: Zakelly Lan
 Attachments: image-2023-11-14-10-56-36-189.png

!image-2023-11-14-10-56-36-189.png!

There is a drop from commit fccf26bb449 to fa8ea3f9a44 .



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] FLIP-364: Improve the restart-strategy

2023-11-13 Thread Jing Ge
Hi Rui,

Thanks for the proposal! I agree with Zhu that any changes of the default
behaviors will have impact on users' jobs in the production environment and
it would be necessary to have users' attention to to avoid
any surprises after upgrading Flink.

@Zhu
for 1, if we change the default values, we will not only change the
behavior of the default restart-strategy but also change the default
behavior of exponential-delay strategy for users who are explicitly
configured to use the strategy with default values in the production.

NIT: @Rui it would be great if you could point out the source code
reference where the restartAttempt will be calculated rapidly,
i.e. +100 with your one task example. All related docs I could find didn't
describe it clearly.

Best regards,
Jing


On Mon, Nov 13, 2023 at 11:55 AM Zhu Zhu  wrote:

> Hi Rui,
> Thanks for creating this FLIP and sorry for jumping in so late into the
> discussion.
>
> The improvements of exponential-delay strategy and making it the default
> strategy looks good it me in general. I have some comments for it, as well
> as for the failure counting.
>
> 1. default values of exponential-delay configuration
> It is mentioned in the FLIP that "the default value of these options are
> not changed, we just change them for default restart-strategy. The default
> restart-strategy just takes effect if checkpointing is enabled and the
> user doesn’t configure the restart-strategy". I'm a bit concerned with
> the inconsistency which may cause confusion to users. So if the proposed
> configuration values work better in most cases, I'm leaning towards to
> change the default values.
>
> 2. the default tolerable failure count
> Currently, the restart-strategy behaves like this by default(if not
> configured):
> 1. job retries on failures indefinitely if checkpointing is enabled
> 2. job fails on any failure if checkpointing is disabled
> Changing it to always restart indefinitely may result in unexpected
> behaviors in production. Therefore, either we should do the same
> thing to exponential-delay, or have an open discussion(also involving
> the user ML) on changing this default behavior.
>
> 3. failure counting
> Flink currently will try to recognize concurrent failures and group them
> together, which can be seen in the web UI. So how about to align the
> failure counting with the concurrent failures computing? This can make it
> more consistent and easier for understanding. It will require changes to
> the concurrent failures computing though, i.e. taking the backoff time
> into consideration. So maybe we can open a seperate FLIP for this change.
>
> Thanks,
> Zhu
>
> Rui Fan <1996fan...@gmail.com> 于2023年11月10日周五 18:22写道:
>
> > I'll start voting next Monday if there isn't any other comment.
> >
> > Best,
> > Rui
> >
> > On Thu, Oct 19, 2023 at 6:59 PM Rui Fan <1996fan...@gmail.com> wrote:
> >
> > > Hi Konstantin and Max,
> > >
> > > Thanks for your feedback!
> > >
> > > Sorry, I forgot to mention the default value of
> > > `restart-strategy.exponential-delay.max-attempts-before-reset-backoff`.
> > >
> > > Retrying forever sounds good to me, I have added it to the FLIP:
> > >
> > > The default value of
> > > `restart-strategy.exponential-delay.max-attempts-before-reset-backoff`
> is
> > > Integer.MAX_VALUE.
> > >
> > > Best,
> > > Rui
> > >
> > > On Thu, Oct 19, 2023 at 6:29 PM Maximilian Michels 
> > wrote:
> > >
> > >> Hey Rui,
> > >>
> > >> +1 for making exponential backoff the default. I agree with Konstantin
> > >> that retrying forever is a good default for exponential backoff
> > >> because oftentimes the issue will resolve eventually. The purpose of
> > >> exponential backoff is precisely to continue to retry without causing
> > >> too much load. However, I'm not against adding an optional max number
> > >> of retries.
> > >>
> > >> -Max
> > >>
> > >> On Thu, Oct 19, 2023 at 11:35 AM Konstantin Knauf 
> > >> wrote:
> > >> >
> > >> > Hi Rui,
> > >> >
> > >> > Thank you for this proposal and working on this. I also agree that
> > >> > exponential back off makes sense as a new default in general. I
> think
> > >> > restarting indefinitely (no max attempts) makes sense by default,
> > >> though,
> > >> > but of course allowing users to change is valuable.
> > >> >
> > >> > So, overall +1.
> > >> >
> > >> > Cheers,
> > >> >
> > >> > Konstantin
> > >> >
> > >> > Am Di., 17. Okt. 2023 um 07:11 Uhr schrieb Rui Fan <
> > >> 1996fan...@gmail.com>:
> > >> >
> > >> > > Hi all,
> > >> > >
> > >> > > I would like to start a discussion on FLIP-364: Improve the
> > >> > > restart-strategy[1]
> > >> > >
> > >> > > As we know, the restart-strategy is critical for flink jobs, it
> > mainly
> > >> > > has two functions:
> > >> > > 1. When an exception occurs in the flink job, quickly restart the
> > job
> > >> > > so that the job can return to the running state.
> > >> > > 2. When a job cannot be recovered after frequent restarts within
> > >> > > a certain period of time, 

[jira] [Created] (FLINK-33538) Create a "best match" option for CEP when using the optional operator in the Pattern API

2023-11-13 Thread Brian Lehman (Jira)
Brian Lehman created FLINK-33538:


 Summary: Create a "best match" option for CEP when using the 
optional operator in the Pattern API 
 Key: FLINK-33538
 URL: https://issues.apache.org/jira/browse/FLINK-33538
 Project: Flink
  Issue Type: Improvement
  Components: Library / CEP
Reporter: Brian Lehman


I’ll try to detail my issue and the initial experimentation I’ve done in the 
CEP source code to show the potential of making this nearly as fast as having 
no optional events in the pattern.

 

Using Flink CEP I have implemented a pattern that detects a sequence of events. 
Each event has internal attributes that are checked and inter-event time deltas 
are also checked as part of the pattern.

 

When I require all events in my sequence (say 10 elements long) Flink CEP works 
well and is super fast at detecting the “perfect” sequences. Should one or more 
of the events not get recorded, I would still like to detect the partial 
sequence. Since I don’t know which events might be missing I must make all of 
the events optional. While this worked, it was significantly slower, to the 
point that it is an unworkable solution.

 

CEP is set to alert as fast as possible – so when everything is optional, once 
it passes a single event the path to final in ComputationState is immediately 
found and the potentialMatch is emitted. If a perfect match is coming, I will 
not find the pattern unless the skip strategy is set to noSkip(). Because all 
are optional, in the instance of a perfect match, CEP wants to emit all 2^10 
possible matches when noSkip() is used. This is what causes the extreme 
performance drop off.

 

I would like to add an option to Flink CEP that allows the “best match” when 
the optional() attribute is used in the Pattern API. The default can still be 
“fastest match” which would operate as it does today. In the “best match” 
scenario, the potentialMatches would be held back (not emitted) until the full 
pattern time has passed.

 

I have made some modifications to the source code (primarily 
org.apache.flink.cep.nfa.NFA) that show this can work and be nearly as fast as 
the perfect match option. My basic strategy is the following:
 * Once a longer match is found, discard all of the shorter length sub-matches 
from partialMatches and potentialMatches.
 * Eg –  [E1, $endState$]
 * [E1, E2, $endState$] - discard the [E1, $endState$] and [E2, $endState$] and 
those partialMatch paths as well
 * [E1, E2, E3, $endState$] – discard the [E1, E2, $endState$], etc.


 * DeweyNumber allows a way to quickly assess match lengths, by comparing 
DeweyNumber.lengths.
 * DeweyNumber provides a way to assess matches, with its versioning – matches 
tend to use a version of zero (eg. 1.0.0.0.0.0 is a perfect match to the first 
6 events of the pattern)
 * Delay emitting the “best match” until the options are exhausted, eliminating 
the shorter matches as quickly as possible.

 

I’m at a place in my modifications where it would be very beneficial to work 
with an expert that would understand how best to accomplish this without 
impacting current functionality/performance. I’m happy to collaborate and share 
my work if that helps. My current modifications are a combination of mods and 
logs to allow me to see what is going on internally in the CEP processing.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [VOTE] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects

2023-11-13 Thread Mingliang Liu
+1 (non-binding)

Thanks for driving this.

On Mon, Nov 13, 2023 at 12:37 AM Hang Ruan  wrote:

> +1(non-binding)
>
> Best,
> Hang
>
> Jing Ge  于2023年11月13日周一 16:36写道:
>
> > +1(binding)
> > Thanks!
> >
> > Best Regards,
> > Jing
> >
> > On Mon, Nov 13, 2023 at 8:34 AM Zhu Zhu  wrote:
> >
> > > +1 (binding)
> > >
> > > Thanks,
> > > Zhu
> > >
> > > Xia Sun  于2023年11月13日周一 15:02写道:
> > >
> > > > +1 (non-binding)
> > > >
> > > > Best,
> > > > Xia
> > > >
> > > > Samrat Deb  于2023年11月13日周一 12:37写道:
> > > >
> > > > > +1 (non binding)
> > > > >
> > > > > Bests,
> > > > > Samrat
> > > > >
> > > > > On Mon, 13 Nov 2023 at 9:10 AM, Yangze Guo 
> > wrote:
> > > > >
> > > > > > +1 (binding)
> > > > > >
> > > > > > Best,
> > > > > > Yangze Guo
> > > > > >
> > > > > > On Mon, Nov 13, 2023 at 11:35 AM weijie guo <
> > > guoweijieres...@gmail.com
> > > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > +1(binding)
> > > > > > >
> > > > > > > Best regards,
> > > > > > >
> > > > > > > Weijie
> > > > > > >
> > > > > > >
> > > > > > > Lijie Wang  于2023年11月13日周一 10:40写道:
> > > > > > >
> > > > > > > > +1 (binding)
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Lijie
> > > > > > > >
> > > > > > > > Yuepeng Pan  于2023年11月10日周五 18:32写道:
> > > > > > > >
> > > > > > > > > +1(non-binding)
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Roc
> > > > > > > > >
> > > > > > > > > On 2023/11/10 03:58:10 Junrui Lee wrote:
> > > > > > > > > > Hi everyone,
> > > > > > > > > >
> > > > > > > > > > Thank you to everyone for the feedback on FLIP-381:
> > Deprecate
> > > > > > > > > configuration
> > > > > > > > > > getters/setters that return/set complex Java objects[1]
> > which
> > > > has
> > > > > > been
> > > > > > > > > > discussed in this thread [2].
> > > > > > > > > >
> > > > > > > > > > I would like to start a vote for it. The vote will be
> open
> > > for
> > > > at
> > > > > > least
> > > > > > > > > 72
> > > > > > > > > > hours (excluding weekends) unless there is an objection
> or
> > > not
> > > > > > enough
> > > > > > > > > votes.
> > > > > > > > > >
> > > > > > > > > > [1]
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992
> > > > > > > > > > [2]
> > > > > > https://lists.apache.org/thread/y5owjkfxq3xs9lmpdbl6d6jmqdgbjqxo
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>


[jira] [Created] (FLINK-33537) Docs version nav is missing 1.18 as an option

2023-11-13 Thread Robin Moffatt (Jira)
Robin Moffatt created FLINK-33537:
-

 Summary: Docs version nav is missing 1.18 as an option
 Key: FLINK-33537
 URL: https://issues.apache.org/jira/browse/FLINK-33537
 Project: Flink
  Issue Type: Bug
  Components: Documentation
Reporter: Robin Moffatt
 Attachments: image-2023-11-13-17-51-58-830.png

The docs site is missing 1.18 from the version navigation in the bottom-left  
!image-2023-11-13-17-51-58-830.png! 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (FLINK-33536) Flink Table API CSV streaming sink throws "IOException: Stream closed"

2023-11-13 Thread Prabhu Joseph (Jira)
Prabhu Joseph created FLINK-33536:
-

 Summary: Flink Table API CSV streaming sink throws "IOException: 
Stream closed"
 Key: FLINK-33536
 URL: https://issues.apache.org/jira/browse/FLINK-33536
 Project: Flink
  Issue Type: Bug
  Components: FileSystems, Table SQL / API
Affects Versions: 1.18.0
Reporter: Prabhu Joseph


Flink Table API CSV streaming sink throws "IOException: Stream closed". Prior 
to Flink 1.18, CSV streaming sink used to fail with 
"S3RecoverableFsDataOutputStream cannot sync state to S3" which is fixed by 
[FLINK-28513|https://issues.apache.org/jira/browse/FLINK-28513]. The fix seems 
not complete, it fails with this issue now.

*Repro*

{code}
SET 'execution.runtime-mode' = 'streaming';

create table dummy_table (
  id int,
  data string
) with (
  'connector' = 'filesystem',
  'path' = 's3://prabhuflinks3/dummy_table/',
  'format' = 'csv'
);

INSERT INTO dummy_table SELECT * FROM (VALUES (1, 'Hello World'), (2, 'Hi'), 
(2, 'Hi'), (3, 'Hello'), (3, 'World'), (4, 'ADD'), (5, 'LINE'));

{code}


*Error*

{code}
Caused by: java.io.IOException: Stream closed.
at 
org.apache.flink.core.fs.RefCountedFileWithStream.requireOpened(RefCountedFileWithStream.java:76)
at 
org.apache.flink.core.fs.RefCountedFileWithStream.write(RefCountedFileWithStream.java:52)
at 
org.apache.flink.core.fs.RefCountedBufferingFileStream.flush(RefCountedBufferingFileStream.java:104)
at 
org.apache.flink.fs.s3.common.writer.S3RecoverableFsDataOutputStream.closeAndUploadPart(S3RecoverableFsDataOutputStream.java:209)
at 
org.apache.flink.fs.s3.common.writer.S3RecoverableFsDataOutputStream.closeForCommit(S3RecoverableFsDataOutputStream.java:177)
at 
org.apache.flink.streaming.api.functions.sink.filesystem.OutputStreamBasedPartFileWriter.closeForCommit(OutputStreamBasedPartFileWriter.java:75)
at 
org.apache.flink.streaming.api.functions.sink.filesystem.BulkPartWriter.closeForCommit(BulkPartWriter.java:65)
at 
org.apache.flink.streaming.api.functions.sink.filesystem.Bucket.closePartFile(Bucket.java:263)
at 
org.apache.flink.streaming.api.functions.sink.filesystem.Bucket.onProcessingTime(Bucket.java:379)
at 
org.apache.flink.streaming.api.functions.sink.filesystem.Buckets.onProcessingTime(Buckets.java:338)
at 
org.apache.flink.connector.file.table.stream.AbstractStreamingWriter.endInput(AbstractStreamingWriter.java:155)
at 
org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.endOperatorInput(StreamOperatorWrapper.java:96)
at 
org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.lambda$finish$0(StreamOperatorWrapper.java:149)
at 
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:93)
at 
org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.finish(StreamOperatorWrapper.java:149)
at 
org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.finish(StreamOperatorWrapper.java:156)
at 
org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.finishOperators(RegularOperatorChain.java:115)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.endData(StreamTask.java:619)
at 
org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.lambda$completeProcessing$0(SourceStreamTask.java:367)
at 
org.apache.flink.util.function.FunctionUtils.lambda$asCallable$5(FunctionUtils.java:126)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:93)
at 
org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90)
at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMail(MailboxProcessor.java:398)
at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMailsWhenDefaultActionUnavailable(MailboxProcessor.java:367)
at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:352)
at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:229)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:858)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:807)
at 
org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:953)
at 
org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:932)
at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:746)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:562)
at 

[ANNOUNCE] Starting with Flink 1.19 Release Sync

2023-11-13 Thread Lincoln Lee
Hi everyone,

With the official release of 1.18, we're moving on to the next release
cycle,

I'm happy to share with you that the first release sync meeting of 1.19
will

happen tomorrow on Tuesday, Nov 14th at 9am (UTC+1) / 4pm (UTC+8).

At first, it will be every 2 weeks and when the feature freeze gets closer,
we'll move to weekly meetings.

Welcome and feel free to join us and share your ideas about the new release
cycle!

Details of joining the release sync can be found in the 1.19 release wiki
page[1].

All contributors are invited to update the same wiki page[1] and include
features targeting the 1.19 release.

Looking forward to seeing you all in the meeting!

[1] https://cwiki.apache.org/confluence/display/FLINK/1.19+Release

Best regards,
Martijn, Jing, Yun and Lincoln


Re: [VOTE] Release flink-connector-gcp-pubsub v3.0.2, release candidate #1

2023-11-13 Thread Danny Cranmer
This vote is now closed due to the Flink version error in the pom reported
by Leonard.

Thanks all.

On Thu, Nov 9, 2023 at 3:19 PM Martijn Visser 
wrote:

> I agree with Leonard. We should also not update 1.16.0 to 1.18.0, but
> to the lowest supported Flink version of this release (in this case,
> 1.17.0)
>
> On Thu, Nov 9, 2023 at 3:08 AM Leonard Xu  wrote:
> >
> > Thanks Danny for the reply.
> >
> > -1 (binding)
> >
> > Let’s fix the outdated version in the source code and spin a new rc2.
> >
> > I’d like to open a PR to fix it, and hope everything OK in your Flink
> Forward Trip.
> >
> > Best,
> > Leonard
> >
> >
> >
> > > Thanks for helping to verify the release. The 1.16.0 Flink version in
> the
> > > pom is a miss, ideally it should be updated to 1.18.0, additionally
> this
> > > should have been updated to 1.17.x previously. I would not consider it
> a
> > > hard blocker since the Maven build overrides this variable based on the
> > > provided -Dflink.version and the 1.17/1.18 binaries are valid. However
> it
> > > is non ideal that the default version in the source is outdated. Given
> that
> > > we are yet to receive any binding votes I am happy to spin an rc2
> however
> > > am a bit busy this week at Flink Forward. I will consider this vote
> > > open for now unless you make your -1 binding.
> > >
> > > Thanks,
> > > Danny
> > >
> > > On Tue, Nov 7, 2023 at 8:02 PM Leonard Xu  wrote:
> > >
> > >> Thanks Danny for driving this.  I'm considering -1, please correct me
> if I
> > >> understand wrong.
> > >>
> >  * The sources can be compiled and unit tests pass with flink.version
> > >> 1.17.1
> >  and flink.version 1.18.0
> > 
> >  * Nexus has two staged artifact ids for 3.0.2-1.17 and 3.0.2-1.18
> >  - flink-connector-gcp-pubsub (.jar, -javadoc.jar, -sources.jar and
> .pom)
> >  - flink-connector-gcp-pubsub-parent (only .pom)
> > >>
> > >>
> > >> This release aims to support Flink 1.17 and new released Flink
> 1.18,but
> > >> why is the version in pom file [1] still 1.16.0 ?  IIUC, it should be
> > >> 1.17.0  according the process [2].
> > >>
> > >> Best,
> > >> Leonard
> > >>
> > >> [1]
> > >>
> https://github.com/apache/flink-connector-gcp-pubsub/blob/v3.0.2-rc1/pom.xml#L51
> > >> [1]
> > >>
> https://cwiki.apache.org/confluence/display/FLINK/Externalized+Connector+development
> > >>
> > >>
> > >>> 2023年11月7日 下午12:03,Samrat Deb  写道:
> > >>>
> > >>> +1(non-binding)
> > >>>
> > >>> - Checked release notes
> > >>> - Verified checksums and signatures
> > >>> - Verified no binaries in release
> > >>> - Build connector from source
> > >>>
> > >>> Bests,
> > >>> Samrat
> > >>>
> > >>> On Mon, 6 Nov 2023 at 8:20 PM, Ryan Skraba
>  > >>>
> > >>> wrote:
> > >>>
> >  Hello! +1 (non-binding)
> > 
> >  One note: the parent pom still has 1.16.0 for the Maven property of
> >  flink.version for both 1.17 and 1.18 releases.
> > 
> >  I've validated the source for the RC1:
> >  flink-connector-gcp-pubsub-3.0.2-src.tgz at r65060
> >  * The sha512 checksum is OK.
> >  * The source file is signed correctly.
> >  * The signature 0F79F2AFB2351BC29678544591F9C1EC125FD8DB is found
> in the
> >  KEYS file, and on https://keyserver.ubuntu.com/
> >  * The source file is consistent with the GitHub tag v3.0.2-rc1,
> which
> >  corresponds to commit 4c6be836e6c0f36ef5711f12d7b935254e7d248d
> >  - The files explicitly excluded by create_pristine_sources (such as
> >  .gitignore and the submodule tools/releasing/shared) are not
> present.
> >  * Has a LICENSE file and a NOTICE file
> >  * Does not contain any compiled binaries.
> > 
> > 
> > 
> >  I did a simple smoke test on an emulated Pub/Sub with the 1.18
> version.
> > 
> >  All my best, Ryan Skraba
> > 
> > >>
> > >>
> >
>


Re: Slack invite expired, please share an active invite

2023-11-13 Thread Martijn Visser
Hi all,

Thanks for reporting, the Slack invite link is now also updated on the website.

Best regards,

Martijn

On Mon, Nov 13, 2023 at 12:20 PM  wrote:
>
> Thanks Yun, the link worked.
>
> Best,
> Chinmay
>
> > On Nov 13, 2023, at 4:46 PM, Yun Tang  wrote:
> >
> > Hi Chinmay,
> >
> > You can try this link 
> > https://join.slack.com/t/apache-flink/shared_invite/zt-1ta0su2np-lCCV6xD7XeKjwQuHMOTBIA
> >
> > And the community will fix the expired invitation soon.
> >
> > Best
> > Yun Tang
> > 
> > From: bhatchin...@icloud.com.INVALID 
> > Sent: Monday, November 13, 2023 19:01
> > To: dev@flink.apache.org 
> > Subject: Slack invite expired, please share an active invite
> >
> > Hi,
> >
> > I want to join the Flink Slack community, but the invite in the 
> > documentation  has 
> > expired.
> > Can any existing member share an active invite? :)
> >
> > Thanks,
> > Chinmay
> >
>


Re: Slack invite expired, please share an active invite

2023-11-13 Thread bhatchinmay
Thanks Yun, the link worked.

Best,
Chinmay

> On Nov 13, 2023, at 4:46 PM, Yun Tang  wrote:
> 
> Hi Chinmay,
> 
> You can try this link 
> https://join.slack.com/t/apache-flink/shared_invite/zt-1ta0su2np-lCCV6xD7XeKjwQuHMOTBIA
> 
> And the community will fix the expired invitation soon.
> 
> Best
> Yun Tang
> 
> From: bhatchin...@icloud.com.INVALID 
> Sent: Monday, November 13, 2023 19:01
> To: dev@flink.apache.org 
> Subject: Slack invite expired, please share an active invite
> 
> Hi,
> 
> I want to join the Flink Slack community, but the invite in the documentation 
>  has expired.
> Can any existing member share an active invite? :)
> 
> Thanks,
> Chinmay
> 



Re: Slack invite expired, please share an active invite

2023-11-13 Thread Yun Tang
Hi Chinmay,

You can try this link 
https://join.slack.com/t/apache-flink/shared_invite/zt-1ta0su2np-lCCV6xD7XeKjwQuHMOTBIA

And the community will fix the expired invitation soon.

Best
Yun Tang

From: bhatchin...@icloud.com.INVALID 
Sent: Monday, November 13, 2023 19:01
To: dev@flink.apache.org 
Subject: Slack invite expired, please share an active invite

Hi,

I want to join the Flink Slack community, but the invite in the documentation 
 has expired.
Can any existing member share an active invite? :)

Thanks,
Chinmay



Slack invite expired, please share an active invite

2023-11-13 Thread bhatchinmay
Hi,

I want to join the Flink Slack community, but the invite in the documentation 
 has expired.
Can any existing member share an active invite? :)

Thanks,
Chinmay



Re: [DISCUSS] FLIP-364: Improve the restart-strategy

2023-11-13 Thread Zhu Zhu
Hi Rui,
Thanks for creating this FLIP and sorry for jumping in so late into the
discussion.

The improvements of exponential-delay strategy and making it the default
strategy looks good it me in general. I have some comments for it, as well
as for the failure counting.

1. default values of exponential-delay configuration
It is mentioned in the FLIP that "the default value of these options are
not changed, we just change them for default restart-strategy. The default
restart-strategy just takes effect if checkpointing is enabled and the
user doesn’t configure the restart-strategy". I'm a bit concerned with
the inconsistency which may cause confusion to users. So if the proposed
configuration values work better in most cases, I'm leaning towards to
change the default values.

2. the default tolerable failure count
Currently, the restart-strategy behaves like this by default(if not
configured):
1. job retries on failures indefinitely if checkpointing is enabled
2. job fails on any failure if checkpointing is disabled
Changing it to always restart indefinitely may result in unexpected
behaviors in production. Therefore, either we should do the same
thing to exponential-delay, or have an open discussion(also involving
the user ML) on changing this default behavior.

3. failure counting
Flink currently will try to recognize concurrent failures and group them
together, which can be seen in the web UI. So how about to align the
failure counting with the concurrent failures computing? This can make it
more consistent and easier for understanding. It will require changes to
the concurrent failures computing though, i.e. taking the backoff time
into consideration. So maybe we can open a seperate FLIP for this change.

Thanks,
Zhu

Rui Fan <1996fan...@gmail.com> 于2023年11月10日周五 18:22写道:

> I'll start voting next Monday if there isn't any other comment.
>
> Best,
> Rui
>
> On Thu, Oct 19, 2023 at 6:59 PM Rui Fan <1996fan...@gmail.com> wrote:
>
> > Hi Konstantin and Max,
> >
> > Thanks for your feedback!
> >
> > Sorry, I forgot to mention the default value of
> > `restart-strategy.exponential-delay.max-attempts-before-reset-backoff`.
> >
> > Retrying forever sounds good to me, I have added it to the FLIP:
> >
> > The default value of
> > `restart-strategy.exponential-delay.max-attempts-before-reset-backoff` is
> > Integer.MAX_VALUE.
> >
> > Best,
> > Rui
> >
> > On Thu, Oct 19, 2023 at 6:29 PM Maximilian Michels 
> wrote:
> >
> >> Hey Rui,
> >>
> >> +1 for making exponential backoff the default. I agree with Konstantin
> >> that retrying forever is a good default for exponential backoff
> >> because oftentimes the issue will resolve eventually. The purpose of
> >> exponential backoff is precisely to continue to retry without causing
> >> too much load. However, I'm not against adding an optional max number
> >> of retries.
> >>
> >> -Max
> >>
> >> On Thu, Oct 19, 2023 at 11:35 AM Konstantin Knauf 
> >> wrote:
> >> >
> >> > Hi Rui,
> >> >
> >> > Thank you for this proposal and working on this. I also agree that
> >> > exponential back off makes sense as a new default in general. I think
> >> > restarting indefinitely (no max attempts) makes sense by default,
> >> though,
> >> > but of course allowing users to change is valuable.
> >> >
> >> > So, overall +1.
> >> >
> >> > Cheers,
> >> >
> >> > Konstantin
> >> >
> >> > Am Di., 17. Okt. 2023 um 07:11 Uhr schrieb Rui Fan <
> >> 1996fan...@gmail.com>:
> >> >
> >> > > Hi all,
> >> > >
> >> > > I would like to start a discussion on FLIP-364: Improve the
> >> > > restart-strategy[1]
> >> > >
> >> > > As we know, the restart-strategy is critical for flink jobs, it
> mainly
> >> > > has two functions:
> >> > > 1. When an exception occurs in the flink job, quickly restart the
> job
> >> > > so that the job can return to the running state.
> >> > > 2. When a job cannot be recovered after frequent restarts within
> >> > > a certain period of time, Flink will not retry but will fail the
> job.
> >> > >
> >> > > The current restart-strategy support for function 2 has some issues:
> >> > > 1. The exponential-delay doesn't have the max attempts mechanism,
> >> > > it means that flink will restart indefinitely even if it fails
> >> frequently.
> >> > > 2. For multi-region streaming jobs and all batch jobs, the failure
> of
> >> > > each region will increase the total number of job failures by +1,
> >> > > even if these failures occur at the same time. If the number of
> >> > > failures increases too quickly, it will be difficult to set a
> >> reasonable
> >> > > number of retries.
> >> > > If the maximum number of failures is set too low, the job can easily
> >> > > reach the retry limit, causing the job to fail. If set too high,
> some
> >> jobs
> >> > > will never fail.
> >> > >
> >> > > In addition, when the above two problems are solved, we can also
> >> > > discuss whether exponential-delay can replace fixed-delay as the
> >> > > default restart-strategy. In theory, 

[jira] [Created] (FLINK-33535) Support autoscaler for session jobs

2023-11-13 Thread Gyula Fora (Jira)
Gyula Fora created FLINK-33535:
--

 Summary: Support autoscaler for session jobs
 Key: FLINK-33535
 URL: https://issues.apache.org/jira/browse/FLINK-33535
 Project: Flink
  Issue Type: New Feature
  Components: Autoscaler, Kubernetes Operator
Reporter: Gyula Fora


Currently the operator autoscaler is not enabled for session jobs. There are 
some issues around the actual scaling of the jobs, such as: 

https://issues.apache.org/jira/browse/FLINK-33534

Furthermore there is a bug which in any case prevents the submission of these 
jobs due to how the jobid is generated, causing collisions if we only change 
the parallelism override without the metadata generation.

We could consider still enabling the autoscaler for a limited 1.18 rescale api 
support



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[VOTE] Release 1.16.3, release candidate #1

2023-11-13 Thread Rui Fan
Hi everyone,

Please review and vote on the release candidate #1 for the version 1.16.3,

as follows:

[ ] +1, Approve the release

[ ] -1, Do not approve the release (please provide specific comments)


The complete staging area is available for your review, which includes:
* JIRA release notes [1],

* the official Apache source release and binary convenience releases to be
deployed to dist.apache.org [2], which are signed with the key with
fingerprint B2D64016B940A7E0B9B72E0D7D0528B28037D8BC [3],

* all artifacts to be deployed to the Maven Central Repository [4],

* source code tag "release-1.16.3-rc1" [5],

* website pull request listing the new release and adding announcement blog
post [6].


The vote will be open for at least 72 hours. It is adopted by majority
approval, with at least 3 PMC affirmative votes.


[1]
https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12353259

[2] https://dist.apache.org/repos/dist/dev/flink/flink-1.16.3-rc1/

[3] https://dist.apache.org/repos/dist/release/flink/KEYS

[4] https://repository.apache.org/content/repositories/orgapacheflink-1670/

[5] https://github.com/apache/flink/releases/tag/release-1.16.3-rc1

[6] https://github.com/apache/flink-web/pull/698

Thanks,
Release Manager


Re: [VOTE] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects

2023-11-13 Thread Hang Ruan
+1(non-binding)

Best,
Hang

Jing Ge  于2023年11月13日周一 16:36写道:

> +1(binding)
> Thanks!
>
> Best Regards,
> Jing
>
> On Mon, Nov 13, 2023 at 8:34 AM Zhu Zhu  wrote:
>
> > +1 (binding)
> >
> > Thanks,
> > Zhu
> >
> > Xia Sun  于2023年11月13日周一 15:02写道:
> >
> > > +1 (non-binding)
> > >
> > > Best,
> > > Xia
> > >
> > > Samrat Deb  于2023年11月13日周一 12:37写道:
> > >
> > > > +1 (non binding)
> > > >
> > > > Bests,
> > > > Samrat
> > > >
> > > > On Mon, 13 Nov 2023 at 9:10 AM, Yangze Guo 
> wrote:
> > > >
> > > > > +1 (binding)
> > > > >
> > > > > Best,
> > > > > Yangze Guo
> > > > >
> > > > > On Mon, Nov 13, 2023 at 11:35 AM weijie guo <
> > guoweijieres...@gmail.com
> > > >
> > > > > wrote:
> > > > > >
> > > > > > +1(binding)
> > > > > >
> > > > > > Best regards,
> > > > > >
> > > > > > Weijie
> > > > > >
> > > > > >
> > > > > > Lijie Wang  于2023年11月13日周一 10:40写道:
> > > > > >
> > > > > > > +1 (binding)
> > > > > > >
> > > > > > > Best,
> > > > > > > Lijie
> > > > > > >
> > > > > > > Yuepeng Pan  于2023年11月10日周五 18:32写道:
> > > > > > >
> > > > > > > > +1(non-binding)
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Roc
> > > > > > > >
> > > > > > > > On 2023/11/10 03:58:10 Junrui Lee wrote:
> > > > > > > > > Hi everyone,
> > > > > > > > >
> > > > > > > > > Thank you to everyone for the feedback on FLIP-381:
> Deprecate
> > > > > > > > configuration
> > > > > > > > > getters/setters that return/set complex Java objects[1]
> which
> > > has
> > > > > been
> > > > > > > > > discussed in this thread [2].
> > > > > > > > >
> > > > > > > > > I would like to start a vote for it. The vote will be open
> > for
> > > at
> > > > > least
> > > > > > > > 72
> > > > > > > > > hours (excluding weekends) unless there is an objection or
> > not
> > > > > enough
> > > > > > > > votes.
> > > > > > > > >
> > > > > > > > > [1]
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992
> > > > > > > > > [2]
> > > > > https://lists.apache.org/thread/y5owjkfxq3xs9lmpdbl6d6jmqdgbjqxo
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
>


Re: [VOTE] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects

2023-11-13 Thread Jing Ge
+1(binding)
Thanks!

Best Regards,
Jing

On Mon, Nov 13, 2023 at 8:34 AM Zhu Zhu  wrote:

> +1 (binding)
>
> Thanks,
> Zhu
>
> Xia Sun  于2023年11月13日周一 15:02写道:
>
> > +1 (non-binding)
> >
> > Best,
> > Xia
> >
> > Samrat Deb  于2023年11月13日周一 12:37写道:
> >
> > > +1 (non binding)
> > >
> > > Bests,
> > > Samrat
> > >
> > > On Mon, 13 Nov 2023 at 9:10 AM, Yangze Guo  wrote:
> > >
> > > > +1 (binding)
> > > >
> > > > Best,
> > > > Yangze Guo
> > > >
> > > > On Mon, Nov 13, 2023 at 11:35 AM weijie guo <
> guoweijieres...@gmail.com
> > >
> > > > wrote:
> > > > >
> > > > > +1(binding)
> > > > >
> > > > > Best regards,
> > > > >
> > > > > Weijie
> > > > >
> > > > >
> > > > > Lijie Wang  于2023年11月13日周一 10:40写道:
> > > > >
> > > > > > +1 (binding)
> > > > > >
> > > > > > Best,
> > > > > > Lijie
> > > > > >
> > > > > > Yuepeng Pan  于2023年11月10日周五 18:32写道:
> > > > > >
> > > > > > > +1(non-binding)
> > > > > > >
> > > > > > > Best,
> > > > > > > Roc
> > > > > > >
> > > > > > > On 2023/11/10 03:58:10 Junrui Lee wrote:
> > > > > > > > Hi everyone,
> > > > > > > >
> > > > > > > > Thank you to everyone for the feedback on FLIP-381: Deprecate
> > > > > > > configuration
> > > > > > > > getters/setters that return/set complex Java objects[1] which
> > has
> > > > been
> > > > > > > > discussed in this thread [2].
> > > > > > > >
> > > > > > > > I would like to start a vote for it. The vote will be open
> for
> > at
> > > > least
> > > > > > > 72
> > > > > > > > hours (excluding weekends) unless there is an objection or
> not
> > > > enough
> > > > > > > votes.
> > > > > > > >
> > > > > > > > [1]
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992
> > > > > > > > [2]
> > > > https://lists.apache.org/thread/y5owjkfxq3xs9lmpdbl6d6jmqdgbjqxo
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >
>


Re: [VOTE] Release flink-shaded 16.2, release candidate #1

2023-11-13 Thread Yuxin Tan
Thanks weijie for driving the new release!

+1 (non-binding)

- Built from source code succeeded
- Verified signatures
- Verified hashsums
- Reviewed the web PR

Best,
Yuxin


weijie guo  于2023年11月13日周一 15:57写道:

> Hi everyone,
>
> Please review and vote on the release candidate #1 for the version
> 16.2, as follows:
>
> [ ] +1, Approve the release
>
> [ ] -1, Do not approve the release (please provide specific comments)
>
>
>
>
>
> The complete staging area is available for your review, which includes:
>
> * JIRA release notes [1],
>
> * the official Apache source release to be deployed to dist.apache.org
>  [2], which are signed with the key with fingerprint
> 8D56AE6E7082699A4870750EA4E8C4C05EE6861F [3],
>
> * all artifacts to be deployed to the Maven Central Repository [4],
>
> * source code tag "release-16.2-rc1" [5],
>
> * website pull request listing the new release [6].
>
>
>
> The vote will be open for at least 72 hours. It is adopted by majority
> approval, with at least 3 PMC affirmative votes.
>
>
>
> Thanks,
>
> Release Manager
>
>
>
> [1] https://issues.apache.org/jira/projects/FLINK/versions/12353810
>
> [2] https://dist.apache.org/repos/dist/dev/flink/flink-shaded-16.2-rc1
>
> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
>
> [4]
> https://repository.apache.org/content/repositories/orgapacheflink-1671/
>
> [5] https://github.com/apache/flink-shaded/releases/tag/release-16.2-rc1
>
> [6] https://github.com/apache/flink-web/pull/697
>


[VOTE] Release 1.17.2, release candidate #1

2023-11-13 Thread Yun Tang
Hi everyone,

Please review and vote on the release candidate #1 for the version 1.17.2,

as follows:

[ ] +1, Approve the release

[ ] -1, Do not approve the release (please provide specific comments)


The complete staging area is available for your review, which includes:
* JIRA release notes [1],

* the official Apache source release and binary convenience releases to be 
deployed to dist.apache.org [2], which are signed with the key with fingerprint 
2E0E1AB5D39D55E608071FB9F795C02A4D2482B3 [3],

* all artifacts to be deployed to the Maven Central Repository [4],

* source code tag "release-1.17.2-rc1" [5],

* website pull request listing the new release and adding announcement blog 
post [6].


The vote will be open for at least 72 hours. It is adopted by majority 
approval, with at least 3 PMC affirmative votes.


[1] 
https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12353260

[2] https://dist.apache.org/repos/dist/dev/flink/flink-1.17.2-rc1/

[3] https://dist.apache.org/repos/dist/release/flink/KEYS

[4] https://repository.apache.org/content/repositories/orgapacheflink-1669/

[5] https://github.com/apache/flink/releases/tag/release-1.17.2-rc1

[6] https://github.com/apache/flink-web/pull/696

Thanks,
Release Manager