Re: [VOTE] FLIP-309: Support using larger checkpointing interval when source is processing backlog

2023-06-29 Thread feng xiangyu
+1 (non-binding)

Best,
Xiangyu

yuxia  于2023年6月29日周四 20:44写道:

> +1 (binding)
>
> Best regards,
> Yuxia
>
> - 原始邮件 -
> 发件人: "Yuepeng Pan" 
> 收件人: "dev" 
> 发送时间: 星期四, 2023年 6 月 29日 下午 8:21:14
> 主题: Re: [VOTE] FLIP-309: Support using larger checkpointing interval when
> source is processing backlog
>
> +1  non-binding.
>
>
> Best.
> Yuepeng Pan
>
>
>  Replied Message 
> | From | Jingsong Li |
> | Date | 06/29/2023 13:25 |
> | To | dev |
> | Cc | flink.zhouyunfeng |
> | Subject | Re: [VOTE] FLIP-309: Support using larger checkpointing
> interval when source is processing backlog |
> +1 binding
>
> On Thu, Jun 29, 2023 at 11:03 AM Dong Lin  wrote:
> >
> > Hi all,
> >
> > We would like to start the vote for FLIP-309: Support using larger
> > checkpointing interval when source is processing backlog [1]. This FLIP
> was
> > discussed in this thread [2].
> >
> > Flink 1.18 release will feature freeze on July 11. We hope to make this
> > feature available in Flink 1.18.
> >
> > The vote will be open until at least July 4th (at least 72 hours),
> following
> > the consensus voting process.
> >
> > Cheers,
> > Yunfeng and Dong
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-309%3A+Support+using+larger+checkpointing+interval+when+source+is+processing+backlog
> > [2] https://lists.apache.org/thread/l1l7f30h7zldjp6ow97y70dcthx7tl37
>


Re: [DISCUSS] FLIP-309: Enable operators to trigger checkpoints dynamically

2023-06-28 Thread feng xiangyu
Hi Dong,

Thanks for your quick reply. I think this has truly solved our problem and
will enable us upgrade our existing jobs more seamless.

Best,
Xiangyu

Dong Lin  于2023年6月29日周四 10:50写道:

> Hi Feng,
>
> Thanks for the feedback. Yes, you can configure the
> execution.checkpointing.interval-during-backlog to effectively disable
> checkpoint during backlog.
>
> Prior to your comment, the FLIP allows users to do this by setting the
> config value to something large (e.g. 365 day). After thinking about this
> more, we think it is more usable to allow users to achieve this goal by
> setting the config value to 0. This is consistent with the existing
> behavior of execution.checkpointing.interval -- the checkpoint is disabled
> if user set execution.checkpointing.interval to 0.
>
> We have updated the description of
> execution.checkpointing.interval-during-backlog
> to say the following:
> ... it is not null, the value must either be 0, which means the checkpoint
> is disabled during backlog, or be larger than or equal to
> execution.checkpointing.interval.
>
> Does this address your need?
>
> Best,
> Dong
>
>
>
> On Thu, Jun 29, 2023 at 9:23 AM feng xiangyu  wrote:
>
> > Hi Dong and Yunfeng,
> >
> > Thanks for the proposal, your flip sounds very useful from my
> perspective.
> > In our business, when we using hybrid source in production we also met
> the
> > problem described in your flip.
> > In our solution, we tend to skip making any checkpoints before all batch
> > tasks have finished and resume the periodic checkpoint only in streaming
> > phrase. Within this flip, we can solve our problem in a more generic way.
> >
> > However, I am wondering if we still want to skip making any checkpoints
> > during historical phrase, can we set this configuration
> > "execution.checkpointing.interval-during-backlog" equals "-1" to cover
> this
> > case?
> >
> > Best,
> > Xiangyu
> >
> > Hang Ruan  于2023年6月28日周三 16:30写道:
> >
> > > Thanks for Dong and Yunfeng's work.
> > >
> > > The FLIP looks good to me. This new version is clearer to understand.
> > >
> > > Best,
> > > Hang
> > >
> > > Dong Lin  于2023年6月27日周二 16:53写道:
> > >
> > > > Thanks Jack, Jingsong, and Zhu for the review!
> > > >
> > > > Thanks Zhu for the suggestion. I have updated the configuration name
> as
> > > > suggested.
> > > >
> > > > On Tue, Jun 27, 2023 at 4:45 PM Zhu Zhu  wrote:
> > > >
> > > > > Thanks Dong and Yunfeng for creating this FLIP and driving this
> > > > discussion.
> > > > >
> > > > > The new design looks generally good to me. Increasing the
> checkpoint
> > > > > interval when the job is processing backlogs is easier for users to
> > > > > understand and can help in more scenarios.
> > > > >
> > > > > I have one comment about the new configuration.
> > > > > Naming the new configuration
> > > > > "execution.checkpointing.interval-during-backlog" would be better
> > > > > according to Flink config naming convention.
> > > > > It is also because that nested config keys should be avoided. See
> > > > > FLINK-29372 for more details.
> > > > >
> > > > > Thanks,
> > > > > Zhu
> > > > >
> > > > > Jingsong Li  于2023年6月27日周二 15:45写道:
> > > > > >
> > > > > > Looks good to me!
> > > > > >
> > > > > > Thanks Dong, Yunfeng and all for your discussion and design.
> > > > > >
> > > > > > Best,
> > > > > > Jingsong
> > > > > >
> > > > > > On Tue, Jun 27, 2023 at 3:35 PM Jark Wu 
> wrote:
> > > > > > >
> > > > > > > Thank you Dong for driving this FLIP.
> > > > > > >
> > > > > > > The new design looks good to me!
> > > > > > >
> > > > > > > Best,
> > > > > > > Jark
> > > > > > >
> > > > > > > > 2023年6月27日 14:38,Dong Lin  写道:
> > > > > > > >
> > > > > > > > Thank you Leonard for the review!
> > > > > > > >
> > > > > > > > Hi Piotr, do you have any comments on the latest proposal?
> > > > > > > >
> > >

