Re: Rolling policy when using StreamingFileSink for bulk-encoded output

2019-07-03 Thread Kostas Kloudas
; > > Would you mind granting me the contributor access for the following > > account ? This way I can assign the JIRA to myself. > >*yxu-apache > > <https://issues.apache.org/jira/secure/ViewProfile.jspa?name=yxu-apache > >* > > > > Many tha

[DISCUSS] Graceful Shutdown Handling by UDFs.

2019-07-04 Thread Kostas Kloudas
Hi all, In many cases, UDFs (User Defined Functions) need to be able to perform application-specific actions when they stop in an orderly manner. Currently, Flink's UDFs, and more specifically the RichFunction which exposes lifecycle-related hooks, only have a close() method which is called in any

[FLIP-47] Savepoints vs Checkpoints

2019-07-08 Thread Kostas Kloudas
Hi Devs, Currently there is a number of efforts around checkpoints/savepoints, as reflected by the number of FLIPs. From a quick look FLIP-34, FLIP-41, FLIP-43, and FLIP-45 are all directly related to these topics. This reflects the importance of these two notions/features to the users of the fram

Re: [DISCUSS] Graceful Shutdown Handling by UDFs.

2019-07-08 Thread Kostas Kloudas
uot;cancel" should be consider an ungraceful shutdown, so that the > Job could be restarted from last (retained) checkpoint (as right now). > > Cheers, > > Konstantin > > On Thu, Jul 4, 2019 at 3:21 PM Kostas Kloudas wrote: > > > Hi all, > > > > In many c

Re: [ANNOUNCE] Rong Rong becomes a Flink committer

2019-07-11 Thread Kostas Kloudas
Congratulations Rong! On Thu, Jul 11, 2019 at 4:40 PM Jark Wu wrote: > Congratulations Rong Rong! > Welcome on board! > > On Thu, 11 Jul 2019 at 22:25, Fabian Hueske wrote: > >> Hi everyone, >> >> I'm very happy to announce that Rong Rong accepted the offer of the Flink >> PMC to become a commi

Re: [ANNOUNCE] Jiangjie (Becket) Qin has been added as a committer to the Flink project

2019-07-18 Thread Kostas Kloudas
Congratulations Becket! Kostas On Thu, Jul 18, 2019 at 11:21 AM Guowei Ma wrote: > Congrats Becket! > > Best, > Guowei > > > Terry Wang 于2019年7月18日周四 下午5:17写道: > > > Congratulations Becket! > > > > > 在 2019年7月18日,下午5:09,Dawid Wysakowicz 写道: > > > > > > Congratulations Becket! Good to have you

Re: Will StreamingFileSink.forBulkFormat(...) support overriding OnCheckpointRollingPolicy?

2019-07-23 Thread Kostas Kloudas
Hi Elkhan, As Till pointed out, the fact that your files remain in-progress is the expected behaviour as the StreamingFileSink assumes checkpointing is enabled in order to work. There are no plans to change this behaviour but an issue that may be relevant for you is https://issues.apache.org/jira/

Re: Suspicious watermark of operators after restore from checkpoint

2019-08-07 Thread Kostas Kloudas
Hi Jan, Two pointers that may help you explain the behavior are the following: 1) If you have a custom watermark generator, I do not think that Flink checks if it emits only monotonically increasing watermarks. This is the responsibility of the generator itself. This means that although you opera

Re: Suspicious watermark of operators after restore from checkpoint

2019-08-07 Thread Kostas Kloudas
: > Hi Kostas, > > thanks for reaction, comments inline. > > On 8/7/19 1:59 PM, Kostas Kloudas wrote: > > Hi Jan, > > > > Two pointers that may help you explain the behavior are the following: > > > > 1) If you have a custom watermark generator, I do no

Re: Suspicious watermark of operators after restore from checkpoint

2019-08-07 Thread Kostas Kloudas
But are they chained together? Could you provide the code from your job, at least until operator A? On Wed, Aug 7, 2019 at 3:03 PM Jan Lukavský wrote: > Actually, operator A is intermediate, source is preceding it. > > On 8/7/19 2:44 PM, Kostas Kloudas wrote: > > Hi Jan, > &

Re: Suspicious watermark of operators after restore from checkpoint

2019-08-07 Thread Kostas Kloudas
obgraph.png > [2] https://transfer.sh/iDg1A/watermarks.png > > On 8/7/19 3:04 PM, Kostas Kloudas wrote: > > But are they chained together? Could you provide the code from your job, > at > > least until operator A? > > > > On Wed, Aug 7, 2019 at 3:03 PM Jan Lu

Re: [DISCUSS] Drop stale class Program

2019-08-14 Thread Kostas Kloudas
Hi all, It is nice to have this discussion. I am totally up for removing the unused Program interface, as this will simplify a lot of other code paths in the ClusterClient and elsewhere. Also about the easier integration of Flink with other frameworks, there is another discussion in the mailing

