Hi Ning,

The problem here first of all is that RocksDB java JNI client diverged from 
RocksDB cpp code in status.h,
as mentioned in the Flink issue you refer to.

Flink 1.6 uses RocksDB 5.7.5 java client. 
The JNI code there misses these status subcodes:
kNoSpace = 4,
kDeadlock = 5,
kStaleFile = 6,
kMemoryLimit = 7
which could be potential problems in the job.

kNoSpace is only one of them.
Another probable cause could be kStaleFile, some file system IO problem.
kDeadlock seems to be used only with transactions, so not relevant.
kMemoryLimit means that write batch exceeded max size, but we do not have limit 
for it as I understand.

It would be easier to debug if RocksDB JNI client would at least log the 
unknown subcode but i do not see any easy way to log it in the current version, 
without rebuilding RocksDB and subsequently Flink.

In master branch, java Status and status.h are also unsynced. You could report 
this issue in RocksDB repo, along with extending exception logging message with 
the number of unknown error code. Flink community plans to upgrade to the 
latest RocksDB version again in one of the next Flink releases.

Best,
Andrey

> On 25 Oct 2018, at 04:31, Ning Shi <nings...@gmail.com> wrote:
> 
> Hi,
> 
> We are doing some performance testing on a 12 node cluster with 8 task
> slots per TM. Every 15 minutes or so, the job would run into the
> following exception.
> 
> java.lang.IllegalArgumentException: Illegal value provided for SubCode.
>       at org.rocksdb.Status$SubCode.getSubCode(Status.java:109)
>       at org.rocksdb.Status.<init>(Status.java:30)
>       at org.rocksdb.RocksDB.put(Native Method)
>       at org.rocksdb.RocksDB.put(RocksDB.java:511)
>       at 
> org.apache.flink.contrib.streaming.state.AbstractRocksDBAppendingState.updateInternal(AbstractRocksDBAppendingState.java:80)
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBReducingState.add(RocksDBReducingState.java:99)
>       at 
> org.apache.flink.streaming.runtime.operators.windowing.WindowOperator.processElement(WindowOperator.java:358)
>       at 
> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:202)
>       at 
> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:105)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:300)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
>       at java.lang.Thread.run(Thread.java:745)
> 
> I saw an outstanding issue with similar exception in [1]. The ticket
> description suggests that it was due to out of disk error, but in our
> case, we have plenty of disk left on all TMs.
> 
> Has anyone run into this before? If so, is there a fix or workaround?
> 
> Thanks,
> 
> [1] https://issues.apache.org/jira/browse/FLINK-9233
> 
> --
> Ning

Reply via email to