> context.getOperatorStateStore().getUnionListState(stateDescriptor);
> > > }
> > > }
> > >
> > > @Override
> > > public void run(SourceContext ctx) throws Exception {
> > > while (running) {
> > >
eDescriptor);
> > }
> > }
> >
> > @Override
> > public void run(SourceContext ctx) throws Exception {
> > while (running) {
> > synchronized (ctx.getCheckpointLock()) {
> > ctx.collect(ThreadLocalRandom.current().nextInt());
&g
gt; @Override
> public void cancel() {
> running = false;
> }
>}
> }
>
> Best
> Yun Tang
>
> From: Stephan Ewen
> Sent: Saturday, May 16, 2020 18:56
> To: dev
> Cc: Till Rohrmann ; Piotr Nowojs
ng = false;
}
}
}
Best
Yun Tang
From: Stephan Ewen
Sent: Saturday, May 16, 2020 18:56
To: dev
Cc: Till Rohrmann ; Piotr Nowojski
Subject: Re: [DISCUSS] increase "state.backend.fs.memory-threshold" from 1K to
100K
Okay, thank you for all the feedback.
So we should defin
;
> > [1]
> >
> https://github.com/apache/flink/blob/32bd0944d0519093c0a4d5d809c6636eb3a7fc31/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java#L752
> >
> > Best
> > Yun Tang
> >
> > ________
> > From: Stephan Ewen
could be a trade-off, which benefit perhaps
> > 99% users, but might bring harmful effects to 1% user with large-scale
> > flink jobs.
> >
> >
> > [1]
> >
> https://github.com/apache/flink/blob/c1ea6fcfd05c72a68739bda8bd16a2d1c15522c0/flink-runtime/src/main/java/org/apache/flink/runtime/checkp
t
Yun Tang
From: Stephan Ewen
Sent: Friday, May 15, 2020 16:53
To: dev
Cc: Till Rohrmann ; Piotr Nowojski
Subject: Re: [DISCUSS] increase "state.backend.fs.memory-threshold" from 1K to
100K
I see, thanks for all the input.
I agree with Yun Tang that t
t;
> From: Yu Li
> Sent: Thursday, May 14, 2020 23:51
> To: Till Rohrmann
> Cc: dev ; Piotr Nowojski
> Subject: Re: [DISCUSS] increase "state.backend.fs.memory-threshold" from
> 1K to 100K
>
> TL;DR: I have some reservations but tend
emory-threshold" from 1K to
100K
TL;DR: I have some reservations but tend to be +1 for the proposal,
meanwhile suggest we have a more thorough solution in the long run.
Please correct me if I'm wrong, but it seems the root cause of the issue is
too many small files generated.
I have some c
TL;DR: I have some reservations but tend to be +1 for the proposal,
meanwhile suggest we have a more thorough solution in the long run.
Please correct me if I'm wrong, but it seems the root cause of the issue is
too many small files generated.
I have some concerns for the case of session cluster
I cannot say much about the concrete value but if our users have problems
with the existing default values, then it makes sense to me to change it.
One thing to check could be whether it is possible to provide a meaningful
exception in case that the state size exceeds the frame size. At the
moment
The parameter "state.backend.fs.memory-threshold" decides when a state will
become a file and when it will be stored inline with the metadata (to avoid
excessive amounts of small files).
By default, this threshold is 1K - so every state above that size becomes a
file. For many cases, this threshol
12 matches
Mail list logo