[jira] [Created] (FLINK-14301) add documentation for temp and temp system functions and new function resolution order

2019-09-30 Thread Bowen Li (Jira)
Bowen Li created FLINK-14301: Summary: add documentation for temp and temp system functions and new function resolution order Key: FLINK-14301 URL: https://issues.apache.org/jira/browse/FLINK-14301

Re: [VOTE] FLIP-57: Rework FunctionCatalog

2019-09-30 Thread Bowen Li
Hi all, I've updated the FLIP wiki with the following changes: - Lifespan of temp functions are not tied to those of catalogs and databases. Users can create temp functions even though catalogs/dbs in their fully qualified names don't even exist. - some new SQL commands - "SHOW FUNCTIONS" -

Re: [DISCUSS] FLIP-68: Extend Core Table System with Modular Plugins

2019-09-30 Thread Bowen Li
Hi Timo, Re 1) I agree. I renamed the title to "Extend Core Table System with Pluggable Modules" and all internal references Re 2) First, I'll rename the API to useModules(). The design doesn't forbid users to call useModules() multi times. Objects in modules are loaded on demand instead of

Re: [VOTE] FLIP-57: Rework FunctionCatalog

2019-09-30 Thread Bowen Li
Hi, I think above are some valid points, and we can adopt the suggestions. To elaborate a bit on the new SQL syntax, it would imply that, unlike "SHOW FUNCTION" which only return function names, "SHOW ALL [TEMPORARY] FUNCTIONS" would return functions' fully qualified names with catalog and db

[jira] [Created] (FLINK-14300) org.apache.flink.streaming.runtime.tasks.StreamTask#invoke leaks threads if org.apache.flink.streaming.runtime.tasks.OperatorChain fails to be constructed

2019-09-30 Thread Marcos Klein (Jira)
Marcos Klein created FLINK-14300: Summary: org.apache.flink.streaming.runtime.tasks.StreamTask#invoke leaks threads if org.apache.flink.streaming.runtime.tasks.OperatorChain fails to be constructed Key: FLINK-14300

[jira] [Created] (FLINK-14299) Factor status and system metrics out of JobManagerMetricGroup

2019-09-30 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-14299: - Summary: Factor status and system metrics out of JobManagerMetricGroup Key: FLINK-14299 URL: https://issues.apache.org/jira/browse/FLINK-14299 Project: Flink

Re: REST API / JarRunHandler: More flexibility for launching jobs

2019-09-30 Thread Thomas Weise
The following methods of job submission are relevant (but first two are just general Flink): 1) Flink CLI that submits the job from a Java client environment. 2) Non-JVM tools like the FlinkK8sOperator that use the REST API to launch the job (and optionally also upload the jar, which otherwise

Re: [DISCUSS] FLIP-68: Extend Core Table System with Modular Plugins

2019-09-30 Thread Timo Walther
Hi Bowen, thanks for this proposal after our discussion around the FunctionCatalog rework. I like the architecture proposed in the FLIP because it is also based on existing concepts and just slightly modifies the code base. However, I would like to discuss some unanswered questions: 1)

[jira] [Created] (FLINK-14298) Change LeaderContender#getAddress into LeaderContender#getDescription

2019-09-30 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-14298: - Summary: Change LeaderContender#getAddress into LeaderContender#getDescription Key: FLINK-14298 URL: https://issues.apache.org/jira/browse/FLINK-14298 Project:

Re: [VOTE] FLIP-68: Extend Core Table System with Modular Plugins

2019-09-30 Thread Timo Walther
Hi Bowen, thanks for postponing the voting and sorry for the inconvenience. For the future, we should avoid starting voting threads if there hasn't been a single response in the [DISCUSS] thread. Instead, the owner of the FLIP should proactively try to ping people for feedback. Even if the

[jira] [Created] (FLINK-14297) Temporal Table Function Build Side does not accept a constant key

2019-09-30 Thread Jira
Benoît Paris created FLINK-14297: Summary: Temporal Table Function Build Side does not accept a constant key Key: FLINK-14297 URL: https://issues.apache.org/jira/browse/FLINK-14297 Project: Flink

Re: [DISCUSS] FLIP-54: Evolve ConfigOption and Configuration

