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/09/14 04:56:37 UTC

[GitHub] [pulsar] gaoran10 opened a new pull request, #17632: [improve] Cancel the loadShedding task when closing pulsar service

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

   ### Motivation
   
   The PR https://github.com/apache/pulsar/pull/16408 supported loadBalancerSheddingIntervalMinutes dynamic configuration, but the scheduled load-shedding task may block pulsar service close operation, the pulsar service need to wait until the scheduled load-shedding task executed or reach broker shutdown timeout.
   
   ### Modifications
   
   Cancel the scheduled task when closing the pulsar service.
   
   ### Verifying this change
   
   Add a test to verify the pulsar service could be closed gracefully in time.
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If the box was checked, please highlight the changes*
   
   - [ ] Dependencies (add or upgrade a dependency)
   - [ ] The public API
   - [ ] The schema
   - [ ] The default values of configurations
   - [ ] The binary protocol
   - [ ] The REST endpoints
   - [ ] The admin CLI options
   - [ ] Anything that affects deployment
   
   ### Documentation
   
   <!-- DO NOT REMOVE THIS SECTION. CHECK THE PROPER BOX ONLY. -->
   
   - [ ] `doc-required` 
   (Your PR needs to update docs and you will update later)
   
   - [x] `doc-not-needed` 
   (Please explain why)
   
   - [ ] `doc` 
   (Your PR contains doc changes)
   
   - [ ] `doc-complete`
   (Docs have been already added)
   


