Skip to content

Commit

Permalink
[fix][tests] Fix resource leak in tests. Call cleanup before setup (a…
Browse files Browse the repository at this point in the history
…pache#17743)

- when setup is called explicitly, cleanup should be issued before it
- remove unnecessary "resetConfig" method calls
  - resetConfig gets called as part of internalCleanup
  - when resetConfig is in cleanup, it's possible to override config before calling setup
    - this allows using setup & cleanup methods instead of using
      error prone internalSetup and internalCleanup methods which might leave something behind
  • Loading branch information
lhotari authored Sep 21, 2022
1 parent 2b9ffac commit d1a9a82
Show file tree
Hide file tree
Showing 36 changed files with 110 additions and 160 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@ protected void setup() throws Exception {
protected void cleanup() throws Exception {
super.internalCleanup();
useStaticPorts = false;
resetConfig();
}

@Override
Expand Down Expand Up @@ -105,6 +104,7 @@ public void testGetWorkerServiceException() throws Exception {

@Test
public void testAdvertisedAddress() throws Exception {
cleanup();
useStaticPorts = true;
setup();
assertEquals(pulsar.getAdvertisedAddress(), "localhost");
Expand All @@ -117,6 +117,7 @@ public void testAdvertisedAddress() throws Exception {

@Test
public void testAdvertisedListeners() throws Exception {
cleanup();
// don't use dynamic ports when using advertised listeners (#12079)
useStaticPorts = true;
conf.setAdvertisedListeners("internal:pulsar://gateway:6650, internal:pulsar+ssl://gateway:6651");
Expand All @@ -132,6 +133,7 @@ public void testAdvertisedListeners() throws Exception {

@Test
public void testDynamicBrokerPort() throws Exception {
cleanup();
useStaticPorts = false;
setup();
assertEquals(pulsar.getAdvertisedAddress(), "localhost");
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,6 @@ public class AdminApiClusterTest extends MockedPulsarServiceBaseTest {
@BeforeMethod
@Override
public void setup() throws Exception {
resetConfig();
super.internalSetup();
admin.clusters()
.createCluster(CLUSTER, ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,6 @@ public class AdminApiHealthCheckTest extends MockedPulsarServiceBaseTest {
@BeforeMethod
@Override
public void setup() throws Exception {
resetConfig();
super.internalSetup();
admin.clusters().createCluster("test",
ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,6 @@ public void setup() throws Exception {
@Override
public void cleanup() throws Exception {
super.internalCleanup();
resetConfig();
}

@Test(timeOut = 30000)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,6 @@ private String parseResponseEntity(Object entity) throws Exception {
@BeforeMethod
@Override
protected void setup() throws Exception {
resetConfig();
super.internalSetup();
// Create tenant, namespace, topic
admin.clusters().createCluster(clusterName, ClusterData.builder().serviceUrl(brokerUrl.toString()).build());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,6 @@ protected void setup() throws Exception {
@Override
protected void cleanup() throws Exception {
super.internalCleanup();
resetConfig();
}

@Test(timeOut = 10000)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,6 @@ public void initNamespace() throws Exception {
@Override
@BeforeMethod
public void setup() throws Exception {
resetConfig();
conf.setTopicLevelPoliciesEnabled(false);
conf.setSystemTopicEnabled(false);
conf.setClusterName(testLocalCluster);
Expand Down Expand Up @@ -1557,10 +1556,9 @@ public void testRetentionPolicyValidation() throws Exception {
public void testMaxTopicsPerNamespace() throws Exception {
cleanup();
conf.setMaxTopicsPerNamespace(15);
super.internalSetup();
setup();

String namespace = "testTenant/ns1";
admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(brokerUrl.toString()).build());
String namespace = BrokerTestUtil.newUniqueName("testTenant/ns1");
TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"),
Set.of("use"));
admin.tenants().createTenant("testTenant", tenantInfo);
Expand All @@ -1572,7 +1570,7 @@ public void testMaxTopicsPerNamespace() throws Exception {
assertEquals(10, admin.namespaces().getMaxTopicsPerNamespace(namespace));

// check create partitioned/non-partitioned topics using namespace policy
String topic = "persistent://testTenant/ns1/test_create_topic_v";
String topic = "persistent://" + namespace + "/test_create_topic_v";
admin.topics().createPartitionedTopic(topic + "1", 2);
admin.topics().createPartitionedTopic(topic + "2", 3);
admin.topics().createPartitionedTopic(topic + "3", 4);
Expand Down Expand Up @@ -1610,9 +1608,8 @@ public void testMaxTopicsPerNamespace() throws Exception {
conf.setMaxTopicsPerNamespace(0);
conf.setDefaultNumPartitions(3);
conf.setAllowAutoTopicCreationType("partitioned");
super.internalSetup();
setup();

admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(brokerUrl.toString()).build());
admin.tenants().createTenant("testTenant", tenantInfo);
admin.namespaces().createNamespace(namespace, Set.of("use"));
admin.namespaces().setMaxTopicsPerNamespace(namespace, 10);
Expand Down Expand Up @@ -1640,9 +1637,8 @@ public void testMaxTopicsPerNamespace() throws Exception {
conf.setMaxTopicsPerNamespace(0);
conf.setDefaultNumPartitions(1);
conf.setAllowAutoTopicCreationType("non-partitioned");
super.internalSetup();
setup();

admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(brokerUrl.toString()).build());
admin.tenants().createTenant("testTenant", tenantInfo);
admin.namespaces().createNamespace(namespace, Set.of("use"));
admin.namespaces().setMaxTopicsPerNamespace(namespace, 3);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,6 @@ public void initNamespace() throws Exception {
@Override
@BeforeMethod
public void setup() throws Exception {
resetConfig();
conf.setClusterName(testLocalCluster);
super.internalSetup();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@ public class TopicMessageTTLTest extends MockedPulsarServiceBaseTest {
@BeforeMethod
@Override
protected void setup() throws Exception {
resetConfig();
this.conf.setTtlDurationDefaultInSeconds(3600);
super.internalSetup();

Expand Down Expand Up @@ -125,15 +124,9 @@ public void testGetMessageTTL() throws Exception {

@Test
public void testTopicPolicyDisabled() throws Exception {
super.internalCleanup();
cleanup();
this.conf.setTopicLevelPoliciesEnabled(false);
super.internalSetup();

admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"), Set.of("test"));
admin.tenants().createTenant(this.testTenant, tenantInfo);
admin.namespaces().createNamespace(testTenant + "/" + testNamespace, Set.of("test"));
admin.topics().createPartitionedTopic(testTopic, 2);
setup();

try {
admin.topics().getMessageTTL(testTopic);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,6 @@ protected void setup() throws Exception {
@Override
public void cleanup() throws Exception {
super.internalCleanup();
this.resetConfig();
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,6 @@ public class V1_AdminApi2Test extends MockedPulsarServiceBaseTest {
@BeforeMethod
@Override
public void setup() throws Exception {
resetConfig();
conf.setTopicLevelPoliciesEnabled(false);
conf.setSystemTopicEnabled(false);
conf.setLoadBalancerEnabled(true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import lombok.Cleanup;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.http.HttpStatus;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.api.Consumer;
Expand Down Expand Up @@ -80,13 +81,19 @@
@Test(groups = "broker-admin")
public class AdminApiTransactionTest extends MockedPulsarServiceBaseTest {

@BeforeMethod
@Override
protected void setup() throws Exception {
protected ServiceConfiguration getDefaultConf() {
ServiceConfiguration conf = super.getDefaultConf();
conf.setEnablePackagesManagement(true);
conf.setPackagesManagementStorageProvider(MockedPackagesStorageProvider.class.getName());
conf.setTransactionCoordinatorEnabled(true);
conf.setTransactionBufferSnapshotMaxTransactionCount(1);
return conf;
}

@BeforeMethod
@Override
protected void setup() throws Exception {
super.internalSetup();
admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"), Set.of("test"));
Expand Down Expand Up @@ -566,9 +573,9 @@ public void testGetPendingAckInternalStats() throws Exception {

@Test(timeOut = 20000)
public void testTransactionNotEnabled() throws Exception {
stopBroker();
cleanup();
conf.setTransactionCoordinatorEnabled(false);
super.internalSetup();
setup();
try {
admin.transactions().getCoordinatorInternalStats(1, false);
} catch (PulsarAdminException ex) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -353,7 +353,7 @@ protected PulsarService newPulsarService(ServiceConfiguration conf) throws Excep
protected void setupBrokerMocks(PulsarService pulsar) throws Exception {
// Override default providers with mocked ones
doReturn(mockBookKeeperClientFactory).when(pulsar).newBookKeeperClientFactory();

PulsarMetadataEventSynchronizer synchronizer = StringUtils
.isNotBlank(pulsar.getConfig().getMetadataSyncEventTopic())
? new PulsarMetadataEventSynchronizer(pulsar, pulsar.getConfig().getMetadataSyncEventTopic())
Expand Down Expand Up @@ -517,7 +517,7 @@ public static void setFieldValue(Class<?> clazz, Object classObj, String fieldNa
field.set(classObj, fieldValue);
}

protected static ServiceConfiguration getDefaultConf() {
protected ServiceConfiguration getDefaultConf() {
ServiceConfiguration configuration = new ServiceConfiguration();
configuration.setAdvertisedAddress("localhost");
configuration.setClusterName(configClusterName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,6 @@ protected void setup() throws Exception {
@Override
protected void cleanup() throws Exception {
super.internalCleanup();
resetConfig();
}

// method for resetting state explicitly
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ public class InactiveTopicDeleteTest extends BrokerTestBase {

@BeforeMethod
protected void setup() throws Exception {
resetConfig();
//No-op
}

@AfterMethod(alwaysRun = true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@ protected void setup() throws Exception {
@Override
protected void cleanup() throws Exception {
super.internalCleanup();
resetConfig();
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1327,7 +1327,7 @@ public void testProducerQueueFullBlocking() throws Exception {
.create();

// 2. Stop broker
super.internalCleanup();
cleanup();

// 2. producer publish messages
long startTime = System.nanoTime();
Expand Down Expand Up @@ -1374,7 +1374,7 @@ public void testProducerQueueFullNonBlocking() throws Exception {
.create();

// 2. Stop broker
super.internalCleanup();
cleanup();

// 2. producer publish messages
long startTime = System.nanoTime();
Expand Down Expand Up @@ -1930,7 +1930,7 @@ public void testProducerBusy() throws Exception {

@Test
public void testHttpLookupWithNotFoundError() throws Exception {
stopBroker();
cleanup();
isTcpLookup = false;
setup();
try {
Expand Down Expand Up @@ -2017,7 +2017,7 @@ public <T extends Topic> T create(String topic, ManagedLedger ledger, BrokerServ
if(topicClazz == NonPersistentTopic.class) {
return (T) new NonPersistentTopic(topic, brokerService);
}else {
return (T) new PersistentTopic(topic, ledger, brokerService);
return (T) new PersistentTopic(topic, ledger, brokerService);
}
} catch (Exception e) {
throw new IllegalStateException(e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,6 @@ public class TopicDuplicationTest extends ProducerConsumerBase {
@BeforeMethod
@Override
protected void setup() throws Exception {
resetConfig();
this.conf.setBrokerDeduplicationEnabled(true);
super.internalSetup();
super.producerBaseSetup();
Expand Down Expand Up @@ -221,14 +220,12 @@ public void testDuplicationSnapshotApi() throws Exception {

@Test(timeOut = 30000)
public void testTopicPolicyTakeSnapshot() throws Exception {
super.internalCleanup();
resetConfig();
cleanup();
conf.setBrokerDeduplicationEnabled(true);
conf.setBrokerDeduplicationSnapshotFrequencyInSeconds(1);
conf.setBrokerDeduplicationSnapshotIntervalSeconds(7);
conf.setBrokerDeduplicationEntriesInterval(20000);
super.internalSetup();
super.producerBaseSetup();
setup();

final String topicName = testTopic + UUID.randomUUID().toString();
final String producerName = "my-producer";
Expand Down Expand Up @@ -330,14 +327,12 @@ public void testDuplicationSnapshot() throws Exception {
}

private void testTakeSnapshot(boolean enabledSnapshot) throws Exception {
super.internalCleanup();
resetConfig();
cleanup();
conf.setBrokerDeduplicationEnabled(true);
conf.setBrokerDeduplicationSnapshotFrequencyInSeconds(enabledSnapshot ? 1 : 0);
conf.setBrokerDeduplicationSnapshotIntervalSeconds(1);
conf.setBrokerDeduplicationEntriesInterval(20000);
super.internalSetup();
super.producerBaseSetup();
setup();

final String topicName = testTopic + UUID.randomUUID().toString();
final String producerName = "my-producer";
Expand Down Expand Up @@ -407,14 +402,12 @@ public void testNamespacePolicyApi() throws Exception {

@Test(timeOut = 30000)
public void testNamespacePolicyTakeSnapshot() throws Exception {
super.internalCleanup();
resetConfig();
cleanup();
conf.setBrokerDeduplicationEnabled(true);
conf.setBrokerDeduplicationSnapshotFrequencyInSeconds(1);
conf.setBrokerDeduplicationSnapshotIntervalSeconds(3);
conf.setBrokerDeduplicationEntriesInterval(20000);
super.internalSetup();
super.producerBaseSetup();
setup();

final String topicName = testTopic + UUID.randomUUID().toString();
final String producerName = "my-producer";
Expand Down Expand Up @@ -459,14 +452,12 @@ public void testNamespacePolicyTakeSnapshot() throws Exception {

@Test(timeOut = 30000)
public void testDisableNamespacePolicyTakeSnapshot() throws Exception {
super.internalCleanup();
resetConfig();
cleanup();
conf.setBrokerDeduplicationEnabled(true);
conf.setBrokerDeduplicationSnapshotFrequencyInSeconds(1);
conf.setBrokerDeduplicationSnapshotIntervalSeconds(1);
conf.setBrokerDeduplicationEntriesInterval(20000);
super.internalSetup();
super.producerBaseSetup();
setup();

final String topicName = testTopic + UUID.randomUUID().toString();
final String producerName = "my-producer";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,6 @@ protected void setup() throws Exception {
@Override
protected void cleanup() throws Exception {
super.internalCleanup();
resetConfig();
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,18 +96,16 @@ protected void setup() throws Exception {
@Override
protected void cleanup() throws Exception {
super.internalCleanup();
resetConfig();
}

@Test
public void testPublishRateLimitedTimes() throws Exception {
cleanup();
checkPublishRateLimitedTimes(true);
cleanup();
checkPublishRateLimitedTimes(false);
}

private void checkPublishRateLimitedTimes(boolean preciseRateLimit) throws Exception {
cleanup();
if (preciseRateLimit) {
conf.setBrokerPublisherThrottlingTickTimeMillis(10000000);
conf.setMaxPublishRatePerTopicInMessages(1);
Expand Down
Loading

0 comments on commit d1a9a82

Please sign in to comment.