Skip to content

Commit

Permalink
[Tests] Fix flaky org.mockito.exceptions.misusing.WrongTypeOfReturnVa…
Browse files Browse the repository at this point in the history
…lue issue in broker tests (apache#13621)

- replace spying of a instance with a spy created by Mockito with given
  constructor arguments
  • Loading branch information
lhotari committed Jan 5, 2022
1 parent 1f10281 commit 17a971a
Show file tree
Hide file tree
Showing 48 changed files with 181 additions and 192 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@
public class OffloadPrefixReadTest extends MockedBookKeeperTestCase {
@Test
public void testOffloadRead() throws Exception {
MockLedgerOffloader offloader = spy(new MockLedgerOffloader());
MockLedgerOffloader offloader = spy(MockLedgerOffloader.class);
ManagedLedgerConfig config = new ManagedLedgerConfig();
config.setMaxEntriesPerLedger(10);
config.setMinimumRolloverTime(0, TimeUnit.SECONDS);
Expand Down Expand Up @@ -122,7 +122,7 @@ public void testOffloadRead() throws Exception {

@Test
public void testBookkeeperFirstOffloadRead() throws Exception {
MockLedgerOffloader offloader = spy(new MockLedgerOffloader());
MockLedgerOffloader offloader = spy(MockLedgerOffloader.class);
MockClock clock = new MockClock();
offloader.getOffloadPolicies()
.setManagedLedgerOffloadedReadPriority(OffloadedReadPriority.BOOKKEEPER_FIRST);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import java.util.Optional;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;

import org.apache.bookkeeper.client.PulsarMockBookKeeper;
import org.apache.bookkeeper.common.util.OrderedScheduler;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.pulsar.broker;

import java.util.UUID;
import org.mockito.Mockito;

/**
* Holds util methods used in test.
Expand All @@ -29,4 +30,18 @@ public static String newUniqueName(String prefix) {
return prefix + "-" + UUID.randomUUID();
}

/**
* Creates a Mockito spy directly without an intermediate instance to spy.
* This is to address flaky test issue where a spy created with a given instance fails with
* {@link org.mockito.exceptions.misusing.WrongTypeOfReturnValue} exception.
*
* @param classToSpy the class to spy
* @param args the constructor arguments to use when creating the spy instance
* @return a spy of the provided class created with given constructor arguments
*/
public static <T> T spyWithClassAndConstructorArgs(Class<T> classToSpy, Object... args) {
return Mockito.mock(classToSpy, Mockito.withSettings()
.useConstructor(args)
.defaultAnswer(Mockito.CALLS_REAL_METHODS));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ protected void setup() throws Exception {
new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test")));
admin.namespaces().createNamespace("prop/ns-abc");
admin.namespaces().setNamespaceReplicationClusters("prop/ns-abc", Sets.newHashSet("test"));
persistentTopics = spy(new PersistentTopics());
persistentTopics = spy(PersistentTopics.class);
persistentTopics.setServletContext(new MockServletContext());
persistentTopics.setPulsar(pulsar);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,17 +119,17 @@ public void setup() throws Exception {
conf.setClusterName(configClusterName);
super.internalSetup();

clusters = spy(new Clusters());
clusters = spy(Clusters.class);
clusters.setPulsar(pulsar);
doReturn("test").when(clusters).clientAppId();
doNothing().when(clusters).validateSuperUserAccess();

properties = spy(new Properties());
properties = spy(Properties.class);
properties.setPulsar(pulsar);
doReturn("test").when(properties).clientAppId();
doNothing().when(properties).validateSuperUserAccess();

namespaces = spy(new Namespaces());
namespaces = spy(Namespaces.class);
namespaces.setServletContext(new MockServletContext());
namespaces.setPulsar(pulsar);
doReturn("test").when(namespaces).clientAppId();
Expand All @@ -138,15 +138,15 @@ public void setup() throws Exception {
doNothing().when(namespaces).validateAdminAccessForTenant("other-tenant");
doNothing().when(namespaces).validateAdminAccessForTenant("new-property");

brokers = spy(new Brokers());
brokers = spy(Brokers.class);
brokers.setPulsar(pulsar);
doReturn("test").when(brokers).clientAppId();
doNothing().when(brokers).validateSuperUserAccess();

uriField = PulsarWebResource.class.getDeclaredField("uri");
uriField.setAccessible(true);

persistentTopics = spy(new PersistentTopics());
persistentTopics = spy(PersistentTopics.class);
persistentTopics.setServletContext(new MockServletContext());
persistentTopics.setPulsar(pulsar);
doReturn("test").when(persistentTopics).clientAppId();
Expand All @@ -156,11 +156,11 @@ public void setup() throws Exception {
doNothing().when(persistentTopics).validateAdminAccessForTenant("other-tenant");
doNothing().when(persistentTopics).validateAdminAccessForTenant("prop-xyz");

resourceQuotas = spy(new ResourceQuotas());
resourceQuotas = spy(ResourceQuotas.class);
resourceQuotas.setServletContext(new MockServletContext());
resourceQuotas.setPulsar(pulsar);

brokerStats = spy(new BrokerStats());
brokerStats = spy(BrokerStats.class);
brokerStats.setServletContext(new MockServletContext());
brokerStats.setPulsar(pulsar);

Expand All @@ -169,7 +169,7 @@ public void setup() throws Exception {
doReturn("test").when(persistentTopics).clientAppId();
doReturn(mock(AuthenticationDataHttps.class)).when(persistentTopics).clientAuthData();

schemasResource = spy(new SchemasResource());
schemasResource = spy(SchemasResource.class);
schemasResource.setServletContext(new MockServletContext());
schemasResource.setPulsar(pulsar);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -158,7 +158,7 @@ public void setup() throws Exception {
conf.setClusterName(testLocalCluster);
super.internalSetup();

namespaces = spy(new Namespaces());
namespaces = spy(Namespaces.class);
namespaces.setServletContext(new MockServletContext());
namespaces.setPulsar(pulsar);
doReturn(false).when(namespaces).isRequestHttps();
Expand Down Expand Up @@ -1093,7 +1093,7 @@ public void testValidateTopicOwnership() throws Exception {
ownership.setAccessible(true);
ownership.set(pulsar.getNamespaceService(), MockOwnershipCache);
TopicName topicName = TopicName.get(testNs.getPersistentTopicName("my-topic"));
PersistentTopics topics = spy(new PersistentTopics());
PersistentTopics topics = spy(PersistentTopics.class);
topics.setServletContext(new MockServletContext());
topics.setPulsar(pulsar);
doReturn(false).when(topics).isRequestHttps();
Expand Down Expand Up @@ -1257,7 +1257,7 @@ public void testSubscribeRate() throws Exception {
admin.tenants().deleteTenant("my-tenants");
}

class MockLedgerOffloader implements LedgerOffloader {
public static class MockLedgerOffloader implements LedgerOffloader {
ConcurrentHashMap<Long, UUID> offloads = new ConcurrentHashMap<Long, UUID>();
ConcurrentHashMap<Long, UUID> deletes = new ConcurrentHashMap<Long, UUID>();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.pulsar.broker.admin;

import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyString;
Expand Down Expand Up @@ -115,7 +116,7 @@ public void initPersistentTopics() throws Exception {
@BeforeMethod
protected void setup() throws Exception {
super.internalSetup();
persistentTopics = spy(new PersistentTopics());
persistentTopics = spy(PersistentTopics.class);
persistentTopics.setServletContext(new MockServletContext());
persistentTopics.setPulsar(pulsar);
doReturn(false).when(persistentTopics).isRequestHttps();
Expand All @@ -125,7 +126,7 @@ protected void setup() throws Exception {
doNothing().when(persistentTopics).validateAdminAccessForTenant(this.testTenant);
doReturn(mock(AuthenticationDataHttps.class)).when(persistentTopics).clientAuthData();

nonPersistentTopic = spy(new NonPersistentTopics());
nonPersistentTopic = spy(NonPersistentTopics.class);
nonPersistentTopic.setServletContext(new MockServletContext());
nonPersistentTopic.setPulsar(pulsar);
doReturn(false).when(nonPersistentTopic).isRequestHttps();
Expand All @@ -137,7 +138,7 @@ protected void setup() throws Exception {

PulsarResources resources =
spy(new PulsarResources(pulsar.getLocalMetadataStore(), pulsar.getConfigurationMetadataStore()));
doReturn(spy(new TopicResources(pulsar.getLocalMetadataStore()))).when(resources).getTopicResources();
doReturn(spyWithClassAndConstructorArgs(TopicResources.class, pulsar.getLocalMetadataStore())).when(resources).getTopicResources();
Whitebox.setInternalState(pulsar, "pulsarResources", resources);

admin.clusters().createCluster("use", ClusterData.builder().serviceUrl("http://broker-use.com:8080").build());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ public class ResourceGroupsTest extends MockedPulsarServiceBaseTest {
@Override
protected void setup() throws Exception {
super.internalSetup();
resourcegroups = spy(new ResourceGroups());
resourcegroups = spy(ResourceGroups.class);
resourcegroups.setServletContext(new MockServletContext());
resourcegroups.setPulsar(pulsar);
doReturn(false).when(resourcegroups).isRequestHttps();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.pulsar.broker.auth;

import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
Expand Down Expand Up @@ -321,7 +322,7 @@ protected void setupBrokerMocks(PulsarService pulsar) throws Exception {
doReturn(createLocalMetadataStore()).when(pulsar).createLocalMetadataStore();
doReturn(createConfigurationMetadataStore()).when(pulsar).createConfigurationMetadataStore();

Supplier<NamespaceService> namespaceServiceSupplier = () -> spy(new NamespaceService(pulsar));
Supplier<NamespaceService> namespaceServiceSupplier = () -> spyWithClassAndConstructorArgs(NamespaceService.class, pulsar);
doReturn(namespaceServiceSupplier).when(pulsar).getNamespaceServiceProvider();

doReturn(sameThreadOrderedSafeExecutor).when(pulsar).getOrderedExecutor();
Expand Down Expand Up @@ -385,7 +386,7 @@ public static MockZooKeeper createMockZooKeeperGlobal() {
}

public static NonClosableMockBookKeeper createMockBookKeeper(OrderedExecutor executor) throws Exception {
return spy(new NonClosableMockBookKeeper(executor));
return spyWithClassAndConstructorArgs(NonClosableMockBookKeeper.class, executor);
}

// Prevent the MockBookKeeper instance from being closed when the broker is restarted within a test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.pulsar.broker.loadbalance;

import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs;
import com.google.common.collect.Sets;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -74,7 +75,7 @@ public void anErrorShouldBeThrowBeforeLeaderElected() throws PulsarServerExcepti
config.setAdvertisedAddress("localhost");
config.setZookeeperServers("127.0.0.1" + ":" + bkEnsemble.getZookeeperPort());
@Cleanup
PulsarService pulsar = Mockito.spy(new MockPulsarService(config));
PulsarService pulsar = spyWithClassAndConstructorArgs(MockPulsarService.class, config);
pulsar.start();

// mock pulsar.getLeaderElectionService() in a thread safe way
Expand Down Expand Up @@ -134,7 +135,7 @@ private void checkLookupException(String tenant, String namespace, PulsarClient
}
}

private static class MockPulsarService extends PulsarService {
public static class MockPulsarService extends PulsarService {

public MockPulsarService(ServiceConfiguration config) {
super(config);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.pulsar.broker.loadbalance;

import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs;
import static org.mockito.Mockito.atLeastOnce;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
Expand Down Expand Up @@ -105,7 +106,7 @@ void setup() throws Exception {
bkEnsemble.start();

// Start broker 1
ServiceConfiguration config1 = spy(new ServiceConfiguration());
ServiceConfiguration config1 = spy(ServiceConfiguration.class);
config1.setClusterName("use");
config1.setWebServicePort(Optional.of(0));
config1.setZookeeperServers("127.0.0.1" + ":" + bkEnsemble.getZookeeperPort());
Expand Down Expand Up @@ -333,7 +334,7 @@ public void testLoadReportParsing() throws Exception {

@Test(enabled = true)
public void testDoLoadShedding() throws Exception {
SimpleLoadManagerImpl loadManager = spy(new SimpleLoadManagerImpl(pulsar1));
SimpleLoadManagerImpl loadManager = spyWithClassAndConstructorArgs(SimpleLoadManagerImpl.class, pulsar1);
PulsarResourceDescription rd = new PulsarResourceDescription();
rd.put("memory", new ResourceUsage(1024, 4096));
rd.put("cpu", new ResourceUsage(10, 100));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ public void setUp() throws Exception {
pulsar = mock(PulsarService.class);
ns = mock(NamespaceService.class);
auth = mock(AuthorizationService.class);
config = spy(new ServiceConfiguration());
config = spy(ServiceConfiguration.class);
config.setClusterName("use");
clusters = new TreeSet<>();
clusters.add("use");
Expand Down Expand Up @@ -104,7 +104,7 @@ public void setUp() throws Exception {
@Test
public void crossColoLookup() throws Exception {

TopicLookup destLookup = spy(new TopicLookup());
TopicLookup destLookup = spy(TopicLookup.class);
doReturn(false).when(destLookup).isRequestHttps();
destLookup.setPulsar(pulsar);
doReturn("null").when(destLookup).clientAppId();
Expand All @@ -130,7 +130,7 @@ public void crossColoLookup() throws Exception {
@Test
public void testLookupTopicNotExist() throws Exception {

MockTopicLookup destLookup = spy(new MockTopicLookup());
MockTopicLookup destLookup = spy(MockTopicLookup.class);
doReturn(false).when(destLookup).isRequestHttps();
destLookup.setPulsar(pulsar);
doReturn("null").when(destLookup).clientAppId();
Expand Down Expand Up @@ -158,7 +158,7 @@ public void testLookupTopicNotExist() throws Exception {
assertEquals(restException.getResponse().getStatus(), Status.NOT_FOUND.getStatusCode());
}

static class MockTopicLookup extends TopicLookup {
public static class MockTopicLookup extends TopicLookup {
@Override
protected void validateClusterOwnership(String s) {
// do nothing
Expand All @@ -171,7 +171,7 @@ public void testNotEnoughLookupPermits() throws Exception {
BrokerService brokerService = pulsar.getBrokerService();
doReturn(new Semaphore(0)).when(brokerService).getLookupRequestSemaphore();

TopicLookup destLookup = spy(new TopicLookup());
TopicLookup destLookup = spy(TopicLookup.class);
doReturn(false).when(destLookup).isRequestHttps();
destLookup.setPulsar(pulsar);
doReturn("null").when(destLookup).clientAppId();
Expand Down Expand Up @@ -209,7 +209,7 @@ public void testValidateReplicationSettingsOnNamespace() throws Exception {
// doReturn(Optional.of(policies2)).when(policiesCache)
// .get(AdminResource.path(POLICIES, property, cluster, ns2));

TopicLookup destLookup = spy(new TopicLookup());
TopicLookup destLookup = spy(TopicLookup.class);
doReturn(false).when(destLookup).isRequestHttps();
destLookup.setPulsar(pulsar);
doReturn("null").when(destLookup).clientAppId();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ public class TopicLookupTest extends PulsarWebResourceTest {

@Override
protected ResourceConfig configure() {
resource = spy(new TestableTopicLookup());
resource = spy(TestableTopicLookup.class);
return new ResourceConfig().register(resource);
}

Expand All @@ -66,7 +66,7 @@ public void testListenerName() {
assertEquals(resource.actualListenerName, "query");
}

private static class TestableTopicLookup extends TopicLookup {
public static class TestableTopicLookup extends TopicLookup {
private String actualListenerName;

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.pulsar.broker.namespace;

import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.spy;
Expand Down Expand Up @@ -115,7 +116,7 @@ protected void startBroker() throws Exception {
conf.setWebServicePortTls(Optional.of(0));
serviceConfigurationList.add(conf);

PulsarService pulsar = spy(new PulsarService(conf));
PulsarService pulsar = spyWithClassAndConstructorArgs(PulsarService.class, conf);

setupBrokerMocks(pulsar);
pulsar.start();
Expand All @@ -129,7 +130,7 @@ protected void setupBrokerMocks(PulsarService pulsar) throws Exception {
MockZooKeeperSession mockZooKeeperSession = MockZooKeeperSession.newInstance(mockZooKeeper);
doReturn(new ZKMetadataStore(mockZooKeeperSession)).when(pulsar).createLocalMetadataStore();
doReturn(new ZKMetadataStore(mockZooKeeperSession)).when(pulsar).createConfigurationMetadataStore();
Supplier<NamespaceService> namespaceServiceSupplier = () -> spy(new NamespaceService(pulsar));
Supplier<NamespaceService> namespaceServiceSupplier = () -> spyWithClassAndConstructorArgs(NamespaceService.class, pulsar);
doReturn(namespaceServiceSupplier).when(pulsar).getNamespaceServiceProvider();

SameThreadOrderedSafeExecutor executor = new SameThreadOrderedSafeExecutor();
Expand All @@ -153,7 +154,7 @@ public static MockZooKeeper createMockZooKeeper() throws Exception {
}

public static NonClosableMockBookKeeper createMockBookKeeper(OrderedExecutor executor) throws Exception {
return spy(new NonClosableMockBookKeeper(executor));
return spyWithClassAndConstructorArgs(NonClosableMockBookKeeper.class, executor);
}

// Prevent the MockBookKeeper instance from being closed when the broker is restarted within a test
Expand Down
Loading

0 comments on commit 17a971a

Please sign in to comment.