Compare commits

...

25 Commits

Author SHA1 Message Date
Loïc Mathieu
44a8c7d63a chore(version): version 0.20.3 2024-12-06 14:13:14 +01:00
Florian Hussonnois
56afa318cd fix(core): fix cannot create Metric from null in Worker class
fix: kestra-io/kestra-ee#2417
2024-12-06 13:29:50 +01:00
Loïc Mathieu
620f894a4d fix(core): catch errors on task run
Fixes https://github.com/kestra-io/kestra-ee/issues/2416
2024-12-06 11:42:15 +01:00
YannC
37287d5e4c fix(ui): axios missing content type 2024-12-06 10:41:20 +01:00
brian.mulier
c653a1adc3 fix(jdbc): topology was built across all tenants 2024-12-06 09:53:17 +01:00
Piyush Bhaskar
1abfa5e23e chore(ui): improve bulk actions design in the executions listing (#6240)
Co-authored-by: Piyush-r-bhaskar <piyush.bhaskar@gmail.com>
Co-authored-by: Miloš Paunović <paun992@hotmail.com>
2024-12-05 14:51:13 +01:00
Manoj Balaraj
03d8855309 fix(ui): properly handle pebble expression if it contains dash character (#6062)
Co-authored-by: manu2931 <manojb912@gmai.com>
Co-authored-by: Miloš Paunović <paun992@hotmail.com>
2024-12-05 14:11:40 +01:00
Loïc Mathieu
0ab7b0e57a chore(version): upgrade to 0.20.2 2024-12-05 11:04:52 +01:00
Loïc Mathieu
5f8de5106b Revert "feat(core): Add displayName to flow level outputs(backend) (#5605)"
This reverts commit a5741aa424.

This reverts commit 42f721fdec.

This reverts commit 0de24c4448.
2024-12-05 10:30:51 +01:00
Miloš Paunović
c749301944 fix(ui): filter out system labels from executing using prefill (#6311) 2024-12-05 09:21:24 +01:00
Piyush Bhaskar
0831e9d356 chore(ui): remove default editor outline (#6303)
Co-authored-by: Piyush-r-bhaskar <piyush.bhaskar@gmail.com>
2024-12-05 08:37:23 +01:00
Miloš Paunović
9e4d36e70d fix(ui): only apply editor padding on main editor (#6310) 2024-12-05 08:34:12 +01:00
Ludovic DEHON
2bbb7a83b8 chore(version): update to version 'v0.20.1'. 2024-12-04 22:36:36 +01:00
Piyush Bhaskar
bad60b4663 chore(ui): Improvement in Welcome Page. (#6077)
* chore(ui): Improvement in Welcome Page.

* Update Welcome.vue | scoped the styling

* fix bad merge

* remove special behavior of navbar on welcome

* finish the welcome page (thank you)

* fix: better adaptive layout

* use container queries and flex for better responsive design

* chore(translations): auto generate values for languages other than english

---------

Co-authored-by: Barthélémy Ledoux <ledouxb@me.com>
Co-authored-by: Bart Ledoux <bledoux@kestra.io>
Co-authored-by: Miloš Paunović <paun992@hotmail.com>
Co-authored-by: GitHub Action <actions@github.com>
2024-12-04 14:38:28 +01:00
Abhishek Pawar
4b1c700b5e fix(ui): handle logs selector overflow in a good manner (#6224)
Co-authored-by: MilosPaunovic <paun992@hotmail.com>
2024-12-04 14:18:23 +01:00
Ian Cheng
1323c95785 feat(ui): add right click menu on file tree view in editor (#5936)
Co-authored-by: Miloš Paunović <paun992@hotmail.com>
2024-12-04 13:21:04 +01:00
Loïc Mathieu
3e726b5848 fix(core, webserver): properly close the queue on Flux.onFinally
Two fixes:
- close the queue onFinally and not onComplete and onCancel to take into accunt errors.
- close the queue onFinally in the execution creation as now it is only done on the success path and not even via a Flux lifecycle method

This may fix or improve some incosistent behavior reported by users on the webserver.
2024-12-04 12:18:05 +01:00
Loïc Mathieu
97ad281566 fix(core): Correctly parse Content-Disposition in the Download task
Fixes #6270
2024-12-04 12:16:46 +01:00
Nitin Bisht
31f6e3fe25 chore(ui): amend spacing on plugins page (#6223)
Co-authored-by: MilosPaunovic <paun992@hotmail.com>
2024-12-04 08:55:20 +01:00
Joe Celaster
97f16e989b chore(ui): remove search field background on single plugin page (#6220)
Co-authored-by: MilosPaunovic <paun992@hotmail.com>
2024-12-04 08:49:34 +01:00
Manoj Balaraj
b72fb29377 fix(ui): improve debug outputs expression on initial load (#6094)
Co-authored-by: manu2931 <manojb912@gmai.com>
Co-authored-by: Miloš Paunović <paun992@hotmail.com>
2024-12-04 08:45:34 +01:00
Piyush Bhaskar
e45bbdb9e7 chore(ui): add top and left padding to editor component (#6191)
Co-authored-by: Piyush-r-bhaskar <piyush.bhaskar@gmail.com>
Co-authored-by: MilosPaunovic <paun992@hotmail.com>
2024-12-04 08:39:41 +01:00
Ines Qian
178ee0e7df chore(ui): properly highlight selected options in all of the filter dropdowns (#6173)
Co-authored-by: Miloš Paunović <paun992@hotmail.com>
2024-12-04 08:36:28 +01:00
Florian Hussonnois
aa1ba59983 chore(version): update to version 'v0.20.0'. 2024-12-03 12:20:53 +01:00
Loïc Mathieu
2e9a0d132a fix(core): possible NPE when the Executor didn't have the flow 2024-12-03 12:19:50 +01:00
49 changed files with 1063 additions and 496 deletions

View File

@@ -54,7 +54,11 @@ public abstract class AbstractWorkerCallable implements Callable<State.Type> {
try {
return doCall();
} catch (Exception e) {
} catch (Throwable e) {
// Catching Throwable is usually a bad idea.
// However, here, we want to be sure that the task fails whatever happens,
// and some plugins may throw errors, for example, for dependency issues or worst,
// bad behavior that throws errors and not exceptions.
return this.exceptionHandler(e);
} finally {
shutdownLatch.countDown();

View File

@@ -379,7 +379,9 @@ public class ExecutorService {
if (flow.getOutputs() != null) {
RunContext runContext = runContextFactory.of(executor.getFlow(), executor.getExecution());
try {
Map<String, Object> outputs = flowInputOutput.flowOutputsToMap(flow.getOutputs());
Map<String, Object> outputs = flow.getOutputs()
.stream()
.collect(HashMap::new, (map, entry) -> map.put(entry.getId(), entry.getValue()), Map::putAll);
outputs = runContext.render(outputs);
outputs = flowInputOutput.typedOutputs(flow, executor.getExecution(), outputs);
newExecution = newExecution.withOutputs(outputs);
@@ -1046,7 +1048,7 @@ public class ExecutorService {
* WARNING: ATM, only the first violation will update the execution.
*/
public Executor handleExecutionChangedSLA(Executor executor) throws QueueException {
if (ListUtils.isEmpty(executor.getFlow().getSla()) || executor.getExecution().getState().isTerminated()) {
if (executor.getFlow() == null || ListUtils.isEmpty(executor.getFlow().getSla()) || executor.getExecution().getState().isTerminated()) {
return executor;
}

View File

@@ -1,7 +1,5 @@
package io.kestra.core.runners;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableMap;
@@ -9,7 +7,12 @@ import io.kestra.core.encryption.EncryptionService;
import io.kestra.core.exceptions.IllegalVariableEvaluationException;
import io.kestra.core.exceptions.KestraRuntimeException;
import io.kestra.core.models.executions.Execution;
import io.kestra.core.models.flows.*;
import io.kestra.core.models.flows.Data;
import io.kestra.core.models.flows.DependsOn;
import io.kestra.core.models.flows.Flow;
import io.kestra.core.models.flows.Input;
import io.kestra.core.models.flows.RenderableInput;
import io.kestra.core.models.flows.Type;
import io.kestra.core.models.flows.input.FileInput;
import io.kestra.core.models.flows.input.InputAndValue;
import io.kestra.core.models.flows.input.ItemTypeInterface;
@@ -28,7 +31,6 @@ import jakarta.inject.Inject;
import jakarta.inject.Singleton;
import jakarta.validation.ConstraintViolationException;
import jakarta.validation.constraints.NotNull;
import org.reactivestreams.Publisher;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono;
@@ -43,8 +45,6 @@ import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalTime;
import java.util.AbstractMap;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
@@ -85,21 +85,6 @@ public class FlowInputOutput {
this.secretKey = Optional.ofNullable(secretKey);
}
/**
* Transform a list of flow outputs to a Map of output id -> output value map.
* An Output value map is a map with value and displayName.
*/
public Map<String, Object> flowOutputsToMap(List<Output> flowOutputs) {
return ListUtils.emptyOnNull(flowOutputs)
.stream()
.collect(HashMap::new, (map, entry) -> {
final HashMap<String, Object> entryInfo = new HashMap<>();
entryInfo.put("value", entry.getValue());
entryInfo.put("displayName", Optional.ofNullable(entry.getDisplayName()).orElse(entry.getId()));
map.put(entry.getId(), entryInfo);
}, Map::putAll);
}
/**
* Validate all the inputs of a given execution of a flow.
*
@@ -370,21 +355,9 @@ public class FlowInputOutput {
.getOutputs()
.stream()
.map(output -> {
final HashMap<String, Object> current;
final Object currentValue;
Object current = in == null ? null : in.get(output.getId());
try {
current = in == null ? null : JSON_MAPPER.readValue(
JSON_MAPPER.writeValueAsString(in.get(output.getId())), new TypeReference<>() {});
} catch (JsonProcessingException e) {
throw new RuntimeException(e);
}
if (current == null) {
currentValue = null;
} else {
currentValue = current.get("value");
}
try {
return parseData(execution, output, currentValue)
return parseData(execution, output, current)
.map(entry -> {
if (output.getType().equals(Type.SECRET)) {
return new AbstractMap.SimpleEntry<>(
@@ -395,26 +368,12 @@ public class FlowInputOutput {
return entry;
});
} catch (Exception e) {
throw output.toConstraintViolationException(e.getMessage(), currentValue);
throw output.toConstraintViolationException(e.getMessage(), current);
}
})
.filter(Optional::isPresent)
.map(Optional::get)
.collect(HashMap::new,
(map, entry) -> {
map.compute(entry.getKey(), (key, existingValue) -> {
if (existingValue == null) {
return entry.getValue();
}
if (existingValue instanceof List) {
((List<Object>) existingValue).add(entry.getValue());
return existingValue;
}
return new ArrayList<>(Arrays.asList(existingValue, entry.getValue()));
});
},
Map::putAll
);
.collect(HashMap::new, (map, entry) -> map.put(entry.getKey(), entry.getValue()), Map::putAll);
// Ensure outputs are compliant with tasks outputs.
return JacksonMapper.toMap(results);
@@ -432,7 +391,7 @@ public class FlowInputOutput {
final Type elementType = data instanceof ItemTypeInterface itemTypeInterface ? itemTypeInterface.getItemType() : null;
return Optional.of(new AbstractMap.SimpleEntry<>(
Optional.ofNullable(data.getDisplayName()).orElse(data.getId()),
data.getId(),
parseType(execution, data.getType(), data.getId(), elementType, current)
));
}

View File

@@ -51,6 +51,8 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static io.kestra.core.models.flows.State.Type.*;
import static io.kestra.core.server.Service.ServiceState.TERMINATED_FORCED;
@@ -180,11 +182,16 @@ public class Worker implements Service, Runnable, AutoCloseable {
return Collections.emptySet();
}
return Set.of(
Metric.of(this.metricRegistry.findGauge(MetricRegistry.METRIC_WORKER_JOB_THREAD_COUNT)),
Metric.of(this.metricRegistry.findGauge(MetricRegistry.METRIC_WORKER_JOB_PENDING_COUNT)),
Metric.of(this.metricRegistry.findGauge(MetricRegistry.METRIC_WORKER_JOB_RUNNING_COUNT))
Stream<String> metrics = Stream.of(
MetricRegistry.METRIC_WORKER_JOB_THREAD_COUNT,
MetricRegistry.METRIC_WORKER_JOB_PENDING_COUNT,
MetricRegistry.METRIC_WORKER_JOB_RUNNING_COUNT
);
return metrics
.flatMap(metric -> Optional.ofNullable(metricRegistry.findGauge(metric)).stream())
.map(Metric::of)
.collect(Collectors.toSet());
}
@Override

View File

@@ -61,12 +61,7 @@ public class ExecutionLogService {
}
}));
}, FluxSink.OverflowStrategy.BUFFER)
.doOnCancel(() -> {
if (disposable.get() != null) {
disposable.get().run();
}
})
.doOnComplete(() -> {
.doFinally(ignored -> {
if (disposable.get() != null) {
disposable.get().run();
}

View File

@@ -8,7 +8,6 @@ import io.kestra.core.models.annotations.Example;
import io.kestra.core.models.annotations.Plugin;
import io.kestra.core.models.annotations.PluginProperty;
import io.kestra.core.models.executions.Execution;
import io.kestra.core.models.flows.Flow;
import io.kestra.core.models.property.Property;
import io.kestra.core.models.executions.TaskRun;
import io.kestra.core.models.executions.TaskRunAttempt;
@@ -37,13 +36,16 @@ import lombok.ToString;
import jakarta.validation.constraints.NotEmpty;
import jakarta.validation.constraints.NotNull;
import org.apache.commons.lang3.stream.Streams;
import java.time.ZonedDateTime;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
@SuperBuilder
@ToString
@@ -160,7 +162,7 @@ public class Subflow extends Task implements ExecutableTask<Subflow.Output>, Chi
@Override
public List<SubflowExecution<?>> createSubflowExecutions(RunContext runContext,
FlowExecutorInterface flowExecutorInterface,
Flow currentFlow,
io.kestra.core.models.flows.Flow currentFlow,
Execution currentExecution,
TaskRun currentTaskRun) throws InternalException {
Map<String, Object> inputs = new HashMap<>();
@@ -186,7 +188,7 @@ public class Subflow extends Task implements ExecutableTask<Subflow.Output>, Chi
public Optional<SubflowExecutionResult> createSubflowExecutionResult(
RunContext runContext,
TaskRun taskRun,
Flow flow,
io.kestra.core.models.flows.Flow flow,
Execution execution
) {
// we only create a worker task result when the execution is terminated
@@ -202,16 +204,25 @@ public class Subflow extends Task implements ExecutableTask<Subflow.Output>, Chi
.executionId(execution.getId())
.state(execution.getState().getCurrent());
FlowInputOutput flowInputOutput = ((DefaultRunContext)runContext).getApplicationContext().getBean(FlowInputOutput.class); // this is hacking
final Map<String, Object> subflowOutputs = Optional
.ofNullable(flow.getOutputs())
.map(outputs -> flowInputOutput.flowOutputsToMap(flow.getOutputs()))
.map(outputs -> flowInputOutput.typedOutputs(flow, execution, outputs))
.map(outputs -> outputs
.stream()
.collect(Collectors.toMap(
io.kestra.core.models.flows.Output::getId,
io.kestra.core.models.flows.Output::getValue)
)
)
.orElseGet(() -> isOutputsAllowed ? this.getOutputs() : null);
if (subflowOutputs != null) {
try {
builder.outputs(runContext.render(subflowOutputs));
Map<String, Object> outputs = runContext.render(subflowOutputs);
FlowInputOutput flowInputOutput = ((DefaultRunContext)runContext).getApplicationContext().getBean(FlowInputOutput.class); // this is hacking
if (flow.getOutputs() != null && flowInputOutput != null) {
outputs = flowInputOutput.typedOutputs(flow, execution, outputs);
}
builder.outputs(outputs);
} catch (Exception e) {
runContext.logger().warn("Failed to extract outputs with the error: '{}'", e.getLocalizedMessage(), e);
var state = this.isAllowFailure() ? this.isAllowWarning() ? State.Type.SUCCESS : State.Type.WARNING : State.Type.FAILED;

View File

@@ -23,6 +23,8 @@ import java.io.FileOutputStream;
import java.net.URI;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.List;
import java.util.Map;
@@ -162,21 +164,31 @@ public class Download extends AbstractHttp implements RunnableTask<Download.Outp
}
}
// Note: this is a naive basic implementation that may bot cover all possible use cases.
// Note: this is a basic implementation that should cover all possible use cases.
// If this is not enough, we should find some helper method somewhere to cover all possible rules of the Content-Disposition header.
private String filenameFromHeader(RunContext runContext, String contentDisposition) {
try {
String[] parts = contentDisposition.split(" ");
// Content-Disposition parts are separated by ';'
String[] parts = contentDisposition.split(";");
String filename = null;
for (String part : parts) {
if (part.startsWith("filename")) {
filename = part.substring(part.lastIndexOf('=') + 1);
String stripped = part.strip();
if (stripped.startsWith("filename")) {
filename = stripped.substring(stripped.lastIndexOf('=') + 1);
}
if (part.startsWith("filename*")) {
if (stripped.startsWith("filename*")) {
// following https://datatracker.ietf.org/doc/html/rfc5987 the filename* should be <ENCODING>'(lang)'<filename>
filename = part.substring(part.lastIndexOf('\'') + 2, part.length() - 1);
filename = stripped.substring(stripped.lastIndexOf('\'') + 2, stripped.length() - 1);
}
}
// filename may be in double-quotes
if (filename != null && filename.charAt(0) == '"') {
filename = filename.substring(1, filename.length() - 1);
}
// if filename contains a path: use only the last part to avoid security issues due to host file overwriting
if (filename != null && filename.contains(File.separator)) {
filename = filename.substring(filename.lastIndexOf(File.separator) + 1);
}
return filename;
} catch (Exception e) {
// if we cannot parse the Content-Disposition header, we return null

View File

@@ -2,7 +2,9 @@ package io.kestra.core.runners;
import io.kestra.core.junit.annotations.KestraTest;
import io.kestra.core.models.executions.Execution;
import io.kestra.core.models.flows.*;
import io.kestra.core.models.flows.DependsOn;
import io.kestra.core.models.flows.Input;
import io.kestra.core.models.flows.Type;
import io.kestra.core.models.flows.input.FileInput;
import io.kestra.core.models.flows.input.InputAndValue;
import io.kestra.core.models.flows.input.IntInput;
@@ -28,11 +30,6 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.aMapWithSize;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.is;
@KestraTest
class FlowInputOutputTest {
@@ -252,17 +249,6 @@ class FlowInputOutputTest {
);
}
@Test
@SuppressWarnings("unchecked")
void flowOutputsToMap() {
Flow flow = Flow.builder().id("flow").outputs(List.of(Output.builder().id("output").value("something").build())).build();
Map<String, Object> stringObjectMap = flowInputOutput.flowOutputsToMap(flow.getOutputs());
assertThat(stringObjectMap, aMapWithSize(1));
assertThat(stringObjectMap.get("output"), notNullValue());
assertThat(((Map<String, Object>) stringObjectMap.get("output")).get("value"), is("something"));
}
private static final class MemoryCompletedFileUpload implements CompletedFileUpload {
private final String name;

View File

@@ -39,13 +39,4 @@ class FlowOutputTest extends AbstractMemoryRunnerTest {
assertThat(execution.getOutputs(), nullValue());
assertThat(execution.getState().getCurrent(), is(State.Type.FAILED));
}
@Test
void shouldGetSuccessExecutionForFlowWithOutputsDisplayName() throws TimeoutException, QueueException {
Execution execution = runnerUtils.runOne(null, NAMESPACE, "flow-with-outputs-display-name", null, null);
assertThat(execution.getOutputs(), aMapWithSize(1));
assertThat(execution.getOutputs().get("key"), nullValue());
assertThat(execution.getOutputs().get("Sample Output"), is("{\"value\":\"flow-with-outputs-display-name\"}"));
assertThat(execution.getState().getCurrent(), is(State.Type.SUCCESS));
}
}

View File

@@ -1,22 +1,25 @@
package io.kestra.plugin.core.flow;
import io.kestra.core.junit.annotations.KestraTest;
import io.kestra.core.exceptions.IllegalVariableEvaluationException;
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.Output;
import io.kestra.core.models.flows.State;
import io.kestra.core.models.flows.Type;
import io.kestra.core.runners.DefaultRunContext;
import io.kestra.core.runners.RunContext;
import io.kestra.core.runners.RunContextFactory;
import io.kestra.core.runners.SubflowExecutionResult;
import io.micronaut.context.annotation.Property;
import jakarta.inject.Inject;
import io.micronaut.context.ApplicationContext;
import org.hamcrest.Matchers;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.time.Instant;
import java.util.Collections;
@@ -29,14 +32,26 @@ import static org.hamcrest.Matchers.hasProperty;
import static org.hamcrest.Matchers.is;
import static org.junit.jupiter.api.Assertions.assertTrue;
@KestraTest
@ExtendWith(MockitoExtension.class)
@MockitoSettings(strictness = Strictness.LENIENT)
class SubflowTest {
private static final Logger LOG = LoggerFactory.getLogger(SubflowTest.class);
private static final State DEFAULT_SUCCESS_STATE = State.of(State.Type.SUCCESS, List.of(new State.History(State.Type.CREATED, Instant.now()), new State.History(State.Type.RUNNING, Instant.now()), new State.History(State.Type.SUCCESS, Instant.now())));
public static final String EXECUTION_ID = "executionId";
@Inject
private RunContextFactory runContextFactory;
@Mock
private DefaultRunContext runContext;
@Mock
private ApplicationContext applicationContext;
@BeforeEach
void beforeEach() {
Mockito.when(runContext.logger()).thenReturn(LOG);
Mockito.when(runContext.getApplicationContext()).thenReturn(applicationContext);
}
@Test
void shouldNotReturnResultForExecutionNotTerminated() {
@@ -44,7 +59,6 @@ class SubflowTest {
.builder()
.state(State.of(State.Type.CREATED, Collections.emptyList()))
.build();
RunContext runContext = runContextFactory.of();
Optional<SubflowExecutionResult> result = new Subflow().createSubflowExecutionResult(
runContext,
@@ -59,14 +73,14 @@ class SubflowTest {
@SuppressWarnings("deprecation")
@Test
void shouldNotReturnOutputsForSubflowOutputsDisabled() {
// Given
Mockito.when(applicationContext.getProperty(Subflow.PLUGIN_FLOW_OUTPUTS_ENABLED, Boolean.class))
.thenReturn(Optional.of(false));
Map<String, Object> outputs = Map.of("key", "value");
Subflow subflow = Subflow.builder()
.outputs(outputs)
.build();
DefaultRunContext defaultRunContext = (DefaultRunContext) runContextFactory.of();
DefaultRunContext runContext = Mockito.mock(DefaultRunContext.class);
Mockito.when(runContext.pluginConfiguration(Subflow.PLUGIN_FLOW_OUTPUTS_ENABLED)).thenReturn(Optional.of(false));
Mockito.when(runContext.getApplicationContext()).thenReturn(defaultRunContext.getApplicationContext());
// When
Optional<SubflowExecutionResult> result = subflow.createSubflowExecutionResult(
@@ -81,6 +95,7 @@ class SubflowTest {
Map<String, Object> expected = Subflow.Output.builder()
.executionId(EXECUTION_ID)
.state(DEFAULT_SUCCESS_STATE.getCurrent())
.outputs(Collections.emptyMap())
.build()
.toMap();
assertThat(result.get().getParentTaskRun().getOutputs(), is(expected));
@@ -94,10 +109,14 @@ class SubflowTest {
@SuppressWarnings("deprecation")
@Test
void shouldReturnOutputsForSubflowOutputsEnabled() {
void shouldReturnOutputsForSubflowOutputsEnabled() throws IllegalVariableEvaluationException {
// Given
Mockito.when(applicationContext.getProperty(Subflow.PLUGIN_FLOW_OUTPUTS_ENABLED, Boolean.class))
.thenReturn(Optional.of(true));
Map<String, Object> outputs = Map.of("key", "value");
RunContext runContext = runContextFactory.of(outputs);
Mockito.when(runContext.render(Mockito.anyMap())).thenReturn(outputs);
Subflow subflow = Subflow.builder()
.outputs(outputs)
@@ -129,10 +148,13 @@ class SubflowTest {
}
@Test
void shouldOnlyReturnOutputsFromFlowOutputs() {
void shouldOnlyReturnOutputsFromFlowOutputs() throws IllegalVariableEvaluationException {
// Given
Output output = Output.builder().id("key").value("value").type(Type.STRING).build();
RunContext runContext = runContextFactory.of(Map.of(output.getId(), output.getValue()));
Mockito.when(applicationContext.getProperty(Subflow.PLUGIN_FLOW_OUTPUTS_ENABLED, Boolean.class))
.thenReturn(Optional.of(true));
Output output = Output.builder().id("key").value("value").build();
Mockito.when(runContext.render(Mockito.anyMap())).thenReturn(Map.of(output.getId(), output.getValue()));
Flow flow = Flow.builder()
.outputs(List.of(output))
.build();

View File

@@ -134,7 +134,26 @@ class DownloadTest {
Download.Output output = task.run(runContext);
assertThat(output.getUri().toString(), containsString("filename.jpg"));
assertThat(output.getUri().toString(), endsWith("filename.jpg"));
}
@Test
void contentDispositionWithPath() throws Exception {
EmbeddedServer embeddedServer = applicationContext.getBean(EmbeddedServer.class);
embeddedServer.start();
Download task = Download.builder()
.id(DownloadTest.class.getSimpleName())
.type(DownloadTest.class.getName())
.uri(embeddedServer.getURI() + "/content-disposition")
.build();
RunContext runContext = TestsUtils.mockRunContext(this.runContextFactory, task, ImmutableMap.of());
Download.Output output = task.run(runContext);
assertThat(output.getUri().toString(), not(containsString("/secure-path/")));
assertThat(output.getUri().toString(), endsWith("filename.jpg"));
}
@Test
@@ -177,5 +196,11 @@ class DownloadTest {
return HttpResponse.ok("Hello World".getBytes())
.header(HttpHeaders.CONTENT_DISPOSITION, "attachment; filename=\"filename.jpg\"");
}
@Get("content-disposition-path")
public HttpResponse<byte[]> contentDispositionWithPath() {
return HttpResponse.ok("Hello World".getBytes())
.header(HttpHeaders.CONTENT_DISPOSITION, "attachment; filename=\"/secure-path/filename.jpg\"");
}
}
}

View File

@@ -1,13 +0,0 @@
id: flow-with-outputs-display-name
namespace: io.kestra.tests
tasks:
- id: return
type: io.kestra.plugin.core.debug.Return
format: "{{ flow.id }}"
outputs:
- id: "key"
value: "{{ outputs.return }}"
type: STRING
displayName: Sample Output

View File

@@ -1,11 +1,6 @@
id: subflow-grand-child
namespace: io.kestra.tests
outputs:
- id: myResult
type: STRING
value: something
tasks:
- id: firstLevel
type: io.kestra.plugin.core.log.Log

View File

@@ -1,4 +1,4 @@
version=0.20.0-SNAPSHOT
version=0.20.3
org.gradle.parallel=true
org.gradle.caching=true

View File

@@ -8,8 +8,8 @@ import io.kestra.core.exceptions.InternalException;
import io.kestra.core.metrics.MetricRegistry;
import io.kestra.core.models.executions.*;
import io.kestra.core.models.executions.statistics.ExecutionCount;
import io.kestra.core.models.flows.*;
import io.kestra.core.models.flows.Flow;
import io.kestra.core.models.flows.*;
import io.kestra.core.models.flows.sla.*;
import io.kestra.core.models.tasks.ExecutableTask;
import io.kestra.core.models.tasks.Task;
@@ -47,9 +47,11 @@ import org.jooq.Configuration;
import org.slf4j.event.Level;
import java.io.IOException;
import java.time.*;
import java.time.Duration;
import java.time.Instant;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -294,7 +296,7 @@ public class JdbcExecutor implements ExecutorInterface, Service {
flowTopologyService
.topology(
flow,
this.allFlows
this.allFlows.stream().filter(f -> Objects.equals(f.getTenantId(), flow.getTenantId())).toList()
)
)
.distinct()

View File

@@ -0,0 +1,120 @@
<svg width="164" height="116" viewBox="0 0 164 116" fill="none" xmlns="http://www.w3.org/2000/svg">
<ellipse cx="82.0001" cy="57.058" rx="81.8075" ry="50.9725" fill="#5C29DC" fill-opacity="0.05"/>
<ellipse cx="82.0002" cy="57.0577" rx="71.8178" ry="44.7482" fill="#5C29DC" fill-opacity="0.22"/>
<ellipse cx="82.0002" cy="57.0581" rx="60.1184" ry="37.4585" fill="#5C29DC"/>
<g filter="url(#filter0_d_4459_37617)">
<path d="M84.8423 99.1473L146.085 64.5658C150.538 62.0508 150.595 55.6558 146.186 53.0626L84.9445 17.0409C82.8563 15.8126 80.2648 15.8194 78.183 17.0585L17.6657 53.0806C13.2967 55.6812 13.3531 62.0268 17.7676 64.5494L78.2843 99.1305C80.3147 100.291 82.8059 100.297 84.8423 99.1473Z" fill="#1F232C"/>
</g>
<g filter="url(#filter1_d_4459_37617)">
<path d="M84.8423 92.5061L146.085 57.9247C150.538 55.4097 150.595 49.0147 146.186 46.4215L84.9445 10.3998C82.8563 9.17152 80.2648 9.17828 78.183 10.4174L17.6657 46.4395C13.2967 49.0401 13.3531 55.3857 17.7676 57.9083L78.2843 92.4894C80.3147 93.6497 82.8059 93.656 84.8423 92.5061Z" fill="#1F232C"/>
</g>
<path d="M84.8423 95.8269L146.085 61.2455C150.538 58.7305 150.595 52.3355 146.186 49.7423L84.9445 13.7206C82.8563 12.4923 80.2648 12.4991 78.183 13.7382L17.6657 49.7603C13.2967 52.3609 13.3531 58.7065 17.7676 61.2291L78.2843 95.8102C80.3147 96.9705 82.8059 96.9768 84.8423 95.8269Z" fill="#1F232C"/>
<g filter="url(#filter2_d_4459_37617)">
<path d="M84.8423 90.1834L146.085 55.6019C150.538 53.087 150.595 46.6919 146.186 44.0987L84.9445 8.07703C82.8563 6.84877 80.2648 6.85553 78.183 8.09467L17.6657 44.1167C13.2967 46.7173 13.3531 53.0629 17.7676 55.5855L78.2843 90.1666C80.3147 91.3269 82.8059 91.3333 84.8423 90.1834Z" fill="#11192B"/>
<path d="M84.6791 89.8943L145.921 55.3129C150.153 52.9237 150.206 46.8484 146.018 44.3849L84.7762 8.36315C82.7924 7.1963 80.3305 7.20272 78.3528 8.37991L17.8355 44.402C13.6849 46.8725 13.7385 52.9008 17.9323 55.2973L78.449 89.8784C80.3779 90.9807 82.7445 90.9867 84.6791 89.8943Z" stroke="url(#paint0_linear_4459_37617)" stroke-width="0.663881"/>
</g>
<g clip-path="url(#clip0_4459_37617)">
<mask id="mask0_4459_37617" style="mask-type:alpha" maskUnits="userSpaceOnUse" x="14" y="7" width="136" height="85">
<path d="M84.6791 89.8943L145.921 55.3129C150.153 52.9237 150.206 46.8484 146.018 44.3849L84.7762 8.36315C82.7924 7.1963 80.3305 7.20272 78.3528 8.37991L17.8355 44.402C13.6849 46.8725 13.7385 52.9008 17.9323 55.2973L78.449 89.8784C80.3779 90.9807 82.7445 90.9867 84.6791 89.8943Z" fill="#11192B" stroke="url(#paint1_linear_4459_37617)" stroke-width="0.663881"/>
</mask>
<g mask="url(#mask0_4459_37617)">
<g filter="url(#filter3_f_4459_37617)">
<rect x="55.8655" y="56.9795" width="43.8063" height="15.8446" transform="rotate(180 55.8655 56.9795)" fill="#8874B2"/>
</g>
<g filter="url(#filter4_f_4459_37617)">
<rect x="139.261" y="82.3198" width="43.8063" height="32.7457" transform="rotate(180 139.261 82.3198)" fill="#8874B2"/>
</g>
</g>
</g>
<g filter="url(#filter5_di_4459_37617)">
<path d="M87.578 47.3682C87.5701 46.458 86.311 45.7311 84.7658 45.7446L78.4142 45.8C76.869 45.8135 75.6228 46.5623 75.6308 47.4725L75.6634 51.2139C75.6714 52.1241 76.9304 52.851 78.4756 52.8375L84.8273 52.7821C86.3724 52.7686 87.6186 52.0198 87.6107 51.1096L87.578 47.3682Z" fill="#A950FF"/>
<path d="M72.9007 38.8646C72.8929 37.9707 71.6564 37.2568 70.1388 37.27L63.6871 37.3263C62.1696 37.3396 60.9457 38.0749 60.9535 38.9688L60.9866 42.7692C60.9944 43.6631 62.231 44.377 63.7485 44.3638L70.2003 44.3075C71.7178 44.2942 72.9417 43.5589 72.9339 42.665L72.9007 38.8646Z" fill="#A950FF"/>
<path d="M102.105 38.61C102.097 37.7161 100.861 37.0022 99.343 37.0154L92.8912 37.0717C91.3737 37.085 90.1498 37.8203 90.1576 38.7142L90.1908 42.5146C90.1986 43.4085 91.4351 44.1224 92.9526 44.1092L99.4044 44.0529C100.922 44.0396 102.146 43.3043 102.138 42.4104L102.105 38.61Z" fill="#E9C1FF"/>
<path d="M93.0156 45.6723C91.4704 45.6857 90.2242 46.4345 90.2322 47.3447L90.2648 51.0861C90.2728 51.9963 91.5318 52.7232 93.077 52.7097L99.4287 52.6543C100.974 52.6408 102.22 51.892 102.212 50.9818L102.179 47.2404C102.171 46.3303 100.912 45.6033 99.3672 45.6168L93.0156 45.6723Z" fill="#CD88FF"/>
<path d="M93.0413 54.274C91.5238 54.2872 90.2999 55.0226 90.3077 55.9165L90.3409 59.7169C90.3487 60.6108 91.5852 61.3247 93.1028 61.3115L99.5545 61.2552C101.072 61.2419 102.296 60.5065 102.288 59.6126L102.255 55.8123C102.247 54.9184 101.011 54.2045 99.4931 54.2177L93.0413 54.274Z" fill="#A950FF"/>
<path d="M78.3389 37.1986C76.7937 37.2121 75.5475 37.9609 75.5554 38.8711L75.5881 42.6125C75.596 43.5227 76.8551 44.2496 78.4003 44.2361L84.7519 44.1807C86.2971 44.1672 87.5433 43.4184 87.5353 42.5082L87.5027 38.7668C87.4947 37.8566 86.2357 37.1297 84.6905 37.1432L78.3389 37.1986Z" fill="#CD88FF"/>
<path d="M66.3843 54.7446C62.9323 54.7747 60.1482 56.4475 60.1659 58.481C60.1837 60.5144 62.9965 62.1384 66.4486 62.1083C69.9007 62.0781 72.6847 60.4053 72.667 58.3719C72.6493 56.3385 69.8364 54.7145 66.3843 54.7446Z" fill="#F62E76"/>
</g>
<g filter="url(#filter6_d_4459_37617)">
<path d="M119.151 55.6691L128.653 74.5943C128.776 74.8393 128.959 75.0494 129.184 75.2054C129.41 75.3614 129.671 75.4583 129.944 75.4873C130.216 75.5162 130.492 75.4763 130.745 75.3711C130.999 75.2659 131.221 75.0989 131.393 74.8853L133.178 65.0719L142.018 61.6387C142.187 61.4278 142.301 61.1783 142.35 60.9125C142.399 60.6467 142.382 60.3729 142.299 60.1155C142.217 59.8581 142.072 59.6252 141.877 59.4375C141.683 59.2498 141.445 59.1133 141.184 59.04L121.061 53.3653C120.756 53.2792 120.433 53.2835 120.13 53.3777C119.828 53.4719 119.559 53.6519 119.357 53.8958C119.154 54.1398 119.027 54.4372 118.991 54.752C118.955 55.0668 119.01 55.3854 119.151 55.6691Z" fill="white"/>
</g>
<defs>
<filter id="filter0_d_4459_37617" x="6.45604" y="15.4606" width="150.97" height="99.814" filterUnits="userSpaceOnUse" color-interpolation-filters="sRGB">
<feFlood flood-opacity="0" result="BackgroundImageFix"/>
<feColorMatrix in="SourceAlpha" type="matrix" values="0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 127 0" result="hardAlpha"/>
<feOffset dy="7.30269"/>
<feGaussianBlur stdDeviation="3.98328"/>
<feComposite in2="hardAlpha" operator="out"/>
<feColorMatrix type="matrix" values="0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0.25 0"/>
<feBlend mode="normal" in2="BackgroundImageFix" result="effect1_dropShadow_4459_37617"/>
<feBlend mode="normal" in="SourceGraphic" in2="effect1_dropShadow_4459_37617" result="shape"/>
</filter>
<filter id="filter1_d_4459_37617" x="4.4644" y="0.852951" width="154.953" height="103.797" filterUnits="userSpaceOnUse" color-interpolation-filters="sRGB">
<feFlood flood-opacity="0" result="BackgroundImageFix"/>
<feColorMatrix in="SourceAlpha" type="matrix" values="0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 127 0" result="hardAlpha"/>
<feOffset dy="1.32776"/>
<feGaussianBlur stdDeviation="4.9791"/>
<feComposite in2="hardAlpha" operator="out"/>
<feColorMatrix type="matrix" values="0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0.25 0"/>
<feBlend mode="normal" in2="BackgroundImageFix" result="effect1_dropShadow_4459_37617"/>
<feBlend mode="normal" in="SourceGraphic" in2="effect1_dropShadow_4459_37617" result="shape"/>
</filter>
<filter id="filter2_d_4459_37617" x="11.4351" y="5.50094" width="141.011" height="89.8558" filterUnits="userSpaceOnUse" color-interpolation-filters="sRGB">
<feFlood flood-opacity="0" result="BackgroundImageFix"/>
<feColorMatrix in="SourceAlpha" type="matrix" values="0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 127 0" result="hardAlpha"/>
<feOffset dy="1.32776"/>
<feGaussianBlur stdDeviation="1.49373"/>
<feComposite in2="hardAlpha" operator="out"/>
<feColorMatrix type="matrix" values="0 0 0 0 0.253904 0 0 0 0 0.093523 0 0 0 0 0.968327 0 0 0 0.25 0"/>
<feBlend mode="normal" in2="BackgroundImageFix" result="effect1_dropShadow_4459_37617"/>
<feBlend mode="normal" in="SourceGraphic" in2="effect1_dropShadow_4459_37617" result="shape"/>
</filter>
<filter id="filter3_f_4459_37617" x="-31.6608" y="-2.5851" width="131.246" height="103.284" filterUnits="userSpaceOnUse" color-interpolation-filters="sRGB">
<feFlood flood-opacity="0" result="BackgroundImageFix"/>
<feBlend mode="normal" in="SourceGraphic" in2="BackgroundImageFix" result="shape"/>
<feGaussianBlur stdDeviation="21.8599" result="effect1_foregroundBlur_4459_37617"/>
</filter>
<filter id="filter4_f_4459_37617" x="51.7347" y="5.85435" width="131.246" height="120.185" filterUnits="userSpaceOnUse" color-interpolation-filters="sRGB">
<feFlood flood-opacity="0" result="BackgroundImageFix"/>
<feBlend mode="normal" in="SourceGraphic" in2="BackgroundImageFix" result="shape"/>
<feGaussianBlur stdDeviation="21.8599" result="effect1_foregroundBlur_4459_37617"/>
</filter>
<filter id="filter5_di_4459_37617" x="58.838" y="37.0151" width="44.7778" height="27.7493" filterUnits="userSpaceOnUse" color-interpolation-filters="sRGB">
<feFlood flood-opacity="0" result="BackgroundImageFix"/>
<feColorMatrix in="SourceAlpha" type="matrix" values="0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 127 0" result="hardAlpha"/>
<feOffset dy="1.32776"/>
<feGaussianBlur stdDeviation="0.663881"/>
<feComposite in2="hardAlpha" operator="out"/>
<feColorMatrix type="matrix" values="0 0 0 0 0.222242 0 0 0 0 0.247729 0 0 0 0 0.604557 0 0 0 0.6 0"/>
<feBlend mode="normal" in2="BackgroundImageFix" result="effect1_dropShadow_4459_37617"/>
<feBlend mode="normal" in="SourceGraphic" in2="effect1_dropShadow_4459_37617" result="shape"/>
<feColorMatrix in="SourceAlpha" type="matrix" values="0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 127 0" result="hardAlpha"/>
<feOffset dy="0.995821"/>
<feGaussianBlur stdDeviation="1.32776"/>
<feComposite in2="hardAlpha" operator="arithmetic" k2="-1" k3="1"/>
<feColorMatrix type="matrix" values="0 0 0 0 1 0 0 0 0 0.923763 0 0 0 0 0.923763 0 0 0 0.45 0"/>
<feBlend mode="normal" in2="shape" result="effect2_innerShadow_4459_37617"/>
</filter>
<filter id="filter6_d_4459_37617" x="110.407" y="48.1595" width="40.5442" height="39.3401" filterUnits="userSpaceOnUse" color-interpolation-filters="sRGB">
<feFlood flood-opacity="0" result="BackgroundImageFix"/>
<feColorMatrix in="SourceAlpha" type="matrix" values="0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 127 0" result="hardAlpha"/>
<feOffset dy="3.42944"/>
<feGaussianBlur stdDeviation="4.2868"/>
<feComposite in2="hardAlpha" operator="out"/>
<feColorMatrix type="matrix" values="0 0 0 0 0.0969792 0 0 0 0 0.127894 0 0 0 0 0.2375 0 0 0 1 0"/>
<feBlend mode="normal" in2="BackgroundImageFix" result="effect1_dropShadow_4459_37617"/>
<feBlend mode="normal" in="SourceGraphic" in2="effect1_dropShadow_4459_37617" result="shape"/>
</filter>
<linearGradient id="paint0_linear_4459_37617" x1="109.337" y1="36.0149" x2="77.2234" y2="84.8818" gradientUnits="userSpaceOnUse">
<stop stop-color="#F9C1FF"/>
<stop offset="1" stop-color="#A950FF"/>
</linearGradient>
<linearGradient id="paint1_linear_4459_37617" x1="109.337" y1="36.0149" x2="77.2234" y2="84.8818" gradientUnits="userSpaceOnUse">
<stop stop-color="#F9C1FF"/>
<stop offset="1" stop-color="#A950FF"/>
</linearGradient>
<clipPath id="clip0_4459_37617">
<rect width="148.229" height="85.9522" fill="white" transform="matrix(-1 0 0 1 156.114 6.08545)"/>
</clipPath>
</defs>
</svg>

After

Width:  |  Height:  |  Size: 11 KiB

View File

@@ -1,3 +1,3 @@
* Explore our [documentation](https://kestra.io/docs)
* Learn the [concepts](https://kestra.io/docs/concepts)
* Browse Kestra [integrations](https://kestra.io/plugins)
* [Video Tutorials](https://kestra.io/tutorial-videos/all)
* [Documentation](https://kestra.io/docs)
* [Blueprints](https://kestra.io/blueprints)

View File

@@ -1,3 +1 @@
Our community of data engineers and developers are here to help.
[Join our Slack](https://kestra.io/slack)
Ask any question in our Slack community. If you are stuck, we are help to help you. ✋

View File

@@ -1,3 +1 @@
Follow each step one by one with this advanced tutorial.
[Follow the tutorial](/ui/flows/new?reset=true)
Chhose your use case and follow a step-by-step guide to learn Kestra 's features and capabilities. ❤️

View File

@@ -186,5 +186,4 @@ code {
}
}
}
</style>

View File

@@ -73,40 +73,48 @@
@update:select-all="toggleAllSelection"
@unselect="toggleAllUnselected"
>
<!-- Always visible buttons -->
<el-button v-if="canUpdate" :icon="StateMachine" @click="changeStatusDialogVisible = !changeStatusDialogVisible">
{{ $t("change state") }}
</el-button>
<el-button v-if="canUpdate" :icon="Restart" @click="restartExecutions()">
{{ $t("restart") }}
</el-button>
<el-button v-if="canCreate" :icon="PlayBoxMultiple" @click="replayExecutions()">
{{ $t("replay") }}
</el-button>
<el-button v-if="canUpdate" :icon="StateMachine" @click="changeStatusDialogVisible = !changeStatusDialogVisible">
{{ $t("change state") }}
</el-button>
<el-button v-if="canUpdate" :icon="StopCircleOutline" @click="killExecutions()">
{{ $t("kill") }}
</el-button>
<el-button v-if="canDelete" :icon="Delete" @click="deleteExecutions()">
{{ $t("delete") }}
</el-button>
<el-button
v-if="canUpdate"
:icon="LabelMultiple"
@click="isOpenLabelsModal = !isOpenLabelsModal"
>
{{ $t("Set labels") }}
</el-button>
<el-button v-if="canUpdate" :icon="PlayBox" @click="resumeExecutions()">
{{ $t("resume") }}
</el-button>
<el-button v-if="canUpdate" :icon="PauseBox" @click="pauseExecutions()">
{{ $t("pause") }}
</el-button>
<el-button v-if="canUpdate" :icon="QueueFirstInLastOut" @click="unqueueExecutions()">
{{ $t("unqueue") }}
</el-button>
<el-button v-if="canUpdate" :icon="RunFast" @click="forceRunExecutions()">
{{ $t("force run") }}
</el-button>
<!-- Dropdown with additional actions -->
<el-dropdown>
<el-button>
<DotsVertical />
</el-button>
<template #dropdown>
<el-dropdown-menu>
<el-dropdown-item v-if="canUpdate" :icon="LabelMultiple" @click=" isOpenLabelsModal = !isOpenLabelsModal">
{{ $t("Set labels") }}
</el-dropdown-item>
<el-dropdown-item v-if="canUpdate" :icon="PlayBox" @click="resumeExecutions()">
{{ $t("resume") }}
</el-dropdown-item>
<el-dropdown-item v-if="canUpdate" :icon="PauseBox" @click="pauseExecutions()">
{{ $t("pause") }}
</el-dropdown-item>
<el-dropdown-item v-if="canUpdate" :icon="QueueFirstInLastOut" @click="unqueueExecutions()">
{{ $t("unqueue") }}
</el-dropdown-item>
<el-dropdown-item v-if="canUpdate" :icon="RunFast" @click="forceRunExecutions()">
{{ $t("force run") }}
</el-dropdown-item>
</el-dropdown-menu>
</template>
</el-dropdown>
</bulk-select>
<el-dialog
v-if="isOpenLabelsModal"
@@ -344,6 +352,7 @@
import SelectTable from "../layout/SelectTable.vue";
import PlayBox from "vue-material-design-icons/PlayBox.vue";
import PlayBoxMultiple from "vue-material-design-icons/PlayBoxMultiple.vue";
import DotsVertical from "vue-material-design-icons/DotsVertical.vue";
import Restart from "vue-material-design-icons/Restart.vue";
import Delete from "vue-material-design-icons/Delete.vue";
import StopCircleOutline from "vue-material-design-icons/StopCircleOutline.vue";

View File

@@ -27,6 +27,7 @@
@previous="previousLogForLevel(logLevel)"
@next="nextLogForLevel(logLevel)"
@close="logCursor = undefined"
class="w-100"
/>
</el-form-item>
<el-form-item>

View File

@@ -127,13 +127,28 @@
const debugEditor = ref(null);
const debugExpression = ref("");
const computedDebugValue = computed(() => {
const task = selectedTask()?.taskId;
if(!task) return "";
const formatTask = (task) => {
if (!task) return "";
return task.includes("-") ? `["${task}"]` : `.${task}`;
};
const path = expandedValue.value;
if(!path) return `{{ outputs.${task} }}`
const formatPath = (path) => {
if (!path.includes("-")) return `.${path}`;
return `{{ outputs.${path} }}`
const bracketIndex = path.indexOf("[");
const task = path.substring(0, bracketIndex);
const rest = path.substring(bracketIndex);
return `["${task}"]${rest}`;
}
let task = selectedTask()?.taskId;
if (!task) return "";
let path = expandedValue.value;
if (!path) return `{{ outputs${formatTask(task)} }}`;
return `{{ outputs${formatPath(path)} }}`;
});
const debugError = ref("");
const debugStackTrace = ref("");
@@ -225,11 +240,18 @@
if (!task) return;
selected.value = [task.value];
expandedValue.value = task.value;
const child = task.children?.[1];
if (child) {
selected.value.push(child.value);
expandedValue.value = child.path
expandedValue.value = child.path;
const grandChild = child.children?.[1];
if (grandChild) {
selected.value.push(grandChild.value);
expandedValue.value = grandChild.path;
}
}
debugCollapse.value = "debug";

View File

@@ -53,6 +53,11 @@
:key="comparator.value"
:value="comparator"
:label="comparator.label"
:class="{
selected: current.some(
(c) => c.comparator === comparator,
),
}"
@click="() => comparatorCallback(comparator)"
/>
</template>
@@ -62,6 +67,11 @@
:key="filter.value"
:value="filter"
:label="filter.label"
:class="{
selected: current.some((c) =>
c.value.includes(filter.value),
),
}"
@click="() => valueCallback(filter)"
/>
</template>
@@ -140,6 +150,9 @@
} = useFilters(props.prefix);
const select = ref<InstanceType<typeof ElSelect> | null>(null);
const updateHoveringIndex = (index) => {
select.value.states.hoveringIndex = index >= 0 ? index : 0;
};
const emptyLabel = ref(t("filters.empty"));
const INITIAL_DROPDOWNS = {
first: {shown: true, value: {}},
@@ -208,7 +221,9 @@
dropdowns.value.second = {shown: false, index: -1};
dropdowns.value.third = {shown: true, index: current.value.length - 1};
select.value.states.hoveringIndex = 0;
// Set hover index to the selected comparator for highlighting
const index = valueOptions.value.findIndex((o) => o.value === value.value);
updateHoveringIndex(index);
};
const dropdownClosedCallback = (visible) => {
if (!visible) {
@@ -216,6 +231,12 @@
// If last filter item selection was not completed, remove it from array
if (current.value?.at(-1)?.value?.length === 0) current.value.pop();
} else {
// Highlight all selected items by setting hoveringIndex to match the first selected item
const index = valueOptions.value.findIndex((o) => {
return current.value.some((c) => c.value.includes(o.value));
});
updateHoveringIndex(index);
}
};
const valueCallback = (filter, isDate = false) => {
@@ -225,6 +246,12 @@
if (index === -1) values.push(filter.value);
else values.splice(index, 1);
// Update the hover index for better UX
const hoverIndex = valueOptions.value.findIndex(
(o) => o.value === filter.value,
);
updateHoveringIndex(hoverIndex);
} else {
const match = current.value.find((v) => v.label === "absolute_date");
if (match) match.value = [filter];

View File

@@ -132,7 +132,7 @@
},
fillInputsFromExecution(){
// Add all labels except the one from flow to prevent duplicates
this.executionLabels = this.getExecutionLabels();
this.executionLabels = this.getExecutionLabels().filter(item => !item.key.startsWith("system."));
if (!this.flow.inputs) {
return;

View File

@@ -524,6 +524,7 @@
.custom-dark-vs-theme {
.monaco-editor, .monaco-editor-background {
outline: none;
background-color: $input-bg;
--vscode-editor-background: $input-bg;
--vscode-breadcrumb-background: $input-bg;

View File

@@ -1,5 +1,10 @@
<template>
<div v-show="explorerVisible" class="p-3 sidebar" @click="$refs.tree.setCurrentKey(undefined)">
<div
v-show="explorerVisible"
class="p-3 sidebar"
@click="$refs.tree.setCurrentKey(undefined)"
@contextmenu.prevent="onTabContextMenu"
>
<div class="d-flex flex-row">
<el-select
v-model="filter"
@@ -29,10 +34,7 @@
:persistent="false"
popper-class="text-base"
>
<el-button
class="px-2"
@click="toggleDialog(true, 'file')"
>
<el-button class="px-2" @click="toggleDialog(true, 'file')">
<FilePlus />
</el-button>
</el-tooltip>
@@ -94,10 +96,7 @@
:persistent="false"
popper-class="text-base"
>
<el-button
class="px-2"
@click="exportFiles()"
>
<el-button class="px-2" @click="exportFiles()">
<FolderDownloadOutline />
</el-button>
</el-tooltip>
@@ -110,7 +109,9 @@
:load="loadNodes"
:data="items"
highlight-current
:allow-drop="(_, drop, dropType) => !drop.data?.leaf || dropType !== 'inner'"
:allow-drop="
(_, drop, dropType) => !drop.data?.leaf || dropType !== 'inner'
"
draggable
node-key="id"
v-loading="items === undefined"
@@ -127,7 +128,12 @@
})
: undefined
"
@node-drag-start="nodeBeforeDrag = {parent: $event.parent.data.id, path: getPath($event.data.id)}"
@node-drag-start="
nodeBeforeDrag = {
parent: $event.parent.data.id,
path: getPath($event.data.id),
}
"
@node-drop="nodeMoved"
@keydown.delete.prevent="deleteKeystroke"
>
@@ -141,13 +147,19 @@
<template #default="{data, node}">
<el-dropdown
:ref="`dropdown__${data.id}`"
@contextmenu.prevent.stop="toggleDropdown(`dropdown__${data.id}`)"
@contextmenu.prevent.stop="
toggleDropdown(`dropdown__${data.id}`)
"
trigger="contextmenu"
class="w-100"
>
<el-row justify="space-between" class="w-100">
<el-col class="w-100">
<TypeIcon :name="data.fileName" :folder="!data.leaf" class="me-2" />
<TypeIcon
:name="data.fileName"
:folder="!data.leaf"
class="me-2"
/>
<span class="filename"> {{ data.fileName }}</span>
</el-col>
</el-row>
@@ -174,7 +186,7 @@
true,
!data.leaf ? 'folder' : 'file',
data.fileName,
node
node,
)
"
>
@@ -182,18 +194,16 @@
$t(
`namespace files.rename.${
!data.leaf ? "folder" : "file"
}`
}`,
)
}}
</el-dropdown-item>
<el-dropdown-item
@click="confirmRemove(node)"
>
<el-dropdown-item @click="confirmRemove(node)">
{{
$t(
`namespace files.delete.${
!data.leaf ? "folder" : "file"
}`
}`,
)
}}
</el-dropdown-item>
@@ -308,7 +318,7 @@
{{
Array.isArray(confirmation.node?.data?.children)
? $t(
"namespace files.dialog.folder_deletion_description"
"namespace files.dialog.folder_deletion_description",
)
: $t("namespace files.dialog.file_deletion_description")
}}
@@ -324,6 +334,22 @@
</div>
</template>
</el-dialog>
<el-menu
v-if="tabContextMenu.visible"
:style="{
left: `${tabContextMenu.x}px`,
top: `${tabContextMenu.y}px`,
}"
class="tabs-context"
>
<el-menu-item @click="toggleDialog(true, 'file')">
{{ $t("namespace files.create.file") }}
</el-menu-item>
<el-menu-item @click="toggleDialog(true, 'folder')">
{{ $t("namespace files.create.folder") }}
</el-menu-item>
</el-menu>
</div>
</template>
@@ -332,7 +358,7 @@
import Utils from "../../utils/utils";
import FileExplorerEmpty from "../../assets/icons/file_explorer_empty.svg"
import FileExplorerEmpty from "../../assets/icons/file_explorer_empty.svg";
import Magnify from "vue-material-design-icons/Magnify.vue";
import FilePlus from "vue-material-design-icons/FilePlus.vue";
@@ -340,7 +366,7 @@
import PlusBox from "vue-material-design-icons/PlusBox.vue";
import FolderDownloadOutline from "vue-material-design-icons/FolderDownloadOutline.vue";
import TypeIcon from "../utils/icons/Type.vue"
import TypeIcon from "../utils/icons/Type.vue";
const DIALOG_DEFAULTS = {
visible: false,
@@ -361,8 +387,8 @@
props: {
currentNS: {
type: String,
default: null
}
default: null,
},
},
components: {
Magnify,
@@ -370,7 +396,7 @@
FolderPlus,
PlusBox,
FolderDownloadOutline,
TypeIcon
TypeIcon,
},
data() {
return {
@@ -385,7 +411,8 @@
confirmation: {visible: false, data: {}},
items: undefined,
nodeBeforeDrag: undefined,
searchResults: []
searchResults: [],
tabContextMenu: {visible: false, x: 0, y: 0},
};
},
computed: {
@@ -401,7 +428,12 @@
if (item.type === "Directory") {
const folderPath = `${basePath}${item.fileName}`;
paths.push(folderPath);
paths.push(...extractPaths(`${folderPath}/`, item.children ?? []));
paths.push(
...extractPaths(
`${folderPath}/`,
item.children ?? [],
),
);
}
});
return paths;
@@ -411,7 +443,10 @@
},
},
methods: {
...mapMutations("editor", ["toggleExplorerVisibility", "changeOpenedTabs"]),
...mapMutations("editor", [
"toggleExplorerVisibility",
"changeOpenedTabs",
]),
...mapActions("namespace", [
"createDirectory",
"readDirectory",
@@ -425,8 +460,7 @@
]),
sorted(items) {
return items.sort((a, b) => {
if (a.type === "Directory" && b.type !== "Directory")
return -1;
if (a.type === "Directory" && b.type !== "Directory") return -1;
else if (a.type !== "Directory" && b.type === "Directory")
return 1;
@@ -451,11 +485,13 @@
},
async loadNodes(node, resolve) {
if (node.level === 0) {
const payload = {namespace: this.currentNS ?? this.$route.params.namespace};
const payload = {
namespace: this.currentNS ?? this.$route.params.namespace,
};
const items = await this.readDirectory(payload);
this.renderNodes(items);
this.items = this.sorted(this.items)
this.items = this.sorted(this.items);
}
if (node.level >= 1) {
@@ -470,7 +506,7 @@
...item,
id: Utils.uid(),
leaf: item.type === "File",
}))
})),
);
// eslint-disable-next-line no-inner-declarations
@@ -481,34 +517,39 @@
items[index].children = newChildren;
} else if (Array.isArray(item.children)) {
// Recursively search in children array
updateChildren(
item.children,
path,
newChildren
);
updateChildren(item.children, path, newChildren);
}
});
}
};
updateChildren(this.items, this.getPath(node.data.id), children);
updateChildren(
this.items,
this.getPath(node.data.id),
children,
);
resolve(children);
}
},
async searchFilesList(value) {
if(!value) return;
if (!value) return;
const results = await this.searchFiles({namespace: this.currentNS ?? this.$route.params.namespace, query: value});
this.searchResults = results.map(result => result.replace(/^\/*/, ""));
const results = await this.searchFiles({
namespace: this.currentNS ?? this.$route.params.namespace,
query: value,
});
this.searchResults = results.map((result) =>
result.replace(/^\/*/, ""),
);
return this.searchResults;
},
chooseSearchResults(item){
chooseSearchResults(item) {
this.changeOpenedTabs({
action: "open",
name: item.split("/").pop(),
extension: item.split(".")[1],
path: item,
})
});
this.filter = "";
},
@@ -586,7 +627,10 @@
});
} catch (e) {
this.$refs.tree.remove(draggedNode.data.id);
this.$refs.tree.append(draggedNode.data, this.nodeBeforeDrag.parent);
this.$refs.tree.append(
draggedNode.data,
this.nodeBeforeDrag.parent,
);
}
},
focusCreationInput() {
@@ -628,7 +672,7 @@
const folderIndex = currentFolder.findIndex(
(item) =>
typeof item === "object" &&
item.fileName === folderName
item.fileName === folderName,
);
if (folderIndex === -1) {
// If the folder doesn't exist, create it
@@ -636,7 +680,7 @@
id: Utils.uid(),
fileName: folderName,
children: [],
type: "Directory"
type: "Directory",
};
currentFolder.push(newFolder);
this.sorted(currentFolder);
@@ -656,7 +700,8 @@
const content = await this.readFile(file);
this.importFileDirectory({
namespace: this.currentNS ?? this.$route.params.namespace,
namespace:
this.currentNS ?? this.$route.params.namespace,
content,
path: `${folderPath}/${fileName}`,
});
@@ -668,7 +713,7 @@
extension ? `.${extension}` : ""
}`,
extension,
type: "File"
type: "File",
});
} else {
// Process files at root level (not in any folder)
@@ -676,7 +721,8 @@
const [name, extension] = file.name.split(".");
this.importFileDirectory({
namespace: this.currentNS ?? this.$route.params.namespace,
namespace:
this.currentNS ?? this.$route.params.namespace,
content,
path: file.name,
});
@@ -688,13 +734,13 @@
}`,
extension,
leaf: !!extension,
type: "File"
type: "File",
});
}
}
this.$toast().success(
this.$t("namespace files.import.success")
this.$t("namespace files.import.success"),
);
} catch (error) {
this.$toast().error(this.$t("namespace files.import.error"));
@@ -705,7 +751,9 @@
}
},
exportFiles() {
this.exportFileDirectory({namespace: this.currentNS ?? this.$route.params.namespace});
this.exportFileDirectory({
namespace: this.currentNS ?? this.$route.params.namespace,
});
},
async addFile({file, creation, shouldReset = true}) {
let FILE;
@@ -733,14 +781,15 @@
extension,
content,
leaf,
type: "File"
type: "File",
};
const path = `${this.dialog.folder ? `${this.dialog.folder}/` : ""}${NAME}`;
if (creation) {
if ((await this.searchFilesList(path)).includes(path)) {
this.$toast().error(this.$t("namespace files.create.already_exists"));
this.$toast().error(
this.$t("namespace files.create.already_exists"),
);
return;
}
await this.createFile({
@@ -755,7 +804,7 @@
action: "open",
name: NAME,
path,
extension: extension
extension: extension,
});
this.dialog.folder = path.substring(0, path.lastIndexOf("/"));
@@ -767,16 +816,28 @@
} else {
const SELF = this;
(function pushItemToFolder(basePath = "", array, pathParts) {
for (const item of array) {
const folderPath = `${basePath}${item.fileName}`;
if (folderPath === SELF.dialog.folder && Array.isArray(item.children)) {
item.children = SELF.sorted([...item.children, NEW]);
if (
folderPath === SELF.dialog.folder &&
Array.isArray(item.children)
) {
item.children = SELF.sorted([
...item.children,
NEW,
]);
return true; // Return true if the folder is found and item is pushed
}
if (Array.isArray(item.children) && pushItemToFolder(`${folderPath}/`, item.children, pathParts.slice(1))) {
if (
Array.isArray(item.children) &&
pushItemToFolder(
`${folderPath}/`,
item.children,
pathParts.slice(1),
)
) {
// Return true if the folder is found and item is pushed in recursive call
return true;
}
@@ -787,7 +848,9 @@
const folderPath = `${basePath}${pathParts[0]}`;
if (folderPath === SELF.dialog.folder) {
const newFolder = SELF.folderNode(pathParts[0], [NEW]);
const newFolder = SELF.folderNode(pathParts[0], [
NEW,
]);
array.push(newFolder);
array = SELF.sorted(array);
@@ -797,7 +860,11 @@
array.push(newFolder);
array = SELF.sorted(array);
return pushItemToFolder(`${basePath}${pathParts[0]}/`, newFolder.children, pathParts.slice(1));
return pushItemToFolder(
`${basePath}${pathParts[0]}/`,
newFolder.children,
pathParts.slice(1),
);
}
return false;
@@ -812,7 +879,10 @@
this.confirmation = {visible: true, node};
},
async removeItem() {
const {node, node: {data}} = this.confirmation;
const {
node,
node: {data},
} = this.confirmation;
await this.deleteFileDirectory({
namespace: this.currentNS ?? this.$route.params.namespace,
@@ -832,7 +902,11 @@
},
deleteKeystroke() {
if (this.$refs.tree.getCurrentNode()) {
this.confirmRemove(this.$refs.tree.getNode(this.$refs.tree.getCurrentNode().id));
this.confirmRemove(
this.$refs.tree.getNode(
this.$refs.tree.getCurrentNode().id,
),
);
}
},
async addFolder(folder, creation) {
@@ -842,7 +916,7 @@
fileName: this.dialog.name,
};
const NEW = this.folderNode(fileName, folder?.children ?? [])
const NEW = this.folderNode(fileName, folder?.children ?? []);
if (creation) {
const path = `${
@@ -873,7 +947,12 @@
item.children = SELF.sorted(item.children);
return true; // Return true if the folder is found and item is pushed
} else if (Array.isArray(item.children)) {
if (pushItemToFolder(`${folderPath}/`, item.children)) {
if (
pushItemToFolder(
`${folderPath}/`,
item.children,
)
) {
return true; // Return true if the folder is found and item is pushed in recursive call
}
}
@@ -890,8 +969,8 @@
fileName,
leaf: false,
children: children ?? [],
type: "Directory"
}
type: "Directory",
};
},
getPath(name) {
const nodes = this.$refs.tree.getNodePath(name);
@@ -906,7 +985,20 @@
} catch (_error) {
this.$toast().error(this.$t("namespace files.path.error"));
}
}
},
onTabContextMenu(event) {
this.tabContextMenu = {
visible: true,
x: event.clientX,
y: event.clientY,
};
document.addEventListener("click", this.hideTabContextMenu);
},
hideTabContextMenu() {
this.tabContextMenu.visible = false;
document.removeEventListener("click", this.hideTabContextMenu);
},
},
watch: {
flow: {
@@ -929,99 +1021,116 @@
</script>
<style lang="scss">
.filter .el-input__wrapper {
padding-right: 0px;
.filter .el-input__wrapper {
padding-right: 0px;
}
.el-tree {
height: calc(100% - 64px);
overflow: hidden auto;
.el-tree__empty-block {
height: auto;
}
.el-tree {
height: calc(100% - 64px);
overflow: hidden auto;
&::-webkit-scrollbar {
width: 2px;
}
.el-tree__empty-block {
height: auto;
}
&::-webkit-scrollbar-track {
background: var(--card-bg);
}
&::-webkit-scrollbar {
width: 2px;
}
&::-webkit-scrollbar-thumb {
background: var(--bs-primary);
border-radius: 0px;
}
&::-webkit-scrollbar-track {
background: var(--card-bg);
}
.node {
--el-tree-node-content-height: 36px;
--el-tree-node-hover-bg-color: transparent;
line-height: 36px;
&::-webkit-scrollbar-thumb {
background: var(--bs-primary);
border-radius: 0px;
}
.node {
--el-tree-node-content-height: 36px;
--el-tree-node-hover-bg-color: transparent;
line-height: 36px;
.el-tree-node__content {
width: 100%;
}
.el-tree-node__content {
width: 100%;
}
}
}
</style>
<style lang="scss" scoped>
@import "@kestra-io/ui-libs/src/scss/variables.scss";
@import "@kestra-io/ui-libs/src/scss/variables.scss";
.sidebar {
background: var(--card-bg);
border-right: 1px solid var(--bs-border-color);
.sidebar {
background: var(--card-bg);
border-right: 1px solid var(--bs-border-color);
.empty {
position: relative;
top: 100px;
text-align: center;
color: white;
.empty {
position: relative;
top: 100px;
text-align: center;
color: white;
html.light & {
color: $tertiary;
}
& img {
margin-bottom: 2rem;
}
& h3 {
font-size: var(--font-size-lg);
font-weight: 500;
margin-bottom: .5rem;
}
& p {
font-size: var(--font-size-sm);
}
html.light & {
color: $tertiary;
}
:deep(.el-button):not(.el-dialog .el-button) {
border: 0;
background: none;
outline: none;
opacity: 0.5;
padding-left: calc(var(--spacer) / 2);
padding-right: calc(var(--spacer) / 2);
&.el-button--primary {
opacity: 1;
}
& img {
margin-bottom: 2rem;
}
.hidden {
display: none;
& h3 {
font-size: var(--font-size-lg);
font-weight: 500;
margin-bottom: 0.5rem;
}
.filename {
& p {
font-size: var(--font-size-sm);
}
}
:deep(.el-button):not(.el-dialog .el-button) {
border: 0;
background: none;
outline: none;
opacity: 0.5;
padding-left: calc(var(--spacer) / 2);
padding-right: calc(var(--spacer) / 2);
&.el-button--primary {
opacity: 1;
}
}
.hidden {
display: none;
}
.filename {
font-size: var(--el-font-size-small);
color: var(--el-text-color-regular);
&:hover {
color: var(--el-text-color-primary);
}
}
ul.tabs-context {
position: fixed;
z-index: 9999;
border: 1px solid var(--bs-border-color);
& li {
height: 30px;
padding: 16px;
font-size: var(--el-font-size-small);
color: var(--el-text-color-regular);
color: var(--bs-gray-900);
&:hover {
color: var(--el-text-color-primary);
color: var(--bs-secondary);
}
}
}
</style>
}
</style>

View File

@@ -720,6 +720,10 @@
height: 100%;
outline: none;
}
.main-editor > #editorWrapper .monaco-editor {
padding: 1rem 0 0 1rem;
}
</style>
<style lang="scss">

View File

@@ -1,5 +1,5 @@
<template>
<nav data-component="FILENAME_PLACEHOLDER" class="d-flex w-100 gap-3 top-bar" v-if="displayNavBar">
<nav data-component="FILENAME_PLACEHOLDER" class="d-flex w-100 gap-3 top-bar">
<div class="d-flex flex-column flex-grow-1 flex-shrink-1 overflow-hidden top-title">
<el-breadcrumb v-if="breadcrumb">
<el-breadcrumb-item v-for="(item, x) in breadcrumb" :key="x">
@@ -74,9 +74,6 @@
...mapState("bookmarks", ["pages"]),
...mapGetters("core", ["guidedProperties"]),
...mapGetters("auth", ["user"]),
displayNavBar() {
return this.$route?.name !== "welcome";
},
tourEnabled(){
// Temporary solution to not showing the tour menu item for EE
return this.tutorialFlows?.length && !Object.keys(this.user).length

View File

@@ -1,23 +1,14 @@
<template>
<el-row justify="space-between" :gutter="20">
<el-col
<div class="onboarding-bottom">
<onboarding-card
v-for="card in cards"
:key="card.title"
:xs="24"
:sm="12"
:md="12"
:lg="6"
:xl="6"
class="pb-4"
>
<onboarding-card
:title="card.title"
:content="card.content"
:category="card.category"
:link="card.link"
/>
</el-col>
</el-row>
:title="card.title"
:content="card.content"
:category="card.category"
:link="card.link"
/>
</div>
</template>
<script>
import {mapGetters} from "vuex";
@@ -30,17 +21,13 @@
data() {
return {
cards: [
{
title: this.$t("welcome.started.title"),
category: "started",
},
{
title: this.$t("welcome.product-tour.title"),
category: "product",
},
{
title: this.$t("welcome.doc.title"),
title: this.$t("welcome.tutorial.title"),
category: "docs",
},
{
@@ -54,4 +41,15 @@
...mapGetters("core", ["guidedProperties"])
}
}
</script>
</script>
<style lang="scss" scoped>
.onboarding-bottom {
display: flex;
gap: 1rem;
margin-top: 1.5rem;
justify-items: center;
flex-wrap: wrap;
max-width: 1000px;
}
</style>

View File

@@ -1,22 +1,35 @@
<template>
<el-card>
<template #header>
<img :src="img" alt="">
</template>
<div class="content row">
<p class="fw-bold text-uppercase smaller-text">
{{ title }}
</p>
<markdown :source="mdContent" class="mt-4" />
<el-card class="box-card">
<div class="card-content">
<div class="card-header">
<el-link v-if="isOpenInNewCategory" :underline="false" :icon="OpenInNew" :href="getLink()" target="_blank" />
</div>
<div class="icon-title">
<el-icon size="25px">
<component :is="getIcon()" />
</el-icon>
<div class="card">
<h5 class="cat_title">
{{ title }}
</h5>
<div class="cat_description">
<markdown :source="mdContent" />
</div>
</div>
</div>
</div>
</el-card>
</template>
<script setup>
import OpenInNew from "vue-material-design-icons/OpenInNew.vue"
import Monitor from "vue-material-design-icons/Monitor.vue"
import Slack from "vue-material-design-icons/Slack.vue"
import PlayBox from "vue-material-design-icons/PlayBoxMultiple.vue"
</script>
<script>
import imageStarted from "../../assets/onboarding/onboarding-started-dark.svg"
import imageHelp from "../../assets/onboarding/onboarding-help-dark.svg"
import imageDoc from "../../assets/onboarding/onboarding-docs-dark.svg"
import imageProduct from "../../assets/onboarding/onboarding-product-dark.svg"
import Markdown from "../layout/Markdown.vue";
import Utils from "../../utils/utils.js";
@@ -47,6 +60,26 @@
.then((module) => {
this.markdownContent = module.default;
})
},
getIcon() {
switch (this.category) {
case "help":
return Slack;
case "docs":
return PlayBox;
case "product":
return Monitor;
default:
return Monitor;
}
},
getLink() {
// Define links for the specific categories
const links = {
help: "https://kestra.io/slack",
docs: "https://kestra.io/docs"
};
return links[this.category] || "#"; // Default to "#" if no link is found
}
},
computed: {
@@ -57,48 +90,67 @@
}
return ""
},
img() {
switch (this.category) {
case "started":
return imageStarted;
case "help":
return imageHelp;
case "docs":
return imageDoc;
case "product":
return imageProduct;
}
return imageStarted
},
mdContent() {
return this.markdownContent;
},
isOpenInNewCategory() {
// Define which categories should show the OpenInNew icon
return this.category === "help" || this.category === "docs";
}
}
}
</script>
<style scoped lang="scss">
a:hover {
text-decoration: none;
}
.el-card {
background-color: var(--card-bg);
border-color: var(--el-border-color);
box-shadow: var(--el-box-shadow);
position: relative;
min-width: 250px;
flex: 1;
cursor: pointer;
&:deep(.el-card__header) {
padding: 0;
}
position: relative;
height: 100%;
cursor: pointer;
}
.smaller-text {
font-size: 0.86em;
.box-card {
.card-header {
position: absolute;
top: 5px;
right: 5px;
}
.cat_title {
width: 100%;
margin: 3px 0 10px;
padding-left: 20px;
font-weight: 600;
font-size: var(--el-font-size-small);
}
.cat_description {
width: 100%;
margin: 0;
padding-left: 20px;
}
}
p {
margin-bottom: 0;
.icon-title {
display: inline-flex;
&.icon-title-left {
margin-right: 10px;
}
}
img {
width: 100%;
height: 100%;
.el-link {
font-size: 20px;
}
</style>

View File

@@ -1,90 +1,136 @@
<template>
<el-col class="main">
<el-row :gutter="20">
<el-col :xs="24" :sm="24" :md="24" :lg="14" :xl="14" class="mb-4">
<el-card class="px-3 pt-4">
<el-row justify="space-around" class="p-5">
<el-col :xs="24" :sm="24" :md="24" :lg="12" :xl="12" justify="space-between">
<el-row class="mb-5" justify="center">
<img class="img-fluid" :src="logo" alt="Kestra Logo">
</el-row>
<el-row justify="center">
<router-link :to="{name: 'flows/create'}">
<el-button size="large" type="primary">
<Plus />
{{ $t("welcome button create") }}
</el-button>
</router-link>
</el-row>
</el-col>
<el-col :xs="24" :sm="24" :md="24" :lg="12" :xl="12" justify="center" class="mt-4">
<img :src="codeImage" class="img-fluid" alt="code example">
</el-col>
</el-row>
</el-card>
</el-col>
<el-col :xs="24" :sm="24" :md="24" :lg="10" :xl="10" class="mb-4">
<iframe
width="100%"
height="100%"
src="https://www.youtube.com/embed/a2BZ7vOihjg?si=gHZuap7frp5c8HVx"
frameborder="0"
allow="accelerometer; autoplay; clipboard-write; encrypted-media; gyroscope; picture-in-picture; web-share"
allowfullscreen
/>
</el-col>
</el-row>
<onboarding-bottom />
</el-col>
<top-nav-bar v-if="topbar" :title="routeInfo.title">
<template #additional-right>
<ul>
<li>
<el-button v-if="canCreate" tag="router-link" :to="{name: 'flows/create', query: {namespace: $route.query.namespace}}" :icon="Plus" type="primary">
{{ $t('create_flow') }}
</el-button>
</li>
</ul>
</template>
</top-nav-bar>
<div class="main">
<div class="section-1">
<div class="section-1-main">
<div class="section-content">
<img
:src="logo"
alt="Kestra"
class="section-1-img img-fluid"
width="180px"
>
<h2 class="section-1-title">
{{ $t("homeDashboard.wel_text") }}
</h2>
<p class="section-1-desc">
{{ $t("homeDashboard.start") }}
</p>
<router-link :to="{name: 'flows/create'}">
<el-button
:icon="Plus"
size="large"
type="primary"
class="px-3 p-4 section-1-link product-link"
>
{{ $t("welcome button create") }}
</el-button>
</router-link>
<el-button
:icon="Play"
tag="a"
href="https://www.youtube.com/watch?v=a2BZ7vOihjg"
target="_blank"
class="p-3 px-4 mt-0 mb-lg-5 watch"
>
Watch Video
</el-button>
</div>
<div class="mid-bar mb-3">
<div class="title title--center-line">
{{ $t("homeDashboard.guide") }}
</div>
</div>
<onboarding-bottom />
</div>
</div>
</div>
</template>
<script>
import {mapGetters} from "vuex";
<script setup>
import Plus from "vue-material-design-icons/Plus.vue";
import Play from "vue-material-design-icons/Play.vue";
</script>
<script>
import {mapGetters, mapState} from "vuex";
import OnboardingBottom from "./OnboardingBottom.vue";
import onboardingImage from "../../assets/onboarding/onboarding-dark.svg"
import onboardingImageLight from "../../assets/onboarding/onboarding-light.svg"
import codeImageDark from "../../assets/onboarding/onboarding-code-dark.svg"
import codeImageLight from "../../assets/onboarding/onboarding-code-light.svg"
import kestraWelcome from "../../assets/onboarding/kestra_welcome.svg";
import TopNavBar from "../../components/layout/TopNavBar.vue";
import RouteContext from "../../mixins/routeContext";
import RestoreUrl from "../../mixins/restoreUrl";
import permission from "../../models/permission";
import action from "../../models/action";
export default {
name: "CreateFlow",
mixins: [RouteContext, RestoreUrl],
components: {
OnboardingBottom,
Plus
TopNavBar
},
data() {
return {
onboardingImage,
props: {
topbar: {
type: Boolean,
default: true
}
},
computed: {
...mapGetters("core", ["guidedProperties"]),
...mapState("auth", ["user"]),
logo() {
// get theme
return (localStorage.getItem("theme") || "light") === "light" ? onboardingImageLight : onboardingImage;
return (localStorage.getItem("theme") || "light") === "light" ? kestraWelcome : kestraWelcome;
},
codeImage() {
return (localStorage.getItem("theme") || "light") === "light" ? codeImageLight : codeImageDark;
routeInfo() {
return {
title: this.$t("homeDashboard.welcome")
};
},
canCreate() {
return this.user && this.user.hasAnyActionOnAnyNamespace(permission.FLOW, action.CREATE);
}
}
}
</script>
<style scoped lang="scss">
.main {
margin: 3rem 1rem 1rem;
padding: 3rem 1rem 1rem;
background: var(--el-text-color-primary);
background: radial-gradient(ellipse at top, rgba(102,51,255,0.6) 0%, rgba(253, 253, 253, 0) 20%);
background-size: 4000px;
background-position: center;
height: 100%;
width: auto;
display: flex;
flex-direction: column;
container-type: inline-size;
@media (min-width: 768px) {
margin: 3rem 2rem 1rem;
padding: 3rem 2rem 1rem;
}
@media (min-width: 992px) {
margin: 3rem 3rem 1rem;
padding: 3rem 3rem 1rem;
}
@media (min-width: 1920px) {
margin: 3rem 10rem 1rem;
padding: 3rem 10rem 1rem;
}
}
@@ -93,8 +139,114 @@
height: auto;
}
.el-button {
font-size: var(--font-size-lg);
margin-bottom: calc(var(--spacer) * 2);
.product-link, .watch {
background: var(--el-button-bg-color);
color: var(--el-button-text-color);
font-weight: 700;
border-radius: 5px;
border: 1px solid var(--el-button-border-color);
text-decoration: none;
font-size: var(--el-font-size-small);
width: 200px;
margin-bottom: calc(var(--spacer));
}
.watch {
font-weight: 500;
background-color: var(--el-bg-color);
color: var(--el-text-color-regular);
font-size: var(--el-font-size-small);
}
.main .section-1 {
display: flex;
flex-grow: 1;
justify-content: center;
align-items: center;
border-radius: var(--bs-border-radius);
}
.section-1-main {
.section-content {
width: 100%;
display: flex;
flex-direction: column;
align-items: center;
.section-1-title {
line-height: var(--el-font-line-height-primary);
text-align: center;
font-size: var(--el-font-size-extra-large);
font-weight: 600;
color: var(--el-text-color-regular);
}
.section-1-desc {
line-height: var(--el-font-line-height-primary);
font-weight: 500;
font-size: 1rem;
text-align: center;
color: var(--el-text-color-regular);
}
}
.mid-bar {
margin-top: 50px;
.title {
font-weight: 500;
color: var(--bs-gray-900-lighten-5);
display: flex;
align-items: center;
white-space: nowrap;
font-size: var(--el-font-size-extra-small);
&--center-line {
text-align: center;
padding: 0;
&::before,
&::after {
content: "";
background-color: var(--bs-gray-600-lighten-10);
height: 2px;
width: 50%;
}
&::before {
margin-right: 1rem;
}
&::after {
margin-left: 1rem;
}
}
}
}
}
@container (max-width: 20px) {
.main .section-1 .section-1-main {
width: 90%;
}
}
@container (max-width: 50px) {
.main .section-1 .section-1-main {
padding-top: 30px;
}
.section-1 .section-1-main .container {
width: 76%;
}
.title--center-line {
&::before,
&::after {
width: 50%;
}
}
}
</style>

View File

@@ -233,7 +233,7 @@
.plugin-card {
display: flex;
width: 232px;
width: 100%;
min-width: 130px;
padding: 8px 16px;
align-items: center;

View File

@@ -1,7 +1,7 @@
<template>
<div class="plugins-list">
<el-input
class="search p-2"
class="p-2 bg-transparent search"
:placeholder="$t('pluginPage.search', {count: countPlugin})"
v-model="searchInput"
clearable

View File

@@ -52,7 +52,7 @@
>
<div class="left">
<div>
<div class="title">
<div class="ps-0 title">
{{ blueprint.title }}
</div>
<div v-if="!system" class="tags text-uppercase">

View File

@@ -360,7 +360,7 @@
},
"cancel": "Abbrechen",
"homeDashboard": {
"title": "Übersicht",
"title": "Dashboard",
"today": "Heute",
"yesterday": "Gestern",
"last28Days": "Letzte 28 Tage",
@@ -369,7 +369,11 @@
"namespacesErrorExecutions": "Ausführungsfehler pro Namespace",
"failedExecutions": "Fehlgeschlagene Ausführungen",
"errorLogs": "Fehler-Logs",
"no executions": "Bereit, Ihren Flow in Aktion zu sehen?"
"no executions": "Bereit, Ihren Flow in Aktion zu sehen?",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"welcome": "Welcome to Kestra"
},
"executions replayed": "<code>{executionCount}</code> Ausführung(en) erneut abgespielt",
"executions resumed": "<code>{executionCount}</code> Ausführung(en) fortgesetzt",
@@ -654,6 +658,9 @@
},
"need-help": {
"title": "Brauchen Sie Hilfe?"
},
"tutorial": {
"title": "Tutorial"
}
},
"pluginPage": {
@@ -819,7 +826,7 @@
},
"curl": {
"command": "cURL-Befehl",
"note": "Die Inputs vom Typ SECRET und FILE müssen immer bei der Ausführung angegeben werden."
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "Temporale Ansicht",

View File

@@ -410,7 +410,11 @@
},
"cancel": "Cancel",
"homeDashboard": {
"title": "Overview",
"title": "Dashboard",
"welcome": "Welcome to Kestra",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"today": "Today",
"yesterday": "Yesterday",
"last28Days": "Last 28 days",
@@ -719,6 +723,9 @@
"doc": {
"title": "Docs"
},
"tutorial": {
"title": "Tutorial"
},
"need-help": {
"title": "Need help?"
}
@@ -888,7 +895,7 @@
},
"curl": {
"command": "cURL command",
"note": "For SECRET and FILE-type inputs, adjust the command to match the actual value."
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "Temporal view",

View File

@@ -360,7 +360,7 @@
},
"cancel": "Cancelar",
"homeDashboard": {
"title": "Resumen",
"title": "Dashboard",
"today": "Hoy",
"yesterday": "Ayer",
"last28Days": "Últimos 28 días",
@@ -369,7 +369,11 @@
"namespacesErrorExecutions": "Errores de ejecuciones por namespace",
"failedExecutions": "Ejecuciones FAILED",
"errorLogs": "Logs de errores",
"no executions": "¿Listo para ver tu flow en acción?"
"no executions": "¿Listo para ver tu flow en acción?",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"welcome": "Welcome to Kestra"
},
"executions replayed": "<code>{executionCount}</code> ejecución(es) reproducidas",
"executions resumed": "<code>{executionCount}</code> ejecución(es) reanudadas",
@@ -654,6 +658,9 @@
},
"need-help": {
"title": "¿Necesitas ayuda?"
},
"tutorial": {
"title": "Tutorial"
}
},
"pluginPage": {
@@ -819,7 +826,7 @@
},
"curl": {
"command": "Comando cURL",
"note": "Ten en cuenta que para el tipo de input SECRET y FILE, el comando debe adaptarse para coincidir con los valores reales."
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "Vista temporal",

View File

@@ -360,7 +360,7 @@
},
"cancel": "Annuler",
"homeDashboard": {
"title": "Vue d'ensemble",
"title": "Dashboard",
"today": "Aujourd'hui",
"yesterday": "Hier",
"last28Days": "Dernier 28 jours",
@@ -369,7 +369,11 @@
"namespacesErrorExecutions": "Exécutions en erreur par espace de nom",
"failedExecutions": "Exécutions en échec",
"errorLogs": "Journaux d'erreurs",
"no executions": "Prêt à voir votre flow Kestra en action ?"
"no executions": "Prêt à voir votre flow Kestra en action ?",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"welcome": "Welcome to Kestra"
},
"executions resumed": "<code>{executionCount}</code> exécution(s) reprise(s)",
"executions replayed": "<code>{executionCount}</code> exécution(s) rejouée(s)",
@@ -654,6 +658,9 @@
},
"need-help": {
"title": "Besoin d'aide ?"
},
"tutorial": {
"title": "Tutorial"
}
},
"pluginPage": {
@@ -819,7 +826,7 @@
},
"curl": {
"command": "Commande cURL",
"note": "Notez que pour les types d'entrée SECRET et FILE, la commande doit être adaptée pour correspondre aux valeurs réelles."
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "Vue temporelle",

View File

@@ -360,7 +360,7 @@
},
"cancel": "रद्द करें",
"homeDashboard": {
"title": "सारांश",
"title": "Dashboard",
"today": "आज",
"yesterday": "कल",
"last28Days": "पिछले 28 दिन",
@@ -369,7 +369,11 @@
"namespacesErrorExecutions": "प्रति namespace निष्पादन त्रुटियाँ",
"failedExecutions": "असफल निष्पादन",
"errorLogs": "त्रुटि Logs",
"no executions": "क्या आप अपने flow को क्रियान्वित होते देखना चाहते हैं?"
"no executions": "क्या आप अपने flow को क्रियान्वित होते देखना चाहते हैं?",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"welcome": "Welcome to Kestra"
},
"executions replayed": "<code>{executionCount}</code> निष्पादन पुनः चलाए गए",
"executions resumed": "<code>{executionCount}</code> निष्पादन फिर से शुरू किए गए",
@@ -654,6 +658,9 @@
},
"need-help": {
"title": "मदद चाहिए?"
},
"tutorial": {
"title": "Tutorial"
}
},
"pluginPage": {
@@ -819,7 +826,7 @@
},
"curl": {
"command": "cURL कमांड",
"note": "ध्यान दें की SECRET और FILE इनपुट प्रकार के लिए, कमांड को वास्तविक मानों से मेल खाने के लिए समायोजित करना होगा।"
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "Temporal दृश्य",

View File

@@ -360,7 +360,7 @@
},
"cancel": "Annulla",
"homeDashboard": {
"title": "Panoramica",
"title": "Dashboard",
"today": "Oggi",
"yesterday": "Ieri",
"last28Days": "Ultimi 28 giorni",
@@ -369,7 +369,11 @@
"namespacesErrorExecutions": "Errori di esecuzione per namespace",
"failedExecutions": "Esecuzioni FAILED",
"errorLogs": "Error logs",
"no executions": "Pronto a vedere il tuo flow in azione?"
"no executions": "Pronto a vedere il tuo flow in azione?",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"welcome": "Welcome to Kestra"
},
"executions replayed": "<code>{executionCount}</code> esecuzione/i ripetute",
"executions resumed": "<code>{executionCount}</code> esecuzione/i riprese",
@@ -654,6 +658,9 @@
},
"need-help": {
"title": "Hai bisogno di aiuto?"
},
"tutorial": {
"title": "Tutorial"
}
},
"pluginPage": {
@@ -819,7 +826,7 @@
},
"curl": {
"command": "Comando cURL",
"note": "Nota che per il tipo di input SECRET e FILE, il comando deve essere adattato per corrispondere ai valori reali."
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "Vista temporale",

View File

@@ -360,7 +360,7 @@
},
"cancel": "キャンセル",
"homeDashboard": {
"title": "概要",
"title": "Dashboard",
"today": "今日",
"yesterday": "昨日",
"last28Days": "過去28日間",
@@ -369,7 +369,11 @@
"namespacesErrorExecutions": "namespaceごとの実行エラー",
"failedExecutions": "FAILED実行",
"errorLogs": "エラーログ",
"no executions": "flowを実行してみませんか"
"no executions": "flowを実行してみませんか",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"welcome": "Welcome to Kestra"
},
"executions replayed": "<code>{executionCount}</code>件の実行が再実行されました",
"executions resumed": "<code>{executionCount}</code>件の実行が再開されました",
@@ -654,6 +658,9 @@
},
"need-help": {
"title": "ヘルプが必要ですか?"
},
"tutorial": {
"title": "Tutorial"
}
},
"pluginPage": {
@@ -819,7 +826,7 @@
},
"curl": {
"command": "cURLコマンド",
"note": "SECRETおよびFILE入力タイプの場合、コマンドは実際の値に合わせて調整する必要があります。"
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "Temporalビュー",

View File

@@ -360,7 +360,7 @@
},
"cancel": "취소",
"homeDashboard": {
"title": "개요",
"title": "Dashboard",
"today": "오늘",
"yesterday": "어제",
"last28Days": "지난 28일",
@@ -369,7 +369,11 @@
"namespacesErrorExecutions": "namespace별 실행 오류",
"failedExecutions": "실패한 실행",
"errorLogs": "오류 로그",
"no executions": "flow를 실행해 보시겠습니까?"
"no executions": "flow를 실행해 보시겠습니까?",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"welcome": "Welcome to Kestra"
},
"executions replayed": "<code>{executionCount}</code> 개의 실행이 재실행되었습니다",
"executions resumed": "<code>{executionCount}</code> 개의 실행이 재개되었습니다",
@@ -654,6 +658,9 @@
},
"need-help": {
"title": "도움이 필요하십니까?"
},
"tutorial": {
"title": "Tutorial"
}
},
"pluginPage": {
@@ -819,7 +826,7 @@
},
"curl": {
"command": "cURL 명령어",
"note": "SECRET FILE input 유형의 경우, 명령어는 실제 값에 맞게 조정되어야 합니다."
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "Temporal 보기",

View File

@@ -360,7 +360,7 @@
},
"cancel": "Anuluj",
"homeDashboard": {
"title": "Przegląd",
"title": "Dashboard",
"today": "Dzisiaj",
"yesterday": "Wczoraj",
"last28Days": "Ostatnie 28 dni",
@@ -369,7 +369,11 @@
"namespacesErrorExecutions": "Błędy wykonania na namespace",
"failedExecutions": "Nieudane wykonania",
"errorLogs": "Logi błędów",
"no executions": "Gotowy, aby zobaczyć swój flow w akcji?"
"no executions": "Gotowy, aby zobaczyć swój flow w akcji?",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"welcome": "Welcome to Kestra"
},
"executions replayed": "<code>{executionCount}</code> wykonanie(a) odtworzone",
"executions resumed": "<code>{executionCount}</code> wykonanie(a) wznowione",
@@ -654,6 +658,9 @@
},
"need-help": {
"title": "Potrzebujesz pomocy?"
},
"tutorial": {
"title": "Tutorial"
}
},
"pluginPage": {
@@ -819,7 +826,7 @@
},
"curl": {
"command": "Komenda cURL",
"note": "Zauważ, że dla typu input SECRET i FILE, komenda musi być dostosowana do rzeczywistych wartości."
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "Widok czasowy",

View File

@@ -360,7 +360,7 @@
},
"cancel": "Cancelar",
"homeDashboard": {
"title": "Visão Geral",
"title": "Dashboard",
"today": "Hoje",
"yesterday": "Ontem",
"last28Days": "Últimos 28 dias",
@@ -369,7 +369,11 @@
"namespacesErrorExecutions": "Erros de execuções por namespace",
"failedExecutions": "Execuções FAILED",
"errorLogs": "Logs de erros",
"no executions": "Pronto para ver seu flow em ação?"
"no executions": "Pronto para ver seu flow em ação?",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"welcome": "Welcome to Kestra"
},
"executions replayed": "<code>{executionCount}</code> execução(ões) reproduzida(s)",
"executions resumed": "<code>{executionCount}</code> execução(ões) retomada(s)",
@@ -654,6 +658,9 @@
},
"need-help": {
"title": "Precisa de ajuda?"
},
"tutorial": {
"title": "Tutorial"
}
},
"pluginPage": {
@@ -819,7 +826,7 @@
},
"curl": {
"command": "Comando cURL",
"note": "Note que para os tipos de input SECRET e FILE, o comando deve ser ajustado para corresponder aos valores reais."
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "Visualização temporal",

View File

@@ -360,7 +360,7 @@
},
"cancel": "Отмена",
"homeDashboard": {
"title": "Обзор",
"title": "Dashboard",
"today": "Сегодня",
"yesterday": "Вчера",
"last28Days": "Последние 28 дней",
@@ -369,7 +369,11 @@
"namespacesErrorExecutions": "Ошибки выполнения по namespace",
"failedExecutions": "Неудачные выполнения",
"errorLogs": "Логи ошибок",
"no executions": "Готовы увидеть ваш flow в действии?"
"no executions": "Готовы увидеть ваш flow в действии?",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"welcome": "Welcome to Kestra"
},
"executions replayed": "<code>{executionCount}</code> выполнение(я) воспроизведено",
"executions resumed": "<code>{executionCount}</code> выполнение(я) возобновлено",
@@ -654,6 +658,9 @@
},
"need-help": {
"title": "Нужна помощь?"
},
"tutorial": {
"title": "Tutorial"
}
},
"pluginPage": {
@@ -819,7 +826,7 @@
},
"curl": {
"command": "Команда cURL",
"note": "Обратите внимание, что для типов input SECRET и FILE команда должна быть адаптирована для соответствия реальным значениям."
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "Временной вид",

View File

@@ -360,7 +360,7 @@
},
"cancel": "取消",
"homeDashboard": {
"title": "概览",
"title": "Dashboard",
"today": "今天",
"yesterday": "昨天",
"last28Days": "最近28天",
@@ -369,7 +369,11 @@
"namespacesErrorExecutions": "每个命名空间的执行错误",
"failedExecutions": "失败的执行",
"errorLogs": "错误日志",
"no executions": "准备好看到你的流程在行动了吗?"
"no executions": "准备好看到你的流程在行动了吗?",
"wel_text": "Welcome!",
"start": "Let's get your first workflow up and running.",
"guide": "Need guidance to execute your first flow?",
"welcome": "Welcome to Kestra"
},
"executions replayed": "<code>{executionCount}</code> 个执行已重放",
"executions resumed": "<code>{executionCount}</code> 个执行已恢复",
@@ -654,6 +658,9 @@
},
"need-help": {
"title": "需要帮助?"
},
"tutorial": {
"title": "Tutorial"
}
},
"pluginPage": {
@@ -819,7 +826,7 @@
},
"curl": {
"command": "cURL命令",
"note": "请注意对于SECRET和FILE输入类型命令必须适应以匹配实际值。"
"note": "Note that for SECRET and FILE input type, the command must be accommodate to match the real values."
},
"logs_view": {
"raw": "时间视图",

View File

@@ -117,7 +117,7 @@ export default (callback, store, router) => {
if (!refreshing) {
refreshing = true;
try {
await instance.post("/oauth/access_token?grant_type=refresh_token");
await instance.post("/oauth/access_token?grant_type=refresh_token", null, {headers: {"Content-Type": "application/json"}});
toRefreshQueue.forEach(({config, resolve, reject}) => {
instance.request(config).then(response => { resolve(response) }).catch(error => { reject(error) })
})

View File

@@ -587,6 +587,7 @@ public class ExecutionController {
Flow flow = flowService.getFlowIfExecutableOrThrow(tenantService.resolveTenant(), namespace, id, revision);
List<Label> parsedLabels = parseLabels(labels);
Execution current = Execution.newExecution(flow, null, parsedLabels, scheduleDate);
final AtomicReference<Runnable> disposable = new AtomicReference<>();
Mono<CompletableFuture<ExecutionResponse>> handle = flowInputOutput.readExecutionInputs(flow, current, inputs)
.handle((executionInputs, sink) -> {
Execution executionWithInputs = current.withInputs(executionInputs);
@@ -598,7 +599,6 @@ public class ExecutionController {
if (!wait) {
future.complete(ExecutionResponse.fromExecution(executionWithInputs, executionUrl(executionWithInputs)));
} else {
final AtomicReference<Runnable> disposable = new AtomicReference<>();
disposable.set(this.executionQueue.receive(either -> {
if (either.isRight()) {
log.error("Unable to deserialize the execution: {}", either.getRight().getMessage());
@@ -607,7 +607,6 @@ public class ExecutionController {
Execution item = either.getLeft();
if (item.getId().equals(executionWithInputs.getId()) && this.isStopFollow(flow, item)) {
disposable.get().run();
future.complete(ExecutionResponse.fromExecution(item, executionUrl(item)));
}
}));
@@ -617,7 +616,11 @@ public class ExecutionController {
sink.error(e);
}
});
return handle.flatMap(Mono::fromFuture);
return handle.flatMap(Mono::fromFuture).doFinally(ignored -> {
if (disposable.get() != null) {
disposable.get().run();
}
});
}
private URI executionUrl(Execution execution) {
@@ -1511,12 +1514,7 @@ public class ExecutionController {
cancel.set(receive);
}, FluxSink.OverflowStrategy.BUFFER)
.doOnCancel(() -> {
if (cancel.get() != null) {
cancel.get().run();
}
})
.doOnComplete(() -> {
.doFinally(ignored -> {
if (cancel.get() != null) {
cancel.get().run();
}