[ 
https://issues.apache.org/jira/browse/FLINK-17506?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17102907#comment-17102907
 ] 

David Artiga commented on FLINK-17506:
--------------------------------------

Applied your change to our fork. Still using '/tmp'. Could it be related to 
[this|https://github.com/apache/flink/blob/571c0892fa30a34751b575c10b78472eed8849e2/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java#L282]
 and not calling {{Builder::setConfiguration()}} anywhere 
[here|https://github.com/apache/flink/blob/571c0892fa30a34751b575c10b78472eed8849e2/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/KeyedStateInputFormat.java#L139-L143]?

Including stacktrace:
{code:java}
2020-05-08 21:55:08
org.apache.flink.runtime.JobException: Recovery is suppressed by 
NoRestartBackoffTimeStrategy
        at 
org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:110)
        at 
org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:76)
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:192)
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:186)
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:180)
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:462)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:367)
        at sun.reflect.GeneratedMethodAccessor31.invoke(Unknown Source)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:279)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:194)
        at 
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
        at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
        at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
        at akka.actor.Actor$class.aroundReceive(Actor.scala:517)
        at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
        at akka.actor.ActorCell.invoke(ActorCell.scala:561)
        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
        at akka.dispatch.Mailbox.run(Mailbox.scala:225)
        at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
        at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
        at 
akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
        at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
        at 
akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
Caused by: java.io.IOException: Failed to restore state backend
        at 
org.apache.flink.state.api.input.KeyedStateInputFormat.getStreamOperatorStateContext(KeyedStateInputFormat.java:177)
        at 
org.apache.flink.state.api.input.KeyedStateInputFormat.open(KeyedStateInputFormat.java:145)
        at 
org.apache.flink.state.api.input.KeyedStateInputFormat.open(KeyedStateInputFormat.java:63)
        at 
org.apache.flink.runtime.operators.DataSourceTask.invoke(DataSourceTask.java:173)
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.Exception: Exception while creating 
StreamOperatorStateContext.
        at 
org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:191)
        at 
org.apache.flink.state.api.input.KeyedStateInputFormat.getStreamOperatorStateContext(KeyedStateInputFormat.java:168)
        ... 6 more
Caused by: org.apache.flink.util.FlinkException: Could not restore keyed state 
backend for 
707eed42a9b74f065cc8bb6798b04782_707eed42a9b74f065cc8bb6798b04782_(8/128) from 
any of the 1 provided restore options.
        at 
org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
        at 
org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.keyedStatedBackend(StreamTaskStateInitializerImpl.java:304)
        at 
org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:131)
        ... 7 more
Caused by: org.apache.flink.runtime.state.BackendBuildingException: Caught 
unexpected exception.
        at 
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackendBuilder.build(RocksDBKeyedStateBackendBuilder.java:324)
        at 
org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createKeyedStateBackend(RocksDBStateBackend.java:560)
        at 
org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$keyedStatedBackend$1(StreamTaskStateInitializerImpl.java:288)
        at 
org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:142)
        at 
org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:121)
        ... 9 more
Caused by: org.rocksdb.RocksDBException: While appending to file: 
/tmp/flink-io-b9e45ad7-9765-4225-ab36-210804b9fa36/job_c543bf4d4f8ed7396bf4bc3523ab8ab9_op_707eed42a9b74f065cc8bb6798b04782_707eed42a9b74f065cc8bb6798b04782__8_128__uuid_117bcd9f-f325-4561-9f12-f10e71e7e5ae/db/000509.sst:
 No space left on device
        at org.rocksdb.RocksDB.write0(Native Method)
        at org.rocksdb.RocksDB.write(RocksDB.java:602)
        at 
org.apache.flink.contrib.streaming.state.RocksDBWriteBatchWrapper.flush(RocksDBWriteBatchWrapper.java:112)
        at 
org.apache.flink.contrib.streaming.state.RocksDBWriteBatchWrapper.flushIfNeeded(RocksDBWriteBatchWrapper.java:133)
        at 
org.apache.flink.contrib.streaming.state.RocksDBWriteBatchWrapper.put(RocksDBWriteBatchWrapper.java:94)
        at 
org.apache.flink.contrib.streaming.state.restore.RocksDBFullRestoreOperation.restoreKVStateData(RocksDBFullRestoreOperation.java:236)
        at 
org.apache.flink.contrib.streaming.state.restore.RocksDBFullRestoreOperation.restoreKeyGroupsInStateHandle(RocksDBFullRestoreOperation.java:168)
        at 
org.apache.flink.contrib.streaming.state.restore.RocksDBFullRestoreOperation.restore(RocksDBFullRestoreOperation.java:151)
        at 
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackendBuilder.build(RocksDBKeyedStateBackendBuilder.java:256)
        ... 13 more
 {code}

> SavepointEnvironment does not honour 'io.tmp.dirs' property
> -----------------------------------------------------------
>
>                 Key: FLINK-17506
>                 URL: https://issues.apache.org/jira/browse/FLINK-17506
>             Project: Flink
>          Issue Type: Bug
>          Components: API / State Processor
>    Affects Versions: 1.11.0, 1.10.2
>            Reporter: David Artiga
>            Assignee: Seth Wiesman
>            Priority: Major
>              Labels: pull-request-available
>
> {{SavepointEnvironment}} [creates an 
> IOManagerAsync|https://github.com/apache/flink/blob/d6439c8d0e7792961635e3e4297c3dbfb01938e3/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java#L106]
>  using its [default 
> constructor|https://github.com/apache/flink/blob/d6439c8d0e7792961635e3e4297c3dbfb01938e3/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsync.java#L62],
>  meaning it [uses env var 
> "java.io.tmpdir"|https://github.com/apache/flink/blob/d6439c8d0e7792961635e3e4297c3dbfb01938e3/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java#L227]
>  instead of values from "io.tmp.dirs" config property,



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to