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

Kyle commented on FLINK-25670:
------------------------------

When I set payload_max_bytes to 1GB,
{code:java}
    kind: io.statefun.endpoints.v2/http
    spec:
      functions: example/*
      urlPathTemplate: http://functions.statefun.svc.cluster.local:8000/statefun
      transport:
        type: io.statefun.transports.v1/async
        payload_max_bytes: 1073741824
        timeouts:
          call: 2min {code}
Another error happens:
{code:java}
2022-01-23 02:39:16,469 INFO  
org.apache.flink.statefun.flink.core.nettyclient.NettyRequest [] - Retry #3 
ToFunctionRequestSummary(address=Address(example, hello, 1111), batchSize=1, 
totalSizeInBytes=80, numberOfStates=2) ,About to sleep for 16
2022-01-23 02:39:24,687 WARN  
org.apache.flink.statefun.flink.core.nettyclient.NettyRequest [] - Exception 
caught while trying to deliver a message: (attempt 
#3)ToFunctionRequestSummary(address=Address(example, hello, 1111), batchSize=1, 
totalSizeInBytes=80, numberOfStates=2)
java.lang.OutOfMemoryError: Direct buffer memory
        at java.nio.Bits.reserveMemory(Unknown Source) ~[?:?]
        at java.nio.DirectByteBuffer.<init>(Unknown Source) ~[?:?]
        at java.nio.ByteBuffer.allocateDirect(Unknown Source) ~[?:?]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:755)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.PoolArena$DirectArena.newUnpooledChunk(PoolArena.java:745)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.PoolArena.allocateHuge(PoolArena.java:262)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.PoolArena.allocate(PoolArena.java:232)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.PoolArena.allocate(PoolArena.java:147)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:356)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:187)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:178)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.CompositeByteBuf.allocBuffer(CompositeByteBuf.java:1853)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.CompositeByteBuf.consolidate0(CompositeByteBuf.java:1732)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.CompositeByteBuf.consolidateIfNeeded(CompositeByteBuf.java:559)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.CompositeByteBuf.addComponent(CompositeByteBuf.java:266)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.buffer.CompositeByteBuf.addComponent(CompositeByteBuf.java:222)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageAggregator.appendPartialContent(MessageAggregator.java:333)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageAggregator.decode(MessageAggregator.java:298)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)
 ~[statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.CombinedChannelDuplexHandler$DelegatingChannelHandlerContext.fireChannelRead(CombinedChannelDuplexHandler.java:436)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:324)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:311)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:425)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:276)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.CombinedChannelDuplexHandler.channelRead(CombinedChannelDuplexHandler.java:251)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:792)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe$1.run(AbstractEpollChannel.java:387)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:472)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:384)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
 [statefun-flink-distribution.jar:3.1.1]
        at 
org.apache.flink.shaded.netty4.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
 [statefun-flink-distribution.jar:3.1.1]
        at java.lang.Thread.run(Unknown Source) [?:?]
 {code}
I tried to change the reserved memory of taskmanager in flink-conf.yaml in the 
config:
{code:java}
    jobmanager.memory.process.size: 1g
    taskmanager.memory.process.size: 4g
 {code}
However it doesn't seem to take effect. The error still happens.
{code:java}
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -  Starting 
TaskManager (Version: 1.13.5, Scala: 2.12, Rev:0ff28a7, 
Date:2021-12-14T23:26:04+01:00)
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -  OS current 
user: root
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -  Current 
Hadoop/Kerberos user: <no hadoop dependency found>
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -  JVM: OpenJDK 
64-Bit Server VM - Oracle Corporation - 11/11.0.13+8
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -  Maximum heap 
size: 1588 MiBytes
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -  JAVA_HOME: 
/usr/local/openjdk-11
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -  No Hadoop 
Dependency available
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -  JVM Options:
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
-XX:+UseG1GC
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
-Xmx1664299798
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
-Xms1664299798
2022-01-23 02:57:31,346 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
-XX:MaxDirectMemorySize=493921243
2022-01-23 02:57:31,347 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
-XX:MaxMetaspaceSize=268435456
2022-01-23 02:57:31,347 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
-Dlog.file=/opt/flink/log/flink--taskexecutor-0-statefun-worker-f9cd87c8b-kctkv.log
2022-01-23 02:57:31,347 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
-Dlog4j.configuration=file:/opt/flink/conf/log4j-console.properties
2022-01-23 02:57:31,347 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
-Dlog4j.configurationFile=file:/opt/flink/conf/log4j-console.properties
2022-01-23 02:57:31,347 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
-Dlogback.configurationFile=file:/opt/flink/conf/logback-console.xml
2022-01-23 02:57:31,347 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -  Program 
Arguments:
2022-01-23 02:57:31,348 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
--configDir
2022-01-23 02:57:31,348 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
/opt/flink/conf
2022-01-23 02:57:31,348 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,348 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.memory.network.min=359703515b
2022-01-23 02:57:31,348 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,348 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.cpu.cores=1.0
2022-01-23 02:57:31,348 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,348 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.memory.task.off-heap.size=0b
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.memory.jvm-metaspace.size=268435456b
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
external-resources=none
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.memory.jvm-overhead.min=429496736b
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.memory.framework.off-heap.size=134217728b
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.memory.network.max=359703515b
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.memory.framework.heap.size=134217728b
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.memory.managed.size=1438814063b
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.memory.task.heap.size=1530082070b
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.numberOfTaskSlots=1
2022-01-23 02:57:31,349 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     -D
2022-01-23 02:57:31,350 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
taskmanager.memory.jvm-overhead.max=429496736b
2022-01-23 02:57:31,350 INFO  
org.apache.flink.runtime.taskexecutor.TaskManagerRunner      [] -     
-Djobmanager.rpc.address=statefun-master
 {code}
How can I enlarge the reserved memory for taskmanager?

> StateFun: Unable to handle oversize HTTP message if state size is large
> -----------------------------------------------------------------------
>
>                 Key: FLINK-25670
>                 URL: https://issues.apache.org/jira/browse/FLINK-25670
>             Project: Flink
>          Issue Type: Bug
>          Components: Stateful Functions
>    Affects Versions: statefun-3.1.1
>            Reporter: Kyle
>            Priority: Major
>         Attachments: 00-module.yaml, functions.py
>
>
> Per requirement we need to handle state which is about 500MB large (72MB 
> state allocated in commented code as attached). However the HTTP message 
> limit disallows us to send back large state to StateFun cluster after saving 
> state in Stateful Function.
> Another question is whether large data is allowed to send to Stateful 
> Function from ingress.
>  
> 2022-01-17 07:57:18,416 WARN  
> org.apache.flink.statefun.flink.core.nettyclient.NettyRequest [] - Exception 
> caught while trying to deliver a message: (attempt 
> #10)ToFunctionRequestSummary(address=Address(example, hello, 5555), 
> batchSize=1, totalSizeInBytes=80, numberOfStates=2)
> org.apache.flink.shaded.netty4.io.netty.handler.codec.TooLongFrameException: 
> Response entity too large: DefaultHttpResponse(decodeResult: success, 
> version: HTTP/1.1)
> HTTP/1.1 200 OK
> Content-Type: application/octet-stream
> Content-Length: 40579630
> Date: Mon, 17 Jan 2022 07:57:18 GMT
> Server: Python/3.9 aiohttp/3.8.1
>         at 
> org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObjectAggregator.handleOversizedMessage(HttpObjectAggregator.java:276)
>  ~[statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObjectAggregator.handleOversizedMessage(HttpObjectAggregator.java:87)
>  ~[statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageAggregator.invokeHandleOversizedMessage(MessageAggregator.java:404)
>  ~[statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageAggregator.decode(MessageAggregator.java:254)
>  ~[statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)
>  ~[statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.CombinedChannelDuplexHandler$DelegatingChannelHandlerContext.fireChannelRead(CombinedChannelDuplexHandler.java:436)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:324)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:311)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:425)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:276)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.CombinedChannelDuplexHandler.channelRead(CombinedChannelDuplexHandler.java:251)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:792)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:475)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:378)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
>  [statefun-flink-distribution.jar:3.1.1]
>         at 
> org.apache.flink.shaded.netty4.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
>  [statefun-flink-distribution.jar:3.1.1]
>         at java.lang.Thread.run(Unknown Source) [?:?]



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to