Re: Error due to Hadoop version mismatch

2016-01-04 Thread Robert Metzger
Your Flink installation has Hadoop 2.6.0 included, on the other machine,
there is a Hadoop version installed, which is most likely a 1.x or even a
0.x version.
Are you sure that the host "ubuntu-171" has the ip 10.13.182.171 and that
the hadoop installation in the "/home/ubuntu/Documents/hadoop-2.6.0/"
directory is listening on port 9000 ?

On Mon, Jan 4, 2016 at 3:03 PM, Kashmar, Ali  wrote:

> Hi Max,
>
> Both commands return nothing. Those variables aren’t set.
>
> The only software I installed on these machines is Flink and Java.
>
> -Ali
>
> On 2015-12-28, 6:42 AM, "Maximilian Michels"  wrote:
>
> >Hi Ali,
> >
> >The warning about the native Hadoop libraries is nothing to worry
> >about. The native modules are platform-optimized modules which may be
> >used to improve performance. They are not necessary for Hadoop to
> >function correctly.
> >
> >The exception message implies that you are using is a very old version
> >of Hadoop. Do you have other Hadoop versions installed on the same
> >machine? We have had people using Flink 0.10.0 with Hadoop 2.6.0
> >without any problems.
> >
> >On the cluster machines, what is the output of these commands?
> >
> >echo $HADOOP_CLASSPATH
> >echo $HADOOP_CONF_DIR
> >
> >
> >Thanks,
> >Max
> >
> >On Wed, Dec 23, 2015 at 3:53 PM, Kashmar, Ali 
> wrote:
> >> Hi Max,
> >>
> >> I have the same output for the Task Manager:
> >>
> >> 11:25:04,274 INFO  org.apache.flink.runtime.taskmanager.TaskManager
> >>   -  Hadoop version: 2.6.0
> >>
> >> I do get this line at the beginning of both job and task manager log
> >>files:
> >>
> >> 11:25:04,100 WARN  org.apache.hadoop.util.NativeCodeLoader
> >>   - Unable to load native-hadoop library for your platform... using
> >> builtin-java classes where applicable
> >>
> >> Do you think it has anything to do with it?
> >>
> >> Thanks,
> >> Ali
> >>
> >> On 2015-12-23, 7:30 AM, "Maximilian Michels"  wrote:
> >>
> >>>Hi Ali,
> >>>
> >>>Could you please also post the Hadoop version output of the task
> >>>manager log files? It looks like the task managers are running a
> >>>different Hadoop version.
> >>>
> >>>Thanks,
> >>>Max
> >>>
> >>>On Tue, Dec 22, 2015 at 4:28 PM, Kashmar, Ali 
> >>>wrote:
>  Hi Robert,
> 
>  I found the version in the job manager log file:
> 
>  17:33:49,636 INFO  org.apache.flink.runtime.jobmanager.JobManager
>    -  Hadoop version: 2.6.0
> 
>  But the Hadoop installation I have is saying this:
> 
>  ubuntu@ubuntu-171:~/Documents/hadoop-2.6.0$ bin/hadoop version
>  Hadoop 2.6.0
>  Subversion https://git-wip-us.apache.org/repos/asf/hadoop.git -r
>  e3496499ecb8d220fba99dc5ed4c99c8f9e33bb1
>  Compiled by jenkins on 2014-11-13T21:10Z
>  Compiled with protoc 2.5.0
>  From source with checksum 18e43357c8f927c0695f1e9522859d6a
>  This command was run using
> 
> /home/ubuntu/Documents/hadoop-2.6.0/share/hadoop/common/hadoop-common-2
> .6
> .0
>  .jar
> 
> 
>  So one of them is lying to me? :)
> 
>  Ali
> 
>  On 2015-12-22, 10:16 AM, "Robert Metzger" 
> wrote:
> 
> >Hi Ali,
> >
> >the TaskManagers and the JobManager is logging the Hadoop version on
> >startup.
> >
> >On Tue, Dec 22, 2015 at 4:10 PM, Kashmar, Ali 
> >wrote:
> >
> >> Hello,
> >>
> >> I¹m trying to use HDFS as store for Flink checkpoints so I
> >>downloaded
> >>the
> >> Hadoop 2.6.0/Scala 2.10 version of Flink and installed it. I also
> >> downloaded Hadoop 2.6.0 separately from the Hadoop website and set
> >>up
> >>HDFS
> >> on a separate machine. When I start Flink I get the following error:
> >>
> >> 17:34:13,047 INFO  org.apache.flink.runtime.jobmanager.JobManager
> >>   - Status of job 9ba32a08bc0ec02810bf5d2710842f72 (Protocol
> >>Event
> >> Processing) changed to FAILED.
> >> java.lang.Exception: Call to registerInputOutput() of invokable
> >>failed
> >> at
> >>org.apache.flink.runtime.taskmanager.Task.run(Task.java:529)
> >> at java.lang.Thread.run(Thread.java:745)
> >> Caused by: java.io.IOException: The given file URI (hdfs://
> >> 10.13.182.171:9000/user/flink/checkpoints) points to the HDFS
> >>NameNode
> >>at
> >> 10.13.182.171:9000, but the File System could not be initialized
> >>with
> >> that address: Server IPC version 9 cannot communicate with client
> >>version 4
> >> at
> >>
> >>org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.initialize(HadoopFi
> >>le
> >>Sy
> >>stem.java:337)
> >> at
> >>org.apache.flink.core.fs.FileSystem.get(FileSystem.java:253)
> >> at
> >>
> >>org.apache.flink.runtime.state.filesystem.FsStateBackend.(FsSta
> >>te
> 

Re: Error due to Hadoop version mismatch

2016-01-04 Thread Robert Metzger
Okay, then, my previous statement is false. From the stack trace, it seems
that Flink is using an older Hadoop version. The DFSClient and RPC classes
look different in Hadoop 2.6.0.
Max checked already for some environment variables. Is $CLASSPATH set? Did
you install Hadoop only by downloading the binaries or did you use
something like the ubuntu package manager? (those packages sometimes place
jar files in some lib/ folders)
Did you put something additionally into the lib/ folder of Flink?

I think when Flink is starting up, its also logging the classpath of the
JVM. Can you post it here? (Maybe you need to start with DEBUG log level)


On Mon, Jan 4, 2016 at 8:46 PM, Kashmar, Ali  wrote:

> Hi Robert,
>
> On the ubuntu host, the port 9000 is up:
>
> ubuntu@ubuntu-171:~$ netstat -tulpn
> 
> tcp0  0 10.13.182.171:9000  0.0.0.0:*   LISTEN
>  8849/java
> 
>
> And the process using this port is:
>
> ubuntu@ubuntu-171:~$ ps -ef | grep 8849
> ubuntu8849 1  0  2015 ?00:25:00
> /usr/lib/jvm/java-8-oracle/bin/java -Dproc_namenode -Xmx1000m
> -Djava.net.preferIPv4Stack=true
> -Dhadoop.log.dir=/home/ubuntu/Documents/hadoop-2.6.0/logs
> -Dhadoop.log.file=hadoop.log
> -Dhadoop.home.dir=/home/ubuntu/Documents/hadoop-2.6.0
> -Dhadoop.id.str=ubuntu -Dhadoop.root.logger=INFO,console
> -Djava.library.path=/home/ubuntu/Documents/hadoop-2.6.0/lib/native
> -Dhadoop.policy.file=hadoop-policy.xml -Djava.net.preferIPv4Stack=true
> -Djava.net.preferIPv4Stack=true -Djava.net.preferIPv4Stack=true
> -Dhadoop.log.dir=/home/ubuntu/Documents/hadoop-2.6.0/logs
> -Dhadoop.log.file=hadoop-ubuntu-namenode-ubuntu-171.log
> -Dhadoop.home.dir=/home/ubuntu/Documents/hadoop-2.6.0
> -Dhadoop.id.str=ubuntu -Dhadoop.root.logger=INFO,RFA
> -Djava.library.path=/home/ubuntu/Documents/hadoop-2.6.0/lib/native
> -Dhadoop.policy.file=hadoop-policy.xml -Djava.net.preferIPv4Stack=true
> -Dhadoop.security.logger=INFO,RFAS -Dhdfs.audit.logger=INFO,NullAppender
> -Dhadoop.security.logger=INFO,RFAS -Dhdfs.audit.logger=INFO,NullAppender
> -Dhadoop.security.logger=INFO,RFAS -Dhdfs.audit.logger=INFO,NullAppender
> -Dhadoop.security.logger=INFO,RFAS
> org.apache.hadoop.hdfs.server.namenode.NameNode
>
> And the hadoop version is:
>
> ubuntu@ubuntu-171:~$ Documents/hadoop-2.6.0/bin/hadoop version
> Hadoop 2.6.0
> Subversion https://git-wip-us.apache.org/repos/asf/hadoop.git -r
> e3496499ecb8d220fba99dc5ed4c99c8f9e33bb1
> Compiled by jenkins on 2014-11-13T21:10Z
> Compiled with protoc 2.5.0
> From source with checksum 18e43357c8f927c0695f1e9522859d6a
> This command was run using
> /home/ubuntu/Documents/hadoop-2.6.0/share/hadoop/common/hadoop-common-2.6.0
> .jar
>
>
>
> -Ali
>
>
>
> On 2016-01-04, 2:20 PM, "Robert Metzger"  wrote:
>
> >Your Flink installation has Hadoop 2.6.0 included, on the other machine,
> >there is a Hadoop version installed, which is most likely a 1.x or even a
> >0.x version.
> >Are you sure that the host "ubuntu-171" has the ip 10.13.182.171 and that
> >the hadoop installation in the "/home/ubuntu/Documents/hadoop-2.6.0/"
> >directory is listening on port 9000 ?
> >
> >On Mon, Jan 4, 2016 at 3:03 PM, Kashmar, Ali  wrote:
> >
> >> Hi Max,
> >>
> >> Both commands return nothing. Those variables aren’t set.
> >>
> >> The only software I installed on these machines is Flink and Java.
> >>
> >> -Ali
> >>
> >> On 2015-12-28, 6:42 AM, "Maximilian Michels"  wrote:
> >>
> >> >Hi Ali,
> >> >
> >> >The warning about the native Hadoop libraries is nothing to worry
> >> >about. The native modules are platform-optimized modules which may be
> >> >used to improve performance. They are not necessary for Hadoop to
> >> >function correctly.
> >> >
> >> >The exception message implies that you are using is a very old version
> >> >of Hadoop. Do you have other Hadoop versions installed on the same
> >> >machine? We have had people using Flink 0.10.0 with Hadoop 2.6.0
> >> >without any problems.
> >> >
> >> >On the cluster machines, what is the output of these commands?
> >> >
> >> >echo $HADOOP_CLASSPATH
> >> >echo $HADOOP_CONF_DIR
> >> >
> >> >
> >> >Thanks,
> >> >Max
> >> >
> >> >On Wed, Dec 23, 2015 at 3:53 PM, Kashmar, Ali 
> >> wrote:
> >> >> Hi Max,
> >> >>
> >> >> I have the same output for the Task Manager:
> >> >>
> >> >> 11:25:04,274 INFO  org.apache.flink.runtime.taskmanager.TaskManager
> >> >>   -  Hadoop version: 2.6.0
> >> >>
> >> >> I do get this line at the beginning of both job and task manager log
> >> >>files:
> >> >>
> >> >> 11:25:04,100 WARN  org.apache.hadoop.util.NativeCodeLoader
> >> >>   - Unable to load native-hadoop library for your platform...
> >>using
> >> >> builtin-java classes where applicable
> >> >>
> >> >> Do you think it has anything to do with it?
> >> >>
> >> >> Thanks,
> >> >> Ali
> >> >>
> >> >> On 2015-12-23, 7:30 AM, "Maximilian Michels"  

Re: [gelly] Spargel model rework

2016-01-04 Thread Vasiliki Kalavri
Hello squirrels and happy new year!

I'm reviving this thread to share some results and discuss next steps.

Using the Either type I was able to get rid of redundant messages and
vertex state. During the past few weeks, I have been running experiments,
which show that the performance of this "Pregel" model has improved a lot :)
In [1], you can see the speedup of GSA and Pregel over Spargel, for SSSP
and Connected Components (CC), for the Livejournal (68m edges), Orkut (117m
edges) and Wikipedia (340m edges) datasets.

