You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by xianwen jin <xi...@gmail.com> on 2021/08/10 13:03:44 UTC

State Processor API with EmbeddedRocksDBStateBackend

Hi Flink Community,
I have an issue regarding create a savepoint with EmbeddedRocksDBStateBackend.
Since I have to use dataset for state processor api, it seems that the checkpoint storage is always set to jobmanager even though I pass:
state.checkpoint-storage: filesystem
state.checkpoints.dir: 's3://flink-bucket01/checkpoints/saldo-es/state'
in the flink-conf.yaml.
This causes an exception when the state to be written is more than 5 MB:
java.io.IOException: Size of the state is larger than the maximum permitted memory-backed state. Size=100091190 , maxSize=5242880 . Consider using a different state backend, like the File System State backend.

There's a way to set checkpoint storage to filesystem when I'm using state processor api?

Thanks for any advice or help

Re: State Processor API with EmbeddedRocksDBStateBackend

Posted by Seth Wiesman <sj...@gmail.com>.
Hi Xianwen,

Looks like the State Processor API needs to be updated for the new state
backend factory stack. For now, just use RocksDBStateBackend and it will
work as intended.

I've opened a ticket: https://issues.apache.org/jira/browse/FLINK-23728


Seth

On Wed, Aug 11, 2021 at 2:08 AM xianwen jin <xi...@gmail.com> wrote:

> Hi Yun,
>
> Here's the stack trace
>
> java.util.concurrent.ExecutionException: java.io.IOException: Size of the
> state is larger than the maximum permitted memory-backed state.
> Size=100091190 , maxSize=5242880 . Consider using a different state
> backend, like the File System State backend.
>         at java.util.concurrent.FutureTask.report(FutureTask.java:122)
> ~[?:1.8.0_292]
>         at java.util.concurrent.FutureTask.get(FutureTask.java:192)
> ~[?:1.8.0_292]
>         at
> org.apache.flink.runtime.concurrent.FutureUtils.runIfNotDoneAndGet(FutureUtils.java:636)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:54)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.state.api.output.SnapshotUtils.snapshot(SnapshotUtils.java:67)
> ~[?:?]
>         at
> org.apache.flink.state.api.output.operators.KeyedStateBootstrapOperator.endInput(KeyedStateBootstrapOperator.java:90)
> ~[?:?]
>         at
> org.apache.flink.state.api.output.BoundedStreamTask.processInput(BoundedStreamTask.java:107)
> ~[?:?]
>         at
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:204)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:681)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask.executeInvoke(StreamTask.java:636)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:647)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:620)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.state.api.output.BoundedOneInputStreamTaskRunner.mapPartition(BoundedOneInputStreamTaskRunner.java:80)
> ~[?:?]
>         at
> org.apache.flink.runtime.operators.MapPartitionDriver.run(MapPartitionDriver.java:113)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:519)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:360)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:779)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_292]
> Caused by: java.io.IOException: Size of the state is larger than the
> maximum permitted memory-backed state. Size=100091190 , maxSize=5242880 .
> Consider using a different state backend, like the File System State
> backend.
>         at
> org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory.checkSize(MemCheckpointStreamFactory.java:61)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory$MemoryCheckpointOutputStream.closeAndGetBytes(MemCheckpointStreamFactory.java:141)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory$MemoryCheckpointOutputStream.closeAndGetHandle(MemCheckpointStreamFactory.java:121)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.state.CheckpointStreamWithResultProvider$PrimaryStreamOnly.closeAndFinalizeCheckpointStreamResult(CheckpointStreamWithResultProvider.java:75)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.state.FullSnapshotAsyncWriter.get(FullSnapshotAsyncWriter.java:87)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.state.SnapshotStrategyRunner$1.callInternal(SnapshotStrategyRunner.java:91)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.state.SnapshotStrategyRunner$1.callInternal(SnapshotStrategyRunner.java:88)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.state.AsyncSnapshotCallable.call(AsyncSnapshotCallable.java:78)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> ~[?:1.8.0_292]
>         at
> org.apache.flink.runtime.concurrent.FutureUtils.runIfNotDoneAndGet(FutureUtils.java:633)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         ... 16 more
> 2021-08-09 15:59:53,314 INFO
> org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionFailoverStrategy
> [] - Calculating tasks to restart to recover the failed task
> 488bec663292a28f42ac2a625db66756_0.
> 2021-08-09 15:59:53,315 INFO
> org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionFailoverStrategy
> [] - 25 tasks should be restarted to recover the failed task
> 488bec663292a28f42ac2a625db66756_0.
> 2021-08-09 15:59:53,364 INFO
> org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Job
> PipelineName (43e80a9e125b45e6be8ac263f1c8fd4e) switched from state RUNNING
> to FAILING.
> org.apache.flink.runtime.JobException: Recovery is suppressed by
> NoRestartBackoffTimeStrategy
>         at
> org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:138)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:82)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:207)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:197)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:188)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:677)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:79)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:435)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at sun.reflect.GeneratedMethodAccessor65.invoke(Unknown Source)
> ~[?:?]
>         at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> ~[?:1.8.0_292]
>         at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_292]
>         at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:305)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:212)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:77)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:158)
> ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at scala.PartialFunction.applyOrElse(PartialFunction.scala:123)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at akka.actor.Actor.aroundReceive(Actor.scala:517)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at akka.actor.Actor.aroundReceive$(Actor.scala:515)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at akka.actor.ActorCell.invoke(ActorCell.scala:561)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at akka.dispatch.Mailbox.run(Mailbox.scala:225)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>         at
> akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
>
> Thanks.
>
> On 2021/08/11 02:30:41, "Yun Gao" <yu...@aliyun.com> wrote:
> > Hi Xianwen,
> >
> > Could you also attach the full stack of the exception~?
> >
> > Very thanks
> >
> >
> > ------------------------------------------------------------------
> > Sender:xianwen jin<xi...@gmail.com>
> > Date:2021/08/10 21:03:44
> > Recipient:<us...@flink.apache.org>
> > Theme:State Processor API with EmbeddedRocksDBStateBackend
> >
> > Hi Flink Community,
> > I have an issue regarding create a savepoint with
> EmbeddedRocksDBStateBackend.
> > Since I have to use dataset for state processor api, it seems that the
> checkpoint storage is always set to jobmanager even though I pass:
> > state.checkpoint-storage: filesystem
> > state.checkpoints.dir: 's3://flink-bucket01/checkpoints/saldo-es/state'
> > in the flink-conf.yaml.
> > This causes an exception when the state to be written is more than 5 MB:
> > java.io.IOException: Size of the state is larger than the maximum
> permitted memory-backed state. Size=100091190 , maxSize=5242880 . Consider
> using a different state backend, like the File System State backend.
> >
> > There's a way to set checkpoint storage to filesystem when I'm using
> state processor api?
> >
> > Thanks for any advice or help
> >
>

