Re: Left join with unbalanced dataset

2016-02-03 Thread Fabian Hueske
ue...@gmail.com] > *Envoyé :* mercredi 3 février 2016 10:51 > > *À :* user@flink.apache.org > *Objet :* Re: Left join with unbalanced dataset > > > > Hi Arnauld, > > in a previous mail you said: > "Note that I did not rebuild & reinstall flink, I ju

Re: Left join with unbalanced dataset

2016-02-03 Thread Fabian Hueske
TupleSerializer.java:30) >> >> at >> org.apache.flink.runtime.io.disk.ChannelReaderInputViewIterator.next(ChannelReaderInputViewIterator.java:86) >> >> at >> org.apache.flink.runtime.operators.sort.MergeIterator$HeadStream.nextHead(MergeIterator.java:151)

Re: Left join with unbalanced dataset

2016-02-03 Thread Stephan Ewen
t; at > org.apache.flink.runtime.memory.AbstractPagedInputView.advance(AbstractPagedInputView.java:159) > > at > org.apache.flink.runtime.memory.AbstractPagedInputView.read(AbstractPagedInputView.java:213) > > at > org.apache.flink.api.java.typeutils.runtime.D

Re: Left join with unbalanced dataset

2016-02-02 Thread Stephan Ewen
in memory for all nodes and use a hash > map upon reduction to get all A.join(B) lines. B is not that small, but I > think it will still save RAM. > > > > Best regards, > > Arnaud > > > > -Message d'origine- > > De : Ufuk Celebi [mailto:u...@apache.o

Re: Left join with unbalanced dataset

2016-02-02 Thread Gábor Gévay
eduction to get all A.join(B) lines. B is not that small, but I think > it will still save RAM. > > Best regards, > Arnaud > > -Message d'origine- > De : Ufuk Celebi [mailto:u...@apache.org] > Envoyé : mardi 2 février 2016 15:27 > À : user@flink.apache.org > Objet

RE: Left join with unbalanced dataset

2016-02-02 Thread LINZ, Arnaud
Message d'origine- De : Ufuk Celebi [mailto:u...@apache.org] Envoyé : mardi 2 février 2016 15:27 À : user@flink.apache.org Objet : Re: Left join with unbalanced dataset > On 02 Feb 2016, at 15:15, LINZ, Arnaud wrote: > > Hi, > > Running again with more RAM made the treate

Re: Left join with unbalanced dataset

2016-02-02 Thread Ufuk Celebi
> On 02 Feb 2016, at 15:15, LINZ, Arnaud wrote: > > Hi, > > Running again with more RAM made the treatement go further, but Yarn still > killed one container for memory consumption. I will experiment various memory > parameters. OK, the killing of the container probably triggered the Remote

Re: Left join with unbalanced dataset

2016-02-02 Thread Robert Metzger
Hi Arnaud, you can retrieve the logs of a yarn application by calling "yarn logs -applicationId ". Its going to output you the logs of all Taskmanagers and the job manager in one stream. I would pipe the output into a file and then search for the position where the log for the failing taskmanager

RE: Left join with unbalanced dataset

2016-02-02 Thread LINZ, Arnaud
Hi, Running again with more RAM made the treatement go further, but Yarn still killed one container for memory consumption. I will experiment various memory parameters. How do I retrieve the log of a specific task manager post-mortem? I don't use a permanent Flink/Yarn container (it's killed u

Re: Left join with unbalanced dataset

2016-02-02 Thread Ufuk Celebi
> On 02 Feb 2016, at 14:31, LINZ, Arnaud wrote: > > Hi, > > Unfortunalety, it still fails, but with a different error (see below). > Note that I did not rebuild & reinstall flink, I just used a 0.10-SNAPSHOT > compiled jar submitted as a batch job using the "0.10.0" flink installation. This m

RE: Left join with unbalanced dataset

2016-02-02 Thread LINZ, Arnaud
nel.nio.NioEventLoop.run(NioEventLoop.java:357) -Message d'origine- De : Ufuk Celebi [mailto:u...@apache.org] Envoyé : mardi 2 février 2016 13:52 À : user@flink.apache.org Objet : Re: Left join with unbalanced dataset > On 02 Feb 2016, at 13:28, LINZ, Arnaud wrote: > > Thanks, > I’

Re: Left join with unbalanced dataset

2016-02-02 Thread Ufuk Celebi
> On 02 Feb 2016, at 13:28, LINZ, Arnaud wrote: > > Thanks, > I’m using the official 0.10 release. I will try to use the 0.10 snapshot. > > FYI, setting the heap cut-off ratio to 0.5 lead to the following error : That’s the error Stephan was referring to. Does the snapshot version fix it for

RE: Left join with unbalanced dataset

2016-02-02 Thread LINZ, Arnaud
ecoder$1.cumulate(ByteToMessageDecoder.java:92) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:228) De : ewenstep...@gmail.com [mailto:ewenstep...@gmail.com] De la part de Stephan Ewen Envoyé : mardi 2 février 2016 11:30 À : user@flink.apache.org Ob

Re: Left join with unbalanced dataset

2016-02-02 Thread Stephan Ewen
n/log/application_1453202008841_2794/container_e11_1453202008841_2794_01_25/taskmanager.log > -Dlogback.configurationFile=file:logback.xml > -Dlog4j.configuration=file:log4j.properties > org.apache.flink.yarn.YarnTaskManagerRunner --configDir . --streamingMode > batch

RE: Left join with unbalanced dataset

2016-02-02 Thread LINZ, Arnaud
:58:54,471 INFO org.apache.flink.yarn.YarnJobManager De : LINZ, Arnaud Envoyé : lundi 1 février 2016 09:40 À : user@flink.apache.org Objet : RE: Left join with unbalanced dataset Hi, Thanks, I can’t believe I missed the outer join operators… Will try them and will keep you informed. I use the

RE: Left join with unbalanced dataset

2016-02-01 Thread LINZ, Arnaud
doubling the amount of memory did not solve the problem. Arnaud De : ewenstep...@gmail.com [mailto:ewenstep...@gmail.com] De la part de Stephan Ewen Envoyé : dimanche 31 janvier 2016 20:57 À : user@flink.apache.org Objet : Re: Left join with unbalanced dataset Hi! YARN killing the application

Re: Left join with unbalanced dataset

2016-01-31 Thread Stephan Ewen
Hi! YARN killing the application seems strange. The memory use that YARN sees should not change even when one node gets a lot or data. Can you share what version of Flink (plus commit hash) you are using and whether you use off-heap memory or not? Thanks, Stephan On Sun, Jan 31, 2016 at 10:47

Re: Left join with unbalanced dataset

2016-01-31 Thread Till Rohrmann
Hi Arnaud, the unmatched elements of A will only end up on the same worker node if they all share the same key. Otherwise, they will be evenly spread out across your cluster. However, I would also recommend you to use Flink's leftOuterJoin. Cheers, Till On Sun, Jan 31, 2016 at 5:27 AM, Chiwan Pa

Re: Left join with unbalanced dataset

2016-01-30 Thread Chiwan Park
Hi Arnaud, To join two datasets, the community recommends using join operation rather than cogroup operation. For left join, you can use leftOuterJoin method. Flink’s optimizer decides distributed join execution strategy using some statistics of the datasets such as size of the dataset. Additio

Left join with unbalanced dataset

2016-01-30 Thread LINZ, Arnaud
Hello, I have a very big dataset A to left join with a dataset B that is half its size. That is to say, half of A records will be matched with one record of B, and the other half with null values. I used a CoGroup for that, but my batch fails because yarn kills the container due to memory prob