mirror of
https://github.com/kestra-io/kestra.git
synced 2025-12-25 02:14:38 -05:00
fix(system): avoid creating multiple worker job queue
We created miltiple worker job queue because the bean was in the prototype scope. This was needed only for tests as they are closing it. Switching to singleton and rebuilding the context of the test that needs it fixes the issue.
This commit is contained in:
@@ -1066,18 +1066,16 @@ public class Worker implements Service, Runnable, AutoCloseable {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method should only be used on tests.
|
||||||
|
* It shut down the worker without waiting for tasks to end,
|
||||||
|
* and without closing the queue, so tests can launch and shutdown a worker manually without closing the queue.
|
||||||
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public void shutdown() {
|
public void shutdown() {
|
||||||
// initiate shutdown
|
// initiate shutdown
|
||||||
shutdown.compareAndSet(false, true);
|
shutdown.compareAndSet(false, true);
|
||||||
|
|
||||||
try {
|
|
||||||
// close the WorkerJob queue to stop receiving new JobTask execution.
|
|
||||||
workerJobQueue.close();
|
|
||||||
} catch (IOException e) {
|
|
||||||
log.error("Failed to close the WorkerJobQueue");
|
|
||||||
}
|
|
||||||
|
|
||||||
// close all queues and shutdown now
|
// close all queues and shutdown now
|
||||||
this.receiveCancellations.forEach(Runnable::run);
|
this.receiveCancellations.forEach(Runnable::run);
|
||||||
this.executorService.shutdownNow();
|
this.executorService.shutdownNow();
|
||||||
|
|||||||
@@ -23,6 +23,7 @@ import jakarta.inject.Inject;
|
|||||||
import jakarta.inject.Named;
|
import jakarta.inject.Named;
|
||||||
import org.junit.jupiter.api.Assertions;
|
import org.junit.jupiter.api.Assertions;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
import org.junit.jupiter.api.TestInstance;
|
||||||
import reactor.core.publisher.Flux;
|
import reactor.core.publisher.Flux;
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
@@ -35,7 +36,7 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||||||
import static io.kestra.core.utils.Rethrow.throwSupplier;
|
import static io.kestra.core.utils.Rethrow.throwSupplier;
|
||||||
import static org.assertj.core.api.Assertions.assertThat;
|
import static org.assertj.core.api.Assertions.assertThat;
|
||||||
|
|
||||||
@KestraTest
|
@KestraTest(rebuildContext = true)
|
||||||
class WorkerTest {
|
class WorkerTest {
|
||||||
@Inject
|
@Inject
|
||||||
ApplicationContext applicationContext;
|
ApplicationContext applicationContext;
|
||||||
|
|||||||
@@ -33,7 +33,7 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
|
||||||
@KestraTest
|
@KestraTest(rebuildContext = true)
|
||||||
abstract public class AbstractSchedulerTest {
|
abstract public class AbstractSchedulerTest {
|
||||||
@Inject
|
@Inject
|
||||||
protected ApplicationContext applicationContext;
|
protected ApplicationContext applicationContext;
|
||||||
|
|||||||
@@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit;
|
|||||||
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
|
||||||
@KestraTest
|
@KestraTest(rebuildContext = true)
|
||||||
class TriggerTest {
|
class TriggerTest {
|
||||||
@Inject
|
@Inject
|
||||||
private ApplicationContext applicationContext;
|
private ApplicationContext applicationContext;
|
||||||
|
|||||||
@@ -42,7 +42,7 @@ public class H2QueueFactory implements QueueFactoryInterface {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@Prototype // must be prototype so we can create two Worker in the same application context for testing purpose.
|
@Singleton
|
||||||
@Named(QueueFactoryInterface.WORKERJOB_NAMED)
|
@Named(QueueFactoryInterface.WORKERJOB_NAMED)
|
||||||
@Bean(preDestroy = "close")
|
@Bean(preDestroy = "close")
|
||||||
public QueueInterface<WorkerJob> workerJob() {
|
public QueueInterface<WorkerJob> workerJob() {
|
||||||
|
|||||||
@@ -42,7 +42,7 @@ public class MysqlQueueFactory implements QueueFactoryInterface {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@Prototype // must be prototype so we can create two Worker in the same application context for testing purpose.
|
@Singleton
|
||||||
@Named(QueueFactoryInterface.WORKERJOB_NAMED)
|
@Named(QueueFactoryInterface.WORKERJOB_NAMED)
|
||||||
@Bean(preDestroy = "close")
|
@Bean(preDestroy = "close")
|
||||||
public QueueInterface<WorkerJob> workerJob() {
|
public QueueInterface<WorkerJob> workerJob() {
|
||||||
|
|||||||
@@ -42,7 +42,7 @@ public class PostgresQueueFactory implements QueueFactoryInterface {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@Prototype // must be prototype so we can create two Worker in the same application context for testing purpose.
|
@Singleton
|
||||||
@Named(QueueFactoryInterface.WORKERJOB_NAMED)
|
@Named(QueueFactoryInterface.WORKERJOB_NAMED)
|
||||||
@Bean(preDestroy = "close")
|
@Bean(preDestroy = "close")
|
||||||
public QueueInterface<WorkerJob> workerJob() {
|
public QueueInterface<WorkerJob> workerJob() {
|
||||||
|
|||||||
@@ -111,7 +111,7 @@ public abstract class JdbcServiceLivenessCoordinatorTest {
|
|||||||
CountDownLatch resubmitLatch = new CountDownLatch(1);
|
CountDownLatch resubmitLatch = new CountDownLatch(1);
|
||||||
|
|
||||||
// create first worker
|
// create first worker
|
||||||
Worker worker = applicationContext.createBean(Worker.class, IdUtils.create(), 1, null);
|
Worker worker = applicationContext.createBean(TestMethodScopedWorker.class, IdUtils.create(), 1, null);
|
||||||
worker.run();
|
worker.run();
|
||||||
|
|
||||||
Flux<WorkerTaskResult> receive = TestsUtils.receive(workerTaskResultQueue, either -> {
|
Flux<WorkerTaskResult> receive = TestsUtils.receive(workerTaskResultQueue, either -> {
|
||||||
@@ -130,7 +130,7 @@ public abstract class JdbcServiceLivenessCoordinatorTest {
|
|||||||
worker.shutdown(); // stop processing task
|
worker.shutdown(); // stop processing task
|
||||||
|
|
||||||
// create second worker (this will revoke previously one).
|
// create second worker (this will revoke previously one).
|
||||||
Worker newWorker = applicationContext.createBean(Worker.class, IdUtils.create(), 1, null);
|
Worker newWorker = applicationContext.createBean(TestMethodScopedWorker.class, IdUtils.create(), 1, null);
|
||||||
newWorker.run();
|
newWorker.run();
|
||||||
boolean resubmitLatchAwait = resubmitLatch.await(10, TimeUnit.SECONDS);
|
boolean resubmitLatchAwait = resubmitLatch.await(10, TimeUnit.SECONDS);
|
||||||
assertThat(resubmitLatchAwait).isTrue();
|
assertThat(resubmitLatchAwait).isTrue();
|
||||||
@@ -147,7 +147,7 @@ public abstract class JdbcServiceLivenessCoordinatorTest {
|
|||||||
CountDownLatch resubmitLatch = new CountDownLatch(1);
|
CountDownLatch resubmitLatch = new CountDownLatch(1);
|
||||||
|
|
||||||
// create first worker
|
// create first worker
|
||||||
Worker worker = applicationContext.createBean(Worker.class, IdUtils.create(), 1, "workerGroupKey");
|
Worker worker = applicationContext.createBean(TestMethodScopedWorker.class, IdUtils.create(), 1, "workerGroupKey");
|
||||||
worker.run();
|
worker.run();
|
||||||
|
|
||||||
Flux<WorkerTaskResult> receive = TestsUtils.receive(workerTaskResultQueue, either -> {
|
Flux<WorkerTaskResult> receive = TestsUtils.receive(workerTaskResultQueue, either -> {
|
||||||
@@ -166,7 +166,7 @@ public abstract class JdbcServiceLivenessCoordinatorTest {
|
|||||||
worker.shutdown(); // stop processing task
|
worker.shutdown(); // stop processing task
|
||||||
|
|
||||||
// create second worker (this will revoke previously one).
|
// create second worker (this will revoke previously one).
|
||||||
Worker newWorker = applicationContext.createBean(Worker.class, IdUtils.create(), 1, "workerGroupKey");
|
Worker newWorker = applicationContext.createBean(TestMethodScopedWorker.class, IdUtils.create(), 1, "workerGroupKey");
|
||||||
newWorker.run();
|
newWorker.run();
|
||||||
boolean resubmitLatchAwait = resubmitLatch.await(10, TimeUnit.SECONDS);
|
boolean resubmitLatchAwait = resubmitLatch.await(10, TimeUnit.SECONDS);
|
||||||
assertThat(resubmitLatchAwait).isTrue();
|
assertThat(resubmitLatchAwait).isTrue();
|
||||||
@@ -181,7 +181,7 @@ public abstract class JdbcServiceLivenessCoordinatorTest {
|
|||||||
void taskResubmitSkipExecution() throws Exception {
|
void taskResubmitSkipExecution() throws Exception {
|
||||||
CountDownLatch runningLatch = new CountDownLatch(1);
|
CountDownLatch runningLatch = new CountDownLatch(1);
|
||||||
|
|
||||||
Worker worker = applicationContext.createBean(Worker.class, IdUtils.create(), 8, null);
|
Worker worker = applicationContext.createBean(TestMethodScopedWorker.class, IdUtils.create(), 8, null);
|
||||||
worker.run();
|
worker.run();
|
||||||
|
|
||||||
WorkerTask workerTask = workerTask(Duration.ofSeconds(5));
|
WorkerTask workerTask = workerTask(Duration.ofSeconds(5));
|
||||||
@@ -203,7 +203,7 @@ public abstract class JdbcServiceLivenessCoordinatorTest {
|
|||||||
assertThat(runningLatchAwait).isTrue();
|
assertThat(runningLatchAwait).isTrue();
|
||||||
worker.shutdown();
|
worker.shutdown();
|
||||||
|
|
||||||
Worker newWorker = applicationContext.createBean(Worker.class, IdUtils.create(), 1, null);
|
Worker newWorker = applicationContext.createBean(TestMethodScopedWorker.class, IdUtils.create(), 1, null);
|
||||||
newWorker.run();
|
newWorker.run();
|
||||||
|
|
||||||
// wait a little to be sure there is no resubmit
|
// wait a little to be sure there is no resubmit
|
||||||
@@ -215,7 +215,7 @@ public abstract class JdbcServiceLivenessCoordinatorTest {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
void shouldReEmitTriggerWhenWorkerIsDetectedAsNonResponding() throws Exception {
|
void shouldReEmitTriggerWhenWorkerIsDetectedAsNonResponding() throws Exception {
|
||||||
Worker worker = applicationContext.createBean(Worker.class, IdUtils.create(), 1, null);
|
Worker worker = applicationContext.createBean(TestMethodScopedWorker.class, IdUtils.create(), 1, null);
|
||||||
worker.run();
|
worker.run();
|
||||||
|
|
||||||
WorkerTrigger workerTrigger = workerTrigger(Duration.ofSeconds(5));
|
WorkerTrigger workerTrigger = workerTrigger(Duration.ofSeconds(5));
|
||||||
@@ -236,7 +236,7 @@ public abstract class JdbcServiceLivenessCoordinatorTest {
|
|||||||
receiveTrigger.blockLast();
|
receiveTrigger.blockLast();
|
||||||
worker.shutdown();
|
worker.shutdown();
|
||||||
|
|
||||||
Worker newWorker = applicationContext.createBean(Worker.class, IdUtils.create(), 1, null);
|
Worker newWorker = applicationContext.createBean(TestMethodScopedWorker.class, IdUtils.create(), 1, null);
|
||||||
newWorker.run();
|
newWorker.run();
|
||||||
assertThat(countDownLatch.await(30, TimeUnit.SECONDS)).isTrue();
|
assertThat(countDownLatch.await(30, TimeUnit.SECONDS)).isTrue();
|
||||||
|
|
||||||
@@ -246,7 +246,7 @@ public abstract class JdbcServiceLivenessCoordinatorTest {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
void shouldReEmitTriggerToTheSameWorkerGroup() throws Exception {
|
void shouldReEmitTriggerToTheSameWorkerGroup() throws Exception {
|
||||||
Worker worker = applicationContext.createBean(Worker.class, IdUtils.create(), 1, "workerGroupKey");
|
Worker worker = applicationContext.createBean(TestMethodScopedWorker.class, IdUtils.create(), 1, "workerGroupKey");
|
||||||
worker.run();
|
worker.run();
|
||||||
|
|
||||||
WorkerTrigger workerTrigger = workerTrigger(Duration.ofSeconds(5), "workerGroupKey");
|
WorkerTrigger workerTrigger = workerTrigger(Duration.ofSeconds(5), "workerGroupKey");
|
||||||
@@ -267,7 +267,7 @@ public abstract class JdbcServiceLivenessCoordinatorTest {
|
|||||||
receiveTrigger.blockLast();
|
receiveTrigger.blockLast();
|
||||||
worker.shutdown();
|
worker.shutdown();
|
||||||
|
|
||||||
Worker newWorker = applicationContext.createBean(Worker.class, IdUtils.create(), 1, "workerGroupKey");
|
Worker newWorker = applicationContext.createBean(TestMethodScopedWorker.class, IdUtils.create(), 1, "workerGroupKey");
|
||||||
newWorker.run();
|
newWorker.run();
|
||||||
assertThat(countDownLatch.await(30, TimeUnit.SECONDS)).isTrue();
|
assertThat(countDownLatch.await(30, TimeUnit.SECONDS)).isTrue();
|
||||||
|
|
||||||
|
|||||||
Reference in New Issue
Block a user