Roman Khachatryan created FLINK-38463:
-
Summary: Support TTL in SinkUpsertMaterializerV2
Key: FLINK-38463
URL: https://issues.apache.org/jira/browse/FLINK-38463
Project: Flink
Issue Type
+1 (binding)
Thanks for driving this!
Regards,
Roman
On Tue, Sep 23, 2025, 08:06 Yuepeng Pan wrote:
> +1 (non-binding)
>
> Best regards,
> Yuepeng Pan
>
> Gabor Somogyi 于2025年9月23日周二 12:46写道:
>
> > +1 (binding)
> >
> > G
> >
> >
> > On Tue, Sep 23, 2025 at 6:26 AM Rui Fan <1996fan...@gmail.co
gt; > > > >
> > > > > > +1(binding)
> > > > > >
> > > > > > Best,
> > > > > > Lincoln Lee
> > > > > >
> > > > > > Efrat Levitan efrat890...@gmail.com 于2025年9月16日周二 00:55写道:
> >
Roman Khachatryan created FLINK-38376:
-
Summary: AsyncStateStreamingJoinOperator should not OOM due to
collecting all matching records in memory
Key: FLINK-38376
URL: https://issues.apache.org/jira/browse
tly internally without
> considering the
> compatibility of configuration options.
>
> I don't have a strong opinion on this, if it's really necessary to retain
> flexibility.
>
> Best,
> Rui
>
> On Thu, Sep 11, 2025 at 11:02 PM Roman Khachatryan
> wrote:
Hi everyone,
I'd like to start a vote on FLIP-544 SinkUpsertMaterializer V2
[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/display/FLINK/FLIP-544%3A+SinkUpsertM
data expansion will help ease performance
> bottlenecks when this operator can’t be fully avoided.
>
> Thanks for pushing this forward!
>
> Best,
> Lincoln Lee
>
>
> Roman Khachatryan 于2025年9月12日周五 16:38写道:
>
> > Hey Rui,
> >
> > Thanks for clarifyi
on.
>
> In the first version, I think list size might be enough as an internal
> strategy,
> and we can dynamically choose different thresholds for heap and rocksdb.
>
> Looking forward to your feedback!
>
> Best,
> Rui
>
> On Mon, Sep 1, 2025 at 3:13 PM Piotr Nowojsk
Hi Rui, thanks for driving this!
This would be a very useful addition to the Unaligned Checkpoints.
I have no comments on the proposal as we already discussed it offline,
Looking forward to it being implemented and released!
Regards,
Roman
On Thu, Sep 11, 2025 at 3:52 PM Gabor Somogyi
wrote:
Roman Khachatryan created FLINK-38308:
-
Summary: Prevent excessive logging from parsing SQL
timestamps/dates
Key: FLINK-38308
URL: https://issues.apache.org/jira/browse/FLINK-38308
Project: Flink
like to open a discussion proposing the ability to enable
> > > > flamegraphs at runtime and make their configuration i.e number of
> > > samples,
> > > > delay between samples, and stack depth *dynamically adjustable via
> the
> > > Web
> > > >
live through specific
> number of records), or even be advanced by special control records for
> Datastream users. This kind of user-controlled time advancement is what I
> said "manually controllable". Such flexibility could be broadly beneficial.
>
> We’ve encountered cas
be optimized to single copy for a non-upsert-key
> scenario?
> 4. For the TTL mechanism part, I would suggest an 'event-time based ttl',
> which allows the user to specify insertion time for each insert/update
> operation and a manually controllable `TtlTimeProvider` (instead o
Hi everyone,
I would like to start a discussion about FLIP-544 SinkUpsertMaterializer V2
[1].
SinkUpsertMaterializer is an operator in Flink that reconciles out of order
changelog events before sending them to an upsert sink. In some cases (that
we see in our production), performance of this oper
Roman Khachatryan created FLINK-38199:
-
Summary: Add a benchmark for SinkUpsertMaterializer
Key: FLINK-38199
URL: https://issues.apache.org/jira/browse/FLINK-38199
Project: Flink
Issue
Roman Khachatryan created FLINK-37955:
-
Summary: StreamingJoinOperator should not OOM due to collecting
all matching records in memory
Key: FLINK-37955
URL: https://issues.apache.org/jira/browse/FLINK-37955
Roman Khachatryan created FLINK-37853:
-
Summary: FLIP-530: Dynamic job configuration
Key: FLINK-37853
URL: https://issues.apache.org/jira/browse/FLINK-37853
Project: Flink
Issue Type
t; Thanks,
> Gustavo
>
> On Wed, 21 May 2025 at 08:32, Kartikey Pant
> wrote:
>
> > Hi,
> >
> > +1 (non-binding)
> >
> > Thanks, Kartikey.
> >
> > On Wed, May 21, 2025 at 12:59 AM Roman Khachatryan
> > wrote:
> >
> > >
Hi everyone,
I'd like to start a vote on FLIP-530: Dynamic job configuration
[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/x/uglKFQ
[2]
https://lists.apache.o
Thanks everyone for the discussion!
I'm going to start a voting thread soon unless there are other suggestions
or objections.
Regards,
Roman
On Sat, May 17, 2025 at 2:01 PM Roman Khachatryan wrote:
> Thanks Chesnay, I like your idea of returning 403 for non-white-listed
> optio
ge and resource requirements
> >> change
> >> could trigger a rescale for Adaptive Scheduler. So rescale history can
> >> probably
> >> include both. If we want to show the configuration change history, it
> might
> >> be
> >> more appropriate
configuration changes don't
> respect the delay mechanism?
>
> Please correct me if anything is wrong, thanks!
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-487%3A+Show+history+of+rescales+in+Web+UI+for+AdaptiveScheduler
> [2]
>
> https://cwiki
ils
> could be tracked.
>
> 3. How does it co-works with other dynamic requests ? For example, it
> modifies the parallelisms together with '
> /jobs/:jobid/resource-requirements'.
>
> On Fri, May 9, 2025 at 5:00 AM Roman Khachatryan wrote:
>
> > Hi everyone,
Hi everyone,
I would like to start a discussion about FLIP-530: Dynamic job
configuration [1].
In some cases, it is desirable to change Flink job configuration after it
was submitted to Flink, for example:
- Troubleshooting (e.g. increase checkpoint timeout or failure threshold)
- Performance opt
+1 (binding)
Regards,
Roman
On Thu, Mar 27, 2025 at 1:18 AM Efrat Levitan wrote:
> +1 (non-binding)
>
> Thanks for addressing this issue!
> Efrat
>
>
> On Wed, 26 Mar 2025 at 17:35 Arvid Heise wrote:
>
> > Dear devs,
> >
> > I'd like to start the voting on FLIP-511 [1].
> > Don't hesitate to
+1 (binding)
Regards,
Roman
On Mon, Mar 17, 2025 at 11:06 AM Efrat Levitan
wrote:
> Hi everyone,
>
> I'd like to start a vote on FLIP-513: Split-level watermark metrics [1]
> which has been discussed in this thread [2].
> The vote will be open for at least 72 hours unless there is an objection
+1 (binding)
Regards,
Roman
On Fri, Dec 6, 2024 at 10:19 AM Piotr Nowojski wrote:
> Hi all!
>
> I would like to open the vote for FLIP-481 [1]. It has been discussed here
> [2].
>
> The vote will remain open for at least 72 hours (excluding the weekend).
>
> Best,
> Piotrek
>
> [1] https://cwi
+1 (binding)
Regards,
Roman
On Fri, Dec 6, 2024 at 10:21 AM Piotr Nowojski wrote:
> Hi all!
>
> I would like to open the vote for FLIP-483 [1]. It has been discussed here
> [2].
>
> The vote will remain open for at least 72 hours (excluding the weekend).
>
> Best,
> Piotrek
>
> [1] https://cwi
+1 (binding)
Regards,
Roman
On Fri, Dec 6, 2024 at 10:21 AM Piotr Nowojski wrote:
> Hi all!
>
> I would like to open the vote for FLIP-484 [1]. It has been discussed here
> [2].
>
> The vote will remain open for at least 72 hours (excluding the weekend).
>
> Best,
> Piotrek
>
> [1] https://cwi
+1 (binding)
Regards,
Roman
On Fri, Dec 6, 2024 at 10:18 AM Piotr Nowojski wrote:
> Hi all!
>
> I would like to open the vote for FLIP-482 [1]. It has been discussed here
> [2].
>
> The vote will remain open for at least 72 hours (excluding the weekend).
>
> Best,
> Piotrek
>
> [1] https://cwi
. Also AFAIR it's not clear how to convert a custom
> string into Otel's `Context` class . But I might be wrong with this last
> one.
>
> Best,
> Piotrek
>
>
> czw., 14 lis 2024 o 13:21 Roman Khachatryan napisał(a):
>
> > Hi Piotr, thanks for the proposal
;s used by Flink itself; or is very common.
>
> This introduced to fill in Otel's Body field
> `io.opentelemetry.api.logs.LogRecordBuilder#setBody`
>
> Best,
> Piotrek
>
> czw., 14 lis 2024 o 11:03 Roman Khachatryan napisał(a):
>
> > Hi Piotr, thanks for the pro
+1,
Thanks for volunteering!
Regards,
Roman
On Mon, Nov 25, 2024 at 5:35 AM Zakelly Lan wrote:
> +1 for this
>
> Thanks for driving!
>
> Best,
> Zakelly
>
> On Mon, Nov 25, 2024 at 12:15 PM weijie guo
> wrote:
>
> > Thanks for driving this!
> >
> > +1 for this release and rm.
> >
> > Best re
Roman Khachatryan created FLINK-36733:
-
Summary: Don't transition task to RUNNING until the inputs are
recovered (UC)
Key: FLINK-36733
URL: https://issues.apache.org/jira/browse/FLINK-
Hi Piotr,
Adding OTel implementation makes sense, +1 for the proposal.
Thanks
Regards,
Roman
On Thu, Nov 7, 2024 at 2:37 PM Piotr Nowojski wrote:
> Hi all!
>
> I would like to open up for discussion a new FLIP-482 [1].
>
> Motivation
> FLIP-481 [2] is adding the EventReporter interface. Howe
Hi Piotr, thanks for the proposal,
Can you please clarify
1. The scope of the variables added - is it only the last transformation?
Do I understand correctly, that chaining does NOT affect this scoping?
2. Is Python API going to be supported as well?
Thanks
Regards,
Roman
On Thu, Nov 7, 2024
Hi Piotr, thanks for the proposal,
I see the need for reporting child spans, however I have a couple of
questions about the proposed design:
1. Why do we give up on the idea of reporting child spans independently
from the parent? I couldn't find much details in the Rejected Alternatives
section
Roman Khachatryan created FLINK-36714:
-
Summary: sstmerge/CompactionScheduler termination can be
interrupted
Key: FLINK-36714
URL: https://issues.apache.org/jira/browse/FLINK-36714
Project: Flink
Hi Piotr, thanks for the proposal!
I think this would be a very valuable addition to Flink as it would
simplify operations a lot
(disclaimer: we already use it in our internal Flink version)
I have a couple of remarks regarding the FLIP:
1. Should it list the events that would be emitted in the
Roman Khachatryan created FLINK-36710:
-
Summary: JobIDLoggingITCase fails on SourceCoordinator logs
Key: FLINK-36710
URL: https://issues.apache.org/jira/browse/FLINK-36710
Project: Flink
Roman Khachatryan created FLINK-36623:
-
Summary: Improve logging in DefaultStateTransitionManager
Key: FLINK-36623
URL: https://issues.apache.org/jira/browse/FLINK-36623
Project: Flink
Roman Khachatryan created FLINK-36030:
-
Summary: Don't use negative values for changelog recovery tests
Key: FLINK-36030
URL: https://issues.apache.org/jira/browse/FLINK-36030
Project:
Roman Khachatryan created FLINK-35970:
-
Summary: Update documentation about FLINK-26050 (small file
compaction)
Key: FLINK-35970
URL: https://issues.apache.org/jira/browse/FLINK-35970
Project
+1 (binding)
Regards,
Roman
On Wed, Jul 31, 2024 at 2:39 PM Rui Fan <1996fan...@gmail.com> wrote:
> +1(binding)
>
> Best,
> Rui
>
> Timo Walther 于2024年7月31日 周三17:47写道:
>
> > +1 (binding)
> >
> > Thanks for fixing this critical bug.
> >
> > Regards,
> > Timo
> >
> > On 31.07.24 09:51, Stefan Ric
Roman Khachatryan created FLINK-35933:
-
Summary: Skip distributing maxAllowedWatermark if there are no
subtasks
Key: FLINK-35933
URL: https://issues.apache.org/jira/browse/FLINK-35933
Project
Roman Khachatryan created FLINK-35787:
-
Summary: DefaultSlotStatusSyncer might bring down JVM (exit code
239 instead of a proper shutdown)
Key: FLINK-35787
URL: https://issues.apache.org/jira/browse/FLINK
Roman Khachatryan created FLINK-35786:
-
Summary: NPE in BlobServer / shutdownHook
Key: FLINK-35786
URL: https://issues.apache.org/jira/browse/FLINK-35786
Project: Flink
Issue Type: Bug
Roman Khachatryan created FLINK-35769:
-
Summary: State files might not be deleted on task cancellation
Key: FLINK-35769
URL: https://issues.apache.org/jira/browse/FLINK-35769
Project: Flink
Roman Khachatryan created FLINK-35742:
-
Summary: Don't create RocksDB CF if task cancellation is in
progress
Key: FLINK-35742
URL: https://issues.apache.org/jira/browse/FLINK-35742
Project:
+1 (binding)
Thanks for pushing this and updating the FLIP
Regards,
Roman
On Wed, Jun 26, 2024 at 9:27 AM Piotr Nowojski wrote:
> Thanks for pointing this out Zakelly. After the discussion on the dev
> mailing list, I have updated the `PathsCopyingFileSystem` to merge its
> functionalities wi
Roman Khachatryan created FLINK-35557:
-
Summary: MemoryManager only reserves memory per consumer type once
Key: FLINK-35557
URL: https://issues.apache.org/jira/browse/FLINK-35557
Project: Flink
Roman Khachatryan created FLINK-35556:
-
Summary: Wrong constant in
RocksDBSharedResourcesFactory.SLOT_SHARED_MANAGED
Key: FLINK-35556
URL: https://issues.apache.org/jira/browse/FLINK-35556
Roman Khachatryan created FLINK-35501:
-
Summary: Use common thread pools when transferring RocksDB state
files
Key: FLINK-35501
URL: https://issues.apache.org/jira/browse/FLINK-35501
Project
+1 (binding)
Regards,
Roman
On Mon, May 6, 2024 at 11:56 AM gongzhongqiang
wrote:
> +1 (non-binding)
>
> Best,
> Zhongqiang Gong
>
> yue ma 于2024年5月6日周一 10:54写道:
>
> > Hi everyone,
> >
> > Thanks for all the feedback, I'd like to start a vote on the FLIP-447:
> > Upgrade FRocksDB from 6.20.3
Hi Piotr,
+1 for the proposal, the recovery time improvements are significant IMO
Thanks for pushing this
Regards,
Roman
On Tue, Apr 30, 2024 at 3:15 PM Piotr Nowojski wrote:
> Hi all!
>
> I would like to put under discussion:
>
> FLIP-444: Native file copy support
> https://cwiki.apache.org
Thanks for the proposal, I definitely see the need for this improvement, +1.
Regards,
Roman
On Tue, Apr 30, 2024 at 3:11 PM Piotr Nowojski wrote:
> Hi Yanfei,
>
> Thanks for the feedback!
>
> > 1. Currently when AbstractStreamOperator or AbstractStreamOperatorV2
> > processes a watermark, the
Congrats, well deserved!
Regards,
Roman
On Thu, Apr 18, 2024 at 9:06 AM xiangyu feng wrote:
> Congratulations, Zakelly!
>
>
> Regards,
> Xiangyu Feng
>
> yh z 于2024年4月18日周四 14:27写道:
>
> > Congratulations Zakelly!
> >
> > Best regards,
> > Yunhong (swuferhong)
> >
> > gongzhongqiang 于2024年4月1
Hi,
Thanks for writing the proposal and preparing the upgrade.
FRocksDB definitely needs to be kept in sync with the upstream and the new
APIs are necessary for faster rescaling.
We're already using a similar version internally.
I reviewed the FLIP and it looks good to me (disclaimer: I took pa
Roman Khachatryan created FLINK-34994:
-
Summary: JobIDLoggingITCase fails because of "checkpoint
confirmation for unknown task"
Key: FLINK-34994
URL: https://issues.apache.org/jira/browse/F
+1 (binding)
Regards,
Roman
On Fri, Mar 29, 2024 at 8:08 AM yue ma wrote:
> +1 (non-binding)
>
> Yanfei Lei 于2024年3月27日周三 18:28写道:
>
> > Hi everyone,
> >
> > Thanks for all the feedback about the FLIP-425: Asynchronous Execution
> > Model [1]. The discussion thread is here [2].
> >
> > The vo
+1 (binding)
Regards,
Roman
On Fri, Mar 29, 2024 at 7:01 AM Xintong Song wrote:
> +1 (binding)
>
> Best,
>
> Xintong
>
>
>
> On Fri, Mar 29, 2024 at 12:51 PM Yuepeng Pan
> wrote:
>
> > +1(non-binding)
> >
> > Best,
> > Yuepeng Pan
> >
> >
> > On 2024/03/29 03:03:53 Yunfeng Zhou wrote:
> > > +
Roman Khachatryan created FLINK-34559:
-
Summary: TVF Window Aggregations might stuck
Key: FLINK-34559
URL: https://issues.apache.org/jira/browse/FLINK-34559
Project: Flink
Issue Type
Roman Khachatryan created FLINK-34420:
-
Summary: Various YARN tests fail after failing to download
hadoop.tar.gz
Key: FLINK-34420
URL: https://issues.apache.org/jira/browse/FLINK-34420
Project
Roman Khachatryan created FLINK-34417:
-
Summary: Add JobID to logging MDC
Key: FLINK-34417
URL: https://issues.apache.org/jira/browse/FLINK-34417
Project: Flink
Issue Type: Improvement
Roman Khachatryan created FLINK-34344:
-
Summary: Wrong JobID in CheckpointStatsTracker
Key: FLINK-34344
URL: https://issues.apache.org/jira/browse/FLINK-34344
Project: Flink
Issue Type
+1 (binding)
Regards,
Roman
On Wed, Nov 22, 2023 at 12:55 PM Rui Fan <1996fan...@gmail.com> wrote:
> +1(binding)
>
> Thanks for driving this proposal!
>
> Best,
> Rui
>
> On Wed, Nov 22, 2023 at 7:44 PM Piotr Nowojski
> wrote:
>
> > Hi All,
> >
> > I'd like to start a vote on the FLIP-386: Su
+1 (binding)
Regards,
Roman
On Wed, Nov 22, 2023, 7:30 AM Hangxiang Yu wrote:
> +1(binding)
>
> On Wed, Nov 22, 2023 at 10:29 AM Rui Fan <1996fan...@gmail.com> wrote:
>
> > +1(binding)
> >
> > Best,
> > Rui
> >
> > On Wed, Nov 22, 2023 at 1:20 AM Piotr Nowojski
> > wrote:
> >
> > > Hi All,
> >
+1 (binding)
Regards,
Roman
On Wed, Nov 22, 2023, 7:08 AM Zakelly Lan wrote:
> +1(non-binding)
>
> Best,
> Zakelly
>
> On Wed, Nov 22, 2023 at 3:04 PM Hangxiang Yu wrote:
>
> > +1 (binding)
> > Thanks for driving this again!
> >
> > On Wed, Nov 22, 2023 at 10:30 AM Rui Fan <1996fan...@gmail.co
+1 (binding)
Thanks for the proposal
Regards,
Roman
On Wed, Nov 22, 2023, 10:08 AM Piotr Nowojski
wrote:
> Thanks Rui!
>
> +1 (binding)
>
> Best,
> Piotrek
>
> śr., 22 lis 2023 o 08:05 Hangxiang Yu napisał(a):
>
> > +1 (binding)
> > Thanks for your efforts!
> >
> > On Mon, Nov 20, 2023 at 11
tart with the simplest min/max/sum/avg, and let's see in
> which direction (if any) we need to evolve
> that. Alternative to percentiles is previously mentioned to report
> separately each subtask's initialisation/checkpointing span.
>
> Best,
> Piotrek
>
> [1]
&
Roman Khachatryan created FLINK-33590:
-
Summary: CheckpointStatsTracker.totalNumberOfSubTasks not updated
Key: FLINK-33590
URL: https://issues.apache.org/jira/browse/FLINK-33590
Project: Flink
Thanks for the proposal,
Can you please explain:
1. why the existing MetricGroup interface can't be used? It already had
methods to add metrics and spans ...
2. IIUC, based on these numbers, we're going to report span(s). Shouldn't
the backend report them as spans?
3. How is the implementation s
Thanks Piotr, the proposal totally makes sense to me.
Does it depend on FLIP-384 for voting?
Otherwise, we could probably start the vote already as there're no counter
proposals or objections.
Regards,
Roman
On Tue, Nov 7, 2023, 1:19 PM Piotr Nowojski
wrote:
> Hey, sorry for the misclick. Fixe
Thanks for the proposal,
Starting with the minimal functionality and expanding if necessary as the
FLIP describes makes a lot of sense to me.
Regards,
Roman
On Wed, Nov 15, 2023, 9:31 PM Jing Ge wrote:
> Hi Piotr,
>
> Sorry for the late reply and thanks for the proposal, it looks awesome!
>
>
Roman Khachatryan created FLINK-33442:
-
Summary: UnsupportedOperationException thrown from
RocksDBIncrementalRestoreOperation
Key: FLINK-33442
URL: https://issues.apache.org/jira/browse/FLINK-33442
Congratulations, Yanfey!
Regards,
Roman
On Wed, Aug 9, 2023 at 12:49 PM Benchao Li wrote:
> Congrats, YanFei!
>
> Jing Ge 于2023年8月8日周二 17:41写道:
>
> > Congrats, YanFei!
> >
> > Best regards,
> > Jing
> >
> > On Tue, Aug 8, 2023 at 3:04 PM Yangze Guo wrote:
> >
> > > Congrats, Yanfei!
> > >
>
Congratulations, Hangxiang!
Regards,
Roman
On Wed, Aug 9, 2023 at 12:49 PM Benchao Li wrote:
> Congrats, Hangxiang!
>
> Jing Ge 于2023年8月8日周二 17:44写道:
>
> > Congrats, Hangxiang!
> >
> > Best regards,
> > Jing
> >
> > On Tue, Aug 8, 2023 at 3:04 PM Yangze Guo wrote:
> >
> > > Congrats, Hangxia
+1 (binding)
The FLIP LGTM, thanks Panos!
Regards,
Roman
On Thu, Apr 20, 2023 at 1:33 PM Hong Teoh wrote:
> +1 (non-binding)
>
> Thank you for driving this effort, Panagiotis.
>
> Regards,
> Hong
>
>
> > On 20 Apr 2023, at 12:16, David Morávek wrote:
> >
> > Thanks for the update!
> >
> > +1
Roman Khachatryan created FLINK-31601:
-
Summary: While waiting for resources, resources check might be
scheduled unlimited number of times (Adaptive Scheduler)
Key: FLINK-31601
URL: https://issues.apache.org
+1 (binding)
Thanks David, and everyone involved :)
Regards,
Roman
On Wed, Mar 1, 2023 at 8:01 AM Gyula Fóra wrote:
> +1 (binding)
>
> Looking forward to this :)
>
> Gyula
>
> On Wed, 1 Mar 2023 at 04:02, feng xiangyu wrote:
>
> > +1 (non-binding)
> >
> > ConradJam 于2023年3月1日周三 10:37写道:
>
Roman Khachatryan created FLINK-31261:
-
Summary: Make AdaptiveScheduler aware of the (local) state size
Key: FLINK-31261
URL: https://issues.apache.org/jira/browse/FLINK-31261
Project: Flink
Hi,
Thanks for the update, I think distinguishing the rescaling behaviour and
the desired parallelism declaration is important.
Having the ability to specify min parallelism might be useful in
environments with multiple jobs: Scheduler will then have an option to stop
the less suitable job.
In ot
Congratulations Rui!
Regards,
Roman
On Mon, Feb 20, 2023 at 5:58 PM Anton Kalashnikov
wrote:
> Congrats Rui!
>
> --
> Best regards,
> Anton Kalashnikov
>
> On 20.02.23 17:53, Matthias Pohl wrote:
> > Congratulations, Rui :)
> >
> > On Mon, Feb 20, 2023 at 5:10 PM Jing Ge
> wrote:
> >
> >> Con
Congratulations Anton, well deserved!
Regards,
Roman
On Tue, Feb 21, 2023 at 9:34 AM Martijn Visser
wrote:
> Congratulations Anton!
>
> On Tue, Feb 21, 2023 at 8:08 AM Lincoln Lee
> wrote:
>
> > Congratulations, Anton!
> >
> > Best,
> > Lincoln Lee
> >
> >
> > Guowei Ma 于2023年2月21日周二 15:05写道
the suggestion. I have a test about different exchange types,
> forward
> and rescale, and the results show no differences from the all-to-all type,
> which
> is also understandable, because the network memory usage is calculated
> with numChannels, independent of the edge type.
>
>
Hi everyone,
Thanks for the proposal and the discussion.
I couldn't find much details on how exactly the values of
ExclusiveBuffersPerChannel and FloatingBuffersPerGate are calculated.
I guess that
- the threshold evaluation is done on JM
- floating buffers calculation is done on TM based on the
Roman Khachatryan created FLINK-30073:
-
Summary: Managed memory can be wasted if rocksdb memory is
fixed-per-slot
Key: FLINK-30073
URL: https://issues.apache.org/jira/browse/FLINK-30073
Project
gt; > >
> > > As an alternative, I'd suggest introducing a variant of
> > > RocksDBStateBackend, that shares memory across slots and does not use
> > > managed memory. This basically means the shared memory is not
> considered
> > as
> > > part of manage
the example, how is the memory size of
> unmanaged part calculated?
> 3. For fine-grained-resource-management, the control
> of cpuCores, taskHeapMemory can still work, right? And I am a little
> worried that too many memory-about configuration options are complicated
> for users to
Roman Khachatryan created FLINK-29985:
-
Summary: SlotTable not close on TM termination
Key: FLINK-29985
URL: https://issues.apache.org/jira/browse/FLINK-29985
Project: Flink
Issue Type
r" in the example, how is the memory size of
> unmanaged part calculated?
> 3. For fine-grained-resource-management, the control
> of cpuCores, taskHeapMemory can still work, right? And I am a little
> worried that too many memory-about configuration options are complicated
&
Hi everyone,
I'd like to discuss sharing RocksDB memory across slots as proposed in
FLINK-29928 [1].
Since 1.10 / FLINK-7289 [2], it is possible to:
- share these objects among RocksDB instances of the same slot
- bound the total memory usage by all RocksDB instances of a TM
However, the memory
Roman Khachatryan created FLINK-29928:
-
Summary: Allow sharing (RocksDB) memory between slots
Key: FLINK-29928
URL: https://issues.apache.org/jira/browse/FLINK-29928
Project: Flink
Issue
Roman Khachatryan created FLINK-29158:
-
Summary: Fix logging in DefaultCompletedCheckpointStore
Key: FLINK-29158
URL: https://issues.apache.org/jira/browse/FLINK-29158
Project: Flink
Roman Khachatryan created FLINK-29157:
-
Summary: Clarify the contract between CompletedCheckpointStore and
SharedStateRegistry
Key: FLINK-29157
URL: https://issues.apache.org/jira/browse/FLINK-29157
Roman Khachatryan created FLINK-28976:
-
Summary: Changelog 1st materialization delayed unneccesarily
Key: FLINK-28976
URL: https://issues.apache.org/jira/browse/FLINK-28976
Project: Flink
Roman Khachatryan created FLINK-28931:
-
Summary: BlockingPartitionBenchmark doesn't compile
Key: FLINK-28931
URL: https://issues.apache.org/jira/browse/FLINK-28931
Project: Flink
Roman Khachatryan created FLINK-28647:
-
Summary: Remove separate error handling and adjust documentation
for CLAIM mode + RocksDB native savepoint
Key: FLINK-28647
URL: https://issues.apache.org/jira/browse
Roman Khachatryan created FLINK-28597:
-
Summary: Empty checkpoint folders not deleted on job cancellation
if their shared state is still in use
Key: FLINK-28597
URL: https://issues.apache.org/jira/browse
Hi,
Thanks for the proposal Yun, I think that's a good idea and it could
solve the issue you mentioned (FLINK-26590) in many cases (though not all,
depending on deletion speed; but in practice it may be enough).
Having a separate interface (BulkDeletingFileSystem) would probably help in
increment
1 - 100 of 392 matches
Mail list logo