Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[fix][broker] topic policy deadlock block metadata thread. #23786

Merged
merged 9 commits into from
Dec 30, 2024

Conversation

mattisonchao
Copy link
Member

Motivation

Found one Java-level deadlock:
=============================
"metadata-store-10-1":
  waiting to lock monitor 0x00007fb363ef6a60 (object 0x00000000bf8b3b88, a java.util.concurrent.ConcurrentHashMap$Node),
  which is held by "broker-client-shared-internal-executor-5-1"

"broker-client-shared-internal-executor-5-1":
  waiting to lock monitor 0x00007fb36617a5b0 (object 0x00000000bdf00940, a java.util.concurrent.ConcurrentHashMap$ReservationNode),
  which is held by "pulsar-io-3-6"

"pulsar-io-3-6":
  waiting to lock monitor 0x00007fb363ef6a60 (object 0x00000000bf8b3b88, a java.util.concurrent.ConcurrentHashMap$Node),
  which is held by "broker-client-shared-internal-executor-5-1"

Java stack information for the threads listed above:
===================================================
"metadata-store-10-1":
	at java.util.concurrent.ConcurrentHashMap.compute([email protected]/Unknown Source)
	- waiting to lock <0x00000000bf8b3b88> (a java.util.concurrent.ConcurrentHashMap$Node)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService.lambda$getTopicPoliciesAsync$10(SystemTopicBasedTopicPoliciesService.java:271)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService$$Lambda/0x0000000100bbdac0.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniAcceptNow([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.uniAcceptStage([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.thenAccept([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService.getTopicPoliciesAsync(SystemTopicBasedTopicPoliciesService.java:271)
	at org.apache.pulsar.broker.service.BrokerService.getTopicPoliciesBypassSystemTopic(BrokerService.java:1186)
	at org.apache.pulsar.broker.service.BrokerService.lambda$getTopic$37(BrokerService.java:1080)
	at org.apache.pulsar.broker.service.BrokerService$$Lambda/0x00000001007af710.apply(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniComposeStage([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.thenCompose([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.BrokerService.getTopic(BrokerService.java:1076)
	at org.apache.pulsar.broker.service.BrokerService.getTopic(BrokerService.java:1040)
	at org.apache.pulsar.broker.service.BrokerService.getTopic(BrokerService.java:1035)
	at org.apache.pulsar.broker.service.ServerCnx.lambda$handleSubscribe$18(ServerCnx.java:1331)
	at org.apache.pulsar.broker.service.ServerCnx$$Lambda/0x0000000100ba4230.apply(Unknown Source)
	at java.util.concurrent.CompletableFuture$UniCompose.tryFire([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.postComplete([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.complete([email protected]/Unknown Source)
	at org.apache.pulsar.metadata.impl.ZKMetadataStore.handleGetResult(ZKMetadataStore.java:289)
	at org.apache.pulsar.metadata.impl.ZKMetadataStore.lambda$batchOperation$7(ZKMetadataStore.java:233)
	at org.apache.pulsar.metadata.impl.ZKMetadataStore$$Lambda/0x00000001004c1740.run(Unknown Source)
	at java.util.concurrent.Executors$RunnableAdapter.call([email protected]/Unknown Source)
	at java.util.concurrent.FutureTask.run([email protected]/Unknown Source)
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run([email protected]/Unknown Source)
	at java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/Unknown Source)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/Unknown Source)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.runWith([email protected]/Unknown Source)
	at java.lang.Thread.run([email protected]/Unknown Source)
"broker-client-shared-internal-executor-5-1":
	at java.util.concurrent.ConcurrentHashMap.transfer([email protected]/Unknown Source)
	- waiting to lock <0x00000000bdf00940> (a java.util.concurrent.ConcurrentHashMap$ReservationNode)
	at java.util.concurrent.ConcurrentHashMap.addCount([email protected]/Unknown Source)
	at java.util.concurrent.ConcurrentHashMap.computeIfAbsent([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.BrokerService.lambda$getTopic$36(BrokerService.java:1110)
	at org.apache.pulsar.broker.service.BrokerService$$Lambda/0x00000001007ada38.apply(Unknown Source)
	at java.util.concurrent.CompletableFuture$UniCompose.tryFire([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.postComplete([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.complete([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService.lambda$getTopicPoliciesAsync$9(SystemTopicBasedTopicPoliciesService.java:280)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService$$Lambda/0x0000000100c36818.apply(Unknown Source)
	at java.util.concurrent.ConcurrentHashMap.compute([email protected]/Unknown Source)
	- locked <0x00000000bf8b3b88> (a java.util.concurrent.ConcurrentHashMap$Node)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService.lambda$getTopicPoliciesAsync$10(SystemTopicBasedTopicPoliciesService.java:271)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService$$Lambda/0x0000000100bbdac0.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture$UniAccept.tryFire([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.postComplete([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.complete([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService.lambda$initPolicesCache$21(SystemTopicBasedTopicPoliciesService.java:463)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService$$Lambda/0x0000000100c2be30.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniWhenComplete([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.postComplete([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.complete([email protected]/Unknown Source)
	at org.apache.pulsar.client.impl.MultiTopicsConsumerImpl.lambda$hasMessageAvailableAsync$37(MultiTopicsConsumerImpl.java:879)
	at org.apache.pulsar.client.impl.MultiTopicsConsumerImpl$$Lambda/0x0000000100c2b980.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniWhenComplete([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.postComplete([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.complete([email protected]/Unknown Source)
	at org.apache.pulsar.client.impl.ConsumerImpl.lambda$completehasMessageAvailableWithValue$65(ConsumerImpl.java:2574)
	at org.apache.pulsar.client.impl.ConsumerImpl$$Lambda/0x0000000100c35920.run(Unknown Source)
	at java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/Unknown Source)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/Unknown Source)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.runWith([email protected]/Unknown Source)
	at java.lang.Thread.run([email protected]/Unknown Source)
"pulsar-io-3-6":
	at java.util.concurrent.ConcurrentHashMap.compute([email protected]/Unknown Source)
	- waiting to lock <0x00000000bf8b3b88> (a java.util.concurrent.ConcurrentHashMap$Node)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService.lambda$getTopicPoliciesAsync$10(SystemTopicBasedTopicPoliciesService.java:271)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService$$Lambda/0x0000000100bbdac0.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniAcceptNow([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.uniAcceptStage([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.thenAccept([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService.getTopicPoliciesAsync(SystemTopicBasedTopicPoliciesService.java:271)
	at org.apache.pulsar.broker.service.BrokerService.getTopicPoliciesBypassSystemTopic(BrokerService.java:1186)
	at org.apache.pulsar.broker.service.BrokerService.getManagedLedgerConfig(BrokerService.java:1906)
	at org.apache.pulsar.broker.service.BrokerService.getManagedLedgerFactoryForTopic(BrokerService.java:1273)
	at org.apache.pulsar.broker.service.BrokerService.fetchTopicPropertiesAsync(BrokerService.java:1658)
	at org.apache.pulsar.broker.service.BrokerService.lambda$checkOwnershipAndCreatePersistentTopic$76(BrokerService.java:1694)
	at org.apache.pulsar.broker.service.BrokerService$$Lambda/0x00000001007b3220.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniAcceptNow([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.uniAcceptStage([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.thenAccept([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.BrokerService.checkOwnershipAndCreatePersistentTopic(BrokerService.java:1689)
	at org.apache.pulsar.broker.service.BrokerService.lambda$loadOrCreatePersistentTopic$66(BrokerService.java:1624)
	at org.apache.pulsar.broker.service.BrokerService$$Lambda/0x00000001007b2ba0.run(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniRunNow([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.uniRunStage([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.thenRun([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.BrokerService.loadOrCreatePersistentTopic(BrokerService.java:1620)
	at org.apache.pulsar.broker.service.BrokerService.lambda$getTopic$35(BrokerService.java:1111)
	at org.apache.pulsar.broker.service.BrokerService$$Lambda/0x0000000100c36e50.apply(Unknown Source)
	at java.util.concurrent.ConcurrentHashMap.computeIfAbsent([email protected]/Unknown Source)
	- locked <0x00000000bdf00940> (a java.util.concurrent.ConcurrentHashMap$ReservationNode)
	at org.apache.pulsar.broker.service.BrokerService.lambda$getTopic$36(BrokerService.java:1110)
	at org.apache.pulsar.broker.service.BrokerService$$Lambda/0x00000001007ada38.apply(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniComposeStage([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.thenCompose([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.BrokerService.lambda$getTopic$37(BrokerService.java:1087)
	at org.apache.pulsar.broker.service.BrokerService$$Lambda/0x00000001007af710.apply(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniComposeStage([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.thenCompose([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.BrokerService.getTopic(BrokerService.java:1076)
	at org.apache.pulsar.broker.service.BrokerService.getTopic(BrokerService.java:1040)
	at org.apache.pulsar.broker.service.BrokerService.getTopic(BrokerService.java:1035)
	at org.apache.pulsar.broker.service.ServerCnx.lambda$handleSubscribe$18(ServerCnx.java:1331)
	at org.apache.pulsar.broker.service.ServerCnx$$Lambda/0x0000000100ba4230.apply(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniComposeStage([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.thenCompose([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.ServerCnx.lambda$handleSubscribe$24(ServerCnx.java:1330)
	at org.apache.pulsar.broker.service.ServerCnx$$Lambda/0x0000000100b9ec00.apply(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniApplyNow([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.uniApplyStage([email protected]/Unknown Source)
	at java.util.concurrent.CompletableFuture.thenApply([email protected]/Unknown Source)
	at org.apache.pulsar.broker.service.ServerCnx.handleSubscribe(ServerCnx.java:1271)
	at org.apache.pulsar.common.protocol.PulsarDecoder.channelRead(PulsarDecoder.java:243)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
	at io.netty.handler.flow.FlowControlHandler.dequeue(FlowControlHandler.java:202)
	at io.netty.handler.flow.FlowControlHandler.channelRead(FlowControlHandler.java:164)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
	at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:346)
	at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:333)
	at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:455)
	at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:290)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
	at io.netty.handler.flush.FlushConsolidationHandler.channelRead(FlushConsolidationHandler.java:152)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
	at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1357)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
	at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:868)
	at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166)
	at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:788)
	at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
	at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
	at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.runWith([email protected]/Unknown Source)
	at java.lang.Thread.run([email protected]/Unknown Source)

Modifications

  • Move the callback logic out of lock scope.

Verifying this change

  • Make sure that the change passes the CI checks.

Documentation

  • doc-not-needed

@mattisonchao mattisonchao self-assigned this Dec 26, 2024
@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Dec 26, 2024
@mattisonchao mattisonchao requested a review from coderzc December 26, 2024 09:09
@mattisonchao mattisonchao reopened this Dec 26, 2024
@BewareMyPower BewareMyPower added release/3.3.4 release/4.0.2 type/bug The PR fixed a bug or issue reported a bug labels Dec 26, 2024
Copy link
Contributor

@BewareMyPower BewareMyPower left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In short, I don't think this change fixes the deadlock.

Deadlock analysis

In thread A (broker-client-shared-internal-executor-5-1):

return topics.computeIfAbsent(topicName.toString(), (tpName) ->
        loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies));

It already holds the lock of SystemTopicBasedTopicPoliciesService#policyCacheInitMap (via policyCacheInitMap.compute method) and tries to acquire the lock of BrokerService#topics.

Thread B (pulsar-io-3-6) is blocked at the same line, while it already holds the lock of BrokerService#topics and it goes into the loadOrCreatePersistentTopic method. However, it eventually calls SystemTopicBasedTopicPoliciesService.getTopicPoliciesAsync and tries to acquire the lock of SystemTopicBasedTopicPoliciesService#policyCacheInitMap.

Thread C (metadata-store-10-1) is just blocked at acquiring the lock of policyCacheInitMap like what thread B is blocked. However, the root cause is the deadlock between thread A and B.

Why I think the changes do not work

You've made the following changes:

  1. Move the policyCacheInitMap.compute call from thenAccept to thenComposeAsync.
  2. Don't call whenComplete in the thenComposeAsync method.

The 1st change just moves the thread C from the metadata store thread pool to the CompletableFuture#ASYNC_POOL thread pool. It doesn't affect the deadlock between thread A and B. The 2nd change actually does not make a difference because in the existing code, it's already called in the thread pool of CompletableFuture#ASYNC_POOL via CompletableFuture.runAsync.

@BewareMyPower
Copy link
Contributor

BewareMyPower commented Dec 26, 2024

The pulsar-io-3-6 thread that holds the lock of BrokerService#topics is blocked at:

    public CompletableFuture<ManagedLedgerFactory> getManagedLedgerFactoryForTopic(TopicName topicName) {
        // NOTE: it calls `getTopicPoliciesBypassSystemTopic` and tries to acquire the lock of
        //.  `SystemTopicBasedTopicPoliciesService#policyCacheInitMap`
        return getManagedLedgerConfig(topicName)

called by

    protected CompletableFuture<Map<String, String>> fetchTopicPropertiesAsync(TopicName topicName) {
        if (!topicName.isPartitioned()) {
            return getManagedLedgerFactoryForTopic(topicName).thenCompose(

called by

    private void checkOwnershipAndCreatePersistentTopic(/* ... */) {
        TopicName topicName = TopicName.get(topic);
        pulsar.getNamespaceService().isServiceUnitActiveAsync(topicName)
                .thenAccept(isActive -> {
                    if (isActive) {
                        CompletableFuture<Map<String, String>> propertiesFuture;
                        if (properties == null) {
                            //Read properties from storage when loading topic.
                            propertiesFuture = fetchTopicPropertiesAsync(topicName); // HERE

called by

    protected CompletableFuture<Optional<Topic>> loadOrCreatePersistentTopic(final String topic,
            boolean createIfMissing, Map<String, String> properties, @Nullable TopicPolicies topicPolicies) {
        /* ... */
        checkTopicNsOwnership(topic)
                .thenRun(() -> {
                    final Semaphore topicLoadSemaphore = topicLoadRequestSemaphore.get();

                    if (topicLoadSemaphore.tryAcquire()) {
                        checkOwnershipAndCreatePersistentTopic(topic, createIfMissing, topicFuture, // HERE
                                properties, topicPolicies);

Here checkTopicNsOwnership and isServiceUnitActiveAsync futures all completed immediately in the current thread so the callback of topics.computeIfAbsent is blocked at acquiring the lock of SystemTopicBasedTopicPoliciesService#policyCacheInitMap.

I think the correct fix is to move checkTopicNsOwnership(topic).thenRun to checkTopicNsOwnership(topic).thenRunAsync (or isServiceUnitActiveAsync(topicName).thenAcceptAsync) so that the callback of topics.computeIfAbsent won't be blocked.

@codecov-commenter
Copy link

codecov-commenter commented Dec 26, 2024

Codecov Report

All modified and coverable lines are covered by tests ✅

Project coverage is 74.16%. Comparing base (bbc6224) to head (646772a).
Report is 816 commits behind head on master.

Additional details and impacted files

Impacted file tree graph

@@             Coverage Diff              @@
##             master   #23786      +/-   ##
============================================
+ Coverage     73.57%   74.16%   +0.59%     
+ Complexity    32624    31764     -860     
============================================
  Files          1877     1853      -24     
  Lines        139502   143369    +3867     
  Branches      15299    16277     +978     
============================================
+ Hits         102638   106331    +3693     
+ Misses        28908    28669     -239     
- Partials       7956     8369     +413     
Flag Coverage Δ
inttests 26.66% <63.15%> (+2.08%) ⬆️
systests 23.19% <63.15%> (-1.14%) ⬇️
unittests 73.68% <100.00%> (+0.83%) ⬆️

Flags with carried forward coverage won't be shown. Click here to find out more.

Files with missing lines Coverage Δ
.../service/SystemTopicBasedTopicPoliciesService.java 75.12% <100.00%> (+0.93%) ⬆️

... and 1024 files with indirect coverage changes

Copy link
Contributor

@BewareMyPower BewareMyPower left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The deadlock is caused by locks from two concurrent hash maps (policyCacheInitMap and BrokerService#topics), so fixing one (policyCacheInitMap) of them in this PR is okay for me.

@mattisonchao mattisonchao merged commit 86f8a84 into apache:master Dec 30, 2024
52 checks passed
@lhotari
Copy link
Member

lhotari commented Jan 2, 2025

@mattisonchao Please create a separate PR for cherry-picking to branch-3.3 since there are major merge conflicts. The merge conflicts seem to be mainly due to lack of #23319 changes in branch-3.3 .

lhotari pushed a commit that referenced this pull request Jan 2, 2025
hanmz pushed a commit to hanmz/pulsar that referenced this pull request Feb 12, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
cherry-picked/branch-4.0 doc-not-needed Your PR changes do not impact docs ready-to-test release/4.0.2 type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants