You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/05/17 12:59:10 UTC

[GitHub] [pulsar] nicoloboschi opened a new pull request, #15638: [fix][tests] Ensure thread pools are disposed after some tests

nicoloboschi opened a new pull request, #15638:
URL: https://github.com/apache/pulsar/pull/15638

   ### Motivation
   
   There are some tests that occasionally fails with non-sense mockito errors like this one:
   ```
   org.apache.pulsar.broker.PulsarServerException: 
   org.mockito.exceptions.misusing.WrongTypeOfReturnValue: 
   BrokerService cannot be returned by getConfiguration()
   getConfiguration() should return ServiceConfiguration
   ***
   If you're unsure why you're getting above error read on.
   Due to the nature of the syntax above problem might occur because:
   1. This exception *might* occur in wrongly written multi-threaded tests.
      Please refer to Mockito FAQ on limitations of concurrency testing.
   2. A spy is stubbed using when(spy.foo()).then() syntax. It is safer to stub spies - 
      - with doReturn|Throw() family of methods. More in javadocs for Mockito.spy() method.
   
   	at org.apache.pulsar.broker.service.BrokerService.close(BrokerService.java:685)
   	at java.base/java.lang.invoke.MethodHandle.invokeWithArguments(MethodHandle.java:710)
   	at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:418)
   Caused by: org.mockito.exceptions.misusing.WrongTypeOfReturnValue: 
   BrokerService cannot be returned by getConfiguration()
   getConfiguration() should return ServiceConfiguration
   ***
   If you're unsure why you're getting above error read on.
   Due to the nature of the syntax above problem might occur because:
   1. This exception *might* occur in wrongly written multi-threaded tests.
      Please refer to Mockito FAQ on limitations of concurrency testing.
   2. A spy is stubbed using when(spy.foo()).then() syntax. It is safer to stub spies - 
      - with doReturn|Throw() family of methods. More in javadocs for Mockito.spy() method.
   
   	at org.apache.pulsar.broker.PulsarService.getConfiguration(PulsarService.java:608)
   	at org.apache.pulsar.broker.service.BrokerService.shutdownEventLoopGracefully(BrokerService.java:846)
   	at java.base/java.lang.invoke.MethodHandle.invokeWithArguments(MethodHandle.java:710)
   	at org.apache.pulsar.broker.service.BrokerService.shutdownEventLoopGracefully(BrokerService.java:846)
   	at org.apache.pulsar.broker.service.BrokerService.closeAsync(BrokerService.java:757)
   	at java.base/java.lang.invoke.MethodHandle.invokeWithArguments(MethodHandle.java:710)
   	at org.apache.pulsar.broker.service.BrokerService.closeAsync(BrokerService.java:720)
   	at org.apache.pulsar.broker.service.BrokerService.close(BrokerService.java:680)
   	... 2 more
   ```
   
   After investigating I noticed that MetadataStore threads can still receive notifications even if the current test method is finished. Also other thread pools are not shutdown in the correct way and the test must wait for their terminations in order to avoid mixing up Mockito mocked objects.  
   
   ### Modifications
   - Enhanced `ManagedLedgerFactoryImpl#closeAsync` method
   - PulsarService#close sync method now wait for EventLoopGroup to be terminated
   - Introduced new method closeAsync for MetadataStore class to enhance parallelism while disposing resources in PulsarService and tests
   - Fixed a couple of tests which manually create executors and Netty event loop groups
   
   This is related to past flaky-tests like:
   - https://github.com/apache/pulsar/issues/13808
   - https://github.com/apache/pulsar/pull/14006
   - https://github.com/apache/pulsar/issues/13620
   
     
   - [x] `no-need-doc` 


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] lhotari commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and CoordinationService

