Re: [DISCUSS] Support notifyOnMaster for notifyCheckpointComplete

2019-09-10 Thread Stephan Ewen
Hi all! I think it would be time to rethink the Sink API as a whole, like we did with the Source API in FLIP-27. It would be nice to have proper design that handles all this consistently, rather than adding one more hook. For example: - For batch, you can already use the existing "finalize on m

[jira] [Created] (FLINK-14039) Flink Kinesis consumer: configurable per-shard consumption rate when running in adaptive mode

2019-09-10 Thread Ying Xu (Jira)
Ying Xu created FLINK-14039: --- Summary: Flink Kinesis consumer: configurable per-shard consumption rate when running in adaptive mode Key: FLINK-14039 URL: https://issues.apache.org/jira/browse/FLINK-14039 P

Re: [DISCUSS] FLIP-67: Global partitions lifecycle

2019-09-10 Thread Till Rohrmann
Thanks Chesnay for drafting the FLIP and starting this discussion. I have a couple of comments: * I know that I've also coined the terms global/local result partition but maybe it is not the perfect name. Maybe we could rethink the terminology and call them persistent result partitions? * Nit: I

Re: ClassLoader created by BlobLibraryCacheManager is not using context classloader

2019-09-10 Thread Till Rohrmann
Hi Jan, sorry for my late response. I think the main concern about using the context class loader is the unpredictability and magic it adds to a component where you actually don't wanna be surprised. Moreover, if we now add support for the context class loader, then at a later time another compone

Re: [DISCUSS] Retrieval services in non-high-availability scenario

2019-09-10 Thread Zili Chen
Hi Till, Thanks for your reply. I agree point 3 and 4 in your email worth a separated thread to discuss. Let me answer your questions and concerns in point 1 and 2 respectively. 1.Lifecycle of LeaderServer and requirement to implement it LeaderServer starts on cluster entrypoint and its lifecycl

[jira] [Created] (FLINK-14040) Enable a cron job to run MiniCluster tests for schedulerNG

2019-09-10 Thread Zhu Zhu (Jira)
Zhu Zhu created FLINK-14040: --- Summary: Enable a cron job to run MiniCluster tests for schedulerNG Key: FLINK-14040 URL: https://issues.apache.org/jira/browse/FLINK-14040 Project: Flink Issue Type:

Re: [DISCUSS] Contribute Pulsar Flink connector back to Flink

