[jira] [Created] (FLINK-16416) Shutdown the task manager gracefully in standalone mode

2020-03-03 Thread Yangze Guo (Jira)
Yangze Guo created FLINK-16416: -- Summary: Shutdown the task manager gracefully in standalone mode Key: FLINK-16416 URL: https://issues.apache.org/jira/browse/FLINK-16416 Project: Flink Issue Typ

Re: [DISCUSS] FLIP-109: Improve Hive dependencies out-of-box experience

2020-03-03 Thread Jingsong Lee
Hi all, After some offline discussion, we have some divergence about using lower version hive client to support higher version hive server. The question is whether the lost feature of higher hive version is worth it. I would like to pause this FLIP, until reach a direct agreement, or one day we c

[jira] [Created] (FLINK-16415) Add a version property to module.yaml

2020-03-03 Thread Igal Shilman (Jira)
Igal Shilman created FLINK-16415: Summary: Add a version property to module.yaml Key: FLINK-16415 URL: https://issues.apache.org/jira/browse/FLINK-16415 Project: Flink Issue Type: Improvement

Re: Flink Web UI display nothing in k8s when use ingress

2020-03-03 Thread LakeShen
In my thought , I think I should config the correct flink jobserver for flink task LakeShen 于2020年3月4日周三 下午2:07写道: > Hi community, > now we plan to move all flink tasks to k8s cluster. For one flink > task , we want to see this flink task web ui . First , we create the k8s > Service to e

Re: Flink dev blog

2020-03-03 Thread Yu Li
Big +1 on adding a dev blog and starting with wiki. And +1 to promote the fully polished articles to blog web with a formal process. The latter one also brings up another good-to-have improvement that adding categories and navigation in our blog so people could easily find different topics like re

Re: Contributor permission application

2020-03-03 Thread Yu Li
Welcome to the Flink community yutao! In addition to what Congxian mentioned, you may also find the guide of code contribution process [1] useful, especially the "Requirements for a Jira ticket to get consensus" part. Best Regards, Yu [1] https://flink.apache.org/contributing/contribute-code.htm

Flink Web UI display nothing in k8s when use ingress

