ext steps...
>
>
>
> On Thu, Jan 8, 2015 at 2:07 PM, Robert Metzger
> wrote:
>
> > Please wait with merging 268. There is still a minor open issue.
> >
> > On Thu, Jan 8, 2015 at 1:28 PM, Márton Balassi >
> > wrote:
> >
> > > +1 for mer
t; > [runtime] Fix KryoSerializer to not swallow EOFExceptions)
> >
> >
> >
> > On Thu, Jan 8, 2015 at 11:55 AM, Robert Metzger
> > wrote:
> >
> > > +1 for merging pull request 268. It makes Flink even more compatible
> with
> > > hadoop.
>
utils.converters.FloatArrayConverter
> > [WARNING] - org.apache.commons.beanutils.NestedNullException
> > [WARNING] - org.apache.commons.beanutils.ConvertingWrapDynaBean
> > [WARNING] - org.apache.commons.beanutils.converters.LongArrayConverter
> > [WARNING] - org.ap
Hi,
I very much like the "PIG Journal" here:
https://cwiki.apache.org/confluence/display/PIG/Pig+Journal
Its basically a nice view (however outdated in that case) on whats going on
in the PIG community. You can see finished features on the top, current
features being developed in the middle and id
Robert Metzger created FLINK-1370:
-
Summary: Build website automatically
Key: FLINK-1370
URL: https://issues.apache.org/jira/browse/FLINK-1370
Project: Flink
Issue Type: Task
rc archive
- the source archive builds (mvn clean install) (including rat license
header check, checkstyle and all tests)
- gpg signatures are valid: gpg --verify flink-0.8.0-src.tgz.asc
flink-0.8.0-src.tgz
On Wed, Jan 7, 2015 at 7:55 PM, Robert Metzger wrote:
> -1
>
> There is a
-1
There is a major issue with the hadoop1 and hadoop2 versions: We made
hadoop2 the default profile, so the 0.8.0 version will pull the hadoop2
dependencies.
We basically need an explicit 0.8.0-hadoop1 version now.
The release candidate contains a 0.8.0 version which has hadoop2 activated
by defa
Amazing work, thank you!
On Wed, Jan 7, 2015 at 4:43 PM, Timo Walther wrote:
> Hi everyone,
>
> as some of you may already know, I have worked on a patch for OpenJDK that
> introduces a compiler option for javac which allows to compile and run
> programs with Java 8 Lambda Expressions in Flink w
I would only start a vote if somebody objects.
How about adding this rule to our website, to make it even more official. I
would like to establish a document that contains all the rules we agreed on.
Similarly to the coding guidelines (
http://flink.apache.org/coding_guidelines.html) we could esta
it
> -
> -... and import it as a Maven project into your Eclipse. You have to
> manually add the previously built mongo-hadoop jar-file as a dependency.
> -You can now press the "Run" button and see how Stratosphere
> executes the little program. It was running
ure). But please correct me
> > if I'm wrong.
> >
> > On Mon, Jan 5, 2015 at 11:20 PM, Henry Saputra
> > wrote:
> >
> >> When you said "finish the release" meaning distribute/publish a TLP
> >> release in the incubating branch repo?
> &
i, Jan 2, 2015 at 10:37 AM, Robert Metzger wrote:
> Thanks for the update.
>
> Happy new year to everyone.
>
> It seems that we have to wait for INFRA to finish the LDAP group creation
> for the new TLP first, since this is connected to the maven repository.
> If nothing happe
Do we have any tests for the scala streaming API ?
With tests and documentation missing there is very little time.
I hope that we get the first release candidate out tomorrow.
On Mon, Jan 5, 2015 at 3:33 PM, Gyula Fóra wrote:
> Yes, we can cherry-pick it, thats not a problem.
>
> On Mon, Jan 5,
Hi,
I'm fine with consolidating some of the sub-modules. I'm currently
preparing a pull request for YARN which will move flink-yarn out of the
"flink-addons".
Regarding the "flink-" prefix: I'm voting to keep the prefix because the
resulting jars will contain "flink" in their name. Its much easie
> near.
> > > I also fell motivated to write down my experiences in a documentation
> to
> > > spare some time for up and coming release managers.
> > >
> > > Best,
> > >
> > > Marton
> > >
> > > On Fri, Dec 19, 2014 at 9
> 0.8.0
> > > > That gives the code a bit more testing/use and us a bit more time to
> > > > thoroughly think the implications through.
> > > >
> > > >> On Tue, Dec 16, 2014 at 5:30 PM, Timo Walther
> > > wrote:
> > >
Robert Metzger created FLINK-1338:
-
Summary: Remove incubating disclaimers and update version name
Key: FLINK-1338
URL: https://issues.apache.org/jira/browse/FLINK-1338
Project: Flink
Issue
Hi,
Sorry that you ran into that issue. The problem is that the "fat jar
builder" is removing all packages in the "org.apache.flink" namespace.
Moving the class containing the main() to a new location (say
net.internetmemory) will probably resolve the issue.
Best,
Robert
On Thu, Dec 18, 2014 at
Thank you for putting it to the mailing list!
I'm super excited.
On Wed, Dec 17, 2014 at 9:49 PM, Alan Gates wrote:
>
> Congratulations, we're a TLP!
>
> Alan.
>
> *From:* Brett Porter
> *Date:* December 17, 2014 at 12:18
> *To:* committ...@apache.org
> *Subject:* ASF Board Meeting Summary -
Hi,
it is indeed a bit annoying to work in parts of the system where you need
classes from both systems. But I think thats "only" the case in the YARN,
Hadoop Compat and FileSystem code.
I'm not sure if its a good idea to rename classes such as "Configuration"
or "Path", which are used everywhere
Thank you Stephan!
Now its time to merge the Akka/Jobmanager rework into the master.
@Henry: I'm trying to tag the JIRAs with the correct version. So you should
be able to find the issues fixed in the release.
I would also like to include the fix for
https://issues.apache.org/jira/browse/FLINK-1
Robert Metzger created FLINK-1333:
-
Summary: Getter/Setter recognition for POJO fields with generics
is not working
Key: FLINK-1333
URL: https://issues.apache.org/jira/browse/FLINK-1333
Project
I've worked on adding Kryo support to Flink today. I would really like to
include this into the release. Many users were complaining about this (
https://issues.apache.org/jira/browse/FLINK-610).
I've based my work on top of Timo's pull request. If we're not going to
merge it, I will have to rebas
Hi,
DFF seems to be an interesting tool. There is no Flink support for it yet,
but it should be possible to use it as a back-end.
On Sat, Dec 13, 2014 at 8:33 AM, sirinath wrote:
>
> Hi,
>
> Is it possible to support DFF (http://ddf.io/) in Flink?
>
> Suminda
>
>
>
> --
> View this message in co
breath of types of
> >> metric could and should be collected.
> >>
> >> - Henry
> >>
> >> [1] https://dropwizard.github.io/metrics/3.1.0/getting-started/
> >> [2] https://spark.apache.org/docs/1.0.1/monitoring.html
> >> [3] https://issu
Thank you Gyula. Can you past the document contents into our wiki?
On Thu, Dec 11, 2014 at 1:58 PM, Gyula Fóra wrote:
> Hey all,
>
> For tomorrow's hangout Paris wrote us a nice summary of our current plans
> for the streaming api.
>
> You can find it here:
>
> https://docs.google.com/document/d
+1
Everybody stumbled across this.
On Thu, Dec 11, 2014 at 8:55 AM, Stephan Ewen wrote:
> How about we link this Screenshot from the faq as well and add the pointers
> to the dev guide...
>
+1.
I think adding the TypeHints pull request to the release is a good idea.
This allows us to give users a workaround if the type extraction is not
working.
I'll have a look at the Hbase hadoop1 support.
On Wed, Dec 10, 2014 at 7:35 PM, Stephan Ewen wrote:
> Hi everyone!
>
> I would suggest t
> > We could do it ourselves I believe as long as that channel is available.
> >
> > Should we go with #flink, #asfflink, or #apacheflink as name of the
> > channel?
> > Other suggestion is welcomed.
> >
> > - Henry
> >
> > On Tue, Dec 9, 2014 at 1:03 A
+1
I'm idling on #flink @ freenode. (I guess thats the place where most of the
Apache Projects have their channel).
What do we have to do to "get" the channel?
On Tue, Dec 9, 2014 at 6:23 AM, Chiwan Park wrote:
> +1
> Nice Idea!
>
> —
> Chiwan Park (Sent with iPhone)
>
>
> > On Dec 9, 2014, a
I agree with Ufuk.
I'm uncertain if its a good idea to delay the 0.8.0 for these "last minute"
streaming features.
I have the impression that the current master is really stable and very
well tested. If we now bring in some prototypes and last minute features,
we probably end up discussing a 0.8.1
I'm voting to release the current master as Flink 0.8.0.
Once we have merged these two major changes (
https://github.com/apache/incubator-flink/pull/149,
https://github.com/apache/incubator-flink/pull/254) in the queue we will
need some time until we reach a well tested, stable state again.
On M
Hey Nils,
I have played around a bit with a little prototype. You can find the code
here: https://github.com/rmetzger/incubator-flink/tree/flink456 (its
another branch in my repo).
You can see the changes that I applied on top of Till's Akka branch here:
https://github.com/rmetzger/incubator-flink
Robert Metzger created FLINK-1304:
-
Summary: Add tests for the I/O Format wrappers
Key: FLINK-1304
URL: https://issues.apache.org/jira/browse/FLINK-1304
Project: Flink
Issue Type
It seems to depend on the issue ;)
"FLINK-1074" is the first hit on google.
Searching for "site:issues.apache.org/jira FLINK" will show you which
issues are indexed.
I can recommend searching for JIRAs with google mail ;)
On Fri, Dec 5, 2014 at 4:46 PM, Kostas Tzoumas wrote:
> Hey,
>
> I have n
Robert Metzger created FLINK-1302:
-
Summary: JDBCInputFormat does not implement NonParallelInput
interface
Key: FLINK-1302
URL: https://issues.apache.org/jira/browse/FLINK-1302
Project: Flink
e and
> > hereby are appointed to serve as the initial members of the
> > Apache Flink Project:
> >
> > * Marton Balassi
> > * Ufuk Celebi
> > * Stephan Ewen
> > * Gyula Fora
> > * Alan Gates
As possible workarounds, you could
a) Implement your own serialization by implementing the "Value" interface.
b) Use the Hadoop
http://hadoop.apache.org/docs/r2.3.0/api/org/apache/hadoop/io/MapWritable.html
MapWritable Class. You have to use hadoop's LongWritable and IntWritable
for the types of th
;
> > >
> > > 2014-12-02 14:43 GMT+01:00 Ufuk Celebi :
> > >
> > > > Have you also thought about adding the statistics collection with the
> > > > writers, i.e. the collector or record writer?
> > > >
> > > > If all you care abou
Yes. I also got the impression that you are looking for something slightly
different.
It is probably easier for you right now to "hack" something into the system
to get these statistics.
On Tue, Dec 2, 2014 at 2:25 PM, Alexander Alexandrov <
alexander.s.alexand...@gmail.com> wrote:
> I checked t
The thread mentioned by Ufuk is an ongoing discussion, thats why there is
no JIRA yet.
To my understanding, its a student doing a project on Flink.
Also, I would like to give you the same advice I already gave to Nils: I
would highly recommend using Till's Akka branch for starting to work on
that.
ScalaCsvInputFormat using the CaseClassSerializer won't work
> > properly anymore with the CaseClassSerializer set to be stateful.
> >
> >
> > On Mon, Dec 1, 2014 at 12:50 PM, Ufuk Celebi wrote:
> >
> > > On Mon, Dec 1, 2014 at 12:15 PM, Robert Metzge
Robert Metzger created FLINK-1295:
-
Summary: Add option to Flink client to start a YARN session per job
Key: FLINK-1295
URL: https://issues.apache.org/jira/browse/FLINK-1295
Project: Flink
Robert Metzger created FLINK-1294:
-
Summary: Add Performance and Tuning Guide to documentation
Key: FLINK-1294
URL: https://issues.apache.org/jira/browse/FLINK-1294
Project: Flink
Issue Type
> >> anyways.
> >>
> >> In this context, I would indeed suggest some form of Github.
> >>
> >> Herzliche Grüße
> >> Volker Markl
> >>
> >> Von meinem iPad gesendet, daher evtl. kurz und knappe Nachricht
> >>
> >&
I looked through the commits since the last release and there is a huge
amount of improvements we could include.
I was thinking about limiting the commits for the bugfix release to a
absolute minimum. Only those issues users complained about or cause the
system not to work at all should be included
What solved my recent Scala / IntelliJ problems (which started occurring
after migrating to Intellij 14) was adding this jar
"/home/robert/.m2/repository/org/scalamacros/paradise_2.10.4/2.0.1/paradise_2.10.4-2.0.1.jar"
on the scala compiler settings page in IntelliJ as a compiler plugin.
(And I thi
Kostas said in his first mail that we made two successful releases so far.
Actually, we had 3 releases in the incubator (0.6, 0.6.1, 0.7.0). A bug fix
release is not a big effort for the community code-wise, but we did the
whole voting/compliance check three times so far ;)
I'm subscribed to a few
Robert Metzger created FLINK-1288:
-
Summary: YARN ApplicationMaster sometimes fails to allocate the
specified number of workers
Key: FLINK-1288
URL: https://issues.apache.org/jira/browse/FLINK-1288
Robert Metzger created FLINK-1287:
-
Summary: Improve File Input Split assignment
Key: FLINK-1287
URL: https://issues.apache.org/jira/browse/FLINK-1287
Project: Flink
Issue Type: Improvement
Hi Nils,
I'm not sure if its a good idea to use JMX. I fear that we are
overengineering something here for features that we don't really need.
I don't know any tools that can evaluate these JMX information (I think
thats the main argument for using JMX).
Also doing this kind of monitoring (connect
Hi,
great idea!
Maybe we should use the Wiki for such a list? It would make it easier for
users to just drop a link to a github repo of an algorithm implementation.
On Mon, Nov 24, 2014 at 4:05 PM, Kruse, Sebastian
wrote:
> Hi everyone,
>
> at HPI, we recently had the idea of a projects page
+1 for Marton as a release manager. Let me know if you need any help.
I'm trying to find some time today to collect a list of commits I'd like to
include.
On Mon, Nov 24, 2014 at 10:51 AM, Márton Balassi
wrote:
> +1 There are a couple of streaming bugfix commits that I'd like to push
> there.
>
Hi,
currently, we are not compressing the network data. However, it should be
easy to add support for that.
Ufuk can probably elaborate on the details, but he told me some time ago
that we can add code to compress outgoing network buffers using snappy or
other fast compression algorithms. In parti
Robert Metzger created FLINK-1266:
-
Summary: Generalize Flink's DistributedFileSystemClass to a Hadoop
FileSystem wrapper
Key: FLINK-1266
URL: https://issues.apache.org/jira/browse/FLINK-1266
Pr
Hi everyone,
In the PPMC we have voted to add Timo Walther as a new committer to Flink.
Congrats Timo! Keep up the good work.
Cheers,
Robert
Robert Metzger created FLINK-1252:
-
Summary: Add a serializer for Date
Key: FLINK-1252
URL: https://issues.apache.org/jira/browse/FLINK-1252
Project: Flink
Issue Type: Improvement
Robert Metzger created FLINK-1251:
-
Summary: CollectionInputFormat is not able to handle Enums
Key: FLINK-1251
URL: https://issues.apache.org/jira/browse/FLINK-1251
Project: Flink
Issue Type
Hi,
Maybe its an issue with the classpath? As far as I know is Hadoop reading
the configuration files from the classpath. Maybe is the hbase-site.xml
file not accessible through the classpath when running on the cluster?
On Wed, Nov 12, 2014 at 7:40 PM, Flavio Pompermaier
wrote:
> Today we tried
Robert Metzger created FLINK-1236:
-
Summary: Add support for localization of Hadoop Input Splits
Key: FLINK-1236
URL: https://issues.apache.org/jira/browse/FLINK-1236
Project: Flink
Issue
Cool! Its good to see meetup talks about Flink every month now.
I've tweeted about it.
We do not really have an events page. There was something in the news
section recently, but I think a dedicated page for past and upcoming talks
would be much better.
On Wed, Nov 12, 2014 at 10:27 AM, Stephan
Robert Metzger created FLINK-1234:
-
Summary: Make Hadoop2 profile default
Key: FLINK-1234
URL: https://issues.apache.org/jira/browse/FLINK-1234
Project: Flink
Issue Type: Improvement
> You can add:
> >>
> >> - New integration with other Apache Projects : MRQL and TEZ
> >> (Hadoop/YARN/HBase has always been)
> >>
> >> Stephan
> >>
> >>
> >> On Wed, Oct 29, 2014 at 5:58 PM, Márton Balassi <
> b
Hi,
here is what I've drafted for the report:
Flink
Flink is an open source system for expressive, declarative, fast, and
efficient data analysis. Flink combines the scalability and programming
flexibility of distributed MapReduce-lik
I personally think that one of the most important goals of our project is
to get new users trying it out. Therefore, we need to avoid any possible
entry barrier.
Sadly, it seems that people (in particular companies) are still using Java
6, even though its not being maintained anymore and potentiall
Hi,
the vote has passed with +3 binding votes from the IPMC.
+1 votes are from:
Justin Mclean
Alan Gates
Henry Saputra
Thank you for checking the release. I'll publish the release now
On Sun, Oct 26, 2014 at 1:28 AM, Justin Mclean
wrote:
> Hi,
>
> +1 binding
>
> - vote good
> - artefact n
Hi all,
This is our second try to release Flink 0.7.0-incubating. The last time we
found a critical issue while the vote was running, so I had to cancel the
vote here.
The 0.7.0-incubating release will be the second major-version release of
Flink in the incubator.
The vote thread can be found he
Thanks everybody for voting on this release.
Lets hope we don't find a bug in the last minute ... ;)
Binding votes:
Fabian Hueske
Kostas Tzoumas
Henry Saputra
Stephan Ewen
Robert Metzger
Ufuk Celebi
makes +6 binding votes for the release.
I'll post the vote in the incubator once t
PM, Henry Saputra
> wrote:
>
>> Signature files look good
>> Checksum files look good
>> NOTICE and LICENSE files look good
>> No 3rd party executables in source artifact
>> Source compile and tests passed
>> Run simple example in standalone.
>&g
Yes, for sorted groups, you need to use Pojos or Tuples.
I think you have to split the input lines manually, with a mapper.
How about using a TupleN<...> with only the fields you need? (returned by
the mapper)
if you need all fields, you could also use a Tuple2 where
the first position is the sort
Robert Metzger created FLINK-1179:
-
Summary: Add button to JobManager web interface to request stack
trace of a TaskManager
Key: FLINK-1179
URL: https://issues.apache.org/jira/browse/FLINK-1179
+1 Very good idea.
On Mon, Oct 20, 2014 at 2:37 PM, Márton Balassi
wrote:
> +1 on the Budapest side. We're using Flink co-located with a HDFS2 cluster.
>
> On Mon, Oct 20, 2014 at 2:15 PM, Stephan Ewen wrote:
>
> > Hi!
> >
> > I was wondering whether it makes sense to make hadoop 2 the default
Robert Metzger created FLINK-1172:
-
Summary: Links to APIs inside Iterations guide broken
Key: FLINK-1172
URL: https://issues.apache.org/jira/browse/FLINK-1172
Project: Flink
Issue Type: Bug
ow my progress. :)
>
> On Sat, Oct 18, 2014 at 2:59 PM, Robert Metzger
> wrote:
>
> > I don't know if its okay for you to start with a project outside our main
> > codebase, but we really need a QA-bot for validating pull requests:
> > https://issues.apache.
Please vote on releasing the following candidate as Apache Flink
(incubating) version 0.7.0
This release will be the second major release for Flink in the incubator.
-
The commit to be voted on is in the branch "release-0.7.0-rc2" (commi
+1
What about the sporadically failing scala tests you've mentioned. Are they
contained in #159?
On Sat, Oct 18, 2014 at 12:36 AM, Stephan Ewen wrote:
> How about including these Pull Requests in the new release candidate:
>
>
> InputSplit Localization -
> https://github.com/apache/incubator-fl
23 AM, Robert Metzger
wrote:
> Hi Ajay,
>
> new contributors are always very welcome in our community.
> There is a "How to contribute guide" on our website:
> http://flink.incubator.apache.org/how-to-contribute.html
> It contains some general advice on how to start.
>
&g
Hi Ajay,
new contributors are always very welcome in our community.
There is a "How to contribute guide" on our website:
http://flink.incubator.apache.org/how-to-contribute.html
It contains some general advice on how to start.
If you want, we can discuss together which issue you want to work on i
e those commits
> affect user facing code anyway.
>
> Cheers,
>
> Marton
>
> On Fri, Oct 17, 2014 at 4:53 PM, Robert Metzger
> wrote:
>
> > There were 4 commits to the streaming subsystem after the 0.7-branch.
> > Should they end up in the 0.7 release or are they
.
On Fri, Oct 17, 2014 at 3:11 PM, Robert Metzger wrote:
> As per discussion here (
> http://mail-archives.apache.org/mod_mbox/incubator-flink-dev/201410.mbox/%3CJIRA.12748855.141353790.284973.1413537933789%40Atlassian.JIRA%3E),
> I invalidate the release candidate voted on here.
&
, Oct 16, 2014 at 9:06 PM, Robert Metzger wrote:
> The vote has passed with:
> +1 votes: 6 (all PPMC votes)
> -1 votes: 0
>
>
> Thanks everybody for helping to check the release!
> I'll post on the incubator list once this message appears in the archives.
>
>
> R
Yes, that was intentionally.
> >
> > The whole point of using a parallel engine is to process large datasets.
> > Otherwise you could do it in Python on a single box...
> > Remote reads will severely impact the performance and might cause
> > significant performance reg
ian Hueske wrote:
> This is a critical issue and sounds bit like a release blocker for 0.7 to
> me.
>
> Other opinions?
>
> 2014-10-17 11:25 GMT+02:00 Robert Metzger (JIRA) :
>
> > Robert Metzger created FLINK-1170:
> > -
&g
Robert Metzger created FLINK-1170:
-
Summary: Localization of InputSplits is not working properly
Key: FLINK-1170
URL: https://issues.apache.org/jira/browse/FLINK-1170
Project: Flink
Issue
Hi all,
The Flink incubator community has successfully voted for releasing version
0.7.0-incubating.
The vote thread can be found here:
http://mail-archives.apache.org/mod_mbox/incubator-flink-dev/201410.mbox/%3CCAGr9p8DeNoa0R0%3DyFjmr4hbBwJP%2BvFJdeCFsh8ozjcQ8z7Ds9Q%40mail.gmail.com%3E
The vote
2014-10-16 18:27:08, "Kostas Tzoumas" wrote:
> >No, you don't need to install HDFS. You can use Flink without HDFS.
> >
> >On Thu, Oct 16, 2014 at 12:17 PM, Robert Metzger
> >wrote:
> >
> >> Hi,
> >>
> >> good to hear that you
The vote has passed with:
+1 votes: 6 (all PPMC votes)
-1 votes: 0
Thanks everybody for helping to check the release!
I'll post on the incubator list once this message appears in the archives.
Robert
On Thu, Oct 16, 2014 at 6:43 PM, Robert Metzger wrote:
> +1
>
>
> I'
joscha Krettek
> wrote:
> > +1
> >
> > I downloaded the hadoop2 release, ran local, submitted a job to the
> > web-frontend, checked job-manager interface.
> >
> > On Wed, Oct 15, 2014 at 2:34 PM, Robert Metzger
> wrote:
> >> I forgot to put a
Hi,
good to hear that you are using Flink.
HDFS is a distributed file system for reliably storing huge amounts of
data. Many users of Flink use HDFS to store all kinds of data on it.
This includes both input data for their jobs and also results.
Often, Flink and HDFS are installed next to each ot
e and passed tests
> Run simple examples in standalone
>
> +1
>
> - Henry
>
>
> On Mon, Oct 13, 2014 at 9:59 AM, Robert Metzger
> wrote:
> > Please vote on releasing the following candidate as Apache Flink
> > (incubating) version 0.7.0
> >
> >
Robert Metzger created FLINK-1166:
-
Summary: Add a QA bot to Flink that is testing pull requests
Key: FLINK-1166
URL: https://issues.apache.org/jira/browse/FLINK-1166
Project: Flink
Issue
Please vote on releasing the following candidate as Apache Flink
(incubating) version 0.7.0
This release will be the second major release for Flink in the incubator.
-
The commit to be voted on is in the branch "release-0.7.0-rc1"
(commi
ed all binary packages (except the YARN one) on Windows. Started local
> mode and executed a random example.
> All worked fine.
>
> 2014-10-11 19:21 GMT+02:00 Robert Metzger :
>
> > Yes, on different computers.
> >
> > Another issue that came in my mind regarding the
Robert Metzger created FLINK-1157:
-
Summary: Document TaskManager Slots
Key: FLINK-1157
URL: https://issues.apache.org/jira/browse/FLINK-1157
Project: Flink
Issue Type: Improvement
Robert Metzger created FLINK-1155:
-
Summary: Add Squirrel Logo to Flink website
Key: FLINK-1155
URL: https://issues.apache.org/jira/browse/FLINK-1155
Project: Flink
Issue Type: Task
gt; Seems that some timers do not get canceled...
>
> On Sat, Oct 11, 2014 at 12:07 PM, Robert Metzger
> wrote:
>
> > WordCount [Java Application]
> > org.apache.flink.examples.java.wordcount.WordCount at localhost:38214
> > (Suspended)
> > Daemon System Thread [Sig
TimerThread.run() line: 505
Thread [DestroyJavaVM] (Suspended)
/opt/java/bin/java (Oct 11, 2014, 12:02:52 PM)
On Fri, Oct 10, 2014 at 7:31 PM, Stephan Ewen wrote:
> Which threads are active when the shutdown hangs?
> Am 10.10.2014 19:25 schrieb "Robert Metzger" :
>
> > Ah damn
jira/browse/FLINK-1152 )
> >
> > Has the Scala API been updated to reflect the newly integrated POJO
> support?
> >
> > Could somebody who's using Windows try it there?
> >
> > We should check if the web-client shows the optimizer plans of the
> examples
> >
ould check if the web-client shows the optimizer plans of the examples
properly and if they all run without data.
On Wed, Oct 8, 2014 at 8:50 PM, Robert Metzger wrote:
> No, its just a dry run and I want to do some tests on our cluster first.
>
> Last time, I created a lot of release
Hi,
Thank you for reporting the issue.
We have the pull request to fix the issue already merged:
https://github.com/apache/incubator-flink/pull/136/files but we didn't
upload the latest 0.7 documentation to the website.
I'll upload the latest 0.7 documentation to the website ...
Best,
Robert
On
1 - 100 of 464 matches
Mail list logo