Compare commits

..

8 Commits

Author SHA1 Message Date
brian.mulier
b3e858ffb8 fix(tests): wait for KILL in TestExecution cleaner 2025-11-07 20:24:10 +01:00
brian.mulier
94fa411b2b fix(tests): remove ThreadLocal + add distinct on TestExecution cleaner 2025-11-07 16:47:58 +01:00
brian.mulier
91486169e1 fix(tests): add TestQueueFactory setup in beforeEach
Otherwise the rebuildContext was creating a new applicationContext that doesn't contain the testExecutions
2025-11-07 16:15:02 +01:00
brian.mulier
0ca9a90ebf feat(tests): intercept created executions through queue proxy & kill them if running after test 2025-11-07 16:15:02 +01:00
Miloš Paunović
58e5efe767 refactor(core): uniform .gitignore file for javascript (#12802) 2025-11-07 14:09:41 +01:00
Miloš Paunović
c3c46ae336 chore(flows): amend flow export filename to include namespace and id parameters (#12800)
Closes https://github.com/kestra-io/kestra/issues/12790.
2025-11-07 13:57:33 +01:00
Miloš Paunović
f8bb59f76e refactor(core): replace soon-to-be-deprecated button attribute (#12796)
Resolving console warnings.

https://element-plus.org/en-US/component/button#link-button
2025-11-07 13:29:40 +01:00
Miloš Paunović
0c4425b030 chore(deps): regular dependency update (#12785)
Performing a weekly round of dependency updates in the NPM ecosystem to keep everything up to date.
2025-11-07 11:38:46 +01:00
24 changed files with 835 additions and 529 deletions

7
.gitignore vendored
View File

@@ -32,12 +32,13 @@ ui/node_modules
ui/.env.local
ui/.env.*.local
webserver/src/main/resources/ui
yarn.lock
webserver/src/main/resources/views
ui/coverage
ui/stats.html
ui/.frontend-gradle-plugin
ui/utils/CHANGELOG.md
ui/test-report.junit.xml
*storybook.log
storybook-static
### Docker
/.env
@@ -57,6 +58,4 @@ core/src/main/resources/gradle.properties
# Allure Reports
**/allure-results/*
*storybook.log
storybook-static
/jmh-benchmarks/src/main/resources/gradle.properties

View File

@@ -3,6 +3,7 @@ package io.kestra.core.queues;
import io.kestra.core.exceptions.DeserializationException;
import io.kestra.core.models.Pauseable;
import io.kestra.core.utils.Either;
import org.apache.commons.lang3.NotImplementedException;
import java.io.Closeable;
import java.util.List;
@@ -54,4 +55,20 @@ public interface QueueInterface<T> extends Closeable, Pauseable {
}
Runnable receive(String consumerGroup, Class<?> queueType, Consumer<Either<T, DeserializationException>> consumer, boolean forUpdate);
default void deleteByKey(String key) throws QueueException {
throw new NotImplementedException();
}
default void deleteByKeys(List<String> keys) throws QueueException {
throw new NotImplementedException();
}
default void emitOnly(String consumerGroup, T message) throws QueueException {
throw new NotImplementedException();
}
default Runnable receiveBatch(Class<?> queueType, Consumer<List<Either<T, DeserializationException>>> consumer) {
throw new NotImplementedException();
}
}

View File

@@ -15,6 +15,7 @@ import io.kestra.core.services.FlowListenersInterface;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
@@ -30,8 +31,8 @@ public class FlowListeners implements FlowListenersInterface {
private final AtomicBoolean isStarted = new AtomicBoolean(false);
private final QueueInterface<FlowInterface> flowQueue;
private final List<FlowWithSource> flows;
private final List<Consumer<List<FlowWithSource>>> consumers = new ArrayList<>();
private final List<BiConsumer<FlowWithSource, FlowWithSource>> consumersEach = new ArrayList<>();
private final List<Consumer<List<FlowWithSource>>> consumers = new CopyOnWriteArrayList<>();
private final List<BiConsumer<FlowWithSource, FlowWithSource>> consumersEach = new CopyOnWriteArrayList<>();
private final PluginDefaultService pluginDefaultService;

View File

@@ -2,6 +2,10 @@ package io.kestra.core.utils;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Function;
import java.util.function.Predicate;
public class ListUtils {
public static <T> List<T> emptyOnNull(List<T> list) {
@@ -71,4 +75,13 @@ public class ListUtils {
throw new IllegalArgumentException("%s in not an instance of List of String".formatted(object));
}
}
private static <T> Predicate<T> distinctByKeyPredicate(Function<? super T,Object> keyExtractor) {
Map<Object,Boolean> seen = new ConcurrentHashMap<>();
return t -> seen.putIfAbsent(keyExtractor.apply(t), Boolean.TRUE) == null;
}
public static <T> List<T> distinctByKey(List<T> withDuplicates, Function<? super T, Object> keyExtractor) {
return withDuplicates.stream().filter(distinctByKeyPredicate(keyExtractor)).toList();
}
}

View File

@@ -14,13 +14,10 @@ import java.util.Date;
import java.util.Map;
import jakarta.inject.Inject;
import org.junit.jupiter.api.TestInstance;
import org.junit.jupiter.api.TestInstance.Lifecycle;
import static org.assertj.core.api.Assertions.assertThat;
@KestraTest
@TestInstance(Lifecycle.PER_CLASS)
class DateFilterTest {
public static final ZonedDateTime NOW = ZonedDateTime.parse("2013-09-08T16:19:12.123456+01");

View File

@@ -1,7 +1,11 @@
package io.kestra.core.utils;
import io.kestra.core.models.executions.Execution;
import io.kestra.core.models.flows.State;
import org.junit.jupiter.api.Test;
import java.time.Duration;
import java.time.Instant;
import java.util.Collections;
import java.util.List;
@@ -52,4 +56,19 @@ class ListUtilsTest {
assertThrows(IllegalArgumentException.class, () -> ListUtils.convertToListString("not a list"));
assertThrows(IllegalArgumentException.class, () -> ListUtils.convertToListString(List.of(1, 2, 3)));
}
}
@Test
void distinctByKey() {
List<Execution> executions = List.of(
Execution.builder().id("1").flowId("flow1").state(State.of(State.Type.CREATED, Collections.emptyList())).build(),
Execution.builder().id("1").flowId("flow1").state(State.of(State.Type.RUNNING, List.of(new State.History(State.Type.CREATED, Instant.now().minus(Duration.ofSeconds(2)))))).build(),
Execution.builder().id("2").flowId("flow2").build()
);
List<Execution> distinctExecutions = ListUtils.distinctByKey(executions, Execution::getId);
assertThat(distinctExecutions.size()).isEqualTo(2);
assertThat(distinctExecutions.stream().map(Execution::getId)).containsExactlyInAnyOrder("1", "2");
assertThat(distinctExecutions.stream().filter(e -> e.getId().equals("1")).findFirst().get().getState().getCurrent()).isEqualTo(State.Type.CREATED);
}
}

View File

@@ -1,10 +1,7 @@
package io.kestra.runner.mysql;
import io.kestra.jdbc.runner.JdbcRunnerRetryTest;
import org.junit.jupiter.api.TestInstance;
import org.junit.jupiter.api.TestInstance.Lifecycle;
@TestInstance(Lifecycle.PER_CLASS)
public class MysqlRunnerRetryTest extends JdbcRunnerRetryTest {
}

View File

@@ -293,7 +293,7 @@ public class JdbcExecutor implements ExecutorInterface {
Await.until(() -> this.allFlows != null, Duration.ofMillis(100), Duration.ofMinutes(5));
this.receiveCancellations.addFirst(((JdbcQueue<Execution>) this.executionQueue).receiveBatch(
this.receiveCancellations.addFirst(this.executionQueue.receiveBatch(
Executor.class,
executions -> {
List<CompletableFuture<Void>> futures = executions.stream()
@@ -302,7 +302,7 @@ public class JdbcExecutor implements ExecutorInterface {
CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).join();
}
));
this.receiveCancellations.addFirst(((JdbcQueue<WorkerTaskResult>) this.workerTaskResultQueue).receiveBatch(
this.receiveCancellations.addFirst(this.workerTaskResultQueue.receiveBatch(
Executor.class,
workerTaskResults -> {
List<CompletableFuture<Void>> futures = workerTaskResults.stream()
@@ -1121,14 +1121,14 @@ public class JdbcExecutor implements ExecutorInterface {
// IMPORTANT: this must be done before emitting the last execution message so that all consumers are notified that the execution ends.
// NOTE: we may also purge ExecutionKilled events, but as there may not be a lot of them, it may not be worth it.
if (cleanExecutionQueue && isTerminated) {
((JdbcQueue<Execution>) executionQueue).deleteByKey(executor.getExecution().getId());
executionQueue.deleteByKey(executor.getExecution().getId());
}
// emit for other consumers than the executor if no failure
if (hasFailure) {
this.executionQueue.emit(executor.getExecution());
} else {
((JdbcQueue<Execution>) this.executionQueue).emitOnly(null, executor.getExecution());
this.executionQueue.emitOnly(null, executor.getExecution());
}
Execution execution = executor.getExecution();
@@ -1206,8 +1206,8 @@ public class JdbcExecutor implements ExecutorInterface {
List<String> taskRunKeys = executor.getExecution().getTaskRunList().stream()
.map(taskRun -> taskRun.getId())
.toList();
((JdbcQueue<WorkerTaskResult>) workerTaskResultQueue).deleteByKeys(taskRunKeys);
((JdbcQueue<WorkerJob>) workerJobQueue).deleteByKeys(taskRunKeys);
workerTaskResultQueue.deleteByKeys(taskRunKeys);
workerJobQueue.deleteByKeys(taskRunKeys);
}
}
} catch (QueueException e) {

View File

@@ -39,10 +39,10 @@ import lombok.extern.slf4j.Slf4j;
@JdbcRunnerEnabled
public class JdbcIndexer implements Indexer {
private final LogRepositoryInterface logRepository;
private final JdbcQueue<LogEntry> logQueue;
private final QueueInterface<LogEntry> logQueue;
private final MetricRepositoryInterface metricRepository;
private final JdbcQueue<MetricEntry> metricQueue;
private final QueueInterface<MetricEntry> metricQueue;
private final MetricRegistry metricRegistry;
private final List<Runnable> receiveCancellations = new ArrayList<>();
@@ -67,9 +67,9 @@ public class JdbcIndexer implements Indexer {
QueueService queueService
) {
this.logRepository = logRepository;
this.logQueue = (JdbcQueue<LogEntry>) logQueue;
this.logQueue = logQueue;
this.metricRepository = metricRepositor;
this.metricQueue = (JdbcQueue<MetricEntry>) metricQueue;
this.metricQueue = metricQueue;
this.metricRegistry = metricRegistry;
this.eventPublisher = eventPublisher;
this.skipExecutionService = skipExecutionService;
@@ -91,7 +91,7 @@ public class JdbcIndexer implements Indexer {
this.sendBatch(metricQueue, metricRepository);
}
protected <T> void sendBatch(JdbcQueue<T> queueInterface, SaveRepositoryInterface<T> saveRepositoryInterface) {
protected <T> void sendBatch(QueueInterface<T> queueInterface, SaveRepositoryInterface<T> saveRepositoryInterface) {
this.receiveCancellations.addFirst(queueInterface.receiveBatch(Indexer.class, eithers -> {
// first, log all deserialization issues
eithers.stream().filter(either -> either.isRight()).forEach(either -> log.error("unable to deserialize an item: {}", either.getRight().getMessage()));

View File

@@ -34,7 +34,7 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
@KestraTest(rebuildContext = true)
@KestraTest
abstract public class AbstractSchedulerTest {
@Inject
protected ApplicationContext applicationContext;

View File

@@ -42,7 +42,6 @@ import static io.kestra.core.utils.Rethrow.throwConsumer;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.*;
@TestInstance(Lifecycle.PER_CLASS)
public class SchedulerScheduleTest extends AbstractSchedulerTest {
@Inject
protected FlowListeners flowListenersService;
@@ -687,4 +686,4 @@ public class SchedulerScheduleTest extends AbstractSchedulerTest {
Await.until(() -> this.triggerState.findLast(trigger).map(t -> t.getNextExecutionDate().isAfter(lastTrigger.getNextExecutionDate().plusSeconds(10))).orElse(false).booleanValue(), Duration.ofMillis(100), Duration.ofSeconds(20));
}
}
}
}

View File

@@ -1,13 +1,31 @@
package io.kestra.core.junit.extensions;
import io.kestra.core.junit.annotations.KestraTest;
import io.kestra.core.models.executions.Execution;
import io.kestra.core.models.executions.ExecutionKilled;
import io.kestra.core.models.executions.ExecutionKilledExecution;
import io.kestra.core.queues.QueueException;
import io.kestra.core.queues.QueueFactoryInterface;
import io.kestra.core.queues.QueueInterface;
import io.kestra.core.queues.TestQueueFactory;
import io.kestra.core.repositories.ExecutionRepositoryInterface;
import io.kestra.core.runners.TestRunner;
import io.kestra.core.utils.Await;
import io.kestra.core.utils.ListUtils;
import io.kestra.core.utils.TestsUtils;
import io.micronaut.inject.qualifiers.Qualifiers;
import io.micronaut.test.annotation.MicronautTestValue;
import io.micronaut.test.context.TestContext;
import io.micronaut.test.extensions.junit5.MicronautJunit5Extension;
import lombok.extern.slf4j.Slf4j;
import org.junit.jupiter.api.extension.ExtensionContext;
import org.junit.platform.commons.support.AnnotationSupport;
import java.time.Duration;
import java.util.*;
import java.util.concurrent.TimeoutException;
@Slf4j
public class KestraTestExtension extends MicronautJunit5Extension {
private static final ExtensionContext.Namespace NAMESPACE = ExtensionContext.Namespace.create(KestraTestExtension.class);
@@ -42,14 +60,18 @@ public class KestraTestExtension extends MicronautJunit5Extension {
}
@Override
public void beforeAll(ExtensionContext extensionContext) throws Exception {
super.beforeAll(extensionContext);
KestraTest kestraTest = extensionContext.getTestClass()
public void beforeTestExecution(ExtensionContext context) throws Exception {
super.beforeTestExecution(context);
TestQueueFactory.testExecutions.set(new ArrayList<>());
KestraTest kestraTest = context.getTestClass()
.orElseThrow()
.getAnnotation(KestraTest.class);
if (kestraTest.startRunner()){
if (kestraTest.startRunner()) {
TestRunner runner = applicationContext.getBean(TestRunner.class);
if (!runner.isRunning()){
if (!runner.isRunning()) {
runner.setSchedulerEnabled(kestraTest.startScheduler());
runner.setWorkerEnabled(kestraTest.startWorker());
runner.run();
@@ -62,5 +84,82 @@ public class KestraTestExtension extends MicronautJunit5Extension {
super.afterTestExecution(context);
TestsUtils.queueConsumersCleanup();
List<Execution> executionsToKill = TestQueueFactory.testExecutions.get();
if (!executionsToKill.isEmpty()
&& applicationContext.containsBean(ExecutionRepositoryInterface.class)
&& applicationContext.containsBean(QueueInterface.class, Qualifiers.byName(QueueFactoryInterface.KILL_NAMED))) {
ExecutionRepositoryInterface executionRepository = applicationContext.getBean(ExecutionRepositoryInterface.class);
QueueInterface<ExecutionKilled> killQueue = applicationContext.getBean(QueueInterface.class, Qualifiers.byName(QueueFactoryInterface.KILL_NAMED));
KestraTest kestraTest = context.getTestClass()
.orElseThrow()
.getAnnotation(KestraTest.class);
// We only wait for KILLED state if the runner is started, otherwise we just emit the kill event and it may be processed upon starting a test with a runner
List<Execution> killedExecutions = retryingExecutionKill(executionsToKill, executionRepository, killQueue, 10, kestraTest.startRunner());
executionsToKill.removeIf(execution -> killedExecutions.stream().anyMatch(killedExecution ->
Objects.equals(execution.getTenantId(), killedExecution.getTenantId())
&& Objects.equals(execution.getId(), killedExecution.getId())
));
}
}
private List<Execution> retryingExecutionKill(List<Execution> testExecutions, ExecutionRepositoryInterface executionRepository, QueueInterface<ExecutionKilled> killQueue, int retriesLeft, boolean shouldWaitForKill) throws InterruptedException {
try {
List<Execution> runningExecutions = ListUtils.distinctByKey(
testExecutions.stream().flatMap(launchedExecution -> executionRepository.findById(launchedExecution.getTenantId(), launchedExecution.getId()).stream()).toList(),
Execution::getId
).stream().filter(inRepository -> !inRepository.getState().isTerminated()).toList();
runningExecutions.forEach(inRepository -> emitKillMessage(killQueue, inRepository));
if (shouldWaitForKill) {
try {
waitForKilled(executionRepository, runningExecutions);
} catch (TimeoutException e) {
log.warn("Some executions remained in KILLING", e);
}
}
return runningExecutions;
} catch (ConcurrentModificationException e) {
// We intentionally don't use a CopyOnWriteArrayList to retry on concurrent modification exceptions to make sure to get rid of flakiness due to overflowing executions
if (retriesLeft <= 0) {
log.warn("Couldn't kill executions after test execution, due to concurrent modifications, this could impact further tests", e);
return Collections.emptyList();
}
Thread.sleep(100);
return retryingExecutionKill(testExecutions, executionRepository, killQueue, retriesLeft - 1, shouldWaitForKill);
}
}
private void emitKillMessage(QueueInterface<ExecutionKilled> killQueue, Execution inRepository) {
log.warn("Execution {} is still running after test execution, killing it", inRepository.getId());
try {
killQueue.emit(ExecutionKilledExecution.builder()
.tenantId(inRepository.getTenantId())
.executionId(inRepository.getId())
.state(ExecutionKilled.State.REQUESTED)
.isOnKillCascade(true)
.build()
);
} catch (QueueException e) {
log.warn("Couldn't kill execution {} after test execution", inRepository.getId(), e);
}
}
private void waitForKilled(ExecutionRepositoryInterface executionRepository, List<Execution> runningExecutions) throws TimeoutException {
Await.until(() -> runningExecutions.stream()
.map(execution -> executionRepository.findById(execution.getTenantId(), execution.getId()))
.allMatch(maybeExecution -> maybeExecution.map(inRepository -> {
boolean terminated = inRepository.getState().isTerminated();
if (!terminated) {
log.warn("Execution {} has a pending KILL request but is still in state {} ", inRepository.getId(), inRepository.getState().getCurrent());
}
return terminated;
})
.orElse(true))
, Duration.ofMillis(50), Duration.ofSeconds(10));
}
}

View File

@@ -0,0 +1,43 @@
package io.kestra.core.queues;
import io.kestra.core.models.executions.Execution;
import io.micronaut.context.annotation.*;
import jakarta.inject.Named;
import jakarta.inject.Singleton;
import java.lang.reflect.Proxy;
import java.util.*;
@Factory
@Requires(bean = QueueFactoryInterface.class)
public class TestQueueFactory {
public static final InheritableThreadLocal<List<Execution>> testExecutions = new InheritableThreadLocal<>();
private QueueInterface<Execution> delegate;
public TestQueueFactory(QueueFactoryInterface queueFactoryInterface) {
this.delegate = queueFactoryInterface.execution();
}
@SuppressWarnings("unchecked")
@Singleton
@Replaces(named = QueueFactoryInterface.EXECUTION_NAMED)
@Named(QueueFactoryInterface.EXECUTION_NAMED)
@Bean(preDestroy = "close")
public QueueInterface<Execution> execution() {
return (QueueInterface<Execution>) Proxy.newProxyInstance(this.getClass().getClassLoader(), new Class[]{QueueInterface.class}, (proxy, method, args) -> {
try {
if (method.getName().contains("emit")) {
Arrays.stream(args).filter(arg -> arg instanceof Execution).forEach(arg -> {
synchronized (testExecutions.get()) {
testExecutions.get().add((Execution) arg);
}
});
}
return method.invoke(this.delegate, args);
} catch (Exception e) {
throw Optional.ofNullable(e.getCause()).orElse(e);
}
});
}
}

996
ui/package-lock.json generated

File diff suppressed because it is too large Load Diff

View File

@@ -30,7 +30,7 @@
"@vue-flow/core": "^1.47.0",
"@vueuse/core": "^14.0.0",
"ansi-to-html": "^0.7.2",
"axios": "^1.13.1",
"axios": "^1.13.2",
"bootstrap": "^5.3.8",
"buffer": "^6.0.3",
"chart.js": "^4.5.1",
@@ -39,7 +39,7 @@
"cytoscape": "^3.33.0",
"dagre": "^0.8.5",
"el-table-infinite-scroll": "^3.0.7",
"element-plus": "2.11.5",
"element-plus": "2.11.7",
"humanize-duration": "^3.33.1",
"js-yaml": "^4.1.0",
"lodash": "^4.17.21",
@@ -57,16 +57,16 @@
"moment-timezone": "^0.5.46",
"nprogress": "^0.2.0",
"path-browserify": "^1.0.1",
"pdfjs-dist": "^5.4.296",
"pinia": "^3.0.3",
"posthog-js": "^1.281.0",
"pdfjs-dist": "^5.4.394",
"pinia": "^3.0.4",
"posthog-js": "^1.289.0",
"rapidoc": "^9.3.8",
"semver": "^7.7.3",
"shiki": "^3.12.2",
"vue": "^3.5.22",
"shiki": "^3.15.0",
"vue": "^3.5.24",
"vue-axios": "^3.5.2",
"vue-chartjs": "^5.3.2",
"vue-gtag": "^3.6.2",
"vue-chartjs": "^5.3.3",
"vue-gtag": "^3.6.3",
"vue-i18n": "^11.1.12",
"vue-material-design-icons": "^5.3.1",
"vue-router": "^4.6.3",
@@ -80,10 +80,10 @@
"devDependencies": {
"@codecov/vite-plugin": "^1.9.1",
"@esbuild-plugins/node-modules-polyfill": "^0.2.2",
"@eslint/js": "^9.38.0",
"@eslint/js": "^9.39.1",
"@playwright/test": "^1.56.1",
"@rushstack/eslint-patch": "^1.14.1",
"@shikijs/markdown-it": "^3.14.0",
"@shikijs/markdown-it": "^3.15.0",
"@storybook/addon-themes": "^9.1.16",
"@storybook/addon-vitest": "^9.1.16",
"@storybook/test-runner": "^0.23.0",
@@ -91,13 +91,13 @@
"@types/humanize-duration": "^3.27.4",
"@types/js-yaml": "^4.0.9",
"@types/moment": "^2.11.29",
"@types/node": "^24.9.2",
"@types/node": "^24.10.0",
"@types/nprogress": "^0.2.3",
"@types/path-browserify": "^1.0.3",
"@types/semver": "^7.7.1",
"@types/testing-library__jest-dom": "^5.14.9",
"@types/testing-library__user-event": "^4.1.1",
"@typescript-eslint/parser": "^8.46.2",
"@typescript-eslint/parser": "^8.46.3",
"@vitejs/plugin-vue": "^6.0.1",
"@vitejs/plugin-vue-jsx": "^5.1.1",
"@vitest/browser": "^3.2.4",
@@ -107,42 +107,42 @@
"@vueuse/router": "^14.0.0",
"change-case": "5.4.4",
"cross-env": "^10.1.0",
"eslint": "^9.38.0",
"eslint": "^9.39.1",
"eslint-plugin-storybook": "^9.1.16",
"eslint-plugin-vue": "^9.33.0",
"globals": "^16.4.0",
"globals": "^16.5.0",
"husky": "^9.1.7",
"jsdom": "^27.0.1",
"jsdom": "^27.1.0",
"lint-staged": "^16.2.6",
"monaco-editor": "^0.52.2",
"monaco-yaml": "5.3.1",
"patch-package": "^8.0.1",
"playwright": "^1.55.0",
"prettier": "^3.6.2",
"rimraf": "^6.0.1",
"rolldown-vite": "^7.1.20",
"rimraf": "^6.1.0",
"rolldown-vite": "^7.2.2",
"rollup-plugin-copy": "^3.5.0",
"sass": "^1.92.3",
"sass": "^1.93.3",
"storybook": "^9.1.16",
"storybook-vue3-router": "^6.0.2",
"ts-node": "^10.9.2",
"typescript": "^5.9.3",
"typescript-eslint": "^8.46.2",
"typescript-eslint": "^8.46.3",
"uuid": "^13.0.0",
"vite": "npm:rolldown-vite@latest",
"vitest": "^3.2.4",
"vue-tsc": "^3.1.2"
"vue-tsc": "^3.1.3"
},
"optionalDependencies": {
"@esbuild/darwin-arm64": "^0.25.11",
"@esbuild/darwin-x64": "^0.25.11",
"@esbuild/linux-x64": "^0.25.11",
"@esbuild/darwin-arm64": "^0.25.12",
"@esbuild/darwin-x64": "^0.25.12",
"@esbuild/linux-x64": "^0.25.12",
"@rollup/rollup-darwin-arm64": "^4.52.5",
"@rollup/rollup-darwin-x64": "^4.52.5",
"@rollup/rollup-linux-x64-gnu": "^4.52.5",
"@swc/core-darwin-arm64": "^1.14.0",
"@swc/core-darwin-x64": "^1.14.0",
"@swc/core-linux-x64-gnu": "^1.14.0"
"@swc/core-darwin-arm64": "^1.15.0",
"@swc/core-darwin-x64": "^1.15.0",
"@swc/core-linux-x64-gnu": "^1.15.0"
},
"overrides": {
"bootstrap": {

View File

@@ -24,7 +24,7 @@
@remove="emit('remove', $event)"
/>
<el-button
type="text"
link
size="small"
class="close"
:icon="Close"

View File

@@ -3,7 +3,7 @@
<el-button
v-if="!!filterKey"
ref="buttonRef"
type="text"
link
size="small"
:icon="PencilOutline"
class="edit-button"

View File

@@ -2,7 +2,7 @@
<div class="filter-header">
<label class="filter-label">{{ label }}</label>
<el-button
type="text"
link
size="small"
:icon="Close"
@click="emits('close')"

View File

@@ -5,7 +5,7 @@
<h6>{{ t("filter.customize columns") }}</h6>
<small>{{ t("filter.drag to reorder columns") }}</small>
</div>
<el-button type="text" :icon="Close" @click="$emit('close')" size="small" class="close-icon" />
<el-button link :icon="Close" @click="$emit('close')" size="small" class="close-icon" />
</div>
<div class="list">

View File

@@ -6,7 +6,7 @@
<small>{{ t("filter.select filter") }}</small>
</div>
<el-button
type="text"
link
:icon="Close"
@click="$emit('close')"
size="small"
@@ -27,7 +27,7 @@
</div>
<el-button
type="text"
link
size="default"
:icon="isSelected(key) ? undefined : Plus"
:class="isSelected(key) ? 'selected' : 'unselected'"

View File

@@ -5,7 +5,7 @@
{{ $t("filter.saved filters") }}
</h6>
<el-button
type="text"
link
:icon="Close"
@click="$emit('close')"
size="small"
@@ -28,7 +28,7 @@
<div class="action-buttons">
<el-tooltip :content="$t('filter.edit filter')" placement="top" effect="light">
<el-button
type="text"
link
size="small"
class="edit-button"
:icon="PencilOutline"
@@ -37,7 +37,7 @@
</el-tooltip>
<el-tooltip :content="$t('filter.delete filter')" placement="top" effect="light">
<el-button
type="text"
link
size="small"
class="delete-button"
:icon="Delete"

View File

@@ -59,12 +59,12 @@
const {t} = useI18n();
const exportYaml = () => {
const src = flowStore.flowYaml
if(!src) {
return;
}
const blob = new Blob([src], {type: "text/yaml"});
localUtils.downloadUrl(window.URL.createObjectURL(blob), "flow.yaml");
if(!flowStore.flow || !flowStore.flowYaml) return;
const {id, namespace} = flowStore.flow;
const blob = new Blob([flowStore.flowYaml], {type: "text/yaml"});
localUtils.downloadUrl(window.URL.createObjectURL(blob), `${namespace}.${id}.yaml`);
};
const flowStore = useFlowStore();

View File

@@ -25,7 +25,7 @@
</div>
<el-button
type="text"
link
size="default"
:icon="isVisible(column) ? EyeOutline : EyeOffOutline"
:class="isVisible(column) ? 'selected' : 'unselected'"

View File

@@ -18,7 +18,6 @@ import lombok.AllArgsConstructor;
import lombok.EqualsAndHashCode;
import lombok.Getter;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
@@ -55,20 +54,15 @@ class BasicAuthServiceTest {
@Inject
private InstanceService instanceService;
@BeforeEach
void setUp() {
stubFor(any(urlMatching(".*"))
.willReturn(aResponse()
.withStatus(404)
.withBody("No stub matched")));
}
@AfterEach
void afterEach() {
void stopApp() {
stubFor(
post(urlEqualTo("/v1/reports/events"))
.willReturn(aResponse().withStatus(200))
);
deleteSetting();
}
@Test
void isBasicAuthInitialized(){
deleteSetting();
@@ -76,22 +70,22 @@ class BasicAuthServiceTest {
new BasicAuthConfiguration(USER_NAME, PASSWORD, null, null)
).config;
basicAuthService.init();
assertThat(basicAuthService.isBasicAuthInitialized()).as("isBasicAuthInitialized after init with basic auth configured with user and password").isTrue();
assertTrue(basicAuthService.isBasicAuthInitialized());
deleteSetting();
assertThat(basicAuthService.isBasicAuthInitialized()).as("not isBasicAuthInitialized when there is no settings").isFalse();
assertFalse(basicAuthService.isBasicAuthInitialized());
basicAuthService.basicAuthConfiguration = new ConfigWrapper(
new BasicAuthConfiguration(USER_NAME, null, null, null)
).config;
basicAuthService.init();
assertThat(basicAuthService.isBasicAuthInitialized()).as("not isBasicAuthInitialized when there is settings but only user name").isFalse();
assertFalse(basicAuthService.isBasicAuthInitialized());
basicAuthService.basicAuthConfiguration = new ConfigWrapper(
new BasicAuthConfiguration(null, null, null, null)
).config;
basicAuthService.init();
assertThat(basicAuthService.isBasicAuthInitialized()).as("not isBasicAuthInitialized when there is settings but no user and password").isFalse();
assertFalse(basicAuthService.isBasicAuthInitialized());
}
@Test
@@ -214,11 +208,6 @@ class BasicAuthServiceTest {
@Test
void initFromYamlConfig() throws TimeoutException {
stubFor(
post(urlEqualTo("/v1/reports/events"))
.willReturn(aResponse().withStatus(200))
);
basicAuthService.basicAuthConfiguration = basicAuthConfiguration;
basicAuthService.init();
assertConfigurationMatchesApplicationYaml();
@@ -248,11 +237,6 @@ class BasicAuthServiceTest {
@Test
void saveValidAuthConfig() throws TimeoutException {
stubFor(
post(urlEqualTo("/v1/reports/events"))
.willReturn(aResponse().withStatus(200))
);
basicAuthService.save(new BasicAuthCredentials(null, USER_NAME, PASSWORD));
awaitOssAuthEventApiCall(USER_NAME);
}