2020-03-03 Thread LakeShen
Hi community, now we plan to move all flink tasks to k8s cluster. For one flink task , we want to see this flink task web ui . First , we create the k8s Service to expose 8081 port of jobmanager, then we use ingress controller so that we can see it outside.But the flink web like this : [im

Re: [DISCUSS] FLIP-110: Support LIKE clause in CREATE TABLE

2020-03-03 Thread Dawid Wysakowicz
Hi Jark, I did investigate the INHERITS clause, but it has a semantic that in my opinion we definitely don't want to support. INHERITS creates a new table with a "link" to the original table. Therefore if you e.g change the schema of the original table it's also reflected in the child table. It's a

[jira] [Created] (FLINK-16414) create udaf/udtf function using sql casuing ValidationException: SQL validation failed. null

2020-03-03 Thread Terry Wang (Jira)
Terry Wang created FLINK-16414: -- Summary: create udaf/udtf function using sql casuing ValidationException: SQL validation failed. null Key: FLINK-16414 URL: https://issues.apache.org/jira/browse/FLINK-16414

[jira] [Created] (FLINK-16413) Reduce hive source parallelism when limit push down

2020-03-03 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-16413: Summary: Reduce hive source parallelism when limit push down Key: FLINK-16413 URL: https://issues.apache.org/jira/browse/FLINK-16413 Project: Flink Issue Typ

Re: [VOTE] FLIP-100: Add Attempt Information

2020-03-03 Thread Jark Wu
+1 from my side. Best, Jark On Wed, 4 Mar 2020 at 11:39, Kurt Young wrote: > LGTM now, +1 from my side. > > Best, > Kurt > > > On Wed, Mar 4, 2020 at 12:27 AM Gary Yao wrote: > >> Hi Yadong, >> >> Thank you for updating the wiki page. >> >> Only one minor suggestion – I would change: >> >>

Re: Creating TemporalTable based on Catalog table in SQL Client

2020-03-03 Thread Bowen Li
Hi Gyula, What line 622 (the link you shared) does is not registering catalogs, but setting an already registered catalog as the current one. As you can see from the method and its comment, catalogs are loaded first before any tables in yaml are registered, so you should be able to achieve what yo

Re: [DISCUSS] FLIP-110: Support LIKE clause in CREATE TABLE

2020-03-03 Thread Jark Wu
Hi Dawid, Thanks for starting this discussion. I like the idea. Once we support more intergrated catalogs, e.g. ConfluentSchemaRegistryCatalog, this problem will be more urgent. Because it's very common to adjust existing tables in catalog slightly. My initial thought was introducing INHERITS key

Re: [VOTE] FLIP-100: Add Attempt Information

2020-03-03 Thread Kurt Young
LGTM now, +1 from my side. Best, Kurt On Wed, Mar 4, 2020 at 12:27 AM Gary Yao wrote: > Hi Yadong, > > Thank you for updating the wiki page. > > Only one minor suggestion – I would change: > > > If show-history is true return the information of attempt. > > to > > > If show-history is

Re: [DISCUSS] FLIP-108: Add GPU support in Flink

2020-03-03 Thread Yangze Guo
Thanks for all the feedbacks. @Becket Regarding the WebUI and GPUInfo, you're right, I'll add them to the Public API section. @Stephan @Becket Regarding the general extended resource mechanism, I second Xintong's suggestion. - It's better to leverage ResourceProfile and ResourceSpec after we sup

Re: Flink dev blog

2020-03-03 Thread Xintong Song
I also like Ufuk's idea. The wiki allows people to post on their works in a quick and easier way. For me and probably many other Chinese folks, writing and polishing a formal article in English usually takes a long time, of which a significant portion is spent on polishing the language. If the blo

[jira] [Created] (FLINK-16412) Disallow embedded metastore in HiveCatalog production code

2020-03-03 Thread Rui Li (Jira)
Rui Li created FLINK-16412: -- Summary: Disallow embedded metastore in HiveCatalog production code Key: FLINK-16412 URL: https://issues.apache.org/jira/browse/FLINK-16412 Project: Flink Issue Type: Bu

Re: [DISCUSS] FLIP-108: Add GPU support in Flink

2020-03-03 Thread Xingbo Huang
Thanks a lot for the FLIP, Yangze. There is no doubt that GPU resource management support will greatly facilitate the development of AI-related applications by PyFlink users. I have only one comment about this wiki: Regarding the names of several GPU configurations, I think it is better to delet

Re: [DISCUSS] FLIP-108: Add GPU support in Flink

2020-03-03 Thread Xintong Song
@Stephan, @Becket, Actually, Yangze, Yang and I also had an offline discussion about making the "GPU Support" as some general "Extended Resource Support". We believe supporting extended resources in a general mechanism is definitely a good and extensible way. The reason we propose this FLIP narrow

Re: Flink dev blog

2020-03-03 Thread Jark Wu
+1 for this. Regarding to the place to hold blogs. Personally, I prefer to use existing blog and separate by tags/categories and title names. Because, the dev blogs are very good learning materials. I believe many users will be interested in these posts. It's just like "Technology Deep Dive" talks

Re: [DISCUSS] FLIP-108: Add GPU support in Flink

2020-03-03 Thread Becket Qin
That's a good point, Stephan. It makes total sense to generalize the resource management to support custom resources. Having that allows users to add new resources by themselves. The general resource management may involve two different aspects: 1. The custom resource type definition. It is suppor

[jira] [Created] (FLINK-16411) Use google maven mirrors in all maven invocations on AZP

2020-03-03 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16411: -- Summary: Use google maven mirrors in all maven invocations on AZP Key: FLINK-16411 URL: https://issues.apache.org/jira/browse/FLINK-16411 Project: Flink

[jira] [Created] (FLINK-16410) PrometheusReporterEndToEndITCase fails with ClassNotFoundException

2020-03-03 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16410: -- Summary: PrometheusReporterEndToEndITCase fails with ClassNotFoundException Key: FLINK-16410 URL: https://issues.apache.org/jira/browse/FLINK-16410 Project: Flink

Re: Flink dev blog

2020-03-03 Thread Ufuk Celebi
+1 on starting with the Wiki. I really like the name "Engine room". Can we name the section in the Wiki like that? In general, if we think that a post or a series of posts would be a good fit for the main blog, it would be pretty straightforward to promote a post from the Engine room to the main bl

[jira] [Created] (FLINK-16409) Use LinkedHashMap for deterministic iterations

2020-03-03 Thread testfixer0 (Jira)
testfixer0 created FLINK-16409: -- Summary: Use LinkedHashMap for deterministic iterations Key: FLINK-16409 URL: https://issues.apache.org/jira/browse/FLINK-16409 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-16408) Bind user code class loader to lifetime of a slot

2020-03-03 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-16408: - Summary: Bind user code class loader to lifetime of a slot Key: FLINK-16408 URL: https://issues.apache.org/jira/browse/FLINK-16408 Project: Flink Issue Typ

Re: SerializableHadoopConfiguration

2020-03-03 Thread Sivaprasanna
BTW, can we leverage flink-shaded-hadoop-2? Reason why I ask, if any Flink module is going to use Hadoop in any way, it will most probably include flink-shaded-hadoop-2 as a dependency. However, flink-shaded modules don't have any source files. Is that a strict convention that the community follows

Re: SerializableHadoopConfiguration

2020-03-03 Thread Sivaprasanna
Hi Arvid, Thanks for the quick reply. Yes, it actually makes sense to avoid Hadoop dependencies from getting into Flink's core modules but I also wonder if it will be an overkill to add flink-hadoop-fs as a dependency just because we want to use a utility class from that module. - Sivaprasanna O

RE: Building with Hadoop 3

2020-03-03 Thread LINZ, Arnaud
Hello, Have you shared it somewhere on the web already? Best, Arnaud De : vino yang Envoyé : mercredi 4 décembre 2019 11:55 À : Márton Balassi Cc : Chesnay Schepler ; Foster, Craig ; u...@flink.apache.org; dev@flink.apache.org Objet : Re: Building with Hadoop 3 Hi Marton, Thanks for your expl

Re: Flink dev blog

2020-03-03 Thread Rong Rong
Big +1 on this. Some of these topics are not only for contributors, but would also be super useful for advance users. One topic I can think of in addition is: Security/Kerberos. Echo on Both Seth's idea, we could have both wiki and PR submission: As Robert mentioned - wiki submission would make th

Re: Flink dev blog

2020-03-03 Thread Dian Fu
Big +1 on this idea. It will benefit both the developers and users a lot. Regarding to the place to hold these blogs, my preference is 3) as I notice that there are already a few high quality blogs on flink web-site[1] and I guess that may be a good place to start with. We just need to figure ou

