LiebingYu opened a new issue, #2318:
URL: https://github.com/apache/fluss/issues/2318

   ### Search before asking
   
   - [x] I searched in the [issues](https://github.com/apache/fluss/issues) and 
found nothing similar.
   
   
   ### Fluss version
   
   main (development)
   
   ### Please describe the bug 🐞
   
   When I run a Flink job to sink data to Fluss, I found the following error:
   ```java
   java.util.concurrent.CompletionException: 
org.apache.fluss.exception.UnknownServerException: 
java.util.ConcurrentModificationException
        at java.base/java.util.HashMap$HashIterator.remove(HashMap.java:1611)
        at 
org.apache.fluss.server.coordinator.MetadataManager.removeSensitiveTableOptions(MetadataManager.java:598)
        at 
org.apache.fluss.server.coordinator.MetadataManager.getTable(MetadataManager.java:618)
        at 
org.apache.fluss.server.RpcServiceBase.getTableInfo(RpcServiceBase.java:269)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
        at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at 
org.apache.fluss.rpc.netty.server.FlussRequestHandler.processRequest(FlussRequestHandler.java:63)
        at 
org.apache.fluss.rpc.netty.server.FlussRequestHandler.processRequest(FlussRequestHandler.java:34)
        at 
org.apache.fluss.rpc.netty.server.RequestProcessor.processRequest(RequestProcessor.java:98)
        at 
org.apache.fluss.rpc.netty.server.RequestProcessor.run(RequestProcessor.java:70)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
        at java.base/java.lang.Thread.run(Thread.java:833)
   
        at 
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
        at 
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
        at 
java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:607)
        at 
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591)
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at 
java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1990)
        at 
org.apache.fluss.rpc.netty.client.ServerConnection$ResponseCallback.onRequestFailure(ServerConnection.java:239)
        at 
org.apache.fluss.rpc.netty.client.NettyClientHandler.channelRead(NettyClientHandler.java:135)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at 
org.apache.fluss.shaded.netty4.io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:286)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at 
org.apache.fluss.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:346)
        at 
org.apache.fluss.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:318)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:800)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:509)
        at 
org.apache.fluss.shaded.netty4.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
        at 
org.apache.fluss.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at 
org.apache.fluss.shaded.netty4.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at 
org.apache.fluss.shaded.netty4.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:879)
   Caused by: org.apache.fluss.exception.UnknownServerException: 
java.util.ConcurrentModificationException
        at java.base/java.util.HashMap$HashIterator.remove(HashMap.java:1611)
        at 
org.apache.fluss.server.coordinator.MetadataManager.removeSensitiveTableOptions(MetadataManager.java:598)
        at 
org.apache.fluss.server.coordinator.MetadataManager.getTable(MetadataManager.java:618)
        at 
org.apache.fluss.server.RpcServiceBase.getTableInfo(RpcServiceBase.java:269)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
        at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at 
org.apache.fluss.rpc.netty.server.FlussRequestHandler.processRequest(FlussRequestHandler.java:63)
        at 
org.apache.fluss.rpc.netty.server.FlussRequestHandler.processRequest(FlussRequestHandler.java:34)
        at 
org.apache.fluss.rpc.netty.server.RequestProcessor.processRequest(RequestProcessor.java:98)
        at 
org.apache.fluss.rpc.netty.server.RequestProcessor.run(RequestProcessor.java:70)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
        at java.base/java.lang.Thread.run(Thread.java:833)
   ```
   
   This error was introduced by https://github.com/apache/fluss/pull/1860. In 
`removeSensitiveTableOptions`, it modifies a shared map. Once `getTable` is 
accessed concurrently, a `ConcurrentModificationException` will be thrown.
   
   ### Solution
   
   _No response_
   
   ### Are you willing to submit a PR?
   
   - [x] I'm willing to submit a PR!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to