Wearing my Cloudera hat I can tell you that we have done this exercise for
our distros of the 3.0 and 3.1 Hadoop versions. We have not contributed
these back just yet, but we are open to do so. If the community is
interested we can contribute those changes back to flink-shaded and suggest
the nece
+1 (binding)
Thank you for proposing this contribution!
On Fri, Nov 1, 2019 at 2:46 PM Konstantin Knauf
wrote:
> +1 (non-binding)
>
> Stateful Functions, already in its current initial release, simplifies the
> development of event-driven application on Flink quite significantly.
>
> On Thu, Oc
+1
On Fri, Aug 23, 2019 at 11:40 AM Kostas Tzoumas wrote:
> +1
>
> On Thu, Aug 22, 2019 at 5:29 PM jincheng sun
> wrote:
>
>> +1
>>
>> Becket Qin 于2019年8月22日 周四16:22写道:
>>
>> > Hi All, so far the votes count as following:
>> >
>> > +1 (Binding): 13 (Aljoscha, Fabian, Kurt, Till, Timo, Max, Step
Hi Aljoscha,
I am in favor of the change. No concerns on my side, just one remark that I
have talked to Sean last week (ccd) and he mentioned that he has faced some
technical issues while driving the transition from 2.10 to 2.12 for Spark.
It had to do with changes in the scope of implicits. You m
Done, my Travis build came in all green:
https://travis-ci.org/mbalassi/flink/builds/181809550
The hotfix is pushed.
On Tue, Dec 6, 2016 at 5:08 PM, Márton Balassi
wrote:
> The travis build is currently broken, if noone has fixed it yet then I
> will push the fix for it.
>
> ht
The travis build is currently broken, if noone has fixed it yet then I will
push the fix for it.
https://travis-ci.org/apache/flink/builds/181765492
The following files have missing licenses:
flink-runtime-web/web-dashboard/assets/images/browserconfig.xml
flink-runtime-web/web-dashboard/assets/i
+1. It keeps it both organized and to a reasonable minimum overhead.
Would you volunteer for starting the mail thread each month then, Kostas?
Best,
Marton
On Tue, Dec 6, 2016 at 6:42 AM, Kostas Tzoumas wrote:
> Hi folks,
>
> I'd like to see how the community feels about a monthly "Who is hir
3, 2016 at 4:42 PM, Márton Balassi
wrote:
> Thanks for bringing up the issue, Fabian.
>
> I am also for unifying the access patterns between the FieldAccessor and
> ExpressionKeys logic.
>
> In terms of Fabian's suggestion to refactor the ExpressionKeys parsing
> l
let you know asap I have something ;-)
> >
> > Best
> >
> > -Original Message-
> > From: Márton Balassi [mailto:balassi.mar...@gmail.com]
> > Sent: viernes, 28 de octubre de 2016 8:50
> > To: dev@flink.apache.org
> > Subject: Re: Apache Flink and
Thanks for bringing up the issue, Fabian.
I am also for unifying the access patterns between the FieldAccessor and
ExpressionKeys logic.
In terms of Fabian's suggestion to refactor the ExpressionKeys parsing
logic and rely on it in the FieldAccessors I think that is nice first step.
It would be g
Hi Ruben,
I am currently not aware of such an effort, but I definitely do agree that
it is an interesting pattern to investigate. As a motivation you could have
a look at the Spark connector implementations to see the Kudu APIs in use.
For that I would recommend the DataSource API implementation t
're currently using Cloudera 5.6.0 and there's HBase 1.0.0...is it
> compatible?
>
> On Tue, Oct 4, 2016 at 12:08 PM, Márton Balassi
> wrote:
>
> > Sounds reasonable to me. +1.
> >
> > On Tue, Oct 4, 2016 at 11:38 AM, Fabian Hueske
> wrote:
> >
&
Thanks for volunteering for RM, Ufuk.
On Tue, Oct 4, 2016 at 12:07 PM, Ufuk Celebi wrote:
> If there are no objections I would like to be the release manager for
> this release.
>
> Futhermore, I would like to add FLINK-4732 (Maven junction plugin
> security issue) to the list of fixes for this
Sounds reasonable to me. +1.
On Tue, Oct 4, 2016 at 11:38 AM, Fabian Hueske wrote:
> Hi everybody,
>
> Flink's TableInputFormat depends on a very old HBase dependency (0.98.11).
>
> We have received user requests (see FLINK-2765 [1]) to update the
> dependency for hadoop-2 to 1.2.
> In addition
Sure, Nikolay. Thanks for your willingness to contribute to Flink. I have
added you to the project, next time you can do it yourself.
Best,
Marton
On Mon, Oct 3, 2016 at 11:33 AM, Nikolay Vasilishin <
nikolay_vasilis...@epam.com> wrote:
> Hi devs.
>
> I'd like to do something for flink. Could y
Go ahead. +1, (binding).
On Tue, Aug 9, 2016 at 6:53 PM, Henry Saputra
wrote:
> Yes, We can do this with reduced time for VOTE
>
> On Tuesday, August 9, 2016, Ufuk Celebi wrote:
>
> > PS: Let me add that no changes other than the release 1.1.1 commit
> > have been added on top of what we had in
Welcome Neelesh, great to have you here. :)
On Sun, Jul 31, 2016, 11:08 Neelesh Salian wrote:
> Hello folks,
>
> I am Neelesh Salian; I recently joined the Flink community and I wanted to
> take this opportunity to formally introduce myself.
>
> I have been working with the Hadoop and Spark ecos
ur e-mail below:
>
> On 08.07.2016 15:13, Márton Balassi wrote:
>
>> Hi Kevin,
>>
>> Thanks for being willing to contribute such an effort. I think it is a
>> completely valid discussion to ask in your organization and please feel
>> free to ask us questions d
Hi Kevin,
Thanks for being willing to contribute such an effort. I think it is a
completely valid discussion to ask in your organization and please feel
free to ask us questions during your evaluation. Putting statements on the
Flink website highlighting the differences would be very tricky though
Hi Alan,
Your contribution is more than welcome. It would be a great addition to
flink-streaming-connectors. At some point we might move some of these to a
"Flink Packages" repository, similarly to the Spark approach, but currently
the best place to have them is the internal connectors.
Robert in
I do like the idea, that seems to be the trend now - the Bigtop community
had a similar initiative recently. [1]
Helps dealing with the "Is it mature enough?" question. :)
[1] http://kaiyzen.github.io/bigtop/
On Mon, Jul 4, 2016 at 5:00 PM, Ufuk Celebi wrote:
> I would like that! +1
>
> On Mon,
I also think that the current mechanism is weird. IMHO it makes sense to
add the flag to both the start and stop scripts.
On Wed, Jun 1, 2016 at 2:09 PM, Ufuk Celebi wrote:
> Yes, it's expected, but you are certainly not the first one to be
> confused by this behaviour.
>
> The reasoning behind
Hey Eron,
Yes, DataSet#collect and count methods implicitly trigger a JobGraph
execution, thus they also trigger writing to any previously defined sinks.
The idea behind this behavior is to enable interactive querying (the one
that you are used to get from a shell environment) and it is also a gre
Hey Vijay,
Depending on the local dependencies is one way to do this. IMHO the more
straight-forward way is to simply place your tests within your version of
Flink in the same project. That way the IDE will use the right version of
the artifact when executing the test.
Best,
Marton
On Sat, May
+1 for the proposal
@ggevay: I do think that it refers to you. :)
On Thu, May 12, 2016 at 10:40 AM, Gábor Gévay wrote:
> Hello,
>
> There are at least three Gábors in the Flink community, :) so
> assuming that the Gábor in the list of maintainers of the DataSet API
> is referring to me, I'll be
Hi Gabor,
I have checked out your branch and tried debugging WordCountPojo to
reproduce the behaviour. I am on a Mac with jdk1.8.0_91. I have received
the following error when trying to access the constructors of the class in
question:
Exception in thread "main" java.lang.VerifyError: (class:
org
uld be added.
> I think that users could be confused.
>
> Regards,
> Chiwan Park
>
> > On Apr 17, 2016, at 3:49 PM, Márton Balassi
> wrote:
> >
> > Hi Gábor,
> >
> > I think that adding the Janino dep to flink-core should be fine, as it
> has
> &g
com/apache/flink/blob/master/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala
>
> On 18 March 2016 at 19:37, Gábor Horváth wrote:
>
> > Thank you! I finalized the project.
> >
> >
> > On 18 March 2016 at 10:29, Már
Just a quick note: "[FLINK-3636] Add ThrottledIterator to WindowJoin jar"
is not needed on the release-1,0 branch as the example rewrite introducing
the ThrottledIterator is not present there. I see that Ufuk has already
pushed the commit there, it does no harm after all.
On Wed, Mar 30, 2016 at 1
le.com/Tuple-performance-and-the-curious-JIT-compiler-td10666.html
> ),
> >> and I wanted to make this information available to be able to
> incorporate
> >> this into that discussion. I have written this draft with the help of
> Gábor
> >> Gévay and Márton Balassi and I am op
Hey,
I think we came to the agreement that this PR is not mergeable right now,
so I am closing it. I personally find it inconsistent to not have the fully
API mirrored in Scala though, but this is something that we can revisit
when prepairing 2.0.
Best,
Marton
On Mon, Mar 14, 2016 at 8:14 PM, S
w methods. Maybe we can
> find a good way to resolve the issue without changing the signature of
> existing methods.
> And for tracking API breaking changes, maybe it makes sense to create a
> 2.0.0 version in JIRA and set the "fix-for" for the issue to 2.0.
>
> On Sun, M
nge because its API breaking.
> One of the promises of the 1.0 release is that we are not breaking any APIs
> in the 1.x.y series of Flink. We can fix those issues with a 2.x release.
>
> On Sun, Mar 13, 2016 at 5:27 AM, Márton Balassi
> wrote:
>
> > The JIRA issue is FLINK
The JIRA issue is FLINK-3610.
On Sat, Mar 12, 2016 at 8:39 PM, Márton Balassi
wrote:
>
> I have just come across a shortcoming of the streaming Scala API: it
> completely lacks the Scala implementation of the DataStreamSink and
> instead the Java version is used. [1]
>
> I wo
Hey,
I have just come across a shortcoming of the streaming Scala API: it
completely lacks the Scala implementation of the DataStreamSink and instead
the Java version is used. [1]
I would regard this as a bug that needs a fix for 1.0.1. Unfortunately this
is also api-breaking.
Will post it to JI
bit more, like
> >> > - when is it decided whether this project takes place?
> >> > - when would results be there?
> >> > - can we expect the results to be usable, i.e., how good is the
> >> student?
> >> > (no offence, but so far the results in GSoC wer
Mar 9, 2016 at 12:38 PM, Márton Balassi
> wrote:
>
> > Hey,
> >
> > I was wondering whether there is a way to access the Configuration from
> an
> > (Stream)ExecutionEnviroment or a RichFunction. Practically I would like
> to
> > set a temporary
Hey,
I was wondering whether there is a way to access the Configuration from an
(Stream)ExecutionEnviroment or a RichFunction. Practically I would like to
set a temporary persist path in the Configuration and access the location
somewhere during the topology.
I have followed the way the streaming
@Fabian: That is my bad, but I think we should be still on time. Pinged Uli
just to make sure. Proposal from Gabor and Jira from me are coming soon.
On Tue, Mar 8, 2016 at 11:43 AM, Fabian Hueske wrote:
> Hi Gabor,
>
> I did not find any Flink proposals for this year's GSoC in JIRA (should be
>
Thanks for initiating this Ufuk. Updated the streaming hashing mention -
whether it is api breaking is questionable, so I would place it last in the
list. But definitely good to mention it there.
On Thu, Mar 3, 2016 at 10:48 AM, Ufuk Celebi wrote:
> Hey all,
>
> let's make sure that we have a go
t; and the core flink
> dependencies with scope "compile"
>
> That way the example should run in the IDE out of the cox, and users that
> reference the libraries will still get the correct packaging (include the
> library in the user jar, but not additionally the core flink j
Great to see that. :)
On Fri, Feb 26, 2016 at 1:56 PM, Theodore Vasiloudis <
theodoros.vasilou...@gmail.com> wrote:
> I'm sure others noticed this as well yesterday, but the project has passed
> 1000 stars on Github,
> just in time for the 1.0 release ;)
>
> Here's to the next 1000!
>
> --Theo
>
.
> > >> >> The "trigger" for creating the release was that the number of
> > blocking
> > >> >> issues is 0 now.
> > >> >>
> > >> >> I did a quick check of the open pull requests yesterday evening and
> > >>
Issued JIRA ticket 3511 to make it referable in other discussions. [1]
[1] https://issues.apache.org/jira/browse/FLINK-3511
On Thu, Feb 25, 2016 at 3:36 PM, Márton Balassi
wrote:
> Recent changes to the build [1] where many libraries got their core
> dependencies (the ones included
Recent changes to the build [1] where many libraries got their core
dependencies (the ones included in the flink-dist fat jar) moved to the
provided scope.
The reasoning was that when submitting to the Flink cluster the application
already has these dependencies, while when a user writes a program
Thanks for creating the candidate Robert and for the heads-up, Slim.
I would like to get a PR [1] in before 1.0.0 as it breaks hashing behavior
of DataStream.keyBy. The PR has the feature implemented and the java tests
adopted, there is still a bit of outstanding fix for the scala tests. Gábor
Hor
Adding to Ufuk's answer: yes, cancelling the job frees up the resources. :)
Best:
Marton
On Fri, Feb 19, 2016 at 12:10 PM, Ufuk Celebi wrote:
> Yes, you can cancel it via the web frontend or the CLI interface [1].
>
> If you can send messages to the JobManager, you can also send a
> CancelJob
Hi guys,
They are at least already registered for serialization [1], so there should
be no intentional conflict as Theo has suggested.
[1]
https://github.com/apache/flink/blob/master/flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/common/FlinkMLTools.scala#L67-L73
Best,
Marton
On T
+1
On Wed, Jan 13, 2016 at 12:37 PM, Matthias J. Sax wrote:
> +1
>
> On 01/13/2016 11:51 AM, Fabian Hueske wrote:
> > @Stephan: You mean all tags should be protected, not only those under
> rel?
> >
> > 2016-01-13 11:43 GMT+01:00 Till Rohrmann :
> >
> >> +1 for protecting the master branch.
> >>
Thanks for writing this up, Gábor. As Aljoscha suggested chaining changes
all of these and makes it very tricky to work with these which should be
clearly documented. That was the reason while some time ago the streaming
API always copied the output of a UDF by default to avoid this ambiguous
cases
Thanks, Robert!
On Fri, Nov 27, 2015 at 5:02 PM, Vasiliki Kalavri wrote:
> Thank you Robert ^^
>
> On 27 November 2015 at 16:23, Till Rohrmann wrote:
>
> > Thanks Robert for being the release manager for 0.10.1
> >
> > On Fri, Nov 27, 2015 at 4:21 PM, Maximilian Michels
> > wrote:
> >
> > > Gr
Additionally as having multiple files under /output1.txt is standard in the
Hadoop ecosystem you can transparently read all the files with
env.readTextFile("/output1.txt").
You can also set parallelism on individual operators (e.g the file writer)
if you really need a single output.
On Fri, Nov 2
+1 for both
As we are planning to restructure the maven projects at the point that
breaks the PRs anyway, so going on step further at this point in time is
reasonable for me.
On Tue, Oct 20, 2015 at 2:37 PM, Matthias J. Sax wrote:
> big +1 for both!
>
> On 10/20/2015 02:31 PM, Ufuk Celebi wrote
Thanks for starting and big +1 for making it more prominent.
On Mon, Oct 19, 2015 at 2:53 PM, Fabian Hueske wrote:
> Thanks for starting this Kostas.
>
> I think the list is quite hidden in the wiki. Should we link from
> flink.apache.org to that page?
>
> Cheers, Fabian
>
> 2015-10-19 14:50 GMT
not doing something stupid and it was a bug
> that
> >> you guys where aware of.
> >>
> >> cheers Martin
> >>
> >> On Tue, Oct 6, 2015 at 8:09 AM, Aljoscha Krettek
> >> wrote:
> >>
> >> > Hi,
> >> >
I like Fabian's approach, you can also share these filters on JIRA.
On Fri, Oct 9, 2015 at 2:55 PM, Fabian Hueske wrote:
> Sorry, that was not the correct link.
> You can create a bookmark for this one:
>
>
> https://issues.apache.org/jira/issues/?jql=project%20%3D%20FLINK%20AND%20resolution%20%
+1
One minor comment: I suppose you implicitly mean that a committer can
shepherd her own PR.
On Wed, Oct 7, 2015 at 10:22 AM, Fabian Hueske wrote:
> @Matthias: That's a good point. Each PR should be backed by a JIRA issue.
> If that's not the case, we have to make the contributor aware of that
unfortunately. [1] Could you maybe produce a minimalistic example that we
can actually execute? :)
[1]
https://github.com/mbalassi/flink/commit/9f1f02d05e2bc2043a8f514d39fbf7753ea7058d
On Mon, Oct 5, 2015 at 10:06 PM, Márton Balassi
wrote:
> Thanks, I am checking it out tomorrow morning.
>
&g
n conjunction with the code. Each ERROR printout in the log relates to
> an accumulator receiving wrong values.
>
> cheers Martin
>
> On Sat, Oct 3, 2015 at 11:29 AM, Márton Balassi
> wrote:
>
>> Hey,
>>
>> Thanks for reporting the problem, Martin. I have not
Hey,
Thanks for reporting the problem, Martin. I have not merged the PR Stephan
is referring to yet. [1] There I am cleaning up some of the internals too.
Just out of curiosity, could you share the code for the failing test please?
[1] https://github.com/apache/flink/pull/1155
On Fri, Oct 2, 201
@Matthias: +1.
On Fri, Oct 2, 2015 at 11:27 AM, Stephan Ewen wrote:
> @matthias +1 for that approach
>
> On Fri, Oct 2, 2015 at 11:21 AM, Matthias J. Sax wrote:
>
> > It think, rename "flink-storm-compatibility-core" to just "flink-storm"
> > would be the cleanest solution.
> >
> > So in flink-
Great to see streaming graduating. :)
I like the outline, both getting rid of staging, having the examples
together and generally flattening the structure are very reasonable to me.
You have listed flink-streaming-examples under flink-streaming-connectors
and left out some less prominent maven mo
@Matthias: Please open the PR. Maybe tag it with [0.10.0-milestone-1].
Thanks,
Marton
On Fri, Sep 25, 2015 at 11:38 AM, Matthias J. Sax wrote:
> Do we need a Jira for the WebClient fix? Or can I just commit it?
>
> If anybody whats to review, please find it here:
> https://github.com/mjsax/fli
Hey Gyula,
I have been recently looking at the streaming UdfOperators and can not
recall a utility for the sources that you are looking for, but maybe I am
also missing it. :)
It would be a convenient addition though.
Best,
Marton
On Sun, Sep 13, 2015 at 8:59 PM, Gyula Fóra wrote:
> Hey All!
Welcome, Matthias :)
On Wed, Sep 2, 2015 at 5:21 PM, Aljoscha Krettek
wrote:
> Welcome, welcome, welcome :D
>
> On Wed, 2 Sep 2015 at 15:53 Fabian Hueske wrote:
>
> > Welcome on board Matthias!
> > On Sep 2, 2015 15:40, "Maximilian Michels" wrote:
> >
> > > Well done, Matthias! Waiting for mor
+1
On Wed, Aug 26, 2015 at 3:11 PM, Maximilian Michels wrote:
> We will have a proper minor release and a preview of 0.10. After all,
> a good compromise.
>
> +1
>
> On Wed, Aug 26, 2015 at 2:57 PM, Chiwan Park
> wrote:
> > Robert's suggestion looks good. +1
> >
> > Sent from my iPhone
> >
> >>
+1 for Vasia's suggestion
On Aug 22, 2015 8:07 PM, "Vasiliki Kalavri"
wrote:
> I just came across 2 more :/
> I'm also in favor of tracking these with JIRA. How about "test-stability"
> for a label?
>
> -V.
>
> On 21 August 2015 at 12:47, Matthias J. Sax >
> wrote:
>
> > I like the idea with the
Welcome Chesnay!
On Thu, Aug 20, 2015 at 7:29 PM, Henry Saputra
wrote:
> Welcome Chesnay!
>
> On Thu, Aug 20, 2015 at 2:18 AM, Robert Metzger
> wrote:
> > The Project Management Committee (PMC) for Apache Flink has asked Chesnay
> > Schepler to become a committer and we are pleased to announce
Hey Christian,
Thanks for the insider view on SPQR. I have to agree with Gyula that
dynamic topology build is not the highest priority for Flink currently, but
certainly a very interesting feature and one that has already been
requested by a couple of users.
As for none of the open source streami
manmat - Mátyás Manninger, aspired to be a GSoC student last year.
On Mon, Jul 27, 2015 at 12:32 PM, Kostas Tzoumas
wrote:
> Cool, I added a link to the website. Keep the corrections coming
>
> On Mon, Jul 27, 2015 at 11:30 AM, Vasiliki Kalavri <
> vasilikikala...@gmail.com> wrote:
>
> > Thanks
et. In each discrete time slice, the data set can be
> treated
> > > > like
> > > > >> a
> > > > >> > regular data set.
> > > > >> >
> > > > >> > Let's kick off a separate design for the d
Hi,
Hadoop is not a necessity for running Flink, but rather an option. Try the
steps of the setup guide. [1]
If you really nee HDFS though to get the best IO performance I would
suggest having Hadoop on all your machines running Flink.
[1]
https://ci.apache.org/projects/flink/flink-docs-release-0
Generally I agree with the new design. Two concerns:
1) Does KeyedDataStream replace GroupedDataStream or is it the latter a
special case of the former?
The KeyedDataStream as described in the design document is a bit unclear
for me. It lists the following usages:
a) It is the first step in bui
As Kostas mentioned the failure mechanisms for streaming and batch
processing are different, but you can expect exactly once processing
guarantees from both of them.
On Thu, Jul 9, 2015 at 2:43 PM, 马国维 wrote:
> hi, everyoneThe doc say Flink Streaming use "Barriers" to ensure
> "exactly once."
I have found two off-by-one issues in the windowing code.
The first may result in duplicate data in the last window and is easy to
fix. [1]
The second may result data being swallowed in the last window, and is also
not difficult to fix. [2]
I've talked to Aljoscha about fixing the second one, an
Hey,
Thanks for picking up the issue. This value can be specified as
"execution-retries.delay" in the flink-conf.yaml. Hence you can check the
associated value in the ConfigConstants [1] and track the way it is used.
It is passed a couple of times, but is ultimately used in ExecutionGraph.
[2]
[1
Hi,
Flink currently does not have explicit Api support for that, but is
definitely possible to do. In fact Gyula (cc-d) mocked up a prototype for a
similar problem some time ago.
The idea needs some refinement to properly support all the viable use cases
though and the streaming Api currently has
Interesting, it does not appear on travis or my local machine, but both run
linux. (Ubuntu 14.10, Java 8, mvn 3.0.5 in the latter case)
On paper the remote-resources plugin is only used for the Eclipse
integration and should not even effect the maven build itself, at least
that the comment says in
@Matthias: Your point of working with a minimal number of clear concepts is
desirable to say the least. :)
The reasoning behind the KeyedDatastream is to associate Flink persisted
operator state with the keys of the data that produced it, so that stateful
computation becomes scalabe in the future.
Thanks, Max. I would like to present the streaming stuff a bit differently,
updating it.
On Tue, Jun 23, 2015 at 1:59 PM, Maximilian Michels wrote:
> Hi everyone,
>
> With the vote now being passed, we should think about the release
> announcement. I've created a document which basically contain
Added a ticket, so we can refer to it.
https://issues.apache.org/jira/browse/FLINK-2257
On Mon, Jun 22, 2015 at 2:14 PM, Ufuk Celebi wrote:
>
> On 22 Jun 2015, at 14:00, Maximilian Michels wrote:
>
> > Hi Marton,
> >
> > Thanks for spotting this issue. It is a bug we should list under known
>
Hey,
Due to the effort invested to the Kafka connector mainly by Robert and
Gabor Hermann we are going to ship a fairly nice solution for reading from
and writing to Kafka with 0.9.0. This is the most prominent streaming
connector currently, and rightfully so as pipeline level end-to-end exactly
o
Hey,
I have found that open and close methods of streaming RichWindowFunctions
are not called. I have the fix [1] as I did implement a fix for a similar
issue some time ago, [2] sorry for not realizing it back then.
[1] https://github.com/apache/flink/pull/855
[2] https://issues.apache.org/jira/b
Thanks, Ted - great news. :)
@Stephan: Streaming is mentioned, but not really prominent. "... could be
just the ticket for fusing streaming analytics with historical analytics."
On Sat, Jun 20, 2015 at 8:23 PM, Ted Dunning wrote:
> On Sat, Jun 20, 2015 at 12:52 AM, Stephan Ewen wrote:
>
> > T
+1
Verified checksums and signatures
Built from source
Run bundled batch examples on a local setup
Run streaming example with checkpointed operator state on cluster, killed
taskmanagers underneath
On Wed, Jun 17, 2015 at 2:35 AM, Maximilian Michels wrote:
> Dear community!
>
> After careful tes
@Max: The PR is good to go on my side. Does the job, could be a bit nicer
though. Added to the document.
On Tue, Jun 16, 2015 at 10:54 PM, Aljoscha Krettek
wrote:
> I added the two relevant Table API commits to the release doc.
>
> On Tue, 16 Jun 2015 at 21:49 Maximilian Michels wrote:
>
> > +1
Ok, never mind. I got it: it is decided later dependent on the hadoop
profile. Makes sense.
On Tue, Jun 16, 2015 at 1:24 PM, Márton Balassi
wrote:
> Hey,
>
> While reading through the flink-parent pom I have stumbled upon this. [1]
>
> error
>
> I am just curious why do w
Hey,
While reading through the flink-parent pom I have stumbled upon this. [1]
error
I am just curious why do we use this. :)
[1] https://github.com/apache/flink/blob/master/pom.xml#L72
@Aljoscha:
1) I think this just means that you can set the state backend on a
taskmanager basis.
3) This is a serious issue then. Is it work when you set it in the
flink-conf.yaml?
On Mon, Jun 15, 2015 at 3:17 PM, Aljoscha Krettek
wrote:
> So, during my testing of the state checkpointing on a cl
As for outstanding issues I think streaming is good to go as far as I know.
I am personally against including all libraries - at least speaking for the
streaming connectors. Robert, Stephan and myself had a detailed discussion
on that some time ago and the disadvantage of having all the libraries i
@Till: This also apples to the streaming connectors.
On Fri, Jun 12, 2015 at 9:45 AM, Till Rohrmann wrote:
> Hi guys,
>
> I just noticed while testing the TableAPI on the cluster that it is not
> part of the dist module. Therefore, programs using the TableAPI will only
> run when you put the Tab
l forward
> ("DataStream.forward()") would not work correctly.
>
> Please correct me, if I got something mixed up.
>
>
> -Matthias
>
> On 06/10/2015 02:42 PM, Márton Balassi wrote:
> > Hey,
> >
> > As the storm-compatibility-core b
As for the current streaming dependencies: flink-java is not needed at all,
flink-clients is only needed in specific cases. Thanks for spotting it, I
am cleaning it.
As for Gabor's problem: maybe we should move the CaseClassTypeInfo to
flink-core then given Till's comment. Any opinions on that?
O
Hey,
As the storm-compatibility-core build goes fine this is a dependency issue
with storm-compatibility-examples. As a first try replace:
org.apache.flink
flink-streaming-core
${project.version}
test
tests
with
org.apache.flink
flink-streaming-core
${project.version}
test
As for the streaming commit cherry-picked to the release branch:
This is an unfortunate communication issue, let us make sure that we
clearly communicate similar issues in the future.
As for FLINK-2192: This is essentially a duplicate issue of the testability
of the streaming iteration. Not a bloc
I agree that for the sake of the above mentioned use cases it is reasonable
to add this to the release with the right documentation, for machine
learning potentially loosing one round of feedback data should not matter.
Let us not block prominent users until the next release on this.
On Wed, Jun
Added F7 Running against Kafka cluster for me in the doc. Doing it
tomorrow.
On Mon, Jun 8, 2015 at 7:00 PM, Chiwan Park wrote:
> Hi. I’m very excited about preparing a new major release. :)
> I just picked two tests. I will report status as soon as possible.
>
> Regards,
> Chiwan Park
>
> > On
The problem is still there. @Aljoscha: It would be great if you could take
it.
On Mon, Jun 8, 2015 at 9:41 AM, Gyula Fóra wrote:
> I agree with Marton. I thought Aljoscha was working on that.
>
> On Monday, June 8, 2015, Márton Balassi wrote:
>
> > FLINK-2054 is definite
FLINK-2054 is definitely a problem if it persists. Sorry for missing it,
solving it asap.
On Mon, Jun 8, 2015 at 7:18 AM, Ufuk Celebi wrote:
>
> On 08 Jun 2015, at 00:22, Robert Metzger wrote:
>
> > What about https://issues.apache.org/jira/browse/FLINK-2177 and
> > https://issues.apache.org/ji
Our codebase github mirror is out of sync for at least 8 hours. I have
filed a JIRA ticket for Infra. [1]
[1] https://issues.apache.org/jira/browse/INFRA-9777
101 - 200 of 283 matches
Mail list logo