Re: [DISCUSS] FLIP-108: Add GPU support in Flink

2020-03-03 Thread Stephan Ewen
Thank you for writing this FLIP. I cannot really give much input into the mechanics of GPU-aware scheduling and GPU allocation, as I have no experience with that. One thought I had when reading the proposal is if it makes sense to look at the "GPU Manager" as an "External Resource Manager", and G

Re: [VOTE] FLIP-100: Add Attempt Information

2020-03-03 Thread Gary Yao
Hi Yadong, Thank you for updating the wiki page. Only one minor suggestion – I would change: > If show-history is true return the information of attempt. to > If show-history is true, information for all attempts including previous ones will be returned That being said, FLIP-100 looks

[jira] [Created] (FLINK-16407) Remove "FAQ" section from flink website

2020-03-03 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16407: -- Summary: Remove "FAQ" section from flink website Key: FLINK-16407 URL: https://issues.apache.org/jira/browse/FLINK-16407 Project: Flink Issue Type: Task

[jira] [Created] (FLINK-16406) Increase default value for JVM Metaspace to minimise its OutOfMemoryError

2020-03-03 Thread Andrey Zagrebin (Jira)
Andrey Zagrebin created FLINK-16406: --- Summary: Increase default value for JVM Metaspace to minimise its OutOfMemoryError Key: FLINK-16406 URL: https://issues.apache.org/jira/browse/FLINK-16406 Proje

Re: Flink dev blog

