You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Shashank Agarwal (JIRA)" <ji...@apache.org> on 2018/03/05 14:46:00 UTC

[jira] [Created] (FLINK-8867) Rocksdb checkpointing failing with fs.default-scheme: hdfs:// config

Shashank Agarwal created FLINK-8867:
---------------------------------------

             Summary: Rocksdb checkpointing failing with fs.default-scheme: hdfs:// config
                 Key: FLINK-8867
                 URL: https://issues.apache.org/jira/browse/FLINK-8867
             Project: Flink
          Issue Type: Bug
          Components: Configuration, State Backends, Checkpointing, YARN
    Affects Versions: 1.4.1, 1.4.2
            Reporter: Shashank Agarwal
             Fix For: 1.5.0, 1.4.3


In our setup, when we put an entry in our Flink_conf file for default schema.

{code}
fs.default-scheme: hdfs://mydomain.com:8020/flink
{code}

Than application with rocksdb state backend fails with the following exception. When we remove this config it works fine. It's working fine with other state backends.

{code}
AsynchronousException{java.lang.Exception: Could not materialize checkpoint 1 for operator order ip stream (1/1).}
	at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:948)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.Exception: Could not materialize checkpoint 1 for operator order ip stream (1/1).
	... 6 more
Caused by: java.util.concurrent.ExecutionException: java.lang.IllegalStateException
	at java.util.concurrent.FutureTask.report(FutureTask.java:122)
	at java.util.concurrent.FutureTask.get(FutureTask.java:192)
	at org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
	at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:894)
	... 5 more
	Suppressed: java.lang.Exception: Could not properly cancel managed keyed state future.
		at org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:91)
		at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.cleanup(StreamTask.java:976)
		at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:939)
		... 5 more
	Caused by: java.util.concurrent.ExecutionException: java.lang.IllegalStateException
		at java.util.concurrent.FutureTask.report(FutureTask.java:122)
		at java.util.concurrent.FutureTask.get(FutureTask.java:192)
		at org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
		at org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:66)
		at org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:89)
		... 7 more
	Caused by: java.lang.IllegalStateException
		at org.apache.flink.util.Preconditions.checkState(Preconditions.java:179)
		at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBIncrementalSnapshotOperation.materializeSnapshot(RocksDBKeyedStateBackend.java:926)
		at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$1.call(RocksDBKeyedStateBackend.java:389)
		at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$1.call(RocksDBKeyedStateBackend.java:386)
		at java.util.concurrent.FutureTask.run(FutureTask.java:266)
		at org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:40)
		at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:894)
		... 5 more
	[CIRCULAR REFERENCE:java.lang.IllegalStateException]
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)