Regarding next steps, if no objections, I will open a Jira for adding a
Pregel iteration abstraction to Gelly. The Gelly guide has to be updated to
reflect the spectrum of iteration abstractions that we have discussed in
this thread, i.e. Pregel -> Spargel (Scatter-Gather) -> GSA.

I think it might also be a good idea to do some renaming. Currently, we
call the Spargel iteration "vertex-centric", which fits better to the
Pregel abstraction. I propose we rename the spargel iteration into
"scatter-gather" or "signal-collect" (where it was first introduced [2]).
Any other ideas?

Thanks,
-Vasia.

[1]:
https://drive.google.com/file/d/0BzQJrI2eGlyYRTRjMkp1d3R6eVE/view?usp=sharing
[2]: http://link.springer.com/chapter/10.1007/978-3-642-17746-0_48

On 11 November 2015 at 11:05, Stephan Ewen  wrote:

> See: https://issues.apache.org/jira/browse/FLINK-3002
>
> On Wed, Nov 11, 2015 at 10:54 AM, Stephan Ewen  wrote:
>
> > "Either" an "Optional" types are quite useful.
> >
> > Let's add them to the core Java API.
> >
> > On Wed, Nov 11, 2015 at 10:00 AM, Vasiliki Kalavri <
> > vasilikikala...@gmail.com> wrote:
> >
> >> Thanks Fabian! I'll try that :)
> >>
> >> On 10 November 2015 at 22:31, Fabian Hueske  wrote:
> >>
> >> > You could implement a Java Either type (similar to Scala's Either)
> that
> >> > either has a Message or the VertexState and a corresponding
> >> TypeInformation
> >> > and TypeSerializer that serializes a byte flag to indicate which both
> >> types
> >> > is used.
> >> > It might actually make sense, to add a generic Either type to the Java
> >> API
> >> > in general (similar to the Java Tuples with resemble the Scala
> Tuples).
> >> >
> >> > Cheers, Fabian
> >> >
> >> > 2015-11-10 22:16 GMT+01:00 Vasiliki Kalavri <
> vasilikikala...@gmail.com
> >> >:
> >> >
> >> > > Hi,
> >> > >
> >> > > after running a few experiments, I can confirm that putting the
> >> combiner
> >> > > after the flatMap is indeed more efficient.
> >> > >
> >> > > I ran SSSP and Connected Components with Spargel, GSA, and the
> Pregel
> >> > model
> >> > > and the results are the following:
> >> > >
> >> > > - for SSSP, Spargel is always the slowest, GSA is a ~1.2x faster and
> >> > Pregel
> >> > > is ~1.1x faster without combiner, ~1.3x faster with combiner.
> >> > > - for Connected Components, Spargel and GSA perform similarly, while
> >> > Pregel
> >> > > is 1.4-1.6x slower.
> >> > >
> >> > > To start with, this is much better than I expected :)
> >> > > However, there is a main shortcoming in my current implementation
> that
> >> > > negatively impacts performance:
> >> > > Since the compute function coGroup needs to output both new vertex
> >> values
> >> > > and new messages, I emit a wrapping tuple that contains both vertex
> >> state
> >> > > and messages and then filter them out based on a boolean field. The
> >> > problem
> >> > > is that since I cannot emit null fields, I emit a dummy message for
> >> each
> >> > > new vertex state and a dummy vertex state for each new message. That
> >> > > essentially means that the intermediate messages result is double in
> >> > size,
> >> > > if say the vertex values are of the same type as the messages (can
> be
> >> > worse
> >> > > if the vertex values are more complex).
> >> > > So my question is, is there a way to avoid this redundancy, by
> either
> >> > > emitting null fields or by creating an operator that could emit 2
> >> > different
> >> > > types of tuples?
> >> > >
> >> > > Thanks!
> >> > > -Vasia.
> >> > >
> >> > > On 9 November 2015 at 15:20, Fabian Hueske 
> wrote:
> >> > >
> >> > > > Hi Vasia,
> >> > > >
> >> > > > sorry for the late reply.
> >> > > > I don't think there is a big difference. In both cases, the
> >> > partitioning
> >> > > > and sorting happens at the end of the iteration.
> >> > > > If the groupReduce is applied before the workset is returned, the
> >> > sorting
> >> > > > happens on the filtered result (after the flatMap) which might be
> a
> >> > > little
> >> > > > bit more efficient (depending on the ratio of messages and
> solution
> >> set
> >> > > > updates). Also it does not require that the initial workset is
> >> sorted
> >> > for
> >> > > > the first groupReduce.
> >> > > >
> >> > > > I would put it at the end.
> >> > > >
> >> > > > Cheers, Fabian
> >> > > >
> >> > > > 2015-11-05 17:19 GMT+01:00 Vasiliki Kalavri <
> >> vasilikikala...@gmail.com
> 