2020-03-03 Thread Yadong Xie
Hi all maybe we can use markdown & GitHub to make the submission easy to review I have set up a similar blog for Flink-china blog before(deprecated), glad to offer help if needed here is the link: https://github.com/flink-china/doc Seth Wiesman 于2020年3月3日周二 下午10:51写道: > For lack of a better wa

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

2020-03-03 Thread Yang Wang
Hi Peter, Having the application mode does not mean we will drop the cluster-deploy option. I just want to share some thoughts about “Application Mode”. 1. The application mode could cover the per-job sematic. Its lifecyle is bound to the user `main()`. And all the jobs in the user main will be

Re: Flink dev blog

2020-03-03 Thread Robert Metzger
I fully agree with Seth! I proposed the Wiki to make the submission process as frictionless as possible. If we publish this on the official Flink website, we will have an involved process of slow reviews, incorporating feedback etc., making such a publication painful. In the wiki, people can basi

Re: Flink dev blog

2020-03-03 Thread Seth Wiesman
For lack of a better way to put this, I think the location depends on the level of effort you want to put into writing these articles. If they are informal design documents then I think the wiki is the way to go. If you want to have them be more polished then the existing blog. This means going t

Re: [DISCUSS] FLIP-107: Reading table columns from different parts of source records

2020-03-03 Thread Jark Wu
Thanks Dawid, I have two more questions. > SupportsMetadata Introducing SupportsMetadata sounds good to me. But I have some questions regarding to this interface. 1) How do the source know what the expected return type of each metadata? 2) Where to put the metadata fields? Append to the existing

Re: Flink dev blog

2020-03-03 Thread Arvid Heise
I think there is enough to positive to start setting it up. That begs the question: in which format. Following possibilities exist: 1) Use wiki as Robert pointed out. 2) Add new blog. 3) Use existing blog and separate by tags #user, #expert, #dev (can be mixed). Start page could filter on #user by

[jira] [Created] (FLINK-16405) Add hive-metastore dependency in connecting to hive docs

2020-03-03 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-16405: -- Summary: Add hive-metastore dependency in connecting to hive docs Key: FLINK-16405 URL: https://issues.apache.org/jira/browse/FLINK-16405 Project: Flink Issue Ty

Re: [DISCUSS] Remove Eclipse-specific plugin configurations

2020-03-03 Thread Chesnay Schepler
You may very well may be ;) This gives us one more reason to remove this stuff. On 03/03/2020 14:52, Flavio Pompermaier wrote: Yes, in my experience.. I always asked myself if I was the only one using Eclipse.. :D On Tue, Mar 3, 2020 at 2:33 PM Chesnay Schepler > wr

[jira] [Created] (FLINK-16404) Solve the potential deadlock problem when reducing exclusive buffers to zero

2020-03-03 Thread Zhijiang (Jira)
Zhijiang created FLINK-16404: Summary: Solve the potential deadlock problem when reducing exclusive buffers to zero Key: FLINK-16404 URL: https://issues.apache.org/jira/browse/FLINK-16404 Project: Flink

Re: [DISCUSS] Remove Eclipse-specific plugin configurations

2020-03-03 Thread Flavio Pompermaier
Yes, in my experience.. I always asked myself if I was the only one using Eclipse.. :D On Tue, Mar 3, 2020 at 2:33 PM Chesnay Schepler wrote: > To clarify, the whole lifecycle-mapping business is both unnecessary and > actively harmful? > > On 03/03/2020 14:18, Flavio Pompermaier wrote: > > Sorr

[jira] [Created] (FLINK-16403) Solve the potential deadlock problem when reducing exclusive buffers to zero

2020-03-03 Thread Zhijiang (Jira)
Zhijiang created FLINK-16403: Summary: Solve the potential deadlock problem when reducing exclusive buffers to zero Key: FLINK-16403 URL: https://issues.apache.org/jira/browse/FLINK-16403 Project: Flink

Re: Flink dev blog

2020-03-03 Thread Piotr Nowojski
+1 for the idea :) And fully agree to clearly separate them. I think the original idea was writing about some recent changes in the Flink’s code base, that could affect other Flink developers (contributors/committers). Like for example some new ideas/future directions that we want to follow. Es