Re: [DISCUSS] Drop stale class Program

2019-08-14 Thread Kostas Kloudas
Most users of Flink don’t actually know it exists and > > it is only making our code more complicated. Going forward with the new > > Client API discussions will be a lot easier without it as well. > > > > Best, > > Aljoscha > > > > > On 14. Aug

[DISCUSS] FLIP-52: Remove legacy Program interface.

2019-08-14 Thread Kostas Kloudas
Hi all, As discussed in [1] , the Program interface seems to be outdated and there seems to be no objection to remove it. Given that this interface is PublicEvolving, its removal should pass through a FLIP and this discussion and the associated FLIP are exactly for that purpose. Please let me kn

Re: [ANNOUNCE] Andrey Zagrebin becomes a Flink committer

2019-08-14 Thread Kostas Kloudas
Congratulations Andrey! Well deserved! Kostas On Wed, Aug 14, 2019 at 4:04 PM Yun Tang wrote: > > Congratulations Andrey. > > Best > Yun Tang > > From: Xintong Song > Sent: Wednesday, August 14, 2019 21:40 > To: Oytun Tez > Cc: Zili Chen ; Till Rohrmann ; > de

Re: [DISCUSS] FLIP-52: Remove legacy Program interface.

2019-08-15 Thread Kostas Kloudas
t, > > > tison. > > > > > > [1] > > > > > http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/REST-API-JarRunHandler-More-flexibility-for-launching-jobs-td31026.html#a31168 > > > > > > > > > Stephan Ewen 于2019年8月14日周三 下午7:50写道:

[VOTE] FLIP-52: Remove legacy Program interface.

2019-08-21 Thread Kostas Kloudas
Hi all, Following the FLIP process, this is a voting thread dedicated to the FLIP-52. As shown from the corresponding discussion thread [1], we seem to agree that the Program interface can be removed, so let's make it also official with a vote. Cheers, Kostas [1] https://lists.apache.org/thre

Re: [VOTE] FLIP-52: Remove legacy Program interface.

2019-08-21 Thread Kostas Kloudas
Thanks a lot! We have the 3 +1's so we can move forward. Cheers, Kostas On Wed, Aug 21, 2019 at 1:30 PM Chesnay Schepler wrote: > > +1 > > On 21/08/2019 13:23, Timo Walther wrote: > > +1 > > > > Am 21.08.19 um 13:21 schrieb Stephan Ewen: > >> +1 >

Re: [DISCUSS] Flink client api enhancement for downstream project

2019-08-23 Thread Kostas Kloudas
Hi all, On the topic of web submission, I agree with Till that it only seems to complicate things. It is bad for security, job isolation (anybody can submit/cancel jobs), and its implementation complicates some parts of the code. So, if it were to redesign the WebUI, maybe this part could be left

Re: [CODE-STYLE] Builder pattern

2019-08-27 Thread Kostas Kloudas
Hi all, I agree with Arvid, although for point 2 I would be less strict. @Piotr, for the side note you mentioned, and from the description you mention in the mail for example I, it seems that the need to pass parameters in the build() is not an inherent need of the build pattern but it can be mit

Re: [DISCUSS] Flink client api enhancement for downstream project

2019-08-30 Thread Kostas Kloudas
ClusterEntryPoint actually uses the user code class loader > > > with child first class loading [1]. Hence, I hope that this problem will > > be > > > resolved soon. > > > > > > [1] https://issues.apache.org/jira/browse/FLINK-13840 > > > > > >

Re: [DISCUSS] Releasing Flink 1.8.2

2019-09-02 Thread Kostas Kloudas
Hi all, I think this should be also considered a blocker https://issues.apache.org/jira/browse/FLINK-13940. It is not a regression but it can result to data loss. I think I can have a quick fix by tomorrow. Cheers, Kostas On Mon, Sep 2, 2019 at 12:01 PM jincheng sun wrote: > > Thanks for all o

Re: [DISCUSS] Releasing Flink 1.8.2

2019-09-03 Thread Kostas Kloudas
very important bug, And great thanks for > helping fix it Kostas! > > Best, Jincheng > > Kostas Kloudas 于2019年9月2日周一 下午7:20写道: > > > Hi all, > > > > I think this should be also considered a blocker > > https://issues.apache.org/jira/browse/FLINK-13940.

Re: [DISCUSS] Releasing Flink 1.8.2

2019-09-03 Thread Kostas Kloudas
> in the future. > So should we update the fixVersion of FLINK-13940? > > Best, > Jark > > On Tue, 3 Sep 2019 at 21:33, Kostas Kloudas wrote: > > > Thanks for waiting! > > > > A fix for FLINK-13940 has been merged on 1.8, 1.9 and the master under > > FLIN

Re: [DISCUSS] Flink client api enhancement for downstream project

