[cherry-pick][branch-2.10] Fix deadlock causes session notification not to work (#19754) (#19768)

Co-authored-by: Qiang Zhao <mattisonchao@apache.org>
Master https://github.com/apache/pulsar/pull/19754
### Motivation

This is a namespace bundle double-owners problem. We found it in the memory dumps.

The memory dumps show that the notification thread has been blocked for a long time by the leader election deadlock, And many notifications are blocked in the executor queue. This causes we can't to revalidate the locks, and they are still thinking them working well.

For private reasons, I can't share the namespace bundle snapshot, but the blocked thread easily explains it.

<img width="1061" alt="image" src="https://user-images.githubusercontent.com/74767115/223670419-c4319f44-f1e1-4361-8c79-04c7f0dabe3c.png"> ^^ blocked thread

<img width="949" alt="image" src="https://user-images.githubusercontent.com/74767115/223672000-fb4ce22c-6a45-4cb6-9d23-c36e3afbc93a.png">

^^ executor queue

### Modifications

- Avoid putting the new task to single thread executor causes deadlock.

(cherry picked from commit cbd799f05eb26aeac5ffd5bbb9751ad9e5928dd3)
diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java
index 86fac33..8e1d2cd 100644
--- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java
+++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java
@@ -19,6 +19,7 @@
 package org.apache.pulsar.metadata.coordination.impl;
 
 import com.fasterxml.jackson.databind.type.TypeFactory;
+import com.google.common.annotations.VisibleForTesting;
 import java.util.EnumSet;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -111,13 +112,13 @@
             } else {
                 return tryToBecomeLeader();
             }
-        }).thenComposeAsync(leaderElectionState -> {
+        }).thenCompose(leaderElectionState -> {
             // make sure that the cache contains the current leader
             // so that getLeaderValueIfPresent works on all brokers
             cache.refresh(path);
             return cache.get(path)
                     .thenApply(__ -> leaderElectionState);
-        }, executor);
+        });
     }
 
     private synchronized CompletableFuture<LeaderElectionState> handleExistingLeaderValue(GetResult res) {
@@ -336,4 +337,9 @@
             }
         }
     }
+
+    @VisibleForTesting
+    protected ScheduledExecutorService getSchedulerExecutor() {
+        return executor;
+    }
 }
diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImplTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImplTest.java
new file mode 100644
index 0000000..8f70ab5
--- /dev/null
+++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImplTest.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.metadata.coordination.impl;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
+import lombok.Cleanup;
+import org.apache.pulsar.metadata.BaseMetadataStoreTest;
+import org.apache.pulsar.metadata.api.MetadataStoreConfig;
+import org.apache.pulsar.metadata.api.coordination.CoordinationService;
+import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
+import org.testng.annotations.Test;
+
+public class LeaderElectionImplTest extends BaseMetadataStoreTest {
+
+    @Test(dataProvider = "impl", timeOut = 10000)
+    public void validateDeadLock(String provider, Supplier<String> urlSupplier)
+            throws Exception {
+        if (provider.equals("Memory") || provider.equals("RocksDB")) {
+            // There are no multiple sessions for the local memory provider
+            return;
+        }
+
+        @Cleanup
+        MetadataStoreExtended store = MetadataStoreExtended.create(urlSupplier.get(),
+                MetadataStoreConfig.builder().build());
+
+        String path = newKey();
+
+        @Cleanup
+        CoordinationService cs = new CoordinationServiceImpl(store);
+
+        @Cleanup
+        LeaderElectionImpl<String> le = (LeaderElectionImpl<String>) cs.getLeaderElection(String.class,
+                path, __ -> {
+                });
+        final CompletableFuture<Void> blockFuture = new CompletableFuture<>();
+        // simulate handleSessionNotification method logic
+        le.getSchedulerExecutor().execute(() -> {
+            try {
+                le.elect("test-2").join();
+                blockFuture.complete(null);
+            } catch (Throwable ex) {
+                blockFuture.completeExceptionally(ex);
+            }
+        });
+        blockFuture.join();
+    }
+}