stefan-egli commented on code in PR #43: URL: https://github.com/apache/sling-org-apache-sling-event/pull/43#discussion_r2079827515
########## src/main/java/org/apache/sling/event/impl/jobs/config/JobManagerConfiguration.java: ########## @@ -199,6 +202,34 @@ static JobManagerConfiguration newForTest(ResourceResolverFactory resourceResolv /** The topology capabilities. */ private volatile TopologyCapabilities topologyCapabilities; + /** The condition that determines if the job manager is enabled. */ + @Reference( + target = "(osgi.condition.id=true)", + cardinality = ReferenceCardinality.OPTIONAL, + policy = ReferencePolicy.DYNAMIC + ) + private volatile Condition condition; Review Comment: * this should have a more specific name * "somehow/somewhere" we should try to hide requiring this condition behind a toggle ########## src/main/java/org/apache/sling/event/impl/jobs/queues/JobQueueImpl.java: ########## @@ -310,6 +314,11 @@ public void run() { } private void startJob(final JobHandler handler) { + if (!services.configuration.isEnable()) { + logger.debug("JobManager is disabled, stopping job {} in queue {}", handler.getJob().getId(), queueName); + handler.finished(Job.JobState.STOPPED, true, null); Review Comment: that wouldn't be good to stop the job here. we should instead not touch the job state at all. at this point no damage was done to the job yet - we can still leave everything as is, the job hasn't been started yet. ########## src/main/java/org/apache/sling/event/impl/jobs/queues/QueueManager.java: ########## @@ -191,6 +193,11 @@ protected void deactivate() { * @see java.lang.Runnable#run() */ void maintain() { + // Skip maintenance if job manager is disabled + if (!this.configuration.isEnable()) { + logger.debug("JobManager is disabled, skipping maintenance"); + return; + } Review Comment: I'm not sure if we should avoid the `QueueManager.maintain` or the `JobQueueImpl.maintain` .. one of the two I think should be skipped indeed. Reason I'm hesitant about avoiding the `QueueManager.maintain` is that it also has the "remove idle job queues" logic - so I wonder if part of this method should still be executed? ########## src/main/java/org/apache/sling/event/impl/jobs/config/JobManagerConfiguration.java: ########## @@ -199,6 +202,34 @@ static JobManagerConfiguration newForTest(ResourceResolverFactory resourceResolv /** The topology capabilities. */ private volatile TopologyCapabilities topologyCapabilities; + /** The condition that determines if the job manager is enabled. */ + @Reference( + target = "(osgi.condition.id=true)", + cardinality = ReferenceCardinality.OPTIONAL, + policy = ReferencePolicy.DYNAMIC + ) + private volatile Condition condition; + + /** + * Get the condition that determines if the job manager is enabled. + * @return The condition or null if the job manager is disabled + */ + public boolean isEnable() { Review Comment: name should be an adjective, not a verb. in addition I think it should be qualified a bit better. It's not so much about the jobmanager as a whole being enabled or not - it is only about whether jobs should be started or not. they should still be assigned for example - that shouldn't be prevented during startup/termination. ########## src/test/java/org/apache/sling/event/impl/jobs/JobManagerConditionTest.java: ########## @@ -0,0 +1,216 @@ +/* + * 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.sling.event.impl.jobs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.*; + +import java.lang.reflect.Field; +import java.util.HashMap; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.sling.commons.scheduler.Scheduler; +import org.apache.sling.commons.threads.ThreadPool; +import org.apache.sling.commons.threads.ThreadPoolManager; +import org.apache.sling.event.impl.jobs.config.JobManagerConfiguration; +import org.apache.sling.event.impl.jobs.config.QueueConfigurationManager; +import org.apache.sling.event.impl.jobs.config.TopologyCapabilities; +import org.apache.sling.event.impl.jobs.config.InternalQueueConfiguration; +import org.apache.sling.event.impl.support.Environment; +import org.apache.sling.event.jobs.jmx.QueuesMBean; +import org.apache.sling.event.impl.jobs.queues.QueueManager; +import org.apache.sling.event.impl.jobs.stats.StatisticsManager; +import org.apache.sling.event.jobs.Job; +import org.apache.sling.event.jobs.Job.JobState; +import org.apache.sling.event.jobs.consumer.JobConsumer; +import org.apache.sling.testing.mock.sling.junit.SlingContext; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.MockitoJUnitRunner; +import org.mockito.stubbing.Answer; +import org.osgi.service.condition.Condition; +import org.osgi.service.event.EventAdmin; + +import com.codahale.metrics.MetricRegistry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@RunWith(MockitoJUnitRunner.class) +public class JobManagerConditionTest { Review Comment: (similar comment to JobQueueImplTest : not seeing a lot of testing of the condition itself) ########## src/main/java/org/apache/sling/event/impl/jobs/queues/QueueManager.java: ########## @@ -82,7 +82,7 @@ public class QueueManager implements Runnable, EventHandler, ConfigurationChangeListener { - static QueueManager newForTest(EventAdmin eventAdmin, JobConsumerManager jobConsumerManager, + public static QueueManager newForTest(EventAdmin eventAdmin, JobConsumerManager jobConsumerManager, Review Comment: (same comment here about trying to avoid this change) ########## src/main/java/org/apache/sling/event/impl/jobs/queues/QueueManager.java: ########## @@ -161,6 +161,8 @@ protected void activate(final Map<String, Object> props) { queueServices.statisticsManager = statisticsManager; queueServices.eventingThreadPool = this.threadPool; this.configuration.addListener(this); + // Start jobs immediately on startup + this.maintain(); Review Comment: I don't think this is needed. upon change of the new condition to true we should try an immediate maintain, but this one here changes the logic in general ########## src/main/java/org/apache/sling/event/impl/jobs/queues/OutdatedJobQueueInfo.java: ########## @@ -23,7 +23,7 @@ /** * Encapsulates data required to survive queue outdating */ -class OutdatedJobQueueInfo { +public class OutdatedJobQueueInfo { Review Comment: wondering if this could be avoided (then again this is the implementation bundle, but if it can be avoided without much effort I'd prefer that) ########## src/test/java/org/apache/sling/event/impl/jobs/JobQueueImplTest.java: ########## @@ -0,0 +1,169 @@ +/* + * 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.sling.event.impl.jobs; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import org.apache.sling.event.impl.jobs.config.InternalQueueConfiguration; +import org.apache.sling.event.impl.jobs.config.JobManagerConfiguration; +import org.apache.sling.event.impl.jobs.queues.JobQueueImpl; +import org.apache.sling.event.impl.jobs.queues.OutdatedJobQueueInfo; +import org.apache.sling.event.impl.jobs.queues.QueueJobCache; +import org.apache.sling.event.impl.jobs.queues.QueueServices; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Map; + +public class JobQueueImplTest { + + private JobQueueImpl jobQueue; + private QueueServices services; + private JobManagerConfiguration configuration; + private Logger logger; + private String testQueue = "testQueue"; + + @Before + public void setUp() { + configuration = mock(JobManagerConfiguration.class); + services = spy(QueueServices.class); + logger = mock(Logger.class); + InternalQueueConfiguration internalConfig = mock(InternalQueueConfiguration.class); + services.configuration = configuration; + when(configuration.isEnable()).thenReturn(false); + when(internalConfig.getMaxParallel()).thenReturn(5); + when(internalConfig.getRetryDelayInMs()).thenReturn(1000L); + try { + Constructor<JobQueueImpl> constructor = JobQueueImpl.class.getDeclaredConstructor(String.class, InternalQueueConfiguration.class, QueueServices.class, QueueJobCache.class, OutdatedJobQueueInfo.class); + constructor.setAccessible(true); + + jobQueue = constructor.newInstance(testQueue, internalConfig, services, null, null); + } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + throw new RuntimeException(e); + } + // Use reflection to set the logger field + try { + Field loggerField = JobQueueImpl.class.getDeclaredField("logger"); + loggerField.setAccessible(true); + loggerField.set(jobQueue, logger); + } catch (IllegalAccessException | NoSuchFieldException e) { + throw new RuntimeException(e); + } + } + + @Test + public void testStartJobsWhenDisabled() { + jobQueue.startJobs(); + + verify(logger).debug("JobManager is disabled, skipping job starts for queue {}", testQueue); + verifyNoMoreInteractions(logger); Review Comment: I think we need more stable test that doesn't rely on this log output ########## src/test/java/org/apache/sling/event/impl/jobs/JobQueueImplTest.java: ########## @@ -0,0 +1,169 @@ +/* + * 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.sling.event.impl.jobs; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import org.apache.sling.event.impl.jobs.config.InternalQueueConfiguration; +import org.apache.sling.event.impl.jobs.config.JobManagerConfiguration; +import org.apache.sling.event.impl.jobs.queues.JobQueueImpl; +import org.apache.sling.event.impl.jobs.queues.OutdatedJobQueueInfo; +import org.apache.sling.event.impl.jobs.queues.QueueJobCache; +import org.apache.sling.event.impl.jobs.queues.QueueServices; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Map; + +public class JobQueueImplTest { Review Comment: general comment : I don't see this test really testing the logic a lot. It seems mostly testing indirect aspects rather than the influence of the new readiness condition ########## src/main/java/org/apache/sling/event/impl/jobs/config/JobManagerConfiguration.java: ########## @@ -199,6 +202,34 @@ static JobManagerConfiguration newForTest(ResourceResolverFactory resourceResolv /** The topology capabilities. */ private volatile TopologyCapabilities topologyCapabilities; + /** The condition that determines if the job manager is enabled. */ + @Reference( + target = "(osgi.condition.id=true)", + cardinality = ReferenceCardinality.OPTIONAL, + policy = ReferencePolicy.DYNAMIC + ) + private volatile Condition condition; + + /** + * Get the condition that determines if the job manager is enabled. + * @return The condition or null if the job manager is disabled + */ + public boolean isEnable() { + return condition != null; + } + + /** + * Notify all queues about condition change + */ + private void notifyConditionChange() { + synchronized ( this.listeners ) { + final TopologyCapabilities topologyCapabilities = this.topologyCapabilities; + for(final ConfigurationChangeListener listener : this.listeners) { + listener.configurationChanged(topologyCapabilities != null && condition != null); + } + } Review Comment: I don't fully understand the intention here. The method is currently not invokved, so it's a bit difficult to understand what it is intended to be used for. It seems it is a clone of `notifyListener` but has different idea of what `active` means. In any case: I don't think we should fiddle with the `active` state for example for JobScheduler. Jobs should still be scheduled even if the job starts are not allowed. ########## src/test/java/org/apache/sling/event/impl/jobs/JobQueueImplTest.java: ########## @@ -0,0 +1,169 @@ +/* + * 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.sling.event.impl.jobs; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import org.apache.sling.event.impl.jobs.config.InternalQueueConfiguration; +import org.apache.sling.event.impl.jobs.config.JobManagerConfiguration; +import org.apache.sling.event.impl.jobs.queues.JobQueueImpl; +import org.apache.sling.event.impl.jobs.queues.OutdatedJobQueueInfo; +import org.apache.sling.event.impl.jobs.queues.QueueJobCache; +import org.apache.sling.event.impl.jobs.queues.QueueServices; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Map; + +public class JobQueueImplTest { + + private JobQueueImpl jobQueue; + private QueueServices services; + private JobManagerConfiguration configuration; + private Logger logger; + private String testQueue = "testQueue"; + + @Before + public void setUp() { + configuration = mock(JobManagerConfiguration.class); + services = spy(QueueServices.class); + logger = mock(Logger.class); + InternalQueueConfiguration internalConfig = mock(InternalQueueConfiguration.class); + services.configuration = configuration; + when(configuration.isEnable()).thenReturn(false); + when(internalConfig.getMaxParallel()).thenReturn(5); + when(internalConfig.getRetryDelayInMs()).thenReturn(1000L); + try { + Constructor<JobQueueImpl> constructor = JobQueueImpl.class.getDeclaredConstructor(String.class, InternalQueueConfiguration.class, QueueServices.class, QueueJobCache.class, OutdatedJobQueueInfo.class); + constructor.setAccessible(true); + + jobQueue = constructor.newInstance(testQueue, internalConfig, services, null, null); + } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + throw new RuntimeException(e); + } + // Use reflection to set the logger field + try { + Field loggerField = JobQueueImpl.class.getDeclaredField("logger"); + loggerField.setAccessible(true); + loggerField.set(jobQueue, logger); + } catch (IllegalAccessException | NoSuchFieldException e) { + throw new RuntimeException(e); + } + } + + @Test + public void testStartJobsWhenDisabled() { + jobQueue.startJobs(); + + verify(logger).debug("JobManager is disabled, skipping job starts for queue {}", testQueue); + verifyNoMoreInteractions(logger); + } + + @Test + public void testStartJob() { + // Mock a JobHandler and its behavior + JobHandler jobHandler = mock(JobHandler.class, RETURNS_DEEP_STUBS); + String jobId = "testJob"; + when(jobHandler.getJob().getId()).thenReturn(jobId); + + // Use reflection to access the private processingJobsLists field + try { + Field processingJobsListsField = JobQueueImpl.class.getDeclaredField("processingJobsLists"); + processingJobsListsField.setAccessible(true); + @SuppressWarnings("unchecked") + Map<String, JobHandler> processingJobsLists = (Map<String, JobHandler>) processingJobsListsField.get(jobQueue); + processingJobsLists.put(testQueue, jobHandler); + } catch (Exception e) { + throw new RuntimeException("Failed to access processingJobsLists field", e); + } + + // Call the private startJob method using reflection + try { + Method startJobMethod = JobQueueImpl.class.getDeclaredMethod("startJob", JobHandler.class); + startJobMethod.setAccessible(true); + startJobMethod.invoke(jobQueue, jobHandler); + } catch (Exception e) { + throw new RuntimeException("Failed to invoke startJob method", e); + } + + // Verify the behavior + verify(logger).debug("JobManager is disabled, stopping job {} in queue {}", jobId, testQueue); + verifyNoMoreInteractions(logger); + } + + @Test + public void testQueueShutdown() { + // Enable the configuration + when(configuration.isEnable()).thenReturn(true); + + // Shut down the queue + jobQueue.close(); + + // Attempt to start jobs + jobQueue.startJobs(); + + // Verify that no jobs were started + verify(logger).info("Stopped job queue {}", testQueue); + verify(logger, never()).debug("Starting job queue {}", testQueue); Review Comment: (similar to a previous comment: I don't think we should rely on log output) ########## src/test/java/org/apache/sling/event/impl/jobs/JobQueueImplTest.java: ########## @@ -0,0 +1,169 @@ +/* + * 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.sling.event.impl.jobs; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import org.apache.sling.event.impl.jobs.config.InternalQueueConfiguration; +import org.apache.sling.event.impl.jobs.config.JobManagerConfiguration; +import org.apache.sling.event.impl.jobs.queues.JobQueueImpl; +import org.apache.sling.event.impl.jobs.queues.OutdatedJobQueueInfo; +import org.apache.sling.event.impl.jobs.queues.QueueJobCache; +import org.apache.sling.event.impl.jobs.queues.QueueServices; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Map; + +public class JobQueueImplTest { + + private JobQueueImpl jobQueue; + private QueueServices services; + private JobManagerConfiguration configuration; + private Logger logger; + private String testQueue = "testQueue"; + + @Before + public void setUp() { + configuration = mock(JobManagerConfiguration.class); + services = spy(QueueServices.class); + logger = mock(Logger.class); + InternalQueueConfiguration internalConfig = mock(InternalQueueConfiguration.class); + services.configuration = configuration; + when(configuration.isEnable()).thenReturn(false); + when(internalConfig.getMaxParallel()).thenReturn(5); + when(internalConfig.getRetryDelayInMs()).thenReturn(1000L); + try { + Constructor<JobQueueImpl> constructor = JobQueueImpl.class.getDeclaredConstructor(String.class, InternalQueueConfiguration.class, QueueServices.class, QueueJobCache.class, OutdatedJobQueueInfo.class); + constructor.setAccessible(true); + + jobQueue = constructor.newInstance(testQueue, internalConfig, services, null, null); + } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + throw new RuntimeException(e); + } + // Use reflection to set the logger field + try { + Field loggerField = JobQueueImpl.class.getDeclaredField("logger"); + loggerField.setAccessible(true); + loggerField.set(jobQueue, logger); + } catch (IllegalAccessException | NoSuchFieldException e) { + throw new RuntimeException(e); + } + } + + @Test + public void testStartJobsWhenDisabled() { + jobQueue.startJobs(); + + verify(logger).debug("JobManager is disabled, skipping job starts for queue {}", testQueue); + verifyNoMoreInteractions(logger); + } + + @Test + public void testStartJob() { + // Mock a JobHandler and its behavior + JobHandler jobHandler = mock(JobHandler.class, RETURNS_DEEP_STUBS); + String jobId = "testJob"; + when(jobHandler.getJob().getId()).thenReturn(jobId); + + // Use reflection to access the private processingJobsLists field + try { + Field processingJobsListsField = JobQueueImpl.class.getDeclaredField("processingJobsLists"); + processingJobsListsField.setAccessible(true); + @SuppressWarnings("unchecked") + Map<String, JobHandler> processingJobsLists = (Map<String, JobHandler>) processingJobsListsField.get(jobQueue); + processingJobsLists.put(testQueue, jobHandler); + } catch (Exception e) { + throw new RuntimeException("Failed to access processingJobsLists field", e); + } + + // Call the private startJob method using reflection + try { + Method startJobMethod = JobQueueImpl.class.getDeclaredMethod("startJob", JobHandler.class); + startJobMethod.setAccessible(true); + startJobMethod.invoke(jobQueue, jobHandler); + } catch (Exception e) { + throw new RuntimeException("Failed to invoke startJob method", e); + } + + // Verify the behavior + verify(logger).debug("JobManager is disabled, stopping job {} in queue {}", jobId, testQueue); + verifyNoMoreInteractions(logger); + } + + @Test + public void testQueueShutdown() { + // Enable the configuration + when(configuration.isEnable()).thenReturn(true); + + // Shut down the queue + jobQueue.close(); + + // Attempt to start jobs + jobQueue.startJobs(); + + // Verify that no jobs were started + verify(logger).info("Stopped job queue {}", testQueue); + verify(logger, never()).debug("Starting job queue {}", testQueue); + } + + @Test + public void testQueueStartupAndShutdown() { + // Enable the configuration + when(configuration.isEnable()).thenReturn(true); + // Mock a valid QueueJobCache + QueueJobCache mockCache = mock(QueueJobCache.class); + try { + Field cacheField = JobQueueImpl.class.getDeclaredField("cache"); + cacheField.setAccessible(true); + cacheField.set(jobQueue, mockCache); + } catch (Exception e) { + throw new RuntimeException("Failed to set mock cache", e); + } + + // Start jobs + jobQueue.startJobs(); + + try { + Field runningField = JobQueueImpl.class.getDeclaredField("running"); + runningField.setAccessible(true); + boolean isRunning = (boolean) runningField.get(jobQueue); + assertTrue(isRunning); + } catch (NoSuchFieldException | IllegalAccessException e) { + throw new RuntimeException("Failed to access running field", e); + } + + //stop the queue + jobQueue.close(); + try { + Field runningField = JobQueueImpl.class.getDeclaredField("running"); + runningField.setAccessible(true); + boolean isRunning = (boolean) runningField.get(jobQueue); + assertTrue(!isRunning); + } catch (NoSuchFieldException | IllegalAccessException e) { + throw new RuntimeException("Failed to access running field", e); + } Review Comment: two aspects: if we'd want to check for `running` then we might add a package-protected isRunning only used for tests - would make the test less brittle. the second one though: I'm wondering if this is really testing something. the running flag is a result of the calls to the constructor and close - without any involvement of the condition...? -- 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: dev-unsubscr...@sling.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org