2019-09-04 Thread Kostas Kloudas
ili Chen 于2019年8月31日周六 下午12:52写道: >> >> Great Kostas! Looking forward to your POC! >> >> Best, >> tison. >> >> >> Jeff Zhang 于2019年8月30日周五 下午11:07写道: >>> >>> Awesome, @Kostas Looking forward your POC. >>> >>> Kostas Klo

Re: [DISCUSS] Flink client api enhancement for downstream project

2019-09-04 Thread Kostas Kloudas
, 2019 at 11:46 AM Kostas Kloudas wrote: > > Great idea Tison! > > I will create the umbrella issue and post it here so that we are all > on the same page! > > Cheers, > Kostas > > On Wed, Sep 4, 2019 at 11:36 AM Zili Chen wrote: > > > > Hi Kostas & A

Re: [DISCUSS] Features for Apache Flink 1.10

2019-09-06 Thread Kostas Kloudas
Hi Gary, Thanks for kicking off the feature discussion. +1 for Gary and Yu as release managers. Cheers, Kostas On Fri, Sep 6, 2019 at 5:06 PM Gary Yao wrote: > > Hi community, > > Since Apache Flink 1.9.0 has been released more than 2 weeks ago, I want to > start kicking off the discussion abo

Re: [ANNOUNCE] Kostas Kloudas joins the Flink PMC

2019-09-09 Thread Kostas Kloudas
Qin > Send Time:2019 Sep. 9 (Mon.) 10:47 > To:dev > Subject:Re: [ANNOUNCE] Kostas Kloudas joins the Flink PMC > > Congrats, Kostas! > > On Sun, Sep 8, 2019 at 11:48 PM myasuka wrote: > > > Congratulations Kostas! > > > > Best > > Yun Tang >

Re: [DISCUSS] StreamingFile with ParquetBulkWriter bucketing limitations

2019-09-09 Thread Kostas Kloudas
Hi Enrico, Sorry for the late reply. I think your understanding is correct. The best way to do it is to write your own ParquetBulkWriter and the corresponding factory. Out of curiosity, I guess that in the BucketingSink you were using the AvroKeyValueSinkWriter, right? Cheers, Kostas On Fri, Au

Re: [DISCUSS] Flink client api enhancement for downstream project

2019-09-11 Thread Kostas Kloudas
oof >>> >> of >>> >> concept implementation for the new JobClient interface, maybe with a new >>> >> method executeAsync() in the environment that returns the JobClient and >>> >> implement the methods to see how that works and to see wh

Re: performances of S3 writing with many buckets in parallel

2020-02-07 Thread Kostas Kloudas
Hi Enrico, Nice to hear from you and thanks for checking it out! This can be helpful for people using the BucketingSink but I would recommend you to switch to the StreamingFileSink which is the "new version" of the BucketingSink. In fact the BucketingSink is going to be removed in one of the foll

Re: [VOTE] Release 1.10.0, release candidate #3

2020-02-10 Thread Kostas Kloudas
Hi all, +1 (binding) - Built Flink locally - Tested quickstart by writing simple, WordCount-like streaming jobs - Submitted them to Yarn both "per-job" and "session" mode - used the yarn-session CLI to start/stop sessions Thanks a lot Gary and Yu for managing the release. Cheers, Kostas On M

Re: [DISCUSS] Improvements on FLIP Process

2020-02-18 Thread Kostas Kloudas
+1 to what Aljoscha and Timo are proposing. I would lean towards eliminating Google Docs altogether. I think they served a purpose when discussions were among 3 to 4 people but with the current size of the community and the amount of participants per discussion they become difficult to follow. Be

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

2020-03-02 Thread Kostas Kloudas
ML :) Thanks for starting the discussion and I hope that soon we will be able to vote on the FLIP. Cheers, Kostas On Thu, Jan 16, 2020 at 3:40 AM Yang Wang wrote: > > Hi all, > > Thanks a lot for the feedback from @Kostas Kloudas. Your all concerns are > on point. The FLIP-85 is m

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

2020-03-02 Thread Kostas Kloudas
e between per-job and application mode? > > For per-job, user `main()` is always executed on client side. And For > application mode, user > `main()` could be executed on client or master side(configured via cli > option). > Right? We need to have a clear concept. Otherwise, the us

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

2020-03-03 Thread Kostas Kloudas
r mode > implementation for per job cluster. > > To minimize the confusion of users, I think it would be better just an option > of per job cluster for each type of cluster manager. How do you think? > > > Best Regards > Peter Huang > > > > > > > > &g

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

2020-03-05 Thread Kostas Kloudas
Also from my side +1 to start voting. Cheers, Kostas On Thu, Mar 5, 2020 at 7:45 AM tison wrote: > > +1 to star voting. > > Best, > tison. > > > Yang Wang 于2020年3月5日周四 下午2:29写道: >> >> Hi Peter, >> Really thanks for your response. >> >> Hi