[jira] [Created] (FLINK-16402) Alter table fails on Hive catalog

2020-03-03 Thread Gyula Fora (Jira)
Gyula Fora created FLINK-16402: -- Summary: Alter table fails on Hive catalog Key: FLINK-16402 URL: https://issues.apache.org/jira/browse/FLINK-16402 Project: Flink Issue Type: Bug Compo

Re: Flink dev blog

2020-03-03 Thread Arvid Heise
Thx for the good feedback. I'd like to pursue the topic. Regarding the scope. How about we add a banner on each article that describes the intended audience and the scope? Example (two phase commit sink): This article is primarily intended for Flink contributors and expert users. Users would only

Re: [VOTE] FLIP-100: Add Attempt Information

2020-03-03 Thread Yadong Xie
Hi all The rest API part has been updated with Gary and Till's suggestions here is the link: https://cwiki.apache.org/confluence/display/FLINK/FLIP-100%3A+Add+Attempt+Information Yadong Xie 于2020年3月3日周二 下午9:14写道: > Hi Chesnay > > most discussions in this vote are about the more feature/demo req

Re: [DISCUSS] Remove Eclipse-specific plugin configurations

2020-03-03 Thread Chesnay Schepler
To clarify, the whole lifecycle-mapping business is both unnecessary and actively harmful? On 03/03/2020 14:18, Flavio Pompermaier wrote: Sorry for the late reply. I'm using the latest Eclipse (2019-12 R) and if I create a project using the Flink 1.10 archetype Eclipse doesn't reconstruct corre

Re: [DISCUSS] Remove Eclipse-specific plugin configurations

2020-03-03 Thread Flavio Pompermaier
Sorry for the late reply. I'm using the latest Eclipse (2019-12 R) and if I create a project using the Flink 1.10 archetype Eclipse doesn't reconstruct correctly the sources folders. If I remove the lifecycle-mapping plugin from the build section everything works as expected. About Flink developme

[jira] [Created] (FLINK-16401) Support map type for JsonRowSerializationSchema

2020-03-03 Thread Benchao Li (Jira)
Benchao Li created FLINK-16401: -- Summary: Support map type for JsonRowSerializationSchema Key: FLINK-16401 URL: https://issues.apache.org/jira/browse/FLINK-16401 Project: Flink Issue Type: Impro

Creating TemporalTable based on Catalog table in SQL Client

2020-03-03 Thread Gyula Fóra
Hi all! I was testing the TemporalTable functionality in the SQL client while using the Hive Catalog and I ran into the following problem. I have a table created in the Hive catalog and I want to create a temporal table over it. As we cannot create temporal tables in SQL directly I have to defin

Re: [VOTE] FLIP-100: Add Attempt Information

2020-03-03 Thread Yadong Xie
Hi Chesnay most discussions in this vote are about the more feature/demo request in POC or discussion about response format, the main proposal the web UI part which is not changed and the discussion about the response is converging, the response format discussion could happen either here or at th

[DISCUSS] FLIP-110: Support LIKE clause in CREATE TABLE

2020-03-03 Thread Dawid Wysakowicz
Hi devs, I wanted to bring another improvement proposal up for a discussion. Often users need to adjust existing tables slightly. This is especially useful when users need to enhance a table created from an external tool (e.g. HIVE) with Flink's specific information such as e.g watermarks. It can

[Reminder] Service Unavailable in JIRA

2020-03-03 Thread Xingbo Huang
Hi, all developers. Currently Service Unavailable in JIRA. Not only the flink community, other communities also cannot access JIRA Best, Xingbo

Re: [VOTE] FLIP-100: Add Attempt Information

2020-03-03 Thread Chesnay Schepler
I suggest to cancel this vote. Several discussion items have been brought up during the vote, some of which are still unresolved, others which resulted in changes to the proposal. My conclusion is that this proposal needs more discussions. On 20/02/2020 10:46, Yadong Xie wrote: Hi all I wa

Re: Flink dev blog

