Hi,
I think Thomas raised very good questions and would like to know your
opinions if we want to move connectors out of flink in this version.
(1) is the connector API already stable?
> Separate releases would only make sense if the core Flink surface is
> fairly stable though. As evident from Ic
Hi all,
Zhu Zhu and I propose to introduce a new job scheduler to Flink: adaptive
batch job scheduler. The new scheduler can automatically decide
parallelisms of job vertices for batch jobs, according to the size of data
volume each vertex needs to process.
Major benefits of this scheduler inclu
Yue Ma created FLINK-24597:
--
Summary: RocksdbStateBackend getKeysAndNamespaces would return
duplicate data when using MapState
Key: FLINK-24597
URL: https://issues.apache.org/jira/browse/FLINK-24597
Project
Jingsong Lee created FLINK-24596:
Summary: Bugs in sink.buffer-flush before upsert-kafka
Key: FLINK-24596
URL: https://issues.apache.org/jira/browse/FLINK-24596
Project: Flink
Issue Type: Bug
Pavel Penkov created FLINK-24595:
Summary: Programmatic configuration of S3 doesn't pass parameters
to Hadoop FS
Key: FLINK-24595
URL: https://issues.apache.org/jira/browse/FLINK-24595
Project: Flink
Could you clarify what release cadence you're thinking of? There's quite
a big range that fits "more frequent than Flink" (per-commit, daily,
weekly, bi-weekly, monthly, even bi-monthly).
On 19/10/2021 14:15, Martijn Visser wrote:
Hi all,
I think it would be a huge benefit if we can achieve m
+1 (non-binding)
verified:
- build from source code
- run batch job using native k8s
- restore streaming job written in Apache Beam (java, scala 2.11)
- restore streaming job written in java (scala 2.11)
All works and I don't see anything weird in logs.
Thanks!
L.
On Tue, Oct 12, 2021 at 7:22 P
Azocan Kara created FLINK-24594:
---
Summary: /jobs endpoint returns same job id twice (suspended and
running) until stopped
Key: FLINK-24594
URL: https://issues.apache.org/jira/browse/FLINK-24594
Project:
Seth Wiesman created FLINK-24593:
Summary: Create Table API Quickstart
Key: FLINK-24593
URL: https://issues.apache.org/jira/browse/FLINK-24593
Project: Flink
Issue Type: Improvement
Sergey Nuyanzin created FLINK-24592:
---
Summary: FlinkSQL multiline parser improvements
Key: FLINK-24592
URL: https://issues.apache.org/jira/browse/FLINK-24592
Project: Flink
Issue Type: Impr
Thank you Chesnay and all the others who have contributed!
Best regards,
Martijn
On Tue, 19 Oct 2021 at 15:03, Chesnay Schepler wrote:
> We have unanimously approved this release:
>
> Binding votes:
> - Dian
> - Arvid
> - Chesnay
>
> On 12/10/2021 19:21, Chesnay Schepler wrote:
> > Hi everyone
We have unanimously approved this release:
Binding votes:
- Dian
- Arvid
- Chesnay
On 12/10/2021 19:21, Chesnay Schepler wrote:
Hi everyone,
Please review and vote on the release candidate #1 for the version
1.13.3, as follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (p
+1 (binding)
On 19/10/2021 13:59, Arvid Heise wrote:
+1 (binding)
- build from source on scala 2_12 profile
- ran standalone cluster with examples
Best,
Arvid
On Tue, Oct 19, 2021 at 4:48 AM Dian Fu wrote:
+1 (binding)
- verified the checksum and signature
- checked the dependency change
TBH I think you're overestimating how much work it is to create a
non-Flink release. Having done most of the flink-shaded releases, I
really don't see an issue of even doing weekly releases with that process.
We can not reduce the number of votes AFAIK; the ASF seems very clear on
that matter
Hey all,
I don't have much to add to the general discussion. Just a single
comment on:
that we could adjust the bylaws for the connectors such that we need
fewer PMCs to approve a release. Would it be enough to have one PMC
vote per connector release?
I think it's not an option. This
Thank you, Arvid & team, for working on this.
I would also favor one connector repository under the ASF. This will
already force us to provide better tools and more stable APIs, which
connectors developed outside of Apache Flink will benefit from, too.
Besides simplifying the formal release proce
Okay I think it is clear that the majority would like to keep connectors
under the Apache Flink umbrella. That means we will not be able to have
per-connector repositories and project management, automatic dependency
bumping with Dependabot, or semi-automatic releases.
So then I'm assuming the dir
Hi all,
I think it would be a huge benefit if we can achieve more frequent releases
of connectors, which are not bound to the release cycle of Flink itself. I
agree that in order to get there, we need to have stable interfaces which
are trustworthy and reliable, so they can be safely used by those
+1 (binding)
- build from source on scala 2_12 profile
- ran standalone cluster with examples
Best,
Arvid
On Tue, Oct 19, 2021 at 4:48 AM Dian Fu wrote:
> +1 (binding)
>
> - verified the checksum and signature
> - checked the dependency changes since 1.13.2. There is only one dependency
> cha
Krzysztof Dziolak created FLINK-24591:
-
Summary: Kafka Producer fails with SecurityException when using
cluster.intercept-user-system-exit
Key: FLINK-24591
URL: https://issues.apache.org/jira/browse/FLINK-2459
Hi,
I'm not sure I understand your question. Are you looking for a way to
define multiple view in SQL? Please try define multiple view by define
multiple create view query, you could find create view syntax in document
[1]
Please let me know if I misunderstand your requirement.
[1]
https://nightli
Chesnay Schepler created FLINK-24590:
Summary: Consider removing timeout from
FlinkMatchers#futureWillCompleteExceptionally
Key: FLINK-24590
URL: https://issues.apache.org/jira/browse/FLINK-24590
Anton Kalashnikov created FLINK-24589:
-
Summary: FLIP-183: Buffer debloating 1.2
Key: FLINK-24589
URL: https://issues.apache.org/jira/browse/FLINK-24589
Project: Flink
Issue Type: New Fea
Marios Trivyzas created FLINK-24588:
---
Summary: Optimisation of chained cast calls can lead to unexpected
behaviour
Key: FLINK-24588
URL: https://issues.apache.org/jira/browse/FLINK-24588
Project: Fl
Hi,
I went through the flink documents, multiple view creation through flink
approach not able to findout, please help me here for implementation.
Thanks in advance.
Regards,
Mahesh Kumar GH
TO:@Arvid Heise @Caizhi Weng
thanks for your reply, as the stream api code become more and more complicated,
we will add more dependenies from third pary.
This kind of problem will be inevitable. If we only rely on this kind of trick
like shade the dependencies to solve this kind of problem,
Shiao-An Yuan created FLINK-24587:
-
Summary: Let PubSub source support changing subscriptions
Key: FLINK-24587
URL: https://issues.apache.org/jira/browse/FLINK-24587
Project: Flink
Issue Type
Ingo Bürk created FLINK-24586:
-
Summary: SQL functions should return STRING instead of
VARCHAR(2000)
Key: FLINK-24586
URL: https://issues.apache.org/jira/browse/FLINK-24586
Project: Flink
Issue
28 matches
Mail list logo