Re: [PROPOSAL] Reverse the dependency from flink-streaming-java to flink-client

2020-03-05 Thread Kostas Kloudas
Big +1 also from my side. This will eliminate some work-arounds used so far to bypass the module structure (like code using reflection to extract a JobGraph from a Pipeline). I agree with Stephan that with proper documentation, release notes and tooling update, it will hopefully not be a big hass

[VOTE] [FLIP-85] Flink Application Mode

2020-03-06 Thread Kostas Kloudas
Hi all, I would like to start the vote for FLIP-85 [1], which is discussed and reached a consensus in the discussion thread [2]. Given that it is Friday, the vote will be open until March. 11th (72h starting on Monday), unless there is an objection or not enough votes. Thanks, Kostas [1] https

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

2020-03-09 Thread Kostas Kloudas
terEntrypoint's responsibility. >>>>> >>>>> >Will there be an option of not tearing the cluster down? >>>>> >>>>> I think the answer is "No" because the cluster is designed to be bounded >>>>> with an Appli

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

2020-03-10 Thread Kostas Kloudas
s/jars are built into the image. Cheers, Kostas On Tue, Mar 10, 2020 at 3:05 PM Yang Wang wrote: > > Thanks for your response. > > @Kostas Kloudas Could we update the cli changes and how to fetch the > user jars to FLIP document? I think other dev or users may have the similar > q

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

2020-03-10 Thread Kostas Kloudas
Hi all, The FLIP was updated under the section "First Version Deliverables". Cheers, Kostas On Tue, Mar 10, 2020 at 4:10 PM Kostas Kloudas wrote: > > Hi all, > > Yes I will do that. From the discussion, I will add that: > 1) for the cli, we are planning to add a &quo

Re: [VOTE] [FLIP-85] Flink Application Mode

2020-03-12 Thread Kostas Kloudas
Hi all, Thanks for the votes. We have 3 binding +1 votes: Tison, Rong Rong, and Aljoscha and 0 negative ones. I will move the FLIP to accepted and will start opening JIRAs. Thanks a lot, Kostas On Thu, Mar 12, 2020 at 11:17 AM Aljoscha Krettek wrote: > > +1 (binding) > > Aljoscha

Re: [DISCUSS] Drop Bucketing Sink

2020-03-12 Thread Kostas Kloudas
Hi Robert, +1 for dropping the BucketingSink. In any case, it has not been maintained for quite some time. Cheers, Kostas On Thu, Mar 12, 2020 at 3:41 PM Robert Metzger wrote: > > Hi all, > > I'm currently investigating a failing end to end test for the bucketing > sink [1]. > The bucketing sin

Re: [Discussion] Job generation / submission hooks & Atlas integration

2020-03-12 Thread Kostas Kloudas
Hi all, I do not have a strong opinion on the topic yet, but I would like to share my thoughts on this. In the solution proposing a wrapping AtlasExecutor around the Flink Executors, if we allow the user to use the CLI to submit jobs, then this means that the CLI code may have to change so that i

Re: [Discussion] Job generation / submission hooks & Atlas integration

2020-03-12 Thread Kostas Kloudas
Hi again, Just to clarify, I am not against exposing the Pipeline if this will lead to a "clean" solution. And, I. forgot to say that the last solution, if adopted, would have to work on the JobGraph, which may not be that desirable. Kostas On Thu, Mar 12, 2020 at 8:26 PM Kostas Klou

Re: [Discussion] Job generation / submission hooks & Atlas integration

2020-03-12 Thread Kostas Kloudas
he pattern :) For not always being able to return the > pipeline, this might be expected depending on how the JobClient, so we need > to handle it some way. > > > On Thu, Mar 12, 2020 at 8:30 PM Kostas Kloudas wrote: > > > Hi again, > > > > Just to clarify, I am not

Re: [Discussion] Job generation / submission hooks & Atlas integration

2020-03-13 Thread Kostas Kloudas
> > > that could be registered in the StreamExecutionEnvironment. This meant > > that > > > we could work on the StreamGraph, register metadata in the JobGraph, and > > > execute the actual atlas registration logic in the JobMaster when the job > > > starts.

Re: [DISCUSS] Drop Bucketing Sink

2020-03-17 Thread Kostas Kloudas
; Best, > > > > >> David > > > > >> > > > > >> [1] > > > > >> > > > > >> > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-46%3A+Graceful+Shutdown+Handling+by+UDFs

Re: [DISCUSS] FLIP-115: Filesystem connector in Table

2020-03-18 Thread Kostas Kloudas
Hi all, I also agree with Stephan on this! It has been more than a year now that most of our efforts have had the "unify" / "unification"/ etc either on their title or in their core and this has been the focus of all our resources. By deviating from this now, we only put more stress on other team

Re: [DISCUSS] FLIP-126: Unify (and separate) Watermark Assigners