2020-03-03 Thread Jingsong Li
+1 for this proposal. I have a lot of desired topics in table and batch. I also second Seth and Stephan 's comment separate this in a clear way. Have concerns that maybe easy to confuse new users. If I am a beginner and find a bunch of deep documents, I need to further distinguish which is effecti

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

2020-03-03 Thread Kostas Kloudas
Hi Peter, I understand your point. This is why I was also a bit torn about the name and my proposal was a bit aligned with yours (something along the lines of "cluster deploy" mode). But many of the other participants in the discussion suggested the "Application Mode". I think that the reasoning

Re: SerializableHadoopConfiguration

2020-03-03 Thread Arvid Heise
Hi Sivaprasanna, we actually want to remove Hadoop from all core modules, so we could not place it in some very common place like flink-core. But I think the module flink-hadoop-fs could be a fitting place. On Tue, Mar 3, 2020 at 11:25 AM Sivaprasanna wrote: > Hi > > The flink-sequence-file mo

Re: Flink dev blog

2020-03-03 Thread Flavio Pompermaier
Big +1 from my side. I'd be very interested in what Jeff proposed, in particular everything related to client part (job submission, workflow management, callbacks on submission/success/failure, etc). Something I can't find anywhere is also how to query Flink states..would it be possible to have som

SerializableHadoopConfiguration

2020-03-03 Thread Sivaprasanna
Hi The flink-sequence-file module has a class named SerializableHadoopConfiguration[1] which is nothing but a wrapper class for Hadoop Configuration. I believe this class can be moved to a common module since this is not necessarily tightly coupled with sequence-file module, and also because it ca

Re: [DISCUSS] FLIP-56: Dynamic Slot Allocation

2020-03-03 Thread Xintong Song
Hi Shaoxun, You're right, that supporting end-to-end fine grained resource management is a huge plan, and FLIP-56 is only one step towards it. Regarding your questions: First, does "specific request" for the slots mean the requesting slot > profile contains detailed information about memory and

Re: Flink dev blog

2020-03-03 Thread Jeff Zhang
+1 for this proposal. I am preparing some articles for how to use Flink on Zeppelin, although it is not closely related with this topic, but should be helpful for users to get started with Flink. Till Rohrmann 于2020年3月3日周二 下午5:39写道: > I like the idea. +1 from my side. > > Potential topics: > -

Re: Flink dev blog

2020-03-03 Thread Till Rohrmann
I like the idea. +1 from my side. Potential topics: - Scheduling - Cluster partitions - Memory configuration - Recovery Cheers, Till On Tue, Mar 3, 2020 at 3:56 AM Xintong Song wrote: > Big +1. Thanks for the idea, Arvid. > > I'd be excited to read such blogs. > > And we would also be happy to

[jira] [Created] (FLINK-16400) HdfsKindTest.testS3Kind fails in Hadoop 2.4.1 nightly test

2020-03-03 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16400: -- Summary: HdfsKindTest.testS3Kind fails in Hadoop 2.4.1 nightly test Key: FLINK-16400 URL: https://issues.apache.org/jira/browse/FLINK-16400 Project: Flink

Re: [DISCUSS] FLIP-107: Reading table columns from different parts of source records

2020-03-03 Thread Dawid Wysakowicz
Hi, 1. I thought a bit more on how the source would emit the columns and I now see its not exactly the same as regular columns. I see a need to elaborate a bit more on that in the FLIP as you asked, Jark. I do agree mostly with Danny on how we should do that. One additional things I would introdu

[jira] [Created] (FLINK-16399) An OOM error occurred in Flink Job Manager

2020-03-03 Thread jiangbo (Jira)
jiangbo created FLINK-16399: --- Summary: An OOM error occurred in Flink Job Manager Key: FLINK-16399 URL: https://issues.apache.org/jira/browse/FLINK-16399 Project: Flink Issue Type: Bug Co

Re: [DISCUSS] FLIP-107: Reading table columns from different parts of source records

2020-03-03 Thread Danny Chan
Thanks Dawid for bringing up this discussion, I think it is a useful feature ~ About how the metadata outputs from source I think it is completely orthogonal, computed column push down is another topic, this should not be a blocker but a promotion, if we do not have any filters on the computed