fix(tests): remove JdbcTestUtils.drop usages as it defeats concurrent test runs

This commit is contained in:
brian.mulier
2025-11-26 11:38:57 +01:00
committed by brian-mulier-p
parent 98678deabb
commit c3d94dc8ff
13 changed files with 34 additions and 154 deletions

View File

@@ -1,20 +1,6 @@
package io.kestra.repository.h2;
import io.kestra.core.repositories.AbstractExecutionServiceTest;
import io.kestra.jdbc.JdbcTestUtils;
import jakarta.inject.Inject;
import org.junit.jupiter.api.BeforeEach;
import java.io.IOException;
import java.net.URISyntaxException;
class H2ExecutionServiceTest extends AbstractExecutionServiceTest {
@Inject
JdbcTestUtils jdbcTestUtils;
@BeforeEach
protected void init() throws IOException, URISyntaxException {
jdbcTestUtils.drop();
jdbcTestUtils.migrate();
}
}
}

View File

@@ -1,20 +1,6 @@
package io.kestra.repository.mysql;
import io.kestra.core.repositories.AbstractExecutionServiceTest;
import io.kestra.jdbc.JdbcTestUtils;
import jakarta.inject.Inject;
import org.junit.jupiter.api.BeforeEach;
import java.io.IOException;
import java.net.URISyntaxException;
public class MysqlExecutionServiceTest extends AbstractExecutionServiceTest {
@Inject
JdbcTestUtils jdbcTestUtils;
@BeforeEach
protected void init() throws IOException, URISyntaxException {
jdbcTestUtils.drop();
jdbcTestUtils.migrate();
}
}

View File

@@ -4,17 +4,13 @@ import io.kestra.core.models.flows.Flow;
import io.kestra.core.models.triggers.multipleflows.AbstractMultipleConditionStorageTest;
import io.kestra.core.models.triggers.multipleflows.MultipleConditionStorageInterface;
import io.kestra.core.models.triggers.multipleflows.MultipleConditionWindow;
import io.kestra.jdbc.JdbcTestUtils;
import io.kestra.repository.mysql.MysqlRepository;
import jakarta.inject.Inject;
import jakarta.inject.Named;
import org.junit.jupiter.api.BeforeEach;
import java.util.List;
class MysqlMultipleConditionStorageTest extends AbstractMultipleConditionStorageTest {
@Inject
JdbcTestUtils jdbcTestUtils;
@Inject
@Named("multipleconditions")
@@ -27,11 +23,4 @@ class MysqlMultipleConditionStorageTest extends AbstractMultipleConditionStorage
protected void save(MultipleConditionStorageInterface multipleConditionStorage, Flow flow, List<MultipleConditionWindow> multipleConditionWindows) {
multipleConditionStorage.save(multipleConditionWindows);
}
@BeforeEach
protected void init() {
jdbcTestUtils.drop();
jdbcTestUtils.migrate();
}
}
}

View File

@@ -1,20 +1,6 @@
package io.kestra.repository.postgres;
import io.kestra.core.repositories.AbstractExecutionServiceTest;
import io.kestra.jdbc.JdbcTestUtils;
import jakarta.inject.Inject;
import org.junit.jupiter.api.BeforeEach;
import java.io.IOException;
import java.net.URISyntaxException;
class PostgresExecutionServiceTest extends AbstractExecutionServiceTest {
@Inject
JdbcTestUtils jdbcTestUtils;
@BeforeEach
protected void init() throws IOException, URISyntaxException {
jdbcTestUtils.drop();
jdbcTestUtils.migrate();
}
}
}

View File

