Re: Drafting a roadmap for Flink

2015-02-09 Thread Dmitriy Lyubimov
fwiw re: shell, this is just scala being incredibly useful. If anything, spark is following scala. So is for example BIDMat/BIDMach (and, sigh* mahout). I don't think differentiation means throwing away common baseline tools, there's gotta be more than that. (I'm of course advocating using shell in

Re: Events not received

2015-02-09 Thread Hermann Gábor
Hey Ufuk, Thanks for the fast reply and fix! It's working now :) Unfortunately the "other side" of the AbstractInvokable lifecycle is causing a bit of a trouble, as the AbstractInvokable does not wait for possible events before finishing, and it throws an IllegalStateException: Tried to send task

Re: Memory segment error when migrating functional code from Flink 0.9 to 0.8

2015-02-09 Thread Andra Lungu
Hi Stephan, I tried the solution with DeltaIteration#setSolutionSetUnManaged(), unfortunatelly the error is still there... Even when I try to run it with just one iteration... Also, I am not sure that the job can be broken into subparts in this particular case. Any other suggestions would be appr

[jira] [Created] (FLINK-1504) Add support for accessing secured HDFS clusters in standalone mode

2015-02-09 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-1504: - Summary: Add support for accessing secured HDFS clusters in standalone mode Key: FLINK-1504 URL: https://issues.apache.org/jira/browse/FLINK-1504 Project: Flink

[jira] [Created] (FLINK-1503) GSoC project: Batch and Streaming integration through new operators and unified API

2015-02-09 Thread Gyula Fora (JIRA)
Gyula Fora created FLINK-1503: - Summary: GSoC project: Batch and Streaming integration through new operators and unified API Key: FLINK-1503 URL: https://issues.apache.org/jira/browse/FLINK-1503 Project:

[jira] [Created] (FLINK-1502) Expose metrics to graphite, ganglia and JMX.

2015-02-09 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-1502: - Summary: Expose metrics to graphite, ganglia and JMX. Key: FLINK-1502 URL: https://issues.apache.org/jira/browse/FLINK-1502 Project: Flink Issue Type: Sub-

[jira] [Created] (FLINK-1501) Integrate metrics library and report basic metrics to JobManager web interface

2015-02-09 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-1501: - Summary: Integrate metrics library and report basic metrics to JobManager web interface Key: FLINK-1501 URL: https://issues.apache.org/jira/browse/FLINK-1501 Projec

[jira] [Created] (FLINK-1500) exampleScalaPrograms.EnumTriangleOptITCase does not finish on Travis

2015-02-09 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-1500: Summary: exampleScalaPrograms.EnumTriangleOptITCase does not finish on Travis Key: FLINK-1500 URL: https://issues.apache.org/jira/browse/FLINK-1500 Project: Flink

Re: Events not received

2015-02-09 Thread Ufuk Celebi
Hey Gabor, can you try this branch: https://github.com/uce/incubator-flink/tree/flink-1496-lost Is this working for you? This is a quick fix. It keeps track of events, which could not be delivered to uninitialized channels and delivers them after they are initialized. (The BufferReader, which

[jira] [Created] (FLINK-1499) Make TaskManager to disconnect from TaskManager in case of a restart

2015-02-09 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-1499: Summary: Make TaskManager to disconnect from TaskManager in case of a restart Key: FLINK-1499 URL: https://issues.apache.org/jira/browse/FLINK-1499 Project: Flink

Re: [DISCUSS] Distributed TPC-H DataGenerator for flink-contrib

2015-02-09 Thread Robert Metzger
Hi Fabian, The legal TPC-H documents are hard to parse ;) I think we don't have any issues with the tools or documents they provide, because we are not modifying their generator tools. Also, we are not publishing any performance numbers. There is one remaining concern and thats the TPC trademark.

Re: Eclipse JDT, Java 8, lambdas