Re: Error due to Hadoop version mismatch

2016-01-04 Thread Kashmar, Ali
Hi Robert,

On the ubuntu host, the port 9000 is up:

ubuntu@ubuntu-171:~$ netstat -tulpn

tcp0  0 10.13.182.171:9000  0.0.0.0:*   LISTEN
 8849/java


And the process using this port is:

ubuntu@ubuntu-171:~$ ps -ef | grep 8849
ubuntu8849 1  0  2015 ?00:25:00
/usr/lib/jvm/java-8-oracle/bin/java -Dproc_namenode -Xmx1000m
-Djava.net.preferIPv4Stack=true
-Dhadoop.log.dir=/home/ubuntu/Documents/hadoop-2.6.0/logs
-Dhadoop.log.file=hadoop.log
-Dhadoop.home.dir=/home/ubuntu/Documents/hadoop-2.6.0
-Dhadoop.id.str=ubuntu -Dhadoop.root.logger=INFO,console
-Djava.library.path=/home/ubuntu/Documents/hadoop-2.6.0/lib/native
-Dhadoop.policy.file=hadoop-policy.xml -Djava.net.preferIPv4Stack=true
-Djava.net.preferIPv4Stack=true -Djava.net.preferIPv4Stack=true
-Dhadoop.log.dir=/home/ubuntu/Documents/hadoop-2.6.0/logs
-Dhadoop.log.file=hadoop-ubuntu-namenode-ubuntu-171.log
-Dhadoop.home.dir=/home/ubuntu/Documents/hadoop-2.6.0
-Dhadoop.id.str=ubuntu -Dhadoop.root.logger=INFO,RFA
-Djava.library.path=/home/ubuntu/Documents/hadoop-2.6.0/lib/native
-Dhadoop.policy.file=hadoop-policy.xml -Djava.net.preferIPv4Stack=true
-Dhadoop.security.logger=INFO,RFAS -Dhdfs.audit.logger=INFO,NullAppender
-Dhadoop.security.logger=INFO,RFAS -Dhdfs.audit.logger=INFO,NullAppender
-Dhadoop.security.logger=INFO,RFAS -Dhdfs.audit.logger=INFO,NullAppender
-Dhadoop.security.logger=INFO,RFAS
org.apache.hadoop.hdfs.server.namenode.NameNode