Re: State Processor API with EmbeddedRocksDBStateBackend

Posted by xianwen jin <xi...@gmail.com>.
Hi Yun,

Here's the stack trace

java.util.concurrent.ExecutionException: java.io.IOException: Size of the state is larger than the maximum permitted memory-backed state. Size=100091190 , maxSize=5242880 . Consider using a different state backend, like the File System State backend.
	at java.util.concurrent.FutureTask.report(FutureTask.java:122) ~[?:1.8.0_292]
	at java.util.concurrent.FutureTask.get(FutureTask.java:192) ~[?:1.8.0_292]
	at org.apache.flink.runtime.concurrent.FutureUtils.runIfNotDoneAndGet(FutureUtils.java:636) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:54) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.state.api.output.SnapshotUtils.snapshot(SnapshotUtils.java:67) ~[?:?]
	at org.apache.flink.state.api.output.operators.KeyedStateBootstrapOperator.endInput(KeyedStateBootstrapOperator.java:90) ~[?:?]
	at org.apache.flink.state.api.output.BoundedStreamTask.processInput(BoundedStreamTask.java:107) ~[?:?]
	at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:204) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:681) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.executeInvoke(StreamTask.java:636) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:647) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:620) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.state.api.output.BoundedOneInputStreamTaskRunner.mapPartition(BoundedOneInputStreamTaskRunner.java:80) ~[?:?]
	at org.apache.flink.runtime.operators.MapPartitionDriver.run(MapPartitionDriver.java:113) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:519) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:360) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:779) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_292]
