Re: [DISCUSS] Release flink-shaded 12.0

2020-09-29 Thread Till Rohrmann
Thanks for volunteering as our release manager Robert! Cheers, Till On Tue, Sep 29, 2020 at 7:36 PM Robert Metzger wrote: > It seems that we have consensus to create a flink-shaded release. > > I'll soon propose a RC. > > On Fri, Sep 25, 2020 at 9:11 AM Konstantin Knauf > wrote: > > > +1 > > >

Re: [VOTE] FLIP-142: Disentangle StateBackends from Checkpointing

2020-09-29 Thread Till Rohrmann
+1 (binding) Cheers, Till On Tue, Sep 29, 2020 at 5:04 PM Kostas Kloudas wrote: > +1 (binding) > > Kostas > > On Tue, Sep 29, 2020 at 4:57 PM Yu Li wrote: > > > > +1 (binding) > > > > Thanks all for the patience of answering / addressing my questions in the > > discussion thread. > > > > Best

[jira] [Created] (FLINK-19475) Implement a timer service that holds a single key at a time

2020-09-29 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-19475: Summary: Implement a timer service that holds a single key at a time Key: FLINK-19475 URL: https://issues.apache.org/jira/browse/FLINK-19475 Project: Flink

[jira] [Created] (FLINK-19474) Implement a state backends that holds a single key at a time

2020-09-29 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-19474: Summary: Implement a state backends that holds a single key at a time Key: FLINK-19474 URL: https://issues.apache.org/jira/browse/FLINK-19474 Project: Flink

[jira] [Created] (FLINK-19473) Implement multi inputs sorting DataInput

2020-09-29 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-19473: Summary: Implement multi inputs sorting DataInput Key: FLINK-19473 URL: https://issues.apache.org/jira/browse/FLINK-19473 Project: Flink Issue Type:

[jira] [Created] (FLINK-19472) Implement a one input sorting DataInput

2020-09-29 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-19472: Summary: Implement a one input sorting DataInput Key: FLINK-19472 URL: https://issues.apache.org/jira/browse/FLINK-19472 Project: Flink Issue Type: S

[jira] [Created] (FLINK-19471) CVE-2020-7712 is reported for flink-streaming-java_2.11:jar:1.11.1

2020-09-29 Thread Jeff Hu (Jira)
Jeff Hu created FLINK-19471: --- Summary: CVE-2020-7712 is reported for flink-streaming-java_2.11:jar:1.11.1 Key: FLINK-19471 URL: https://issues.apache.org/jira/browse/FLINK-19471 Project: Flink Iss

Re: Need help in creating Flink Streaming s3 Job for multiple path reader one by one

2020-09-29 Thread Satyaa Dixit
Hi Guys, Sorry to bother you again, but someone could help me here? Any help in this regard will be much appreciated. Regards, Satya On Tue, Sep 29, 2020 at 2:57 PM Satyaa Dixit wrote: > Hi Guys, > I need one help, any leads will be highly appreciated.I have written a > flink streaming job to

[jira] [Created] (FLINK-19470) ParquetColumnarRowSplitReader::reachEnd returns false after it has reached end

2020-09-29 Thread Rui Li (Jira)
Rui Li created FLINK-19470: -- Summary: ParquetColumnarRowSplitReader::reachEnd returns false after it has reached end Key: FLINK-19470 URL: https://issues.apache.org/jira/browse/FLINK-19470 Project: Flink

Re: [VOTE] FLIP-143: Unified Sink API

2020-09-29 Thread Guowei Ma
Hi all, The voting time for FLIP-143 [1] has passed. I'm closing the vote now. There were 5 votes, 4 of which are binding: - Aljoscha Krettek (binding) - Steven Wu - Kostas Kloudas (binding) - Jingsong Li (binding) - Jiangang Liu (binding) There were no -1 votes. Thus, changes have been accept

[jira] [Created] (FLINK-19469) HBase connector 2.2 failed to download dependencies "org.glassfish:javax.el:jar:3.0.1-b06-SNAPSHOT"

2020-09-29 Thread Dian Fu (Jira)
Dian Fu created FLINK-19469: --- Summary: HBase connector 2.2 failed to download dependencies "org.glassfish:javax.el:jar:3.0.1-b06-SNAPSHOT" Key: FLINK-19469 URL: https://issues.apache.org/jira/browse/FLINK-19469