Re: [DISCUSS] FLIP-309: Enable operators to trigger checkpoints dynamically

2023-06-28 Thread feng xiangyu
Hi Dong and Yunfeng,

Thanks for the proposal, your flip sounds very useful from my perspective.
In our business, when we using hybrid source in production we also met the
problem described in your flip.
In our solution, we tend to skip making any checkpoints before all batch
tasks have finished and resume the periodic checkpoint only in streaming
phrase. Within this flip, we can solve our problem in a more generic way.

However, I am wondering if we still want to skip making any checkpoints
during historical phrase, can we set this configuration
"execution.checkpointing.interval-during-backlog" equals "-1" to cover this
case?

Best,
Xiangyu

Hang Ruan  于2023年6月28日周三 16:30写道:

> Thanks for Dong and Yunfeng's work.
>
> The FLIP looks good to me. This new version is clearer to understand.
>
> Best,
> Hang
>
> Dong Lin  于2023年6月27日周二 16:53写道:
>
> > Thanks Jack, Jingsong, and Zhu for the review!
> >
> > Thanks Zhu for the suggestion. I have updated the configuration name as
> > suggested.
> >
> > On Tue, Jun 27, 2023 at 4:45 PM Zhu Zhu  wrote:
> >
> > > Thanks Dong and Yunfeng for creating this FLIP and driving this
> > discussion.
> > >
> > > The new design looks generally good to me. Increasing the checkpoint
> > > interval when the job is processing backlogs is easier for users to
> > > understand and can help in more scenarios.
> > >
> > > I have one comment about the new configuration.
> > > Naming the new configuration
> > > "execution.checkpointing.interval-during-backlog" would be better
> > > according to Flink config naming convention.
> > > It is also because that nested config keys should be avoided. See
> > > FLINK-29372 for more details.
> > >
> > > Thanks,
> > > Zhu
> > >
> > > Jingsong Li  于2023年6月27日周二 15:45写道:
> > > >
> > > > Looks good to me!
> > > >
> > > > Thanks Dong, Yunfeng and all for your discussion and design.
> > > >
> > > > Best,
> > > > Jingsong
> > > >
> > > > On Tue, Jun 27, 2023 at 3:35 PM Jark Wu  wrote:
> > > > >
> > > > > Thank you Dong for driving this FLIP.
> > > > >
> > > > > The new design looks good to me!
> > > > >
> > > > > Best,
> > > > > Jark
> > > > >
> > > > > > 2023年6月27日 14:38,Dong Lin  写道:
> > > > > >
> > > > > > Thank you Leonard for the review!
> > > > > >
> > > > > > Hi Piotr, do you have any comments on the latest proposal?
> > > > > >
> > > > > > I am wondering if it is OK to start the voting thread this week.
> > > > > >
> > > > > > On Mon, Jun 26, 2023 at 4:10 PM Leonard Xu 
> > > wrote:
> > > > > >
> > > > > >> Thanks Dong for driving this FLIP forward!
> > > > > >>
> > > > > >> Introducing  `backlog status` concept for flink job makes sense
> to
> > > me as
> > > > > >> following reasons:
> > > > > >>
> > > > > >> From concept/API design perspective, it’s more general and
> natural
> > > than
> > > > > >> above proposals as it can be used in HybridSource for bounded
> > > records, CDC
> > > > > >> Source for history snapshot and general sources like KafkaSource
> > for
> > > > > >> historical messages.
> > > > > >>
> > > > > >> From user cases/requirements, I’ve seen many users manually to
> set
> > > larger
> > > > > >> checkpoint interval during backfilling and then set a shorter
> > > checkpoint
> > > > > >> interval for real-time processing in their production
> environments
> > > as a
> > > > > >> flink application optimization. Now, the flink framework can
> make
> > > this
> > > > > >> optimization no longer require the user to set the checkpoint
> > > interval and
> > > > > >> restart the job multiple times.
> > > > > >>
> > > > > >> Following supporting using larger checkpoint for job under
> backlog
> > > status
> > > > > >> in current FLIP, we can explore supporting larger
> > > parallelism/memory/cpu
> > > > > >> for job under backlog status in the future.
> > > > > >>
> > > > > >> In short, the updated FLIP looks good to me.
> > > > > >>
> > > > > >>
> > > > > >> Best,
> > > > > >> Leonard
> > > > > >>
> > > > > >>
> > > > > >>> On Jun 22, 2023, at 12:07 PM, Dong Lin 
> > > wrote:
> > > > > >>>
> > > > > >>> Hi Piotr,
> > > > > >>>
> > > > > >>> Thanks again for proposing the isProcessingBacklog concept.
> > > > > >>>
> > > > > >>> After discussing with Becket Qin and thinking about this more,
> I
> > > agree it
> > > > > >>> is a better idea to add a top-level concept to all source
> > > operators to
> > > > > >>> address the target use-case.
> > > > > >>>
> > > > > >>> The main reason that changed my mind is that
> isProcessingBacklog
> > > can be
> > > > > >>> described as an inherent/nature attribute of every source
> > instance
> > > and
> > > > > >> its
> > > > > >>> semantics does not need to depend on any specific checkpointing
> > > policy.
> > > > > >>> Also, we can hardcode the isProcessingBacklog behavior for the
> > > sources we
> > > > > >>> have considered so far (e.g. HybridSource and MySQL CDC source)
> > > without
> > > > > >>> asking users to explicitly configure the per-source behavior,
> > which
> > > > > >> indeed
> 

