michaeljmarshall commented on issue #16979: URL: https://github.com/apache/pulsar/issues/16979#issuecomment-1213756898
It's clear to me now that the issue is a data race with the `entries` array. After https://github.com/apache/pulsar/pull/16603, entries are published to another thread. I believe they are "safely published" because the calling thread is within a `synchronized` block and the receiving thread passes the `entries` array to a `synchronized` method: https://github.com/apache/pulsar/blob/abff91f881d2677ecf76943c2bfb4f9983fe50b6/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java#L554-L555 The fundamental race comes from the caching. When an entry is expired from the cache, we call `release()`, which deallocates the object. Here is another stack trace I found from running the test. It looks like we already found similar stack traces here https://github.com/apache/pulsar/issues/14436. My analysis aligns with @lhotari's. I'll have a PR up soon. ``` 2022-08-12T23:52:24,841 - ERROR - [broker-topic-workers-OrderedExecutor-3-0:Commands@1859] - [PersistentSubscription{topic=persistent://my-property/my-ns/cache-read, name=sub-1}] [-1] Failed to parse message metadata java.lang.IllegalArgumentException: Invalid unknonwn tag type: 7 at org.apache.pulsar.common.api.proto.LightProtoCodec.skipUnknownField(LightProtoCodec.java:270) ~[classes/:?] at org.apache.pulsar.common.api.proto.MessageMetadata.parseFrom(MessageMetadata.java:1370) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.parseMessageMetadata(Commands.java:445) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.parseMessageMetadata(Commands.java:432) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.peekMessageMetadata(Commands.java:1854) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.peekAndCopyMessageMetadata(Commands.java:1873) ~[classes/:?] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.trySendMessagesToConsumers(PersistentDispatcherMultipleConsumers.java:599) ~[classes/:?] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.java:573) ~[classes/:?] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.lambda$readEntriesComplete$5(PersistentDispatcherMultipleConsumers.java:556) ~[classes/:?] at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[classes/:?] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) ~[bookkeeper-common-4.15.0.jar:4.15.0] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) ~[?:?] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) ~[?:?] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[netty-common-4.1.77.Final.jar:4.1.77.Final] at java.lang.Thread.run(Thread.java:833) ~[?:?] 2022-08-12T23:52:24,874 - ERROR - [broker-topic-workers-OrderedExecutor-3-0:Commands@1859] - [PersistentSubscription{topic=persistent://my-property/my-ns/cache-read, name=sub-1}] [-1] Failed to parse message metadata java.lang.IndexOutOfBoundsException: readerIndex(44) + length(1) exceeds writerIndex(44): UnpooledDuplicatedByteBuf(ridx: 44, widx: 44, cap: 44/44, unwrapped: UnpooledHeapByteBuf(ridx: 0, widx: 44, cap: 44/44)) at io.netty.buffer.AbstractByteBuf.checkReadableBytes0(AbstractByteBuf.java:1442) ~[netty-buffer-4.1.77.Final.jar:4.1.77.Final] at io.netty.buffer.AbstractByteBuf.readByte(AbstractByteBuf.java:730) ~[netty-buffer-4.1.77.Final.jar:4.1.77.Final] at org.apache.pulsar.common.api.proto.LightProtoCodec.readVarInt(LightProtoCodec.java:140) ~[classes/:?] at org.apache.pulsar.common.api.proto.MessageMetadata.parseFrom(MessageMetadata.java:1234) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.parseMessageMetadata(Commands.java:445) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.parseMessageMetadata(Commands.java:432) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.peekMessageMetadata(Commands.java:1854) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.peekAndCopyMessageMetadata(Commands.java:1873) ~[classes/:?] at org.apache.pulsar.broker.service.AbstractBaseDispatcher.lambda$filterEntriesForConsumer$1(AbstractBaseDispatcher.java:117) ~[classes/:?] at java.util.Optional.orElseGet(Optional.java:364) ~[?:?] at org.apache.pulsar.broker.service.AbstractBaseDispatcher.filterEntriesForConsumer(AbstractBaseDispatcher.java:115) ~[classes/:?] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.trySendMessagesToConsumers(PersistentDispatcherMultipleConsumers.java:669) ~[classes/:?] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.java:573) ~[classes/:?] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.lambda$readEntriesComplete$5(PersistentDispatcherMultipleConsumers.java:556) ~[classes/:?] at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[classes/:?] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) ~[bookkeeper-common-4.15.0.jar:4.15.0] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) ~[?:?] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) ~[?:?] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[netty-common-4.1.77.Final.jar:4.1.77.Final] at java.lang.Thread.run(Thread.java:833) ~[?:?] 2022-08-12T23:52:24,865 - ERROR - [broker-topic-workers-OrderedExecutor-2-0:Commands@1859] - [PersistentSubscription{topic=persistent://my-property/my-ns/cache-read, name=sub-0}] [-1] Failed to parse message metadata java.lang.IllegalStateException: Some required fields are missing at org.apache.pulsar.common.api.proto.MessageMetadata.checkRequiredFields(MessageMetadata.java:1378) ~[classes/:?] at org.apache.pulsar.common.api.proto.MessageMetadata.parseFrom(MessageMetadata.java:1373) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.parseMessageMetadata(Commands.java:445) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.parseMessageMetadata(Commands.java:432) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.peekMessageMetadata(Commands.java:1854) ~[classes/:?] at org.apache.pulsar.common.protocol.Commands.peekAndCopyMessageMetadata(Commands.java:1873) ~[classes/:?] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.trySendMessagesToConsumers(PersistentDispatcherMultipleConsumers.java:599) ~[classes/:?] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.java:573) ~[classes/:?] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.lambda$readEntriesComplete$5(PersistentDispatcherMultipleConsumers.java:556) ~[classes/:?] at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[classes/:?] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) ~[bookkeeper-common-4.15.0.jar:4.15.0] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) ~[?:?] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) ~[?:?] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[netty-common-4.1.77.Final.jar:4.1.77.Final] at java.lang.Thread.run(Thread.java:833) ~[?:?] ``` -- 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]
