Release vote just started. If everything works, the release should be out
on Monday.
If you like, you can use the release candidate version and contribute to
the release testing. ;)
Add this to your POM:
rc0
Flink 0.9.1 RC0
https://repository.apache.org/content/repositories/orgapacheflin
Oops, seems that Stephan's email covers my answer plus the plans to provide
transactional sinks :-)
On Thu, Aug 27, 2015 at 1:25 PM, Kostas Tzoumas wrote:
> Note that the definition of "exactly-once" means that records are
> guaranteed to be processed exactly once by Flink operators, and thus st
Note that the definition of "exactly-once" means that records are
guaranteed to be processed exactly once by Flink operators, and thus state
updates to operator state happen exactly once (e.g., if C had a counter
that x1, x2, and x3 incremented, the counter would have a value of 3 and
not a value o
Hi!
The "exactly once" guarantees refer to the state in Flink. It means that
any aggregates and any user-defined state will see each element once.
This guarantee does not automatically translate to the outputs to the
outside world, as Marton said. Exactly once output is only possible (in
general
Dear Zhangrucong,
>From your explanation it seems that you have a good general understanding
of Flink's checkpointing algorithm. Your concern is valid, by default a
sink C with emits tuples to the "outside world" potentially multiple times.
A neat trick to solve this issue for your user defined si
Hi:
The document said Flink can guarantee processing each tuple exactly-once,
but I can not understand how it works.
For example, In Fig 1, C is running between snapshot n-1 and snapshot
n(snapshot n hasn't been generated). After snapshot n-1, C has processed tuple
x1, x2, x3 and alread
I think this commit fixed it in the 0.9 branch
(c7e868416a5b8f61489a221ad3822dea1366d887) so it should be good in the
release.
On Thu, Aug 27, 2015 at 11:52 AM, Chiwan Park wrote:
> Hi Michele,
>
> We’re doing release process for 0.9.1. Ufuk Celebi will start vote for
> 0.9.1 release soon.
>
> R
Hi Michele,
We’re doing release process for 0.9.1. Ufuk Celebi will start vote for 0.9.1
release soon.
Regards,
Chiwan Park
> On Aug 27, 2015, at 6:49 PM, Michele Bertoni
> wrote:
>
> Hi everybody,
> I am still waiting for version 0.9.1 to solve this problem, any idea on when
> it will be r
Hi everybody,
I am still waiting for version 0.9.1 to solve this problem, any idea on when it
will be released?
Thanks
Best,
michele
Il giorno 15/lug/2015, alle ore 15:58, Maximilian Michels
mailto:m...@apache.org>> ha scritto:
I was able to reproduce this problem. It turns out, this has
Matthias has a very good point! Have a look at the System and see what
strikes you as most interesting.
For example
- runtime
- Graph Algorithms
- ML algorithms
- streaming core/connectors
- Storm streaming layer.
On Thu, Aug 27, 2015 at 10:37 AM, Matthias J. Sax <
mj...@informatik.hu
One more thing. Not every open issue is documented in JIRA (even if you
try to do this). You can also have a look into the wiki:
https://cwiki.apache.org/confluence/display/FLINK/Apache+Flink+Home
So if you are interested to work on a specific component you might try
to talk to the main contribute
> On 27 Aug 2015, at 09:33, LINZ, Arnaud wrote:
>
> Hi,
>
> Ok, I’ve created FLINK-2580 to track this issue (and FLINK-2579, which is
> totally unrelated).
Thanks :)
> I think I’m going to set up my dev environment to start contributing a little
> more than just complaining J.
If you nee
Hi,
Ok, I’ve created FLINK-2580 to track this issue (and FLINK-2579, which is
totally unrelated).
I think I’m going to set up my dev environment to start contributing a little
more than just complaining ☺.
Best regards,
Arnaud
De : ewenstep...@gmail.com [mailto:ewenstep...@gmail.com] De la p
13 matches
Mail list logo