2020-04-27 Thread Kostas Kloudas
Hi Aljoscha, Thanks for opening the discussion! I have two comments on the FLIP: 1) we could add lifecycle methods to the Generator, i.e. open()/ close(), probably with a Context as argument: I have not fully thought this through but I think that this is more aligned with the rest of our rich fun

[DISCUSS] FLIP-73: Introducing Executors for job submission

2019-09-24 Thread Kostas Kloudas
Hi all, In the context of the discussion about introducing the Job Client API [1], there was a side-discussion about refactoring the way users submit jobs in Flink. There were many different interesting ideas on the topic and 3 design documents that were trying to tackle both the issue about code

Re: [DISCUSS] FLIP-73: Introducing Executors for job submission

2019-09-25 Thread Kostas Kloudas
mpatibility, it is on (Stream)ExecutionEnvironment#execute. > You're right that we don't stick to blocking to return a JobExecutionResult in > Executor aspect but implementing env.execute with a unique > > Executor#execute(or with suffix Async): CompletableFuture > > what

Re: [DISCUSS] FLIP-73: Introducing Executors for job submission

2019-09-25 Thread Kostas Kloudas
l do it. What do you think? > > Best, > Aljoscha > > > On 25. Sep 2019, at 10:11, Kostas Kloudas wrote: > > > > Hi Thomas and Zili, > > > > As you both said the Executor is a new addition so there are no > > compatibility concerns. > > Backwards co

Re: [DISCUSS] FLIP-74: Flink JobClient API

2019-09-25 Thread Kostas Kloudas
Hi Tison, Thanks for the FLIP and launching the discussion! As a first note, big +1 on providing/exposing a JobClient to the users! Some points that would be nice to be clarified: 1) You mention that we can get rid of the DETACHED mode: I agree that at a high level, given that everything will no

Re: [DISCUSS] Expose multiple level clients

2019-09-25 Thread Kostas Kloudas
Hi Tison, I shared some thoughts on this topic in the FLIP-74 thread, as some of the questions are related to that. Cheers, Kostas On Wed, Sep 25, 2019 at 12:30 PM Zili Chen wrote: > > Hi all, > > While drafting FLIP-74 I notice that a job level client(called JobClient) is > always retrieved f

Re: [DISCUSS] FLIP-73: Introducing Executors for job submission

2019-09-27 Thread Kostas Kloudas
org/x/thread.html/b2e22a45aeb94a8d06b50c4de078f7b23d9ff08b8226918a1a903768@%3Cdev.flink.apache.org%3E > [2] > https://lists.apache.org/x/thread.html/240582148eda905a772d59b2424cb38fa16ab993647824d178cacb02@%3Cdev.flink.apache.org%3E > [3] > https://docs.google.com/document/d/1E-8UjOLz4QPUTx

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 princ

Re: [DISCUSS] FLIP-73: Introducing Executors for job submission

2019-10-02 Thread Kostas Kloudas
Hi all, I agree with Tison that we should disentangle threads so that people can work independently. For FLIP-73: - for Preview/OptimizedPlanEnv: I think they are orthogonal to the Executors work, as they are using the exexute() method because this is the only "entry" to the user program. To thi

Re: [DISCUSS] FLIP-73: Introducing Executors for job submission

2019-10-02 Thread Kostas Kloudas
and use it to execute multiple pipelines, if in the future we choose to do so. Let me know what do you think, Kostas On Wed, Oct 2, 2019 at 7:23 PM Kostas Kloudas wrote: > > Hi all, > > I agree with Tison that we should disentangle threads so that people > can work independently.

Re: [DISCUSS] FLIP-73: Introducing Executors for job submission

2019-10-02 Thread Kostas Kloudas
configured on Executor created > >> and > >> let this one overwhelm that one. > >> > >> I can see it is useful that conceptually we can create an Executor for a > >> series jobs > >> to the same cluster but with different job configuration per pi

Re: [DISCUSS] FLIP-74: Flink JobClient API

2019-10-03 Thread Kostas Kloudas
Hi Tison, I see. Then I would say that as a first step, and to see if people are happy with the result, integration with the production code can be through a new method executeAsync() in the Executor that we discussed earlier. This method could potentially be exposed to ExecutionEnvironment as a

Re: [DISCUSS] FLIP-74: Flink JobClient API

2019-10-04 Thread Kostas Kloudas
luster". > >>>>>>>>> > >>>>>>>>> Besides, refer to the discussion with Till[1], it would be > >>> helpful > >>>> we > >>>>>>>>> follow the same process > >

[VOTE] FLIP-73: Introducing Executors for job submission

2019-10-04 Thread Kostas Kloudas
Hi all, I would like to start the vote for FLIP-73 [1], which is discussed and reached a consensus in the discussion thread[2]. Given that it is Friday, the vote will be open until Oct. 9th (72h starting on Monday), unless there is an objection or not enough votes. Thanks, Kostas [1] https://c