2019-09-10 Thread Yijie Shen
Hi everyone! Thanks for your attention and the promotion of this work. We will prepare a FLIP as soon as possible for more specific discussions. For FLIP-27, it seems that we have not reached a consensus. Therefore, I will explain all the functionalities of the existing connector in the FLIP (in

[jira] [Created] (FLINK-14041) Refactor LeaderRetrievalServiceHostnameResolutionTest and remove StandaloneUtils

2019-09-10 Thread TisonKun (Jira)
TisonKun created FLINK-14041: Summary: Refactor LeaderRetrievalServiceHostnameResolutionTest and remove StandaloneUtils Key: FLINK-14041 URL: https://issues.apache.org/jira/browse/FLINK-14041 Project: Fli

Re: [DISCUSS] Retrieval services in non-high-availability scenario

2019-09-10 Thread Till Rohrmann
Hi Tison, thanks for the detailed response. I put some comments inline: On Tue, Sep 10, 2019 at 10:51 AM Zili Chen wrote: > Hi Till, > > Thanks for your reply. I agree point 3 and 4 in your email worth a > separated > thread to discuss. Let me answer your questions and concerns in point 1 and >

Re: [DISCUSS] Contribute Pulsar Flink connector back to Flink

2019-09-10 Thread Stephan Ewen
Hi all! Nice to see this lively discussion about the Pulsar connector. Some thoughts on the open questions: ## Contribute to Flink or maintain as a community package Looks like the discussion is more going towards contribution. I think that is good, especially if we think that we want to build a

How to implement grouping set in stream

2019-09-10 Thread 刘建刚
I want to implement grouping set in stream. I am new to flink sql. I want to find a example to teach me how to self define rule and implement corresponding operator. Can anyone give me any suggestion?

Re: [DISCUSS] Retrieval services in non-high-availability scenario

2019-09-10 Thread Zili Chen
Hi Till, Thanks for your quick reply. I'd like to narrow the intention of this thread as I posted above >Well, I see your concerns on replace existing stable services hurriedly with a new implementation. Here I list the pros and cons of this replacement. If we agree that it does good I can provid

[jira] [Created] (FLINK-14042) Different RelDataTypes generated for same TemporalTableFunction

2019-09-10 Thread Yuval Itzchakov (Jira)
Yuval Itzchakov created FLINK-14042: --- Summary: Different RelDataTypes generated for same TemporalTableFunction Key: FLINK-14042 URL: https://issues.apache.org/jira/browse/FLINK-14042 Project: Flink

[jira] [Created] (FLINK-14043) SavepointMigrationTestBase is super slow

2019-09-10 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-14043: - Summary: SavepointMigrationTestBase is super slow Key: FLINK-14043 URL: https://issues.apache.org/jira/browse/FLINK-14043 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-14044) Reducing synchronization in AsyncWaitOperator

2019-09-10 Thread Arvid Heise (Jira)
Arvid Heise created FLINK-14044: --- Summary: Reducing synchronization in AsyncWaitOperator Key: FLINK-14044 URL: https://issues.apache.org/jira/browse/FLINK-14044 Project: Flink Issue Type: Impro

[jira] [Created] (FLINK-14045) Rewrite DefaultExecutionSlotAllocator to use SlotProviderStrategy

2019-09-10 Thread Gary Yao (Jira)
Gary Yao created FLINK-14045: Summary: Rewrite DefaultExecutionSlotAllocator to use SlotProviderStrategy Key: FLINK-14045 URL: https://issues.apache.org/jira/browse/FLINK-14045 Project: Flink Is

[jira] [Created] (FLINK-14046) DDL property 'format.fields.#.type' should ignore case

2019-09-10 Thread hailong wang (Jira)
hailong wang created FLINK-14046: Summary: DDL property 'format.fields.#.type' should ignore case Key: FLINK-14046 URL: https://issues.apache.org/jira/browse/FLINK-14046 Project: Flink Issue

Re: [VOTE] Release 1.8.2, release candidate #1

2019-09-10 Thread Till Rohrmann
+1 (binding) - verified checksums and signatures - no binary files in source release - built Flink from source release with Scala 2.12, running all tests - Verified that no new dependencies have been added - Executed simple example jobs locally (worked) Cheers, Till On Tue, Sep 10, 2019 at 8:00

[jira] [Created] (FLINK-14047) Hide secret values when displaying user configuration/global job parameters in web UI

2019-09-10 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-14047: - Summary: Hide secret values when displaying user configuration/global job parameters in web UI Key: FLINK-14047 URL: https://issues.apache.org/jira/browse/FLINK-14047

Re: [DISCUSS] modular built-in functions

2019-09-10 Thread Xuefu Z
HI Bowen, Thank you for sharing your research summaries. The concerns you raised about the modular approach are very valuable and practical. Here are some of my thoughts.. 1. Naming conflicts and resolution. Naming conflicts is likely, and as you suggested, the resolution can be just based on the

Re: [DISCUSS] Support notifyOnMaster for notifyCheckpointComplete

2019-09-10 Thread JingsongLee
+1 to rethink Best, Jingsong Lee -- From:Stephan Ewen Send Time:2019年9月10日(星期二) 15:40 To:dev Cc:JingsongLee Subject:Re: [DISCUSS] Support notifyOnMaster for notifyCheckpointComplete Hi all! I think it would be time to rethink t

Re: [DISCUSS] FLIP-63: Rework table partition support

2019-09-10 Thread JingsongLee
Hi biao, thanks for your feedbacks: Actually, the runtime source partition of runtime is similar to split, which concerns data reading, parallelism and fault tolerance, all the runtime concepts. While table partition is only a virtual concept. Users are more likely to choose which partition to

[jira] [Created] (FLINK-14048) Flink client hangs after trying to kill Yarn Job during deployment

2019-09-10 Thread Gyula Fora (Jira)
Gyula Fora created FLINK-14048: -- Summary: Flink client hangs after trying to kill Yarn Job during deployment Key: FLINK-14048 URL: https://issues.apache.org/jira/browse/FLINK-14048 Project: Flink