And the hadoop version is:

ubuntu@ubuntu-171:~$ Documents/hadoop-2.6.0/bin/hadoop version
Hadoop 2.6.0
Subversion https://git-wip-us.apache.org/repos/asf/hadoop.git -r
e3496499ecb8d220fba99dc5ed4c99c8f9e33bb1
Compiled by jenkins on 2014-11-13T21:10Z
Compiled with protoc 2.5.0
From source with checksum 18e43357c8f927c0695f1e9522859d6a
This command was run using
/home/ubuntu/Documents/hadoop-2.6.0/share/hadoop/common/hadoop-common-2.6.0
.jar



-Ali



On 2016-01-04, 2:20 PM, "Robert Metzger"  wrote:

>Your Flink installation has Hadoop 2.6.0 included, on the other machine,
>there is a Hadoop version installed, which is most likely a 1.x or even a
>0.x version.
>Are you sure that the host "ubuntu-171" has the ip 10.13.182.171 and that
>the hadoop installation in the "/home/ubuntu/Documents/hadoop-2.6.0/"
>directory is listening on port 9000 ?
>
>On Mon, Jan 4, 2016 at 3:03 PM, Kashmar, Ali  wrote:
>
>> Hi Max,
>>
>> Both commands return nothing. Those variables aren’t set.
>>
>> The only software I installed on these machines is Flink and Java.
>>
>> -Ali
>>
>> On 2015-12-28, 6:42 AM, "Maximilian Michels"  wrote:
>>
>> >Hi Ali,
>> >
>> >The warning about the native Hadoop libraries is nothing to worry
>> >about. The native modules are platform-optimized modules which may be
>> >used to improve performance. They are not necessary for Hadoop to
>> >function correctly.
>> >
>> >The exception message implies that you are using is a very old version
>> >of Hadoop. Do you have other Hadoop versions installed on the same
>> >machine? We have had people using Flink 0.10.0 with Hadoop 2.6.0
>> >without any problems.
>> >
>> >On the cluster machines, what is the output of these commands?
>> >
>> >echo $HADOOP_CLASSPATH
>> >echo $HADOOP_CONF_DIR
>> >
>> >
>> >Thanks,
>> >Max
>> >
>> >On Wed, Dec 23, 2015 at 3:53 PM, Kashmar, Ali 
>> wrote:
>> >> Hi Max,
>> >>
>> >> I have the same output for the Task Manager:
>> >>
>> >> 11:25:04,274 INFO  org.apache.flink.runtime.taskmanager.TaskManager
>> >>   -  Hadoop version: 2.6.0
>> >>
>> >> I do get this line at the beginning of both job and task manager log
>> >>files:
>> >>
>> >> 11:25:04,100 WARN  org.apache.hadoop.util.NativeCodeLoader
>> >>   - Unable to load native-hadoop library for your platform...
>>using
>> >> builtin-java classes where applicable
>> >>
>> >> Do you think it has anything to do with it?
>> >>
>> >> Thanks,
>> >> Ali
>> >>
>> >> On 2015-12-23, 7:30 AM, "Maximilian Michels"  wrote:
>> >>
>> >>>Hi Ali,
>> >>>
>> >>>Could you please also post the Hadoop version output of the task
>> >>>manager log files? It looks like the task managers are running a
>> >>>different Hadoop version.
>> >>>
>> >>>Thanks,
>> >>>Max
>> >>>
>> >>>On Tue, Dec 22, 2015 at 4:28 PM, Kashmar, Ali 
>> >>>wrote:
>>  Hi Robert,
>> 
>>  I found the version in the job manager log file:
>> 
>>  17:33:49,636 INFO  org.apache.flink.runtime.jobmanager.JobManager
>>    -  Hadoop version: 2.6.0
>> 
>>  But the Hadoop installation I have is saying this:
>> 
>>  ubuntu@ubuntu-171:~/Documents/hadoop-2.6.0$ bin/hadoop version
>>  Hadoop 2.6.0
>>  Subversion https://git-wip-us.apache.org/repos/asf/hadoop.git -r
>>  e3496499ecb8d220fba99dc5ed4c99c8f9e33bb1
>>  Compiled by jenkins on 2014-11-13T21:10Z
>>  Compiled with protoc 2.5.0
>>  From source with checksum 

[jira] [Created] (FLINK-3199) KafkaITCase.testOneToOneSources

2016-01-04 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created FLINK-3199:
--

 Summary: KafkaITCase.testOneToOneSources
 Key: FLINK-3199
 URL: https://issues.apache.org/jira/browse/FLINK-3199
 Project: Flink
  Issue Type: Bug
  Components: Tests
Reporter: Matthias J. Sax
Priority: Critical


https://travis-ci.org/mjsax/flink/jobs/100167558
{noformat}
Failed tests: 
KafkaITCase.testOneToOneSources:96->KafkaConsumerTestBase.runOneToOneExactlyOnceTest:521->KafkaTestBase.tryExecute:318
 Test failed: The program execution failed: Job execution failed.
{noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: 2015: A Year in Review for Apache Flink

2016-01-04 Thread Till Rohrmann
Happy New Year :-) Hope everyone had a great start into the new year.

On Thu, Dec 31, 2015 at 12:57 PM, Slim Baltagi  wrote:

> Happy New Year to you and your families!
> Let’s  make 2016 the year of Flink: General Availability, faster growth,
> wider industry adoption, …
> Slim Baltagi
> Chicago, US
>
> On Dec 31, 2015, at 5:05 AM, Vasiliki Kalavri 
> wrote:
>
> Happy new year everyone!
> Looking forward to all the great things the Apache Flink community will
> accomplish in 2016 :))
>
> Greetings from snowy Greece!
> -Vasia.
>
> On 31 December 2015 at 04:22, Henry Saputra 
> wrote:
>
>> Dear All,
>>
>> It is almost end of 2015 and it has been busy and great year for Apache
>> Flink =)
>>
>> Robert Metzger had posted great blog summarizing Apache Flink grow for
>> this year:
>>
>>   https://flink.apache.org/news/2015/12/18/a-year-in-review.html
>>
>> Happy New Year everyone and thanks for being part of this great community!
>>
>>
>> Thanks,
>>
>> - Henry
>>
>
>
>