2019-09-30 Thread Timo Walther
Hi all, thanks for all the feedback we have received so far. Also from the offline feedback that I have received so far, it seems this is a very sensitive topic. I think we have reached consensus that we want to improve the configuration experience in Flink. We just need to figure out the

Re: [VOTE] FLIP-57: Rework FunctionCatalog

2019-09-30 Thread Timo Walther
Hi all, I support Fabian's arguments. In my opinion, temporary objects should just be an additional layer on top of the regular catalog/database lookup logic. Thus, a temporary table or function has always highest precedence and should be stable within the local session. Otherwise it could

[jira] [Created] (FLINK-14296) SqlNodes in the parser module should use an Optional for optional parameters

2019-09-30 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-14296: Summary: SqlNodes in the parser module should use an Optional for optional parameters Key: FLINK-14296 URL: https://issues.apache.org/jira/browse/FLINK-14296

Re: [VOTE] FLIP-57: Rework FunctionCatalog

2019-09-30 Thread Fabian Hueske
Hi all, Sorry for the late reply. I think it might lead to confusing situations if temporary functions (or any temporary db objects for that matter) are bound to the life cycle of an (external) db/catalog. Imaging a situation where you create a temp function in a db in an external catalog and

[jira] [Created] (FLINK-14295) Nightly flink-runtime failed with java 11

2019-09-30 Thread Yu Li (Jira)
Yu Li created FLINK-14295: - Summary: Nightly flink-runtime failed with java 11 Key: FLINK-14295 URL: https://issues.apache.org/jira/browse/FLINK-14295 Project: Flink Issue Type: Bug

[DISCUSS] FLIP-76: Unaligned checkpoints

2019-09-30 Thread Arvid Heise
Hi Devs, I would like to start the formal discussion about FLIP-76 [1], which improves the checkpoint latency in systems under backpressure, where a checkpoint can take hours to complete in the worst case. I recommend the thread "checkpointing under backpressure" [2] to get a good idea why users

[jira] [Created] (FLINK-14294) Nightly flink-quickstart-scala end-to-end test failed due to missing javax.tools.ToolProvider

2019-09-30 Thread Yu Li (Jira)
Yu Li created FLINK-14294: - Summary: Nightly flink-quickstart-scala end-to-end test failed due to missing javax.tools.ToolProvider Key: FLINK-14294 URL: https://issues.apache.org/jira/browse/FLINK-14294

Re: [DISCUSS] FLIP-75: Flink Web UI Improvement Proposal

2019-09-30 Thread Xintong Song
@Yadong 2. I agree that we can update the task executor ui after flip-56 is done. But I would suggest keep it on discussion to come up with a proper ui design for task executor resources. I don't think the mentioned image from flip-56 is a good choice. That image is a simplified figure with cpu

[jira] [Created] (FLINK-14293) Lazy execution of queries might produce wrong results

2019-09-30 Thread Jark Wu (Jira)
Jark Wu created FLINK-14293: --- Summary: Lazy execution of queries might produce wrong results Key: FLINK-14293 URL: https://issues.apache.org/jira/browse/FLINK-14293 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-14292) Provide terasort examples

2019-09-30 Thread liupengcheng (Jira)
liupengcheng created FLINK-14292: Summary: Provide terasort examples Key: FLINK-14292 URL: https://issues.apache.org/jira/browse/FLINK-14292 Project: Flink Issue Type: Improvement

Re: [DISCUSS] FLIP-54: Evolve ConfigOption and Configuration

2019-09-30 Thread Kostas Kloudas
Hi all, I would also like to give a +1 for supporting lists as config options with the delimeter being a parameter (if we cannot find a consensus). To some extent the current codebase has already solved the issue by already having lists as options, but the problem is that so far there was no

Re: [DISCUSS] Remove uncompleted YARNHighAvailabilityService

2019-09-30 Thread Zili Chen
Thanks for your reply Till. I will wait a bit for other thoughts, and create JIRA and start progress if no further objections. Best, tison. Till Rohrmann 于2019年9月30日周一 下午5:51写道: > Hi Tison, > > I agree that unused HA implementations can be removed since they are dead > code. If we should

Re: [DISCUSS] FLIP-75: Flink Web UI Improvement Proposal

