Re: [VOTE] FLIP-90: Support SQL 2016-2017 JSON functions in Flink SQL

2020-01-06 Thread Jingsong Li
+1 non-binding. Thanks Forward for driving this. Considering that it is made up of independent and certain things from SQL standard and Calcite, I think it can be started as soon as possible. Best, Jingsong Lee On Tue, Dec 31, 2019 at 5:09 PM Forward Xu wrote: > Hi all, > > I'd like to start

Re: [DISCUSS] Set default planner for SQL Client to Blink planner in 1.10 release

2020-01-06 Thread Jark Wu
Thanks everyone for the feedback! We have reached a clear consensus that the proposal is accepted. There are 3 binding votes from: - Timo - Hequn - Jark I will make sure that this change is applied to Flink code base and created FLINK-15495 to track this. Best, Jark On Mon, 6 Jan 2020 at

[jira] [Created] (FLINK-15497) Topn

2020-01-06 Thread Kurt Young (Jira)
Kurt Young created FLINK-15497: -- Summary: Topn Key: FLINK-15497 URL: https://issues.apache.org/jira/browse/FLINK-15497 Project: Flink Issue Type: Bug Reporter: Kurt Young --

Re: [DISCUSS] JDBC exactly-once sink

2020-01-06 Thread Jingsong Li
Hi Piotr, We already have "JDBCUpsertOutputFormat". It is mainly proposed for Table/SQL API, but DataStream can use it to have the semantics of upsert. But it has weaker semantics than exactly once, and can only guarantee the final consistency. Best, Jingsong Lee On Mon, Jan 6, 2020 at 8:07 PM

[jira] [Created] (FLINK-15496) Remove RegisterApplicationMasterResponseReflector

2020-01-06 Thread Zili Chen (Jira)
Zili Chen created FLINK-15496: - Summary: Remove RegisterApplicationMasterResponseReflector Key: FLINK-15496 URL: https://issues.apache.org/jira/browse/FLINK-15496 Project: Flink Issue Type:

Re: [DISCUSS] FLIP-90: Support SQL 2016-2017 JSON functions in Flink SQL

2020-01-06 Thread Forward Xu
Thanks Jark for checking the doc. hi,Timo, please help to check to see if there is anything else to add. Best, Forward Jark Wu 于2020年1月6日周一 下午2:58写道: > Thanks Forward for the updating. It is much more clearer now about the > returning type, especially JSON_VALUE. > The design doc looks good to

[jira] [Created] (FLINK-15495) Set default planner for SQL Client to Blink planner

2020-01-06 Thread Jark Wu (Jira)
Jark Wu created FLINK-15495: --- Summary: Set default planner for SQL Client to Blink planner Key: FLINK-15495 URL: https://issues.apache.org/jira/browse/FLINK-15495 Project: Flink Issue Type:

[jira] [Created] (FLINK-15494) time field index wrong in LogicalWindowAggregateRuleBase

2020-01-06 Thread Benchao Li (Jira)
Benchao Li created FLINK-15494: -- Summary: time field index wrong in LogicalWindowAggregateRuleBase Key: FLINK-15494 URL: https://issues.apache.org/jira/browse/FLINK-15494 Project: Flink Issue

Re: Hi Guys

2020-01-06 Thread Xintong Song
Hi, Welcome to the Apache Flink community. You do not need a contributor permission for opening jira tickets. Once the ticket is opened, discussed and consensus are reached, you can ask a committer to assign the ticket and start working on it. Please refer to the community contribution guidelines

Re: Why aren't Range Scans Supported in MapState?

2020-01-06 Thread Yun Tang
Hi Aaron We would usually compare MapState with Map in java language. And your request is not what typical Map interface but SortedMap defines. I think it's reasonable If we introduce SortedMapState interface in Flink. However, there existed several challenges for RocksDB. * RocksDB only

Hi Guys

2020-01-06 Thread ????
Hi Guys, I want to contribute to Apache Flink. Would you please give me the permission as a contributor? My JIRA ID is mackson.

[jira] [Created] (FLINK-15493) FlinkKafkaInternalProducerITCase.testProducerWhenCommitEmptyPartitionsToOutdatedTxnCoordinator failed on travis

2020-01-06 Thread Dian Fu (Jira)
Dian Fu created FLINK-15493: --- Summary: FlinkKafkaInternalProducerITCase.testProducerWhenCommitEmptyPartitionsToOutdatedTxnCoordinator failed on travis Key: FLINK-15493 URL:

Re: [DISCUSS] FLIP-85: Delayed Job Graph Generation

2020-01-06 Thread Peter Huang
Dear All, Happy new year! According to existing feedback from the community, we revised the doc with the consideration of session cluster support, and concrete interface changes needed and execution plan. Please take one more round of review at your most convenient time.

[jira] [Created] (FLINK-15492) flink-clients test failing on Red Hat 7.6 PowerPC Linux

2020-01-06 Thread Ronald O. Edmark (Jira)
Ronald O. Edmark created FLINK-15492: Summary: flink-clients test failing on Red Hat 7.6 PowerPC Linux Key: FLINK-15492 URL: https://issues.apache.org/jira/browse/FLINK-15492 Project: Flink

Why aren't Range Scans Supported in MapState?

2020-01-06 Thread Aaron Langford
Hey Flink Community, RocksDB natively supports range scans for keys. Is there a good story why we don't have a way to ask for keys by range or prefix currently in MapState? Aaron

Re: [DISCUSS] Some feedback after trying out the new FLIP-49 memory configurations

2020-01-06 Thread Andrey Zagrebin
Thank you for more explanation Stephan and feedback Jingsong, I see the point now. ‘taskmanager.memory.flink.size’ is indeed simpler to understand for the newcomers in the default config because it includes less types of memory to consider. Most of the time beginners will think about heap size