@@ -4,17 +4,13 @@ import io.kestra.core.models.flows.Flow;
import io.kestra.core.models.triggers.multipleflows.AbstractMultipleConditionStorageTest;
import io.kestra.core.models.triggers.multipleflows.MultipleConditionStorageInterface;
import io.kestra.core.models.triggers.multipleflows.MultipleConditionWindow;
import io.kestra.jdbc.JdbcTestUtils;
import io.kestra.repository.postgres.PostgresRepository;
import jakarta.inject.Inject;
import jakarta.inject.Named;
import org.junit.jupiter.api.BeforeEach;
import java.util.List;
class PostgresMultipleConditionStorageTest extends AbstractMultipleConditionStorageTest {
@Inject
JdbcTestUtils jdbcTestUtils;
@Inject
@Named("multipleconditions")
@@ -27,11 +23,4 @@ class PostgresMultipleConditionStorageTest extends AbstractMultipleConditionStor
protected void save(MultipleConditionStorageInterface multipleConditionStorage, Flow flow, List<MultipleConditionWindow> multipleConditionWindows) {
multipleConditionStorage.save(multipleConditionWindows);
}
@BeforeEach
protected void init() {
jdbcTestUtils.drop();
jdbcTestUtils.migrate();
}
}
}

View File

@@ -3,16 +3,14 @@ package io.kestra.jdbc;
import io.micronaut.flyway.FlywayConfigurationProperties;
import io.micronaut.flyway.FlywayMigrator;
import jakarta.annotation.PostConstruct;
import lombok.SneakyThrows;
import org.jooq.DSLContext;
import jakarta.inject.Inject;
import jakarta.inject.Singleton;
import lombok.SneakyThrows;
import org.jooq.DSLContext;
import org.jooq.Table;
import org.jooq.impl.DSL;
import javax.sql.DataSource;
import java.util.List;
import static io.kestra.core.utils.Rethrow.throwPredicate;
@@ -46,7 +44,7 @@ public class JdbcTestUtils {
.getTables()
.stream()
.filter(throwPredicate(table -> (table.getSchema().getName().equals(dataSource.getConnection().getCatalog())) ||
table.getSchema().getName().equals("public") || // for Postgres
table.getSchema().getName().equals("public") || // for Postgres
table.getSchema().getName().equals("dbo") // for SQLServer
))
.filter(table -> tableConfigs.getTableConfigs().stream().anyMatch(conf -> conf.table().equalsIgnoreCase(table.getName())))
@@ -54,6 +52,10 @@ public class JdbcTestUtils {
});
}
/**
* This should never be used ideally in OSS as it defeats the concurrent test runs and may drop a table in the middle of another test
*/
@Deprecated
@SneakyThrows
public void drop() {
dslContextWrapper.transaction((configuration) -> {
@@ -62,6 +64,11 @@ public class JdbcTestUtils {
this.tables.forEach(t -> dslContext.delete(t).execute());
});
}
/**
* This should never be used ideally in OSS as it defeats the concurrent test runs and may drop a table in the middle of another test
*/
@Deprecated
public void migrate() {
dslContextWrapper.transaction((configuration) -> {
flywayMigrator.run(config, dataSource);

View File

@@ -1,17 +1,6 @@
package io.kestra.jdbc.repository;
import io.kestra.core.repositories.AbstractKvMetadataRepositoryTest;
import io.kestra.jdbc.JdbcTestUtils;
import jakarta.inject.Inject;
import org.junit.jupiter.api.BeforeEach;
public abstract class AbstractJdbcKvMetadataRepositoryTest extends AbstractKvMetadataRepositoryTest {
@Inject
JdbcTestUtils jdbcTestUtils;
@BeforeEach
protected void init() {
jdbcTestUtils.drop();
jdbcTestUtils.migrate();
}
}

View File

@@ -1,20 +1,16 @@
package io.kestra.jdbc.repository;
import io.kestra.core.server.ServerConfig;
import io.kestra.core.server.ServerInstance;
import io.kestra.core.server.Service;
import io.kestra.core.server.ServiceInstance;
import io.kestra.core.server.ServiceStateTransition;
import io.kestra.core.server.ServiceType;
import io.kestra.core.server.WorkerTaskRestartStrategy;
import io.kestra.core.junit.annotations.KestraTest;
import io.kestra.core.server.*;
import io.kestra.core.utils.IdUtils;
import io.kestra.core.utils.Network;
import io.kestra.jdbc.JdbcTestUtils;
import io.kestra.core.junit.annotations.KestraTest;
import jakarta.inject.Inject;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.parallel.Execution;
import org.junit.jupiter.api.parallel.ExecutionMode;
import java.time.Duration;
import java.time.Instant;
@@ -23,8 +19,6 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import org.junit.jupiter.api.parallel.Execution;
import org.junit.jupiter.api.parallel.ExecutionMode;
import static io.kestra.core.server.ServiceStateTransition.Result.FAILED;
import static io.kestra.core.server.ServiceStateTransition.Result.SUCCEEDED;
@@ -261,4 +255,4 @@ public abstract class AbstractJdbcServiceInstanceRepositoryTest {
);
}
}
}
}