Re: Error due to Hadoop version mismatch

2016-01-04 Thread Kashmar, Ali
Hi Max,

Both commands return nothing. Those variables aren’t set.

The only software I installed on these machines is Flink and Java.

-Ali

On 2015-12-28, 6:42 AM, "Maximilian Michels"  wrote:

>Hi Ali,
>
>The warning about the native Hadoop libraries is nothing to worry
>about. The native modules are platform-optimized modules which may be
>used to improve performance. They are not necessary for Hadoop to
>function correctly.
>
>The exception message implies that you are using is a very old version
>of Hadoop. Do you have other Hadoop versions installed on the same
>machine? We have had people using Flink 0.10.0 with Hadoop 2.6.0
>without any problems.
>
>On the cluster machines, what is the output of these commands?
>
>echo $HADOOP_CLASSPATH
>echo $HADOOP_CONF_DIR
>
>
>Thanks,
>Max
>
>On Wed, Dec 23, 2015 at 3:53 PM, Kashmar, Ali  wrote:
>> Hi Max,
>>
>> I have the same output for the Task Manager:
>>
>> 11:25:04,274 INFO  org.apache.flink.runtime.taskmanager.TaskManager
>>   -  Hadoop version: 2.6.0
>>
>> I do get this line at the beginning of both job and task manager log
>>files:
>>
>> 11:25:04,100 WARN  org.apache.hadoop.util.NativeCodeLoader
>>   - Unable to load native-hadoop library for your platform... using
>> builtin-java classes where applicable
>>
>> Do you think it has anything to do with it?
>>
>> Thanks,
>> Ali
>>
>> On 2015-12-23, 7:30 AM, "Maximilian Michels"  wrote:
>>
>>>Hi Ali,
>>>
>>>Could you please also post the Hadoop version output of the task
>>>manager log files? It looks like the task managers are running a
>>>different Hadoop version.
>>>
>>>Thanks,
>>>Max
>>>
>>>On Tue, Dec 22, 2015 at 4:28 PM, Kashmar, Ali 
>>>wrote:
 Hi Robert,

 I found the version in the job manager log file:

 17:33:49,636 INFO  org.apache.flink.runtime.jobmanager.JobManager
   -  Hadoop version: 2.6.0

 But the Hadoop installation I have is saying this:

 ubuntu@ubuntu-171:~/Documents/hadoop-2.6.0$ bin/hadoop version
 Hadoop 2.6.0
 Subversion https://git-wip-us.apache.org/repos/asf/hadoop.git -r
 e3496499ecb8d220fba99dc5ed4c99c8f9e33bb1
 Compiled by jenkins on 2014-11-13T21:10Z
 Compiled with protoc 2.5.0
 From source with checksum 18e43357c8f927c0695f1e9522859d6a
 This command was run using