2019-09-30 Thread Till Rohrmann
For 3. At the moment the log and stdout file serving requires the TaskExecutor to be running. But in some scenarios when having a NFS, it should be enough to know where the file is located. However, this assumption does not hold in the general case. Cheers, Till On Mon, Sep 30, 2019 at 11:43 AM

Re: [DISCUSS] Remove uncompleted YARNHighAvailabilityService

2019-09-30 Thread Till Rohrmann
Hi Tison, I agree that unused HA implementations can be removed since they are dead code. If we should need them in the future, then we can still get them by going back a bit in time. Hence +1 for removing unused HA implementations. Cheers, Till On Mon, Sep 30, 2019 at 10:42 AM Zili Chen

Re: [DISCUSS] FLIP-75: Flink Web UI Improvement Proposal

2019-09-30 Thread Yadong Xie
Hi Xintong Song Thanks for your comments! 1. I think it is a good idea that to align CPU and memory usage with FLIP-49 if it will release in version 1.10 2. We can update the task executor UI design after FLIP-56 merged into master. Actually, the image

Re: [COMMITTER] repo locked due to synchronization issues

2019-09-30 Thread Till Rohrmann
Thanks Rong for volunteering. From my side +1 for using Github repo. Cheers, Till On Fri, Sep 27, 2019 at 9:58 PM Thomas Weise wrote: > +1 for recommendation to use the github repo > > Thanks, > Thomas > > On Fri, Sep 27, 2019 at 9:29 AM Rong Rong wrote: > > > +1 on to state with one

[jira] [Created] (FLINK-14291) Add unit tests for DefaultScheduler to test concurrent failover behavior

2019-09-30 Thread Gary Yao (Jira)
Gary Yao created FLINK-14291: Summary: Add unit tests for DefaultScheduler to test concurrent failover behavior Key: FLINK-14291 URL: https://issues.apache.org/jira/browse/FLINK-14291 Project: Flink

Re: [DISCUSS] Remove uncompleted YARNHighAvailabilityService

2019-09-30 Thread Zili Chen
Any suggestion? IMO it is exactly inactive for quite some time, we can remove these uncompleted codes at least for now and re-introduce if needed. Best, tison. Zili Chen 于2019年9月27日周五 上午9:23写道: > Hi devs, > > Noticed that there are several stale & uncompleted high-availability > services

[jira] [Created] (FLINK-14290) Decouple plan translation from job execution/ClusterClient

2019-09-30 Thread Aljoscha Krettek (Jira)
Aljoscha Krettek created FLINK-14290: Summary: Decouple plan translation from job execution/ClusterClient Key: FLINK-14290 URL: https://issues.apache.org/jira/browse/FLINK-14290 Project: Flink

[VOTE] Release 1.9.1, release candidate #1

2019-09-30 Thread Jark Wu
Hi everyone, Please review and vote on the release candidate #1 for the version 1.9.1, as follows: [ ] +1, Approve the release [ ] -1, Do not approve the release (please provide specific comments) The complete staging area is available for your review, which includes: * JIRA release notes [1],

Re: [DISCUSS] Releasing Flink 1.9.1

2019-09-30 Thread Jark Wu
Thanks Hequn for the reviewing and merging. I'm preparing the RC1 for 1.9.1. Cheers, Jark On Mon, 30 Sep 2019 at 11:59, Hequn Cheng wrote: > Hi, > > @jincheng sun @Dian Fu > Thanks > a lot for reporting and fixing the problem! > @Jark Wu The PR of FLINK-14288 has been merged into > both

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

2019-09-30 Thread Becket Qin
Forgot to say that I agree with Till that it seems a good idea to let TEs register the global partitions to the RM instead of letting JM do it. This simplifies quite a few things. Thanks, Jiangjie (Becket) Qin On Sun, Sep 29, 2019 at 11:25 PM Becket Qin wrote: > Hi Chesnay, > > Thanks for the

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

2019-09-30 Thread Becket Qin
Hi Chesnay, Thanks for the proposal. My understanding of the entire workflow step by step is following: - JM maintains the local and global partition metadata when the task runs to create result partitions. The tasks themselves does not distinguish between local / global partitions. Only the