[jira] [Created] (FLINK-19468) Metrics not returned when data stream / operator name contains "+"

2020-09-29 Thread Boyang Jerry Peng (Jira)
Boyang Jerry Peng created FLINK-19468: - Summary: Metrics not returned when data stream / operator name contains "+" Key: FLINK-19468 URL: https://issues.apache.org/jira/browse/FLINK-19468 Project:

[jira] [Created] (FLINK-19467) Implement HashMapStateBackend and EmbeddedRocksDBStateBackend

2020-09-29 Thread Seth Wiesman (Jira)
Seth Wiesman created FLINK-19467: Summary: Implement HashMapStateBackend and EmbeddedRocksDBStateBackend Key: FLINK-19467 URL: https://issues.apache.org/jira/browse/FLINK-19467 Project: Flink

[jira] [Created] (FLINK-19466) Implement JobManagerCheckpointStorage and FileSystemCheckpointStorage

2020-09-29 Thread Seth Wiesman (Jira)
Seth Wiesman created FLINK-19466: Summary: Implement JobManagerCheckpointStorage and FileSystemCheckpointStorage Key: FLINK-19466 URL: https://issues.apache.org/jira/browse/FLINK-19466 Project: Flink

[jira] [Created] (FLINK-19465) Add CheckpointStorage interface

2020-09-29 Thread Seth Wiesman (Jira)
Seth Wiesman created FLINK-19465: Summary: Add CheckpointStorage interface Key: FLINK-19465 URL: https://issues.apache.org/jira/browse/FLINK-19465 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-19464) Rename CheckpointStorage interface to CheckpointStorageAccess

2020-09-29 Thread Seth Wiesman (Jira)
Seth Wiesman created FLINK-19464: Summary: Rename CheckpointStorage interface to CheckpointStorageAccess Key: FLINK-19464 URL: https://issues.apache.org/jira/browse/FLINK-19464 Project: Flink

[jira] [Created] (FLINK-19463) Disentangle StateBackends from Checkpointing

2020-09-29 Thread Seth Wiesman (Jira)
Seth Wiesman created FLINK-19463: Summary: Disentangle StateBackends from Checkpointing Key: FLINK-19463 URL: https://issues.apache.org/jira/browse/FLINK-19463 Project: Flink Issue Type: Impr

[RESULT][VOTE] FLIP-142: Disentangle StateBackends from Checkpointing

2020-09-29 Thread Seth Wiesman
Hi all, The voting time for FLIP-142 has passed. I'm closing the vote now. - Konstantin (binding) - David Anderson (binding) - Gordon (binding) - Congxian - David Wysakowicz (binding) - Aljoscha (binding) - Yu (binding) - Kostas (binding) Including myself, there were 9 votes, 8 binding. There we

[jira] [Created] (FLINK-19462) Checkpoint statistics for unfinished task snapshots

2020-09-29 Thread Nico Kruber (Jira)
Nico Kruber created FLINK-19462: --- Summary: Checkpoint statistics for unfinished task snapshots Key: FLINK-19462 URL: https://issues.apache.org/jira/browse/FLINK-19462 Project: Flink Issue Type:

[jira] [Created] (FLINK-19461) yarn-sesson.sh -jm -tm arguments have no effect

2020-09-29 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-19461: -- Summary: yarn-sesson.sh -jm -tm arguments have no effect Key: FLINK-19461 URL: https://issues.apache.org/jira/browse/FLINK-19461 Project: Flink Issue Typ

[jira] [Created] (FLINK-19460) AWS Kinesis Producer EXACTLY_ONCE semantic

2020-09-29 Thread Chris Slotterback (Jira)
Chris Slotterback created FLINK-19460: - Summary: AWS Kinesis Producer EXACTLY_ONCE semantic Key: FLINK-19460 URL: https://issues.apache.org/jira/browse/FLINK-19460 Project: Flink Issue T

[jira] [Created] (FLINK-19459) flink-dist won't build locally with newer (3.3+) maven versions

2020-09-29 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-19459: -- Summary: flink-dist won't build locally with newer (3.3+) maven versions Key: FLINK-19459 URL: https://issues.apache.org/jira/browse/FLINK-19459 Project: Flink

Re: [DISCUSS] Release flink-shaded 12.0

2020-09-29 Thread Robert Metzger
It seems that we have consensus to create a flink-shaded release. I'll soon propose a RC. On Fri, Sep 25, 2020 at 9:11 AM Konstantin Knauf wrote: > +1 > > > > On Wed, Sep 23, 2020 at 9:13 AM Yu Li wrote: > > > +1 > > > > Best Regards, > > Yu > > > > > > On Tue, 22 Sep 2020 at 17:49, Robert Met

[jira] [Created] (FLINK-19458) ZooKeeperLeaderElectionITCase.testJobExecutionOnClusterWithLeaderChange: ZooKeeper unexpectedly modified

2020-09-29 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-19458: -- Summary: ZooKeeperLeaderElectionITCase.testJobExecutionOnClusterWithLeaderChange: ZooKeeper unexpectedly modified Key: FLINK-19458 URL: https://issues.apache.org/jira/browse/

[jira] [Created] (FLINK-19457) Port NumberSequenceSource to FLIP-27 source interface

2020-09-29 Thread Stephan Ewen (Jira)
Stephan Ewen created FLINK-19457: Summary: Port NumberSequenceSource to FLIP-27 source interface Key: FLINK-19457 URL: https://issues.apache.org/jira/browse/FLINK-19457 Project: Flink Issue T

Re: [VOTE] FLIP-142: Disentangle StateBackends from Checkpointing

2020-09-29 Thread Kostas Kloudas
+1 (binding) Kostas On Tue, Sep 29, 2020 at 4:57 PM Yu Li wrote: > > +1 (binding) > > Thanks all for the patience of answering / addressing my questions in the > discussion thread. > > Best Regards, > Yu > > > On Thu, 17 Sep 2020 at 14:39, Dawid Wysakowicz > wrote: > > > +1 (binding) > > > > On

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

2020-09-29 Thread Yu Li
Yes let's move on, already cast my vote in the voting thread. Thanks all for the patience answering / addressing my belated questions! Best Regards, Yu On Sun, 27 Sep 2020 at 20:00, Stephan Ewen wrote: > Good to see this FLIP moving. > > From what I understand, the remaining questions are mai

Re: [VOTE] FLIP-142: Disentangle StateBackends from Checkpointing

2020-09-29 Thread Yu Li
+1 (binding) Thanks all for the patience of answering / addressing my questions in the discussion thread. Best Regards, Yu On Thu, 17 Sep 2020 at 14:39, Dawid Wysakowicz wrote: > +1 (binding) > > On 17/09/2020 07:19, Congxian Qiu wrote: > > +1 (non-binding) > > > > Best, > > Congxian > > > >

[jira] [Created] (FLINK-19456) sql client execute insert sql with comment ahead

2020-09-29 Thread ledong Lin (Jira)
ledong Lin created FLINK-19456: -- Summary: sql client execute insert sql with comment ahead Key: FLINK-19456 URL: https://issues.apache.org/jira/browse/FLINK-19456 Project: Flink Issue Type: Bug

Re: [DISCUSS] FLIP-144: Native Kubernetes HA for Flink

2020-09-29 Thread Till Rohrmann
For 1. I was wondering whether we can't write the leader connection information directly when trying to obtain the leadership (trying to update the leader key with one's own value)? This might be a little detail, though. 2. Alright, so we are having a similar mechanism as we have in ZooKeeper with

Re: CheckpointedFunction initialization during checkpoint

2020-09-29 Thread Aljoscha Krettek
Hi Teng, I think if the system is slowed down enough it can happen that some parts of the graph are still restoring while others are already taking a checkpoint. By virtue of how checkpointing works (by sending barriers along the network connections between tasks) this should not be a problem

Re: [DISCUSS] FLIP-146: Improve new TableSource and TableSink interfaces

2020-09-29 Thread Aljoscha Krettek
Hi, I'll only respond regarding the parallelism for now because I need to think some more about DataStream. What I'm saying is that exposing a parallelism only for Table Connectors is not the right thing. If we want to allow sources to tell the system/framework what would be a good paralleli

[jira] [Created] (FLINK-19455) Module 'flink-sql-connector-hive-2.3.6' build fail by maven-enforcer-plugin

2020-09-29 Thread hailong wang (Jira)
hailong wang created FLINK-19455: Summary: Module 'flink-sql-connector-hive-2.3.6' build fail by maven-enforcer-plugin Key: FLINK-19455 URL: https://issues.apache.org/jira/browse/FLINK-19455 Project:

[jira] [Created] (FLINK-19454) Translate page 'Importing Flink into an IDE' into Chinese

2020-09-29 Thread wulei0302 (Jira)
wulei0302 created FLINK-19454: - Summary: Translate page 'Importing Flink into an IDE' into Chinese Key: FLINK-19454 URL: https://issues.apache.org/jira/browse/FLINK-19454 Project: Flink Issue Typ

Re: [VOTE] FLIP-143: Unified Sink API

2020-09-29 Thread 刘建刚
+1 (binding) Best, Liu Jiangang Jingsong Li 于2020年9月29日周二 下午1:36写道: > +1 (binding) > > Best, > Jingsong > > On Mon, Sep 28, 2020 at 3:21 AM Kostas Kloudas wrote: > > > +1 (binding) > > > > @Steven Wu I think there will be opportunities to fine tune the API > > during the implementation. > > >

[jira] [Created] (FLINK-19453) Deprecate old source and sink interfaces

2020-09-29 Thread Timo Walther (Jira)
Timo Walther created FLINK-19453: Summary: Deprecate old source and sink interfaces Key: FLINK-19453 URL: https://issues.apache.org/jira/browse/FLINK-19453 Project: Flink Issue Type: Sub-task

Re: [DISCUSS][Release 1.12] Stale blockers and build instabilities

2020-09-29 Thread Dian Fu
Hi all, I'd like to update the status about the blocker issues and build instabilities as there is only one month left and the number of blocker issues increases a lot compared to last week. == Blockers: https://issues.apache.org/jira/browse/FLINK-18682?filter=12349334

[jira] [Created] (FLINK-19452) statistics of group by CDC data is always 1

2020-09-29 Thread Zhengchao Shi (Jira)
Zhengchao Shi created FLINK-19452: - Summary: statistics of group by CDC data is always 1 Key: FLINK-19452 URL: https://issues.apache.org/jira/browse/FLINK-19452 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-19451) Add HELM chart distribution to StateFun release process

2020-09-29 Thread Igal Shilman (Jira)
Igal Shilman created FLINK-19451: Summary: Add HELM chart distribution to StateFun release process Key: FLINK-19451 URL: https://issues.apache.org/jira/browse/FLINK-19451 Project: Flink Issue

Need help in creating Flink Streaming s3 Job for multiple path reader one by one

2020-09-29 Thread Satyaa Dixit
Hi Guys, I need one help, any leads will be highly appreciated.I have written a flink streaming job to read the data from s3 bucket and push to kafka. Below is the working source that deal with single s3 path: TextInputFormat format = new TextInputFormat(new org.apache.flink.core.fs.Path("s3a://dir

[jira] [Created] (FLINK-19450) Optimize the Python CI Test

2020-09-29 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-19450: Summary: Optimize the Python CI Test Key: FLINK-19450 URL: https://issues.apache.org/jira/browse/FLINK-19450 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-19449) LEAD/LAG cannot work correctly in streaming mode

2020-09-29 Thread Benchao Li (Jira)
Benchao Li created FLINK-19449: -- Summary: LEAD/LAG cannot work correctly in streaming mode Key: FLINK-19449 URL: https://issues.apache.org/jira/browse/FLINK-19449 Project: Flink Issue Type: Bug

Re: Need help in setting up flink 1.10

2020-09-29 Thread David Anderson
Ravi, Note that questions like this are better suited for the user mailing list. According to [1], google cloud storage is supported under the gcs: url scheme. Also, since Flink 1.10, most filesystems must be loaded as plugins, rather than from the lib directory [2]. I don't have experience with

[jira] [Created] (FLINK-19448) CoordinatedSourceITCase.testEnumeratorReaderCommunication hangs

2020-09-29 Thread Dian Fu (Jira)
Dian Fu created FLINK-19448: --- Summary: CoordinatedSourceITCase.testEnumeratorReaderCommunication hangs Key: FLINK-19448 URL: https://issues.apache.org/jira/browse/FLINK-19448 Project: Flink Issue