View File

@@ -1,12 +1,11 @@
package io.kestra.jdbc.runner;
import io.kestra.core.junit.annotations.KestraTest;
import io.kestra.core.runners.DeserializationIssuesCaseTest;
import io.kestra.core.utils.IdUtils;
import io.kestra.jdbc.JdbcTableConfigs;
import io.kestra.jdbc.JdbcTestUtils;
import io.kestra.jdbc.JooqDSLContextWrapper;
import io.kestra.jdbc.repository.AbstractJdbcRepository;
import io.kestra.core.junit.annotations.KestraTest;
import jakarta.inject.Inject;
import org.jooq.*;
import org.jooq.Record;
@@ -19,26 +18,18 @@ import java.util.HashMap;
import java.util.Map;
@KestraTest(startRunner = true)
@TestInstance(TestInstance.Lifecycle.PER_CLASS) // must be per-class to allow calling once init() which took a lot of time
@TestInstance(TestInstance.Lifecycle.PER_CLASS)
// must be per-class to allow calling once init() which took a lot of time
public abstract class AbstractJdbcDeserializationIssuesTest {
@Inject
private DeserializationIssuesCaseTest deserializationIssuesCaseTest;
@Inject
private JdbcTestUtils jdbcTestUtils;
@Inject
private JooqDSLContextWrapper dslContextWrapper;
@Inject
private JdbcTableConfigs jdbcTableConfigs;
@BeforeAll
void init() {
jdbcTestUtils.drop();
jdbcTestUtils.migrate();
}
@Test
void workerTaskDeserializationIssue() throws Exception {
deserializationIssuesCaseTest.workerTaskDeserializationIssue(this::sendToQueue);

View File

@@ -1,5 +1,6 @@
package io.kestra.jdbc.runner;
import io.kestra.core.junit.annotations.KestraTest;
import io.kestra.core.models.flows.FlowInterface;
import io.kestra.core.models.flows.FlowWithSource;
import io.kestra.core.models.property.Property;
@@ -8,14 +9,11 @@ import io.kestra.core.queues.QueueFactoryInterface;
import io.kestra.core.queues.QueueInterface;
import io.kestra.core.runners.Indexer;
import io.kestra.core.runners.WorkerTaskResult;
import io.kestra.core.utils.IdUtils;
import io.kestra.core.utils.TestsUtils;
import io.kestra.plugin.core.debug.Return;
import io.kestra.core.utils.IdUtils;
import io.kestra.jdbc.JdbcTestUtils;
import io.kestra.core.junit.annotations.KestraTest;
import jakarta.inject.Inject;
import jakarta.inject.Named;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import reactor.core.publisher.Flux;
@@ -37,9 +35,6 @@ abstract public class JdbcQueueTest {
@Named(QueueFactoryInterface.WORKERTASKRESULT_NAMED)
protected QueueInterface<WorkerTaskResult> workerTaskResultQueue;
@Inject
JdbcTestUtils jdbcTestUtils;
@Test
void noGroup() throws InterruptedException, QueueException {
CountDownLatch countDownLatch = new CountDownLatch(2);
@@ -133,10 +128,4 @@ abstract public class JdbcQueueTest {
.tasks(Collections.singletonList(Return.builder().id("test").type(Return.class.getName()).format(Property.ofValue("test")).build()))
.build();
}
@BeforeEach
protected void init() {
jdbcTestUtils.drop();
jdbcTestUtils.migrate();
}
}
}

View File

@@ -9,8 +9,6 @@ import io.kestra.core.queues.QueueException;
import io.kestra.core.runners.AbstractRunnerTest;
import io.kestra.core.runners.InputsTest;
import io.kestra.core.utils.TestsUtils;
import io.kestra.jdbc.JdbcTestUtils;
import jakarta.inject.Inject;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.junitpioneer.jupiter.RetryingTest;
@@ -32,15 +30,6 @@ public abstract class JdbcRunnerTest extends AbstractRunnerTest {
public static final String NAMESPACE = "io.kestra.tests";
@Inject
private JdbcTestUtils jdbcTestUtils;
@BeforeAll
public void init(){
jdbcTestUtils.drop();
jdbcTestUtils.migrate();
}
@Test
@LoadFlows({"flows/valids/waitfor-child-task-warning.yaml"})
void waitForChildTaskWarning() throws Exception {

View File

@@ -8,7 +8,6 @@ import io.kestra.core.models.conditions.ConditionContext;
import io.kestra.core.models.executions.Execution;
import io.kestra.core.models.executions.TaskRun;
import io.kestra.core.models.flows.Flow;
import io.kestra.core.models.flows.State;
import io.kestra.core.models.flows.State.Type;
import io.kestra.core.models.tasks.ResolvedTask;
import io.kestra.core.models.tasks.WorkerGroup;
@@ -21,7 +20,6 @@ import io.kestra.core.services.WorkerGroupService;
import io.kestra.core.tasks.test.SleepTrigger;
import io.kestra.core.utils.IdUtils;
import io.kestra.core.utils.TestsUtils;
import io.kestra.jdbc.JdbcTestUtils;
import io.kestra.jdbc.repository.AbstractJdbcWorkerJobRunningRepository;
import io.kestra.plugin.core.flow.Sleep;
import io.micronaut.context.ApplicationContext;
@@ -29,7 +27,10 @@ import io.micronaut.context.annotation.Property;
import io.micronaut.test.annotation.MockBean;
import jakarta.inject.Inject;
import jakarta.inject.Named;
import org.junit.jupiter.api.*;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInstance;
import reactor.core.publisher.Flux;
import java.time.Duration;
@@ -44,16 +45,14 @@ import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
@KestraTest(environments = {"test", "liveness"}, startRunner = true, startWorker = false)
@TestInstance(TestInstance.Lifecycle.PER_CLASS) // must be per-class to allow calling once init() which took a lot of time
@KestraTest(environments = {"test", "liveness"}, startRunner = true, startWorker = false)
@TestInstance(TestInstance.Lifecycle.PER_CLASS)
// must be per-class to allow calling once init() which took a lot of time
@Property(name = "kestra.server-type", value = "EXECUTOR")
public abstract class JdbcServiceLivenessCoordinatorTest {
@Inject
private ApplicationContext applicationContext;
@Inject
private JdbcTestUtils jdbcTestUtils;
@Inject
private TestRunContextFactory runContextFactory;
@@ -84,9 +83,6 @@ public abstract class JdbcServiceLivenessCoordinatorTest {
@BeforeAll
void init() {
jdbcTestUtils.drop();
jdbcTestUtils.migrate();
// Simulate that executor and workers are not running on the same JVM.
jdbcServiceLivenessHandler.setServerInstance(IdUtils.create());
}

View File

@@ -5,7 +5,6 @@ import io.kestra.core.junit.annotations.LoadFlows;
import io.kestra.core.models.executions.Execution;
import io.kestra.core.models.flows.State;
import io.kestra.core.trace.TraceUtils;
import io.kestra.jdbc.JdbcTestUtils;
import io.micronaut.http.HttpRequest;
import io.micronaut.http.MediaType;
import io.micronaut.http.client.annotation.Client;
@@ -17,7 +16,6 @@ import io.opentelemetry.api.trace.Tracer;
import io.opentelemetry.sdk.testing.junit5.OpenTelemetryExtension;
import io.opentelemetry.sdk.trace.data.SpanData;
import jakarta.inject.Inject;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.RegisterExtension;
@@ -34,15 +32,6 @@ public class TracesTest {
@Client("/")
private ReactorHttpClient client;
@Inject
private JdbcTestUtils jdbcTestUtils;
@AfterEach
protected void setup() {
jdbcTestUtils.drop();
jdbcTestUtils.migrate();
}
@Test
@LoadFlows({"flows/valids/minimal.yaml"})
void runningAFlowShouldGenerateTraces() {