[jira] [Created] (FLINK-15491) Fix incorrect path while missing trailing slash of high-availability.zookeeper.path.running-registry parameter.

2020-01-06 Thread Qingxin Wu (Jira)
Qingxin Wu created FLINK-15491: -- Summary: Fix incorrect path while missing trailing slash of high-availability.zookeeper.path.running-registry parameter. Key: FLINK-15491 URL:

[jira] [Created] (FLINK-15490) KafkaITCase.testOneSourceMultiplePartitions fails on Travis

2020-01-06 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-15490: - Summary: KafkaITCase.testOneSourceMultiplePartitions fails on Travis Key: FLINK-15490 URL: https://issues.apache.org/jira/browse/FLINK-15490 Project: Flink

[ANNOUNCE] Flink Forward SF Call for Presentation closing soon!

2020-01-06 Thread Fabian Hueske
Hi all, First of all, Happy New Year to everyone! Many of you probably didn't spent the holidays thinking a lot about Flink. Now, however, is the right time to focus again and decide which talk(s) to submit for Flink Forward San Francisco because the Call for Presentations is closing this

[jira] [Created] (FLINK-15489) Cannot update jobmanager/taskmanager logs

2020-01-06 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-15489: Summary: Cannot update jobmanager/taskmanager logs Key: FLINK-15489 URL: https://issues.apache.org/jira/browse/FLINK-15489 Project: Flink Issue

[jira] [Created] (FLINK-15488) Cannot start a taskmanger if using logback

2020-01-06 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-15488: Summary: Cannot start a taskmanger if using logback Key: FLINK-15488 URL: https://issues.apache.org/jira/browse/FLINK-15488 Project: Flink Issue

[jira] [Created] (FLINK-15487) Expose the new type inference for scalar functions

2020-01-06 Thread Timo Walther (Jira)
Timo Walther created FLINK-15487: Summary: Expose the new type inference for scalar functions Key: FLINK-15487 URL: https://issues.apache.org/jira/browse/FLINK-15487 Project: Flink Issue

Re: [DISCUSS] JDBC exactly-once sink

2020-01-06 Thread Piotr Nowojski
Hi, Also +1 for using the XA. There might be scenarios where WAL could be a better option, but I think XA should be the default/first choice. If there will be a bigger demand for WAL, we can always provide it as an alternative. As Jingsong mentioned, with WAL I would be worried about batch

[jira] [Created] (FLINK-15486) LocalInputChannelTest.testConcurrentConsumeMultiplePartitions failing on ppc64le platform.

2020-01-06 Thread Siddhesh Ghadi (Jira)
Siddhesh Ghadi created FLINK-15486: -- Summary: LocalInputChannelTest.testConcurrentConsumeMultiplePartitions failing on ppc64le platform. Key: FLINK-15486 URL: https://issues.apache.org/jira/browse/FLINK-15486

Re: [VOTE] Rename terminology "Time-windowed Join" to "Interval Join" in Table API & SQL

2020-01-06 Thread Timo Walther
If the semantics of both APIs are 100% equivalent, I'm fine with the renaming. Have you checked that? Regards, Timo On 06.01.20 10:46, Zhenghua Gao wrote: +1 to align the terminology. *Best Regards,* *Zhenghua Gao* On Fri, Jan 3, 2020 at 12:59 PM Jingsong Li wrote: +1 for this

[jira] [Created] (FLINK-15485) Reopen tests like 'ignore TODO/when FLINK-xx is closed/when FLINK-xx is merged' has been finished

2020-01-06 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-15485: -- Summary: Reopen tests like 'ignore TODO/when FLINK-xx is closed/when FLINK-xx is merged' has been finished Key: FLINK-15485 URL: https://issues.apache.org/jira/browse/FLINK-15485

Re: [VOTE] Rename terminology "Time-windowed Join" to "Interval Join" in Table API & SQL

2020-01-06 Thread Zhenghua Gao
+1 to align the terminology. *Best Regards,* *Zhenghua Gao* On Fri, Jan 3, 2020 at 12:59 PM Jingsong Li wrote: > +1 for this documentation change. > Hope less confuse to users. > > Best, > Jingsong Lee > > On Fri, Jan 3, 2020 at 12:09 PM Benchao Li wrote: > > > +1 > > > > It's good to align

Re: [DISCUSS] JDBC exactly-once sink

2020-01-06 Thread Jingsong Li
Thanks Roman for driving this. Although Upsert sink can bring some consistency guarantee to JDBC sink, users have strong requirements to append exactly-once sink too. +1 to use XA distribution transaction. For WAL, as far as my experience is concerned, writing JDBC in large quantities when

[jira] [Created] (FLINK-15484) How does the CEP of flink compare the current data of yesterday with the reference data? Is there a better way to get it continuously from ES

2020-01-06 Thread smallyin (Jira)
smallyin created FLINK-15484: Summary: How does the CEP of flink compare the current data of yesterday with the reference data? Is there a better way to get it continuously from ES Key: FLINK-15484 URL:

[DISCUSS] JDBC exactly-once sink

2020-01-06 Thread Roman Khachatryan
Hi everyone, I'm currently working on exactly-once JDBC sink implementation for Flink. Any ideas and/or feedback are welcome. I've considered the following options: 1. Two-phase commit. This is similar to Kafka sink. XA or database-specific API can be used. In case of XA, each sink subtask acts

[jira] [Created] (FLINK-15483) Support to set environment variables for jobmanager pod

2020-01-06 Thread Yang Wang (Jira)
Yang Wang created FLINK-15483: - Summary: Support to set environment variables for jobmanager pod Key: FLINK-15483 URL: https://issues.apache.org/jira/browse/FLINK-15483 Project: Flink Issue