Posted by GitBox <gi...@apache.org>.
lhotari commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1145882933

   @nicoloboschi Pleae rebase this PR.
   
   It looks like this could help with these issues which happen in PulsarFunctionTlsTest.tearDown:
   ```
   "main" #1 prio=5 os_prio=0 cpu=67703.73ms elapsed=3524.96s tid=0x00007f0b9c0243f0 nid=0xafb waiting on condition  [0x00007f0ba017c000]
      java.lang.Thread.State: WAITING (parking)
   	at jdk.internal.misc.Unsafe.park(java.base@17.0.3/Native Method)
   	- parking to wait for  <0x00000000ce100010> (a java.util.concurrent.CountDownLatch$Sync)
   	at java.util.concurrent.locks.LockSupport.park(java.base@17.0.3/LockSupport.java:211)
   	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(java.base@17.0.3/AbstractQueuedSynchronizer.java:715)
   	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(java.base@17.0.3/AbstractQueuedSynchronizer.java:1047)
   	at java.util.concurrent.CountDownLatch.await(java.base@17.0.3/CountDownLatch.java:230)
   	at org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl.shutdown(ManagedLedgerFactoryImpl.java:635)
   	at org.apache.pulsar.broker.ManagedLedgerClientFactory.close(ManagedLedgerClientFactory.java:134)
   	at org.apache.pulsar.broker.PulsarService.closeAsync(PulsarService.java:460)
   	at org.apache.pulsar.broker.PulsarService.close(PulsarService.java:374)
   	at org.apache.pulsar.functions.worker.PulsarFunctionTlsTest.tearDown(PulsarFunctionTlsTest.java:182)
   ```
   
   This is the [4th most flaky issue currently](https://docs.google.com/spreadsheets/d/165FHpHjs5fHccSsmQM4beeg6brn-zfUjcrXf6xAu4yQ/edit#gid=1452390962).


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nicoloboschi commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
nicoloboschi commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1131815602

   Test failed is not related, I opened an issue: https://github.com/apache/pulsar/issues/15676
   
   


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nicoloboschi closed pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and CoordinationService

Posted by GitBox <gi...@apache.org>.
nicoloboschi closed pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and CoordinationService
URL: https://github.com/apache/pulsar/pull/15638


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Technoboy- closed pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
Technoboy- closed pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore
URL: https://github.com/apache/pulsar/pull/15638


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] lhotari commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
lhotari commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1136813233

   There's a huge amount of [metadata-store-coordination-service threads in the thread dump](https://jstack.review/?https://gist.github.com/lhotari/5c1ac9953d3f33bc371915d73e154d77#tda_1_dump). 


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nicoloboschi commented on a diff in pull request #15638: [fix][test] Fix CoordinatorService, MetadataStore and MockZooKeeper leaks in tests

Posted by GitBox <gi...@apache.org>.
nicoloboschi commented on code in PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#discussion_r919799930


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java:
##########
@@ -249,20 +252,14 @@ public void setup() throws Exception {
 
     @AfterMethod(alwaysRun = true)
     public void teardown() throws Exception {
-        metadataStore.close();

Review Comment:
   yes it's intentional. `store` is created by `PersistentTopicTest`. `metadataStore` is handled by the superclass `MockedBookKeeperTestCase`. So before the `store` was never closed



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] github-actions[bot] commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and CoordinationService

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1176965084

   The pr had no activity for 30 days, mark with Stale label.


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] lhotari commented on a diff in pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and CoordinationService

