From 6bbc3f50b75a9507ee89a8cbf2b85e511fc839e4 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Mon, 24 Feb 2020 16:01:47 +0100 Subject: [PATCH] Avoid race while mocking compactor There seems to be a race when mocking the compactor in MockedPulsarServiceBaseTest which yields errors like: ``` Caused by: java.lang.ClassCastException: org.apache.pulsar.compaction.TwoPhaseCompactor$MockitoMock$1141048386 cannot be cast to org.apache.pulsar.broker.service.BrokerService ``` This obviously causes tests to fail in strange and surprising ways. I've not been able to reproduce, but the issue seems to be with how we mock the compactor. We don't want to have to construct the whole compactor as we just want to spy on it, so we get the current compactor, wrap it in spy and tell the PulsarService instance to return it when getCompactor is called. However, we are doing this after we have already called PulsarService#start(). So there are threads already using the mock structures. The mock structures themselves are not thread safe, so modifying them while they are in use, is not a good idea. The fix is to finish mocking before invoking #start(). Do do this, I've broken out the Compactor construction method in PulsarService, so that alone can be mocked to wrap the object in a spy. --- .../org/apache/pulsar/broker/PulsarService.java | 15 ++++++++------- .../broker/auth/MockedPulsarServiceBaseTest.java | 8 +++++--- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 2584955b496e8..5e515aacf1760 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -932,15 +932,16 @@ protected synchronized ScheduledExecutorService getCompactorExecutor() { return this.compactorExecutor; } + // only public so mockito can mock it + public Compactor newCompactor() throws PulsarServerException { + return new TwoPhaseCompactor(this.getConfiguration(), + getClient(), getBookKeeperClient(), + getCompactorExecutor()); + } + public synchronized Compactor getCompactor() throws PulsarServerException { if (this.compactor == null) { - try { - this.compactor = new TwoPhaseCompactor(this.getConfiguration(), - getClient(), getBookKeeperClient(), - getCompactorExecutor()); - } catch (Exception e) { - throw new PulsarServerException(e); - } + this.compactor = newCompactor(); } return this.compactor; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index 683c220ba5215..33fbea4e9d172 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.auth; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; @@ -246,9 +247,6 @@ protected PulsarService startBroker(ServiceConfiguration conf) throws Exception pulsar.start(); conf.setAuthorizationEnabled(isAuthorizationEnabled); - Compactor spiedCompactor = spy(pulsar.getCompactor()); - doReturn(spiedCompactor).when(pulsar).getCompactor(); - return pulsar; } @@ -261,6 +259,10 @@ protected void setupBrokerMocks(PulsarService pulsar) throws Exception { doReturn(namespaceServiceSupplier).when(pulsar).getNamespaceServiceProvider(); doReturn(sameThreadOrderedSafeExecutor).when(pulsar).getOrderedExecutor(); + + doAnswer((invocation) -> { + return spy(invocation.callRealMethod()); + }).when(pulsar).newCompactor(); } public TenantInfo createDefaultTenantInfo() throws PulsarAdminException {