-- 
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] codelipenghui commented on a diff in pull request #17632: [improve][broker] Cancel the loadShedding task when closing pulsar service

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


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceCloseTest.java:
##########
@@ -0,0 +1,62 @@
+/**
+ * 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.broker;
+
+import static org.mockito.Mockito.spy;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class PulsarServiceCloseTest extends MockedPulsarServiceBaseTest {
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    protected PulsarService startBrokerWithoutAuthorization(ServiceConfiguration conf) throws Exception {
+        conf.setBrokerShutdownTimeoutMs(1000 * 60 * 5);
+        conf.setLoadBalancerSheddingIntervalMinutes(30);
+        PulsarService pulsar = spy(newPulsarService(conf));
+        setupBrokerMocks(pulsar);
+        beforePulsarStartMocks(pulsar);
+        pulsar.start();
+        log.info("Pulsar started. brokerServiceUrl: {} webServiceAddress: {}", pulsar.getBrokerServiceUrl(),
+                pulsar.getWebServiceAddress());
+        return pulsar;
+    }
+
+    @Test(timeOut = 30_000)
+    public void closeInTimeTest() throws Exception {
+        // The pulsar service is not used, so it should be closed gracefully in short time.
+        pulsar.close();

Review Comment:
   Can we add a check for the `LoadSheddingTask.java.future` is cancelled, or we can just add a method like `LoadSheddingTask.isCancelled`



-- 
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- merged pull request #17632: [improve][broker] Cancel the loadShedding task when closing pulsar service

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


-- 
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] gaoran10 commented on a diff in pull request #17632: [improve][broker] Cancel the loadShedding task when closing pulsar service

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


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceCloseTest.java:
##########
@@ -0,0 +1,62 @@
+/**
+ * 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.broker;
+
+import static org.mockito.Mockito.spy;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class PulsarServiceCloseTest extends MockedPulsarServiceBaseTest {
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    protected PulsarService startBrokerWithoutAuthorization(ServiceConfiguration conf) throws Exception {
+        conf.setBrokerShutdownTimeoutMs(1000 * 60 * 5);
+        conf.setLoadBalancerSheddingIntervalMinutes(30);
+        PulsarService pulsar = spy(newPulsarService(conf));
+        setupBrokerMocks(pulsar);
+        beforePulsarStartMocks(pulsar);
+        pulsar.start();
+        log.info("Pulsar started. brokerServiceUrl: {} webServiceAddress: {}", pulsar.getBrokerServiceUrl(),
+                pulsar.getWebServiceAddress());
+        return pulsar;
+    }
+
+    @Test(timeOut = 30_000)
+    public void closeInTimeTest() throws Exception {
+        // The pulsar service is not used, so it should be closed gracefully in short time.
+        pulsar.close();

Review Comment:
   I adjust the test, 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] codelipenghui commented on a diff in pull request #17632: [improve][broker] Cancel the loadShedding task when closing pulsar service

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


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceCloseTest.java:
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.broker;
+
+import static org.mockito.Mockito.spy;
+import static org.testng.AssertJUnit.assertFalse;
+import static org.testng.AssertJUnit.assertTrue;
+
+import java.util.concurrent.ScheduledFuture;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.loadbalance.LoadSheddingTask;
+import org.awaitility.reflect.WhiteboxImpl;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j

Review Comment:
   It's better to have a test group name



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceCloseTest.java:
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.broker;
+
+import static org.mockito.Mockito.spy;
+import static org.testng.AssertJUnit.assertFalse;
+import static org.testng.AssertJUnit.assertTrue;
+
+import java.util.concurrent.ScheduledFuture;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.loadbalance.LoadSheddingTask;
+import org.awaitility.reflect.WhiteboxImpl;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class PulsarServiceCloseTest extends MockedPulsarServiceBaseTest {
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+    }
+
+    @AfterMethod(alwaysRun = true)

Review Comment:
   AfterClass



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceCloseTest.java:
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.broker;
+
+import static org.mockito.Mockito.spy;
+import static org.testng.AssertJUnit.assertFalse;
+import static org.testng.AssertJUnit.assertTrue;
+
+import java.util.concurrent.ScheduledFuture;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.loadbalance.LoadSheddingTask;
+import org.awaitility.reflect.WhiteboxImpl;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class PulsarServiceCloseTest extends MockedPulsarServiceBaseTest {
+
+    @BeforeMethod

Review Comment:
   BeforeClass



-- 
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] poorbarcode commented on a diff in pull request #17632: [improve][broker] Cancel the loadShedding task when closing pulsar service

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadSheddingTask.java:
##########
@@ -52,25 +55,24 @@ public void run() {
         } catch (Exception e) {

Review Comment:
   In the first line of method `run` method, we can add a check to make quick failure:
   
   ```java
   
   public void run(){
       if (isCancel){
           return;
       }
       ...
   }
   
   ```



-- 
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] Jason918 commented on a diff in pull request #17632: [improve][broker] Cancel the loadShedding task when closing pulsar service

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


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceCloseTest.java:
##########
@@ -0,0 +1,62 @@
+/**
+ * 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.broker;
+
+import static org.mockito.Mockito.spy;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class PulsarServiceCloseTest extends MockedPulsarServiceBaseTest {
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    protected PulsarService startBrokerWithoutAuthorization(ServiceConfiguration conf) throws Exception {
+        conf.setBrokerShutdownTimeoutMs(1000 * 60 * 5);
+        conf.setLoadBalancerSheddingIntervalMinutes(30);
+        PulsarService pulsar = spy(newPulsarService(conf));
+        setupBrokerMocks(pulsar);
+        beforePulsarStartMocks(pulsar);
+        pulsar.start();
+        log.info("Pulsar started. brokerServiceUrl: {} webServiceAddress: {}", pulsar.getBrokerServiceUrl(),
+                pulsar.getWebServiceAddress());
+        return pulsar;
+    }
+
+    @Test(timeOut = 30_000)
+    public void closeInTimeTest() throws Exception {
+        // The pulsar service is not used, so it should be closed gracefully in short time.
+        pulsar.close();

Review Comment:
   +1



-- 
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] gaoran10 commented on a diff in pull request #17632: [improve][broker] Cancel the loadShedding task when closing pulsar service

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadSheddingTask.java:
##########
@@ -52,25 +55,24 @@ public void run() {
         } catch (Exception e) {

Review Comment:
   Thanks, good idea.



-- 
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] poorbarcode commented on a diff in pull request #17632: [improve][broker] Cancel the loadShedding task when closing pulsar service

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadSheddingTask.java:
##########
@@ -52,25 +55,24 @@ public void run() {
         } catch (Exception e) {

Review Comment:
   At the first line of method `run`, we can add a check to make quick failure:
   
   ```java
   
   public void run(){
       if (isCancel){
           return;
       }
       ...
   }
   
   ```



-- 
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] gaoran10 commented on a diff in pull request #17632: [improve][broker] Cancel the loadShedding task when closing pulsar service

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


##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceCloseTest.java:
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.broker;
+
+import static org.mockito.Mockito.spy;
+import static org.testng.AssertJUnit.assertFalse;
+import static org.testng.AssertJUnit.assertTrue;
+
+import java.util.concurrent.ScheduledFuture;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.loadbalance.LoadSheddingTask;
+import org.awaitility.reflect.WhiteboxImpl;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j

Review Comment:
   fix



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceCloseTest.java:
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.broker;
+
+import static org.mockito.Mockito.spy;
+import static org.testng.AssertJUnit.assertFalse;
+import static org.testng.AssertJUnit.assertTrue;
+
+import java.util.concurrent.ScheduledFuture;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.loadbalance.LoadSheddingTask;
+import org.awaitility.reflect.WhiteboxImpl;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class PulsarServiceCloseTest extends MockedPulsarServiceBaseTest {
+
+    @BeforeMethod

Review Comment:
   fix



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceCloseTest.java:
##########
@@ -0,0 +1,78 @@
+/**
+ * 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.broker;
+
+import static org.mockito.Mockito.spy;
+import static org.testng.AssertJUnit.assertFalse;
+import static org.testng.AssertJUnit.assertTrue;
+
+import java.util.concurrent.ScheduledFuture;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.loadbalance.LoadSheddingTask;
+import org.awaitility.reflect.WhiteboxImpl;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class PulsarServiceCloseTest extends MockedPulsarServiceBaseTest {
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+    }
+
+    @AfterMethod(alwaysRun = true)

Review Comment:
   fix



-- 
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