Caused by: java.io.IOException: Size of the state is larger than the maximum permitted memory-backed state. Size=100091190 , maxSize=5242880 . Consider using a different state backend, like the File System State backend.
	at org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory.checkSize(MemCheckpointStreamFactory.java:61) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory$MemoryCheckpointOutputStream.closeAndGetBytes(MemCheckpointStreamFactory.java:141) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory$MemoryCheckpointOutputStream.closeAndGetHandle(MemCheckpointStreamFactory.java:121) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.state.CheckpointStreamWithResultProvider$PrimaryStreamOnly.closeAndFinalizeCheckpointStreamResult(CheckpointStreamWithResultProvider.java:75) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.state.FullSnapshotAsyncWriter.get(FullSnapshotAsyncWriter.java:87) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.state.SnapshotStrategyRunner$1.callInternal(SnapshotStrategyRunner.java:91) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.state.SnapshotStrategyRunner$1.callInternal(SnapshotStrategyRunner.java:88) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.state.AsyncSnapshotCallable.call(AsyncSnapshotCallable.java:78) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[?:1.8.0_292]
	at org.apache.flink.runtime.concurrent.FutureUtils.runIfNotDoneAndGet(FutureUtils.java:633) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	... 16 more
2021-08-09 15:59:53,314 INFO  org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionFailoverStrategy [] - Calculating tasks to restart to recover the failed task 488bec663292a28f42ac2a625db66756_0.
2021-08-09 15:59:53,315 INFO  org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionFailoverStrategy [] - 25 tasks should be restarted to recover the failed task 488bec663292a28f42ac2a625db66756_0. 
2021-08-09 15:59:53,364 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Job PipelineName (43e80a9e125b45e6be8ac263f1c8fd4e) switched from state RUNNING to FAILING.
org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy
	at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:138) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:82) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:207) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:197) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:188) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:677) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:79) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:435) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at sun.reflect.GeneratedMethodAccessor65.invoke(Unknown Source) ~[?:?]
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_292]
	at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_292]
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:305) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:212) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:77) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:158) ~[flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at scala.PartialFunction.applyOrElse(PartialFunction.scala:123) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.actor.Actor.aroundReceive(Actor.scala:517) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.actor.Actor.aroundReceive$(Actor.scala:515) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.actor.ActorCell.invoke(ActorCell.scala:561) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.dispatch.Mailbox.run(Mailbox.scala:225) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.dispatch.Mailbox.exec(Mailbox.scala:235) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]
	at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) [flink-dist_2.12-1.13.1-stream1.jar:1.13.1-stream1]

Thanks.

On 2021/08/11 02:30:41, "Yun Gao" <yu...@aliyun.com> wrote: 
> Hi Xianwen,
> 
> Could you also attach the full stack of the exception~?
> 
> Very thanks
> 
> 
> ------------------------------------------------------------------
> Sender:xianwen jin<xi...@gmail.com>
> Date:2021/08/10 21:03:44
> Recipient:<us...@flink.apache.org>
> Theme:State Processor API with EmbeddedRocksDBStateBackend
> 
> Hi Flink Community,
> I have an issue regarding create a savepoint with EmbeddedRocksDBStateBackend.
> Since I have to use dataset for state processor api, it seems that the checkpoint storage is always set to jobmanager even though I pass:
> state.checkpoint-storage: filesystem
> state.checkpoints.dir: 's3://flink-bucket01/checkpoints/saldo-es/state'
> in the flink-conf.yaml.
> This causes an exception when the state to be written is more than 5 MB:
> java.io.IOException: Size of the state is larger than the maximum permitted memory-backed state. Size=100091190 , maxSize=5242880 . Consider using a different state backend, like the File System State backend.
> 
> There's a way to set checkpoint storage to filesystem when I'm using state processor api?
> 
> Thanks for any advice or help
> 

Re: State Processor API with EmbeddedRocksDBStateBackend

Posted by Yun Gao <yu...@aliyun.com>.
Hi Xianwen,

Could you also attach the full stack of the exception~?

Very thanks


------------------------------------------------------------------
Sender:xianwen jin<xi...@gmail.com>
Date:2021/08/10 21:03:44
Recipient:<us...@flink.apache.org>
Theme:State Processor API with EmbeddedRocksDBStateBackend

Hi Flink Community,
I have an issue regarding create a savepoint with EmbeddedRocksDBStateBackend.
Since I have to use dataset for state processor api, it seems that the checkpoint storage is always set to jobmanager even though I pass:
state.checkpoint-storage: filesystem
state.checkpoints.dir: 's3://flink-bucket01/checkpoints/saldo-es/state'
in the flink-conf.yaml.
This causes an exception when the state to be written is more than 5 MB:
java.io.IOException: Size of the state is larger than the maximum permitted memory-backed state. Size=100091190 , maxSize=5242880 . Consider using a different state backend, like the File System State backend.

There's a way to set checkpoint storage to filesystem when I'm using state processor api?

Thanks for any advice or help