2015-02-09 Thread Stephan Ewen
Is it possible to use this compiler for the java 8 quickstart archetypes? On Mon, Feb 9, 2015 at 4:14 PM, Timo Walther wrote: > The fix is included in 4.4.2. However, it seems that even if the compiler > option > "org.eclipse.jdt.core.compiler.codegen.lambda.genericSignature=generate" > is set

[jira] [Created] (FLINK-1498) Spurious failures on Travis for I/O heavy tasks

2015-02-09 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-1498: --- Summary: Spurious failures on Travis for I/O heavy tasks Key: FLINK-1498 URL: https://issues.apache.org/jira/browse/FLINK-1498 Project: Flink Issue Type: Bug

Re: [DISCUSS] Distributed TPC-H DataGenerator for flink-contrib

2015-02-09 Thread Fabian Hueske
I think this is a great tool and would be a nice contribution. I am however not sure about the licensing here. Even though the used library appears to be AL2 licensed, I do not know if there are any restrictions from the Transaction Processing Performance Council (TPC, tpc.org). TPC-H is a benchma

Re: Eclipse JDT, Java 8, lambdas

2015-02-09 Thread Timo Walther
The fix is included in 4.4.2. However, it seems that even if the compiler option "org.eclipse.jdt.core.compiler.codegen.lambda.genericSignature=generate" is set in the project's "org.eclipse.jdt.core.prefs" file, it has no effect. The command line approach works: java -jar ./plugins/org.eclip

[DISCUSS] Distributed TPC-H DataGenerator for flink-contrib

2015-02-09 Thread Robert Metzger
Hi, we recently added the "flink-contrib" module for user contributed tools etc. On one of the last weekends, I've created a distributed tpch generator, based on this libary: https://github.com/airlift/tpch (which is from a PrestoDB developer and available on Maven central). You can find my code

[jira] [Created] (FLINK-1497) No documentation on MiniCluster

2015-02-09 Thread Sergey Dudoladov (JIRA)
Sergey Dudoladov created FLINK-1497: --- Summary: No documentation on MiniCluster Key: FLINK-1497 URL: https://issues.apache.org/jira/browse/FLINK-1497 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-1496) Events at unitialized input channels are lost

2015-02-09 Thread Ufuk Celebi (JIRA)
Ufuk Celebi created FLINK-1496: -- Summary: Events at unitialized input channels are lost Key: FLINK-1496 URL: https://issues.apache.org/jira/browse/FLINK-1496 Project: Flink Issue Type: Bug

Re: Events not received

2015-02-09 Thread Ufuk Celebi
On 09 Feb 2015, at 15:06, Hermann Gábor wrote: > Hi all, > > We've been trying to use events for two-way communication, but it seems > like some of the backward events (at a BufferReader) are not sent at all if > the connection is not initialized yet (i.e. the given input channel is > UNKNOWN).

Events not received

2015-02-09 Thread Hermann Gábor
Hi all, We've been trying to use events for two-way communication, but it seems like some of the backward events (at a BufferReader) are not sent at all if the connection is not initialized yet (i.e. the given input channel is UNKNOWN). Is there a way to make sure the events get to the destinatio

Re: Planning Release 0.8.1

2015-02-09 Thread Felix Neutatz
Yes, that would be great :) 2015-02-09 14:37 GMT+01:00 Robert Metzger : > Yes. > > Do you mean this? https://github.com/apache/flink/pull/287 > > I guess you would like to have this for the Parquet blog post? If so, we > can merge it in my opinion. > > > On Mon, Feb 9, 2015 at 2:30 PM, Felix Neut

[jira] [Created] (FLINK-1495) Make Akka timeout configurable in YARN client.

2015-02-09 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-1495: - Summary: Make Akka timeout configurable in YARN client. Key: FLINK-1495 URL: https://issues.apache.org/jira/browse/FLINK-1495 Project: Flink Issue Type: Im

Re: Planning Release 0.8.1

2015-02-09 Thread Robert Metzger
Yes. Do you mean this? https://github.com/apache/flink/pull/287 I guess you would like to have this for the Parquet blog post? If so, we can merge it in my opinion. On Mon, Feb 9, 2015 at 2:30 PM, Felix Neutatz wrote: > @Robert, does this also include the Kryo Protobuff support? If yes we cou

Re: Planning Release 0.8.1

2015-02-09 Thread Felix Neutatz
@Robert, does this also include the Kryo Protobuff support? If yes we could also ship my changes of the Hadoopinputformat :) Am 09.02.2015 14:27 schrieb "Robert Metzger" : > Cool. > > I'm currently also testing my last change (kryo serializers). I think I'll > start creating the release candidate

Re: Planning Release 0.8.1

2015-02-09 Thread Robert Metzger
Cool. I'm currently also testing my last change (kryo serializers). I think I'll start creating the release candidate in one hour. On Mon, Feb 9, 2015 at 2:24 PM, Márton Balassi wrote: > Went through the streaming commits with Gyula and assembled the ones > missing. Pushing it as soon as travis

Re: Planning Release 0.8.1

2015-02-09 Thread Márton Balassi
Went through the streaming commits with Gyula and assembled the ones missing. Pushing it as soon as travis passes. On Fri, Feb 6, 2015 at 2:26 PM, Robert Metzger wrote: > It seems that quite a few important fixes still need some work until they > are ready. > I'll extend the "deadline" to Monday

[jira] [Created] (FLINK-1494) Build fails on BlobCacheTest

2015-02-09 Thread Fabian Hueske (JIRA)
Fabian Hueske created FLINK-1494: Summary: Build fails on BlobCacheTest Key: FLINK-1494 URL: https://issues.apache.org/jira/browse/FLINK-1494 Project: Flink Issue Type: Bug Componen

Re: Eclipse JDT, Java 8, lambdas

2015-02-09 Thread Timo Walther
Hey, it seems that 4.4.2 also includes the fix (https://projects.eclipse.org/projects/eclipse/releases/4.4.2/bugs) and will be released end of february. I will try Eclipse Luna SR2 RC2 today and check if it is working. Regards, Timo On 09.02.2015 10:05, Nam-Luc Tran wrote: I did try the 4

Re: Memory segment error when migrating functional code from Flink 0.9 to 0.8

2015-02-09 Thread Stephan Ewen
This is actually a problem of the number of memory segments available to the hash table for the solution set. For complex pipelines, memory currently gets too fragmented. There are two workarounds, until we do the dynamic memory management, or break it into shorter pipelines: Break the job up int

Re: Memory segment error when migrating functional code from Flink 0.9 to 0.8

2015-02-09 Thread Till Rohrmann
Hi Andra, have you tried increasing the number of network buffers in your cluster? You can control by the configuration value: taskmanager.network.numberOfBuffers: #numberBuffers Greets, Till On Mon, Feb 9, 2015 at 9:56 AM, Andra Lungu wrote: > Hello everyone, > > I am implementing a graph a

Re: Eclipse JDT, Java 8, lambdas

2015-02-09 Thread Nam-Luc Tran
I did try the 4.5 M4 release and it did not go straightforward. -- View this message in context: http://apache-flink-incubator-mailing-list-archive.1008284.n3.nabble.com/Eclipse-JDT-Java-8-lambdas-tp3664p3688.html Sent from the Apache Flink (Incubator) Mailing List archive. mailing list archiv

Memory segment error when migrating functional code from Flink 0.9 to 0.8

2015-02-09 Thread Andra Lungu
Hello everyone, I am implementing a graph algorithm as part of a course and I will also add it to the Flink- Gelly examples. My problem is that I started developing it in the Gelly repository, which runs on flink 0.9. It works like a charm there, but in order to test in on a cluster to see its rea