/home/ubuntu/Documents/hadoop-2.6.0/share/hadoop/common/hadoop-common-2
.6
.0
 .jar


 So one of them is lying to me? :)

 Ali

 On 2015-12-22, 10:16 AM, "Robert Metzger"  wrote:

>Hi Ali,
>
>the TaskManagers and the JobManager is logging the Hadoop version on
>startup.
>
>On Tue, Dec 22, 2015 at 4:10 PM, Kashmar, Ali 
>wrote:
>
>> Hello,
>>
>> I¹m trying to use HDFS as store for Flink checkpoints so I
>>downloaded
>>the
>> Hadoop 2.6.0/Scala 2.10 version of Flink and installed it. I also
>> downloaded Hadoop 2.6.0 separately from the Hadoop website and set
>>up
>>HDFS
>> on a separate machine. When I start Flink I get the following error:
>>
>> 17:34:13,047 INFO  org.apache.flink.runtime.jobmanager.JobManager
>>   - Status of job 9ba32a08bc0ec02810bf5d2710842f72 (Protocol
>>Event
>> Processing) changed to FAILED.
>> java.lang.Exception: Call to registerInputOutput() of invokable
>>failed
>> at
>>org.apache.flink.runtime.taskmanager.Task.run(Task.java:529)
>> at java.lang.Thread.run(Thread.java:745)
>> Caused by: java.io.IOException: The given file URI (hdfs://
>> 10.13.182.171:9000/user/flink/checkpoints) points to the HDFS
>>NameNode
>>at
>> 10.13.182.171:9000, but the File System could not be initialized
>>with
>> that address: Server IPC version 9 cannot communicate with client
>>version 4
>> at
>>
>>org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.initialize(HadoopFi
>>le
>>Sy
>>stem.java:337)
>> at
>>org.apache.flink.core.fs.FileSystem.get(FileSystem.java:253)
>> at
>>
>>org.apache.flink.runtime.state.filesystem.FsStateBackend.(FsSta
>>te
>>Ba
>>ckend.java:142)
>> at
>>
>>org.apache.flink.runtime.state.filesystem.FsStateBackend.(FsSta
>>te
>>Ba
>>ckend.java:101)
>> at
>>
>>org.apache.flink.runtime.state.filesystem.FsStateBackendFactory.creat
>>eF
>>ro
>>mConfig(FsStateBackendFactory.java:48)
>> at
>>
>>org.apache.flink.streaming.runtime.tasks.StreamTask.createStateBacken
>>d(
>>St
>>reamTask.java:517)
>> at
>>
>>org.apache.flink.streaming.runtime.tasks.StreamTask.registerInputOutp
>>ut
>>(S
>>treamTask.java:171)
>> at

Re: Error due to Hadoop version mismatch

2016-01-04 Thread Kashmar, Ali
Looking at the lib folder revealed the problem. The lib folder on one of
the nodes had libraries for both hadoop 1 and 2. I’m not sure how I ended
up with that but it must have happened while I was copying the dependency
jars to each node. I removed all the jars and started with a fresh copy
and reran the test. It worked this time.

Thank you so much for being patient with me and sorry if I wasted your
time.

-Ali

On 2016-01-04, 3:17 PM, "Robert Metzger"  wrote:

>Okay, then, my previous statement is false. From the stack trace, it seems
>that Flink is using an older Hadoop version. The DFSClient and RPC classes
>look different in Hadoop 2.6.0.
>Max checked already for some environment variables. Is $CLASSPATH set? Did
>you install Hadoop only by downloading the binaries or did you use
>something like the ubuntu package manager? (those packages sometimes place
>jar files in some lib/ folders)
>Did you put something additionally into the lib/ folder of Flink?
>
>I think when Flink is starting up, its also logging the classpath of the
>JVM. Can you post it here? (Maybe you need to start with DEBUG log level)
>
>
>On Mon, Jan 4, 2016 at 8:46 PM, Kashmar, Ali  wrote:
>
>> Hi Robert,
>>
>> On the ubuntu host, the port 9000 is up:
>>
>> ubuntu@ubuntu-171:~$ netstat -tulpn
>> 
>> tcp0  0 10.13.182.171:9000  0.0.0.0:*
>>LISTEN
>>  8849/java
>> 
>>
>> And the process using this port is:
>>
>> ubuntu@ubuntu-171:~$ ps -ef | grep 8849
>> ubuntu8849 1  0  2015 ?00:25:00
>> /usr/lib/jvm/java-8-oracle/bin/java -Dproc_namenode -Xmx1000m
>> -Djava.net.preferIPv4Stack=true
>> -Dhadoop.log.dir=/home/ubuntu/Documents/hadoop-2.6.0/logs
>> -Dhadoop.log.file=hadoop.log
>> -Dhadoop.home.dir=/home/ubuntu/Documents/hadoop-2.6.0
>> -Dhadoop.id.str=ubuntu -Dhadoop.root.logger=INFO,console
>> -Djava.library.path=/home/ubuntu/Documents/hadoop-2.6.0/lib/native
>> -Dhadoop.policy.file=hadoop-policy.xml -Djava.net.preferIPv4Stack=true
>> -Djava.net.preferIPv4Stack=true -Djava.net.preferIPv4Stack=true
>> -Dhadoop.log.dir=/home/ubuntu/Documents/hadoop-2.6.0/logs
>> -Dhadoop.log.file=hadoop-ubuntu-namenode-ubuntu-171.log
>> -Dhadoop.home.dir=/home/ubuntu/Documents/hadoop-2.6.0
>> -Dhadoop.id.str=ubuntu -Dhadoop.root.logger=INFO,RFA
>> -Djava.library.path=/home/ubuntu/Documents/hadoop-2.6.0/lib/native
>> -Dhadoop.policy.file=hadoop-policy.xml -Djava.net.preferIPv4Stack=true
>> -Dhadoop.security.logger=INFO,RFAS -Dhdfs.audit.logger=INFO,NullAppender
>> -Dhadoop.security.logger=INFO,RFAS -Dhdfs.audit.logger=INFO,NullAppender
>> -Dhadoop.security.logger=INFO,RFAS -Dhdfs.audit.logger=INFO,NullAppender
>> -Dhadoop.security.logger=INFO,RFAS
>> org.apache.hadoop.hdfs.server.namenode.NameNode
>>
>> And the hadoop version is:
>>
>> ubuntu@ubuntu-171:~$ Documents/hadoop-2.6.0/bin/hadoop version
>> Hadoop 2.6.0
>> Subversion https://git-wip-us.apache.org/repos/asf/hadoop.git -r
>> e3496499ecb8d220fba99dc5ed4c99c8f9e33bb1
>> Compiled by jenkins on 2014-11-13T21:10Z
>> Compiled with protoc 2.5.0
>> From source with checksum 18e43357c8f927c0695f1e9522859d6a
>> This command was run using
>> 
>>/home/ubuntu/Documents/hadoop-2.6.0/share/hadoop/common/hadoop-common-2.6
>>.0
>> .jar
>>
>>
>>
>> -Ali
>>
>>
>>
>> On 2016-01-04, 2:20 PM, "Robert Metzger"  wrote:
>>
>> >Your Flink installation has Hadoop 2.6.0 included, on the other
>>machine,
>> >there is a Hadoop version installed, which is most likely a 1.x or
>>even a
>> >0.x version.
>> >Are you sure that the host "ubuntu-171" has the ip 10.13.182.171 and
>>that
>> >the hadoop installation in the "/home/ubuntu/Documents/hadoop-2.6.0/"
>> >directory is listening on port 9000 ?
>> >
>> >On Mon, Jan 4, 2016 at 3:03 PM, Kashmar, Ali 
>>wrote:
>> >
>> >> Hi Max,
>> >>
>> >> Both commands return nothing. Those variables aren’t set.
>> >>
>> >> The only software I installed on these machines is Flink and Java.
>> >>
>> >> -Ali
>> >>
>> >> On 2015-12-28, 6:42 AM, "Maximilian Michels"  wrote:
>> >>
>> >> >Hi Ali,
>> >> >
>> >> >The warning about the native Hadoop libraries is nothing to worry
>> >> >about. The native modules are platform-optimized modules which may
>>be
>> >> >used to improve performance. They are not necessary for Hadoop to
>> >> >function correctly.
>> >> >
>> >> >The exception message implies that you are using is a very old
>>version
>> >> >of Hadoop. Do you have other Hadoop versions installed on the same
>> >> >machine? We have had people using Flink 0.10.0 with Hadoop 2.6.0
>> >> >without any problems.
>> >> >
>> >> >On the cluster machines, what is the output of these commands?
>> >> >
>> >> >echo $HADOOP_CLASSPATH
>> >> >echo $HADOOP_CONF_DIR
>> >> >
>> >> >
>> >> >Thanks,
>> >> >Max
>> >> >
>> >> >On Wed, Dec 23, 2015 at 3:53 PM, Kashmar, Ali 
>> >> wrote:
>> >> >> Hi Max,
>> >> >>
>> >> >> I have the same output for the Task