flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Stefan Richter <s.rich...@data-artisans.com>
Subject Re: Unable to use Flink RocksDB state backend due to endianness mismatch
Date Tue, 06 Jun 2017 08:01:20 GMT
Hi,

RocksDB is a native library with JNI binding. It is included as a dependency and does not
build from source when you build Flink. The included jar provides native code for Linux, OSX,
and Windows on x86-64. From the Exception, I would conclude you are using a different CPU
architecture that is big endian. In this case, you would have to compile RocksDB yourself
on your native platform and replace the jar that ships with Flink.

Best,
Stefan

> Am 03.06.2017 um 14:16 schrieb Ziyad Muhammed <mmziyad@gmail.com>:
> 
> Dear all,
> 
> My Flink Job reads from a kafka topic and store the data in a RocksDB state backend,
in order to make use of the queryable state. I'm able to run the job and query the state in
my local machine. But when deploying on the cluster, I'm getting the below error:
> 
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
> at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply$mcV$sp(JobManager.scala:900)
> at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:843)
> at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:843)
> at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
> at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
> at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
> at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
> at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> 
> Caused by: java.lang.IllegalStateException: Could not initialize keyed state backend.
> at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initKeyedState(AbstractStreamOperator.java:286)
> at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:199)
> at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeOperators(StreamTask.java:666)
> at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:654)
> at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:257)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:655)
> at java.lang.Thread.run(Thread.java:745)
> 
> Caused by: java.lang.Exception: Could not load the native RocksDB library
> at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.ensureRocksDBIsLoaded(RocksDBStateBackend.java:483)
> at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createKeyedStateBackend(RocksDBStateBackend.java:235)
> at org.apache.flink.streaming.runtime.tasks.StreamTask.createKeyedStateBackend(StreamTask.java:785)
> at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initKeyedState(AbstractStreamOperator.java:277)
> ... 6 more
> 
> Caused by: java.lang.UnsatisfiedLinkError: /data/hadoop/tmp/rocksdb-lib-32bc6a5551b331596649309a808b287d/librocksdbjni-linux64.so:
/data/hadoop/tmp/rocksdb-lib-32bc6a5551b331596649309a808b287d/librocksdbjni-linux64.so: ELF
file data encoding not big-endian (Possible cause: endianness mismatch)
> at java.lang.ClassLoader$NativeLibrary.load(Native Method)
> at java.lang.ClassLoader.loadLibrary0(ClassLoader.java:1941)
> at java.lang.ClassLoader.loadLibrary(ClassLoader.java:1824)
> at java.lang.Runtime.load0(Runtime.java:809)
> at java.lang.System.load(System.java:1086)
> at org.rocksdb.NativeLibraryLoader.loadLibraryFromJar(NativeLibraryLoader.java:78)
> at org.rocksdb.NativeLibraryLoader.loadLibrary(NativeLibraryLoader.java:56)
> at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.ensureRocksDBIsLoaded(RocksDBStateBackend.java:460)
> I have tried setting the rocksDB state backend at cluster level and job level. when it
is set at job level, I have provided it as a shaded dependency. I have tried compiling the
code on the host cluster machine as well. I get the same error in all the cases. 
> 
> ELF file data encoding not big-endian (Possible cause: endianness mismatch)
> How can I solve this error?
> 
> Thanks in advance
> Ziyad
> 


Mime
View raw message