Re: [ANNOUNCE] Flink Table Store Joins Apache Incubator as Apache Paimon(incubating)

2023-03-27 Thread feng xiangyu
Congrats Yu! Looking forward to contributing to Paimon!

Best Regards,
Xiangyu

yuxia  于2023年3月27日周一 21:01写道:

> congratulations!
>
> Best regards,
> Yuxia
>
>
> 发件人: "Andrew Otto" 
> 收件人: "Matthias Pohl" 
> 抄送: "Jing Ge" , "Leonard Xu" , "Yu
> Li" , "dev" , "User" <
> u...@flink.apache.org>, "user-zh" 
> 发送时间: 星期一, 2023年 3 月 27日 下午 8:57:50
> 主题: Re: [ANNOUNCE] Flink Table Store Joins Apache Incubator as Apache
> Paimon(incubating)
>
> Exciting!
>
> If this ends up working well, Wikimedia Foundation would love to try it
> out!
>
> On Mon, Mar 27, 2023 at 8:39 AM Matthias Pohl via user < [ mailto:
> u...@flink.apache.org | u...@flink.apache.org ] > wrote:
>
>
>
> Congratulations and good luck with pushing the project forward.
>
> On Mon, Mar 27, 2023 at 2:35 PM Jing Ge via user < [ mailto:
> u...@flink.apache.org | u...@flink.apache.org ] > wrote:
>
> BQ_BEGIN
>
> Congrats!
> Best regards,
> Jing
>
> On Mon, Mar 27, 2023 at 2:32 PM Leonard Xu < [ mailto:xbjt...@gmail.com |
> xbjt...@gmail.com ] > wrote:
>
> BQ_BEGIN
>
> Congratulations!
>
> Best,
> Leonard
>
>
> BQ_BEGIN
>
> On Mar 27, 2023, at 5:23 PM, Yu Li < [ mailto:car...@gmail.com |
> car...@gmail.com ] > wrote:
>
> Dear Flinkers,
>
>
>
>
> As you may have noticed, we are pleased to announce that Flink Table Store
> has joined the Apache Incubator as a separate project called Apache
> Paimon(incubating) [1] [2] [3]. The new project still aims at building a
> streaming data lake platform for high-speed data ingestion, change data
> tracking and efficient real-time analytics, with the vision of supporting a
> larger ecosystem and establishing a vibrant and neutral open source
> community.
>
>
>
>
> We would like to thank everyone for their great support and efforts for
> the Flink Table Store project, and warmly welcome everyone to join the
> development and activities of the new project. Apache Flink will continue
> to be one of the first-class citizens supported by Paimon, and we believe
> that the Flink and Paimon communities will maintain close cooperation.
>
>
>
>
> 亲爱的Flinkers,
>
>
>
>
> 正如您可能已经注意到的,我们很高兴地宣布,Flink Table Store 已经正式加入 Apache 孵化器独立孵化 [1] [2]
> [3]。新项目的名字是 Apache
> Paimon(incubating),仍致力于打造一个支持高速数据摄入、流式数据订阅和高效实时分析的新一代流式湖仓平台。此外,新项目将支持更加丰富的生态,并建立一个充满活力和中立的开源社区。
>
>
>
>
>
> 在这里我们要感谢大家对 Flink Table Store 项目的大力支持和投入,并热烈欢迎大家加入新项目的开发和社区活动。Apache Flink
> 将继续作为 Paimon 支持的主力计算引擎之一,我们也相信 Flink 和 Paimon 社区将继续保持密切合作。
>
>
>
>
> Best Regards,
> Yu (on behalf of the Apache Flink PMC and Apache Paimon PPMC)
>
> 致礼,
> 李钰(谨代表 Apache Flink PMC 和 Apache Paimon PPMC)
>
>
>
>
> [1] [ https://paimon.apache.org/ | https://paimon.apache.org/ ]
> [2] [ https://github.com/apache/incubator-paimon |
> https://github.com/apache/incubator-paimon ]
> [3] [ https://cwiki.apache.org/confluence/display/INCUBATOR/PaimonProposal
> | https://cwiki.apache.org/confluence/display/INCUBATOR/PaimonProposal ]
>
>
>
>
>
> BQ_END
>
>
> BQ_END
>
>
> BQ_END
>
>
>


Re: [VOTE] FLIP-291: Externalized Declarative Resource Management

2023-02-28 Thread feng xiangyu
+1  (non-binding)

ConradJam  于2023年3月1日周三 10:37写道:

> +1  (non-binding)
>
> Zhanghao Chen  于2023年3月1日周三 10:18写道:
>
> > Thanks for driving this. +1 (non-binding)
> >
> > Best,
> > Zhanghao Chen
> > 
> > From: David Mor?vek 
> > Sent: Tuesday, February 28, 2023 21:46
> > To: dev 
> > Subject: [VOTE] FLIP-291: Externalized Declarative Resource Management
> >
> > Hi Everyone,
> >
> > I want to start the vote on FLIP-291: Externalized Declarative Resource
> > Management [1]. The FLIP was discussed in this thread [2].
> >
> > The goal of the FLIP is to enable external declaration of the resource
> > requirements of a running job.
> >
> > The vote will last for at least 72 hours (Friday, 3rd of March, 15:00
> CET)
> > unless
> > there is an objection or insufficient votes.
> >
> > [1] https://lists.apache.org/thread/b8fnj127jsl5ljg6p4w3c4wvq30cnybh
> > [2]
> >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-291%3A+Externalized+Declarative+Resource+Management
> >
> > Best,
> > D.
> >
>
>
> --
> Best
>
> ConradJam
>


Re: [DISCUSS] FLIP-291: Externalized Declarative Resource Management

2023-02-13 Thread feng xiangyu
Hi David,

Thanks for your reply.  I think your response totally make sense.  This
flip targets on declaring required resource to ResourceManager instead of
using  ResourceManager to add/remove TMs directly.

Best,
Xiangyu



David Morávek  于2023年2月13日周一 15:46写道:

> Hi everyone,
>
> @Shammon
>
> I'm not entirely sure what "config file" you're referring to. You can, of
> course, override the default parallelism in "flink-conf.yaml", but for
> sinks and sources, the parallelism needs to be tweaked on the connector
> level ("WITH" statement).
>
> This is something that should be achieved with tooling around Flink. We
> want to provide an API on the lowest level that generalizes well. Achieving
> what you're describing should be straightforward with this API.
>
> @Xiangyu
>
> Is it possible for this REST API to declare TM resources in the future?
>
>
> Would you like to add/remove TMs if you use an active Resource Manager?
> This would be out of the scope of this effort since it targets the
> scheduler component only (we make no assumptions about the used Resource
> Manager). Also, the AdaptiveScheduler is only intended to be used for
> Streaming.
>
>  And for streaming jobs, I'm wondering if there is any situation we need to
> > rescale the TM resources of a flink cluster at first and then the
> adaptive
> > scheduler will rescale the per-vertex ResourceProfiles accordingly.
> >
>
> We plan on adding support for the ResourceProfiles (dynamic slot
> allocation) as the next step. Again we won't make any assumptions about the
> used Resource Manager. In other words, this effort ends by declaring
> desired resources to the Resource Manager.
>
> Does that make sense?
>
> @Matthias
>
> We've done another pass on the proposed API and currently lean towards
> having an idempotent PUT API.
> - We don't care too much about multiple writers' scenarios in terms of who
> can write an authoritative payload; this is up to the user of the API to
> figure out
> - It's indeed tricky to achieve atomicity with PATCH API; switching to PUT
> API seems to do the trick
> - We won't allow partial "payloads" anymore, meaning you need to define
> requirements for all vertices in the JobGraph; This is completely fine for
> the programmatic workflows. For DEBUG / DEMO purposes, you can use the GET
> endpoint and tweak the response to avoid writing the whole payload by hand.
>
> WDYT?
>
>
> Best,
> D.
>
> On Fri, Feb 10, 2023 at 11:21 AM feng xiangyu 
> wrote:
>
> > Hi David,
> >
> > Thanks for creating this flip. I think this work it is very useful,
> > especially in autoscaling scenario.  I would like to share some questions
> > from my view.
> >
> > 1, Is it possible for this REST API to declare TM resources in the
> future?
> > I'm asking because we are building the autoscaling feature for Flink OLAP
> > Session Cluster in ByteDance. We need to rescale the cluster's resource
> on
> > TM level instead of Job level. It would be very helpful if we have a REST
> > API for out external Autoscaling service to use.
> >
> > 2, And for streaming jobs, I'm wondering if there is any situation we
> need
> > to rescale the TM resources of a flink cluster at first and then the
> > adaptive scheduler will rescale the per-vertex ResourceProfiles
> > accordingly.
> >
> > best.
> > Xiangyu
> >
> > Shammon FY  于2023年2月9日周四 11:31写道:
> >
> > > Hi David
> > >
> > > Thanks for your answer.
> > >
> > > > Can you elaborate more about how you'd intend to use the endpoint? I
> > > think we can ultimately introduce a way of re-declaring "per-vertex
> > > defaults," but I'd like to understand the use case bit more first.
> > >
> > > For this issue, I mainly consider the consistency of user configuration
> > and
> > > job runtime. For sql jobs, users usually set specific parallelism for
> > > source and sink, and set a global parallelism for other operators.
> These
> > > config items are stored in a config file. For some high-priority jobs,
> > > users may want to manage them manually.
> > > 1. When users need to scale the parallelism, they should update the
> > config
> > > file and restart flink job, which may take a long time.
> > > 2. After providing the REST API, users can just send a request to the
> job
> > > via REST API quickly after updating the config file.
> > > The configuration in the running job and config file should be the
> same.
> > > What do you think of this?
> > >
&

Re: [DISCUSS] FLIP-291: Externalized Declarative Resource Management

2023-02-10 Thread feng xiangyu
Hi David,

Thanks for creating this flip. I think this work it is very useful,
especially in autoscaling scenario.  I would like to share some questions
from my view.

1, Is it possible for this REST API to declare TM resources in the future?
I'm asking because we are building the autoscaling feature for Flink OLAP
Session Cluster in ByteDance. We need to rescale the cluster's resource on
TM level instead of Job level. It would be very helpful if we have a REST
API for out external Autoscaling service to use.

2, And for streaming jobs, I'm wondering if there is any situation we need
to rescale the TM resources of a flink cluster at first and then the
adaptive scheduler will rescale the per-vertex ResourceProfiles
accordingly.

best.
Xiangyu

Shammon FY  于2023年2月9日周四 11:31写道:

> Hi David
>
> Thanks for your answer.
>
> > Can you elaborate more about how you'd intend to use the endpoint? I
> think we can ultimately introduce a way of re-declaring "per-vertex
> defaults," but I'd like to understand the use case bit more first.
>
> For this issue, I mainly consider the consistency of user configuration and
> job runtime. For sql jobs, users usually set specific parallelism for
> source and sink, and set a global parallelism for other operators. These
> config items are stored in a config file. For some high-priority jobs,
> users may want to manage them manually.
> 1. When users need to scale the parallelism, they should update the config
> file and restart flink job, which may take a long time.
> 2. After providing the REST API, users can just send a request to the job
> via REST API quickly after updating the config file.
> The configuration in the running job and config file should be the same.
> What do you think of this?
>
> best.
> Shammon
>
>
>
> On Tue, Feb 7, 2023 at 4:51 PM David Morávek 
> wrote:
>
> > Hi everyone,
> >
> > Let's try to answer the questions one by one.
> >
> > *@ConradJam*
> >
> > when the number of "slots" is insufficient, can we can stop users
> rescaling
> > > or throw something to tell user "less avaliable slots to upgrade,
> please
> > > checkout your alivalbe slots" ?
> > >
> >
> > The main property of AdaptiveScheduler is that it can adapt to "available
> > resources," which means you're still able to make progress even though
> you
> > didn't get all the slots you've asked for. Let's break down the pros and
> > cons of this property.
> >
> > - (plus) If you lose a TM for some reason, you can still recover even if
> it
> > doesn't come back. We still need to give it some time to eliminate
> > unnecessary rescaling, which can be controlled by setting
> > "resource-stabilization-timeout."
> > - (plus) The resources can arrive with a significant delay. For example,
> > you're unable to spawn enough TMs on time because you've run out of
> > resources in your k8s cluster, and you need to wait for the cluster auto
> > scaler to kick in and add new nodes to the cluster. In this scenario,
> > you'll be able to start making progress faster, at the cost of multiple
> > rescalings (once the remaining resources arrive).
> > - (plus) This plays well with the declarative manner of today's
> > infrastructure. For example, you tell k8s that you need 10 TMs, and
> you'll
> > eventually get them.
> > - (minus) In the case of large state jobs, the cost of multiple
> rescalings
> > might outweigh the above.
> >
> > We've already touched on the solution to this problem on the FLIP. Please
> > notice the parallelism knob being a range with a lower and upper bound.
> > Setting both the lower and upper bound to the same value could give the
> > behavior you're describing at the cost of giving up some properties that
> AS
> > gives you (you'd be falling back to the DefaultScheduler's behavior).
> >
> > when user upgrade job-vertx-parallelism . I want to have an interface to
> > > query the current update parallel execution status, so that the user or
> > > program can understand the current status
> > >
> >
> > This is a misunderstanding. We're not introducing the RESCALE endpoint.
> > This endpoint allows you to re-declare the resources needed to run the
> job.
> > Once you reach the desired resources (you get more resources than the
> lower
> > bound defines), your job will run.
> >
> > We can expose a similar endpoint to "resource requirements" to give you
> an
> > overview of the resources the vertices already have. You can already get
> > this from the REST API, so exposing this in yet another way should be
> > considered carefully.
> >
> > *@Matthias*
> >
> > I'm wondering whether it makes sense to add some kind of resource ID to
> the
> > > REST API.
> >
> >
> > That's a good question. I want to think about that and get back to the
> > question later. My main struggle when thinking about this is, "if this
> > would be an idempotent POST endpoint," would it be any different?
> >
> > How often do we allow resource requirements to be changed?
> >
> >
> > There shall be no rate limiting on the