Hi Martijn,
Sorry for later reply. This feature is only supported in DataStream and doesn't
be supported in MATCH_RECOGNIZE because the SQL syntax of MATCH_RECOGNIZE does
not contain the semantics of this feature, which requires modification of the
SQL syntax. The support above MATCH_RECOGNIZE
lincoln lee created FLINK-27849:
---
Summary: Harden correctness for non-deterministic updates present
in the changelog pipeline
Key: FLINK-27849
URL: https://issues.apache.org/jira/browse/FLINK-27849
Proj
Xintong Song created FLINK-27848:
Summary: ZooKeeperLeaderElectionDriver keeps writing leader
information, using up zxid
Key: FLINK-27848
URL: https://issues.apache.org/jira/browse/FLINK-27848
Project
Hi Yue,
Thanks for providing the benefit of this feature. After this feature is merged,
you are welcome to try this feature in business scenarios.
Regards,
Nicholas Jiang
On 2022/05/06 11:31:48 yue ma wrote:
> hi Nicholas,
>
> Thanks for bringing this discussion, we also think it's a useful fe
Hi Dian,
Thanks for your feedback about supprting the within between events feature. I
have updated the FLIP for introduction of 'Pattern#within(WithinType
withInType, Time windowTime)' interface. Regarding your comments, I have the
following thoughts:
- Regarding the API, the name `partialWit
+1 (binding)
Thanks Lincoln for driving the FLIP and discussion.
Best,
Yun Gao
--
From:Jingsong Li
Send Time:2022 May 31 (Tue.) 11:19
To:dev
Subject:Re: [VOTE] FLIP-232: Add Retry Support For Async I/O In DataStream API
+1 (bind
Hi,
Regarding the drop operation, with some offline discussion with Dalong and Zhu,
we think that listening in the client side might be problematic since it would
exit
after submitting the jobs in detached mode, thus the operation might need to
be in the JobMaster side.
For the listener interf
Lincoln Thanks! Looks very good.
Best,
Jingsong
On Tue, May 31, 2022 at 12:09 PM Lincoln Lee wrote:
> Thanks Jingsong for the reminder! There is indeed some cleanup work to be
> done, I'll update the doc today.
>
> Best,
> Lincoln Lee
>
>
> Jingsong Li 于2022年5月31日周二 11:29写道:
>
> > Hi Lincoln,
Hi everyone,
Please review and vote on the release candidate #3 for the version 1.0.0 of
Apache Flink Kubernetes Operator,
as follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific comments)
**Release Overview**
As an overview, the release consists of th
Thanks Jingsong for the reminder! There is indeed some cleanup work to be
done, I'll update the doc today.
Best,
Lincoln Lee
Jingsong Li 于2022年5月31日周二 11:29写道:
> Hi Lincoln,
>
> Some minor comments for FLIP document (Does not affect voting):
>
> - Maybe we can remove `The current processing f
Hi Lincoln,
Some minor comments for FLIP document (Does not affect voting):
- Maybe we can remove `The current processing flow chart` part, at first
impression I thought this was part of `Proposed Changes`, it's right under
`Proposed Changes`.
- About `We tried two implementations`, I think we ca
+1 (binding)
Thanks for driving.
Best,
Jingsong
On Tue, May 31, 2022 at 11:05 AM Gen Luo wrote:
> +1 (non-binding)
>
> On Mon, May 30, 2022 at 3:50 PM Jark Wu wrote:
>
> > +1 (binding)
> >
> > Best,
> > Jark
> >
> > On Mon, 30 May 2022 at 15:40, Lincoln Lee
> wrote:
> >
> > > Dear Flink deve
+1 (non-binding)
On Mon, May 30, 2022 at 3:50 PM Jark Wu wrote:
> +1 (binding)
>
> Best,
> Jark
>
> On Mon, 30 May 2022 at 15:40, Lincoln Lee wrote:
>
> > Dear Flink developers,
> >
> > Thanks for all your feedback for FLIP-232: Add Retry Support For Async
> I/O
> > In DataStream API[1] on the
Jingsong Lee created FLINK-27847:
Summary: SchemaManager supports schema evolution
Key: FLINK-27847
URL: https://issues.apache.org/jira/browse/FLINK-27847
Project: Flink
Issue Type: Sub-task
Jingsong Lee created FLINK-27846:
Summary: Schema evolution for data file reading
Key: FLINK-27846
URL: https://issues.apache.org/jira/browse/FLINK-27846
Project: Flink
Issue Type: Sub-task
Jingsong Lee created FLINK-27845:
Summary: Introduce Column Type Schema evolution
Key: FLINK-27845
URL: https://issues.apache.org/jira/browse/FLINK-27845
Project: Flink
Issue Type: Sub-task
Jingsong Lee created FLINK-27844:
Summary: Schema evolution for ManifestFileMeta
Key: FLINK-27844
URL: https://issues.apache.org/jira/browse/FLINK-27844
Project: Flink
Issue Type: Sub-task
Jingsong Lee created FLINK-27843:
Summary: Schema evolution for data file meta
Key: FLINK-27843
URL: https://issues.apache.org/jira/browse/FLINK-27843
Project: Flink
Issue Type: Sub-task
#contribution-helps is meant for new contributors to ask non-technical
questions. E.g., looking for starter issues or reviewers (just don't DM
people).
For having a #dev/development, I'm a little concerned this may encourage
people to start discussions directly in slack. IMHO, we may want to stick
On Mon, May 30, 2022 at 7:35 AM Martijn Visser
wrote:
> Hi Chen,
>
> I think the best starting point would be to create a FLIP [1]. One of the
> important topics from my point of view is to make sure that such changes
> are not only available for SQL users, but are also being considered for
> Tab
Hello everyone,
I would like to start a discussion about FLIP-236: Asynchronous Job Submission
[1].
During the processing of a job submission request, Flink runs the “main” method
in a previously submitted JAR.
This method may contain operations that take arbitrarily long to complete or
genuin
Thanks for raising it.
But I wonder what do you mean by saying "dynamic overwrite schema type
information loaded from HiveCatalog".
For such case, what does the HiveCatalog store?
Best regards,
Yuxia
- 原始邮件 -
发件人: "Chen Qin"
收件人: "dev"
发送时间: 星期一, 2022年 5 月 30日 上午 1:04:29
主题: Table API
Jing Ge created FLINK-27842:
---
Summary: Rename ndv to granularityNumber
Key: FLINK-27842
URL: https://issues.apache.org/jira/browse/FLINK-27842
Project: Flink
Issue Type: Improvement
Compo
Development channel (#dev/development) makes sense to me. It’s related to
actual development related questions instead of the contrib process.
Gyula
On Mon, 30 May 2022 at 19:19, Jing Ge wrote:
> Good idea, thanks! Is the channel #contribution-helps a good fit for it? Or
> should we just rename
Good idea, thanks! Is the channel #contribution-helps a good fit for it? Or
should we just rename it to #development? For me, development is a subset
of contribution.
Best regards,
Jing
On Mon, May 30, 2022 at 4:28 PM Robert Metzger wrote:
> Thanks a lot for kicking this off.
>
> Is there a rea
Hi Xingbo,
+1
Thanks for driving this.
Best regards,
Jing
On Mon, May 30, 2022 at 4:51 PM Martijn Visser
wrote:
> Hi Xingbo,
>
> +1 to release Flink 1.14.5.
>
> Best regards,
>
> Martijn
>
> Op do 26 mei 2022 om 10:19 schreef Dian Fu :
>
> > Hi Xingbo,
> >
> > Thanks for driving this release.
Thanks Robert for the reminder. Thanks Martijn for sharing the link.
Is the feature freeze deadline on July 25 fixed or will it be adjusted
accordingly?
Best regards,
Jing
On Mon, May 30, 2022 at 4:56 PM Martijn Visser
wrote:
> Yes, we will have the release meeting tomorrow. Looking forward to
Hi Jingsong and devs!
I agree that custom reloading would be very useful, so I changed
recently proposed ReloadTime to customizable ReloadStrategy and its
default realization FixedDelayReloadStrategy. I updated the FLIP, you
can look at the new design [1]. From my point of view, the
disadvantage o
Feifan Wang created FLINK-27841:
---
Summary: RocksDB cache miss increase in 1.15
Key: FLINK-27841
URL: https://issues.apache.org/jira/browse/FLINK-27841
Project: Flink
Issue Type: Improvement
lijixiang created FLINK-27839:
-
Summary:
org.apache.flink.kafka.shaded.org.apache.kafka.common.errors.InterruptException
Key: FLINK-27839
URL: https://issues.apache.org/jira/browse/FLINK-27839
Project: Fl
lijixiang created FLINK-27840:
-
Summary:
org.apache.flink.kafka.shaded.org.apache.kafka.common.errors.DisconnectException
Key: FLINK-27840
URL: https://issues.apache.org/jira/browse/FLINK-27840
Project: F
Yes, we will have the release meeting tomorrow. Looking forward to everyone
who wants to participate. For those that are looking for the invite link,
see https://cwiki.apache.org/confluence/display/FLINK/1.16+Release
Best regards,
Martijn
Op ma 30 mei 2022 om 16:39 schreef Robert Metzger :
> I
Hi Xingbo,
+1 to release Flink 1.14.5.
Best regards,
Martijn
Op do 26 mei 2022 om 10:19 schreef Dian Fu :
> Hi Xingbo,
>
> Thanks for driving this release. +1 for 1.14.5 as there are already nearly
> 100 commits [1] since 1.14.4.
>
> Regards,
> Dian
>
> [1] https://github.com/apache/flink/comp
I assume we'll have the next release planning meeting tomorrow?
I'm also bringing this up also as a reminder for other folks who might be
interested in joining.
On Wed, May 11, 2022 at 5:55 AM Xintong Song wrote:
> I'd like to kindly remind that, if someone adds / modifies release notes of
> a
Hi Chen,
I think the best starting point would be to create a FLIP [1]. One of the
important topics from my point of view is to make sure that such changes
are not only available for SQL users, but are also being considered for
Table API, DataStream and/or Python. There might be reasons why not to
Thanks a lot for kicking this off.
Is there a reason why we haven't set up a #development channel yet?
I have a short question that is more suitable for that channel, compared to
the dev@ list ;)
On Mon, May 23, 2022 at 7:52 AM Xintong Song wrote:
> Hi Kyle,
>
> I've sent an invitation to your
Shengkai Fang created FLINK-27838:
-
Summary: Support timeout mechansim for executeStatement API
Key: FLINK-27838
URL: https://issues.apache.org/jira/browse/FLINK-27838
Project: Flink
Issue Ty
Shengkai Fang created FLINK-27837:
-
Summary: Support statement set in the SQL Gateway
Key: FLINK-27837
URL: https://issues.apache.org/jira/browse/FLINK-27837
Project: Flink
Issue Type: Sub-ta
>
> I thought we could enable Adaptive Scheduler, so adding or removing a task
> manager is the same as restarting a job when we use an adaptive scheduler.
> Do I miss anything ?
It is true for standalone mode since adding/removing a TaskManager pod is
fully controlled by users(or external tools)
Hi Jacky,
Some performance improvements for Flame Graphs related to how the stack
traces are collected were recently merged [1]. Since you are working on a
real-life environment with high parallelism, could you maybe run a couple
of tests to verify that everything still works as expected after the
Nico Kruber created FLINK-27836:
---
Summary: RocksDBMapState iteration may stop too early for
var-length prefixes
Key: FLINK-27836
URL: https://issues.apache.org/jira/browse/FLINK-27836
Project: Flink
Thanks for starting this discussion. I think it makes sense to make a clear
definition and goal of ITCases and e2e tests. For me, compared to e2e
tests, ITCases are more lightweight and only focus on some specific part of
the e2e pipeline. E2e tests cover the whole pipeline and could be
considered
Thanks for the clarifications. I agree then that it indeed makes sense to
move the related classes into connector-test-utils for now and reconsider
when and if concrete use cases of FlinkContainerTestEnvironment outside of
the connectors' scope come up.
On Wed, May 25, 2022 at 5:23 PM Chesnay Sch
Hi Becket,
Thanks for summing this up. Just one correction:
> Piotr prefers option 2, his opinions are:
> e) It is OK that the code itself in option 2 indicates the developers
that a feature is optional. We will rely on the documentation to correct
that and clarify that the feature is actually
Hi Jing Ge,
What do you mean about the "impact on the block cache used by HBase"?
In my understanding, the connector cache and HBase cache are totally two
things.
The connector cache is a local/client cache, and the HBase cache is a
server cache.
> does it make sense to have a no-cache solution a
Thanks Jark for your quick response and the consensus!
And I will update the FLIP after Jingsong or other developers confirm that
there is no problem.
Best,
Lincoln Lee
Jark Wu 于2022年5月30日周一 15:49写道:
> Thanks for the update.
>
> The unified lookup hint looks good to me.
> And thanks for expla
+1 (binding)
Best,
Jark
On Mon, 30 May 2022 at 15:40, Lincoln Lee wrote:
> Dear Flink developers,
>
> Thanks for all your feedback for FLIP-232: Add Retry Support For Async I/O
> In DataStream API[1] on the discussion thread[2].
>
> I'd like to start a vote for it. The vote will be open for at
Thanks for the update.
The unified lookup hint looks good to me.
And thanks for explaining ALLOW_UNORDERED.
Best,
Jark
On Mon, 30 May 2022 at 15:31, Lincoln Lee wrote:
> Hi Jark & Jingsong,
>
> Thanks for your feedback!
>
> 1.) support retry on sync lookup
> I also agree with supporting it, th
Caizhi Weng created FLINK-27835:
---
Summary: Introduce LeafPredicate interface and children method in
Predicate
Key: FLINK-27835
URL: https://issues.apache.org/jira/browse/FLINK-27835
Project: Flink
Hi everyone,
I started a vote for this FLIP [1], please vote there or ask additional
questions here. [2]
[1] https://lists.apache.org/thread/qp9y1k0gldxymzmrso0xgsrwh15n6clc
[2] https://lists.apache.org/thread/pgm3bf8vd5vqchlm29n6cro0gz4pbd3g
Best,
Lincoln Lee
Jark Wu 于2022年5月27日周五 14:44写道:
Dear Flink developers,
Thanks for all your feedback for FLIP-232: Add Retry Support For Async I/O
In DataStream API[1] on the discussion thread[2].
I'd like to start a vote for it. The vote will be open for at least 72
hours unless there is an objection or not enough votes.
[1]
https://cwiki.apa
Hi Jark & Jingsong,
Thanks for your feedback!
1.) support retry on sync lookup
I also agree with supporting it, this will be useful for connectors that
don't have asynchronous lookup implementations and can also solve the ASYNC
non-target problem to some extent(because the retrying is blocking fo
52 matches
Mail list logo