Posted by GitBox <gi...@apache.org>.
lhotari commented on code in PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#discussion_r918754279


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -597,6 +601,8 @@ private CompletableFuture<Void> addTimeoutHandling(CompletableFuture<Void> futur
                 Duration.ofMillis(Math.max(1L, getConfiguration().getBrokerShutdownTimeoutMs())),
                 shutdownExecutor, () -> FutureUtil.createTimeoutException("Timeout in close", getClass(), "close"));
         future.handle((v, t) -> {
+            LOG.info("Shutdown timed out after {} ms", getConfiguration().getBrokerShutdownTimeoutMs());
+            LOG.info(ThreadDumpUtil.buildThreadDiagnosticString());

Review Comment:
   does this need `if (t != null) {` ?



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java:
##########
@@ -249,20 +252,20 @@ public void setup() throws Exception {
 
     @AfterMethod(alwaysRun = true)
     public void teardown() throws Exception {
-        metadataStore.close();
-        brokerService.getTopics().clear();
-        brokerService.close(); //to clear pulsarStats
-        try {
-            pulsar.close();
-        } catch (Exception e) {
-            log.warn("Failed to close pulsar service", e);
-            throw e;
-        }
-
-        executor.shutdownNow();
-        if (eventLoopGroup != null) {
-            eventLoopGroup.shutdownGracefully().get();
-        }
+        List<CompletableFuture<Void>> futures = new ArrayList<>();
+        futures.add(brokerService.closeAsync());
+        futures.add(pulsar.closeAsync());
+        futures.add(GracefulExecutorServicesShutdown.initiate()
+                .timeout(Duration.ZERO)
+                .shutdown(executor)
+                .handle());
+        final CompletableFuture<Void> eventLoopGroupCloseFuture = new CompletableFuture<>();
+        eventLoopGroup.shutdownGracefully().sync().addListener(f -> {
+            eventLoopGroupCloseFuture.complete(null);
+        });

Review Comment:
   this doesn't look right. 
   
   What is the intention of calling `.sync()`? that would wait for the shutdown to happen before proceeding. The future isn't needed in that case at all so that's why the `.addListener` doesn't make sense. 
   
   There's [org.apache.pulsar.common.util.netty.NettyFutureUtil#toCompletableFuture](https://github.com/apache/pulsar/blob/12d43a86d9fb228b498914155bc3a6194874a2a2/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/NettyFutureUtil.java#L36) to adapt a Netty Future to a CompletableFuture.
   
   



##########
pulsar-common/src/main/java/org/apache/pulsar/common/util/GracefulExecutorServicesShutdown.java:
##########
@@ -114,4 +121,16 @@ public CompletableFuture<Void> handle() {
         return new GracefulExecutorServicesTerminationHandler(timeout, terminationTimeout,
                 executorServices).getFuture();
     }
+
+    public static CompletableFuture<Void> shutdownEventLoopGracefully(EventLoopGroup eventLoopGroup,
+                                                                      long brokerShutdownTimeoutMs) {
+        long quietPeriod = Math.min((long) (
+                        GRACEFUL_SHUTDOWN_QUIET_PERIOD_RATIO_OF_TOTAL_TIMEOUT * brokerShutdownTimeoutMs),
+                GRACEFUL_SHUTDOWN_QUIET_PERIOD_MAX_MS);
+        long timeout = (long) (GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT * brokerShutdownTimeoutMs);
+        return NettyFutureUtil.toCompletableFutureVoid(
+                eventLoopGroup.shutdownGracefully(quietPeriod,
+                        timeout, TimeUnit.MILLISECONDS));
+    }
+

Review Comment:
   this method doesn't really belong to this class. org.apache.pulsar.common.util.netty.EventLoopUtil is a better location for this method.



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java:
##########
@@ -238,12 +240,19 @@ public void teardown() throws Exception {
         if (channel != null) {
             channel.close();
         }
-        pulsar.close();
-        brokerService.close();
-        executor.shutdownNow();
-        if (eventLoopGroup != null) {
-            eventLoopGroup.shutdownGracefully().get();
-        }
+        List<CompletableFuture<Void>> futures = new ArrayList<>();
+        futures.add(brokerService.closeAsync());
+        futures.add(pulsar.closeAsync());
+        futures.add(GracefulExecutorServicesShutdown.initiate()
+                .timeout(Duration.ZERO)
+                .shutdown(executor)
+                .handle());
+        final CompletableFuture<Void> eventLoopGroupCloseFuture = new CompletableFuture<>();
+        eventLoopGroup.shutdownGracefully().sync().addListener(f -> {
+            eventLoopGroupCloseFuture.complete(null);
+        });

Review Comment:
   this doesn't look right. (please check first comment about this code pattern)



##########
pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java:
##########
@@ -190,12 +195,18 @@ public void testHttpLookupRedirect() throws Exception {
     @AfterMethod(alwaysRun = true)
     @Override
     protected void cleanup() throws Exception {
-        if (this.executorService != null) {
-            this.executorService.shutdownNow();
-        }
-        if (eventExecutors != null) {
-            eventExecutors.shutdownGracefully();
-        }
+        List<CompletableFuture<Void>> futures = new ArrayList<>();
+        futures.add(pulsar.closeAsync());
+        futures.add(GracefulExecutorServicesShutdown.initiate()
+                .timeout(Duration.ZERO)
+                .shutdown(executorService)
+                .handle());
+        final CompletableFuture<Void> eventLoopGroupCloseFuture = new CompletableFuture<>();
+        eventExecutors.shutdownGracefully().sync().addListener(f -> {
+            eventLoopGroupCloseFuture.complete(null);
+        });

Review Comment:
   this doesn't look right. (please check first comment about this code pattern)



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java:
##########
@@ -208,19 +211,20 @@ public CompletableFuture<Boolean> checkInitializedBefore(PersistentSubscription
 
     @AfterMethod(alwaysRun = true)
     public void teardown() throws Exception {
-        brokerMock.close(); //to clear pulsarStats
-        try {
-            pulsarMock.close();
-        } catch (Exception e) {
-            log.warn("Failed to close pulsar service", e);
-            throw e;
-        }
-
+        List<CompletableFuture<Void>> futures = new ArrayList<>();
+        futures.add(brokerMock.closeAsync());
+        futures.add(pulsarMock.closeAsync());
+        futures.add(GracefulExecutorServicesShutdown.initiate()
+                .timeout(Duration.ZERO)
+                .shutdown(executor)
+                .handle());
+        final CompletableFuture<Void> eventLoopGroupCloseFuture = new CompletableFuture<>();
+        eventLoopGroup.shutdownGracefully().sync().addListener(f -> {
+            eventLoopGroupCloseFuture.complete(null);
+        });

Review Comment:
   this doesn't look right. (please check first comment about this code pattern)



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] lhotari commented on a diff in pull request #15638: [fix][test] Fix CoordinatorService, MetadataStore and MockZooKeeper leaks in tests

Posted by GitBox <gi...@apache.org>.
lhotari commented on code in PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#discussion_r919805326


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java:
##########
@@ -249,20 +252,14 @@ public void setup() throws Exception {
 
     @AfterMethod(alwaysRun = true)
     public void teardown() throws Exception {
-        metadataStore.close();

Review Comment:
   it seems that there's both store and metadataStore (defined in MockedBookKeeperTestCase). I think that metadataStore state should be cleaned up in MockedBookKeeperTestCase.tearDown method.



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] lhotari commented on a diff in pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
lhotari commented on code in PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#discussion_r881336111


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStore.java:
##########
@@ -159,4 +159,10 @@ public interface MetadataStore extends AutoCloseable {
      * @return the metadata cache object
      */
     <T> MetadataCache<T> getMetadataCache(MetadataSerde<T> serde);
+
+    /**
+     * Async close the current instance.
+     * @return
+     */
+    CompletableFuture<Void> closeAsync();

Review Comment:
   Changing the MetadataStore interface would require a PIP. We cannot do that. Please revert this part of the changes. It will also resolve the unclosed metadata store threads issue seen in the thread dumps.



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nicoloboschi commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and CoordinationService

Posted by GitBox <gi...@apache.org>.
nicoloboschi commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1181781405

   @lhotari PTAL


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] lhotari commented on a diff in pull request #15638: [fix][test] Fix CoordinatorService, MetadataStore and MockZooKeeper leaks in tests

Posted by GitBox <gi...@apache.org>.
lhotari commented on code in PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#discussion_r919791585


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java:
##########
@@ -249,20 +252,14 @@ public void setup() throws Exception {
 
     @AfterMethod(alwaysRun = true)
     public void teardown() throws Exception {
-        metadataStore.close();

Review Comment:
   is it fine to replace this with `store.close()`?



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nicoloboschi commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
nicoloboschi commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1134804841

   /pulsarbot rerun-failure-checks


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Technoboy- commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1129948755

   Will reopen later to re-run the test.


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nicoloboschi commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
nicoloboschi commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1131391094

   /pulsarbot rerun-failure-checks


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Technoboy- commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and CoordinationService

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1138048442

   In the past few days, when investigating the problem of pulsar close being hanged, another solution is to add an interface for asynchronous close. The idea is the same as this patch. However, I think this also requires other resources of the close method to be changed, so it may be better to send an email to discuss.
   @nicoloboschi @lhotari 


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] lhotari merged pull request #15638: [fix][test] Fix CoordinatorService, MetadataStore and MockZooKeeper leaks in tests

Posted by GitBox <gi...@apache.org>.
lhotari merged PR #15638:
URL: https://github.com/apache/pulsar/pull/15638


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Technoboy- closed pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
Technoboy- closed pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore
URL: https://github.com/apache/pulsar/pull/15638


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nicoloboschi closed pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
nicoloboschi closed pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore
URL: https://github.com/apache/pulsar/pull/15638


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nicoloboschi commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
nicoloboschi commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1132890324

   /pulsarbot rerun-failure-checks


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nicoloboschi commented on a diff in pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
nicoloboschi commented on code in PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#discussion_r881350316


##########
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStore.java:
##########
@@ -159,4 +159,10 @@ public interface MetadataStore extends AutoCloseable {
      * @return the metadata cache object
      */
     <T> MetadataCache<T> getMetadataCache(MetadataSerde<T> serde);
+
+    /**
+     * Async close the current instance.
+     * @return
+     */
+    CompletableFuture<Void> closeAsync();

Review Comment:
   I will revert it but it is unrelated with the thread leak.
   
   `metadata-store-coordination-service` threads come from `CoordinationServiceImpl` and the executor is never shutdown.
   
    



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nicoloboschi commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
nicoloboschi commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1132990538

   /pulsarbot rerun-failure-checks


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] lhotari commented on pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
lhotari commented on PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#issuecomment-1135628681

   @nicoloboschi I see that you have had to rerun tests quite many times. what is the source of flakiness? Have the problems already been reported as GH issues?


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nicoloboschi closed pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore

Posted by GitBox <gi...@apache.org>.
nicoloboschi closed pull request #15638: [fix] Ensure resources are disposed in a sync way while closing ManagedLedgerFactory and MetadataStore
URL: https://github.com/apache/pulsar/pull/15638


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] lhotari commented on a diff in pull request #15638: [fix][test] Fix CoordinatorService, MetadataStore and MockZooKeeper leaks in tests

Posted by GitBox <gi...@apache.org>.
lhotari commented on code in PR #15638:
URL: https://github.com/apache/pulsar/pull/15638#discussion_r919809721


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java:
##########
@@ -249,20 +252,14 @@ public void setup() throws Exception {
 
     @AfterMethod(alwaysRun = true)
     public void teardown() throws Exception {
-        metadataStore.close();

Review Comment:
   @nicoloboschi Good catch!



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org