Re: [DISCUSS] FLIP-73: Introducing Executors for job submission

2019-10-10 Thread Kostas Kloudas
; > > >> Executor.execute() to return a CompletableFuture and to completely > > > remove > > > >> the “detached” logic from ClusterClient. That way, the new components > > > make > > > >> no distinction between “detached” and “attached” but

Re: [VOTE] FLIP-73: Introducing Executors for job submission

2019-10-10 Thread Kostas Kloudas
as Weise wrote: > > > > +1 > > > > > > On Fri, Oct 4, 2019 at 8:56 AM Zili Chen wrote: > > > >> Thanks for your works Kostas! > >> > >> +1 for FLIP-73 > >> > >> Best, > >> tison > >> > >> > &g

Re: [DISCUSS] FLIP-73: Introducing Executors for job submission

2019-10-10 Thread Kostas Kloudas
er client or a job client? As discussed previously > I think conceptually it is a job client? > > Best, > tison. > > > Kostas Kloudas 于2019年10月10日周四 下午5:08写道: > > > Hi Tison, > > > > I would say that as a first step, and until we see that the interfaces > &g

Re: [VOTE] FLIP-74: Flink JobClient API

2019-10-11 Thread Kostas Kloudas
Hi all, I only have two minor comments before voting and they have to do with the following: 1) In the discussion, we agreed to remove the cancelWithSavepoint() from the JobClient as this is deprecated in the rest API. This is not in the FLIP. 2) The section "ClusterDescriptor or Executor(FLIP-73

Re: [VOTE] FLIP-74: Flink JobClient API

2019-10-11 Thread Kostas Kloudas
from the FLIP. But > to emphasize that we make the decision to exclude it I add it to reject > alternatives. > > (2) Updated FLIP to reflect the consensus :-) > > Best, > tison. > > > Kostas Kloudas 于2019年10月11日周五 下午5:12写道: > > > Hi all, > > > >

Re: [PROPOSAL] Contribute Stateful Functions to Apache Flink

2019-10-14 Thread Kostas Kloudas
Hi all, Big +1 for contributing Stateful Functions to Flink and as for the main question at hand, I would vote for putting it in the main repository. I understand that this can couple the release cadence of Flink and Stateful Functions although I think the pros of having a "you break it, you fix

Re: [DISCUSS] FLIP policy for introducing config option keys

2019-10-15 Thread Kostas Kloudas
Hi Aljoscha, Given that config option keys are user-facing and any change there is breaking, I think there should be a discussion about them and a FLIP, where people have to actually vote for it seems to be the right place. I understand that this is tedious (and actually I will also have to open s

Re: performances of S3 writing with many buckets in parallel

2019-10-16 Thread Kostas Kloudas
Hi Enrico, Thanks for opening the discussion! One thing to note that may help s that the hadoop S3 FS tries to imitate a filesystem on top of S3: - before writing a key it checks if the "parent directory" exists by checking for a key with the prefix up to the last "/" - it creates empty marker f

Re: [DISCUSS] Stateful Functions - in which form to contribute? (same or different repository)

2019-10-16 Thread Kostas Kloudas
Hi all, Although in the initial thread I said that, in general, I would prefer having one repository, I understand that arguments presented here and I think it makes sense for such a young project to have its own repository. So +1 from my side, with an asterisk about hoping that eventually the pr

Re: [DISCUSS] FLIP-59: Enable execution configuration from Configuration object

2019-10-16 Thread Kostas Kloudas
Hi all, Thanks for opening the discussion! I like the idea, so +1 from my side and actually this is aligned with our intensions for the FLIP-73 effort. For the naming convention of the parameters introduced in the FLIP, my proposal would be have the full word "execution" instead of the shorter "

Fwd: Verifying correctness of StreamingFileSink (Kafka -> S3)

2019-10-16 Thread Kostas Kloudas
StringEncoder<>("UTF-8")) .withBucketAssigner(assignerB) .build(); myInputFromKafka.addSink(sinkA) myInputFromKafka.addSink(sinkB) Cheers, Kostas -- Forwarded message - From: amran dean Date: Wed, Oct 16, 2019 at 8:43 PM Subject: Re: Verifying correctness of StreamingFileSink

Re: [DISCUSS] FLIP-59: Enable execution configuration from Configuration object

2019-10-17 Thread Kostas Kloudas
o Walther wrote: > > Hi Kostas, > > can we still discuss the naming of the properties? For me, having > "execution" and "exector" as prefixes might be confusing in the future > and difficult to identify if you scan through a list of properties. > > How about

FLIP-81: Executor-related new ConfigOptions

2019-10-21 Thread Kostas Kloudas
Hi all, As part of FLIP-73 (the Executors effort), we would like to introduce some new configuration options. These are needed in order to be able to map all the options that the user can specify either programmatically or through the CLI into configuration options. The bylaws specify that every

Re: [VOTE] Accept Stateful Functions into Apache Flink

2019-10-21 Thread Kostas Kloudas
+1 (binding) On Mon, Oct 21, 2019 at 5:18 PM Aljoscha Krettek wrote: > > +1 (binding) > > Aljoscha > > > On 21. Oct 2019, at 16:18, Thomas Weise wrote: > > > > +1 (binding) > > > > > > On Mon, Oct 21, 2019 at 7:10 AM Timo Walther wrote: > > > >> +1 (binding) > >> > >> Thanks, > >> Timo > >> > >

Re: FLIP-81: Executor-related new ConfigOptions

2019-10-22 Thread Kostas Kloudas
it more > > prominent that we don’t like the “attached”, “shutdown-on-attached-exit”, > > and “dynamic-properties” and that we only add them for backwards > > compatibility. > > > > Best, > > Aljoscha > > > >> On 21. Oct 2019, at 17:48,

Re: FLIP-81: Executor-related new ConfigOptions

2019-10-23 Thread Kostas Kloudas
> experimental and clearly describing what they do and imply. > > Best, > Aljoscha > > > On 22. Oct 2019, at 14:26, Kostas Kloudas wrote: > > > > Hi Dawid, > > > > For the first comment, I am also up for re-using as many options as > > possible. My

[VOTE] FLIP-81: Executor-related new ConfigOptions

2019-10-23 Thread Kostas Kloudas
Hi all, This is the voting thread for FLIP-81, as the title says. The FLIP can be found in [1] and the discussion thread in [2]. As per the bylaws, the vote will stay open until Friday 26-10 (3 days) or until it gets 3 votes. Thank you, Kostas [1] https://cwiki.apache.org/confluence/pages/view

Re: [VOTE] FLIP-81: Executor-related new ConfigOptions

2019-10-28 Thread Kostas Kloudas
; >>>> > >>>>> On 23. Oct 2019, at 10:41, Zili Chen wrote: > >>>>> > >>>>> Thanks for starting this voting process. I have looked at the FLIP > >> and > >>>> the > >>>>> disc

Re: [VOTE] FLIP-59: Enable execution configuration from Configuration object

2019-11-14 Thread Kostas Kloudas
+1 (binding) On Tue, Nov 12, 2019 at 10:20 AM tison wrote: > > +1 (binding) > > Best, > tison. > > > Aljoscha Krettek 于2019年11月12日周二 下午4:37写道: > > > +1 (binding) > > > > > On 8. Nov 2019, at 09:45, Danny Chan wrote: > > > > > > Thanks for driving on this, David. > > > > > > +1(non-binding) from

Documentation tasks for release-1.10

2019-12-16 Thread Kostas Kloudas
Hi all, With the feature-freeze for the release-1.10 already past us, it is time to focus a little bit on documenting the new features that the community added to this release, and improving the already existing documentation based on questions that we see in Flink's mailing lists. To this end, I

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

2020-01-15 Thread Kostas Kloudas
Hi all, I am writing here as the discussion on the Google Doc seems to be a bit difficult to follow. I think that in order to be able to make progress, it would be helpful to focus on per-job mode for now. The reason is that: 1) making the (unique) JobSubmitHandler responsible for creating the j

Re: [ANNOUNCE] Yu Li became a Flink committer

2020-01-23 Thread Kostas Kloudas
Congratulations Yu and welcome! On Thu, Jan 23, 2020 at 2:28 PM Till Rohrmann wrote: > > Congrats Yu :-) > > On Thu, Jan 23, 2020 at 2:02 PM Yang Wang wrote: > > > Congratulations, Yu. > > > > > > Best, > > Yang > > > > tison 于2020年1月23日周四 下午7:07写道: > > > > > Congratulations! > > > > > > Best,

Re: Caveats of multi-execute() Flink programs

2020-01-23 Thread Kostas Kloudas
Hi all, I agree with Stephan that it is reasonable to assume that multiple independent jobs should not be in the same "main()" or "application" or "program" or environment. But if/when dataStream subsumes dataSet, then batch programs will also have checkpoints/savepoints and in that case it is n

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

2020-01-30 Thread Kostas Kloudas
Hi all, +1 (binding) - Compiled locally - Built simple jobs using quickstart - Submitted on Yarn both per-job and session cluster Cheers, Kostas On Thu, Jan 30, 2020 at 6:26 AM Hequn Cheng wrote: > > Hi everyone, > > Thanks a lot for your checking and voting for the release! > I’ll summarize t

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

2020-02-03 Thread Kostas Kloudas
+1 (binding) - Built Flink locally - Tested quickstart by writing simple, WordCount-like jobs - Submitted them to Yarn both "per-job" and "session" mode For Thomas' comment, I agree that in this release we change how some of the execution options are propagated through the stack. This was done as

Re: CoProcessFunction doesn't support timer on keyed stream

2018-04-26 Thread Kostas Kloudas
Hi Ken, It is true that there is not reason for not having access to the timerService from the processElement of the keyed side. On the other side (the non-keyed side) you cannot set timers because timers are bound to a specific key. Now, if one stream is broadcasted and the other is keyed,

Re: CoProcessFunction doesn't support timer on keyed stream

2018-04-26 Thread Kostas Kloudas
ll the documentation at localhost:4000 and the Broadcast State one at: http://localhost:4000/dev/stream/state/broadcast_state.html <http://localhost:4000/dev/stream/state/broadcast_state.html> Any feedback is welcomed! Cheers, Kostas > On Apr 26, 2018, at 11:09 AM, Kostas Kl

Re: [ANNOUNCE] Two new committers: Xingcan Cui and Nico Kruber

2018-05-10 Thread Kostas Kloudas
Congratulations to both of you guys! > On May 10, 2018, at 9:41 AM, Amit Jain wrote: > > Congrats! > > On Thu, May 10, 2018 at 10:10 AM, Xingcan Cui wrote: >> Thanks, everyone! >> >> It’s an honor which inspires me to devote more to our community. >> >> Regards, >> Xingcan >> >>> On May 10,

Re: [ANNOUNCE] New committer Piotr Nowojski

2018-06-22 Thread Kostas Kloudas
Congratulations! On Fri, Jun 22, 2018, 21:33 Shuyi Chen wrote: > Congratulations! > > > On Fri, Jun 22, 2018 at 12:30 PM Matthias J. Sax wrote: > > > -BEGIN PGP SIGNED MESSAGE- > > Hash: SHA512 > > > > Congrats! > > > > On 6/22/18 12:28 PM, Stefan Richter wrote: > > > Congrats Piotr! >

Re: [ANNOUNCE] New committer: Sihua Zhou

2018-06-22 Thread Kostas Kloudas
Congratulations! On Fri, Jun 22, 2018, 21:33 Shuyi Chen wrote: > Congratulations! > > On Fri, Jun 22, 2018 at 11:08 AM Matthias J. Sax wrote: > > > -BEGIN PGP SIGNED MESSAGE- > > Hash: SHA512 > > > > Congrats! > > > > On 6/22/18 10:33 AM, shimin yang wrote: > > > Congrats! > > > > > > O

Re: Adding a part suffix setter to the BucketingSink

2018-07-12 Thread Kostas Kloudas
Hi Lakshmi, Since Flink-1.5 you have the ability to set the part suffix. As you said, you only want the .gzip to be the suffix of the final (or “completed”) part files, which is exactly what is currently supported. If you want also intermediate files to have this suffix, then you can always set

Re: Adding a part suffix setter to the BucketingSink

2018-07-12 Thread Kostas Kloudas
this issue - https://issues.apache.org/ > jira/browse/FLINK-9603. Thanks for pointing out, I'll make sure to use the > 1.5.3 version of the sink. > > Thanks > Lakshmi > > > On Thu, Jul 12, 2018 at 4:55 AM, vino yang wrote: > >> Hi Kostas, good job! >> >

Cherry picking FLINK-9753 and FLINK-9753 on release-1.6

2018-07-20 Thread Kostas Kloudas
Hi all, We just merged on the master [FLINK-9753] (commit 66b1f854a0250bdd048808d40f93aa2990476841) and [FLINK-9903] (commit b56c75ca375049b1d2c80d2d0945ae1ae04eb39e). These two commits introduce: 1) a big refactoring to the new StreamingFileSink which allows it to support bulk formats in gene

Re: Cherry picking FLINK-9753 and FLINK-9753 on release-1.6

2018-07-23 Thread Kostas Kloudas
ay, I would still like to see this happen, because it let me have more >> alternative to upload data with parquet >> format. Thanks for your great efforts on this. >> >> Best Regards, >> Tony Wei >> >> https://issues.apache.org/jira/brows

Re: Support Hadoop 2.6 for StreamingFileSink

2018-08-21 Thread Kostas Kloudas
Hi Artsem, Till is correct in that getting rid of the “valid-length” file was a design decision for the new StreamingFileSink since the beginning. The motivation was that users were reporting that essentially it was very cumbersome to use. In general, when the BucketingSink gets deprecated, I

Re: [ANNOUNCE] New committer Gary Yao

2018-09-07 Thread Kostas Kloudas
Congratulations Gary! Well deserved! Cheers, Kostas > On Sep 7, 2018, at 4:43 PM, Fabian Hueske wrote: > > Congratulations Gary! > > 2018-09-07 16:29 GMT+02:00 Thomas Weise : > >> Congrats, Gary! >> >> On Fri, Sep 7, 2018 at 4:17 PM Dawid Wysakowicz >> wrote: >> >>> Congratulations Gary! W

  1   2   3   4   5   >