From fed16513c7d88b3adaa8b968ba09c0bc376da777 Mon Sep 17 00:00:00 2001 From: Anoop Panicker Date: Fri, 10 Aug 2018 00:07:15 -0700 Subject: [PATCH 1/4] store large payloads in external storage - part 1 --- .../config/ConductorClientConfiguration.java | 40 +++++ .../DefaultConductorClientConfiguration.java | 39 +++++ .../conductor/client/http/ClientBase.java | 67 ++++++--- .../conductor/client/http/MetadataClient.java | 15 ++ .../conductor/client/http/TaskClient.java | 57 ++++++-- .../conductor/client/http/WorkflowClient.java | 74 +++++++++- .../client/task/WorkflowTaskCoordinator.java | 14 +- .../client/task/WorkflowTaskMetrics.java | 10 +- .../conductor/client/util/PayloadStorage.java | 129 ++++++++++++++++ .../client/worker/TestPropertyFactory.java | 44 +++--- .../conductor/common/metadata/tasks/Task.java | 3 +- .../common/metadata/tasks/TaskResult.java | 25 +++- .../workflow/StartWorkflowRequest.java | 15 +- .../common/run/ExternalStorageLocation.java | 47 ++++++ .../common/utils/ExternalPayloadStorage.java | 57 ++++++++ contribs/dependencies.lock | 72 +++++++-- core/build.gradle | 4 +- core/dependencies.lock | 36 +++++ .../conductor/core/config/Configuration.java | 32 ++-- .../core/execution/DeciderService.java | 2 +- .../core/utils/DummyPayloadStorage.java | 42 ++++++ .../core/utils/S3PayloadStorage.java | 138 ++++++++++++++++++ .../conductor/service/ExecutionService.java | 25 +++- .../conductor/service/TaskService.java | 57 ++++++-- .../conductor/service/WorkflowService.java | 10 ++ es2-persistence/dependencies.lock | 54 +++++++ es5-persistence/dependencies.lock | 54 +++++++ jersey/dependencies.lock | 54 +++++++ .../server/resources/TaskResource.java | 8 + .../server/resources/WorkflowResource.java | 9 ++ redis-persistence/dependencies.lock | 54 +++++++ server/dependencies.lock | 88 +++++++++-- .../conductor/server/ConductorConfig.java | 19 +-- .../conductor/server/ConductorServer.java | 17 ++- .../conductor/server/ServerModule.java | 15 +- server/src/main/resources/server.properties | 2 +- test-harness/dependencies.lock | 40 ++++- .../tests/integration/End2EndTests.java | 5 +- .../utils/MockExternalPayloadStorage.java | 38 +++++ .../conductor/tests/utils/TestModule.java | 17 +-- 40 files changed, 1355 insertions(+), 173 deletions(-) create mode 100644 client/src/main/java/com/netflix/conductor/client/config/ConductorClientConfiguration.java create mode 100644 client/src/main/java/com/netflix/conductor/client/config/DefaultConductorClientConfiguration.java create mode 100644 client/src/main/java/com/netflix/conductor/client/util/PayloadStorage.java create mode 100644 common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java create mode 100644 common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java create mode 100644 core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java create mode 100644 core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java create mode 100644 test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java diff --git a/client/src/main/java/com/netflix/conductor/client/config/ConductorClientConfiguration.java b/client/src/main/java/com/netflix/conductor/client/config/ConductorClientConfiguration.java new file mode 100644 index 0000000000..3acf0ed6be --- /dev/null +++ b/client/src/main/java/com/netflix/conductor/client/config/ConductorClientConfiguration.java @@ -0,0 +1,40 @@ +/* + * Copyright 2018 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.conductor.client.config; + +public interface ConductorClientConfiguration { + + /** + * @return the workflow input payload threshold in KB, + * beyond which the payload will be processed based on {@link ConductorClientConfiguration#isExternalPayloadStorageEnabled()}. + */ + int getWorkflowInputPayloadThresholdKB(); + + /** + * @return the task output payload threshold in KB, + * beyond which the payload will be processed based on {@link ConductorClientConfiguration#isExternalPayloadStorageEnabled()}. + */ + int getTaskOutputPayloadThresholdKB(); + + /** + * @return the flag which controls the use of external storage for storing workflow/task + * input and output JSON payloads with size greater than threshold. + * If it is set to true, the payload is stored in external location. + * If it is set to false, the payload is rejected and the task/workflow execution fails. + */ + boolean isExternalPayloadStorageEnabled(); +} diff --git a/client/src/main/java/com/netflix/conductor/client/config/DefaultConductorClientConfiguration.java b/client/src/main/java/com/netflix/conductor/client/config/DefaultConductorClientConfiguration.java new file mode 100644 index 0000000000..65873d48a4 --- /dev/null +++ b/client/src/main/java/com/netflix/conductor/client/config/DefaultConductorClientConfiguration.java @@ -0,0 +1,39 @@ +/* + * Copyright 2018 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.conductor.client.config; + +/** + * A default implementation of {@link ConductorClientConfiguration} + * where external payload storage is disabled. + */ +public class DefaultConductorClientConfiguration implements ConductorClientConfiguration { + + @Override + public int getWorkflowInputPayloadThresholdKB() { + return 5120; + } + + @Override + public int getTaskOutputPayloadThresholdKB() { + return 3072; + } + + @Override + public boolean isExternalPayloadStorageEnabled() { + return false; + } +} diff --git a/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java b/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java index a72eb678ad..9cf8969f06 100644 --- a/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java +++ b/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java @@ -1,4 +1,4 @@ -/** +/* * Copyright 2016 Netflix, Inc. *

* Licensed under the Apache License, Version 2.0 (the "License"); @@ -13,17 +13,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -/** - * - */ + package com.netflix.conductor.client.http; import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; +import com.netflix.conductor.client.config.ConductorClientConfiguration; +import com.netflix.conductor.client.config.DefaultConductorClientConfiguration; import com.netflix.conductor.client.exceptions.ConductorClientException; import com.netflix.conductor.client.exceptions.ErrorResponse; +import com.netflix.conductor.client.util.PayloadStorage; +import com.netflix.conductor.common.metadata.tasks.TaskResult; +import com.netflix.conductor.common.run.ExternalStorageLocation; import com.sun.jersey.api.client.Client; import com.sun.jersey.api.client.ClientHandler; import com.sun.jersey.api.client.ClientHandlerException; @@ -38,6 +41,7 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.UriBuilder; +import java.io.ByteArrayInputStream; import java.io.IOException; import java.net.URI; import java.util.Collection; @@ -56,15 +60,23 @@ public abstract class ClientBase { protected ObjectMapper objectMapper; + protected PayloadStorage payloadStorage; + + protected ConductorClientConfiguration conductorClientConfiguration; + protected ClientBase() { this(new DefaultClientConfig(), null); } - protected ClientBase(ClientConfig clientConfig) { - this(clientConfig, null); + protected ClientBase(ClientConfig config) { + this(config, null); + } + + protected ClientBase(ClientConfig config, ClientHandler handler) { + this(config, new DefaultConductorClientConfiguration(), handler); } - protected ClientBase(ClientConfig clientConfig, ClientHandler handler) { + protected ClientBase(ClientConfig config, ConductorClientConfiguration clientConfiguration, ClientHandler handler) { objectMapper = new ObjectMapper(); objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); objectMapper.configure(DeserializationFeature.FAIL_ON_IGNORED_PROPERTIES, false); @@ -72,14 +84,18 @@ protected ClientBase(ClientConfig clientConfig, ClientHandler handler) { objectMapper.setSerializationInclusion(Include.NON_NULL); objectMapper.setSerializationInclusion(Include.NON_EMPTY); + payloadStorage = new PayloadStorage(); + JacksonJsonProvider provider = new JacksonJsonProvider(objectMapper); - clientConfig.getSingletons().add(provider); + config.getSingletons().add(provider); if (handler == null) { - this.client = Client.create(clientConfig); + this.client = Client.create(config); } else { - this.client = new Client(handler, clientConfig); + this.client = new Client(handler, config); } + + conductorClientConfiguration = clientConfiguration; } public void setRootURI(String root) { @@ -173,18 +189,19 @@ protected T getForEntity(String url, Object[] queryParams, GenericType re return getForEntity(url, queryParams, response -> response.getEntity(responseType), uriVariables); } - private T getForEntity(String url, Object[] queryParams, Function entityPvoider, Object... uriVariables) { + private T getForEntity(String url, Object[] queryParams, Function entityProvider, Object... uriVariables) { URI uri = null; - ClientResponse clientResponse = null; + ClientResponse clientResponse; try { uri = getURIBuilder(root + url, queryParams).build(uriVariables); clientResponse = client.resource(uri) .accept(MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN) .get(ClientResponse.class); if (clientResponse.getStatus() < 300) { - return entityPvoider.apply(clientResponse); + return entityProvider.apply(clientResponse); + } else { - throw new UniformInterfaceException(clientResponse); // let handleUniformInterfaceException to handle unexpected response consistently + throw new UniformInterfaceException(clientResponse); } } catch (UniformInterfaceException e) { handleUniformInterfaceException(e, uri); @@ -194,11 +211,26 @@ private T getForEntity(String url, Object[] queryParams, Function (3 * 1024 * 1024)) { //There is hard coded since there is no easy way to pass a config in here - taskResult.setReasonForIncompletion(String.format("The TaskResult payload: %d is greater than the permissible 3MB", taskResultSize)); - taskResult.setStatus(TaskResult.Status.FAILED_WITH_TERMINAL_ERROR); - taskResult.setOutputData(null); + if (taskResultSize > conductorClientConfiguration.getTaskOutputPayloadThresholdKB() * 1024) { + if (conductorClientConfiguration.isExternalPayloadStorageEnabled()) { + String externalStoragePath = useExternalPayloadStorage("tasks", taskOutputBytes, taskResultSize); + taskResult.setExternalPayloadStoragePath(externalStoragePath); + taskResult.setOutputData(null); + } else { + taskResult.setReasonForIncompletion(String.format("The TaskResult payload size: %d is greater than the permissible 3MB", taskResultSize)); + taskResult.setStatus(TaskResult.Status.FAILED_WITH_TERMINAL_ERROR); + taskResult.setOutputData(null); + } } } catch (IOException e) { - logger.error("Unable to update task with task result: {}", taskResult, e); - throw new RuntimeException("Unable to update task", e); + String errorMsg = String.format("Unable to update task: %s with task result", taskResult.getTaskId()); + logger.error(errorMsg, e); + throw new ConductorClientException(errorMsg, e); } postForEntityWithRequestOnly("tasks", taskResult); } @@ -424,4 +446,9 @@ public void registerTaskDefs(List taskDefs) { Preconditions.checkNotNull(taskDefs, "Task defs cannot be null"); postForEntity("metadata/taskdefs", taskDefs); } + + public String getExternalPayloadUri(String taskId) { + Preconditions.checkArgument(StringUtils.isNotBlank(taskId), "task id cannot be blank"); + return getForEntity("tasks/externalpayloaduri/{taskId}", null, String.class, taskId); + } } diff --git a/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java b/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java index f91a04ad41..3abb995b6d 100644 --- a/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java +++ b/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java @@ -16,6 +16,9 @@ package com.netflix.conductor.client.http; import com.google.common.base.Preconditions; +import com.netflix.conductor.client.config.ConductorClientConfiguration; +import com.netflix.conductor.client.exceptions.ConductorClientException; +import com.netflix.conductor.client.task.WorkflowTaskMetrics; import com.netflix.conductor.common.metadata.workflow.RerunWorkflowRequest; import com.netflix.conductor.common.metadata.workflow.StartWorkflowRequest; import com.netflix.conductor.common.metadata.workflow.WorkflowDef; @@ -27,7 +30,11 @@ import com.sun.jersey.api.client.config.ClientConfig; import com.sun.jersey.api.client.filter.ClientFilter; import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.util.List; import java.util.Map; @@ -43,6 +50,8 @@ public class WorkflowClient extends ClientBase { private static GenericType> searchResultWorkflowSummary = new GenericType>() { }; + private static final Logger logger = LoggerFactory.getLogger(WorkflowClient.class); + /** * Creates a default task client */ @@ -66,8 +75,8 @@ public WorkflowClient(ClientConfig config, ClientHandler handler) { } /** - * @param config config REST Client configuration - * @param handler handler Jersey client handler. Useful when plugging in various http client interaction modules (e.g. ribbon) + * @param config REST Client configuration + * @param handler Jersey client handler. Useful when plugging in various http client interaction modules (e.g. ribbon) * @param filters Chain of client side filters to be applied per request */ public WorkflowClient(ClientConfig config, ClientHandler handler, ClientFilter... filters) { @@ -77,6 +86,19 @@ public WorkflowClient(ClientConfig config, ClientHandler handler, ClientFilter.. } } + /** + * @param config REST Client configuration + * @param clientConfiguration Specific properties configured for the client, see {@link ConductorClientConfiguration} + * @param handler Jersey client handler. Useful when plugging in various http client interaction modules (e.g. ribbon) + * @param filters Chain of client side filters to be applied per request + */ + public WorkflowClient(ClientConfig config, ConductorClientConfiguration clientConfiguration, ClientHandler handler, ClientFilter... filters) { + super(config, clientConfiguration, handler); + for (ClientFilter filter : filters) { + super.client.addFilter(filter); + } + } + //Metadata Operations @@ -120,12 +142,19 @@ public WorkflowDef getWorkflowDef(String name, Integer version) { * @param correlationId the correlation id * @param input the input to set in the workflow * @return the id of the workflow instance that can be used for tracking + * + * @deprecated This API is deprecated and will be removed in the next version + * use {@link #startWorkflow(StartWorkflowRequest)} instead */ + @Deprecated public String startWorkflow(String name, Integer version, String correlationId, Map input) { Preconditions.checkArgument(StringUtils.isNotBlank(name), "name cannot be blank"); - - Object[] params = new Object[]{"version", version, "correlationId", correlationId}; - return postForEntity("workflow/{name}", input, params, String.class, name); + StartWorkflowRequest startWorkflowRequest = new StartWorkflowRequest(); + startWorkflowRequest.setName(name); + startWorkflowRequest.setVersion(version); + startWorkflowRequest.setCorrelationId(correlationId); + startWorkflowRequest.setInput(input); + return startWorkflow(startWorkflowRequest); } /** @@ -136,6 +165,30 @@ public String startWorkflow(String name, Integer version, String correlationId, */ public String startWorkflow(StartWorkflowRequest startWorkflowRequest) { Preconditions.checkNotNull(startWorkflowRequest, "StartWorkflowRequest cannot be null"); + Preconditions.checkArgument(StringUtils.isNotBlank(startWorkflowRequest.getName()), "Workflow name cannot be null or empty"); + Preconditions.checkArgument(StringUtils.isBlank(startWorkflowRequest.getExternalStoragePath()), "External Storage Path must be blank"); + + String version = startWorkflowRequest.getVersion() != null ? startWorkflowRequest.getVersion().toString() : "latest"; + try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream()) { + objectMapper.writeValue(byteArrayOutputStream, startWorkflowRequest.getInput()); + byte[] workflowInputBytes = byteArrayOutputStream.toByteArray(); + long workflowInputSize = workflowInputBytes.length; + WorkflowTaskMetrics.recordWorkflowInputPayloadSize(startWorkflowRequest.getName(), version, workflowInputSize); + if (workflowInputSize > conductorClientConfiguration.getWorkflowInputPayloadThresholdKB() * 1024) { + if (conductorClientConfiguration.isExternalPayloadStorageEnabled()) { + String externalStoragePath = useExternalPayloadStorage("workflow", workflowInputBytes, workflowInputSize); + startWorkflowRequest.setExternalStoragePath(externalStoragePath); + startWorkflowRequest.setInput(null); + } else { + String errorMsg = String.format("Input payload larger than the allowed threshold of: %d KB", conductorClientConfiguration.getWorkflowInputPayloadThresholdKB()); + throw new ConductorClientException(errorMsg); + } + } + } catch (IOException e) { + String errorMsg = String.format("Unable to start workflow:%s, version:%s", startWorkflowRequest.getName(), version); + logger.error(errorMsg, e); + throw new ConductorClientException(errorMsg, e); + } return postForEntity("workflow", startWorkflowRequest, null, String.class, startWorkflowRequest.getName()); } @@ -201,7 +254,8 @@ public void deleteWorkflow(String workflowId, boolean archiveWorkflow) { */ public List getRunningWorkflow(String workflowName, Integer version) { Preconditions.checkArgument(StringUtils.isNotBlank(workflowName), "Workflow name cannot be blank"); - return getForEntity("workflow/running/{name}", new Object[]{"version", version}, new GenericType>() {}, workflowName); + return getForEntity("workflow/running/{name}", new Object[]{"version", version}, new GenericType>() { + }, workflowName); } /** @@ -219,7 +273,8 @@ public List getWorkflowsByTimePeriod(String workflowName, int version, L Preconditions.checkNotNull(endTime, "End time cannot be null"); Object[] params = new Object[]{"version", version, "startTime", startTime, "endTime", endTime}; - return getForEntity("workflow/running/{name}", params, new GenericType>() {}, workflowName); + return getForEntity("workflow/running/{name}", params, new GenericType>() { + }, workflowName); } /** @@ -344,4 +399,9 @@ public SearchResult search(Integer start, Integer size, String Object[] params = new Object[]{"start", start, "size", size, "sort", sort, "freeText", freeText, "query", query}; return getForEntity("workflow/search", params, searchResultWorkflowSummary); } + + public String getExternalPayloadUri(String workflowId) { + Preconditions.checkArgument(StringUtils.isNotBlank(workflowId), "workflow id cannot be blank"); + return getForEntity("workflow/externalpayloaduri/{workflowId}", null, String.class, workflowId); + } } diff --git a/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java b/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java index d3ed42467e..019bde9d3d 100644 --- a/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java +++ b/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java @@ -273,15 +273,15 @@ public synchronized void init() { }); } - public void shutdown() { - this.scheduledExecutorService.shutdown(); - this.executorService.shutdown(); + public void shutdown() { + this.scheduledExecutorService.shutdown(); + this.executorService.shutdown(); - shutdownExecutorService(this.scheduledExecutorService, SHUTDOWN_WAIT_TIME_IN_SEC); - shutdownExecutorService(this.executorService, SHUTDOWN_WAIT_TIME_IN_SEC); - } + shutdownExecutorService(this.scheduledExecutorService, SHUTDOWN_WAIT_TIME_IN_SEC); + shutdownExecutorService(this.executorService, SHUTDOWN_WAIT_TIME_IN_SEC); + } - private void shutdownExecutorService(ExecutorService executorService, long timeout) { + private void shutdownExecutorService(ExecutorService executorService, long timeout) { try { if (executorService.awaitTermination(timeout, TimeUnit.SECONDS)) { logger.debug("tasks completed, shutting down"); diff --git a/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskMetrics.java b/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskMetrics.java index d7bed563b7..90ce8d7195 100644 --- a/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskMetrics.java +++ b/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskMetrics.java @@ -42,6 +42,8 @@ public class WorkflowTaskMetrics { private static final String TASK_TYPE = "taskType"; + private static final String WORFLOW_TYPE = "workflowType"; + private static final String WORKFLOW_VERSION = "version"; private static final String EXCEPTION = "exception"; private static final String TASK_EXECUTION_QUEUE_FULL = "task_execution_queue_full"; @@ -54,7 +56,9 @@ public class WorkflowTaskMetrics { private static final String TASK_POLL_COUNTER = "task_poll_counter"; private static final String TASK_EXECUTE_TIME = "task_execute_time"; private static final String TASK_POLL_TIME = "task_poll_time"; - public static final String TASK_RESULT_SIZE = "task_result_size"; + private static final String TASK_RESULT_SIZE = "task_result_size"; + private static final String WORKFLOW_INPUT_SIZE = "workflow_input_size"; + private static Registry registry = Spectator.globalRegistry(); @@ -158,5 +162,7 @@ public static void incrementTaskPollCount(String taskType, int taskCount) { getCounter(TASK_POLL_COUNTER, TASK_TYPE, taskType).increment(taskCount); } - + public static void recordWorkflowInputPayloadSize(String workflowType, String version, long payloadSize) { + getGauge(WORKFLOW_INPUT_SIZE, WORFLOW_TYPE, workflowType, WORKFLOW_VERSION, version).getAndSet(payloadSize); + } } diff --git a/client/src/main/java/com/netflix/conductor/client/util/PayloadStorage.java b/client/src/main/java/com/netflix/conductor/client/util/PayloadStorage.java new file mode 100644 index 0000000000..bd729a1532 --- /dev/null +++ b/client/src/main/java/com/netflix/conductor/client/util/PayloadStorage.java @@ -0,0 +1,129 @@ +/* + * Copyright 2018 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.conductor.client.util; + +import com.netflix.conductor.client.exceptions.ConductorClientException; +import com.netflix.conductor.common.run.ExternalStorageLocation; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.HttpURLConnection; +import java.net.MalformedURLException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; + +/** + * An implementation of {@link ExternalPayloadStorage} for storing large JSON payload data. + */ +public class PayloadStorage implements ExternalPayloadStorage { + private static final Logger logger = LoggerFactory.getLogger(PayloadStorage.class); + + /** + * This method is not intended to be used in the client. + * The client makes a request to the server to get the {@link ExternalStorageLocation} + */ + @Override + public ExternalStorageLocation getExternalUri(Operation operation, PayloadType payloadType) { + throw new UnsupportedOperationException(); + } + + /** + * Uploads the payload to the uri specified. + * + * @param path the location to which the object is to be uploaded + * @param payload an {@link InputStream} containing the json payload which is to be uploaded + * @param payloadSize the size of the json payload in bytes + * @throws ConductorClientException if the upload fails due to an invalid path or an error from external storage + */ + @Override + public void upload(String path, InputStream payload, long payloadSize) { + HttpURLConnection connection = null; + try { + URL url = new URI(path).toURL(); + + connection = (HttpURLConnection) url.openConnection(); + connection.setDoOutput(true); + connection.setRequestMethod("PUT"); + + try (BufferedOutputStream bufferedOutputStream = new BufferedOutputStream(connection.getOutputStream())) { + int length; + while ((length = payload.read()) != -1) { + bufferedOutputStream.write(length); + } + + // Check the HTTP response code + int responseCode = connection.getResponseCode(); + logger.debug("Upload completed with HTTP response code: {}", responseCode); + } + } catch (URISyntaxException | MalformedURLException e) { + String errorMsg = String.format("Invalid path specified: %s", path); + logger.error(errorMsg, e); + throw new ConductorClientException(errorMsg, e); + } catch (IOException e) { + String errorMsg = String.format("Error uploading to path: %s", path); + logger.error(errorMsg, e); + throw new ConductorClientException(errorMsg, e); + } finally { + if (connection != null) { + connection.disconnect(); + } + } + } + + /** + * Downloads the payload from the given uri. + * + * @param path the location from where the object is to be downloaded + * @return an inputstream of the payload in the external storage + * @throws ConductorClientException if the download fails due to an invalid path or an error from external storage + */ + @Override + public InputStream download(String path) { + HttpURLConnection connection = null; + try { + URL url = new URI(path).toURL(); + connection = (HttpURLConnection) url.openConnection(); + connection.setDoOutput(false); + + // Check the HTTP response code + int responseCode = connection.getResponseCode(); + if (responseCode == HttpURLConnection.HTTP_OK) { + logger.debug("Download completed with HTTP response code: {}", connection.getResponseCode()); + return connection.getInputStream(); + } + logger.info("No file to download. Response code: {}", responseCode); + return null; + } catch (URISyntaxException | MalformedURLException e) { + String errorMsg = String.format("Invalid path specified: %s", path); + logger.error(errorMsg, e); + throw new ConductorClientException(errorMsg, e); + } catch (IOException e) { + String errorMsg = String.format("Error downloading from path: %s", path); + logger.error(errorMsg, e); + throw new ConductorClientException(errorMsg, e); + } finally { + if (connection != null) { + connection.disconnect(); + } + } + } +} diff --git a/client/src/test/java/com/netflix/conductor/client/worker/TestPropertyFactory.java b/client/src/test/java/com/netflix/conductor/client/worker/TestPropertyFactory.java index c8b4b94604..b037778dc4 100644 --- a/client/src/test/java/com/netflix/conductor/client/worker/TestPropertyFactory.java +++ b/client/src/test/java/com/netflix/conductor/client/worker/TestPropertyFactory.java @@ -13,63 +13,55 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -/** - * - */ + package com.netflix.conductor.client.worker; +import com.netflix.conductor.common.metadata.tasks.TaskResult; +import org.junit.Test; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import org.junit.Test; - -import com.netflix.conductor.common.metadata.tasks.Task; -import com.netflix.conductor.common.metadata.tasks.TaskResult; - /** * @author Viren * */ public class TestPropertyFactory { - + @Test public void testIdentity(){ - Worker worker = Worker.create("Test2", (Task task)->{ - return new TaskResult(task); - }); + Worker worker = Worker.create("Test2", TaskResult::new); assertNotNull(worker.getIdentity()); boolean paused = worker.paused(); assertFalse("Paused? " + paused, paused); } - + @Test public void test() { - - int val = PropertyFactory.getInteger("workerB", "pollingInterval", 100).intValue(); + + int val = PropertyFactory.getInteger("workerB", "pollingInterval", 100); assertEquals("got: " + val, 2, val); assertEquals(100, PropertyFactory.getInteger("workerB", "propWithoutValue", 100).intValue()); - + assertFalse(PropertyFactory.getBoolean("workerB", "paused", true)); //Global value set to 'false' assertTrue(PropertyFactory.getBoolean("workerA", "paused", false)); //WorkerA value set to 'true' - - + + assertEquals(42, PropertyFactory.getInteger("workerA", "batchSize", 42).intValue()); //No global value set, so will return the default value supplied assertEquals(84, PropertyFactory.getInteger("workerB", "batchSize", 42).intValue()); //WorkerB's value set to 84 - assertEquals("domainA", PropertyFactory.getString("workerA", "domain", null)); - assertEquals("domainB", PropertyFactory.getString("workerB", "domain", null)); - assertEquals(null, PropertyFactory.getString("workerC", "domain", null)); // Non Existent + assertEquals("domainA", PropertyFactory.getString("workerA", "domain", null)); + assertEquals("domainB", PropertyFactory.getString("workerB", "domain", null)); + assertNull(PropertyFactory.getString("workerC", "domain", null)); // Non Existent } - + @Test public void testProperty() { - Worker worker = Worker.create("Test", (Task task)->{ - return new TaskResult(task); - }); + Worker worker = Worker.create("Test", TaskResult::new); boolean paused = worker.paused(); assertTrue("Paused? " + paused, paused); } - } diff --git a/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java b/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java index 20b64bf381..b4c9fe3187 100644 --- a/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java +++ b/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java @@ -438,7 +438,8 @@ public String getWorkflowType() { /** - * @param workflowType workflow type + * @param workflowType the name of the workflow + * @return the task object with the workflow type set */ public Task setWorkflowType(String workflowType) { this.workflowType = workflowType; diff --git a/common/src/main/java/com/netflix/conductor/common/metadata/tasks/TaskResult.java b/common/src/main/java/com/netflix/conductor/common/metadata/tasks/TaskResult.java index 1dfb6ec1fb..2fb5e7bb60 100644 --- a/common/src/main/java/com/netflix/conductor/common/metadata/tasks/TaskResult.java +++ b/common/src/main/java/com/netflix/conductor/common/metadata/tasks/TaskResult.java @@ -1,4 +1,4 @@ -/** +/* * Copyright 2016 Netflix, Inc. *

* Licensed under the Apache License, Version 2.0 (the "License"); @@ -13,9 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -/** - * - */ + package com.netflix.conductor.common.metadata.tasks; import java.util.HashMap; @@ -50,6 +48,8 @@ public enum Status { private List logs = new CopyOnWriteArrayList<>(); + private String externalPayloadStoragePath; + public TaskResult(Task task) { this.workflowInstanceId = task.getWorkflowInstanceId(); this.taskId = task.getTaskId(); @@ -191,6 +191,22 @@ public TaskResult log(String log) { return this; } + /** + * + * @return the path where the task output is stored in external storage + */ + public String getExternalPayloadStoragePath() { + return externalPayloadStoragePath; + } + + /** + * + * @param externalPayloadStoragePath path in the external storage where the task output is stored + */ + public void setExternalPayloadStoragePath(String externalPayloadStoragePath) { + this.externalPayloadStoragePath = externalPayloadStoragePath; + } + @Override public String toString() { return "TaskResult{" + @@ -202,6 +218,7 @@ public String toString() { ", status=" + status + ", outputData=" + outputData + ", logs=" + logs + + ", externalPayloadStoragePath='" + externalPayloadStoragePath + '\'' + '}'; } diff --git a/common/src/main/java/com/netflix/conductor/common/metadata/workflow/StartWorkflowRequest.java b/common/src/main/java/com/netflix/conductor/common/metadata/workflow/StartWorkflowRequest.java index 595094b23d..9db60e85cd 100644 --- a/common/src/main/java/com/netflix/conductor/common/metadata/workflow/StartWorkflowRequest.java +++ b/common/src/main/java/com/netflix/conductor/common/metadata/workflow/StartWorkflowRequest.java @@ -7,6 +7,7 @@ public class StartWorkflowRequest { private String name; private Integer version; private String correlationId; + private String externalStoragePath; private Map input = new HashMap<>(); private Map taskToDomain = new HashMap<>(); @@ -19,7 +20,8 @@ public void setName(String name) { public StartWorkflowRequest withName(String name) { this.name = name; return this; - } + } + public Integer getVersion() { return version; } @@ -30,6 +32,7 @@ public StartWorkflowRequest withVersion(Integer version) { this.version = version; return this; } + public String getCorrelationId() { return correlationId; } @@ -40,6 +43,15 @@ public StartWorkflowRequest withCorrelationId(String correlationId) { this.correlationId = correlationId; return this; } + + public String getExternalStoragePath() { + return externalStoragePath; + } + + public void setExternalStoragePath(String externalStoragePath) { + this.externalStoragePath = externalStoragePath; + } + public Map getInput() { return input; } @@ -50,6 +62,7 @@ public StartWorkflowRequest withInput(Map input) { this.input = input; return this; } + public Map getTaskToDomain() { return taskToDomain; } diff --git a/common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java b/common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java new file mode 100644 index 0000000000..85bdc4b4ce --- /dev/null +++ b/common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java @@ -0,0 +1,47 @@ +/* + * Copyright 2016 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.conductor.common.run; + +public class ExternalStorageLocation { + + public String uri; + public String path; + + public String getUri() { + return uri; + } + + public void setUri(String uri) { + this.uri = uri; + } + + public String getPath() { + return path; + } + + public void setPath(String path) { + this.path = path; + } + + @Override + public String toString() { + return "ExternalStorageLocation{" + + "uri='" + uri + '\'' + + ", path='" + path + '\'' + + '}'; + } +} diff --git a/common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java b/common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java new file mode 100644 index 0000000000..b3a867023b --- /dev/null +++ b/common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java @@ -0,0 +1,57 @@ +/* + * Copyright 2018 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.conductor.common.utils; + +import com.netflix.conductor.common.run.ExternalStorageLocation; + +import java.io.InputStream; + +/** + * Interface used to externalize the storage of large JSON payloads in workflow and task input/output + */ +public interface ExternalPayloadStorage { + + enum Operation {READ, WRITE} + + enum PayloadType {WORKFLOW_INPUT, WORKFLOW_OUTPUT, TASK_INPUT, TASK_OUTPUT} + + /** + * Obtain a uri used to store/access a json payload in external storage. + * + * @param operation the type of {@link Operation} to be performed with the uri + * @param payloadType the {@link PayloadType} that is being accessed at the uri + * @return a {@link ExternalStorageLocation} object which contains the uri and the path for the json payload + */ + ExternalStorageLocation getExternalUri(Operation operation, PayloadType payloadType); + + /** + * Upload a json payload to the specified external storage location. + * + * @param path the location to which the object is to be uploaded + * @param payload an {@link InputStream} containing the json payload which is to be uploaded + * @param payloadSize the size of the json payload in bytes + */ + void upload(String path, InputStream payload, long payloadSize); + + /** + * Download the json payload from the specified external storage location. + * + * @param path the location from where the object is to be downloaded + * @return an {@link InputStream} of the json payload at the specified location + */ + InputStream download(String path); +} diff --git a/contribs/dependencies.lock b/contribs/dependencies.lock index 35f95f4197..8683de789b 100644 --- a/contribs/dependencies.lock +++ b/contribs/dependencies.lock @@ -1,7 +1,13 @@ { "compile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { - "locked": "1.11.380", + "locked": "1.11.389", "requested": "latest.release" }, "com.fasterxml.jackson.core:jackson-core": { @@ -112,8 +118,14 @@ } }, "compileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { - "locked": "1.11.380", + "locked": "1.11.389", "requested": "latest.release" }, "com.fasterxml.jackson.core:jackson-core": { @@ -224,8 +236,14 @@ } }, "default": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { - "locked": "1.11.380", + "locked": "1.11.389", "requested": "latest.release" }, "com.fasterxml.jackson.core:jackson-core": { @@ -346,8 +364,14 @@ } }, "runtime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { - "locked": "1.11.380", + "locked": "1.11.389", "requested": "latest.release" }, "com.fasterxml.jackson.core:jackson-core": { @@ -458,8 +482,14 @@ } }, "runtimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { - "locked": "1.11.380", + "locked": "1.11.389", "requested": "latest.release" }, "com.fasterxml.jackson.core:jackson-core": { @@ -570,8 +600,14 @@ } }, "testCompile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { - "locked": "1.11.380", + "locked": "1.11.389", "requested": "latest.release" }, "com.fasterxml.jackson.core:jackson-core": { @@ -698,8 +734,14 @@ } }, "testCompileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { - "locked": "1.11.380", + "locked": "1.11.389", "requested": "latest.release" }, "com.fasterxml.jackson.core:jackson-core": { @@ -826,8 +868,14 @@ } }, "testRuntime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { - "locked": "1.11.380", + "locked": "1.11.389", "requested": "latest.release" }, "com.fasterxml.jackson.core:jackson-core": { @@ -954,8 +1002,14 @@ } }, "testRuntimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { - "locked": "1.11.380", + "locked": "1.11.389", "requested": "latest.release" }, "com.fasterxml.jackson.core:jackson-core": { diff --git a/core/build.gradle b/core/build.gradle index 07cdcd4eca..17a764f97a 100644 --- a/core/build.gradle +++ b/core/build.gradle @@ -8,13 +8,13 @@ dependencies { compile "com.netflix.servo:servo-core:${revServo}" compile "com.netflix.spectator:spectator-api:${revSpectator}" + compile "com.fasterxml.jackson.core:jackson-databind:${revJacksonDatabind}" compile "com.fasterxml.jackson.core:jackson-core:${revJacksonCore}" compile "com.jayway.jsonpath:json-path:${revJsonPath}" - compile "org.apache.commons:commons-lang3:${revCommonsLang3}" - compile "com.spotify:completable-futures:${revSpotifyCompletableFutures}" + compile "com.amazonaws:aws-java-sdk-s3:${revAwsSdk}" testCompile "org.slf4j:slf4j-log4j12:${revSlf4jlog4j}" } diff --git a/core/dependencies.lock b/core/dependencies.lock index 355f6fb4d0..b4a2b8ae79 100644 --- a/core/dependencies.lock +++ b/core/dependencies.lock @@ -1,5 +1,9 @@ { "compile": { + "com.amazonaws:aws-java-sdk-s3": { + "locked": "1.11.86", + "requested": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "locked": "2.7.5", "requested": "2.7.5" @@ -57,6 +61,10 @@ } }, "compileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "locked": "1.11.86", + "requested": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "locked": "2.7.5", "requested": "2.7.5" @@ -114,6 +122,10 @@ } }, "default": { + "com.amazonaws:aws-java-sdk-s3": { + "locked": "1.11.86", + "requested": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "locked": "2.7.5", "requested": "2.7.5" @@ -171,6 +183,10 @@ } }, "runtime": { + "com.amazonaws:aws-java-sdk-s3": { + "locked": "1.11.86", + "requested": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "locked": "2.7.5", "requested": "2.7.5" @@ -228,6 +244,10 @@ } }, "runtimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "locked": "1.11.86", + "requested": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "locked": "2.7.5", "requested": "2.7.5" @@ -285,6 +305,10 @@ } }, "testCompile": { + "com.amazonaws:aws-java-sdk-s3": { + "locked": "1.11.86", + "requested": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "locked": "2.7.5", "requested": "2.7.5" @@ -354,6 +378,10 @@ } }, "testCompileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "locked": "1.11.86", + "requested": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "locked": "2.7.5", "requested": "2.7.5" @@ -423,6 +451,10 @@ } }, "testRuntime": { + "com.amazonaws:aws-java-sdk-s3": { + "locked": "1.11.86", + "requested": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "locked": "2.7.5", "requested": "2.7.5" @@ -492,6 +524,10 @@ } }, "testRuntimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "locked": "1.11.86", + "requested": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "locked": "2.7.5", "requested": "2.7.5" diff --git a/core/src/main/java/com/netflix/conductor/core/config/Configuration.java b/core/src/main/java/com/netflix/conductor/core/config/Configuration.java index bb1a188fa8..ddf4d3c9d7 100644 --- a/core/src/main/java/com/netflix/conductor/core/config/Configuration.java +++ b/core/src/main/java/com/netflix/conductor/core/config/Configuration.java @@ -18,11 +18,11 @@ */ package com.netflix.conductor.core.config; +import com.google.inject.AbstractModule; + import java.util.List; import java.util.Map; -import com.google.inject.AbstractModule; - /** * @author Viren * @@ -33,13 +33,13 @@ public interface Configuration { * * @return time frequency in seconds, at which the workflow sweeper should run to evaluate running workflows. */ - public int getSweepFrequency(); + int getSweepFrequency(); /** * * @return when set to true, the sweep is disabled */ - public boolean disableSweep(); + boolean disableSweep(); /** @@ -47,43 +47,43 @@ public interface Configuration { * @return when set to true, the background task workers executing async system tasks (eg HTTP) are disabled * */ - public boolean disableAsyncWorkers(); + boolean disableAsyncWorkers(); /** * * @return ID of the server. Can be host name, IP address or any other meaningful identifier. Used for logging */ - public String getServerId(); + String getServerId(); /** * * @return Current environment. e.g. test, prod */ - public String getEnvironment(); + String getEnvironment(); /** * * @return name of the stack under which the app is running. e.g. devint, testintg, staging, prod etc. */ - public String getStack(); + String getStack(); /** * * @return APP ID. Used for logging */ - public String getAppId(); + String getAppId(); /** * * @return Data center region. if hosting on Amazon the value is something like us-east-1, us-west-2 etc. */ - public String getRegion(); + String getRegion(); /** * * @return Availability zone / rack. for AWS deployments, the value is something like us-east-1a, etc. */ - public String getAvailabilityZone(); + String getAvailabilityZone(); /** * @@ -91,7 +91,7 @@ public interface Configuration { * @param defaultValue Default value when not specified * @return User defined integer property. */ - public int getIntProperty(String name, int defaultValue); + int getIntProperty(String name, int defaultValue); /** @@ -100,7 +100,7 @@ public interface Configuration { * @param defaultValue Default value when not specified * @return User defined Long property. */ - public long getLongProperty(String name, long defaultValue); + long getLongProperty(String name, long defaultValue); /** * @@ -108,14 +108,14 @@ public interface Configuration { * @param defaultValue Default value when not specified * @return User defined string property. */ - public String getProperty(String name, String defaultValue); + String getProperty(String name, String defaultValue); /** * * @return Returns all the configurations in a map. */ - public Map getAll(); + Map getAll(); /** * @@ -123,7 +123,7 @@ public interface Configuration { * Use this to inject additional modules that should be loaded as part of the Conductor server initialization * If you are creating custom tasks (com.netflix.conductor.core.execution.tasks.WorkflowSystemTask) then initialize them as part of the custom modules. */ - public default List getAdditionalModules() { + default List getAdditionalModules() { return null; } diff --git a/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java b/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java index dd73264fcc..f1e9cf0a59 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java @@ -151,7 +151,7 @@ private DeciderOutcome decide(final WorkflowDef workflowDef, final Workflow work TaskDef taskDefinition = metadataDAO.getTaskDef(pendingTask.getTaskDefName()); if (taskDefinition != null) { checkForTimeout(taskDefinition, pendingTask); - // If the task has not been updated for "responseTimeout" then mark task as TIMED_OUT + // If the task has not been updated for "responseTimeoutSeconds" then mark task as TIMED_OUT if (isResponseTimedOut(taskDefinition, pendingTask)) { timeoutTask(pendingTask); } diff --git a/core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java b/core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java new file mode 100644 index 0000000000..356a0cc896 --- /dev/null +++ b/core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java @@ -0,0 +1,42 @@ +/* + * Copyright 2016 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.conductor.core.utils; + +import com.netflix.conductor.common.run.ExternalStorageLocation; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; + +import java.io.InputStream; + +/** + * + */ +public class DummyPayloadStorage implements ExternalPayloadStorage { + + @Override + public ExternalStorageLocation getExternalUri(Operation operation, PayloadType payloadType) { + return null; + } + + @Override + public void upload(String path, InputStream payload, long payloadSize) { + } + + @Override + public InputStream download(String path) { + return null; + } +} diff --git a/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java b/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java new file mode 100644 index 0000000000..a863cbfea3 --- /dev/null +++ b/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java @@ -0,0 +1,138 @@ +/* + * Copyright 2018 Netflix, Inc. + *

+ * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.conductor.core.utils; + +import com.amazonaws.HttpMethod; +import com.amazonaws.SdkClientException; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; +import com.amazonaws.services.s3.model.GeneratePresignedUrlRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.S3Object; +import com.netflix.conductor.common.run.ExternalStorageLocation; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; +import com.netflix.conductor.core.config.Configuration; +import com.netflix.conductor.core.execution.ApplicationException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.inject.Inject; +import javax.inject.Singleton; +import java.io.InputStream; +import java.net.URISyntaxException; +import java.util.Date; + +/** + * An implementation of {@link ExternalPayloadStorage} using AWS S3 for storing large JSON payload data. + * The S3 client assumes that access to S3 is configured on the instance. + * see DefaultAWSCredentialsProviderChain + */ +@Singleton +public class S3PayloadStorage implements ExternalPayloadStorage { + private static final Logger logger = LoggerFactory.getLogger(S3PayloadStorage.class); + private static final String CONTENT_TYPE = "application/json"; + + private final AmazonS3 s3Client; + private final String bucketName; + private final int expirationSec; + + @Inject + public S3PayloadStorage(Configuration config) { + s3Client = AmazonS3ClientBuilder.standard().withRegion("us-east-1").build(); + bucketName = config.getProperty("s3bucket", ""); + expirationSec = config.getIntProperty("s3signedurlexpirationseconds", 3600); + } + + /** + * @param operation the type of {@link Operation} to be performed + * @param payloadType the {@link PayloadType} that is being accessed + * @return a {@link ExternalStorageLocation} object which contains the pre-signed URL and the s3 object key for the json payload + */ + @Override + public ExternalStorageLocation getExternalUri(Operation operation, PayloadType payloadType) { + try { + ExternalStorageLocation externalStorageLocation = new ExternalStorageLocation(); + + Date expiration = new Date(); + long expTimeMillis = expiration.getTime() + 1000 * expirationSec; + expiration.setTime(expTimeMillis); + + HttpMethod httpMethod = HttpMethod.GET; + if (operation == Operation.WRITE) { + httpMethod = HttpMethod.PUT; + } + + StringBuilder stringBuilder = new StringBuilder(); + if (payloadType == PayloadType.WORKFLOW_INPUT) { + stringBuilder.append("workflow/input/"); + } else if (payloadType == PayloadType.WORKFLOW_OUTPUT) { + stringBuilder.append("workflow/output/"); + } else if (payloadType == PayloadType.TASK_INPUT) { + stringBuilder.append("task/input/"); + } else { + stringBuilder.append("task/output/"); + } + stringBuilder.append(IDGenerator.generate()).append(".json"); + String objectKey = stringBuilder.toString(); + externalStorageLocation.setPath(objectKey); + + GeneratePresignedUrlRequest generatePresignedUrlRequest = new GeneratePresignedUrlRequest(bucketName, objectKey) + .withMethod(httpMethod) + .withExpiration(expiration); + + externalStorageLocation.setUri(s3Client.generatePresignedUrl(generatePresignedUrlRequest).toURI().toASCIIString()); + return externalStorageLocation; + } catch (SdkClientException e) { + String msg = "Error communicating with S3"; + logger.error(msg, e); + throw new ApplicationException(ApplicationException.Code.BACKEND_ERROR, msg, e); + } catch (URISyntaxException e) { + String msg = "Invalid URI Syntax"; + logger.error(msg, e); + throw new ApplicationException(ApplicationException.Code.INTERNAL_ERROR, msg, e); + } + } + + @Override + public void upload(String path, InputStream payload, long payloadSize) { + try { + ObjectMetadata objectMetadata = new ObjectMetadata(); + objectMetadata.setContentType(CONTENT_TYPE); + objectMetadata.setContentLength(payloadSize); + PutObjectRequest request = new PutObjectRequest(bucketName, path, payload, objectMetadata); + s3Client.putObject(request); + } catch (SdkClientException e) { + String msg = "Error communicating with S3"; + logger.error(msg, e); + throw new ApplicationException(ApplicationException.Code.BACKEND_ERROR, msg, e); + } + } + + @Override + public InputStream download(String path) { + try { + S3Object s3Object = s3Client.getObject(new GetObjectRequest(bucketName, path)); + return s3Object.getObjectContent(); + } catch (SdkClientException e) { + String msg = "Error communicating with S3"; + logger.error(msg, e); + throw new ApplicationException(ApplicationException.Code.BACKEND_ERROR, msg, e); + } + } +} diff --git a/core/src/main/java/com/netflix/conductor/service/ExecutionService.java b/core/src/main/java/com/netflix/conductor/service/ExecutionService.java index 05bf8c6e92..5876e6bb27 100644 --- a/core/src/main/java/com/netflix/conductor/service/ExecutionService.java +++ b/core/src/main/java/com/netflix/conductor/service/ExecutionService.java @@ -23,10 +23,14 @@ import com.netflix.conductor.common.metadata.tasks.TaskExecLog; import com.netflix.conductor.common.metadata.tasks.TaskResult; import com.netflix.conductor.common.metadata.workflow.WorkflowDef; +import com.netflix.conductor.common.run.ExternalStorageLocation; import com.netflix.conductor.common.run.SearchResult; import com.netflix.conductor.common.run.TaskSummary; import com.netflix.conductor.common.run.Workflow; import com.netflix.conductor.common.run.WorkflowSummary; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; +import com.netflix.conductor.common.utils.ExternalPayloadStorage.Operation; +import com.netflix.conductor.common.utils.ExternalPayloadStorage.PayloadType; import com.netflix.conductor.core.config.Configuration; import com.netflix.conductor.core.events.queue.Message; import com.netflix.conductor.core.execution.ApplicationException; @@ -80,6 +84,8 @@ public class ExecutionService { private final int maxSearchSize; + private final ExternalPayloadStorage externalPayloadStorage; + private static final int MAX_POLL_TIMEOUT_MS = 5000; private static final int POLL_COUNT_ONE = 1; @@ -87,12 +93,13 @@ public class ExecutionService { private static final int POLLING_TIMEOUT_IN_MS = 100; @Inject - public ExecutionService(WorkflowExecutor wfProvider, ExecutionDAO executionDAO, QueueDAO queueDAO, MetadataDAO metadataDAO, IndexDAO indexDAO, Configuration config) { + public ExecutionService(WorkflowExecutor wfProvider, ExecutionDAO executionDAO, QueueDAO queueDAO, MetadataDAO metadataDAO, IndexDAO indexDAO, Configuration config, ExternalPayloadStorage externalPayloadStorage) { this.workflowExecutor = wfProvider; this.executionDAO = executionDAO; this.queueDAO = queueDAO; this.metadataDAO = metadataDAO; this.indexDAO = indexDAO; + this.externalPayloadStorage = externalPayloadStorage; this.taskRequeueTimeout = config.getIntProperty("task.requeue.timeout", 60_000); this.maxSearchSize = config.getIntProperty("workflow.max.search.size", 5_000); } @@ -340,13 +347,13 @@ public SearchResult search(String query, String freeText, int s try { return new WorkflowSummary(executionDAO.getWorkflow(workflowId,false)); } catch(Exception e) { - logger.error(e.getMessage(), e); + logger.error("Error fetching workflow by id: {}", workflowId, e); return null; } }).filter(Objects::nonNull).collect(Collectors.toList()); int missing = result.getResults().size() - workflows.size(); long totalHits = result.getTotalHits() - missing; - return new SearchResult(totalHits, workflows); + return new SearchResult<>(totalHits, workflows); } public SearchResult searchWorkflowByTasks(String query, String freeText, int start, int size, List sortOptions) { @@ -358,7 +365,7 @@ public SearchResult searchWorkflowByTasks(String query, String String workflowId = taskSummary.getWorkflowId(); return new WorkflowSummary(executionDAO.getWorkflow(workflowId, false)); } catch (Exception e) { - logger.error("Error fetching workflow by id: ", e); + logger.error("Error fetching workflow by id: {}", taskSummary.getWorkflowId(), e); return null; } }) @@ -443,4 +450,14 @@ public List getTaskLogs(String taskId) { return indexDAO.getTaskExecutionLogs(taskId); } + /** + * Get external uri for the payload + * + * @param operation the type of {@link Operation} to be performed + * @param payloadType the {@link PayloadType} at the external uri + * @return the external uri at which the payload is stored/to be stored + */ + public ExternalStorageLocation getPayloadUri(Operation operation, PayloadType payloadType) { + return externalPayloadStorage.getExternalUri(operation, payloadType); + } } diff --git a/core/src/main/java/com/netflix/conductor/service/TaskService.java b/core/src/main/java/com/netflix/conductor/service/TaskService.java index 6558c461c2..eb71a5b596 100644 --- a/core/src/main/java/com/netflix/conductor/service/TaskService.java +++ b/core/src/main/java/com/netflix/conductor/service/TaskService.java @@ -21,8 +21,10 @@ import com.netflix.conductor.common.metadata.tasks.Task; import com.netflix.conductor.common.metadata.tasks.TaskExecLog; import com.netflix.conductor.common.metadata.tasks.TaskResult; +import com.netflix.conductor.common.run.ExternalStorageLocation; import com.netflix.conductor.common.run.SearchResult; import com.netflix.conductor.common.run.TaskSummary; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.dao.QueueDAO; import com.netflix.conductor.metrics.Monitors; import com.netflix.conductor.service.utils.ServiceUtils; @@ -60,9 +62,10 @@ public TaskService(ExecutionService executionService, QueueDAO queueDAO) { /** * Poll for a task of a certain type. + * * @param taskType Task name * @param workerId Id of the workflow - * @param domain Domain of the workflow + * @param domain Domain of the workflow * @return polled {@link Task} */ public Task poll(String taskType, String workerId, String domain) { @@ -78,11 +81,12 @@ public Task poll(String taskType, String workerId, String domain) { /** * Batch Poll for a task of a certain type. + * * @param taskType Task Name * @param workerId Id of the workflow - * @param domain Domain of the workflow - * @param count Number of tasks - * @param timeout Timeout for polling in milliseconds + * @param domain Domain of the workflow + * @param count Number of tasks + * @param timeout Timeout for polling in milliseconds * @return list of {@link Task} */ public List batchPoll(String taskType, String workerId, String domain, Integer count, Integer timeout) { @@ -98,9 +102,10 @@ public List batchPoll(String taskType, String workerId, String domain, Int /** * Get in progress tasks. The results are paginated. + * * @param taskType Task Name * @param startKey Start index of pagination - * @param count Number of entries + * @param count Number of entries * @return list of {@link Task} */ public List getTasks(String taskType, String startKey, Integer count) { @@ -110,7 +115,8 @@ public List getTasks(String taskType, String startKey, Integer count) { /** * Get in progress task for a given workflow id. - * @param workflowId Id of the workflow + * + * @param workflowId Id of the workflow * @param taskReferenceName Task reference name. * @return instance of {@link Task} */ @@ -122,6 +128,7 @@ public Task getPendingTaskForWorkflow(String workflowId, String taskReferenceNam /** * Updates a task. + * * @param taskResult Instance of {@link TaskResult} * @return task Id of the updated task. */ @@ -137,7 +144,8 @@ public String updateTask(TaskResult taskResult) { /** * Ack Task is received. - * @param taskId Id of the task + * + * @param taskId Id of the task * @param workerId Id of the worker * @return `true|false` if task if received or not */ @@ -149,8 +157,9 @@ public String ackTaskReceived(String taskId, String workerId) { /** * Log Task Execution Details. + * * @param taskId Id of the task - * @param log Details you want to log + * @param log Details you want to log */ public void log(String taskId, String log) { ServiceUtils.checkNotNullOrEmpty(taskId, "TaskId cannot be null or empty."); @@ -159,6 +168,7 @@ public void log(String taskId, String log) { /** * Get Task Execution Logs. + * * @param taskId Id of the task. * @return list of {@link TaskExecLog} */ @@ -169,6 +179,7 @@ public List getTaskLogs(String taskId) { /** * Get task by Id. + * * @param taskId Id of the task. * @return instance of {@link Task} */ @@ -180,8 +191,9 @@ public Task getTask(String taskId) { /** * Remove Task from a Task type queue. + * * @param taskType Task Name - * @param taskId ID of the task + * @param taskId ID of the task */ public void removeTaskFromQueue(String taskType, String taskId) { ServiceUtils.checkNotNullOrEmpty(taskType, "TaskType cannot be null or empty."); @@ -191,6 +203,7 @@ public void removeTaskFromQueue(String taskType, String taskId) { /** * Get Task type queue sizes. + * * @param taskTypes List of task types. * @return map of task type as Key and queue size as value. */ @@ -200,6 +213,7 @@ public Map getTaskQueueSizes(List taskTypes) { /** * Get the details about each queue. + * * @return map of queue details. */ public Map>> allVerbose() { @@ -208,6 +222,7 @@ public Map>> allVerbose() { /** * Get the details about each queue. + * * @return map of details about each queue. */ public Map getAllQueueDetails() { @@ -218,6 +233,7 @@ public Map getAllQueueDetails() { /** * Get the last poll data for a given task type. + * * @param taskType Task Name * @return list of {@link PollData} */ @@ -229,6 +245,7 @@ public List getPollData(String taskType) { /** * Get the last poll data for all task types. + * * @return list of {@link PollData} */ public List getAllPollData() { @@ -237,6 +254,7 @@ public List getAllPollData() { /** * Requeue pending tasks for all the running workflows. + * * @return number of tasks requeued. */ public String requeue() { @@ -245,26 +263,37 @@ public String requeue() { /** * Requeue pending tasks. + * * @param taskType Task name. * @return number of tasks requeued. */ public String requeuePendingTask(String taskType) { - ServiceUtils.checkNotNullOrEmpty(taskType,"TaskType cannot be null or empty."); + ServiceUtils.checkNotNullOrEmpty(taskType, "TaskType cannot be null or empty."); return String.valueOf(executionService.requeuePendingTasks(taskType)); } /** * Search for tasks based in payload and other parameters. Use sort options as ASC or DESC e.g. * sort=name or sort=workflowId. If order is not specified, defaults to ASC. - * @param start Start index of pagination - * @param size Number of entries - * @param sort Sorting type ASC|DESC + * + * @param start Start index of pagination + * @param size Number of entries + * @param sort Sorting type ASC|DESC * @param freeText Text you want to search - * @param query Query you want to search + * @param query Query you want to search * @return instance of {@link SearchResult} */ public SearchResult search(int start, int size, String sort, String freeText, String query) { return executionService.getSearchTasks(query, freeText, start, size, sort); } + + /** + * Get the external storage location where the task output payload is stored/to be stored + * + * @return {@link ExternalStorageLocation} containing the uri and the path to the payload is stored in external storage + */ + public ExternalStorageLocation getPayloadUri() { + return executionService.getPayloadUri(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.TASK_OUTPUT); + } } diff --git a/core/src/main/java/com/netflix/conductor/service/WorkflowService.java b/core/src/main/java/com/netflix/conductor/service/WorkflowService.java index ae8af409c6..b170ce761b 100644 --- a/core/src/main/java/com/netflix/conductor/service/WorkflowService.java +++ b/core/src/main/java/com/netflix/conductor/service/WorkflowService.java @@ -5,9 +5,11 @@ import com.netflix.conductor.common.metadata.workflow.SkipTaskRequest; import com.netflix.conductor.common.metadata.workflow.StartWorkflowRequest; import com.netflix.conductor.common.metadata.workflow.WorkflowDef; +import com.netflix.conductor.common.run.ExternalStorageLocation; import com.netflix.conductor.common.run.SearchResult; import com.netflix.conductor.common.run.Workflow; import com.netflix.conductor.common.run.WorkflowSummary; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.core.config.Configuration; import com.netflix.conductor.core.execution.ApplicationException; import com.netflix.conductor.core.execution.WorkflowExecutor; @@ -279,4 +281,12 @@ public SearchResult searchWorkflowsByTasks(int start, int size, return executionService.searchWorkflowByTasks(query, freeText, start, size, ServiceUtils.convertStringToList(sort)); } + /** + * Get the external storage location where the workflow input payload is stored/to be stored + * + * @return {@link ExternalStorageLocation} containing the uri and the path to the payload is stored in external storage + */ + public ExternalStorageLocation getExternalPayloadUri() { + return executionService.getPayloadUri(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.WORKFLOW_INPUT); + } } diff --git a/es2-persistence/dependencies.lock b/es2-persistence/dependencies.lock index 2cb2a634b3..25a979cb31 100644 --- a/es2-persistence/dependencies.lock +++ b/es2-persistence/dependencies.lock @@ -1,5 +1,11 @@ { "compile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -91,6 +97,12 @@ } }, "compileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -205,6 +217,12 @@ } }, "default": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -296,6 +314,12 @@ } }, "runtime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -387,6 +411,12 @@ } }, "runtimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -492,6 +522,12 @@ } }, "testCompile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -591,6 +627,12 @@ } }, "testCompileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -690,6 +732,12 @@ } }, "testRuntime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -798,6 +846,12 @@ } }, "testRuntimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" diff --git a/es5-persistence/dependencies.lock b/es5-persistence/dependencies.lock index 50e8a3ba90..3b0573d567 100644 --- a/es5-persistence/dependencies.lock +++ b/es5-persistence/dependencies.lock @@ -1,5 +1,11 @@ { "compile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -104,6 +110,12 @@ } }, "compileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -208,6 +220,12 @@ } }, "default": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -312,6 +330,12 @@ } }, "runtime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -416,6 +440,12 @@ } }, "runtimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -520,6 +550,12 @@ } }, "testCompile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -632,6 +668,12 @@ } }, "testCompileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -744,6 +786,12 @@ } }, "testRuntime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -856,6 +904,12 @@ } }, "testRuntimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" diff --git a/jersey/dependencies.lock b/jersey/dependencies.lock index 31244b50af..a22e55ecdc 100644 --- a/jersey/dependencies.lock +++ b/jersey/dependencies.lock @@ -1,5 +1,11 @@ { "compile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -99,6 +105,12 @@ } }, "compileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -198,6 +210,12 @@ } }, "default": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -303,6 +321,12 @@ } }, "runtime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -402,6 +426,12 @@ } }, "runtimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -501,6 +531,12 @@ } }, "testCompile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -608,6 +644,12 @@ } }, "testCompileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -715,6 +757,12 @@ } }, "testRuntime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -822,6 +870,12 @@ } }, "testRuntimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" diff --git a/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java b/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java index 0a0e1a53a4..5ce34a3b48 100644 --- a/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java +++ b/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java @@ -19,6 +19,7 @@ import com.netflix.conductor.common.metadata.tasks.Task; import com.netflix.conductor.common.metadata.tasks.TaskExecLog; import com.netflix.conductor.common.metadata.tasks.TaskResult; +import com.netflix.conductor.common.run.ExternalStorageLocation; import com.netflix.conductor.common.run.SearchResult; import com.netflix.conductor.common.run.TaskSummary; import com.netflix.conductor.service.TaskService; @@ -221,4 +222,11 @@ public SearchResult search(@QueryParam("start") @DefaultValue("0") return taskService.search(start, size, sort, freeText, query); } + @GET + @ApiOperation("Get the external uri where the task output payload is to be stored") + @Consumes(MediaType.WILDCARD) + @Path("/externalstoragelocation") + public ExternalStorageLocation getPayloadURI() { + return taskService.getPayloadUri(); + } } diff --git a/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java b/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java index 31dfd92b12..6f62c4a0e4 100644 --- a/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java +++ b/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java @@ -22,6 +22,7 @@ import com.netflix.conductor.common.metadata.workflow.RerunWorkflowRequest; import com.netflix.conductor.common.metadata.workflow.SkipTaskRequest; import com.netflix.conductor.common.metadata.workflow.StartWorkflowRequest; +import com.netflix.conductor.common.run.ExternalStorageLocation; import com.netflix.conductor.common.run.SearchResult; import com.netflix.conductor.common.run.Workflow; import com.netflix.conductor.common.run.WorkflowSummary; @@ -241,4 +242,12 @@ public SearchResult searchWorkflowsByTasks(@QueryParam("start") @QueryParam("query") String query) { return workflowService.searchWorkflowsByTasks(start, size, sort, freeText, query); } + + @GET + @ApiOperation("Get the uri and path of the external storage where the workflow input payload is to be stored") + @Consumes(MediaType.WILDCARD) + @Path("/externalstoragelocation") + public ExternalStorageLocation getExternalPayloadURI() { + return workflowService.getExternalPayloadUri(); + } } diff --git a/redis-persistence/dependencies.lock b/redis-persistence/dependencies.lock index 1b664232c3..4a9952f4de 100644 --- a/redis-persistence/dependencies.lock +++ b/redis-persistence/dependencies.lock @@ -1,5 +1,11 @@ { "compile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -100,6 +106,12 @@ } }, "compileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -200,6 +212,12 @@ } }, "default": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -300,6 +318,12 @@ } }, "runtime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -400,6 +424,12 @@ } }, "runtimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -500,6 +530,12 @@ } }, "testCompile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -612,6 +648,12 @@ } }, "testCompileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -724,6 +766,12 @@ } }, "testRuntime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" @@ -836,6 +884,12 @@ } }, "testRuntimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-core" diff --git a/server/dependencies.lock b/server/dependencies.lock index 1c9a270a66..7876e0d433 100644 --- a/server/dependencies.lock +++ b/server/dependencies.lock @@ -1,10 +1,16 @@ { "compile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -272,11 +278,17 @@ } }, "compileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -544,11 +556,17 @@ } }, "default": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -816,11 +834,17 @@ } }, "grettyProductRuntime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -1148,11 +1172,17 @@ } }, "runtime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -1420,11 +1450,17 @@ } }, "runtimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -1692,11 +1728,17 @@ } }, "springBoot": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -1964,11 +2006,17 @@ } }, "testCompile": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -2244,11 +2292,17 @@ } }, "testCompileClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -2524,11 +2578,17 @@ } }, "testRuntime": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -2804,11 +2864,17 @@ } }, "testRuntimeClasspath": { + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" + }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ diff --git a/server/src/main/java/com/netflix/conductor/server/ConductorConfig.java b/server/src/main/java/com/netflix/conductor/server/ConductorConfig.java index 1b25d70deb..ad9f09a55b 100644 --- a/server/src/main/java/com/netflix/conductor/server/ConductorConfig.java +++ b/server/src/main/java/com/netflix/conductor/server/ConductorConfig.java @@ -18,6 +18,12 @@ */ package com.netflix.conductor.server; +import com.google.inject.AbstractModule; +import com.netflix.conductor.core.config.Configuration; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.net.InetAddress; import java.net.UnknownHostException; import java.util.HashMap; @@ -27,13 +33,6 @@ import java.util.Optional; import java.util.Properties; -import org.apache.commons.lang3.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.inject.AbstractModule; -import com.netflix.conductor.core.config.Configuration; - /** * @author Viren * @@ -98,7 +97,9 @@ public int getIntProperty(String key, int defaultValue) { String val = getProperty(key, Integer.toString(defaultValue)); try{ defaultValue = Integer.parseInt(val); - }catch(NumberFormatException e){} + } catch(NumberFormatException e){ + logger.error("Error parsing the Int value for Key:{} , returning a default value: {}", key, defaultValue); + } return defaultValue; } @@ -132,7 +133,7 @@ public String getProperty(String key, String defaultValue) { public Map getAll() { Map map = new HashMap<>(); Properties props = System.getProperties(); - props.entrySet().forEach(entry -> map.put(entry.getKey().toString(), entry.getValue())); + props.forEach((key, value) -> map.put(key.toString(), value)); return map; } diff --git a/server/src/main/java/com/netflix/conductor/server/ConductorServer.java b/server/src/main/java/com/netflix/conductor/server/ConductorServer.java index 80133f84da..3a8d2ef293 100644 --- a/server/src/main/java/com/netflix/conductor/server/ConductorServer.java +++ b/server/src/main/java/com/netflix/conductor/server/ConductorServer.java @@ -64,6 +64,10 @@ public class ConductorServer { enum DB { redis, dynomite, memory, redis_cluster, mysql } + + enum ExternalPayloadStorage { + S3 + } private ServerModule serverModule; @@ -72,6 +76,8 @@ enum DB { private ConductorConfig conductorConfig; private DB database; + + private ExternalPayloadStorage externalPayloadStorage; public ConductorServer(ConductorConfig conductorConfig) { this.conductorConfig = conductorConfig; @@ -103,12 +109,19 @@ public ConductorServer(ConductorConfig conductorConfig) { Host dynoHost = new Host(host, port, rack, Status.Up); dynoHosts.add(dynoHost); } - }else { //Create a single shard host supplier Host dynoHost = new Host("localhost", 0, conductorConfig.getAvailabilityZone(), Status.Up); dynoHosts.add(dynoHost); } + + String externalPayloadStorageString = conductorConfig.getProperty("workflow.external.payload.storage", ""); + try { + externalPayloadStorage = ConductorServer.ExternalPayloadStorage.valueOf(externalPayloadStorageString); + } catch(IllegalArgumentException e) { + logger.info("External payload storage is not configured, provided: {}, supported values are: {}", externalPayloadStorageString, Arrays.toString(ConductorServer.ExternalPayloadStorage.values()), e); + } + init(dynoClusterName, dynoHosts); } @@ -174,7 +187,7 @@ private void init(String dynoClusterName, List dynoHosts) { break; } - this.serverModule = new ServerModule(jedis, hostSupplier, conductorConfig, database); + this.serverModule = new ServerModule(jedis, hostSupplier, conductorConfig, database, externalPayloadStorage); } private TokenMapSupplier getTokenMapSupplier(List dynoHosts) { diff --git a/server/src/main/java/com/netflix/conductor/server/ServerModule.java b/server/src/main/java/com/netflix/conductor/server/ServerModule.java index e88b734d99..d9b592364f 100644 --- a/server/src/main/java/com/netflix/conductor/server/ServerModule.java +++ b/server/src/main/java/com/netflix/conductor/server/ServerModule.java @@ -20,11 +20,14 @@ import com.google.inject.AbstractModule; import com.google.inject.Provides; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.contribs.http.HttpTask; import com.netflix.conductor.contribs.http.RestClientManager; import com.netflix.conductor.contribs.json.JsonJqTransform; import com.netflix.conductor.core.config.Configuration; import com.netflix.conductor.core.config.CoreModule; +import com.netflix.conductor.core.utils.DummyPayloadStorage; +import com.netflix.conductor.core.utils.S3PayloadStorage; import com.netflix.conductor.dao.RedisWorkflowModule; import com.netflix.conductor.dao.es.index.ElasticSearchModule; import com.netflix.conductor.dao.es5.index.ElasticSearchModuleV5; @@ -58,14 +61,16 @@ public class ServerModule extends AbstractModule { private ConductorServer.DB db; - public ServerModule(JedisCommands jedis, HostSupplier hostSupplier, ConductorConfig conductorConfig, ConductorServer.DB db) { + private ConductorServer.ExternalPayloadStorage externalPayloadStorage; + + public ServerModule(JedisCommands jedis, HostSupplier hostSupplier, ConductorConfig conductorConfig, ConductorServer.DB db, ConductorServer.ExternalPayloadStorage externalPayloadStorage) { this.dynoConn = jedis; this.hostSupplier = hostSupplier; this.conductorConfig = conductorConfig; this.region = conductorConfig.getRegion(); this.localRack = conductorConfig.getAvailabilityZone(); this.db = db; - + this.externalPayloadStorage = externalPayloadStorage; } @Override @@ -101,6 +106,12 @@ protected void configure() { install(additionalModule); } } + + if (externalPayloadStorage == ConductorServer.ExternalPayloadStorage.S3) { + bind(ExternalPayloadStorage.class).to(S3PayloadStorage.class); + } else { + bind(ExternalPayloadStorage.class).to(DummyPayloadStorage.class); + } } @Provides diff --git a/server/src/main/resources/server.properties b/server/src/main/resources/server.properties index e0e4fda704..1830a66d9f 100644 --- a/server/src/main/resources/server.properties +++ b/server/src/main/resources/server.properties @@ -39,5 +39,5 @@ workflow.elasticsearch.index.name=conductor #Elasticsearch major release version. workflow.elasticsearch.version=2 -# For a single node dynomite or redis server, make sure the value below is set to same as rack specified in the "workflow.dynomite.cluster.hosts" property. +# For a single node dynomite or redis server, make sure the value below is set to same as rack specified in the "workflow.dynomite.cluster.hosts" property. EC2_AVAILABILITY_ZONE=us-east-1c diff --git a/test-harness/dependencies.lock b/test-harness/dependencies.lock index 7f23fad650..92da9ea42a 100644 --- a/test-harness/dependencies.lock +++ b/test-harness/dependencies.lock @@ -4,13 +4,19 @@ "firstLevelTransitive": [ "com.netflix.conductor:conductor-client" ], - "locked": "1.11.380" + "locked": "1.11.389" + }, + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -360,13 +366,19 @@ "firstLevelTransitive": [ "com.netflix.conductor:conductor-client" ], - "locked": "1.11.380" + "locked": "1.11.389" + }, + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -716,13 +728,19 @@ "firstLevelTransitive": [ "com.netflix.conductor:conductor-client" ], - "locked": "1.11.380" + "locked": "1.11.389" + }, + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ @@ -1072,13 +1090,19 @@ "firstLevelTransitive": [ "com.netflix.conductor:conductor-client" ], - "locked": "1.11.380" + "locked": "1.11.389" + }, + "com.amazonaws:aws-java-sdk-s3": { + "firstLevelTransitive": [ + "com.netflix.conductor:conductor-core" + ], + "locked": "1.11.86" }, "com.amazonaws:aws-java-sdk-sqs": { "firstLevelTransitive": [ "com.netflix.conductor:conductor-contribs" ], - "locked": "1.11.380" + "locked": "1.11.389" }, "com.fasterxml.jackson.core:jackson-core": { "firstLevelTransitive": [ diff --git a/test-harness/src/test/java/com/netflix/conductor/tests/integration/End2EndTests.java b/test-harness/src/test/java/com/netflix/conductor/tests/integration/End2EndTests.java index 7e780c332f..584a19f669 100644 --- a/test-harness/src/test/java/com/netflix/conductor/tests/integration/End2EndTests.java +++ b/test-harness/src/test/java/com/netflix/conductor/tests/integration/End2EndTests.java @@ -298,11 +298,8 @@ public void testStartWorkflow() { StartWorkflowRequest startWorkflowRequest = new StartWorkflowRequest(); try{ wc.startWorkflow(startWorkflowRequest); - } catch (ConductorClientException e) { - int statuCode = e.getStatus(); - assertEquals(400, statuCode); + } catch (IllegalArgumentException e) { assertEquals("Workflow name cannot be null or empty", e.getMessage()); - assertFalse(e.isRetryable()); } } diff --git a/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java b/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java new file mode 100644 index 0000000000..229be587c0 --- /dev/null +++ b/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java @@ -0,0 +1,38 @@ +/* + * Copyright 2017 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.conductor.tests.utils; + +import com.netflix.conductor.common.run.ExternalStorageLocation; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; + +import java.io.InputStream; + +public class MockExternalPayloadStorage implements ExternalPayloadStorage { + + @Override + public ExternalStorageLocation getExternalUri(Operation operation, PayloadType payloadType) { + return null; + } + + @Override + public void upload(String path, InputStream payload, long payloadSize) { + } + + @Override + public InputStream download(String path) { + return null; + } +} diff --git a/test-harness/src/test/java/com/netflix/conductor/tests/utils/TestModule.java b/test-harness/src/test/java/com/netflix/conductor/tests/utils/TestModule.java index 5ff5aec8e8..c49d06f146 100644 --- a/test-harness/src/test/java/com/netflix/conductor/tests/utils/TestModule.java +++ b/test-harness/src/test/java/com/netflix/conductor/tests/utils/TestModule.java @@ -18,16 +18,9 @@ */ package com.netflix.conductor.tests.utils; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; - import com.google.inject.AbstractModule; import com.google.inject.Provides; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.core.config.Configuration; import com.netflix.conductor.core.config.CoreModule; import com.netflix.conductor.dao.ExecutionDAO; @@ -41,9 +34,14 @@ import com.netflix.conductor.redis.utils.JedisMock; import com.netflix.conductor.server.ConductorConfig; import com.netflix.dyno.queues.ShardSupplier; - import redis.clients.jedis.JedisCommands; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicInteger; + /** * @author Viren * @@ -89,6 +87,7 @@ public String getCurrentShard() { bind(DynoProxy.class).toInstance(proxy); install(new CoreModule()); bind(UserTask.class).asEagerSingleton(); + bind(ExternalPayloadStorage.class).to(MockExternalPayloadStorage.class); } @Provides From 6750f55592123857628ec6a5f9d53f7f37a462a9 Mon Sep 17 00:00:00 2001 From: Anoop Panicker Date: Thu, 23 Aug 2018 16:30:02 -0700 Subject: [PATCH 2/4] use external workflow input and external task output --- .../config/ConductorClientConfiguration.java | 16 +- .../DefaultConductorClientConfiguration.java | 10 + .../conductor/client/http/ClientBase.java | 26 +- .../conductor/client/http/MetadataClient.java | 30 +- .../client/{util => http}/PayloadStorage.java | 51 ++- .../conductor/client/http/TaskClient.java | 39 +- .../conductor/client/http/WorkflowClient.java | 33 +- .../client/task/WorkflowTaskCoordinator.java | 1 + .../client/task/WorkflowTaskMetrics.java | 9 +- .../conductor/common/metadata/tasks/Task.java | 43 ++- .../common/metadata/tasks/TaskResult.java | 15 +- .../workflow/StartWorkflowRequest.java | 10 +- .../common/run/ExternalStorageLocation.java | 12 +- .../conductor/common/run/Workflow.java | 162 +++++--- .../common/utils/ExternalPayloadStorage.java | 2 +- .../common/workflow/TestWorkflowDef.java | 18 +- .../conductor/contribs/http/HttpTask.java | 4 +- .../conductor/contribs/http/TestHttpTask.java | 26 +- .../conductor/core/config/CoreModule.java | 9 +- .../core/events/ActionProcessor.java | 6 +- .../conductor/core/events/EventProcessor.java | 6 +- .../conductor/core/events/EventQueues.java | 25 +- .../core/execution/DeciderService.java | 89 +++-- .../core/execution/ParametersUtils.java | 94 +++-- .../core/execution/WorkflowExecutor.java | 134 +++---- .../execution/mapper/DecisionTaskMapper.java | 3 +- .../execution/mapper/SimpleTaskMapper.java | 5 +- .../core/execution/mapper/WaitTaskMapper.java | 3 +- .../conductor/core/execution/tasks/Event.java | 26 +- .../core/execution/tasks/SubWorkflow.java | 15 +- .../core/utils/DummyPayloadStorage.java | 4 +- .../conductor/core/utils/JsonUtils.java | 2 +- .../core/utils/S3PayloadStorage.java | 42 ++- .../conductor/service/EventService.java | 9 +- .../conductor/service/ExecutionService.java | 12 +- .../conductor/service/MetadataService.java | 9 +- .../conductor/service/TaskService.java | 4 +- .../conductor/service/WorkflowService.java | 6 +- .../core/events/TestEventProcessor.java | 39 +- .../core/execution/TestDeciderOutcomes.java | 4 +- .../core/execution/TestDeciderService.java | 51 +-- .../core/execution/TestWorkflowExecutor.java | 12 +- .../mapper/DecisionTaskMapperTest.java | 22 +- .../mapper/SubWorkflowTaskMapperTest.java | 15 +- .../execution/mapper/WaitTaskMapperTest.java | 12 +- .../core/execution/tasks/TestEvent.java | 108 +++--- .../service/WorkflowServiceTest.java | 58 ++- .../server/resources/TaskResource.java | 8 +- .../server/resources/WorkflowResource.java | 8 +- .../dao/dynomite/RedisExecutionDAO.java | 12 +- .../dao/dynomite/RedisMetadataDAO.java | 27 +- .../dao/dynomite/queue/DynoQueueDAO.java | 38 +- .../conductor/server/ConductorServer.java | 10 +- .../conductor/server/ServerModule.java | 16 +- .../integration/WorkflowServiceTest.java | 348 ++++++++++-------- .../utils/MockExternalPayloadStorage.java | 32 +- 56 files changed, 1050 insertions(+), 780 deletions(-) rename client/src/main/java/com/netflix/conductor/client/{util => http}/PayloadStorage.java (73%) diff --git a/client/src/main/java/com/netflix/conductor/client/config/ConductorClientConfiguration.java b/client/src/main/java/com/netflix/conductor/client/config/ConductorClientConfiguration.java index 3acf0ed6be..55597736cb 100644 --- a/client/src/main/java/com/netflix/conductor/client/config/ConductorClientConfiguration.java +++ b/client/src/main/java/com/netflix/conductor/client/config/ConductorClientConfiguration.java @@ -19,17 +19,29 @@ public interface ConductorClientConfiguration { /** - * @return the workflow input payload threshold in KB, + * @return the workflow input payload size threshold in KB, * beyond which the payload will be processed based on {@link ConductorClientConfiguration#isExternalPayloadStorageEnabled()}. */ int getWorkflowInputPayloadThresholdKB(); /** - * @return the task output payload threshold in KB, + * @return the max value of workflow input payload size threshold in KB, + * beyond which the payload will be rejected regardless external payload storage is enabled. + */ + int getWorkflowInputMaxPayloadThresholdKB(); + + /** + * @return the task output payload size threshold in KB, * beyond which the payload will be processed based on {@link ConductorClientConfiguration#isExternalPayloadStorageEnabled()}. */ int getTaskOutputPayloadThresholdKB(); + /** + * @return the max value of task output payload size threshold in KB, + * beyond which the payload will be rejected regardless external payload storage is enabled. + */ + int getTaskOutputMaxPayloadThresholdKB(); + /** * @return the flag which controls the use of external storage for storing workflow/task * input and output JSON payloads with size greater than threshold. diff --git a/client/src/main/java/com/netflix/conductor/client/config/DefaultConductorClientConfiguration.java b/client/src/main/java/com/netflix/conductor/client/config/DefaultConductorClientConfiguration.java index 65873d48a4..5ea2c12435 100644 --- a/client/src/main/java/com/netflix/conductor/client/config/DefaultConductorClientConfiguration.java +++ b/client/src/main/java/com/netflix/conductor/client/config/DefaultConductorClientConfiguration.java @@ -27,11 +27,21 @@ public int getWorkflowInputPayloadThresholdKB() { return 5120; } + @Override + public int getWorkflowInputMaxPayloadThresholdKB() { + return 10240; + } + @Override public int getTaskOutputPayloadThresholdKB() { return 3072; } + @Override + public int getTaskOutputMaxPayloadThresholdKB() { + return 10240; + } + @Override public boolean isExternalPayloadStorageEnabled() { return false; diff --git a/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java b/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java index 9cf8969f06..b8dd8f8363 100644 --- a/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java +++ b/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java @@ -20,13 +20,13 @@ import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; +import com.google.common.base.Preconditions; import com.netflix.conductor.client.config.ConductorClientConfiguration; import com.netflix.conductor.client.config.DefaultConductorClientConfiguration; import com.netflix.conductor.client.exceptions.ConductorClientException; import com.netflix.conductor.client.exceptions.ErrorResponse; -import com.netflix.conductor.client.util.PayloadStorage; -import com.netflix.conductor.common.metadata.tasks.TaskResult; import com.netflix.conductor.common.run.ExternalStorageLocation; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.sun.jersey.api.client.Client; import com.sun.jersey.api.client.ClientHandler; import com.sun.jersey.api.client.ClientHandlerException; @@ -65,11 +65,11 @@ public abstract class ClientBase { protected ConductorClientConfiguration conductorClientConfiguration; protected ClientBase() { - this(new DefaultClientConfig(), null); + this(new DefaultClientConfig(), new DefaultConductorClientConfiguration(), null); } protected ClientBase(ClientConfig config) { - this(config, null); + this(config, new DefaultConductorClientConfiguration(), null); } protected ClientBase(ClientConfig config, ClientHandler handler) { @@ -84,8 +84,6 @@ protected ClientBase(ClientConfig config, ConductorClientConfiguration clientCo objectMapper.setSerializationInclusion(Include.NON_NULL); objectMapper.setSerializationInclusion(Include.NON_EMPTY); - payloadStorage = new PayloadStorage(); - JacksonJsonProvider provider = new JacksonJsonProvider(objectMapper); config.getSingletons().add(provider); @@ -96,6 +94,8 @@ protected ClientBase(ClientConfig config, ConductorClientConfiguration clientCo } conductorClientConfiguration = clientConfiguration; + payloadStorage = new PayloadStorage(this); + } public void setRootURI(String root) { @@ -181,11 +181,11 @@ private T postForEntity(String url, Object request, Object[] queryParams, Ob return null; } - protected T getForEntity(String url, Object[] queryParams, Class responseType, Object... uriVariables) { + T getForEntity(String url, Object[] queryParams, Class responseType, Object... uriVariables) { return getForEntity(url, queryParams, response -> response.getEntity(responseType), uriVariables); } - protected T getForEntity(String url, Object[] queryParams, GenericType responseType, Object... uriVariables) { + T getForEntity(String url, Object[] queryParams, GenericType responseType, Object... uriVariables) { return getForEntity(url, queryParams, response -> response.getEntity(responseType), uriVariables); } @@ -215,13 +215,15 @@ private T getForEntity(String url, Object[] queryParams, Function getAllWorkflowDefs() { * Removes the workflow definition of a workflow from the conductor server. * It does not remove associated workflows. Use with caution. * - * @param name Name of the workflow to be unregistered. + * @param name Name of the workflow to be unregistered. * @param version Version of the workflow definition to be unregistered. */ public void unregisterWorkflowDef(String name, Integer version) { Preconditions.checkArgument(StringUtils.isNotBlank(name), "Workflow name cannot be blank"); Preconditions.checkNotNull(version, "Version cannot be null"); - delete("metadata/workflow/{name}/{version}", name, version); + delete("metadata/workflow/{name}/{version}", name, version); } // Task Metadata Operations @@ -152,7 +148,7 @@ public void unregisterWorkflowDef(String name, Integer version) { */ public void registerTaskDefs(List taskDefs) { Preconditions.checkNotNull(taskDefs, "Task defs list cannot be null"); - postForEntity("metadata/taskdefs", taskDefs); + postForEntityWithRequestOnly("metadata/taskdefs", taskDefs); } /** diff --git a/client/src/main/java/com/netflix/conductor/client/util/PayloadStorage.java b/client/src/main/java/com/netflix/conductor/client/http/PayloadStorage.java similarity index 73% rename from client/src/main/java/com/netflix/conductor/client/util/PayloadStorage.java rename to client/src/main/java/com/netflix/conductor/client/http/PayloadStorage.java index bd729a1532..e9d296cc21 100644 --- a/client/src/main/java/com/netflix/conductor/client/util/PayloadStorage.java +++ b/client/src/main/java/com/netflix/conductor/client/http/PayloadStorage.java @@ -14,8 +14,9 @@ * limitations under the License. */ -package com.netflix.conductor.client.util; +package com.netflix.conductor.client.http; +import com.amazonaws.util.IOUtils; import com.netflix.conductor.client.exceptions.ConductorClientException; import com.netflix.conductor.common.run.ExternalStorageLocation; import com.netflix.conductor.common.utils.ExternalPayloadStorage; @@ -37,49 +38,65 @@ public class PayloadStorage implements ExternalPayloadStorage { private static final Logger logger = LoggerFactory.getLogger(PayloadStorage.class); + private final ClientBase clientBase; + + public PayloadStorage(ClientBase clientBase) { + this.clientBase = clientBase; + } + /** * This method is not intended to be used in the client. * The client makes a request to the server to get the {@link ExternalStorageLocation} */ @Override - public ExternalStorageLocation getExternalUri(Operation operation, PayloadType payloadType) { - throw new UnsupportedOperationException(); + public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType) { + String uri; + switch (payloadType) { + case WORKFLOW_INPUT: + case WORKFLOW_OUTPUT: + uri = "workflow"; + break; + case TASK_INPUT: + case TASK_OUTPUT: + uri = "tasks"; + break; + default: + throw new ConductorClientException(String.format("Invalid payload type: %s for operation: %s", payloadType.toString(), operation.toString())); + } + return clientBase.getForEntity(String.format("%s/externalstoragelocation", uri), null, ExternalStorageLocation.class); } /** * Uploads the payload to the uri specified. * - * @param path the location to which the object is to be uploaded + * @param uri the location to which the object is to be uploaded * @param payload an {@link InputStream} containing the json payload which is to be uploaded * @param payloadSize the size of the json payload in bytes * @throws ConductorClientException if the upload fails due to an invalid path or an error from external storage */ @Override - public void upload(String path, InputStream payload, long payloadSize) { + public void upload(String uri, InputStream payload, long payloadSize) { HttpURLConnection connection = null; try { - URL url = new URI(path).toURL(); + URL url = new URI(uri).toURL(); connection = (HttpURLConnection) url.openConnection(); connection.setDoOutput(true); connection.setRequestMethod("PUT"); try (BufferedOutputStream bufferedOutputStream = new BufferedOutputStream(connection.getOutputStream())) { - int length; - while ((length = payload.read()) != -1) { - bufferedOutputStream.write(length); - } + IOUtils.copy(payload, bufferedOutputStream); // Check the HTTP response code int responseCode = connection.getResponseCode(); logger.debug("Upload completed with HTTP response code: {}", responseCode); } } catch (URISyntaxException | MalformedURLException e) { - String errorMsg = String.format("Invalid path specified: %s", path); + String errorMsg = String.format("Invalid path specified: %s", uri); logger.error(errorMsg, e); throw new ConductorClientException(errorMsg, e); } catch (IOException e) { - String errorMsg = String.format("Error uploading to path: %s", path); + String errorMsg = String.format("Error uploading to path: %s", uri); logger.error(errorMsg, e); throw new ConductorClientException(errorMsg, e); } finally { @@ -99,6 +116,7 @@ public void upload(String path, InputStream payload, long payloadSize) { @Override public InputStream download(String path) { HttpURLConnection connection = null; + String errorMsg; try { URL url = new URI(path).toURL(); connection = (HttpURLConnection) url.openConnection(); @@ -110,14 +128,15 @@ public InputStream download(String path) { logger.debug("Download completed with HTTP response code: {}", connection.getResponseCode()); return connection.getInputStream(); } - logger.info("No file to download. Response code: {}", responseCode); - return null; + errorMsg = String.format("Unable to download. Response code: %d", responseCode); + logger.error(errorMsg); + throw new ConductorClientException(errorMsg); } catch (URISyntaxException | MalformedURLException e) { - String errorMsg = String.format("Invalid path specified: %s", path); + errorMsg = String.format("Invalid path specified: %s", path); logger.error(errorMsg, e); throw new ConductorClientException(errorMsg, e); } catch (IOException e) { - String errorMsg = String.format("Error downloading from path: %s", path); + errorMsg = String.format("Error downloading from path: %s", path); logger.error(errorMsg, e); throw new ConductorClientException(errorMsg, e); } finally { diff --git a/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java b/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java index 5274acb51f..cdf26f2bbf 100644 --- a/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java +++ b/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java @@ -17,6 +17,7 @@ import com.google.common.base.Preconditions; import com.netflix.conductor.client.config.ConductorClientConfiguration; +import com.netflix.conductor.client.config.DefaultConductorClientConfiguration; import com.netflix.conductor.client.exceptions.ConductorClientException; import com.netflix.conductor.client.task.WorkflowTaskMetrics; import com.netflix.conductor.common.metadata.tasks.PollData; @@ -26,9 +27,11 @@ import com.netflix.conductor.common.metadata.tasks.TaskResult; import com.netflix.conductor.common.run.SearchResult; import com.netflix.conductor.common.run.TaskSummary; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.sun.jersey.api.client.ClientHandler; import com.sun.jersey.api.client.GenericType; import com.sun.jersey.api.client.config.ClientConfig; +import com.sun.jersey.api.client.config.DefaultClientConfig; import com.sun.jersey.api.client.filter.ClientFilter; import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; @@ -68,14 +71,14 @@ public class TaskClient extends ClientBase { * Creates a default task client */ public TaskClient() { - super(); + this(new DefaultClientConfig(), new DefaultConductorClientConfiguration(), null); } /** * @param config REST Client configuration */ public TaskClient(ClientConfig config) { - super(config); + this(config, new DefaultConductorClientConfiguration(), null); } /** @@ -83,7 +86,7 @@ public TaskClient(ClientConfig config) { * @param handler Jersey client handler. Useful when plugging in various http client interaction modules (e.g. ribbon) */ public TaskClient(ClientConfig config, ClientHandler handler) { - super(config, handler); + this(config, new DefaultConductorClientConfiguration(), handler); } /** @@ -92,10 +95,7 @@ public TaskClient(ClientConfig config, ClientHandler handler) { * @param filters Chain of client side filters to be applied per request */ public TaskClient(ClientConfig config, ClientHandler handler, ClientFilter... filters) { - super(config, handler); - for (ClientFilter filter : filters) { - super.client.addFilter(filter); - } + this(config, new DefaultConductorClientConfiguration(), handler, filters); } /** @@ -229,21 +229,25 @@ public Task getPendingTaskForWorkflow(String workflowId, String taskReferenceNam */ public void updateTask(TaskResult taskResult, String taskType) { Preconditions.checkNotNull(taskResult, "Task result cannot be null"); - Preconditions.checkArgument(StringUtils.isBlank(taskResult.getExternalPayloadStoragePath()), "External Storage Path must be blank"); + Preconditions.checkArgument(StringUtils.isBlank(taskResult.getExternalOutputPayloadStoragePath()), "External Storage Path must not be set"); try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream()) { objectMapper.writeValue(byteArrayOutputStream, taskResult.getOutputData()); byte[] taskOutputBytes = byteArrayOutputStream.toByteArray(); long taskResultSize = taskOutputBytes.length; WorkflowTaskMetrics.recordTaskResultPayloadSize(taskType, taskResultSize); - if (taskResultSize > conductorClientConfiguration.getTaskOutputPayloadThresholdKB() * 1024) { - if (conductorClientConfiguration.isExternalPayloadStorageEnabled()) { - String externalStoragePath = useExternalPayloadStorage("tasks", taskOutputBytes, taskResultSize); - taskResult.setExternalPayloadStoragePath(externalStoragePath); + + long payloadSizeThreshold = conductorClientConfiguration.getTaskOutputPayloadThresholdKB() * 1024; + if (taskResultSize > payloadSizeThreshold) { + if (!conductorClientConfiguration.isExternalPayloadStorageEnabled() + || taskResultSize > (conductorClientConfiguration.getTaskOutputMaxPayloadThresholdKB() * 1024)) { + taskResult.setReasonForIncompletion(String.format("The TaskResult payload size: %d is greater than the permissible %d MB", taskResultSize, payloadSizeThreshold)); + taskResult.setStatus(TaskResult.Status.FAILED_WITH_TERMINAL_ERROR); taskResult.setOutputData(null); } else { - taskResult.setReasonForIncompletion(String.format("The TaskResult payload size: %d is greater than the permissible 3MB", taskResultSize)); - taskResult.setStatus(TaskResult.Status.FAILED_WITH_TERMINAL_ERROR); + WorkflowTaskMetrics.incrementExternalPayloadUsedCount(taskType, ExternalPayloadStorage.Operation.WRITE.name(), ExternalPayloadStorage.PayloadType.TASK_OUTPUT.name()); + String externalStoragePath = uploadToExternalPayloadStorage(ExternalPayloadStorage.PayloadType.TASK_OUTPUT, taskOutputBytes, taskResultSize); + taskResult.setExternalOutputPayloadStoragePath(externalStoragePath); taskResult.setOutputData(null); } } @@ -286,7 +290,7 @@ public void log(String taskId, String logMessage) { */ public void logMessageForTask(String taskId, String logMessage) { Preconditions.checkArgument(StringUtils.isNotBlank(taskId), "Task id cannot be blank"); - postForEntity("tasks/" + taskId + "/log", logMessage); + postForEntityWithRequestOnly("tasks/" + taskId + "/log", logMessage); } /** @@ -446,9 +450,4 @@ public void registerTaskDefs(List taskDefs) { Preconditions.checkNotNull(taskDefs, "Task defs cannot be null"); postForEntity("metadata/taskdefs", taskDefs); } - - public String getExternalPayloadUri(String taskId) { - Preconditions.checkArgument(StringUtils.isNotBlank(taskId), "task id cannot be blank"); - return getForEntity("tasks/externalpayloaduri/{taskId}", null, String.class, taskId); - } } diff --git a/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java b/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java index 3abb995b6d..648f32aa59 100644 --- a/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java +++ b/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java @@ -17,6 +17,7 @@ import com.google.common.base.Preconditions; import com.netflix.conductor.client.config.ConductorClientConfiguration; +import com.netflix.conductor.client.config.DefaultConductorClientConfiguration; import com.netflix.conductor.client.exceptions.ConductorClientException; import com.netflix.conductor.client.task.WorkflowTaskMetrics; import com.netflix.conductor.common.metadata.workflow.RerunWorkflowRequest; @@ -25,9 +26,11 @@ import com.netflix.conductor.common.run.SearchResult; import com.netflix.conductor.common.run.Workflow; import com.netflix.conductor.common.run.WorkflowSummary; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.sun.jersey.api.client.ClientHandler; import com.sun.jersey.api.client.GenericType; import com.sun.jersey.api.client.config.ClientConfig; +import com.sun.jersey.api.client.config.DefaultClientConfig; import com.sun.jersey.api.client.filter.ClientFilter; import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; @@ -56,14 +59,14 @@ public class WorkflowClient extends ClientBase { * Creates a default task client */ public WorkflowClient() { - super(); + this(new DefaultClientConfig(), new DefaultConductorClientConfiguration(), null); } /** * @param config REST Client configuration */ public WorkflowClient(ClientConfig config) { - super(config); + this(config, new DefaultConductorClientConfiguration(), null); } /** @@ -71,7 +74,7 @@ public WorkflowClient(ClientConfig config) { * @param handler Jersey client handler. Useful when plugging in various http client interaction modules (e.g. ribbon) */ public WorkflowClient(ClientConfig config, ClientHandler handler) { - super(config, handler); + this(config, new DefaultConductorClientConfiguration(), handler); } /** @@ -80,10 +83,7 @@ public WorkflowClient(ClientConfig config, ClientHandler handler) { * @param filters Chain of client side filters to be applied per request */ public WorkflowClient(ClientConfig config, ClientHandler handler, ClientFilter... filters) { - super(config, handler); - for (ClientFilter filter : filters) { - super.client.addFilter(filter); - } + this(config, new DefaultConductorClientConfiguration(), handler, filters); } /** @@ -142,7 +142,6 @@ public WorkflowDef getWorkflowDef(String name, Integer version) { * @param correlationId the correlation id * @param input the input to set in the workflow * @return the id of the workflow instance that can be used for tracking - * * @deprecated This API is deprecated and will be removed in the next version * use {@link #startWorkflow(StartWorkflowRequest)} instead */ @@ -166,7 +165,7 @@ public String startWorkflow(String name, Integer version, String correlationId, public String startWorkflow(StartWorkflowRequest startWorkflowRequest) { Preconditions.checkNotNull(startWorkflowRequest, "StartWorkflowRequest cannot be null"); Preconditions.checkArgument(StringUtils.isNotBlank(startWorkflowRequest.getName()), "Workflow name cannot be null or empty"); - Preconditions.checkArgument(StringUtils.isBlank(startWorkflowRequest.getExternalStoragePath()), "External Storage Path must be blank"); + Preconditions.checkArgument(StringUtils.isBlank(startWorkflowRequest.getExternalInputPayloadStoragePath()), "External Storage Path must not be set"); String version = startWorkflowRequest.getVersion() != null ? startWorkflowRequest.getVersion().toString() : "latest"; try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream()) { @@ -176,8 +175,9 @@ public String startWorkflow(StartWorkflowRequest startWorkflowRequest) { WorkflowTaskMetrics.recordWorkflowInputPayloadSize(startWorkflowRequest.getName(), version, workflowInputSize); if (workflowInputSize > conductorClientConfiguration.getWorkflowInputPayloadThresholdKB() * 1024) { if (conductorClientConfiguration.isExternalPayloadStorageEnabled()) { - String externalStoragePath = useExternalPayloadStorage("workflow", workflowInputBytes, workflowInputSize); - startWorkflowRequest.setExternalStoragePath(externalStoragePath); + WorkflowTaskMetrics.incrementExternalPayloadUsedCount(startWorkflowRequest.getName(), ExternalPayloadStorage.Operation.WRITE.name(), ExternalPayloadStorage.PayloadType.WORKFLOW_INPUT.name()); + String externalStoragePath = uploadToExternalPayloadStorage(ExternalPayloadStorage.PayloadType.WORKFLOW_INPUT, workflowInputBytes, workflowInputSize); + startWorkflowRequest.setExternalInputPayloadStoragePath(externalStoragePath); startWorkflowRequest.setInput(null); } else { String errorMsg = String.format("Input payload larger than the allowed threshold of: %d KB", conductorClientConfiguration.getWorkflowInputPayloadThresholdKB()); @@ -341,7 +341,7 @@ public String rerunWorkflow(String workflowId, RerunWorkflowRequest rerunWorkflo */ public void restart(String workflowId) { Preconditions.checkArgument(StringUtils.isNotBlank(workflowId), "workflow id cannot be blank"); - postForEntity1("workflow/{workflowId}/restart", workflowId); + postForEntityWithUriVariablesOnly("workflow/{workflowId}/restart", workflowId); } /** @@ -351,7 +351,7 @@ public void restart(String workflowId) { */ public void retryLastFailedTask(String workflowId) { Preconditions.checkArgument(StringUtils.isNotBlank(workflowId), "workflow id cannot be blank"); - postForEntity1("workflow/{workflowId}/retry", workflowId); + postForEntityWithUriVariablesOnly("workflow/{workflowId}/retry", workflowId); } /** @@ -361,7 +361,7 @@ public void retryLastFailedTask(String workflowId) { */ public void resetCallbacksForInProgressTasks(String workflowId) { Preconditions.checkArgument(StringUtils.isNotBlank(workflowId), "workflow id cannot be blank"); - postForEntity1("workflow/{workflowId}/resetcallbacks", workflowId); + postForEntityWithUriVariablesOnly("workflow/{workflowId}/resetcallbacks", workflowId); } /** @@ -399,9 +399,4 @@ public SearchResult search(Integer start, Integer size, String Object[] params = new Object[]{"start", start, "size", size, "sort", sort, "freeText", freeText, "query", query}; return getForEntity("workflow/search", params, searchResultWorkflowSummary); } - - public String getExternalPayloadUri(String workflowId) { - Preconditions.checkArgument(StringUtils.isNotBlank(workflowId), "workflow id cannot be blank"); - return getForEntity("workflow/externalpayloaduri/{workflowId}", null, String.class, workflowId); - } } diff --git a/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java b/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java index 019bde9d3d..6e2dcfd47c 100644 --- a/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java +++ b/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java @@ -379,6 +379,7 @@ private void execute(Worker worker, Task task) { TaskResult result = null; try { logger.debug("Executing task {} in worker {} at {}", task, worker.getClass().getSimpleName(), worker.getIdentity()); + // TODO result = worker.execute(task); result.setWorkflowInstanceId(task.getWorkflowInstanceId()); result.setTaskId(task.getTaskId()); diff --git a/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskMetrics.java b/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskMetrics.java index 90ce8d7195..39961d5afa 100644 --- a/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskMetrics.java +++ b/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskMetrics.java @@ -45,6 +45,9 @@ public class WorkflowTaskMetrics { private static final String WORFLOW_TYPE = "workflowType"; private static final String WORKFLOW_VERSION = "version"; private static final String EXCEPTION = "exception"; + private static final String NAME = "name"; + private static final String OPERATION = "operation"; + private static final String PAYLOAD_TYPE = "payload_type"; private static final String TASK_EXECUTION_QUEUE_FULL = "task_execution_queue_full"; private static final String TASK_POLL_ERROR = "task_poll_error"; @@ -58,7 +61,7 @@ public class WorkflowTaskMetrics { private static final String TASK_POLL_TIME = "task_poll_time"; private static final String TASK_RESULT_SIZE = "task_result_size"; private static final String WORKFLOW_INPUT_SIZE = "workflow_input_size"; - + private static final String EXTERNAL_PAYLOAD_USED = "external_payload_used"; private static Registry registry = Spectator.globalRegistry(); @@ -165,4 +168,8 @@ public static void incrementTaskPollCount(String taskType, int taskCount) { public static void recordWorkflowInputPayloadSize(String workflowType, String version, long payloadSize) { getGauge(WORKFLOW_INPUT_SIZE, WORFLOW_TYPE, workflowType, WORKFLOW_VERSION, version).getAndSet(payloadSize); } + + public static void incrementExternalPayloadUsedCount(String name, String operation, String payloadType) { + incrementCount(EXTERNAL_PAYLOAD_USED, NAME, name, OPERATION, operation, PAYLOAD_TYPE, payloadType); + } } diff --git a/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java b/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java index b4c9fe3187..7bdd0b60af 100644 --- a/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java +++ b/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java @@ -132,8 +132,11 @@ public boolean isRetriable() { private int rateLimitFrequencyInSeconds; - public Task() { + private String externalInputPayloadStoragePath; + + private String externalOutputPayloadStoragePath; + public Task() { } /** @@ -560,8 +563,35 @@ public void setRateLimitFrequencyInSeconds(int rateLimitFrequencyInSeconds) { this.rateLimitFrequencyInSeconds = rateLimitFrequencyInSeconds; } - public Task copy() { + /** + * @return the external storage path for the task input payload + */ + public String getExternalInputPayloadStoragePath() { + return externalInputPayloadStoragePath; + } + + /** + * @param externalInputPayloadStoragePath the external storage path where the task input payload is stored + */ + public void setExternalInputPayloadStoragePath(String externalInputPayloadStoragePath) { + this.externalInputPayloadStoragePath = externalInputPayloadStoragePath; + } + + /** + * @return the external storage path for the task output payload + */ + public String getExternalOutputPayloadStoragePath() { + return externalOutputPayloadStoragePath; + } + + /** + * @param externalOutputPayloadStoragePath the external storage path where the task output payload is stored + */ + public void setExternalOutputPayloadStoragePath(String externalOutputPayloadStoragePath) { + this.externalOutputPayloadStoragePath = externalOutputPayloadStoragePath; + } + public Task copy() { Task copy = new Task(); copy.setCallbackAfterSeconds(callbackAfterSeconds); copy.setCallbackFromWorker(callbackFromWorker); @@ -584,6 +614,7 @@ public Task copy() { copy.setDomain(domain); copy.setRateLimitPerFrequency(rateLimitPerFrequency); copy.setRateLimitFrequencyInSeconds(rateLimitFrequencyInSeconds); + copy.setExternalInputPayloadStoragePath(externalInputPayloadStoragePath); return copy; } @@ -607,11 +638,11 @@ public String toString() { ", startDelayInSeconds=" + startDelayInSeconds + ", retriedTaskId='" + retriedTaskId + '\'' + ", retried=" + retried + + ", executed=" + executed + ", callbackFromWorker=" + callbackFromWorker + - ", rateLimitFrequencyInSeconds=" + rateLimitFrequencyInSeconds + - ", rateLimitPerFrequency=" + rateLimitPerFrequency + ", responseTimeoutSeconds=" + responseTimeoutSeconds + ", workflowInstanceId='" + workflowInstanceId + '\'' + + ", workflowType='" + workflowType + '\'' + ", taskId='" + taskId + '\'' + ", reasonForIncompletion='" + reasonForIncompletion + '\'' + ", callbackAfterSeconds=" + callbackAfterSeconds + @@ -619,6 +650,10 @@ public String toString() { ", outputData=" + outputData + ", workflowTask=" + workflowTask + ", domain='" + domain + '\'' + + ", rateLimitPerFrequency=" + rateLimitPerFrequency + + ", rateLimitFrequencyInSeconds=" + rateLimitFrequencyInSeconds + + ", externalInputPayloadStoragePath='" + externalInputPayloadStoragePath + '\'' + + ", externalOutputPayloadStoragePath='" + externalOutputPayloadStoragePath + '\'' + '}'; } } diff --git a/common/src/main/java/com/netflix/conductor/common/metadata/tasks/TaskResult.java b/common/src/main/java/com/netflix/conductor/common/metadata/tasks/TaskResult.java index 2fb5e7bb60..aecc801e8e 100644 --- a/common/src/main/java/com/netflix/conductor/common/metadata/tasks/TaskResult.java +++ b/common/src/main/java/com/netflix/conductor/common/metadata/tasks/TaskResult.java @@ -48,7 +48,7 @@ public enum Status { private List logs = new CopyOnWriteArrayList<>(); - private String externalPayloadStoragePath; + private String externalOutputPayloadStoragePath; public TaskResult(Task task) { this.workflowInstanceId = task.getWorkflowInstanceId(); @@ -58,6 +58,7 @@ public TaskResult(Task task) { this.status = Status.valueOf(task.getStatus().name()); this.workerId = task.getWorkerId(); this.outputData = task.getOutputData(); + this.externalOutputPayloadStoragePath = task.getExternalOutputPayloadStoragePath(); } public TaskResult() { @@ -195,16 +196,16 @@ public TaskResult log(String log) { * * @return the path where the task output is stored in external storage */ - public String getExternalPayloadStoragePath() { - return externalPayloadStoragePath; + public String getExternalOutputPayloadStoragePath() { + return externalOutputPayloadStoragePath; } /** * - * @param externalPayloadStoragePath path in the external storage where the task output is stored + * @param externalOutputPayloadStoragePath path in the external storage where the task output is stored */ - public void setExternalPayloadStoragePath(String externalPayloadStoragePath) { - this.externalPayloadStoragePath = externalPayloadStoragePath; + public void setExternalOutputPayloadStoragePath(String externalOutputPayloadStoragePath) { + this.externalOutputPayloadStoragePath = externalOutputPayloadStoragePath; } @Override @@ -218,7 +219,7 @@ public String toString() { ", status=" + status + ", outputData=" + outputData + ", logs=" + logs + - ", externalPayloadStoragePath='" + externalPayloadStoragePath + '\'' + + ", externalOutputPayloadStoragePath='" + externalOutputPayloadStoragePath + '\'' + '}'; } diff --git a/common/src/main/java/com/netflix/conductor/common/metadata/workflow/StartWorkflowRequest.java b/common/src/main/java/com/netflix/conductor/common/metadata/workflow/StartWorkflowRequest.java index 9db60e85cd..9157aa3127 100644 --- a/common/src/main/java/com/netflix/conductor/common/metadata/workflow/StartWorkflowRequest.java +++ b/common/src/main/java/com/netflix/conductor/common/metadata/workflow/StartWorkflowRequest.java @@ -7,7 +7,7 @@ public class StartWorkflowRequest { private String name; private Integer version; private String correlationId; - private String externalStoragePath; + private String externalInputPayloadStoragePath; private Map input = new HashMap<>(); private Map taskToDomain = new HashMap<>(); @@ -44,12 +44,12 @@ public StartWorkflowRequest withCorrelationId(String correlationId) { return this; } - public String getExternalStoragePath() { - return externalStoragePath; + public String getExternalInputPayloadStoragePath() { + return externalInputPayloadStoragePath; } - public void setExternalStoragePath(String externalStoragePath) { - this.externalStoragePath = externalStoragePath; + public void setExternalInputPayloadStoragePath(String externalInputPayloadStoragePath) { + this.externalInputPayloadStoragePath = externalInputPayloadStoragePath; } public Map getInput() { diff --git a/common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java b/common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java index 85bdc4b4ce..81e0f935a0 100644 --- a/common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java +++ b/common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java @@ -16,10 +16,18 @@ package com.netflix.conductor.common.run; +/** + * Describes the location where the JSON payload is stored in external storage. + *

  • + * The location is described using the following fields: + * + * + *
  • + */ public class ExternalStorageLocation { - public String uri; - public String path; + private String uri; + private String path; public String getUri() { return uri; diff --git a/common/src/main/java/com/netflix/conductor/common/run/Workflow.java b/common/src/main/java/com/netflix/conductor/common/run/Workflow.java index 783954452b..c65214eb35 100644 --- a/common/src/main/java/com/netflix/conductor/common/run/Workflow.java +++ b/common/src/main/java/com/netflix/conductor/common/run/Workflow.java @@ -15,76 +15,81 @@ */ package com.netflix.conductor.common.run; +import com.netflix.conductor.common.metadata.Auditable; +import com.netflix.conductor.common.metadata.tasks.Task; + import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; - -import com.netflix.conductor.common.metadata.Auditable; -import com.netflix.conductor.common.metadata.tasks.Task; +import java.util.stream.Collectors; public class Workflow extends Auditable{ - + public enum WorkflowStatus { RUNNING(false, false), COMPLETED(true, true), FAILED(true, false), TIMED_OUT(true, false), TERMINATED(true, false), PAUSED(false, true); - + private boolean terminal; - + private boolean successful; - + WorkflowStatus(boolean terminal, boolean successful){ this.terminal = terminal; this.successful = successful; } - + public boolean isTerminal(){ return terminal; } - + public boolean isSuccessful(){ return successful; } } - + private WorkflowStatus status = WorkflowStatus.RUNNING; - + private long endTime; private String workflowId; - + private String parentWorkflowId; private String parentWorkflowTaskId; private List tasks = new LinkedList<>(); - + private Map input = new HashMap<>(); - + private Map output = new HashMap<>();; - + private String workflowType; - + private int version; - + private String correlationId; - + private String reRunFromWorkflowId; - + private String reasonForIncompletion; - + private int schemaVersion; - + private String event; private Map taskToDomain = new HashMap<>(); private Set failedReferenceTaskNames = new HashSet<>(); + private String externalInputPayloadStoragePath; + + private String externalOutputPayloadStoragePath; + public Workflow(){ - + } /** * @return the status @@ -152,7 +157,7 @@ public List getTasks() { public void setTasks(List tasks) { this.tasks = tasks; } - + /** * @return the input */ @@ -189,40 +194,40 @@ public Map getOutput() { public void setOutput(Map output) { this.output = output; } - + /** - * + * * @return The correlation id used when starting the workflow */ public String getCorrelationId() { return correlationId; } - + /** - * + * * @param correlationId the correlation id */ public void setCorrelationId(String correlationId) { this.correlationId = correlationId; } - + /** - * + * * @return Workflow Type / Definition */ public String getWorkflowType() { return workflowType; } - + /** - * + * * @param workflowType Workflow type */ public void setWorkflowType(String workflowType) { this.workflowType = workflowType; } - - + + /** * @return the version */ @@ -235,23 +240,23 @@ public int getVersion() { public void setVersion(int version) { this.version = version; } - + public String getReRunFromWorkflowId() { return reRunFromWorkflowId; } - + public void setReRunFromWorkflowId(String reRunFromWorkflowId) { this.reRunFromWorkflowId = reRunFromWorkflowId; } - + public String getReasonForIncompletion() { return reasonForIncompletion; } - + public void setReasonForIncompletion(String reasonForIncompletion) { this.reasonForIncompletion = reasonForIncompletion; } - + /** * @return the parentWorkflowId */ @@ -264,7 +269,7 @@ public String getParentWorkflowId() { public void setParentWorkflowId(String parentWorkflowId) { this.parentWorkflowId = parentWorkflowId; } - + /** * @return the parentWorkflowTaskId */ @@ -289,17 +294,17 @@ public int getSchemaVersion() { public void setSchemaVersion(int schemaVersion) { this.schemaVersion = schemaVersion; } - + /** - * + * * @return Name of the event that started the workflow */ public String getEvent() { return event; } - + /** - * + * * @param event Name of the event that started the workflow */ public void setEvent(String event) { @@ -314,16 +319,39 @@ public void setFailedReferenceTaskNames(Set failedReferenceTaskNames) { this.failedReferenceTaskNames = failedReferenceTaskNames; } - @Override - public String toString() { - return workflowType + "." + version + "/" + workflowId + "." + status; + /** + * @return the external storage path of the workflow input payload + */ + public String getExternalInputPayloadStoragePath() { + return externalInputPayloadStoragePath; + } + + /** + * @param externalInputPayloadStoragePath the external storage path where the workflow input payload is stored + */ + public void setExternalInputPayloadStoragePath(String externalInputPayloadStoragePath) { + this.externalInputPayloadStoragePath = externalInputPayloadStoragePath; } - + + /** + * @return the external storage path of the workflow output payload + */ + public String getExternalOutputPayloadStoragePath() { + return externalOutputPayloadStoragePath; + } + + /** + * @param externalOutputPayloadStoragePath the external storage path where the workflow output payload is stored + */ + public void setExternalOutputPayloadStoragePath(String externalOutputPayloadStoragePath) { + this.externalOutputPayloadStoragePath = externalOutputPayloadStoragePath; + } + public Task getTaskByRefName(String refName) { if (refName == null) { throw new RuntimeException("refName passed is null. Check the workflow execution. For dynamic tasks, make sure referenceTaskName is set to a not null value"); } - LinkedList found = new LinkedList(); + LinkedList found = new LinkedList<>(); for (Task t : tasks) { if (t.getReferenceTaskName() == null) { throw new RuntimeException("Task " + t.getTaskDefName() + ", seq=" + t.getSeq() + " does not have reference name specified."); @@ -337,5 +365,41 @@ public Task getTaskByRefName(String refName) { } return found.getLast(); } - + + /** + * @return a deep copy of the workflow instance + * Note: This does not copy the following fields: + *
      + *
    • endTime
    • + *
    • taskToDomain
    • + *
    • failedReferenceTaskNames
    • + *
    • externalInputPayloadStoragePath
    • + *
    • externalOutputPayloadStoragePath
    • + *
    + */ + public Workflow copy() { + Workflow copy = new Workflow(); + copy.setInput(input); + copy.setOutput(output); + copy.setStatus(status); + copy.setWorkflowId(workflowId); + copy.setParentWorkflowId(parentWorkflowId); + copy.setParentWorkflowTaskId(parentWorkflowTaskId); + copy.setReRunFromWorkflowId(reRunFromWorkflowId); + copy.setWorkflowType(workflowType); + copy.setVersion(version); + copy.setCorrelationId(correlationId); + copy.setEvent(event); + copy.setReasonForIncompletion(reasonForIncompletion); + copy.setSchemaVersion(schemaVersion); + copy.setTasks(tasks.stream() + .map(Task::copy) + .collect(Collectors.toList())); + return copy; + } + + @Override + public String toString() { + return workflowType + "." + version + "/" + workflowId + "." + status; + } } \ No newline at end of file diff --git a/common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java b/common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java index b3a867023b..7e17e12b3b 100644 --- a/common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java +++ b/common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java @@ -36,7 +36,7 @@ enum PayloadType {WORKFLOW_INPUT, WORKFLOW_OUTPUT, TASK_INPUT, TASK_OUTPUT} * @param payloadType the {@link PayloadType} that is being accessed at the uri * @return a {@link ExternalStorageLocation} object which contains the uri and the path for the json payload */ - ExternalStorageLocation getExternalUri(Operation operation, PayloadType payloadType); + ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType); /** * Upload a json payload to the specified external storage location. diff --git a/common/src/test/java/com/netflix/conductor/common/workflow/TestWorkflowDef.java b/common/src/test/java/com/netflix/conductor/common/workflow/TestWorkflowDef.java index 21903f39b6..7e77e5f01c 100644 --- a/common/src/test/java/com/netflix/conductor/common/workflow/TestWorkflowDef.java +++ b/common/src/test/java/com/netflix/conductor/common/workflow/TestWorkflowDef.java @@ -18,20 +18,20 @@ */ package com.netflix.conductor.common.workflow; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; +import com.netflix.conductor.common.metadata.workflow.WorkflowDef; +import com.netflix.conductor.common.metadata.workflow.WorkflowTask; +import com.netflix.conductor.common.metadata.workflow.WorkflowTask.Type; +import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import org.junit.Test; - -import com.netflix.conductor.common.metadata.workflow.WorkflowDef; -import com.netflix.conductor.common.metadata.workflow.WorkflowTask; -import com.netflix.conductor.common.metadata.workflow.WorkflowTask.Type; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; /** * @author Viren @@ -47,7 +47,7 @@ private WorkflowTask createTask(int c){ } @Test - public void test() throws Exception { + public void test() { String COND_TASK_WF = "COND_TASK_WF"; List wfts = new ArrayList(10); @@ -78,7 +78,7 @@ public void test() throws Exception { caseTask.setTaskReferenceName("case"); Map> dc = new HashMap<>(); dc.put("c1", Arrays.asList(wfts.get(0), subCaseTask, wfts.get(1))); - dc.put("c2", Arrays.asList(wfts.get(3))); + dc.put("c2", Collections.singletonList(wfts.get(3))); caseTask.setDecisionCases(dc); WorkflowTask finalTask = new WorkflowTask(); diff --git a/contribs/src/main/java/com/netflix/conductor/contribs/http/HttpTask.java b/contribs/src/main/java/com/netflix/conductor/contribs/http/HttpTask.java index 9333f0baad..876c7292d1 100644 --- a/contribs/src/main/java/com/netflix/conductor/contribs/http/HttpTask.java +++ b/contribs/src/main/java/com/netflix/conductor/contribs/http/HttpTask.java @@ -160,9 +160,7 @@ protected HttpResponse httpCall(Input input) throws Exception { if(input.body != null) { builder.entity(input.body); } - input.headers.entrySet().forEach(e -> { - builder.header(e.getKey(), e.getValue()); - }); + input.headers.forEach(builder::header); HttpResponse response = new HttpResponse(); try { diff --git a/contribs/src/test/java/com/netflix/conductor/contribs/http/TestHttpTask.java b/contribs/src/test/java/com/netflix/conductor/contribs/http/TestHttpTask.java index 89eaa32ccf..e68f93f15d 100644 --- a/contribs/src/test/java/com/netflix/conductor/contribs/http/TestHttpTask.java +++ b/contribs/src/test/java/com/netflix/conductor/contribs/http/TestHttpTask.java @@ -26,6 +26,7 @@ import com.netflix.conductor.common.metadata.workflow.WorkflowTask; import com.netflix.conductor.common.metadata.workflow.WorkflowTask.Type; import com.netflix.conductor.common.run.Workflow; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.contribs.http.HttpTask.Input; import com.netflix.conductor.core.config.Configuration; import com.netflix.conductor.core.execution.DeciderService; @@ -53,7 +54,6 @@ import org.junit.BeforeClass; import org.junit.Test; -import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import java.io.BufferedReader; @@ -131,7 +131,7 @@ public void setup() { } @Test - public void testPost() throws Exception { + public void testPost() { Task task = new Task(); Input input = new Input(); @@ -158,7 +158,7 @@ public void testPost() throws Exception { @Test - public void testPostNoContent() throws Exception { + public void testPostNoContent() { Task task = new Task(); Input input = new Input(); @@ -179,7 +179,7 @@ public void testPostNoContent() throws Exception { } @Test - public void testFailure() throws Exception { + public void testFailure() { Task task = new Task(); Input input = new Input(); @@ -199,7 +199,7 @@ public void testFailure() throws Exception { } @Test - public void testTextGET() throws Exception { + public void testTextGET() { Task task = new Task(); Input input = new Input(); @@ -215,7 +215,7 @@ public void testTextGET() throws Exception { } @Test - public void testNumberGET() throws Exception { + public void testNumberGET() { Task task = new Task(); Input input = new Input(); @@ -250,7 +250,7 @@ public void testJsonGET() throws Exception { } @Test - public void testExecute() throws Exception { + public void testExecute() { Task task = new Task(); Input input = new Input(); @@ -261,11 +261,10 @@ public void testExecute() throws Exception { task.setScheduledTime(0); boolean executed = httpTask.execute(workflow, task, executor); assertFalse(executed); - } @Test - public void testOptional() throws Exception { + public void testOptional() { Task task = new Task(); Input input = new Input(); input.setUri("http://localhost:7009/failure"); @@ -297,7 +296,8 @@ public void testOptional() throws Exception { def.getTasks().add(wft); MetadataDAO metadataDAO = mock(MetadataDAO.class); QueueDAO queueDAO = mock(QueueDAO.class); - ParametersUtils parametersUtils = new ParametersUtils(); + ExternalPayloadStorage externalPayloadStorage = mock(ExternalPayloadStorage.class); + ParametersUtils parametersUtils = mock(ParametersUtils.class); Map taskMappers = new HashMap<>(); taskMappers.put("DECISION", new DecisionTaskMapper()); taskMappers.put("DYNAMIC", new DynamicTaskMapper(parametersUtils, metadataDAO)); @@ -309,14 +309,14 @@ public void testOptional() throws Exception { taskMappers.put("SUB_WORKFLOW", new SubWorkflowTaskMapper(parametersUtils, metadataDAO)); taskMappers.put("EVENT", new EventTaskMapper(parametersUtils)); taskMappers.put("WAIT", new WaitTaskMapper(parametersUtils)); - new DeciderService(metadataDAO, queueDAO, taskMappers).decide(workflow, def); + new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorage, taskMappers).decide(workflow, def); System.out.println(workflow.getTasks()); System.out.println(workflow.getStatus()); } @Test - public void testOAuth() throws Exception { + public void testOAuth() { Task task = new Task(); Input input = new Input(); input.setUri("http://localhost:7009/oauth"); @@ -346,7 +346,7 @@ private static class EchoHandler extends AbstractHandler { @Override public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) - throws IOException, ServletException { + throws IOException { if(request.getMethod().equals("GET") && request.getRequestURI().equals("/text")) { PrintWriter writer = response.getWriter(); writer.print(TEXT_RESPONSE); diff --git a/core/src/main/java/com/netflix/conductor/core/config/CoreModule.java b/core/src/main/java/com/netflix/conductor/core/config/CoreModule.java index 7c504e1450..30297331d7 100644 --- a/core/src/main/java/com/netflix/conductor/core/config/CoreModule.java +++ b/core/src/main/java/com/netflix/conductor/core/config/CoreModule.java @@ -1,4 +1,4 @@ -/** +/* * Copyright 2016 Netflix, Inc. *

    * Licensed under the Apache License, Version 2.0 (the "License"); @@ -29,7 +29,6 @@ import com.netflix.conductor.core.events.ActionProcessor; import com.netflix.conductor.core.events.EventProcessor; import com.netflix.conductor.core.events.EventQueueProvider; -import com.netflix.conductor.core.events.EventQueues; import com.netflix.conductor.core.events.queue.dyno.DynoEventQueueProvider; import com.netflix.conductor.core.execution.ParametersUtils; import com.netflix.conductor.core.execution.mapper.DecisionTaskMapper; @@ -50,7 +49,6 @@ import com.netflix.conductor.dao.MetadataDAO; import com.netflix.conductor.dao.QueueDAO; - /** * @author Viren */ @@ -59,7 +57,6 @@ public class CoreModule extends AbstractModule { @Override protected void configure() { install(MultibindingsScanner.asModule()); - requestStaticInjection(EventQueues.class); bind(ActionProcessor.class).asEagerSingleton(); bind(EventProcessor.class).asEagerSingleton(); bind(SystemTaskWorkerCoordinator.class).asEagerSingleton(); @@ -74,7 +71,6 @@ public ParametersUtils getParameterUtils() { return new ParametersUtils(); } - @ProvidesIntoMap @StringMapKey("conductor") @Singleton @@ -108,7 +104,6 @@ public TaskMapper getJoinTaskMapper() { } - @ProvidesIntoMap @StringMapKey("FORK_JOIN_DYNAMIC") @Singleton @@ -164,6 +159,4 @@ public TaskMapper getUserDefinedTaskMapper(ParametersUtils parametersUtils, Meta public TaskMapper getSimpleTaskMapper(ParametersUtils parametersUtils, MetadataDAO metadataDAO) { return new SimpleTaskMapper(parametersUtils, metadataDAO); } - - } diff --git a/core/src/main/java/com/netflix/conductor/core/events/ActionProcessor.java b/core/src/main/java/com/netflix/conductor/core/events/ActionProcessor.java index c9ba401fe8..7aaeffdb17 100644 --- a/core/src/main/java/com/netflix/conductor/core/events/ActionProcessor.java +++ b/core/src/main/java/com/netflix/conductor/core/events/ActionProcessor.java @@ -50,13 +50,15 @@ public class ActionProcessor { private final WorkflowExecutor executor; private final MetadataService metadataService; - private final ParametersUtils parametersUtils = new ParametersUtils(); + private final ParametersUtils parametersUtils; + private final JsonUtils jsonUtils = new JsonUtils(); @Inject - public ActionProcessor(WorkflowExecutor executor, MetadataService metadataService) { + public ActionProcessor(WorkflowExecutor executor, MetadataService metadataService, ParametersUtils parametersUtils) { this.executor = executor; this.metadataService = metadataService; + this.parametersUtils = parametersUtils; } public Map execute(Action action, Object payloadObject, String event, String messageId) { diff --git a/core/src/main/java/com/netflix/conductor/core/events/EventProcessor.java b/core/src/main/java/com/netflix/conductor/core/events/EventProcessor.java index 155fa452e0..6765db179a 100644 --- a/core/src/main/java/com/netflix/conductor/core/events/EventProcessor.java +++ b/core/src/main/java/com/netflix/conductor/core/events/EventProcessor.java @@ -70,6 +70,7 @@ public class EventProcessor { private final MetadataService metadataService; private final ExecutionService executionService; private final ActionProcessor actionProcessor; + private final EventQueues eventQueues; private ExecutorService executorService; private final Map eventToQueueMap = new ConcurrentHashMap<>(); @@ -78,10 +79,11 @@ public class EventProcessor { @Inject public EventProcessor(ExecutionService executionService, MetadataService metadataService, - ActionProcessor actionProcessor, Configuration config) { + ActionProcessor actionProcessor, EventQueues eventQueues, Configuration config) { this.executionService = executionService; this.metadataService = metadataService; this.actionProcessor = actionProcessor; + this.eventQueues = eventQueues; int executorThreadCount = config.getIntProperty("workflow.event.processor.thread.count", 2); if (executorThreadCount > 0) { @@ -120,7 +122,7 @@ private void refresh() { List createdQueues = new LinkedList<>(); events.forEach(event -> eventToQueueMap.computeIfAbsent(event, s -> { - ObservableQueue q = EventQueues.getQueue(event); + ObservableQueue q = eventQueues.getQueue(event); createdQueues.add(q); return q; } diff --git a/core/src/main/java/com/netflix/conductor/core/events/EventQueues.java b/core/src/main/java/com/netflix/conductor/core/events/EventQueues.java index 9aabc40e5a..0c865fd97b 100644 --- a/core/src/main/java/com/netflix/conductor/core/events/EventQueues.java +++ b/core/src/main/java/com/netflix/conductor/core/events/EventQueues.java @@ -14,7 +14,7 @@ * limitations under the License. */ /** - * + * */ package com.netflix.conductor.core.events; @@ -25,34 +25,37 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.inject.Singleton; import java.util.List; import java.util.Map; import java.util.stream.Collectors; /** * @author Viren - * Static holders for internal event queues + * Holders for internal event queues */ +@Singleton public class EventQueues { - - private static Logger logger = LoggerFactory.getLogger(EventQueues.class); - private static ParametersUtils parametersUtils = new ParametersUtils(); + private static final Logger logger = LoggerFactory.getLogger(EventQueues.class); - @Inject - @Named("EventQueueProviders") - public static Map providers; //TODO this is a leaky abstraction, when the static injection is moved to singleton this will be fixed + private final ParametersUtils parametersUtils; - private EventQueues() { + private final Map providers; + + @Inject + public EventQueues(@Named("EventQueueProviders") Map providers, ParametersUtils parametersUtils) { + this.providers = providers; + this.parametersUtils = parametersUtils; } - public static List providers() { + public List getProviders() { return providers.values().stream() .map(p -> p.getClass().getName()) .collect(Collectors.toList()); } - public static ObservableQueue getQueue(String eventType) { + public ObservableQueue getQueue(String eventType) { String event = parametersUtils.replace(eventType).toString(); int index = event.indexOf(':'); if (index == -1) { diff --git a/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java b/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java index f1e9cf0a59..a38da0c00a 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java @@ -18,6 +18,8 @@ */ package com.netflix.conductor.core.execution; +import com.amazonaws.util.IOUtils; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.netflix.conductor.common.metadata.tasks.Task; import com.netflix.conductor.common.metadata.tasks.Task.Status; @@ -27,6 +29,7 @@ import com.netflix.conductor.common.metadata.workflow.WorkflowTask.Type; import com.netflix.conductor.common.run.Workflow; import com.netflix.conductor.common.run.Workflow.WorkflowStatus; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.core.execution.mapper.TaskMapper; import com.netflix.conductor.core.execution.mapper.TaskMapperContext; import com.netflix.conductor.core.utils.IDGenerator; @@ -34,11 +37,14 @@ import com.netflix.conductor.dao.MetadataDAO; import com.netflix.conductor.dao.QueueDAO; import com.netflix.conductor.metrics.Monitors; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.inject.Inject; import javax.inject.Named; +import java.io.IOException; +import java.io.InputStream; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; @@ -48,6 +54,7 @@ import java.util.Optional; import java.util.Set; import java.util.function.Function; +import java.util.function.Predicate; import java.util.stream.Collectors; import static com.netflix.conductor.common.metadata.tasks.Task.Status.COMPLETED_WITH_ERRORS; @@ -71,15 +78,24 @@ public class DeciderService { private final QueueDAO queueDAO; - private ParametersUtils parametersUtils = new ParametersUtils(); + private final ParametersUtils parametersUtils; private final Map taskMappers; + private final ExternalPayloadStorage externalPayloadStorage; + + private ObjectMapper objectMapper = new ObjectMapper(); + + @SuppressWarnings("ConstantConditions") + private final Predicate isNonPendingTask = task -> !task.isRetried() && !task.getStatus().equals(SKIPPED) && !task.isExecuted() || SystemTaskType.isBuiltIn(task.getTaskType()); + @Inject - public DeciderService(MetadataDAO metadataDAO, QueueDAO queueDAO, @Named("TaskMappers") Map taskMappers) { + public DeciderService(MetadataDAO metadataDAO, ParametersUtils parametersUtils, QueueDAO queueDAO, ExternalPayloadStorage externalPayloadStorage, @Named("TaskMappers") Map taskMappers) { this.metadataDAO = metadataDAO; this.queueDAO = queueDAO; + this.parametersUtils = parametersUtils; this.taskMappers = taskMappers; + this.externalPayloadStorage = externalPayloadStorage; } //QQ public method validation of the input params @@ -90,7 +106,7 @@ public DeciderOutcome decide(Workflow workflow, WorkflowDef workflowDef) throws final List tasks = workflow.getTasks(); //In case of a new workflow the list of executedTasks will also be empty List executedTasks = tasks.stream() - .filter(t -> !t.getStatus().equals(SKIPPED) && !t.getStatus().equals(READY_FOR_RERUN)) + .filter(t -> !t.getStatus().equals(SKIPPED) && !t.getStatus().equals(READY_FOR_RERUN) && !t.isExecuted()) .collect(Collectors.toList()); List tasksToBeScheduled = new LinkedList<>(); @@ -119,19 +135,19 @@ private DeciderOutcome decide(final WorkflowDef workflowDef, final Workflow work return outcome; } - // Filter the list of tasks and include only tasks that are not retried, + // Filter the list of tasks and include only tasks that are not retried, not executed, // marked to be skipped and not part of System tasks that is DECISION, FORK, JOIN // This list will be empty for a new workflow being started List pendingTasks = workflow.getTasks() .stream() - .filter(task -> (!task.isRetried() && !task.getStatus().equals(SKIPPED)) || SystemTaskType.isBuiltIn(task.getTaskType())) + .filter(isNonPendingTask) .collect(Collectors.toList()); // Get all the tasks that are ready to rerun or not marked to be skipped // This list will be empty for a new workflow Set executedTaskRefNames = workflow.getTasks() .stream() - .filter(task -> !task.getStatus().equals(SKIPPED) && !task.getStatus().equals(READY_FOR_RERUN)) + .filter(Task::isExecuted) .map(Task::getReferenceTaskName) .collect(Collectors.toSet()); @@ -153,7 +169,7 @@ private DeciderOutcome decide(final WorkflowDef workflowDef, final Workflow work checkForTimeout(taskDefinition, pendingTask); // If the task has not been updated for "responseTimeoutSeconds" then mark task as TIMED_OUT if (isResponseTimedOut(taskDefinition, pendingTask)) { - timeoutTask(pendingTask); + timeoutTask(taskDefinition, pendingTask); } } @@ -203,26 +219,26 @@ private DeciderOutcome decide(final WorkflowDef workflowDef, final Workflow work return outcome; } - private List startWorkflow(Workflow workflow, WorkflowDef def) throws TerminateWorkflowException { + private List startWorkflow(Workflow workflow, WorkflowDef workflowDef) throws TerminateWorkflowException { - logger.debug("Starting workflow " + def.getName() + "/" + workflow.getWorkflowId()); + logger.debug("Starting workflow " + workflowDef.getName() + "/" + workflow.getWorkflowId()); //The tasks will be empty in case of new workflow List tasks = workflow.getTasks(); // Check if the workflow is a re-run case or if it is a new workflow execution if (workflow.getReRunFromWorkflowId() == null || tasks.isEmpty()) { - if (def.getTasks().isEmpty()) { + if (workflowDef.getTasks().isEmpty()) { throw new TerminateWorkflowException("No tasks found to be executed", WorkflowStatus.COMPLETED); } - WorkflowTask taskToSchedule = def.getTasks().getFirst(); //Nothing is running yet - so schedule the first task + WorkflowTask taskToSchedule = workflowDef.getTasks().getFirst(); //Nothing is running yet - so schedule the first task //Loop until a non-skipped task is found while (isTaskSkipped(taskToSchedule, workflow)) { - taskToSchedule = def.getNextTask(taskToSchedule.getTaskReferenceName()); + taskToSchedule = workflowDef.getNextTask(taskToSchedule.getTaskReferenceName()); } //In case of a new workflow a the first non-skippable task will be scheduled - return getTasksToBeScheduled(def, workflow, taskToSchedule, 0); + return getTasksToBeScheduled(workflowDef, workflow, taskToSchedule, 0); } // Get the first task to schedule @@ -242,7 +258,6 @@ private List startWorkflow(Workflow workflow, WorkflowDef def) throws Term }); return Collections.singletonList(rerunFromTask); - } private void updateOutput(final WorkflowDef def, final Workflow workflow) { @@ -435,23 +450,44 @@ boolean isResponseTimedOut(TaskDef taskDefinition, Task task) { return true; } - private void timeoutTask(Task task) { - String reason = "responseTimeout: " + task.getResponseTimeoutSeconds() + " exceeded for the taskId: " + task.getTaskId() + " with Task Definition: " + task.getTaskDefName(); + private void timeoutTask(TaskDef taskDef, Task task) { + String reason = "responseTimeout: " + taskDef.getResponseTimeoutSeconds() + " exceeded for the taskId: " + task.getTaskId() + " with Task Definition: " + task.getTaskDefName(); logger.debug(reason); task.setStatus(TIMED_OUT); task.setReasonForIncompletion(reason); } - public List getTasksToBeScheduled(WorkflowDef def, Workflow workflow, + public List getTasksToBeScheduled(WorkflowDef workflowDef, Workflow workflow, WorkflowTask taskToSchedule, int retryCount) { - return getTasksToBeScheduled(def, workflow, taskToSchedule, retryCount, null); + return getTasksToBeScheduled(workflowDef, workflow, taskToSchedule, retryCount, null); } public List getTasksToBeScheduled(WorkflowDef workflowDefinition, Workflow workflowInstance, WorkflowTask taskToSchedule, int retryCount, String retriedTaskId) { + Workflow workflow = workflowInstance.copy(); + + if (StringUtils.isNotBlank(workflowInstance.getExternalInputPayloadStoragePath())) { + // download the workflow input from external storage here and plug it into the workflow + Map workflowInputParams = downloadFromExternalStorage(workflowInstance.getExternalInputPayloadStoragePath()); + workflow.setInput(workflowInputParams); + } + + workflow.getTasks().stream() + .filter(task -> StringUtils.isNotBlank(task.getExternalInputPayloadStoragePath()) || StringUtils.isNotBlank(task.getExternalOutputPayloadStoragePath())) + .forEach(task -> { + if (StringUtils.isNotBlank(task.getExternalOutputPayloadStoragePath())) { + task.setOutputData(downloadFromExternalStorage(task.getExternalOutputPayloadStoragePath())); + task.setExternalOutputPayloadStoragePath(null); + } + if (StringUtils.isNotBlank(task.getExternalInputPayloadStoragePath())) { + task.setInputData(downloadFromExternalStorage(task.getExternalInputPayloadStoragePath())); + task.setExternalInputPayloadStoragePath(null); + } + }); + Map input = parametersUtils.getTaskInput(taskToSchedule.getInputParameters(), - workflowInstance, null, null); + workflow, null, null); Type taskType = Type.USER_DEFINED; String type = taskToSchedule.getType(); @@ -460,8 +496,8 @@ public List getTasksToBeScheduled(WorkflowDef workflowDefinition, Workflow } // get in progress tasks for this workflow instance - List inProgressTasks = workflowInstance.getTasks().stream() - .filter(pendingTask -> pendingTask.getStatus().equals(Status.IN_PROGRESS)) + List inProgressTasks = workflow.getTasks().stream() + .filter(runningTask -> runningTask.getStatus().equals(Status.IN_PROGRESS)) .map(Task::getReferenceTaskName) .collect(Collectors.toList()); @@ -471,7 +507,7 @@ public List getTasksToBeScheduled(WorkflowDef workflowDefinition, Workflow TaskMapperContext taskMapperContext = TaskMapperContext.newBuilder() .withWorkflowDefinition(workflowDefinition) - .withWorkflowInstance(workflowInstance) + .withWorkflowInstance(workflow) .withTaskDefinition(taskDef) .withTaskToSchedule(taskToSchedule) .withTaskInput(input) @@ -489,6 +525,15 @@ public List getTasksToBeScheduled(WorkflowDef workflowDefinition, Workflow .collect(Collectors.toList()); } + @SuppressWarnings("unchecked") + private Map downloadFromExternalStorage(String path) { + try (InputStream inputStream = externalPayloadStorage.download(path)) { + return objectMapper.readValue(IOUtils.toString(inputStream), Map.class); + } catch (IOException e) { + logger.error("Unable to download payload from external storage path: {}", path, e); + throw new ApplicationException(ApplicationException.Code.INTERNAL_ERROR, e); + } + } private boolean isTaskSkipped(WorkflowTask taskToSchedule, Workflow workflow) { try { diff --git a/core/src/main/java/com/netflix/conductor/core/execution/ParametersUtils.java b/core/src/main/java/com/netflix/conductor/core/execution/ParametersUtils.java index 2530ef4fac..7326f501e6 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/ParametersUtils.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/ParametersUtils.java @@ -1,4 +1,4 @@ -/** +/* * Copyright 2016 Netflix, Inc. *

    * Licensed under the Apache License, Version 2.0 (the "License"); @@ -36,13 +36,11 @@ import java.util.Optional; /** - * - * - * + * Used to parse and resolve the JSONPath bindings in the workflow and task definitions. */ public class ParametersUtils { - private ObjectMapper om = new ObjectMapper(); + private ObjectMapper objectMapper = new ObjectMapper(); private TypeReference> map = new TypeReference>() { }; @@ -54,7 +52,6 @@ public enum SystemParameters { } public ParametersUtils() { - } public Map getTaskInput(Map inputParams, Workflow workflow, @@ -80,69 +77,68 @@ public Map getTaskInputV2(Map input, Workflow wo Map> inputMap = new HashMap<>(); - Map wf = new HashMap<>(); - wf.put("input", workflow.getInput()); - wf.put("output", workflow.getOutput()); - wf.put("status", workflow.getStatus()); - wf.put("workflowId", workflow.getWorkflowId()); - wf.put("parentWorkflowId", workflow.getParentWorkflowId()); - wf.put("parentWorkflowTaskId", workflow.getParentWorkflowTaskId()); - wf.put("workflowType", workflow.getWorkflowType()); - wf.put("version", workflow.getVersion()); - wf.put("correlationId", workflow.getCorrelationId()); - wf.put("reasonForIncompletion", workflow.getReasonForIncompletion()); - wf.put("schemaVersion", workflow.getSchemaVersion()); - - inputMap.put("workflow", wf); - //For new work flow being started the list of tasks will be empty + Map workflowParams = new HashMap<>(); + workflowParams.put("input", workflow.getInput()); + workflowParams.put("output", workflow.getOutput()); + workflowParams.put("status", workflow.getStatus()); + workflowParams.put("workflowId", workflow.getWorkflowId()); + workflowParams.put("parentWorkflowId", workflow.getParentWorkflowId()); + workflowParams.put("parentWorkflowTaskId", workflow.getParentWorkflowTaskId()); + workflowParams.put("workflowType", workflow.getWorkflowType()); + workflowParams.put("version", workflow.getVersion()); + workflowParams.put("correlationId", workflow.getCorrelationId()); + workflowParams.put("reasonForIncompletion", workflow.getReasonForIncompletion()); + workflowParams.put("schemaVersion", workflow.getSchemaVersion()); + + inputMap.put("workflow", workflowParams); + + //For new workflow being started the list of tasks will be empty workflow.getTasks().stream() .map(Task::getReferenceTaskName) - .map(taskRefName -> workflow.getTaskByRefName(taskRefName)) + .map(workflow::getTaskByRefName) .forEach(task -> { - Map taskIO = new HashMap<>(); - taskIO.put("input", task.getInputData()); - taskIO.put("output", task.getOutputData()); - taskIO.put("taskType", task.getTaskType()); + Map taskParams = new HashMap<>(); + taskParams.put("input", task.getInputData()); + taskParams.put("output", task.getOutputData()); + taskParams.put("taskType", task.getTaskType()); if (task.getStatus() != null) { - taskIO.put("status", task.getStatus().toString()); + taskParams.put("status", task.getStatus().toString()); } - taskIO.put("referenceTaskName", task.getReferenceTaskName()); - taskIO.put("retryCount", task.getRetryCount()); - taskIO.put("correlationId", task.getCorrelationId()); - taskIO.put("pollCount", task.getPollCount()); - taskIO.put("taskDefName", task.getTaskDefName()); - taskIO.put("scheduledTime", task.getScheduledTime()); - taskIO.put("startTime", task.getStartTime()); - taskIO.put("endTime", task.getEndTime()); - taskIO.put("workflowInstanceId", task.getWorkflowInstanceId()); - taskIO.put("taskId", task.getTaskId()); - taskIO.put("reasonForIncompletion", task.getReasonForIncompletion()); - taskIO.put("callbackAfterSeconds", task.getCallbackAfterSeconds()); - taskIO.put("workerId", task.getWorkerId()); - inputMap.put(task.getReferenceTaskName(), taskIO); + taskParams.put("referenceTaskName", task.getReferenceTaskName()); + taskParams.put("retryCount", task.getRetryCount()); + taskParams.put("correlationId", task.getCorrelationId()); + taskParams.put("pollCount", task.getPollCount()); + taskParams.put("taskDefName", task.getTaskDefName()); + taskParams.put("scheduledTime", task.getScheduledTime()); + taskParams.put("startTime", task.getStartTime()); + taskParams.put("endTime", task.getEndTime()); + taskParams.put("workflowInstanceId", task.getWorkflowInstanceId()); + taskParams.put("taskId", task.getTaskId()); + taskParams.put("reasonForIncompletion", task.getReasonForIncompletion()); + taskParams.put("callbackAfterSeconds", task.getCallbackAfterSeconds()); + taskParams.put("workerId", task.getWorkerId()); + inputMap.put(task.getReferenceTaskName(), taskParams); }); Configuration option = Configuration.defaultConfiguration() .addOptions(Option.SUPPRESS_EXCEPTIONS); DocumentContext documentContext = JsonPath.parse(inputMap, option); - Map replaced = replace(inputParams, documentContext, taskId); - return replaced; + return replace(inputParams, documentContext, taskId); } //deep clone using json - POJO private Map clone(Map inputTemplate) { try { - byte[] bytes = om.writeValueAsBytes(inputTemplate); - Map cloned = om.readValue(bytes, map); - return cloned; + byte[] bytes = objectMapper.writeValueAsBytes(inputTemplate); + return objectMapper.readValue(bytes, map); } catch (IOException e) { - throw new RuntimeException(e.getMessage(), e); + throw new RuntimeException("Unable to clone input params", e); } } public Map replace(Map input, Object json) { - Object doc = null; + Object doc; if (json instanceof String) { doc = JsonPath.parse(json.toString()); } else { @@ -293,6 +289,4 @@ private Map getTaskInputV1(Workflow workflow, Map input, String event, Map taskToDomain) { - return startWorkflow(name, version, input, correlationId, null, null, event, taskToDomain); + public String startWorkflow(String name, int version, String correlationId, Map input, String externalInputPayloadStoragePath, String event, Map taskToDomain) { + return startWorkflow(name, version, input, externalInputPayloadStoragePath, correlationId, null, null, event, taskToDomain); } public String startWorkflow(String name, int version, Map input, String correlationId, String parentWorkflowId, String parentWorkflowTaskId, String event) { - return startWorkflow(name, version, input, correlationId, parentWorkflowId, parentWorkflowTaskId, event, null); + return startWorkflow(name, version, input, null, correlationId, parentWorkflowId, parentWorkflowTaskId, event, null); } private final Predicate validateLastPolledTime = pd -> pd.getLastPollTime() > System.currentTimeMillis() - (activeWorkerLastPollnSecs * 1000); @@ -134,17 +135,47 @@ public String startWorkflow(String name, int version, Map input, private final Predicate isNonTerminalTask = task -> !task.getStatus().isTerminal(); public String startWorkflow(String workflowName, int workflowVersion, Map workflowInput, - String correlationId, String parentWorkflowId, String parentWorkflowTaskId, - String event, Map taskToDomain) { + String externalInputPayloadStoragePath, String correlationId, String parentWorkflowId, + String parentWorkflowTaskId, String event, Map taskToDomain) { + + // perform validations + validateWorkflow(workflowName, workflowVersion, workflowInput, externalInputPayloadStoragePath); + + //A random UUID is assigned to the work flow instance + String workflowId = IDGenerator.generate(); + + // Persist the Workflow + Workflow workflow = new Workflow(); + workflow.setWorkflowId(workflowId); + workflow.setCorrelationId(correlationId); + workflow.setWorkflowType(workflowName); + workflow.setVersion(workflowVersion); + workflow.setInput(workflowInput); + workflow.setExternalInputPayloadStoragePath(externalInputPayloadStoragePath); + workflow.setStatus(WorkflowStatus.RUNNING); + workflow.setParentWorkflowId(parentWorkflowId); + workflow.setParentWorkflowTaskId(parentWorkflowTaskId); + workflow.setOwnerApp(WorkflowContext.get().getClientApp()); + workflow.setCreateTime(System.currentTimeMillis()); + workflow.setUpdatedBy(null); + workflow.setUpdateTime(null); + workflow.setEvent(event); + workflow.setTaskToDomain(taskToDomain); + executionDAO.createWorkflow(workflow); + logger.info("A new instance of workflow {} created with workflow id {}", workflowName, workflowId); + //then decide to see if anything needs to be done as part of the workflow + decide(workflowId); + return workflowId; + } + /** + * Performs validations for starting a workflow + * + * @throws ApplicationException if the validation fails + */ + private void validateWorkflow(String workflowName, int workflowVersion, Map workflowInput, String externalStoragePath) { try { - //Check if the input to the workflow is not null - if (workflowInput == null) { - logger.error("The input for the workflow {} cannot be NULL", workflowName); - throw new ApplicationException(INVALID_INPUT, "NULL input passed when starting workflow"); - } - //Check if the workflow definition is valid WorkflowDef workflowDefinition = metadataDAO.get(workflowName, workflowVersion); if (workflowDefinition == null) { @@ -163,32 +194,12 @@ public String startWorkflow(String workflowName, int workflowVersion, Map getRunningWorkflowIds(String workflowName) { /** * @param workflowId ID of the workflow to evaluate the state for - * @return true if the workflow has completed (success or failed), false otherwise. + * @return true if the workflow is in terminal state, false otherwise. */ public boolean decide(String workflowId) { - //If it is a new workflow the tasks will be still empty even though include tasks is true + // If it is a new workflow, the tasks will be still empty even though include tasks is true Workflow workflow = executionDAO.getWorkflow(workflowId, true); - WorkflowDef def = metadataDAO.get(workflow.getWorkflowType(), workflow.getVersion()); + WorkflowDef workflowDef = metadataDAO.get(workflow.getWorkflowType(), workflow.getVersion()); try { - DeciderOutcome outcome = deciderService.decide(workflow, def); + DeciderOutcome outcome = deciderService.decide(workflow, workflowDef); if (outcome.isComplete) { completeWorkflow(workflow); return true; @@ -629,9 +640,9 @@ public boolean decide(String workflowId) { decide(workflowId); } - } catch (TerminateWorkflowException tw) { - logger.debug(tw.getMessage(), tw); - terminate(def, workflow, tw); + } catch (TerminateWorkflowException twe) { + logger.info("Execution terminated of workflow: {} of type: {}", workflowId, workflowDef.getName(), twe); + terminate(workflowDef, workflow, twe); return true; } catch (RuntimeException e) { logger.error("Error deciding workflow: {}", workflowId, e); @@ -791,7 +802,7 @@ public void executeSystemTask(WorkflowSystemTask systemTask, String taskId, int } } - public void setTaskDomains(List tasks, Workflow wf) { + private void setTaskDomains(List tasks, Workflow wf) { Map taskToDomain = wf.getTaskToDomain(); if (taskToDomain != null) { // Check if all tasks have the same domain "*" @@ -950,8 +961,8 @@ private boolean rerunWF(String workflowId, String taskId, Map ta break; } else { // If not found look into sub workflows - if (task.getTaskType().equalsIgnoreCase("SUB_WORKFLOW")) { - String subWorkflowId = task.getInputData().get("subWorkflowId").toString(); + if (task.getTaskType().equalsIgnoreCase(SubWorkflow.NAME)) { + String subWorkflowId = task.getInputData().get(SubWorkflow.SUB_WORKFLOW_ID).toString(); if (rerunWF(subWorkflowId, taskId, taskInput, null, null)) { rerunFromTask = task; break; @@ -960,7 +971,6 @@ private boolean rerunWF(String workflowId, String taskId, Map ta } } - if (rerunFromTask != null) { // Remove all tasks after the "rerunFromTask" for (Task task : workflow.getTasks()) { @@ -968,20 +978,20 @@ private boolean rerunWF(String workflowId, String taskId, Map ta executionDAO.removeTask(task.getTaskId()); } } - if (rerunFromTask.getTaskType().equalsIgnoreCase("SUB_WORKFLOW")) { + if (rerunFromTask.getTaskType().equalsIgnoreCase(SubWorkflow.NAME)) { // if task is sub workflow set task as IN_PROGRESS rerunFromTask.setStatus(IN_PROGRESS); - executionDAO.updateTask(rerunFromTask); } else { - // Set the task to rerun + // Set the task to rerun as SCHEDULED rerunFromTask.setStatus(SCHEDULED); if (taskInput != null) { rerunFromTask.setInputData(taskInput); } - rerunFromTask.setExecuted(false); - executionDAO.updateTask(rerunFromTask); addTaskToQueue(rerunFromTask); } + rerunFromTask.setExecuted(false); + executionDAO.updateTask(rerunFromTask); + // and set workflow as RUNNING workflow.setStatus(WorkflowStatus.RUNNING); if (correlationId != null) { diff --git a/core/src/main/java/com/netflix/conductor/core/execution/mapper/DecisionTaskMapper.java b/core/src/main/java/com/netflix/conductor/core/execution/mapper/DecisionTaskMapper.java index 87990378db..c2f9ac88ff 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/mapper/DecisionTaskMapper.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/mapper/DecisionTaskMapper.java @@ -28,6 +28,7 @@ import javax.script.ScriptException; import java.util.Arrays; +import java.util.Collections; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -85,7 +86,7 @@ public List getMappedTasks(TaskMapperContext taskMapperContext) { decisionTask.setScheduledTime(System.currentTimeMillis()); decisionTask.setEndTime(System.currentTimeMillis()); decisionTask.getInputData().put("case", caseValue); - decisionTask.getOutputData().put("caseOutput", Arrays.asList(caseValue)); + decisionTask.getOutputData().put("caseOutput", Collections.singletonList(caseValue)); decisionTask.setTaskId(taskId); decisionTask.setStatus(Task.Status.IN_PROGRESS); decisionTask.setWorkflowTask(taskToSchedule); diff --git a/core/src/main/java/com/netflix/conductor/core/execution/mapper/SimpleTaskMapper.java b/core/src/main/java/com/netflix/conductor/core/execution/mapper/SimpleTaskMapper.java index a18be48021..8da5dd2e76 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/mapper/SimpleTaskMapper.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/mapper/SimpleTaskMapper.java @@ -28,7 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -50,7 +50,6 @@ public SimpleTaskMapper(ParametersUtils parametersUtils, MetadataDAO metadataDAO this.metadataDAO = metadataDAO; } - /** * This method maps a {@link WorkflowTask} of type {@link WorkflowTask.Type#SIMPLE} * to a {@link Task} @@ -93,6 +92,6 @@ public List getMappedTasks(TaskMapperContext taskMapperContext) throws Ter simpleTask.setResponseTimeoutSeconds(taskDefinition.getResponseTimeoutSeconds()); simpleTask.setWorkflowTask(taskToSchedule); simpleTask.setRetriedTaskId(retriedTaskId); - return Arrays.asList(simpleTask); + return Collections.singletonList(simpleTask); } } diff --git a/core/src/main/java/com/netflix/conductor/core/execution/mapper/WaitTaskMapper.java b/core/src/main/java/com/netflix/conductor/core/execution/mapper/WaitTaskMapper.java index e0e37809d1..edc3b153d6 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/mapper/WaitTaskMapper.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/mapper/WaitTaskMapper.java @@ -25,6 +25,7 @@ import org.slf4j.LoggerFactory; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -68,6 +69,6 @@ public List getMappedTasks(TaskMapperContext taskMapperContext) { waitTask.setTaskId(taskId); waitTask.setStatus(Task.Status.IN_PROGRESS); waitTask.setWorkflowTask(taskToSchedule); - return Arrays.asList(waitTask); + return Collections.singletonList(waitTask); } } diff --git a/core/src/main/java/com/netflix/conductor/core/execution/tasks/Event.java b/core/src/main/java/com/netflix/conductor/core/execution/tasks/Event.java index f917508a2c..53040005e1 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/tasks/Event.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/tasks/Event.java @@ -14,7 +14,7 @@ * limitations under the License. */ /** - * + * */ package com.netflix.conductor.core.execution.tasks; @@ -33,6 +33,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.inject.Inject; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -44,18 +45,21 @@ * */ public class Event extends WorkflowSystemTask { - + private static final Logger logger = LoggerFactory.getLogger(Event.class); public static final String NAME = "EVENT"; private final ObjectMapper objectMapper = new ObjectMapper(); - private final ParametersUtils parametersUtils = new ParametersUtils(); + private final ParametersUtils parametersUtils; + private final EventQueues eventQueues; - - public Event() { + @Inject + public Event(EventQueues eventQueues, ParametersUtils parametersUtils) { super(NAME); + this.parametersUtils = parametersUtils; + this.eventQueues = eventQueues; } - + @Override public void start(Workflow workflow, Task task, WorkflowExecutor provider) { @@ -88,7 +92,7 @@ public void start(Workflow workflow, Task task, WorkflowExecutor provider) { public boolean execute(Workflow workflow, Task task, WorkflowExecutor provider) { return false; } - + @Override public void cancel(Workflow workflow, Task task, WorkflowExecutor provider) { Message message = new Message(task.getTaskId(), null, task.getTaskId()); @@ -123,17 +127,17 @@ ObservableQueue getQueue(Workflow workflow, Task task) { } } task.getOutputData().put("event_produced", queueName); - + try { - return EventQueues.getQueue(queueName); + return eventQueues.getQueue(queueName); } catch(IllegalArgumentException e) { logger.error("Error setting up queue: {} for task:{}, workflow:{}", queueName, task.getTaskId(), workflow.getWorkflowId(), e); task.setStatus(Status.FAILED); task.setReasonForIncompletion("Error when trying to access the specified queue/topic: " + sinkValue + ", error: " + e.getMessage()); - return null; + return null; } } - + @Override public boolean isAsync() { return false; diff --git a/core/src/main/java/com/netflix/conductor/core/execution/tasks/SubWorkflow.java b/core/src/main/java/com/netflix/conductor/core/execution/tasks/SubWorkflow.java index c289c85628..445b3dfa50 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/tasks/SubWorkflow.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/tasks/SubWorkflow.java @@ -37,6 +37,7 @@ public class SubWorkflow extends WorkflowSystemTask { private static final Logger logger = LoggerFactory.getLogger(SubWorkflow.class); public static final String NAME = "SUB_WORKFLOW"; + public static final String SUB_WORKFLOW_ID = "subWorkflowId"; public SubWorkflow() { super(NAME); @@ -56,9 +57,9 @@ public void start(Workflow workflow, Task task, WorkflowExecutor provider) { String correlationId = workflow.getCorrelationId(); try { - String subWorkflowId = provider.startWorkflow(name, version, wfInput, correlationId, workflow.getWorkflowId(), task.getTaskId(), null, workflow.getTaskToDomain()); - task.getOutputData().put("subWorkflowId", subWorkflowId); - task.getInputData().put("subWorkflowId", subWorkflowId); + String subWorkflowId = provider.startWorkflow(name, version, wfInput, null, correlationId, workflow.getWorkflowId(), task.getTaskId(), null, workflow.getTaskToDomain()); + task.getOutputData().put(SUB_WORKFLOW_ID, subWorkflowId); + task.getInputData().put(SUB_WORKFLOW_ID, subWorkflowId); task.setStatus(Status.IN_PROGRESS); } catch (Exception e) { task.setStatus(Status.FAILED); @@ -69,9 +70,9 @@ public void start(Workflow workflow, Task task, WorkflowExecutor provider) { @Override public boolean execute(Workflow workflow, Task task, WorkflowExecutor provider) { - String workflowId = (String) task.getOutputData().get("subWorkflowId"); + String workflowId = (String) task.getOutputData().get(SUB_WORKFLOW_ID); if (workflowId == null) { - workflowId = (String) task.getInputData().get("subWorkflowId"); //Backward compatibility + workflowId = (String) task.getInputData().get(SUB_WORKFLOW_ID); //Backward compatibility } if(StringUtils.isEmpty(workflowId)) { @@ -94,9 +95,9 @@ public boolean execute(Workflow workflow, Task task, WorkflowExecutor provider) @Override public void cancel(Workflow workflow, Task task, WorkflowExecutor provider) { - String workflowId = (String) task.getOutputData().get("subWorkflowId"); + String workflowId = (String) task.getOutputData().get(SUB_WORKFLOW_ID); if(workflowId == null) { - workflowId = (String) task.getInputData().get("subWorkflowId"); //Backward compatibility + workflowId = (String) task.getInputData().get(SUB_WORKFLOW_ID); //Backward compatibility } if(StringUtils.isEmpty(workflowId)) { diff --git a/core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java b/core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java index 356a0cc896..debf546da3 100644 --- a/core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java +++ b/core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java @@ -22,12 +22,12 @@ import java.io.InputStream; /** - * + * A dummy implementation of {@link ExternalPayloadStorage} used when no external payload is configured */ public class DummyPayloadStorage implements ExternalPayloadStorage { @Override - public ExternalStorageLocation getExternalUri(Operation operation, PayloadType payloadType) { + public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType) { return null; } diff --git a/core/src/main/java/com/netflix/conductor/core/utils/JsonUtils.java b/core/src/main/java/com/netflix/conductor/core/utils/JsonUtils.java index e09be20d92..26dfd1cef1 100644 --- a/core/src/main/java/com/netflix/conductor/core/utils/JsonUtils.java +++ b/core/src/main/java/com/netflix/conductor/core/utils/JsonUtils.java @@ -91,7 +91,7 @@ private Object getJson(String jsonAsString) { try { return objectMapper.readValue(jsonAsString, Object.class); } catch (Exception e) { - logger.error("Error parsing json string: {}", jsonAsString, e); + logger.info("Unable to parse (json?) string: {}", jsonAsString, e); return jsonAsString; } } diff --git a/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java b/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java index a863cbfea3..8b00c4a46b 100644 --- a/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java +++ b/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java @@ -56,7 +56,7 @@ public class S3PayloadStorage implements ExternalPayloadStorage { public S3PayloadStorage(Configuration config) { s3Client = AmazonS3ClientBuilder.standard().withRegion("us-east-1").build(); bucketName = config.getProperty("s3bucket", ""); - expirationSec = config.getIntProperty("s3signedurlexpirationseconds", 3600); + expirationSec = config.getIntProperty("s3signedurlexpirationseconds", 5); } /** @@ -65,7 +65,7 @@ public S3PayloadStorage(Configuration config) { * @return a {@link ExternalStorageLocation} object which contains the pre-signed URL and the s3 object key for the json payload */ @Override - public ExternalStorageLocation getExternalUri(Operation operation, PayloadType payloadType) { + public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType) { try { ExternalStorageLocation externalStorageLocation = new ExternalStorageLocation(); @@ -78,18 +78,7 @@ public ExternalStorageLocation getExternalUri(Operation operation, PayloadType p httpMethod = HttpMethod.PUT; } - StringBuilder stringBuilder = new StringBuilder(); - if (payloadType == PayloadType.WORKFLOW_INPUT) { - stringBuilder.append("workflow/input/"); - } else if (payloadType == PayloadType.WORKFLOW_OUTPUT) { - stringBuilder.append("workflow/output/"); - } else if (payloadType == PayloadType.TASK_INPUT) { - stringBuilder.append("task/input/"); - } else { - stringBuilder.append("task/output/"); - } - stringBuilder.append(IDGenerator.generate()).append(".json"); - String objectKey = stringBuilder.toString(); + String objectKey = getObjectKey(payloadType); externalStorageLocation.setPath(objectKey); GeneratePresignedUrlRequest generatePresignedUrlRequest = new GeneratePresignedUrlRequest(bucketName, objectKey) @@ -124,6 +113,11 @@ public void upload(String path, InputStream payload, long payloadSize) { } } + /** + * @param path the S3 key of the object + * @return an input stream containing the contents of the object + * Caller is expected to close the input stream. + */ @Override public InputStream download(String path) { try { @@ -135,4 +129,24 @@ public InputStream download(String path) { throw new ApplicationException(ApplicationException.Code.BACKEND_ERROR, msg, e); } } + + private String getObjectKey(PayloadType payloadType) { + StringBuilder stringBuilder = new StringBuilder(); + switch (payloadType) { + case WORKFLOW_INPUT: + stringBuilder.append("workflow/input/"); + break; + case WORKFLOW_OUTPUT: + stringBuilder.append("workflow/output/"); + break; + case TASK_INPUT: + stringBuilder.append("task/input/"); + break; + case TASK_OUTPUT: + stringBuilder.append("task/output/"); + break; + } + stringBuilder.append(IDGenerator.generate()).append(".json"); + return stringBuilder.toString(); + } } diff --git a/core/src/main/java/com/netflix/conductor/service/EventService.java b/core/src/main/java/com/netflix/conductor/service/EventService.java index 3ce97e5f0c..761e3900ef 100644 --- a/core/src/main/java/com/netflix/conductor/service/EventService.java +++ b/core/src/main/java/com/netflix/conductor/service/EventService.java @@ -1,12 +1,10 @@ package com.netflix.conductor.service; -import com.google.common.base.Preconditions; import com.netflix.conductor.annotations.Trace; import com.netflix.conductor.common.metadata.events.EventHandler; import com.netflix.conductor.core.events.EventProcessor; import com.netflix.conductor.core.events.EventQueues; import com.netflix.conductor.service.utils.ServiceUtils; -import org.apache.commons.lang3.StringUtils; import javax.inject.Inject; import javax.inject.Singleton; @@ -21,13 +19,14 @@ public class EventService { private final MetadataService metadataService; - private final EventProcessor eventProcessor; + private final EventQueues eventQueues; @Inject - public EventService(MetadataService metadataService, EventProcessor eventProcessor) { + public EventService(MetadataService metadataService, EventProcessor eventProcessor, EventQueues eventQueues) { this.metadataService = metadataService; this.eventProcessor = eventProcessor; + this.eventQueues = eventQueues; } /** @@ -88,6 +87,6 @@ public List getEventHandlersForEvent(String event, boolean activeO * @return list of registered queue providers. */ public List getEventQueueProviders() { - return EventQueues.providers(); + return eventQueues.getProviders(); } } diff --git a/core/src/main/java/com/netflix/conductor/service/ExecutionService.java b/core/src/main/java/com/netflix/conductor/service/ExecutionService.java index 5876e6bb27..dacdc36edb 100644 --- a/core/src/main/java/com/netflix/conductor/service/ExecutionService.java +++ b/core/src/main/java/com/netflix/conductor/service/ExecutionService.java @@ -104,7 +104,7 @@ public ExecutionService(WorkflowExecutor wfProvider, ExecutionDAO executionDAO, this.maxSearchSize = config.getIntProperty("workflow.max.search.size", 5_000); } - public Task poll(String taskType, String workerId) throws Exception { + public Task poll(String taskType, String workerId) { return poll(taskType, workerId, null); } public Task poll(String taskType, String workerId, String domain) { @@ -172,7 +172,7 @@ public List getPollData(String taskType) { public List getAllPollData() { Map queueSizes = queueDAO.queuesDetail(); - List allPollData = new ArrayList(); + List allPollData = new ArrayList<>(); queueSizes.keySet().forEach(k -> { try { if(!k.contains(QueueUtils.DOMAIN_SEPARATOR)){ @@ -210,8 +210,8 @@ public Task getPendingTaskForWorkflow(String taskReferenceName, String workflowI /** * This method removes the task from the un-acked Queue * - * @param taskId: the taskId that needs to be updated and removed from the unacked queueDAO - * @return True in case of successful removal of the taskId from the un-acked queueDAO + * @param taskId: the taskId that needs to be updated and removed from the unacked queue + * @return True in case of successful removal of the taskId from the un-acked queue */ public boolean ackTaskReceived(String taskId) { return Optional.ofNullable(getTask(taskId)) @@ -457,7 +457,7 @@ public List getTaskLogs(String taskId) { * @param payloadType the {@link PayloadType} at the external uri * @return the external uri at which the payload is stored/to be stored */ - public ExternalStorageLocation getPayloadUri(Operation operation, PayloadType payloadType) { - return externalPayloadStorage.getExternalUri(operation, payloadType); + public ExternalStorageLocation getExternalStorageLocation(Operation operation, PayloadType payloadType) { + return externalPayloadStorage.getLocation(operation, payloadType); } } diff --git a/core/src/main/java/com/netflix/conductor/service/MetadataService.java b/core/src/main/java/com/netflix/conductor/service/MetadataService.java index 15b2550f3c..5ee089e570 100644 --- a/core/src/main/java/com/netflix/conductor/service/MetadataService.java +++ b/core/src/main/java/com/netflix/conductor/service/MetadataService.java @@ -18,7 +18,6 @@ */ package com.netflix.conductor.service; -import com.google.common.base.Preconditions; import com.netflix.conductor.annotations.Trace; import com.netflix.conductor.common.metadata.events.EventHandler; import com.netflix.conductor.common.metadata.tasks.TaskDef; @@ -42,12 +41,14 @@ @Trace public class MetadataService { - private MetadataDAO metadataDAO; + private final MetadataDAO metadataDAO; + private final EventQueues eventQueues; @Inject - public MetadataService(MetadataDAO metadataDAO) { + public MetadataService(MetadataDAO metadataDAO, EventQueues eventQueues) { this.metadataDAO = metadataDAO; + this.eventQueues = eventQueues; } /** @@ -237,6 +238,6 @@ private void validateEvent(EventHandler eh) { ServiceUtils.checkNotNullOrEmpty(eh.getEvent(), "Missing event location"); ServiceUtils.checkNotNullOrEmpty(eh.getActions(), "No actions specified. Please specify at-least one action"); String event = eh.getEvent(); - EventQueues.getQueue(event); + eventQueues.getQueue(event); } } diff --git a/core/src/main/java/com/netflix/conductor/service/TaskService.java b/core/src/main/java/com/netflix/conductor/service/TaskService.java index eb71a5b596..fc2b87b4b7 100644 --- a/core/src/main/java/com/netflix/conductor/service/TaskService.java +++ b/core/src/main/java/com/netflix/conductor/service/TaskService.java @@ -292,8 +292,8 @@ public SearchResult search(int start, int size, String sort, String * * @return {@link ExternalStorageLocation} containing the uri and the path to the payload is stored in external storage */ - public ExternalStorageLocation getPayloadUri() { - return executionService.getPayloadUri(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.TASK_OUTPUT); + public ExternalStorageLocation getExternalStorageLocation() { + return executionService.getExternalStorageLocation(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.TASK_OUTPUT); } } diff --git a/core/src/main/java/com/netflix/conductor/service/WorkflowService.java b/core/src/main/java/com/netflix/conductor/service/WorkflowService.java index b170ce761b..212fd0cc1a 100644 --- a/core/src/main/java/com/netflix/conductor/service/WorkflowService.java +++ b/core/src/main/java/com/netflix/conductor/service/WorkflowService.java @@ -58,7 +58,7 @@ public String startWorkflow(StartWorkflowRequest startWorkflowRequest) { startWorkflowRequest.getVersion())); } return workflowExecutor.startWorkflow(workflowDef.getName(), workflowDef.getVersion(), - startWorkflowRequest.getCorrelationId(), startWorkflowRequest.getInput(), null, + startWorkflowRequest.getCorrelationId(), startWorkflowRequest.getInput(), startWorkflowRequest.getExternalInputPayloadStoragePath(), null, startWorkflowRequest.getTaskToDomain()); } @@ -286,7 +286,7 @@ public SearchResult searchWorkflowsByTasks(int start, int size, * * @return {@link ExternalStorageLocation} containing the uri and the path to the payload is stored in external storage */ - public ExternalStorageLocation getExternalPayloadUri() { - return executionService.getPayloadUri(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.WORKFLOW_INPUT); + public ExternalStorageLocation getExternalStorageLocation() { + return executionService.getExternalStorageLocation(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.WORKFLOW_INPUT); } } diff --git a/core/src/test/java/com/netflix/conductor/core/events/TestEventProcessor.java b/core/src/test/java/com/netflix/conductor/core/events/TestEventProcessor.java index 0a46e2f255..2de2196a93 100644 --- a/core/src/test/java/com/netflix/conductor/core/events/TestEventProcessor.java +++ b/core/src/test/java/com/netflix/conductor/core/events/TestEventProcessor.java @@ -31,8 +31,10 @@ import com.netflix.conductor.core.events.queue.Message; import com.netflix.conductor.core.events.queue.ObservableQueue; import com.netflix.conductor.core.execution.ApplicationException; +import com.netflix.conductor.core.execution.ParametersUtils; import com.netflix.conductor.core.execution.TestConfiguration; import com.netflix.conductor.core.execution.WorkflowExecutor; +import com.netflix.conductor.core.utils.DummyPayloadStorage; import com.netflix.conductor.service.ExecutionService; import com.netflix.conductor.service.MetadataService; import org.junit.Before; @@ -74,14 +76,21 @@ public class TestEventProcessor { private ExecutionService executionService; private WorkflowExecutor workflowExecutor; private ActionProcessor actionProcessor; + private EventQueues eventQueues; + private ParametersUtils parametersUtils; @Before public void setup() { event = "sqs:arn:account090:sqstest1"; queueURI = "arn:account090:sqstest1"; - EventQueueProvider provider = mock(EventQueueProvider.class); + metadataService = mock(MetadataService.class); + executionService = mock(ExecutionService.class); + workflowExecutor = mock(WorkflowExecutor.class); + actionProcessor = mock(ActionProcessor.class); + parametersUtils = new ParametersUtils(); + EventQueueProvider provider = mock(EventQueueProvider.class); queue = mock(ObservableQueue.class); Message[] messages = new Message[1]; messages[0] = new Message("t0", "{\"Type\":\"Notification\",\"MessageId\":\"7e4e6415-01e9-5caf-abaa-37fd05d446ff\",\"Message\":\"{\\n \\\"testKey1\\\": \\\"level1\\\",\\n \\\"metadata\\\": {\\n \\\"testKey2\\\": 123456 }\\n }\",\"Timestamp\":\"2018-08-10T21:22:05.029Z\",\"SignatureVersion\":\"1\"}", "t0"); @@ -92,14 +101,10 @@ public void setup() { when(queue.getName()).thenReturn(queueURI); when(queue.getType()).thenReturn("sqs"); when(provider.getQueue(queueURI)).thenReturn(queue); - EventQueues.providers = new HashMap<>(); - EventQueues.providers.put("sqs", provider); - - metadataService = mock(MetadataService.class); - executionService = mock(ExecutionService.class); - workflowExecutor = mock(WorkflowExecutor.class); - actionProcessor = mock(ActionProcessor.class); + Map providers = new HashMap<>(); + providers.put("sqs", provider); + eventQueues = new EventQueues(providers, parametersUtils); } @Test @@ -155,9 +160,9 @@ public void testEventProcessor() { workflowDef.setName(startWorkflowAction.getStart_workflow().getName()); when(metadataService.getWorkflowDef(any(), any())).thenReturn(workflowDef); - ActionProcessor actionProcessor = new ActionProcessor(workflowExecutor, metadataService); + ActionProcessor actionProcessor = new ActionProcessor(workflowExecutor, metadataService, parametersUtils); - EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, new TestConfiguration()); + EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, eventQueues, new TestConfiguration()); assertNotNull(eventProcessor.getQueues()); assertEquals(1, eventProcessor.getQueues().size()); @@ -214,9 +219,9 @@ public void testEventHandlerWithCondition() { workflowDef.setName(startWorkflowAction.getStart_workflow().getName()); when(metadataService.getWorkflowDef(any(), any())).thenReturn(workflowDef); - ActionProcessor actionProcessor = new ActionProcessor(workflowExecutor, metadataService); + ActionProcessor actionProcessor = new ActionProcessor(workflowExecutor, metadataService, parametersUtils); - EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, new TestConfiguration()); + EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, eventQueues, new TestConfiguration()); assertNotNull(eventProcessor.getQueues()); assertEquals(1, eventProcessor.getQueues().size()); @@ -245,7 +250,7 @@ public void testEventProcessorWithRetriableError() { when(executionService.addEventExecution(any())).thenReturn(true); when(actionProcessor.execute(any(), any(), any(), any())).thenThrow(new ApplicationException(ApplicationException.Code.BACKEND_ERROR, "some retriable error")); - EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, new TestConfiguration()); + EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, eventQueues, new TestConfiguration()); assertNotNull(eventProcessor.getQueues()); assertEquals(1, eventProcessor.getQueues().size()); @@ -276,7 +281,7 @@ public void testEventProcessorWithNonRetriableError() { when(actionProcessor.execute(any(), any(), any(), any())).thenThrow(new ApplicationException(ApplicationException.Code.INVALID_INPUT, "some non-retriable error")); - EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, new TestConfiguration()); + EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, eventQueues, new TestConfiguration()); assertNotNull(eventProcessor.getQueues()); assertEquals(1, eventProcessor.getQueues().size()); @@ -295,7 +300,7 @@ public void testExecuteInvalidAction() { throw new UnsupportedOperationException("error"); }).when(actionProcessor).execute(any(), any(), any(), any()); - EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, new TestConfiguration()); + EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, eventQueues, new TestConfiguration()); EventExecution eventExecution = new EventExecution("id", "messageId"); eventExecution.setStatus(EventExecution.Status.IN_PROGRESS); eventExecution.setEvent("event"); @@ -315,7 +320,7 @@ public void testExecuteNonRetriableApplicationException() { throw new ApplicationException(ApplicationException.Code.INVALID_INPUT, "some non-retriable error"); }).when(actionProcessor).execute(any(), any(), any(), any()); - EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, new TestConfiguration()); + EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, eventQueues, new TestConfiguration()); EventExecution eventExecution = new EventExecution("id", "messageId"); eventExecution.setStatus(EventExecution.Status.IN_PROGRESS); eventExecution.setEvent("event"); @@ -336,7 +341,7 @@ public void testExecuteRetriableApplicationException() { throw new ApplicationException(ApplicationException.Code.BACKEND_ERROR, "some retriable error"); }).when(actionProcessor).execute(any(), any(), any(), any()); - EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, new TestConfiguration()); + EventProcessor eventProcessor = new EventProcessor(executionService, metadataService, actionProcessor, eventQueues, new TestConfiguration()); EventExecution eventExecution = new EventExecution("id", "messageId"); eventExecution.setStatus(EventExecution.Status.IN_PROGRESS); eventExecution.setEvent("event"); diff --git a/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderOutcomes.java b/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderOutcomes.java index c339b9c26b..2f3ff4d943 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderOutcomes.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderOutcomes.java @@ -28,6 +28,7 @@ import com.netflix.conductor.common.metadata.workflow.WorkflowTask; import com.netflix.conductor.common.metadata.workflow.WorkflowTask.Type; import com.netflix.conductor.common.run.Workflow; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.core.execution.DeciderService.DeciderOutcome; import com.netflix.conductor.core.execution.mapper.DecisionTaskMapper; import com.netflix.conductor.core.execution.mapper.DynamicTaskMapper; @@ -86,6 +87,7 @@ public void init() { MetadataDAO metadataDAO = mock(MetadataDAO.class); QueueDAO queueDAO = mock(QueueDAO.class); + ExternalPayloadStorage externalPayloadStorage = mock(ExternalPayloadStorage.class); TaskDef taskDef = new TaskDef(); taskDef.setRetryCount(1); taskDef.setName("mockTaskDef"); @@ -104,7 +106,7 @@ public void init() { taskMappers.put("EVENT", new EventTaskMapper(parametersUtils)); taskMappers.put("WAIT", new WaitTaskMapper(parametersUtils)); - this.deciderService = new DeciderService(metadataDAO, queueDAO, taskMappers); + this.deciderService = new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorage, taskMappers); } @Test diff --git a/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java b/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java index 156e57f312..c07c1ee381 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java @@ -32,6 +32,7 @@ import com.netflix.conductor.common.metadata.workflow.WorkflowTask.Type; import com.netflix.conductor.common.run.Workflow; import com.netflix.conductor.common.run.Workflow.WorkflowStatus; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.core.execution.DeciderService.DeciderOutcome; import com.netflix.conductor.core.execution.mapper.DecisionTaskMapper; import com.netflix.conductor.core.execution.mapper.DynamicTaskMapper; @@ -114,6 +115,7 @@ public static void init() { public void setup() { MetadataDAO metadataDAO = mock(MetadataDAO.class); QueueDAO queueDAO = mock(QueueDAO.class); + ExternalPayloadStorage externalPayloadStorage = mock(ExternalPayloadStorage.class); TaskDef taskDef = new TaskDef(); WorkflowDef workflowDef = new WorkflowDef(); when(metadataDAO.getTaskDef(any())).thenReturn(taskDef); @@ -131,7 +133,7 @@ public void setup() { taskMappers.put("EVENT", new EventTaskMapper(parametersUtils)); taskMappers.put("WAIT", new WaitTaskMapper(parametersUtils)); - deciderService = new DeciderService(metadataDAO, queueDAO, taskMappers); + deciderService = new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorage, taskMappers); workflow = new Workflow(); workflow.getInput().put("requestId", "request id 001"); @@ -167,7 +169,7 @@ public void setup() { } @Test - public void testGetTaskInputV2() throws Exception { + public void testGetTaskInputV2() { workflow.setSchemaVersion(2); Map ip = new HashMap<>(); @@ -209,14 +211,16 @@ public void testGetTaskInputV2Partial() throws Exception { wfmap.put("output", workflow.getOutput()); wfi.put("workflow", wfmap); - workflow.getTasks().stream().map(task -> task.getReferenceTaskName()).forEach(ref -> { - Map taskInput = workflow.getTaskByRefName(ref).getInputData(); - Map taskOutput = workflow.getTaskByRefName(ref).getOutputData(); - Map io = new HashMap<>(); - io.put("input", taskInput); - io.put("output", taskOutput); - wfi.put(ref, io); - }); + workflow.getTasks().stream() + .map(Task::getReferenceTaskName) + .forEach(ref -> { + Map taskInput = workflow.getTaskByRefName(ref).getInputData(); + Map taskOutput = workflow.getTaskByRefName(ref).getOutputData(); + Map io = new HashMap<>(); + io.put("input", taskInput); + io.put("output", taskOutput); + wfi.put(ref, io); + }); workflow.setSchemaVersion(2); @@ -272,7 +276,7 @@ public void testGetTaskInputV2Partial() throws Exception { @SuppressWarnings("unchecked") @Test - public void testGetTaskInput() throws Exception { + public void testGetTaskInput() { Map ip = new HashMap<>(); ip.put("workflowInputParam", "${workflow.input.requestId}"); ip.put("taskOutputParam", "${task2.output.location}"); @@ -315,7 +319,7 @@ public void testGetTaskInput() throws Exception { } @Test - public void testGetTaskInputV1() throws Exception { + public void testGetTaskInputV1() { Map ip = new HashMap<>(); ip.put("workflowInputParam", "workflow.input.requestId"); ip.put("taskOutputParam", "task2.output.location"); @@ -395,12 +399,12 @@ public void testGetNextTask() { } @Test - public void testCaseStatement() throws Exception { + public void testCaseStatement() { WorkflowDef def = createConditionalWF(); Workflow wf = new Workflow(); - wf.setCreateTime(new Long(0)); + wf.setCreateTime(0L); wf.setWorkflowId("a"); wf.setCorrelationId("b"); wf.setWorkflowType(def.getName()); @@ -464,7 +468,7 @@ public void testTaskTimeout() { //Task should be marked as timed out assertEquals(Status.TIMED_OUT, task.getStatus()); - assertTrue(task.getReasonForIncompletion() != null); + assertNotNull(task.getReasonForIncompletion()); assertEquals(1, counter.count()); taskType.setTimeoutPolicy(TimeoutPolicy.ALERT_ONLY); @@ -508,9 +512,6 @@ public void testTaskTimeout() { public void testConcurrentTaskInputCalc() throws InterruptedException { TaskDef def = new TaskDef(); - ParametersUtils pu = new ParametersUtils(); - - Map inputMap = new HashMap<>(); inputMap.put("path", "${workflow.input.inputLocation}"); inputMap.put("type", "${workflow.input.sourceType}"); @@ -542,7 +543,7 @@ public void testConcurrentTaskInputCalc() throws InterruptedException { Workflow workflow = new Workflow(); workflow.setInput(workflowInput); - Map taskInput = pu.getTaskInputV2(new HashMap<>(), workflow, null, def); + Map taskInput = parametersUtils.getTaskInputV2(new HashMap<>(), workflow, null, def); Object reqInputObj = taskInput.get("input"); assertNotNull(reqInputObj); @@ -582,7 +583,7 @@ public void testConcurrentTaskInputCalc() throws InterruptedException { @SuppressWarnings("unchecked") @Test - public void testTaskRetry() throws Exception { + public void testTaskRetry() { workflow.setSchemaVersion(2); @@ -638,7 +639,7 @@ public void testFork() throws Exception { } @Test - public void testDecideSuccessfulWorkflow() throws Exception { + public void testDecideSuccessfulWorkflow() { WorkflowDef workflowDef = createLinearWorkflow(); Workflow workflow = new Workflow(); @@ -688,7 +689,7 @@ public void testDecideSuccessfulWorkflow() throws Exception { } @Test - public void testDecideFailedTask() throws Exception { + public void testDecideFailedTask() { WorkflowDef workflowDef = createLinearWorkflow(); Workflow workflow = new Workflow(); @@ -719,7 +720,7 @@ public void testDecideFailedTask() throws Exception { } @Test - public void testGetTasksToBeScheduled() throws Exception { + public void testGetTasksToBeScheduled() { WorkflowDef workflowDef = createLinearWorkflow(); Workflow workflow = new Workflow(); @@ -746,7 +747,7 @@ public void testGetTasksToBeScheduled() throws Exception { } @Test - public void testIsResponsedTimeOut() throws Exception { + public void testIsResponsedTimeOut() { TaskDef taskDef = new TaskDef(); taskDef.setName("test_rt"); taskDef.setResponseTimeoutSeconds(10); @@ -762,7 +763,7 @@ public void testIsResponsedTimeOut() throws Exception { assertTrue(flag); } - private WorkflowDef createConditionalWF() throws Exception { + private WorkflowDef createConditionalWF() { WorkflowTask workflowTask1 = new WorkflowTask(); workflowTask1.setName("junit_task_1"); diff --git a/core/src/test/java/com/netflix/conductor/core/execution/TestWorkflowExecutor.java b/core/src/test/java/com/netflix/conductor/core/execution/TestWorkflowExecutor.java index d486833480..13e62f7303 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/TestWorkflowExecutor.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/TestWorkflowExecutor.java @@ -25,6 +25,7 @@ import com.netflix.conductor.common.metadata.workflow.WorkflowTask; import com.netflix.conductor.common.metadata.workflow.WorkflowTask.Type; import com.netflix.conductor.common.run.Workflow; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.core.execution.mapper.DecisionTaskMapper; import com.netflix.conductor.core.execution.mapper.DynamicTaskMapper; import com.netflix.conductor.core.execution.mapper.EventTaskMapper; @@ -83,6 +84,7 @@ public void init() { executionDAO = mock(ExecutionDAO.class); metadataDAO = mock(MetadataDAO.class); queueDAO = mock(QueueDAO.class); + ExternalPayloadStorage externalPayloadStorage = mock(ExternalPayloadStorage.class); ObjectMapper objectMapper = new ObjectMapper(); ParametersUtils parametersUtils = new ParametersUtils(); Map taskMappers = new HashMap<>(); @@ -96,12 +98,12 @@ public void init() { taskMappers.put("SUB_WORKFLOW", new SubWorkflowTaskMapper(parametersUtils, metadataDAO)); taskMappers.put("EVENT", new EventTaskMapper(parametersUtils)); taskMappers.put("WAIT", new WaitTaskMapper(parametersUtils)); - DeciderService deciderService = new DeciderService(metadataDAO, queueDAO, taskMappers); - workflowExecutor = new WorkflowExecutor(deciderService, metadataDAO, executionDAO, queueDAO, config); + DeciderService deciderService = new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorage, taskMappers); + workflowExecutor = new WorkflowExecutor(deciderService, metadataDAO, executionDAO, queueDAO, parametersUtils, config); } @Test - public void testScheduleTask() throws Exception { + public void testScheduleTask() { AtomicBoolean httpTaskExecuted = new AtomicBoolean(false); AtomicBoolean http2TaskExecuted = new AtomicBoolean(false); @@ -119,7 +121,6 @@ public void start(Workflow workflow, Task task, WorkflowExecutor executor) { task.setStatus(Status.COMPLETED); super.start(workflow, task, executor); } - }; new WorkflowSystemTask("HTTP2") { @@ -130,7 +131,6 @@ public void start(Workflow workflow, Task task, WorkflowExecutor executor) { task.setStatus(Status.COMPLETED); super.start(workflow, task, executor); } - }; Workflow workflow = new Workflow(); @@ -225,7 +225,7 @@ public void start(Workflow workflow, Task task, WorkflowExecutor executor) { @Test @SuppressWarnings("unchecked") - public void testCompleteWorkflow() throws Exception { + public void testCompleteWorkflow() { Workflow workflow = new Workflow(); workflow.setWorkflowId("1"); workflow.setWorkflowType("test"); diff --git a/core/src/test/java/com/netflix/conductor/core/execution/mapper/DecisionTaskMapperTest.java b/core/src/test/java/com/netflix/conductor/core/execution/mapper/DecisionTaskMapperTest.java index 2f089aa7f7..0b0dde29fc 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/mapper/DecisionTaskMapperTest.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/mapper/DecisionTaskMapperTest.java @@ -12,7 +12,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; -import org.mockito.Mockito; import java.util.Arrays; import java.util.HashMap; @@ -20,13 +19,13 @@ import java.util.List; import java.util.Map; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; public class DecisionTaskMapperTest { - private ParametersUtils parametersUtils = new ParametersUtils(); + private ParametersUtils parametersUtils; private DeciderService deciderService; //Subject private DecisionTaskMapper decisionTaskMapper; @@ -34,14 +33,14 @@ public class DecisionTaskMapperTest { @Rule public ExpectedException expectedException = ExpectedException.none(); - Map ip1; WorkflowTask task1; WorkflowTask task2; WorkflowTask task3; @Before - public void setUp() throws Exception { + public void setUp() { + parametersUtils = new ParametersUtils(); ip1 = new HashMap<>(); ip1.put("p1", "workflow.input.param1"); @@ -67,7 +66,7 @@ public void setUp() throws Exception { } @Test - public void getMappedTasks() throws Exception { + public void getMappedTasks() { //Given //Task Definition @@ -134,7 +133,7 @@ public void getMappedTasks() throws Exception { } @Test - public void getEvaluatedCaseValue() throws Exception { + public void getEvaluatedCaseValue() { WorkflowTask decisionTask = new WorkflowTask(); decisionTask.setType(WorkflowTask.Type.DECISION.name()); @@ -163,7 +162,7 @@ public void getEvaluatedCaseValue() throws Exception { } @Test - public void getEvaluatedCaseValueUsingExpression() throws Exception { + public void getEvaluatedCaseValueUsingExpression() { //Given //Task Definition TaskDef taskDef = new TaskDef(); @@ -246,10 +245,5 @@ public void getEvaluatedCaseValueException() { expectedException.expectMessage("Error while evaluating the script " + decisionTask.getCaseExpression()); decisionTaskMapper.getEvaluatedCaseValue(decisionTask, evaluatorInput); - } - - - - -} \ No newline at end of file +} diff --git a/core/src/test/java/com/netflix/conductor/core/execution/mapper/SubWorkflowTaskMapperTest.java b/core/src/test/java/com/netflix/conductor/core/execution/mapper/SubWorkflowTaskMapperTest.java index e75ade42d8..d879a451c2 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/mapper/SubWorkflowTaskMapperTest.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/mapper/SubWorkflowTaskMapperTest.java @@ -40,15 +40,16 @@ public class SubWorkflowTaskMapperTest { public ExpectedException expectedException = ExpectedException.none(); @Before - public void setUp() throws Exception { + public void setUp() { parametersUtils = mock(ParametersUtils.class); metadataDAO = mock(MetadataDAO.class); subWorkflowTaskMapper = new SubWorkflowTaskMapper(parametersUtils, metadataDAO); deciderService = mock(DeciderService.class); } + @SuppressWarnings("unchecked") @Test - public void getMappedTasks() throws Exception { + public void getMappedTasks() { //Given WorkflowDef workflowDef = new WorkflowDef(); Workflow workflowInstance = new Workflow(); @@ -90,7 +91,7 @@ public void getMappedTasks() throws Exception { @Test - public void getSubWorkflowParams() throws Exception { + public void getSubWorkflowParams() { WorkflowTask workflowTask = new WorkflowTask(); SubWorkflowParams subWorkflowParams = new SubWorkflowParams(); subWorkflowParams.setName("Foo"); @@ -101,7 +102,7 @@ public void getSubWorkflowParams() throws Exception { } @Test - public void getExceptionWhenNoSubWorkflowParamsPassed() throws Exception { + public void getExceptionWhenNoSubWorkflowParamsPassed() { WorkflowTask workflowTask = new WorkflowTask(); workflowTask.setName("FooWorkFLow"); @@ -114,7 +115,7 @@ public void getExceptionWhenNoSubWorkflowParamsPassed() throws Exception { @Test - public void getSubWorkflowVersion() throws Exception { + public void getSubWorkflowVersion() { Map subWorkflowParamMap = new HashMap<>(); subWorkflowParamMap.put("name","FooWorkFlow"); subWorkflowParamMap.put("version","2"); @@ -125,7 +126,7 @@ public void getSubWorkflowVersion() throws Exception { } @Test - public void getSubworkflowVersionFromMeta() throws Exception { + public void getSubworkflowVersionFromMeta() { Map subWorkflowParamMap = new HashMap<>(); WorkflowDef workflowDef = new WorkflowDef(); workflowDef.setName("FooWorkFlow"); @@ -138,7 +139,7 @@ public void getSubworkflowVersionFromMeta() throws Exception { } @Test - public void getSubworkflowVersionFromMetaException() throws Exception { + public void getSubworkflowVersionFromMetaException() { Map subWorkflowParamMap = new HashMap<>(); when(metadataDAO.getLatest(any())).thenReturn(null); diff --git a/core/src/test/java/com/netflix/conductor/core/execution/mapper/WaitTaskMapperTest.java b/core/src/test/java/com/netflix/conductor/core/execution/mapper/WaitTaskMapperTest.java index d200581cce..df6f61def5 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/mapper/WaitTaskMapperTest.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/mapper/WaitTaskMapperTest.java @@ -13,13 +13,12 @@ import java.util.HashMap; import java.util.List; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; public class WaitTaskMapperTest { - @Test - public void getMappedTasks() throws Exception { + public void getMappedTasks() { //Given WorkflowTask taskToSchedule = new WorkflowTask(); @@ -46,10 +45,5 @@ public void getMappedTasks() throws Exception { //Then assertEquals(1, mappedTasks.size()); assertEquals(Wait.NAME, mappedTasks.get(0).getTaskType()); - - - - } - -} \ No newline at end of file +} diff --git a/core/src/test/java/com/netflix/conductor/core/execution/tasks/TestEvent.java b/core/src/test/java/com/netflix/conductor/core/execution/tasks/TestEvent.java index a58c683681..325dc3f029 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/tasks/TestEvent.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/tasks/TestEvent.java @@ -18,29 +18,12 @@ */ package com.netflix.conductor.core.execution.tasks; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.mock; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; - -import com.netflix.conductor.core.events.EventQueues; -import org.junit.Before; -import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - import com.netflix.conductor.common.metadata.tasks.Task; import com.netflix.conductor.common.metadata.tasks.Task.Status; import com.netflix.conductor.common.metadata.workflow.WorkflowTask; import com.netflix.conductor.common.run.Workflow; +import com.netflix.conductor.core.events.EventQueueProvider; +import com.netflix.conductor.core.events.EventQueues; import com.netflix.conductor.core.events.MockQueueProvider; import com.netflix.conductor.core.events.queue.Message; import com.netflix.conductor.core.events.queue.ObservableQueue; @@ -48,6 +31,23 @@ import com.netflix.conductor.core.execution.ParametersUtils; import com.netflix.conductor.core.execution.TestConfiguration; import com.netflix.conductor.dao.QueueDAO; +import org.junit.Before; +import org.junit.Test; +import org.mockito.stubbing.Answer; + +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; /** * @author Viren @@ -55,24 +55,28 @@ */ public class TestEvent { + private EventQueues eventQueues; + private ParametersUtils parametersUtils; + @Before public void setup() { - EventQueues.providers = new HashMap<>(); - EventQueues.providers.put("sqs", new MockQueueProvider("sqs")); - EventQueues.providers.put("conductor", new MockQueueProvider("conductor")); + Map providers = new HashMap<>(); + providers.put("sqs", new MockQueueProvider("sqs")); + providers.put("conductor", new MockQueueProvider("conductor")); + parametersUtils = new ParametersUtils(); + eventQueues = new EventQueues(providers, parametersUtils); } @Test public void testEvent() { System.setProperty("QUEUE_NAME", "queue_name_001"); - ParametersUtils pu = new ParametersUtils(); String eventt = "queue_${QUEUE_NAME}"; - String event = pu.replace(eventt).toString(); + String event = parametersUtils.replace(eventt).toString(); assertNotNull(event); assertEquals("queue_queue_name_001", event); eventt = "queue_9"; - event = pu.replace(eventt).toString(); + event = parametersUtils.replace(eventt).toString(); assertNotNull(event); assertEquals(eventt, event); } @@ -100,7 +104,7 @@ public void testSinkParam() { task.setTaskType(WorkflowTask.Type.EVENT.name()); workflow.getTasks().add(task); - Event event = new Event(); + Event event = new Event(eventQueues, parametersUtils); ObservableQueue queue = event.getQueue(workflow, task); assertNotNull(task.getReasonForIncompletion(), queue); assertEquals("queue_name", queue.getName()); @@ -146,9 +150,9 @@ public void testSinkParam() { assertEquals(Task.Status.FAILED, task.getStatus()); } + @SuppressWarnings("unchecked") @Test - public void test() throws Exception { - Event event = new Event(); + public void test() { Workflow workflow = new Workflow(); workflow.setWorkflowType("testWorkflow"); workflow.setVersion(2); @@ -162,34 +166,28 @@ public void test() throws Exception { String[] publishedQueue = new String[1]; List publishedMessages = new LinkedList<>(); - doAnswer(new Answer() { - - @SuppressWarnings("unchecked") - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - String queueName = invocation.getArgumentAt(0, String.class); - System.out.println(queueName); - publishedQueue[0] = queueName; - List messages = invocation.getArgumentAt(1, List.class); - publishedMessages.addAll(messages); - return null; - } + doAnswer((Answer) invocation -> { + String queueName = invocation.getArgumentAt(0, String.class); + System.out.println(queueName); + publishedQueue[0] = queueName; + List messages = invocation.getArgumentAt(1, List.class); + publishedMessages.addAll(messages); + return null; }).when(dao).push(any(), any()); - doAnswer(new Answer>() { - - @Override - public List answer(InvocationOnMock invocation) throws Throwable { - String messageId = invocation.getArgumentAt(1, String.class); - if(publishedMessages.get(0).getId().equals(messageId)) { - publishedMessages.remove(0); - return Arrays.asList(messageId); - } - return null; + doAnswer((Answer>) invocation -> { + String messageId = invocation.getArgumentAt(1, String.class); + if(publishedMessages.get(0).getId().equals(messageId)) { + publishedMessages.remove(0); + return Collections.singletonList(messageId); } + return null; }).when(dao).remove(any(), any()); - EventQueues.providers.put("conductor", new DynoEventQueueProvider(dao, new TestConfiguration())); + Map providers = new HashMap<>(); + providers.put("conductor", new DynoEventQueueProvider(dao, new TestConfiguration())); + eventQueues = new EventQueues(providers, parametersUtils); + Event event = new Event(eventQueues, parametersUtils); event.start(workflow, task, null); assertEquals(Task.Status.COMPLETED, task.getStatus()); @@ -206,8 +204,8 @@ public List answer(InvocationOnMock invocation) throws Throwable { @Test - public void testFailures() throws Exception { - Event event = new Event(); + public void testFailures() { + Event event = new Event(eventQueues, parametersUtils); Workflow workflow = new Workflow(); workflow.setWorkflowType("testWorkflow"); workflow.setVersion(2); @@ -242,7 +240,7 @@ public void testFailures() throws Exception { @Test public void testDynamicSinks() { - Event event = new Event(); + Event event = new Event(eventQueues, parametersUtils); Workflow workflow = new Workflow(); workflow.setWorkflowType("testWorkflow"); workflow.setVersion(2); @@ -278,8 +276,6 @@ public void testDynamicSinks() { task.getInputData().put("sink", "sns:my_sqs_queue_name"); queue = event.getQueue(workflow, task); assertEquals(Task.Status.FAILED, task.getStatus()); - - } } diff --git a/core/src/test/java/com/netflix/conductor/service/WorkflowServiceTest.java b/core/src/test/java/com/netflix/conductor/service/WorkflowServiceTest.java index 40e128ef21..33955f4957 100644 --- a/core/src/test/java/com/netflix/conductor/service/WorkflowServiceTest.java +++ b/core/src/test/java/com/netflix/conductor/service/WorkflowServiceTest.java @@ -19,7 +19,8 @@ import java.util.List; import java.util.Map; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyInt; @@ -27,7 +28,6 @@ import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyMapOf; import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.isNull; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -40,8 +40,6 @@ public class WorkflowServiceTest { private MetadataService mockMetadata; - private Configuration mockConfig; - private WorkflowService workflowService; @Before @@ -49,15 +47,15 @@ public void before() { this.mockWorkflowExecutor = Mockito.mock(WorkflowExecutor.class); this.mockExecutionService = Mockito.mock(ExecutionService.class); this.mockMetadata = Mockito.mock(MetadataService.class); - this.mockConfig = Mockito.mock(Configuration.class); + Configuration mockConfig = Mockito.mock(Configuration.class); when(mockConfig.getIntProperty(anyString(), anyInt())).thenReturn(5_000); this.workflowService = new WorkflowService(this.mockWorkflowExecutor, this.mockExecutionService, - this.mockMetadata, this.mockConfig); + this.mockMetadata, mockConfig); } @Test - public void testStartWorkflow() throws Exception { + public void testStartWorkflow() { WorkflowDef workflowDef = new WorkflowDef(); workflowDef.setName("test"); workflowDef.setVersion(1); @@ -72,13 +70,13 @@ public void testStartWorkflow() throws Exception { when(mockMetadata.getWorkflowDef(anyString(), anyInt())).thenReturn(workflowDef); when(mockWorkflowExecutor.startWorkflow(anyString(), anyInt(), anyString(), - anyMapOf(String.class, Object.class), any(String.class), + anyMapOf(String.class, Object.class), any(String.class), any(String.class), anyMapOf(String.class, String.class))).thenReturn(workflowID); assertEquals("w112", workflowService.startWorkflow(startWorkflowRequest)); } @Test(expected = ApplicationException.class) - public void testApplicationExceptionStartWorkflowMessage() throws Exception { + public void testApplicationExceptionStartWorkflowMessage() { try { when(mockMetadata.getWorkflowDef(anyString(), anyInt())).thenReturn(null); @@ -95,7 +93,7 @@ public void testApplicationExceptionStartWorkflowMessage() throws Exception { } @Test - public void testStartWorkflowParam() throws Exception { + public void testStartWorkflowParam() { WorkflowDef workflowDef = new WorkflowDef(); workflowDef.setName("test"); workflowDef.setVersion(1); @@ -111,7 +109,7 @@ public void testStartWorkflowParam() throws Exception { } @Test(expected = ApplicationException.class) - public void testApplicationExceptionStartWorkflowMessageParam() throws Exception { + public void testApplicationExceptionStartWorkflowMessageParam() { try { when(mockMetadata.getWorkflowDef(anyString(), anyInt())).thenReturn(null); @@ -143,7 +141,7 @@ public void testGetWorklfowsSingleCorrelationId() { } @Test - public void testGetWorklfowsMultipleCorrelationId() throws Exception { + public void testGetWorklfowsMultipleCorrelationId() { Workflow workflow = new Workflow(); workflow.setCorrelationId("c123"); @@ -165,7 +163,7 @@ public void testGetWorklfowsMultipleCorrelationId() throws Exception { } @Test - public void testGetExecutionStatus() throws Exception { + public void testGetExecutionStatus() { Workflow workflow = new Workflow(); workflow.setCorrelationId("c123"); @@ -174,7 +172,7 @@ public void testGetExecutionStatus() throws Exception { } @Test(expected = ApplicationException.class) - public void testApplicationExceptionGetExecutionStatus() throws Exception { + public void testApplicationExceptionGetExecutionStatus() { try { when(mockExecutionService.getExecutionStatus(anyString(), anyBoolean())).thenReturn(null); workflowService.getExecutionStatus("w123", true); @@ -187,13 +185,13 @@ public void testApplicationExceptionGetExecutionStatus() throws Exception { } @Test - public void testDeleteWorkflow() throws Exception { + public void testDeleteWorkflow() { workflowService.deleteWorkflow("w123", true); verify(mockExecutionService, times(1)).removeWorkflow(anyString(), anyBoolean()); } @Test(expected = ApplicationException.class) - public void testInvalidDeleteWorkflow() throws Exception { + public void testInvalidDeleteWorkflow() { try { workflowService.deleteWorkflow(null, true); } catch (ApplicationException ex) { @@ -205,7 +203,7 @@ public void testInvalidDeleteWorkflow() throws Exception { } @Test(expected = ApplicationException.class) - public void testInvalidWorkflowNameGetRunningWorkflows() throws Exception { + public void testInvalidWorkflowNameGetRunningWorkflows() { try { workflowService.getRunningWorkflows(null, 123, null, null); } catch (ApplicationException ex) { @@ -217,51 +215,51 @@ public void testInvalidWorkflowNameGetRunningWorkflows() throws Exception { } @Test - public void testGetRunningWorkflowsTime() throws Exception{ + public void testGetRunningWorkflowsTime() { workflowService.getRunningWorkflows("test", 1, 100L, 120L); verify(mockWorkflowExecutor, times(1)).getWorkflows(anyString(), anyInt(), anyLong(), anyLong()); } @Test - public void testGetRunningWorkflows() throws Exception{ + public void testGetRunningWorkflows() { workflowService.getRunningWorkflows("test", 1, null, null); verify(mockWorkflowExecutor, times(1)).getRunningWorkflowIds(anyString()); } @Test - public void testDecideWorkflow() throws Exception { + public void testDecideWorkflow() { workflowService.decideWorkflow("test"); verify(mockWorkflowExecutor, times(1)).decide(anyString()); } @Test - public void testPauseWorkflow() throws Exception{ + public void testPauseWorkflow() { workflowService.pauseWorkflow("test"); verify(mockWorkflowExecutor, times(1)).pauseWorkflow(anyString()); } @Test - public void testResumeWorkflow() throws Exception { + public void testResumeWorkflow() { workflowService.resumeWorkflow("test"); verify(mockWorkflowExecutor, times(1)).resumeWorkflow(anyString()); } @Test - public void testSkipTaskFromWorkflow() throws Exception { + public void testSkipTaskFromWorkflow() { workflowService.skipTaskFromWorkflow("test", "testTask", null); verify(mockWorkflowExecutor, times(1)).skipTaskFromWorkflow(anyString(), anyString(), any(SkipTaskRequest.class)); } @Test - public void testRerunWorkflow() throws Exception { + public void testRerunWorkflow() { RerunWorkflowRequest request = new RerunWorkflowRequest(); workflowService.rerunWorkflow("test", request); verify(mockWorkflowExecutor, times(1)).rerun(any(RerunWorkflowRequest.class)); } @Test - public void testRerunWorkflowReturnWorkflowId() throws Exception { + public void testRerunWorkflowReturnWorkflowId() { RerunWorkflowRequest request = new RerunWorkflowRequest(); String workflowId = "w123"; when(mockWorkflowExecutor.rerun(any(RerunWorkflowRequest.class))).thenReturn(workflowId); @@ -269,25 +267,25 @@ public void testRerunWorkflowReturnWorkflowId() throws Exception { } @Test - public void testRestartWorkflow() throws Exception { + public void testRestartWorkflow() { workflowService.restartWorkflow("w123"); verify(mockWorkflowExecutor, times(1)).rewind(anyString()); } @Test - public void testRetryWorkflow() throws Exception { + public void testRetryWorkflow() { workflowService.retryWorkflow("w123"); verify(mockWorkflowExecutor, times(1)).retry(anyString()); } @Test - public void testResetWorkflow() throws Exception{ + public void testResetWorkflow() { workflowService.resetWorkflow("w123"); verify(mockWorkflowExecutor, times(1)).resetCallbacksForInProgressTasks(anyString()); } @Test - public void testTerminateWorkflow() throws Exception { + public void testTerminateWorkflow() { workflowService.terminateWorkflow("w123", "test"); verify(mockWorkflowExecutor, times(1)).terminateWorkflow(anyString(), anyString()); } @@ -308,7 +306,7 @@ public void testSearchWorkflows() { } @Test(expected = ApplicationException.class) - public void testInvalidSizeSearchWorkflows() throws Exception { + public void testInvalidSizeSearchWorkflows() { try { workflowService.searchWorkflows(0,6000,"asc", "*", "*"); } catch (ApplicationException ex) { diff --git a/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java b/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java index 5ce34a3b48..e0b8d5753f 100644 --- a/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java +++ b/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java @@ -116,7 +116,7 @@ public String updateTask(TaskResult taskResult) { @ApiOperation("Ack Task is received") @Consumes({MediaType.WILDCARD}) public String ack(@PathParam("taskId") String taskId, - @QueryParam("workerid") String workerId) throws Exception{ + @QueryParam("workerid") String workerId) { return taskService.ackTaskReceived(taskId, workerId); } @@ -194,7 +194,7 @@ public List getAllPollData() { @POST @Path("/queue/requeue") @ApiOperation("Requeue pending tasks for all the running workflows") - public String requeue() throws Exception { + public String requeue() { return taskService.requeue(); } @@ -226,7 +226,7 @@ public SearchResult search(@QueryParam("start") @DefaultValue("0") @ApiOperation("Get the external uri where the task output payload is to be stored") @Consumes(MediaType.WILDCARD) @Path("/externalstoragelocation") - public ExternalStorageLocation getPayloadURI() { - return taskService.getPayloadUri(); + public ExternalStorageLocation getExternalStorageLocation() { + return taskService.getExternalStorageLocation(); } } diff --git a/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java b/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java index 6f62c4a0e4..5898f8d351 100644 --- a/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java +++ b/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java @@ -80,7 +80,7 @@ public String startWorkflow(StartWorkflowRequest request) { public String startWorkflow(@PathParam("name") String name, @QueryParam("version") Integer version, @QueryParam("correlationId") String correlationId, - Map input) throws Exception { + Map input) { return workflowService.startWorkflow(name, version, correlationId, input); } @@ -166,7 +166,7 @@ public void resumeWorkflow(@PathParam("workflowId") String workflowId) { @Consumes(MediaType.WILDCARD) public void skipTaskFromWorkflow(@PathParam("workflowId") String workflowId, @PathParam("taskReferenceName") String taskReferenceName, - SkipTaskRequest skipTaskRequest) throws Exception { + SkipTaskRequest skipTaskRequest) { workflowService.skipTaskFromWorkflow(workflowId, taskReferenceName, skipTaskRequest); } @@ -247,7 +247,7 @@ public SearchResult searchWorkflowsByTasks(@QueryParam("start") @ApiOperation("Get the uri and path of the external storage where the workflow input payload is to be stored") @Consumes(MediaType.WILDCARD) @Path("/externalstoragelocation") - public ExternalStorageLocation getExternalPayloadURI() { - return workflowService.getExternalPayloadUri(); + public ExternalStorageLocation getExternalStorageLocation() { + return workflowService.getExternalStorageLocation(); } } diff --git a/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisExecutionDAO.java b/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisExecutionDAO.java index aa3aa65cf6..20f3ca049f 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisExecutionDAO.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisExecutionDAO.java @@ -560,12 +560,12 @@ public List getWorkflowsByCorrelationId(String correlationId, boolean } /** - * This is not just an insert or update, the terminal state workflow is removed from the - * QQ there can be partial updates if the node goes down - * TODO add logger statements for different if conditions - * @param workflow - * @param update - * @return + * Inserts a new workflow/ updates an existing workflow in the datastore. + * Additionally, if a workflow is in terminal state, it is removed from the set of pending workflows. + * + * @param workflow the workflow instance + * @param update flag to identify if update or create operation + * @return the workflowId */ private String insertOrUpdateWorkflow(Workflow workflow, boolean update) { Preconditions.checkNotNull(workflow, "workflow object cannot be null"); diff --git a/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisMetadataDAO.java b/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisMetadataDAO.java index c39b853b24..3490e484a7 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisMetadataDAO.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisMetadataDAO.java @@ -15,20 +15,6 @@ */ package com.netflix.conductor.dao.dynomite; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; - -import javax.inject.Inject; - import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.inject.Singleton; @@ -36,7 +22,6 @@ import com.netflix.conductor.common.metadata.events.EventHandler; import com.netflix.conductor.common.metadata.tasks.TaskDef; import com.netflix.conductor.common.metadata.workflow.WorkflowDef; -import com.netflix.conductor.common.run.Workflow; import com.netflix.conductor.core.config.Configuration; import com.netflix.conductor.core.execution.ApplicationException; import com.netflix.conductor.core.execution.ApplicationException.Code; @@ -46,6 +31,18 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.inject.Inject; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + @Singleton @Trace public class RedisMetadataDAO extends BaseDynoDAO implements MetadataDAO { diff --git a/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/queue/DynoQueueDAO.java b/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/queue/DynoQueueDAO.java index 0d5688d213..a7bdc7b7bd 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/queue/DynoQueueDAO.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/queue/DynoQueueDAO.java @@ -35,9 +35,10 @@ import javax.inject.Inject; import javax.inject.Singleton; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -97,10 +98,10 @@ public List getHosts() { init(); } - public DynoQueueDAO(JedisCommands dynoClient, JedisCommands dynoClientRead, ShardSupplier ss, Configuration config) { + public DynoQueueDAO(JedisCommands dynoClient, JedisCommands dynoClientRead, ShardSupplier shardSupplier, Configuration config) { this.dynoClient = dynoClient; - this.dynoClientRead = dynoClient; - this.ss = ss; + this.dynoClientRead = dynoClientRead; + this.ss = shardSupplier; this.config = config; init(); } @@ -121,12 +122,14 @@ public void init() { public void push(String queueName, String id, long offsetTimeInSecond) { Message msg = new Message(id, null); msg.setTimeout(offsetTimeInSecond, TimeUnit.SECONDS); - queues.get(queueName).push(Arrays.asList(msg)); + queues.get(queueName).push(Collections.singletonList(msg)); } @Override public void push(String queueName, List messages) { - List msgs = messages.stream().map(msg -> new Message(msg.getId(), msg.getPayload())).collect(Collectors.toList()); + List msgs = messages.stream() + .map(msg -> new Message(msg.getId(), msg.getPayload())) + .collect(Collectors.toList()); queues.get(queueName).push(msgs); } @@ -138,20 +141,24 @@ public boolean pushIfNotExists(String queueName, String id, long offsetTimeInSec } Message msg = new Message(id, null); msg.setTimeout(offsetTimeInSecond, TimeUnit.SECONDS); - queue.push(Arrays.asList(msg)); + queue.push(Collections.singletonList(msg)); return true; } @Override public List pop(String queueName, int count, int timeout) { List msg = queues.get(queueName).pop(count, timeout, TimeUnit.MILLISECONDS); - return msg.stream().map(Message::getId).collect(Collectors.toList()); + return msg.stream() + .map(Message::getId) + .collect(Collectors.toList()); } @Override public List pollMessages(String queueName, int count, int timeout) { List msgs = queues.get(queueName).pop(count, timeout, TimeUnit.MILLISECONDS); - return msgs.stream().map(msg -> new com.netflix.conductor.core.events.queue.Message(msg.getId(), msg.getPayload(), null)).collect(Collectors.toList()); + return msgs.stream() + .map(msg -> new com.netflix.conductor.core.events.queue.Message(msg.getId(), msg.getPayload(), null)) + .collect(Collectors.toList()); } @Override @@ -185,19 +192,18 @@ public void flush(String queueName) { @Override public Map queuesDetail() { - Map map = queues.queues().stream().collect(Collectors.toMap(queue -> queue.getName(), q -> q.size())); - return map; + return queues.queues().stream() + .collect(Collectors.toMap(DynoQueue::getName, DynoQueue::size)); } @Override public Map>> queuesDetailVerbose() { - Map>> map = queues.queues().stream() - .collect(Collectors.toMap(queue -> queue.getName(), q -> q.shardSizes())); - return map; + return queues.queues().stream() + .collect(Collectors.toMap(DynoQueue::getName, DynoQueue::shardSizes)); } public void processUnacks(String queueName) { - ((RedisDynoQueue)queues.get(queueName)).processUnacks();; + ((RedisDynoQueue)queues.get(queueName)).processUnacks(); } @Override @@ -210,6 +216,6 @@ public boolean setOffsetTime(String queueName, String id, long offsetTimeInSecon @Override public boolean exists(String queueName, String id) { DynoQueue queue = queues.get(queueName); - return queue.get(id) != null; + return Optional.ofNullable(queue.get(id)).isPresent(); } } \ No newline at end of file diff --git a/server/src/main/java/com/netflix/conductor/server/ConductorServer.java b/server/src/main/java/com/netflix/conductor/server/ConductorServer.java index 3a8d2ef293..48af63d348 100644 --- a/server/src/main/java/com/netflix/conductor/server/ConductorServer.java +++ b/server/src/main/java/com/netflix/conductor/server/ConductorServer.java @@ -65,7 +65,7 @@ enum DB { redis, dynomite, memory, redis_cluster, mysql } - enum ExternalPayloadStorage { + enum ExternalPayloadStorageType { S3 } @@ -77,7 +77,7 @@ enum ExternalPayloadStorage { private DB database; - private ExternalPayloadStorage externalPayloadStorage; + private ExternalPayloadStorageType externalPayloadStorageType; public ConductorServer(ConductorConfig conductorConfig) { this.conductorConfig = conductorConfig; @@ -117,9 +117,9 @@ public ConductorServer(ConductorConfig conductorConfig) { String externalPayloadStorageString = conductorConfig.getProperty("workflow.external.payload.storage", ""); try { - externalPayloadStorage = ConductorServer.ExternalPayloadStorage.valueOf(externalPayloadStorageString); + externalPayloadStorageType = ExternalPayloadStorageType.valueOf(externalPayloadStorageString); } catch(IllegalArgumentException e) { - logger.info("External payload storage is not configured, provided: {}, supported values are: {}", externalPayloadStorageString, Arrays.toString(ConductorServer.ExternalPayloadStorage.values()), e); + logger.info("External payload storage is not configured, provided: {}, supported values are: {}", externalPayloadStorageString, Arrays.toString(ExternalPayloadStorageType.values()), e); } init(dynoClusterName, dynoHosts); @@ -187,7 +187,7 @@ private void init(String dynoClusterName, List dynoHosts) { break; } - this.serverModule = new ServerModule(jedis, hostSupplier, conductorConfig, database, externalPayloadStorage); + this.serverModule = new ServerModule(jedis, hostSupplier, conductorConfig, database, externalPayloadStorageType); } private TokenMapSupplier getTokenMapSupplier(List dynoHosts) { diff --git a/server/src/main/java/com/netflix/conductor/server/ServerModule.java b/server/src/main/java/com/netflix/conductor/server/ServerModule.java index d9b592364f..8ec863c645 100644 --- a/server/src/main/java/com/netflix/conductor/server/ServerModule.java +++ b/server/src/main/java/com/netflix/conductor/server/ServerModule.java @@ -39,6 +39,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; +import static com.netflix.conductor.server.ConductorServer.ExternalPayloadStorageType.S3; + /** * @author Viren * @@ -61,16 +63,16 @@ public class ServerModule extends AbstractModule { private ConductorServer.DB db; - private ConductorServer.ExternalPayloadStorage externalPayloadStorage; + private ConductorServer.ExternalPayloadStorageType externalPayloadStorageType; - public ServerModule(JedisCommands jedis, HostSupplier hostSupplier, ConductorConfig conductorConfig, ConductorServer.DB db, ConductorServer.ExternalPayloadStorage externalPayloadStorage) { + public ServerModule(JedisCommands jedis, HostSupplier hostSupplier, ConductorConfig conductorConfig, ConductorServer.DB db, ConductorServer.ExternalPayloadStorageType externalPayloadStorageType) { this.dynoConn = jedis; this.hostSupplier = hostSupplier; this.conductorConfig = conductorConfig; this.region = conductorConfig.getRegion(); this.localRack = conductorConfig.getAvailabilityZone(); this.db = db; - this.externalPayloadStorage = externalPayloadStorage; + this.externalPayloadStorageType = externalPayloadStorageType; } @Override @@ -93,13 +95,13 @@ protected void configure() { // Use ES2 as default. install(new ElasticSearchModule()); } - + install(new CoreModule()); install(new JerseyModule()); - + new HttpTask(new RestClientManager(), conductorConfig); new JsonJqTransform(); - + List additionalModules = conductorConfig.getAdditionalModules(); if(additionalModules != null) { for(AbstractModule additionalModule : additionalModules) { @@ -107,7 +109,7 @@ protected void configure() { } } - if (externalPayloadStorage == ConductorServer.ExternalPayloadStorage.S3) { + if (externalPayloadStorageType == S3) { bind(ExternalPayloadStorage.class).to(S3PayloadStorage.class); } else { bind(ExternalPayloadStorage.class).to(DummyPayloadStorage.class); diff --git a/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java b/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java index 9867c0868b..617744951d 100644 --- a/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java +++ b/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java @@ -139,7 +139,7 @@ public class WorkflowServiceTest { private static final String TEST_WORKFLOW_NAME_3 = "junit_test_wf3"; @Before - public void init() throws Exception { + public void init() { System.setProperty("EC2_REGION", "us-east-1"); System.setProperty("EC2_AVAILABILITY_ZONE", "us-east-1c"); if (registered) { @@ -240,26 +240,83 @@ public void init() throws Exception { def2.setSchemaVersion(2); def2.setTasks(wftasks2); - try { - - WorkflowDef[] wdsf = new WorkflowDef[]{def, def2}; - for (WorkflowDef wd : wdsf) { - metadataService.updateWorkflowDef(wd); - } - createForkJoinWorkflow(); - def.setName(LONG_RUNNING); - metadataService.updateWorkflowDef(def); - } catch (Exception e) { + WorkflowDef[] wdsf = new WorkflowDef[]{def, def2}; + for (WorkflowDef wd : wdsf) { + metadataService.updateWorkflowDef(wd); } + createForkJoinWorkflow(); + def.setName(LONG_RUNNING); + metadataService.updateWorkflowDef(def); taskDefs = metadataService.getTaskDefs(); - registered = true; } + @Test + public void testWorkflowInputWithExternalPayload() { + WorkflowDef found = metadataService.getWorkflowDef(LINEAR_WORKFLOW_T1_T2, 1); + assertNotNull(found); + + String workflowInputPath = "workflow/input"; + String correlationId = "wf_external_storage"; + String workflowId = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2, 1, correlationId, null, workflowInputPath, null, null); + assertNotNull(workflowId); + + Workflow workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertNull("The workflow input should not be persisted", workflow.getInput()); + assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); + assertEquals(workflow.getReasonForIncompletion(), WorkflowStatus.RUNNING, workflow.getStatus()); + assertEquals(1, workflow.getTasks().size()); + + // Polling for the first task + Task task = workflowExecutionService.poll("junit_task_1", "task1.junit.worker"); + assertNotNull(task); + assertEquals("junit_task_1", task.getTaskType()); + assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); + assertEquals(workflowId, task.getWorkflowInstanceId()); + + // update first task with COMPLETED + String taskOutputPath = "task/output"; + task.setOutputData(null); + task.setExternalOutputPayloadStoragePath(taskOutputPath); + task.setStatus(COMPLETED); + workflowExecutionService.updateTask(task); + + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertNull("The workflow input should not be persisted", workflow.getInput()); + assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); + assertEquals(workflow.getReasonForIncompletion(), WorkflowStatus.RUNNING, workflow.getStatus()); + assertEquals(2, workflow.getTasks().size()); + assertNull("The task output should not be persisted", workflow.getTasks().get(0).getOutputData()); + assertEquals(taskOutputPath, workflow.getTasks().get(0).getExternalOutputPayloadStoragePath()); + + // Polling for the second task + task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker"); + assertNotNull(task); + assertEquals("junit_task_2", task.getTaskType()); + assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); + assertEquals(workflowId, task.getWorkflowInstanceId()); + + // update first task with COMPLETED + task.getOutputData().put("op", "success_task2"); + task.setStatus(COMPLETED); + workflowExecutionService.updateTask(task); + + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertNull("The workflow input should not be persisted", workflow.getInput()); + assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); + assertEquals(WorkflowStatus.COMPLETED, workflow.getStatus()); + assertEquals(2, workflow.getTasks().size()); + assertNull("The task output should not be persisted", workflow.getTasks().get(0).getOutputData()); + assertEquals(taskOutputPath, workflow.getTasks().get(0).getExternalOutputPayloadStoragePath()); + } + @Test - public void testWorkflowWithNoTasks() throws Exception { + public void testWorkflowWithNoTasks() { WorkflowDef empty = new WorkflowDef(); empty.setName("empty_workflow"); @@ -290,7 +347,7 @@ public void testTaskDefTemplate() throws Exception { body.put("outputPath", "${workflow.input.outputPath}"); httpRequest.put("body", body); templatedTask.getInputTemplate().put("http_request", httpRequest); - metadataService.registerTaskDef(Arrays.asList(templatedTask)); + metadataService.registerTaskDef(Collections.singletonList(templatedTask)); WorkflowDef templateWf = new WorkflowDef(); templateWf.setName("template_workflow"); @@ -324,17 +381,17 @@ public void testTaskDefTemplate() throws Exception { assertTrue(taskInput.containsKey("http_request")); assertTrue(taskInput.get("http_request") instanceof Map); - ObjectMapper om = new ObjectMapper(); + ObjectMapper objectMapper = new ObjectMapper(); //Use the commented sysout to get the string value //System.out.println(om.writeValueAsString(om.writeValueAsString(taskInput))); String expected = "{\"http_request\":{\"method\":\"GET\",\"vipStack\":\"test_stack\",\"body\":{\"requestDetails\":{\"key1\":\"value1\",\"key2\":42},\"outputPath\":\"s3://bucket/outputPath\",\"inputPaths\":[\"file://path1\",\"file://path2\"]},\"uri\":\"/get/something\"}}"; - assertEquals(expected, om.writeValueAsString(taskInput)); + assertEquals(expected, objectMapper.writeValueAsString(taskInput)); } @Test - public void testWorkflowSchemaVersion() throws Exception { + public void testWorkflowSchemaVersion() { WorkflowDef ver2 = new WorkflowDef(); ver2.setSchemaVersion(2); ver2.setName("Test_schema_version2"); @@ -462,7 +519,7 @@ public void testForkJoin() throws Exception { } @Test - public void testForkJoinNested() throws Exception { + public void testForkJoinNested() { createForkJoinNestedWorkflow(); @@ -561,7 +618,7 @@ public void testForkJoinNested() throws Exception { } @Test - public void testForkJoinFailure() throws Exception { + public void testForkJoinFailure() { try { createForkJoinWorkflow(); @@ -616,7 +673,7 @@ public void testForkJoinFailure() throws Exception { @SuppressWarnings("unchecked") @Test - public void testDynamicForkJoinLegacy() throws Exception { + public void testDynamicForkJoinLegacy() { try { createDynamicForkJoinWorkflowDefsLegacy(); @@ -685,7 +742,7 @@ public void testDynamicForkJoinLegacy() throws Exception { @SuppressWarnings("unchecked") @Test - public void testDynamicForkJoin() throws Exception { + public void testDynamicForkJoin() { createDynamicForkJoinWorkflowDefs(); @@ -809,7 +866,7 @@ public void testDynamicForkJoin() throws Exception { metadataService.updateTaskDef(taskDef); } - private void createForkJoinWorkflow() throws Exception { + private void createForkJoinWorkflow() { WorkflowDef workflowDef = new WorkflowDef(); workflowDef.setName(FORK_JOIN_WF); @@ -862,7 +919,7 @@ private void createForkJoinWorkflow() throws Exception { } - private void createForkJoinWorkflowWithZeroRetry() throws Exception { + private void createForkJoinWorkflowWithZeroRetry() { WorkflowDef def = new WorkflowDef(); def.setName(FORK_JOIN_WF + "_2"); @@ -915,7 +972,7 @@ private void createForkJoinWorkflowWithZeroRetry() throws Exception { } - private void createForkJoinNestedWorkflow() throws Exception { + private void createForkJoinNestedWorkflow() { WorkflowDef def = new WorkflowDef(); def.setName(FORK_JOIN_NESTED_WF); @@ -991,7 +1048,7 @@ private void createForkJoinNestedWorkflow() throws Exception { } - private void createDynamicForkJoinWorkflowDefs() throws Exception { + private void createDynamicForkJoinWorkflowDefs() { WorkflowDef def = new WorkflowDef(); def.setName(DYNAMIC_FORK_JOIN_WF); @@ -1032,7 +1089,7 @@ private void createDynamicForkJoinWorkflowDefs() throws Exception { } @SuppressWarnings("deprecation") - private void createDynamicForkJoinWorkflowDefsLegacy() throws Exception { + private void createDynamicForkJoinWorkflowDefsLegacy() { WorkflowDef def = new WorkflowDef(); def.setName(DYNAMIC_FORK_JOIN_WF_LEGACY); @@ -1067,7 +1124,7 @@ private void createDynamicForkJoinWorkflowDefsLegacy() throws Exception { } - private void createConditionalWF() throws Exception { + private void createConditionalWF() { WorkflowTask wft1 = new WorkflowTask(); wft1.setName("junit_task_1"); @@ -1142,14 +1199,14 @@ private void createConditionalWF() throws Exception { @Test - public void testDefDAO() throws Exception { + public void testDefDAO() { List taskDefs = metadataService.getTaskDefs(); assertNotNull(taskDefs); assertTrue(!taskDefs.isEmpty()); } @Test - public void testSimpleWorkflowFailureWithTerminalError() throws Exception { + public void testSimpleWorkflowFailureWithTerminalError() { clearWorkflows(); @@ -1213,7 +1270,7 @@ public void testSimpleWorkflowFailureWithTerminalError() throws Exception { assertEquals("NON TRANSIENT ERROR OCCURRED: An integration point required to complete the task is down", es.getReasonForIncompletion()); assertEquals(1, junit_task_1.getRetryCount()); //Configured retries at the task definition level assertEquals(0, t1.getRetryCount()); //Actual retries done on the task - assertEquals(true, es.getOutput().containsKey("o1")); + assertTrue(es.getOutput().containsKey("o1")); assertEquals("p1 value", es.getOutput().get("o1")); assertEquals(es.getOutput().get("validationErrors").toString(), "There was a terminal error"); @@ -1224,7 +1281,7 @@ public void testSimpleWorkflowFailureWithTerminalError() throws Exception { @Test - public void testSimpleWorkflow() throws Exception { + public void testSimpleWorkflow() { clearWorkflows(); @@ -1240,16 +1297,10 @@ public void testSimpleWorkflow() throws Exception { logger.info("testSimpleWorkflow.wfid= {}", workflowInstanceId); assertNotNull(workflowInstanceId); - Workflow es = workflowExecutionService.getExecutionStatus(workflowInstanceId, true); - assertNotNull(es); - assertEquals(es.getReasonForIncompletion(), WorkflowStatus.RUNNING, es.getStatus()); - - - es = workflowExecutionService.getExecutionStatus(workflowInstanceId, true); - assertNotNull(es); - assertEquals(WorkflowStatus.RUNNING, es.getStatus()); - assertEquals(1, es.getTasks().size()); //The very first task is the one that should be scheduled. - + Workflow workflow = workflowExecutionService.getExecutionStatus(workflowInstanceId, true); + assertNotNull(workflow); + assertEquals(workflow.getReasonForIncompletion(), WorkflowStatus.RUNNING, workflow.getStatus()); + assertEquals(1, workflow.getTasks().size()); //The very first task is the one that should be scheduled. boolean failed = false; try { @@ -1274,18 +1325,18 @@ public void testSimpleWorkflow() throws Exception { assertEquals(1, tasks.size()); task = tasks.get(0); - Workflow workflow = workflowExecutionService.getExecutionStatus(task.getWorkflowInstanceId(), false); + workflow = workflowExecutionService.getExecutionStatus(task.getWorkflowInstanceId(), false); System.out.println("task workflow = " + workflow.getWorkflowType() + "," + workflow.getInput()); assertEquals(workflowInstanceId, task.getWorkflowInstanceId()); task.getOutputData().put("op", task1Op); task.setStatus(COMPLETED); workflowExecutionService.updateTask(task); - es = workflowExecutionService.getExecutionStatus(workflowInstanceId, false); - assertNotNull(es); - assertNotNull(es.getOutput()); - assertTrue("Found " + es.getOutput().toString(), es.getOutput().containsKey("o3")); - assertEquals("task1.Done", es.getOutput().get("o3")); + workflow = workflowExecutionService.getExecutionStatus(workflowInstanceId, false); + assertNotNull(workflow); + assertNotNull(workflow.getOutput()); + assertTrue("Found " + workflow.getOutput().toString(), workflow.getOutput().containsKey("o3")); + assertEquals("task1.Done", workflow.getOutput().get("o3")); task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker"); assertNotNull(task); @@ -1295,7 +1346,6 @@ public void testSimpleWorkflow() throws Exception { assertNotNull("Found=" + task.getInputData(), task2Input); assertEquals(task1Op, task2Input); - task2Input = (String) task.getInputData().get("tp1"); assertNotNull(task2Input); assertEquals(inputParam1, task2Input); @@ -1304,17 +1354,15 @@ public void testSimpleWorkflow() throws Exception { task.setReasonForIncompletion("unit test failure"); workflowExecutionService.updateTask(task); - - es = workflowExecutionService.getExecutionStatus(workflowInstanceId, true); - assertNotNull(es); - assertEquals(WorkflowStatus.COMPLETED, es.getStatus()); - tasks = es.getTasks(); + workflow = workflowExecutionService.getExecutionStatus(workflowInstanceId, true); + assertNotNull(workflow); + assertEquals(WorkflowStatus.COMPLETED, workflow.getStatus()); + tasks = workflow.getTasks(); assertNotNull(tasks); assertEquals(2, tasks.size()); - assertTrue("Found " + es.getOutput().toString(), es.getOutput().containsKey("o3")); - assertEquals("task1.Done", es.getOutput().get("o3")); - + assertTrue("Found " + workflow.getOutput().toString(), workflow.getOutput().containsKey("o3")); + assertEquals("task1.Done", workflow.getOutput().get("o3")); } @Test @@ -1328,7 +1376,7 @@ public void testSimpleWorkflowWithResponseTimeout() throws Exception { workflowInput.put("param1", inputParam1); workflowInput.put("param2", "p2 value"); String workflowId = workflowExecutor.startWorkflow("RTOWF", 1, correlationId, workflowInput); - System.out.println("testSimpleWorkflowWithResponseTimeout.wfid=" + workflowId); + logger.debug("testSimpleWorkflowWithResponseTimeout.wfid={}", workflowId); assertNotNull(workflowId); Workflow workflow = workflowExecutionService.getExecutionStatus(workflowId, true); @@ -1357,7 +1405,7 @@ public void testSimpleWorkflowWithResponseTimeout() throws Exception { assertNotNull(workflow); assertEquals(WorkflowStatus.RUNNING, workflow.getStatus()); assertEquals(2, workflow.getTasks().size()); - assertTrue(workflow.getTasks().stream().allMatch(t-> t.getReferenceTaskName().equals("task_rt_t1"))); + assertTrue(workflow.getTasks().stream().allMatch(t -> t.getReferenceTaskName().equals("task_rt_t1"))); assertEquals(TIMED_OUT, workflow.getTasks().get(0).getStatus()); assertEquals(SCHEDULED, workflow.getTasks().get(1).getStatus()); @@ -1408,8 +1456,8 @@ public void testSimpleWorkflowWithResponseTimeout() throws Exception { } @Test - public void testWorkflowRerunWithSubWorkflows() throws Exception { - // Execute a workflow + public void testWorkflowRerunWithSubWorkflows() { + // Execute a workflow with sub-workflow String workflowId = this.runWorkflowWithSubworkflow(); // Check it completed Workflow workflow = workflowExecutionService.getExecutionStatus(workflowId, true); @@ -1420,8 +1468,8 @@ public void testWorkflowRerunWithSubWorkflows() throws Exception { // Now lets pickup the first task in the sub workflow and rerun it from there String subWorkflowId = null; for (Task task : workflow.getTasks()) { - if (task.getTaskType().equalsIgnoreCase("SUB_WORKFLOW")) { - subWorkflowId = task.getOutputData().get("subWorkflowId").toString(); + if (task.getTaskType().equalsIgnoreCase(SubWorkflow.NAME)) { + subWorkflowId = task.getOutputData().get(SubWorkflow.SUB_WORKFLOW_ID).toString(); } } assertNotNull(subWorkflowId); @@ -1434,25 +1482,25 @@ public void testWorkflowRerunWithSubWorkflows() throws Exception { } assertNotNull(subWorkflowTask1); - RerunWorkflowRequest request = new RerunWorkflowRequest(); - request.setReRunFromTaskId(subWorkflowTask1.getTaskId()); + RerunWorkflowRequest rerunWorkflowRequest = new RerunWorkflowRequest(); + rerunWorkflowRequest.setReRunFromTaskId(subWorkflowTask1.getTaskId()); Map newInput = new HashMap<>(); newInput.put("p1", "1"); newInput.put("p2", "2"); - request.setTaskInput(newInput); + rerunWorkflowRequest.setTaskInput(newInput); String correlationId = "unit_test_sw_new"; Map input = new HashMap<>(); input.put("param1", "New p1 value"); input.put("param2", "New p2 value"); - request.setCorrelationId(correlationId); - request.setWorkflowInput(input); + rerunWorkflowRequest.setCorrelationId(correlationId); + rerunWorkflowRequest.setWorkflowInput(input); - request.setReRunFromWorkflowId(workflowId); - request.setReRunFromTaskId(subWorkflowTask1.getTaskId()); + rerunWorkflowRequest.setReRunFromWorkflowId(workflowId); + rerunWorkflowRequest.setReRunFromTaskId(subWorkflowTask1.getTaskId()); // Rerun - workflowExecutor.rerun(request); + workflowExecutor.rerun(rerunWorkflowRequest); // The main WF and the sub WF should be in RUNNING state workflow = workflowExecutionService.getExecutionStatus(workflowId, true); @@ -1507,7 +1555,7 @@ public void testWorkflowRerunWithSubWorkflows() throws Exception { } @Test - public void testSimpleWorkflowWithTaskSpecificDomain() throws Exception { + public void testSimpleWorkflowWithTaskSpecificDomain() { clearWorkflows(); createWorkflowDefForDomain(); @@ -1530,7 +1578,7 @@ public void testSimpleWorkflowWithTaskSpecificDomain() throws Exception { task = workflowExecutionService.poll("junit_task_2", "task1.junit.worker", "domain1"); assertNull(task); - String wfid = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2_SW, 1, correlationId, input, null, taskToDomain); + String wfid = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2_SW, 1, correlationId, input, null, null, taskToDomain); System.out.println("testSimpleWorkflow.wfid=" + wfid); assertNotNull(wfid); Workflow wf = workflowExecutor.getWorkflow(wfid, false); @@ -1637,7 +1685,7 @@ public void testSimpleWorkflowWithTaskSpecificDomain() throws Exception { } @Test - public void testSimpleWorkflowWithAllTaskInOneDomain() throws Exception { + public void testSimpleWorkflowWithAllTaskInOneDomain() { clearWorkflows(); createWorkflowDefForDomain(); @@ -1659,7 +1707,7 @@ public void testSimpleWorkflowWithAllTaskInOneDomain() throws Exception { task = workflowExecutionService.poll("junit_task_2", "task1.junit.worker", "domain12"); assertNull(task); - String wfid = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2_SW, 1, correlationId, input, null, taskToDomain); + String wfid = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2_SW, 1, correlationId, input, null, null, taskToDomain); System.out.println("testSimpleWorkflow.wfid=" + wfid); assertNotNull(wfid); Workflow wf = workflowExecutor.getWorkflow(wfid, false); @@ -1749,7 +1797,7 @@ public void testSimpleWorkflowWithAllTaskInOneDomain() throws Exception { } @After - public void clearWorkflows() throws Exception { + public void clearWorkflows() { List workflows = metadataService.getWorkflowDefs().stream() .map(WorkflowDef::getName) .collect(Collectors.toList()); @@ -1759,13 +1807,11 @@ public void clearWorkflows() throws Exception { workflowExecutor.terminateWorkflow(wfid, "cleanup"); } } - queueDAO.queuesDetail().keySet().forEach(queueName -> { - queueDAO.flush(queueName); - }); + queueDAO.queuesDetail().keySet().forEach(queueDAO::flush); } @Test - public void testLongRunning() throws Exception { + public void testLongRunning() { clearWorkflows(); @@ -1777,22 +1823,16 @@ public void testLongRunning() throws Exception { String inputParam1 = "p1 value"; input.put("param1", inputParam1); input.put("param2", "p2 value"); - String wfid = workflowExecutor.startWorkflow(LONG_RUNNING, 1, correlationId, input); - System.out.println("testLongRunning.wfid=" + wfid); - assertNotNull(wfid); - - Workflow es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(WorkflowStatus.RUNNING, es.getStatus()); - + String workflowId = workflowExecutor.startWorkflow(LONG_RUNNING, 1, correlationId, input); + logger.debug("testLongRunning.wfid={}", workflowId); + assertNotNull(workflowId); - es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(WorkflowStatus.RUNNING, es.getStatus()); + Workflow workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertEquals(WorkflowStatus.RUNNING, workflow.getStatus()); // Check the queue - assertEquals(Integer.valueOf(1), workflowExecutionService.getTaskQueueSizes(Arrays.asList("junit_task_1")).get("junit_task_1")); - /// + assertEquals(Integer.valueOf(1), workflowExecutionService.getTaskQueueSizes(Collections.singletonList("junit_task_1")).get("junit_task_1")); Task task = workflowExecutionService.poll("junit_task_1", "task1.junit.worker"); assertNotNull(task); @@ -1800,28 +1840,24 @@ public void testLongRunning() throws Exception { String param1 = (String) task.getInputData().get("p1"); String param2 = (String) task.getInputData().get("p2"); - assertNotNull(param1); assertNotNull(param2); assertEquals("p1 value", param1); assertEquals("p2 value", param2); - - String task1Op = "task1.In.Progress"; - task.getOutputData().put("op", task1Op); + String task1Output = "task1.In.Progress"; + task.getOutputData().put("op", task1Output); task.setStatus(Status.IN_PROGRESS); task.setCallbackAfterSeconds(5); workflowExecutionService.updateTask(task); String taskId = task.getTaskId(); // Check the queue - assertEquals(Integer.valueOf(1), workflowExecutionService.getTaskQueueSizes(Arrays.asList("junit_task_1")).get("junit_task_1")); - /// - + assertEquals(Integer.valueOf(1), workflowExecutionService.getTaskQueueSizes(Collections.singletonList("junit_task_1")).get("junit_task_1")); - es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(WorkflowStatus.RUNNING, es.getStatus()); + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertEquals(WorkflowStatus.RUNNING, workflow.getStatus()); // Polling for next task should not return anything Task task2 = workflowExecutionService.poll("junit_task_2", "task2.junit.worker"); @@ -1837,13 +1873,14 @@ public void testLongRunning() throws Exception { assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); assertEquals(task.getTaskId(), taskId); - task1Op = "task1.Done"; + task1Output = "task1.Done"; List tasks = workflowExecutionService.getTasks(task.getTaskType(), null, 1); assertNotNull(tasks); assertEquals(1, tasks.size()); - assertEquals(wfid, task.getWorkflowInstanceId()); + assertEquals(workflowId, task.getWorkflowInstanceId()); + task = tasks.get(0); - task.getOutputData().put("op", task1Op); + task.getOutputData().put("op", task1Output); task.setStatus(COMPLETED); workflowExecutionService.updateTask(task); @@ -1852,7 +1889,7 @@ public void testLongRunning() throws Exception { assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); String task2Input = (String) task.getInputData().get("tp2"); assertNotNull(task2Input); - assertEquals(task1Op, task2Input); + assertEquals(task1Output, task2Input); task2Input = (String) task.getInputData().get("tp1"); assertNotNull(task2Input); @@ -1862,19 +1899,17 @@ public void testLongRunning() throws Exception { task.setReasonForIncompletion("unit test failure"); workflowExecutionService.updateTask(task); + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertEquals(WorkflowStatus.COMPLETED, workflow.getStatus()); - es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(WorkflowStatus.COMPLETED, es.getStatus()); - tasks = es.getTasks(); + tasks = workflow.getTasks(); assertNotNull(tasks); assertEquals(2, tasks.size()); - - } @Test - public void testResetWorkflowInProgressTasks() throws Exception { + public void testResetWorkflowInProgressTasks() { clearWorkflows(); @@ -1988,7 +2023,7 @@ public void testResetWorkflowInProgressTasks() throws Exception { @Test - public void testConcurrentWorkflowExecutions() throws Exception { + public void testConcurrentWorkflowExecutions() { int count = 3; @@ -2066,7 +2101,7 @@ public void testConcurrentWorkflowExecutions() throws Exception { } @Test - public void testCaseStatements() throws Exception { + public void testCaseStatements() { createConditionalWF(); String correlationId = "testCaseStatements: " + System.currentTimeMillis(); @@ -2134,7 +2169,7 @@ public void testCaseStatements() throws Exception { } - private void validate(String wfid, String[] sequence, String[] executedTasks, int expectedTotalTasks) throws Exception { + private void validate(String wfid, String[] sequence, String[] executedTasks, int expectedTotalTasks) { for (int i = 0; i < sequence.length; i++) { String t = sequence[i]; Task task = getTask(t); @@ -2169,7 +2204,7 @@ private void validate(String wfid, String[] sequence, String[] executedTasks, in } - private Task getTask(String taskType) throws Exception { + private Task getTask(String taskType) { Task task; int count = 2; do { @@ -2189,7 +2224,7 @@ private Task getTask(String taskType) throws Exception { } @Test - public void testRetries() throws Exception { + public void testRetries() { String taskName = "junit_task_2"; TaskDef taskDef = metadataService.getTaskDef(taskName); @@ -2263,7 +2298,7 @@ public void testRetries() throws Exception { } @Test - public void testSuccess() throws Exception { + public void testSuccess() { WorkflowDef found = metadataService.getWorkflowDef(LINEAR_WORKFLOW_T1_T2, 1); assertNotNull(found); @@ -2373,7 +2408,7 @@ public void testSuccess() throws Exception { } @Test - public void testDeciderUpdate() throws Exception { + public void testDeciderUpdate() { WorkflowDef found = metadataService.getWorkflowDef(LINEAR_WORKFLOW_T1_T2, 1); assertNotNull(found); @@ -2405,7 +2440,7 @@ public void testDeciderUpdate() throws Exception { @Test @Ignore //Ignore for now, will improve this in the future - public void testFailurePoints() throws Exception { + public void testFailurePoints() { WorkflowDef found = metadataService.getWorkflowDef(LINEAR_WORKFLOW_T1_T2, 1); assertNotNull(found); @@ -2621,7 +2656,7 @@ public void testDeciderMix() throws Exception { } @Test - public void testFailures() throws Exception { + public void testFailures() { WorkflowDef errorWorkflow = metadataService.getWorkflowDef(FORK_JOIN_WF, 1); assertNotNull("Error workflow is not defined", errorWorkflow); @@ -2717,7 +2752,7 @@ public void testRetryWithForkJoin() throws Exception { } @Test - public void testRetry() throws Exception { + public void testRetry() { String taskName = "junit_task_1"; TaskDef taskDef = metadataService.getTaskDef(taskName); int retryCount = taskDef.getRetryCount(); @@ -2797,7 +2832,7 @@ public void testRetry() throws Exception { } @Test - public void testNonRestartartableWorkflows() throws Exception { + public void testNonRestartartableWorkflows() { String taskName = "junit_task_1"; TaskDef taskDef = metadataService.getTaskDef(taskName); taskDef.setRetryCount(0); @@ -2897,7 +2932,7 @@ public void testNonRestartartableWorkflows() throws Exception { @Test - public void testRestart() throws Exception { + public void testRestart() { String taskName = "junit_task_1"; TaskDef taskDef = metadataService.getTaskDef(taskName); taskDef.setRetryCount(0); @@ -3036,7 +3071,7 @@ public void testTimeout() throws Exception { } @Test - public void testReruns() throws Exception { + public void testReruns() { WorkflowDef found = metadataService.getWorkflowDef(LINEAR_WORKFLOW_T1_T2, 1); assertNotNull(found); @@ -3177,7 +3212,7 @@ public void testReruns() throws Exception { @Test - public void testTaskSkipping() throws Exception { + public void testTaskSkipping() { String taskName = "junit_task_1"; TaskDef taskDef = metadataService.getTaskDef(taskName); @@ -3255,7 +3290,7 @@ public void testTaskSkipping() throws Exception { } @Test - public void testPauseResume() throws Exception { + public void testPauseResume() { WorkflowDef found = metadataService.getWorkflowDef(LINEAR_WORKFLOW_T1_T2, 1); assertNotNull(found); @@ -3364,7 +3399,7 @@ public void testPauseResume() throws Exception { } @Test - public void testSubWorkflow() throws Exception { + public void testSubWorkflow() { createSubWorkflow(); WorkflowDef found = metadataService.getWorkflowDef(WF_WITH_SUB_WF, 1); @@ -3389,11 +3424,11 @@ public void testSubWorkflow() throws Exception { assertNotNull(es); assertNotNull(es.getTasks()); - task = es.getTasks().stream().filter(t -> t.getTaskType().equals(Type.SUB_WORKFLOW.name().toString())).findAny().get(); + task = es.getTasks().stream().filter(t -> t.getTaskType().equals(Type.SUB_WORKFLOW.name())).findAny().get(); assertNotNull(task); assertNotNull(task.getOutputData()); - assertNotNull("Output: " + task.getOutputData().toString() + ", status: " + task.getStatus(), task.getOutputData().get("subWorkflowId")); - String subWorkflowId = task.getOutputData().get("subWorkflowId").toString(); + assertNotNull("Output: " + task.getOutputData().toString() + ", status: " + task.getStatus(), task.getOutputData().get(SubWorkflow.SUB_WORKFLOW_ID)); + String subWorkflowId = task.getOutputData().get(SubWorkflow.SUB_WORKFLOW_ID).toString(); es = workflowExecutionService.getExecutionStatus(subWorkflowId, true); assertNotNull(es); @@ -3432,7 +3467,7 @@ public void testSubWorkflow() throws Exception { } @Test - public void testSubWorkflowFailure() throws Exception { + public void testSubWorkflowFailure() { TaskDef taskDef = metadataService.getTaskDef("junit_task_1"); assertNotNull(taskDef); @@ -3464,11 +3499,11 @@ public void testSubWorkflowFailure() throws Exception { es = workflowExecutionService.getExecutionStatus(wfId, true); assertNotNull(es); assertNotNull(es.getTasks()); - task = es.getTasks().stream().filter(t -> t.getTaskType().equals(Type.SUB_WORKFLOW.name().toString())).findAny().get(); + task = es.getTasks().stream().filter(t -> t.getTaskType().equals(Type.SUB_WORKFLOW.name())).findAny().get(); assertNotNull(task); assertNotNull(task.getOutputData()); - assertNotNull(task.getOutputData().get("subWorkflowId")); - String subWorkflowId = task.getOutputData().get("subWorkflowId").toString(); + assertNotNull(task.getOutputData().get(SubWorkflow.SUB_WORKFLOW_ID)); + String subWorkflowId = task.getOutputData().get(SubWorkflow.SUB_WORKFLOW_ID).toString(); es = workflowExecutionService.getExecutionStatus(subWorkflowId, true); assertNotNull(es); @@ -3496,7 +3531,7 @@ public void testSubWorkflowFailure() throws Exception { } @Test - public void testSubWorkflowFailureInverse() throws Exception { + public void testSubWorkflowFailureInverse() { TaskDef taskDef = metadataService.getTaskDef("junit_task_1"); assertNotNull(taskDef); @@ -3528,11 +3563,11 @@ public void testSubWorkflowFailureInverse() throws Exception { es = workflowExecutionService.getExecutionStatus(wfId, true); assertNotNull(es); assertNotNull(es.getTasks()); - task = es.getTasks().stream().filter(t -> t.getTaskType().equals(Type.SUB_WORKFLOW.name().toString())).findAny().get(); + task = es.getTasks().stream().filter(t -> t.getTaskType().equals(Type.SUB_WORKFLOW.name())).findAny().get(); assertNotNull(task); assertNotNull(task.getOutputData()); - assertNotNull(task.getOutputData().get("subWorkflowId")); - String subWorkflowId = task.getOutputData().get("subWorkflowId").toString(); + assertNotNull(task.getOutputData().get(SubWorkflow.SUB_WORKFLOW_ID)); + String subWorkflowId = task.getOutputData().get(SubWorkflow.SUB_WORKFLOW_ID).toString(); es = workflowExecutionService.getExecutionStatus(subWorkflowId, true); assertNotNull(es); @@ -3550,7 +3585,7 @@ public void testSubWorkflowFailureInverse() throws Exception { } @Test - public void testSubWorkflowRetry() throws Exception { + public void testSubWorkflowRetry() { String taskName = "junit_task_1"; TaskDef taskDef = metadataService.getTaskDef(taskName); int retryCount = metadataService.getTaskDef(taskName).getRetryCount(); @@ -3587,8 +3622,8 @@ public void testSubWorkflowRetry() throws Exception { task = workflow.getTasks().stream().filter(t -> t.getTaskType().equals(Type.SUB_WORKFLOW.name())).findAny().orElse(null); assertNotNull(task); assertNotNull(task.getOutputData()); - assertNotNull("Output: " + task.getOutputData().toString() + ", status: " + task.getStatus(), task.getOutputData().get("subWorkflowId")); - String subWorkflowId = task.getOutputData().get("subWorkflowId").toString(); + assertNotNull("Output: " + task.getOutputData().toString() + ", status: " + task.getStatus(), task.getOutputData().get(SubWorkflow.SUB_WORKFLOW_ID)); + String subWorkflowId = task.getOutputData().get(SubWorkflow.SUB_WORKFLOW_ID).toString(); workflow = workflowExecutionService.getExecutionStatus(subWorkflowId, true); assertNotNull(workflow); @@ -3656,7 +3691,7 @@ public void testSubWorkflowRetry() throws Exception { @Test - public void testWait() throws Exception { + public void testWait() { WorkflowDef workflowDef = new WorkflowDef(); workflowDef.setName("test_wait"); @@ -3697,7 +3732,7 @@ public void testWait() throws Exception { } @Test - public void testEventWorkflow() throws Exception { + public void testEventWorkflow() { TaskDef taskDef = new TaskDef(); taskDef.setName("eventX"); @@ -3745,7 +3780,7 @@ public void testEventWorkflow() throws Exception { } @Test - public void testTaskWithCallbackAfterSecondsInWorkflow() throws Exception { + public void testTaskWithCallbackAfterSecondsInWorkflow() { WorkflowDef workflowDef = metadataService.getWorkflowDef(LINEAR_WORKFLOW_T1_T2, 1); assertNotNull(workflowDef); @@ -3820,7 +3855,7 @@ public void testTaskWithCallbackAfterSecondsInWorkflow() throws Exception { } //@Test - public void testRateLimiting() throws Exception { + public void testRateLimiting() { TaskDef td = new TaskDef(); td.setName("eventX1"); @@ -3875,7 +3910,7 @@ public void testRateLimiting() throws Exception { } } - private void createSubWorkflow() throws Exception { + private void createSubWorkflow() { WorkflowTask wft1 = new WorkflowTask(); wft1.setName("junit_task_5"); @@ -3915,7 +3950,7 @@ private void createSubWorkflow() throws Exception { } - private void verify(String inputParam1, String wfid, String task1Op, boolean fail) throws Exception { + private void verify(String inputParam1, String wfid, String task1Op, boolean fail) { Task task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker"); assertNotNull(task); assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); @@ -4044,7 +4079,7 @@ private void createWFWithResponseTimeout() { metadataService.updateWorkflowDef(def); } - private String runWorkflowWithSubworkflow() throws Exception { + private String runWorkflowWithSubworkflow() { clearWorkflows(); createWorkflowDefForDomain(); @@ -4082,8 +4117,8 @@ private String runWorkflowWithSubworkflow() throws Exception { // Get the sub workflow id String subWorkflowId = null; for (Task t : workflow.getTasks()) { - if (t.getTaskType().equalsIgnoreCase("SUB_WORKFLOW")) { - subWorkflowId = t.getOutputData().get("subWorkflowId").toString(); + if (t.getTaskType().equalsIgnoreCase(SubWorkflow.NAME)) { + subWorkflowId = t.getOutputData().get(SubWorkflow.SUB_WORKFLOW_ID).toString(); } } assertNotNull(subWorkflowId); @@ -4107,11 +4142,6 @@ private String runWorkflowWithSubworkflow() throws Exception { assertEquals(WorkflowStatus.RUNNING, subWorkflow.getStatus()); assertEquals(2, subWorkflow.getTasks().size()); - workflow = workflowExecutionService.getExecutionStatus(workflowId, true); - assertNotNull(workflow); - assertEquals(WorkflowStatus.RUNNING, workflow.getStatus()); - assertEquals(2, workflow.getTasks().size()); - // Poll for second task of the sub workflow and execute it task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker"); assertNotNull(task); @@ -4179,7 +4209,7 @@ private String runAFailedForkJoinWF() throws Exception { return workflowId; } - private void printTaskStatuses(String wfid, String message) throws Exception { + private void printTaskStatuses(String wfid, String message) { Workflow wf = workflowExecutionService.getExecutionStatus(wfid, true); assertNotNull(wf); printTaskStatuses(wf, message); @@ -4187,7 +4217,7 @@ private void printTaskStatuses(String wfid, String message) throws Exception { private boolean printWFTaskDetails = false; - private void printTaskStatuses(Workflow wf, String message) throws Exception { + private void printTaskStatuses(Workflow wf, String message) { if (printWFTaskDetails) { System.out.println(message + " >>> Workflow status " + wf.getStatus().name()); wf.getTasks().forEach(t -> { diff --git a/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java b/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java index 229be587c0..78f7d24463 100644 --- a/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java +++ b/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java @@ -15,15 +15,22 @@ */ package com.netflix.conductor.tests.utils; +import com.fasterxml.jackson.databind.ObjectMapper; import com.netflix.conductor.common.run.ExternalStorageLocation; import com.netflix.conductor.common.utils.ExternalPayloadStorage; +import java.io.ByteArrayInputStream; +import java.io.IOException; import java.io.InputStream; +import java.util.HashMap; +import java.util.Map; public class MockExternalPayloadStorage implements ExternalPayloadStorage { + private ObjectMapper objectMapper = new ObjectMapper(); + @Override - public ExternalStorageLocation getExternalUri(Operation operation, PayloadType payloadType) { + public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType) { return null; } @@ -33,6 +40,27 @@ public void upload(String path, InputStream payload, long payloadSize) { @Override public InputStream download(String path) { - return null; + try { + + Map payload = getPayload(path); + String jsonString = objectMapper.writeValueAsString(payload); + return new ByteArrayInputStream(jsonString.getBytes()); + } catch (IOException e) { + return null; + } + } + + private Map getPayload(String path) { + Map stringObjectMap = new HashMap<>(); + switch (path) { + case "workflow/input": + stringObjectMap.put("param1", "p1 value"); + stringObjectMap.put("param2", "p2 value"); + break; + case "task/output": + stringObjectMap.put("op", "success_task1"); + break; + } + return stringObjectMap; } } From b9a2a80af54dec220d217d24c26bf776ea0f9628 Mon Sep 17 00:00:00 2001 From: Anoop Panicker Date: Wed, 5 Sep 2018 16:35:51 -0700 Subject: [PATCH 3/4] use external payload storage in workflow execution --- .../conductor/client/http/ClientBase.java | 51 ++- .../conductor/client/http/PayloadStorage.java | 18 +- .../conductor/client/http/TaskClient.java | 27 +- .../conductor/client/http/WorkflowClient.java | 22 +- .../client/task/WorkflowTaskCoordinator.java | 3 +- .../conductor/common/metadata/tasks/Task.java | 1 + .../common/run/ExternalStorageLocation.java | 7 +- .../common/utils/ExternalPayloadStorage.java | 4 +- .../conductor/contribs/http/TestHttpTask.java | 35 +- .../conductor/core/config/Configuration.java | 54 ++- .../conductor/core/events/EventProcessor.java | 2 +- .../conductor/core/execution/Code.java | 15 + .../core/execution/DeciderService.java | 145 +++--- .../execution/TerminateWorkflowException.java | 1 - .../core/execution/WorkflowExecutor.java | 13 +- .../mapper/ForkJoinDynamicTaskMapper.java | 1 + .../core/utils/DummyPayloadStorage.java | 2 +- .../utils/ExternalPayloadStorageUtils.java | 179 ++++++++ .../core/utils/S3PayloadStorage.java | 10 +- .../conductor/service/AdminService.java | 15 + .../conductor/service/EventService.java | 15 + .../conductor/service/ExecutionService.java | 5 +- .../conductor/service/TaskService.java | 8 +- .../service/WorkflowBulkService.java | 15 + .../conductor/service/WorkflowService.java | 20 +- .../conductor/service/utils/ServiceUtils.java | 15 + .../core/execution/TestConfiguration.java | 48 +- .../core/execution/TestDeciderOutcomes.java | 31 +- .../core/execution/TestDeciderService.java | 76 +++- .../core/execution/TestWorkflowExecutor.java | 6 +- .../mapper/ForkJoinDynamicTaskMapperTest.java | 14 +- .../ExternalPayloadStorageUtilsTest.java | 153 +++++++ core/src/test/resources/payload.json | 423 ++++++++++++++++++ .../server/resources/TaskResource.java | 4 +- .../server/resources/WorkflowResource.java | 4 +- .../conductor/config/TestConfiguration.java | 44 +- .../conductor/config/TestConfiguration.java | 40 ++ .../conductor/server/ConductorConfig.java | 44 +- .../tests/integration/End2EndTests.java | 161 +++---- .../integration/WorkflowServiceTest.java | 228 +++++++--- .../tests/utils/MockConfiguration.java | 135 ++++++ .../utils/MockExternalPayloadStorage.java | 37 +- .../conductor/tests/utils/TestModule.java | 23 +- test-harness/src/test/resources/payload.json | 423 ++++++++++++++++++ 44 files changed, 2200 insertions(+), 377 deletions(-) create mode 100644 core/src/main/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtils.java create mode 100644 core/src/test/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtilsTest.java create mode 100644 core/src/test/resources/payload.json create mode 100644 test-harness/src/test/java/com/netflix/conductor/tests/utils/MockConfiguration.java create mode 100644 test-harness/src/test/resources/payload.json diff --git a/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java b/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java index b8dd8f8363..90684d47a8 100644 --- a/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java +++ b/client/src/main/java/com/netflix/conductor/client/http/ClientBase.java @@ -36,6 +36,7 @@ import com.sun.jersey.api.client.WebResource.Builder; import com.sun.jersey.api.client.config.ClientConfig; import com.sun.jersey.api.client.config.DefaultClientConfig; +import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,8 +44,10 @@ import javax.ws.rs.core.UriBuilder; import java.io.ByteArrayInputStream; import java.io.IOException; +import java.io.InputStream; import java.net.URI; import java.util.Collection; +import java.util.Map; import java.util.function.Function; /** @@ -76,7 +79,7 @@ protected ClientBase(ClientConfig config, ClientHandler handler) { this(config, new DefaultConductorClientConfiguration(), handler); } - protected ClientBase(ClientConfig config, ConductorClientConfiguration clientConfiguration, ClientHandler handler) { + protected ClientBase(ClientConfig config, ConductorClientConfiguration clientConfiguration, ClientHandler handler) { objectMapper = new ObjectMapper(); objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); objectMapper.configure(DeserializationFeature.FAIL_ON_IGNORED_PROPERTIES, false); @@ -95,7 +98,6 @@ protected ClientBase(ClientConfig config, ConductorClientConfiguration clientCo conductorClientConfiguration = clientConfiguration; payloadStorage = new PayloadStorage(this); - } public void setRootURI(String root) { @@ -108,7 +110,6 @@ protected void delete(String url, Object... uriVariables) { protected void delete(Object[] queryParams, String url, Object... uriVariables) { URI uri = null; - ClientResponse clientResponse = null; try { uri = getURIBuilder(root + url, queryParams).build(uriVariables); client.resource(uri).delete(); @@ -127,28 +128,11 @@ protected void put(String url, Object[] queryParams, Object request, Object... u } } - /** - * @deprecated replaced by {@link #postForEntityWithRequestOnly(String, Object)} ()} - */ - @Deprecated - protected void postForEntity(String url, Object request) { - postForEntityWithRequestOnly(url, request); - } - - protected void postForEntityWithRequestOnly(String url, Object request) { Class type = null; postForEntity(url, request, null, type); } - /** - * @deprecated replaced by {@link #postForEntityWithUriVariablesOnly(String, Object...)} ()} - */ - @Deprecated - protected void postForEntity1(String url, Object... uriVariables) { - postForEntityWithUriVariablesOnly(url, uriVariables); - } - protected void postForEntityWithUriVariablesOnly(String url, Object... uriVariables) { Class type = null; postForEntity(url, null, null, type, uriVariables); @@ -215,19 +199,40 @@ private T getForEntity(String url, Object[] queryParams, Function downloadFromExternalStorage(ExternalPayloadStorage.PayloadType payloadType, String path) { + Preconditions.checkArgument(StringUtils.isNotBlank(path), "uri cannot be blank"); + ExternalStorageLocation externalStorageLocation = payloadStorage.getLocation(ExternalPayloadStorage.Operation.READ, payloadType, path); + try (InputStream inputStream = payloadStorage.download(externalStorageLocation.getUri())) { + return objectMapper.readValue(inputStream, Map.class); + } catch (IOException e) { + String errorMsg = String.format("Unable to download payload frome external storage location: %s", path); + logger.error(errorMsg, e); + throw new ConductorClientException(errorMsg, e); + } + } + private Builder getWebResourceBuilder(URI URI, Object entity) { return client.resource(URI).type(MediaType.APPLICATION_JSON).entity(entity).accept(MediaType.TEXT_PLAIN, MediaType.APPLICATION_JSON); } diff --git a/client/src/main/java/com/netflix/conductor/client/http/PayloadStorage.java b/client/src/main/java/com/netflix/conductor/client/http/PayloadStorage.java index e9d296cc21..47f74026ac 100644 --- a/client/src/main/java/com/netflix/conductor/client/http/PayloadStorage.java +++ b/client/src/main/java/com/netflix/conductor/client/http/PayloadStorage.java @@ -35,12 +35,12 @@ /** * An implementation of {@link ExternalPayloadStorage} for storing large JSON payload data. */ -public class PayloadStorage implements ExternalPayloadStorage { +class PayloadStorage implements ExternalPayloadStorage { private static final Logger logger = LoggerFactory.getLogger(PayloadStorage.class); private final ClientBase clientBase; - public PayloadStorage(ClientBase clientBase) { + PayloadStorage(ClientBase clientBase) { this.clientBase = clientBase; } @@ -49,7 +49,7 @@ public PayloadStorage(ClientBase clientBase) { * The client makes a request to the server to get the {@link ExternalStorageLocation} */ @Override - public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType) { + public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType, String path) { String uri; switch (payloadType) { case WORKFLOW_INPUT: @@ -63,7 +63,7 @@ public ExternalStorageLocation getLocation(Operation operation, PayloadType payl default: throw new ConductorClientException(String.format("Invalid payload type: %s for operation: %s", payloadType.toString(), operation.toString())); } - return clientBase.getForEntity(String.format("%s/externalstoragelocation", uri), null, ExternalStorageLocation.class); + return clientBase.getForEntity(String.format("%s/externalstoragelocation", uri), new Object[]{"path", path}, ExternalStorageLocation.class); } /** @@ -109,16 +109,16 @@ public void upload(String uri, InputStream payload, long payloadSize) { /** * Downloads the payload from the given uri. * - * @param path the location from where the object is to be downloaded + * @param uri the location from where the object is to be downloaded * @return an inputstream of the payload in the external storage * @throws ConductorClientException if the download fails due to an invalid path or an error from external storage */ @Override - public InputStream download(String path) { + public InputStream download(String uri) { HttpURLConnection connection = null; String errorMsg; try { - URL url = new URI(path).toURL(); + URL url = new URI(uri).toURL(); connection = (HttpURLConnection) url.openConnection(); connection.setDoOutput(false); @@ -132,11 +132,11 @@ public InputStream download(String path) { logger.error(errorMsg); throw new ConductorClientException(errorMsg); } catch (URISyntaxException | MalformedURLException e) { - errorMsg = String.format("Invalid path specified: %s", path); + errorMsg = String.format("Invalid uri specified: %s", uri); logger.error(errorMsg, e); throw new ConductorClientException(errorMsg, e); } catch (IOException e) { - errorMsg = String.format("Error downloading from path: %s", path); + errorMsg = String.format("Error downloading from uri: %s", uri); logger.error(errorMsg, e); throw new ConductorClientException(errorMsg, e); } finally { diff --git a/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java b/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java index cdf26f2bbf..438d9cde6c 100644 --- a/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java +++ b/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java @@ -125,7 +125,9 @@ public Task pollTask(String taskType, String workerId, String domain) { Preconditions.checkArgument(StringUtils.isNotBlank(workerId), "Worker id cannot be blank"); Object[] params = new Object[]{"workerid", workerId, "domain", domain}; - return getForEntity("tasks/poll/{taskType}", params, Task.class, taskType); + Task task = getForEntity("tasks/poll/{taskType}", params, Task.class, taskType); + populateTaskInput(task); + return task; } /** @@ -152,7 +154,9 @@ public List batchPollTasksByTaskType(String taskType, String workerId, int Preconditions.checkArgument(count > 0, "Count must be greater than 0"); Object[] params = new Object[]{"workerid", workerId, "count", count, "timeout", timeoutInMillisecond}; - return getForEntity("tasks/poll/batch/{taskType}", params, taskList, taskType); + List tasks = getForEntity("tasks/poll/batch/{taskType}", params, taskList, taskType); + tasks.forEach(this::populateTaskInput); + return tasks; } /** @@ -180,7 +184,22 @@ public List batchPollTasksInDomain(String taskType, String domain, String Preconditions.checkArgument(count > 0, "Count must be greater than 0"); Object[] params = new Object[]{"workerid", workerId, "count", count, "timeout", timeoutInMillisecond, "domain", domain}; - return getForEntity("tasks/poll/batch/{taskType}", params, taskList, taskType); + List tasks = getForEntity("tasks/poll/batch/{taskType}", params, taskList, taskType); + tasks.forEach(this::populateTaskInput); + return tasks; + } + + /** + * Populates the task input from external payload storage if the external storage path is specified. + * + * @param task the task for which the input is to be populated. + */ + private void populateTaskInput(Task task) { + if (StringUtils.isNotBlank(task.getExternalInputPayloadStoragePath())) { + WorkflowTaskMetrics.incrementExternalPayloadUsedCount(task.getTaskDefName(), ExternalPayloadStorage.Operation.READ.name(), ExternalPayloadStorage.PayloadType.TASK_INPUT.name()); + task.setInputData(downloadFromExternalStorage(ExternalPayloadStorage.PayloadType.TASK_INPUT, task.getExternalInputPayloadStoragePath())); + task.setExternalInputPayloadStoragePath(null); + } } /** @@ -448,6 +467,6 @@ public void unregisterTaskDef(String taskType) { @Deprecated public void registerTaskDefs(List taskDefs) { Preconditions.checkNotNull(taskDefs, "Task defs cannot be null"); - postForEntity("metadata/taskdefs", taskDefs); + postForEntityWithRequestOnly("metadata/taskdefs", taskDefs); } } diff --git a/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java b/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java index 648f32aa59..1fb9ca6b49 100644 --- a/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java +++ b/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java @@ -118,7 +118,7 @@ public List getAllWorkflowDefs() { @Deprecated public void registerWorkflow(WorkflowDef workflowDef) { Preconditions.checkNotNull(workflowDef, "Worfklow definition cannot be null"); - postForEntity("metadata/workflow", workflowDef); + postForEntityWithRequestOnly("metadata/workflow", workflowDef); } /** @@ -211,7 +211,9 @@ public Workflow getExecutionStatus(String workflowId, boolean includeTasks) { */ public Workflow getWorkflow(String workflowId, boolean includeTasks) { Preconditions.checkArgument(StringUtils.isNotBlank(workflowId), "workflow id cannot be blank"); - return getForEntity("workflow/{workflowId}", new Object[]{"includeTasks", includeTasks}, Workflow.class, workflowId); + Workflow workflow = getForEntity("workflow/{workflowId}", new Object[]{"includeTasks", includeTasks}, Workflow.class, workflowId); + populateWorkflowOutput(workflow); + return workflow; } /** @@ -228,8 +230,22 @@ public List getWorkflows(String name, String correlationId, boolean in Preconditions.checkArgument(StringUtils.isNotBlank(correlationId), "correlationId cannot be blank"); Object[] params = new Object[]{"includeClosed", includeClosed, "includeTasks", includeTasks}; - return getForEntity("workflow/{name}/correlated/{correlationId}", params, new GenericType>() { + List workflows = getForEntity("workflow/{name}/correlated/{correlationId}", params, new GenericType>() { }, name, correlationId); + workflows.forEach(this::populateWorkflowOutput); + return workflows; + } + + /** + * Populates the workflow output from external payload storage if the external storage path is specified. + * + * @param workflow the workflow for which the output is to be populated. + */ + private void populateWorkflowOutput(Workflow workflow) { + if (StringUtils.isNotBlank(workflow.getExternalOutputPayloadStoragePath())) { + WorkflowTaskMetrics.incrementExternalPayloadUsedCount(workflow.getWorkflowType(), ExternalPayloadStorage.Operation.READ.name(), ExternalPayloadStorage.PayloadType.WORKFLOW_OUTPUT.name()); + workflow.setOutput(downloadFromExternalStorage(ExternalPayloadStorage.PayloadType.WORKFLOW_OUTPUT, workflow.getExternalOutputPayloadStoragePath())); + } } /** diff --git a/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java b/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java index 6e2dcfd47c..c72f5feb2e 100644 --- a/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java +++ b/client/src/main/java/com/netflix/conductor/client/task/WorkflowTaskCoordinator.java @@ -1,4 +1,4 @@ -/** +/* * Copyright 2016 Netflix, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); @@ -379,7 +379,6 @@ private void execute(Worker worker, Task task) { TaskResult result = null; try { logger.debug("Executing task {} in worker {} at {}", task, worker.getClass().getSimpleName(), worker.getIdentity()); - // TODO result = worker.execute(task); result.setWorkflowInstanceId(task.getWorkflowInstanceId()); result.setTaskId(task.getTaskId()); diff --git a/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java b/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java index 7bdd0b60af..7455398254 100644 --- a/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java +++ b/common/src/main/java/com/netflix/conductor/common/metadata/tasks/Task.java @@ -615,6 +615,7 @@ public Task copy() { copy.setRateLimitPerFrequency(rateLimitPerFrequency); copy.setRateLimitFrequencyInSeconds(rateLimitFrequencyInSeconds); copy.setExternalInputPayloadStoragePath(externalInputPayloadStoragePath); + copy.setExternalOutputPayloadStoragePath(externalOutputPayloadStoragePath); return copy; } diff --git a/common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java b/common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java index 81e0f935a0..3bf3faca94 100644 --- a/common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java +++ b/common/src/main/java/com/netflix/conductor/common/run/ExternalStorageLocation.java @@ -18,11 +18,12 @@ /** * Describes the location where the JSON payload is stored in external storage. + *

      *
    • * The location is described using the following fields: - *
        uri: The uri of the json file in external storage
      - *
        path: The relative path of the file in external storage
      - *
    • + *
    • uri: The uri of the json file in external storage
    • + *
    • path: The relative path of the file in external storage
    • + *
    */ public class ExternalStorageLocation { diff --git a/common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java b/common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java index 7e17e12b3b..4497dbecc3 100644 --- a/common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java +++ b/common/src/main/java/com/netflix/conductor/common/utils/ExternalPayloadStorage.java @@ -34,9 +34,11 @@ enum PayloadType {WORKFLOW_INPUT, WORKFLOW_OUTPUT, TASK_INPUT, TASK_OUTPUT} * * @param operation the type of {@link Operation} to be performed with the uri * @param payloadType the {@link PayloadType} that is being accessed at the uri + * @param path (optional) the relative path for which the external storage location object is to be populated. + * If path is not specified, it will be computed and populated. * @return a {@link ExternalStorageLocation} object which contains the uri and the path for the json payload */ - ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType); + ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType, String path); /** * Upload a json payload to the specified external storage location. diff --git a/contribs/src/test/java/com/netflix/conductor/contribs/http/TestHttpTask.java b/contribs/src/test/java/com/netflix/conductor/contribs/http/TestHttpTask.java index e68f93f15d..1f5b604845 100644 --- a/contribs/src/test/java/com/netflix/conductor/contribs/http/TestHttpTask.java +++ b/contribs/src/test/java/com/netflix/conductor/contribs/http/TestHttpTask.java @@ -26,7 +26,6 @@ import com.netflix.conductor.common.metadata.workflow.WorkflowTask; import com.netflix.conductor.common.metadata.workflow.WorkflowTask.Type; import com.netflix.conductor.common.run.Workflow; -import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.contribs.http.HttpTask.Input; import com.netflix.conductor.core.config.Configuration; import com.netflix.conductor.core.execution.DeciderService; @@ -43,6 +42,7 @@ import com.netflix.conductor.core.execution.mapper.TaskMapper; import com.netflix.conductor.core.execution.mapper.UserDefinedTaskMapper; import com.netflix.conductor.core.execution.mapper.WaitTaskMapper; +import com.netflix.conductor.core.utils.ExternalPayloadStorageUtils; import com.netflix.conductor.dao.MetadataDAO; import com.netflix.conductor.dao.QueueDAO; import org.eclipse.jetty.server.Request; @@ -89,7 +89,9 @@ public class TestHttpTask { private HttpTask httpTask; - private WorkflowExecutor executor = mock(WorkflowExecutor.class); + private WorkflowExecutor workflowExecutor; + + private Configuration config; private Workflow workflow = new Workflow(); @@ -125,7 +127,8 @@ public static void cleanup() { @Before public void setup() { RestClientManager rcm = new RestClientManager(); - Configuration config = mock(Configuration.class); + workflowExecutor = mock(WorkflowExecutor.class); + config = mock(Configuration.class); when(config.getServerId()).thenReturn("test_server_id"); httpTask = new HttpTask(rcm, config); } @@ -143,7 +146,7 @@ public void testPost() { input.setMethod("POST"); task.getInputData().put(HttpTask.REQUEST_PARAMETER_NAME, input); - httpTask.start(workflow, task, executor); + httpTask.start(workflow, task, workflowExecutor); assertEquals(task.getReasonForIncompletion(), Task.Status.COMPLETED, task.getStatus()); Map hr = (Map) task.getOutputData().get("response"); Object response = hr.get("body"); @@ -170,7 +173,7 @@ public void testPostNoContent() { input.setMethod("POST"); task.getInputData().put(HttpTask.REQUEST_PARAMETER_NAME, input); - httpTask.start(workflow, task, executor); + httpTask.start(workflow, task, workflowExecutor); assertEquals(task.getReasonForIncompletion(), Task.Status.COMPLETED, task.getStatus()); Map hr = (Map) task.getOutputData().get("response"); Object response = hr.get("body"); @@ -187,13 +190,13 @@ public void testFailure() { input.setMethod("GET"); task.getInputData().put(HttpTask.REQUEST_PARAMETER_NAME, input); - httpTask.start(workflow, task, executor); + httpTask.start(workflow, task, workflowExecutor); assertEquals("Task output: " + task.getOutputData(), Task.Status.FAILED, task.getStatus()); assertEquals(ERROR_RESPONSE, task.getReasonForIncompletion()); task.setStatus(Status.SCHEDULED); task.getInputData().remove(HttpTask.REQUEST_PARAMETER_NAME); - httpTask.start(workflow, task, executor); + httpTask.start(workflow, task, workflowExecutor); assertEquals(Task.Status.FAILED, task.getStatus()); assertEquals(HttpTask.MISSING_REQUEST, task.getReasonForIncompletion()); } @@ -207,7 +210,7 @@ public void testTextGET() { input.setMethod("GET"); task.getInputData().put(HttpTask.REQUEST_PARAMETER_NAME, input); - httpTask.start(workflow, task, executor); + httpTask.start(workflow, task, workflowExecutor); Map hr = (Map) task.getOutputData().get("response"); Object response = hr.get("body"); assertEquals(Task.Status.COMPLETED, task.getStatus()); @@ -223,7 +226,7 @@ public void testNumberGET() { input.setMethod("GET"); task.getInputData().put(HttpTask.REQUEST_PARAMETER_NAME, input); - httpTask.start(workflow, task, executor); + httpTask.start(workflow, task, workflowExecutor); Map hr = (Map) task.getOutputData().get("response"); Object response = hr.get("body"); assertEquals(Task.Status.COMPLETED, task.getStatus()); @@ -240,7 +243,7 @@ public void testJsonGET() throws Exception { input.setMethod("GET"); task.getInputData().put(HttpTask.REQUEST_PARAMETER_NAME, input); - httpTask.start(workflow, task, executor); + httpTask.start(workflow, task, workflowExecutor); Map hr = (Map) task.getOutputData().get("response"); Object response = hr.get("body"); assertEquals(Task.Status.COMPLETED, task.getStatus()); @@ -259,7 +262,7 @@ public void testExecute() { task.getInputData().put(HttpTask.REQUEST_PARAMETER_NAME, input); task.setStatus(Status.SCHEDULED); task.setScheduledTime(0); - boolean executed = httpTask.execute(workflow, task, executor); + boolean executed = httpTask.execute(workflow, task, workflowExecutor); assertFalse(executed); } @@ -271,7 +274,7 @@ public void testOptional() { input.setMethod("GET"); task.getInputData().put(HttpTask.REQUEST_PARAMETER_NAME, input); - httpTask.start(workflow, task, executor); + httpTask.start(workflow, task, workflowExecutor); assertEquals("Task output: " + task.getOutputData(), Task.Status.FAILED, task.getStatus()); assertEquals(ERROR_RESPONSE, task.getReasonForIncompletion()); assertTrue(!task.getStatus().isSuccessful()); @@ -279,7 +282,7 @@ public void testOptional() { task.setStatus(Status.SCHEDULED); task.getInputData().remove(HttpTask.REQUEST_PARAMETER_NAME); task.setReferenceTaskName("t1"); - httpTask.start(workflow, task, executor); + httpTask.start(workflow, task, workflowExecutor); assertEquals(Task.Status.FAILED, task.getStatus()); assertEquals(HttpTask.MISSING_REQUEST, task.getReasonForIncompletion()); assertTrue(!task.getStatus().isSuccessful()); @@ -296,7 +299,7 @@ public void testOptional() { def.getTasks().add(wft); MetadataDAO metadataDAO = mock(MetadataDAO.class); QueueDAO queueDAO = mock(QueueDAO.class); - ExternalPayloadStorage externalPayloadStorage = mock(ExternalPayloadStorage.class); + ExternalPayloadStorageUtils externalPayloadStorageUtils = mock(ExternalPayloadStorageUtils.class); ParametersUtils parametersUtils = mock(ParametersUtils.class); Map taskMappers = new HashMap<>(); taskMappers.put("DECISION", new DecisionTaskMapper()); @@ -309,7 +312,7 @@ public void testOptional() { taskMappers.put("SUB_WORKFLOW", new SubWorkflowTaskMapper(parametersUtils, metadataDAO)); taskMappers.put("EVENT", new EventTaskMapper(parametersUtils)); taskMappers.put("WAIT", new WaitTaskMapper(parametersUtils)); - new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorage, taskMappers).decide(workflow, def); + new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorageUtils, taskMappers).decide(workflow, def); System.out.println(workflow.getTasks()); System.out.println(workflow.getStatus()); @@ -325,7 +328,7 @@ public void testOAuth() { input.setOauthConsumerSecret("someSecret"); task.getInputData().put(HttpTask.REQUEST_PARAMETER_NAME, input); - httpTask.start(workflow, task, executor); + httpTask.start(workflow, task, workflowExecutor); Map response = (Map) task.getOutputData().get("response"); Map body = (Map) response.get("body"); diff --git a/core/src/main/java/com/netflix/conductor/core/config/Configuration.java b/core/src/main/java/com/netflix/conductor/core/config/Configuration.java index ddf4d3c9d7..ab2c795e62 100644 --- a/core/src/main/java/com/netflix/conductor/core/config/Configuration.java +++ b/core/src/main/java/com/netflix/conductor/core/config/Configuration.java @@ -1,4 +1,4 @@ -/** +/* * Copyright 2016 Netflix, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); @@ -13,9 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -/** - * - */ + package com.netflix.conductor.core.config; import com.google.inject.AbstractModule; @@ -85,6 +83,54 @@ public interface Configuration { */ String getAvailabilityZone(); + /** + * + * @return The threshold of the workflow input payload size in KB beyond which the payload will be stored in {@link com.netflix.conductor.common.utils.ExternalPayloadStorage} + */ + Long getWorkflowInputPayloadSizeThresholdKB(); + + /** + * + * @return The maximum threshold of the workflow input payload size in KB beyond which input will be rejected and the workflow will be marked as FAILED + */ + Long getMaxWorkflowInputPayloadSizeThresholdKB(); + + /** + * + * @return The threshold of the workflow output payload size in KB beyond which the payload will be stored in {@link com.netflix.conductor.common.utils.ExternalPayloadStorage} + */ + Long getWorkflowOutputPayloadSizeThresholdKB(); + + /** + * + * @return The maximum threshold of the workflow output payload size in KB beyond which output will be rejected and the workflow will be marked as FAILED + */ + Long getMaxWorkflowOutputPayloadSizeThresholdKB(); + + /** + * + * @return The threshold of the task input payload size in KB beyond which the payload will be stored in {@link com.netflix.conductor.common.utils.ExternalPayloadStorage} + */ + Long getTaskInputPayloadSizeThresholdKB(); + + /** + * + * @return The maximum threshold of the task input payload size in KB beyond which the task input will be rejected and the task will be marked as FAILED_WITH_TERMINAL_ERROR + */ + Long getMaxTaskInputPayloadSizeThresholdKB(); + + /** + * + * @return The threshold of the task output payload size in KB beyond which the payload will be stored in {@link com.netflix.conductor.common.utils.ExternalPayloadStorage} + */ + Long getTaskOutputPayloadSizeThresholdKB(); + + /** + * + * @return The maximum threshold of the task output payload size in KB beyond which the task input will be rejected and the task will be marked as FAILED_WITH_TERMINAL_ERROR + */ + Long getMaxTaskOutputPayloadSizeThresholdKB(); + /** * * @param name Name of the property diff --git a/core/src/main/java/com/netflix/conductor/core/events/EventProcessor.java b/core/src/main/java/com/netflix/conductor/core/events/EventProcessor.java index 6765db179a..528380f856 100644 --- a/core/src/main/java/com/netflix/conductor/core/events/EventProcessor.java +++ b/core/src/main/java/com/netflix/conductor/core/events/EventProcessor.java @@ -143,7 +143,7 @@ private void listen(ObservableQueue queue) { queue.observe().subscribe((Message msg) -> handle(queue, msg)); } - @SuppressWarnings({"unchecked", "ToArrayCallWithZeroLengthArrayArgument"}) + @SuppressWarnings({"unchecked"}) private void handle(ObservableQueue queue, Message msg) { try { executionService.addMessage(queue.getName(), msg); diff --git a/core/src/main/java/com/netflix/conductor/core/execution/Code.java b/core/src/main/java/com/netflix/conductor/core/execution/Code.java index 808efce7f9..18422aad0a 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/Code.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/Code.java @@ -1,3 +1,18 @@ +/* + * Copyright 2016 Netflix, Inc. + *

    + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.conductor.core.execution; import java.util.HashMap; diff --git a/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java b/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java index a38da0c00a..5ccc8c0860 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java @@ -1,4 +1,4 @@ -/** +/* * Copyright 2016 Netflix, Inc. *

    * Licensed under the Apache License, Version 2.0 (the "License"); @@ -13,13 +13,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -/** - * - */ package com.netflix.conductor.core.execution; -import com.amazonaws.util.IOUtils; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.netflix.conductor.common.metadata.tasks.Task; import com.netflix.conductor.common.metadata.tasks.Task.Status; @@ -32,6 +27,7 @@ import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.core.execution.mapper.TaskMapper; import com.netflix.conductor.core.execution.mapper.TaskMapperContext; +import com.netflix.conductor.core.utils.ExternalPayloadStorageUtils; import com.netflix.conductor.core.utils.IDGenerator; import com.netflix.conductor.core.utils.QueueUtils; import com.netflix.conductor.dao.MetadataDAO; @@ -41,10 +37,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; import javax.inject.Inject; import javax.inject.Named; -import java.io.IOException; -import java.io.InputStream; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; @@ -82,20 +77,21 @@ public class DeciderService { private final Map taskMappers; - private final ExternalPayloadStorage externalPayloadStorage; + private final ExternalPayloadStorageUtils externalPayloadStorageUtils; - private ObjectMapper objectMapper = new ObjectMapper(); @SuppressWarnings("ConstantConditions") private final Predicate isNonPendingTask = task -> !task.isRetried() && !task.getStatus().equals(SKIPPED) && !task.isExecuted() || SystemTaskType.isBuiltIn(task.getTaskType()); @Inject - public DeciderService(MetadataDAO metadataDAO, ParametersUtils parametersUtils, QueueDAO queueDAO, ExternalPayloadStorage externalPayloadStorage, @Named("TaskMappers") Map taskMappers) { + public DeciderService(MetadataDAO metadataDAO, ParametersUtils parametersUtils, QueueDAO queueDAO, + ExternalPayloadStorageUtils externalPayloadStorageUtils, + @Named("TaskMappers") Map taskMappers) { this.metadataDAO = metadataDAO; this.queueDAO = queueDAO; this.parametersUtils = parametersUtils; this.taskMappers = taskMappers; - this.externalPayloadStorage = externalPayloadStorage; + this.externalPayloadStorageUtils = externalPayloadStorageUtils; } //QQ public method validation of the input params @@ -193,10 +189,10 @@ private DeciderOutcome decide(final WorkflowDef workflowDef, final Workflow work List nextTasks = getNextTask(workflowDef, workflow, pendingTask); nextTasks.forEach(nextTask -> tasksToBeScheduled.putIfAbsent(nextTask.getReferenceTaskName(), nextTask)); outcome.tasksToBeUpdated.add(pendingTask); - logger.debug("Scheduling Tasks from {}, next = {}", pendingTask.getTaskDefName(), + logger.debug("Scheduling Tasks from {}, next = {} for workflow: {}", pendingTask.getTaskDefName(), nextTasks.stream() .map(Task::getTaskDefName) - .collect(Collectors.toList())); + .collect(Collectors.toList()), workflow.getWorkflowId()); } } @@ -205,12 +201,12 @@ private DeciderOutcome decide(final WorkflowDef workflowDef, final Workflow work .filter(task -> !executedTaskRefNames.contains(task.getReferenceTaskName())) .collect(Collectors.toList()); if (!unScheduledTasks.isEmpty()) { - logger.debug("Scheduling Tasks {} ", unScheduledTasks.stream() + logger.debug("Scheduling Tasks {} for workflow: {}", unScheduledTasks.stream() .map(Task::getTaskDefName) - .collect(Collectors.toList())); + .collect(Collectors.toList()), workflow.getWorkflowId()); outcome.tasksToBeScheduled.addAll(unScheduledTasks); } - updateOutput(workflowDef, workflow); + updateOutput(workflowDef, workflow, null); if (outcome.tasksToBeScheduled.isEmpty() && checkForWorkflowCompletion(workflowDef, workflow)) { logger.debug("Marking workflow as complete. workflow=" + workflow.getWorkflowId() + ", tasks=" + workflow.getTasks()); outcome.isComplete = true; @@ -260,21 +256,31 @@ private List startWorkflow(Workflow workflow, WorkflowDef workflowDef) thr return Collections.singletonList(rerunFromTask); } - private void updateOutput(final WorkflowDef def, final Workflow workflow) { - + void updateOutput(final WorkflowDef def, final Workflow workflow, @Nullable Task task) { List allTasks = workflow.getTasks(); if (allTasks.isEmpty()) { return; } + Map output; Task last; - last = allTasks.get(allTasks.size() - 1); - Map output = last.getOutputData(); + if (task != null) { + last = task; + } else { + last = allTasks.get(allTasks.size() - 1); + } if (!def.getOutputParameters().isEmpty()) { - output = parametersUtils.getTaskInput(def.getOutputParameters(), workflow, null, null); + Workflow workflowInstance = populateWorkflowAndTaskData(workflow); + output = parametersUtils.getTaskInput(def.getOutputParameters(), workflowInstance, null, null); + } else if (StringUtils.isNotBlank(last.getExternalOutputPayloadStoragePath())) { + output = externalPayloadStorageUtils.downloadPayload(last.getExternalOutputPayloadStoragePath()); + } else { + output = last.getOutputData(); } + workflow.setOutput(output); + externalPayloadStorageUtils.verifyAndUpload(workflow, ExternalPayloadStorage.PayloadType.WORKFLOW_OUTPUT); } private boolean checkForWorkflowCompletion(final WorkflowDef def, final Workflow workflow) throws TerminateWorkflowException { @@ -302,15 +308,10 @@ private boolean checkForWorkflowCompletion(final WorkflowDef def, final Workflow return next != null && !taskStatusMap.containsKey(next); }).collect(Collectors.toList()).isEmpty(); - if (allCompletedSuccessfully && noPendingTasks && noPendingSchedule) { - return true; - } - - return false; + return allCompletedSuccessfully && noPendingTasks && noPendingSchedule; } - @VisibleForTesting - List getNextTask(WorkflowDef def, Workflow workflow, Task task) { + private List getNextTask(WorkflowDef def, Workflow workflow, Task task) { // Get the following task after the last completed task if (SystemTaskType.is(task.getTaskType()) && SystemTaskType.DECISION.name().equals(task.getTaskType())) { @@ -373,15 +374,46 @@ Task retry(TaskDef taskDefinition, WorkflowTask workflowTask, Task task, Workflo rescheduled.setStatus(SCHEDULED); rescheduled.setPollCount(0); rescheduled.setInputData(new HashMap<>()); - rescheduled.getInputData().putAll(task.getInputData()); + if (StringUtils.isNotBlank(task.getExternalInputPayloadStoragePath())) { + rescheduled.setExternalInputPayloadStoragePath(task.getExternalInputPayloadStoragePath()); + } else { + rescheduled.getInputData().putAll(task.getInputData()); + } if (workflowTask != null && workflow.getSchemaVersion() > 1) { - Map taskInput = parametersUtils.getTaskInputV2(workflowTask.getInputParameters(), workflow, rescheduled.getTaskId(), taskDefinition); + Workflow workflowInstance = populateWorkflowAndTaskData(workflow); + Map taskInput = parametersUtils.getTaskInputV2(workflowTask.getInputParameters(), workflowInstance, rescheduled.getTaskId(), taskDefinition); rescheduled.getInputData().putAll(taskInput); } + externalPayloadStorageUtils.verifyAndUpload(rescheduled, ExternalPayloadStorage.PayloadType.TASK_INPUT); //for the schema version 1, we do not have to recompute the inputs return rescheduled; } + @VisibleForTesting + Workflow populateWorkflowAndTaskData(Workflow workflow) { + Workflow workflowInstance = workflow.copy(); + + if (StringUtils.isNotBlank(workflow.getExternalInputPayloadStoragePath())) { + // download the workflow input from external storage here and plug it into the workflow + Map workflowInputParams = externalPayloadStorageUtils.downloadPayload(workflow.getExternalInputPayloadStoragePath()); + workflowInstance.setInput(workflowInputParams); + } + + workflowInstance.getTasks().stream() + .filter(task -> StringUtils.isNotBlank(task.getExternalInputPayloadStoragePath()) || StringUtils.isNotBlank(task.getExternalOutputPayloadStoragePath())) + .forEach(task -> { + if (StringUtils.isNotBlank(task.getExternalOutputPayloadStoragePath())) { + task.setOutputData(externalPayloadStorageUtils.downloadPayload(task.getExternalOutputPayloadStoragePath())); + task.setExternalOutputPayloadStoragePath(null); + } + if (StringUtils.isNotBlank(task.getExternalInputPayloadStoragePath())) { + task.setInputData(externalPayloadStorageUtils.downloadPayload(task.getExternalInputPayloadStoragePath())); + task.setExternalInputPayloadStoragePath(null); + } + }); + return workflowInstance; + } + @VisibleForTesting void checkForTimeout(TaskDef taskDef, Task task) { @@ -462,32 +494,12 @@ public List getTasksToBeScheduled(WorkflowDef workflowDef, Workflow workfl return getTasksToBeScheduled(workflowDef, workflow, taskToSchedule, retryCount, null); } - public List getTasksToBeScheduled(WorkflowDef workflowDefinition, Workflow workflowInstance, + public List getTasksToBeScheduled(WorkflowDef workflowDefinition, Workflow workflow, WorkflowTask taskToSchedule, int retryCount, String retriedTaskId) { - Workflow workflow = workflowInstance.copy(); - - if (StringUtils.isNotBlank(workflowInstance.getExternalInputPayloadStoragePath())) { - // download the workflow input from external storage here and plug it into the workflow - Map workflowInputParams = downloadFromExternalStorage(workflowInstance.getExternalInputPayloadStoragePath()); - workflow.setInput(workflowInputParams); - } - - workflow.getTasks().stream() - .filter(task -> StringUtils.isNotBlank(task.getExternalInputPayloadStoragePath()) || StringUtils.isNotBlank(task.getExternalOutputPayloadStoragePath())) - .forEach(task -> { - if (StringUtils.isNotBlank(task.getExternalOutputPayloadStoragePath())) { - task.setOutputData(downloadFromExternalStorage(task.getExternalOutputPayloadStoragePath())); - task.setExternalOutputPayloadStoragePath(null); - } - if (StringUtils.isNotBlank(task.getExternalInputPayloadStoragePath())) { - task.setInputData(downloadFromExternalStorage(task.getExternalInputPayloadStoragePath())); - task.setExternalInputPayloadStoragePath(null); - } - }); - + Workflow workflowInstance = populateWorkflowAndTaskData(workflow); Map input = parametersUtils.getTaskInput(taskToSchedule.getInputParameters(), - workflow, null, null); + workflowInstance, null, null); Type taskType = Type.USER_DEFINED; String type = taskToSchedule.getType(); @@ -496,7 +508,7 @@ public List getTasksToBeScheduled(WorkflowDef workflowDefinition, Workflow } // get in progress tasks for this workflow instance - List inProgressTasks = workflow.getTasks().stream() + List inProgressTasks = workflowInstance.getTasks().stream() .filter(runningTask -> runningTask.getStatus().equals(Status.IN_PROGRESS)) .map(Task::getReferenceTaskName) .collect(Collectors.toList()); @@ -507,7 +519,7 @@ public List getTasksToBeScheduled(WorkflowDef workflowDefinition, Workflow TaskMapperContext taskMapperContext = TaskMapperContext.newBuilder() .withWorkflowDefinition(workflowDefinition) - .withWorkflowInstance(workflow) + .withWorkflowInstance(workflowInstance) .withTaskDefinition(taskDef) .withTaskToSchedule(taskToSchedule) .withTaskInput(input) @@ -520,19 +532,11 @@ public List getTasksToBeScheduled(WorkflowDef workflowDefinition, Workflow // for static forks, each branch of the fork creates a join task upon completion // for dynamic forks, a join task is created with the fork and also with each branch of the fork // a new task must only be scheduled if a task with the same reference name is not in progress for this workflow instance - return taskMappers.get(taskType.name()).getMappedTasks(taskMapperContext).stream() + List tasks = taskMappers.get(taskType.name()).getMappedTasks(taskMapperContext).stream() .filter(task -> !inProgressTasks.contains(task.getReferenceTaskName())) .collect(Collectors.toList()); - } - - @SuppressWarnings("unchecked") - private Map downloadFromExternalStorage(String path) { - try (InputStream inputStream = externalPayloadStorage.download(path)) { - return objectMapper.readValue(IOUtils.toString(inputStream), Map.class); - } catch (IOException e) { - logger.error("Unable to download payload from external storage path: {}", path, e); - throw new ApplicationException(ApplicationException.Code.INTERNAL_ERROR, e); - } + tasks.forEach(task -> externalPayloadStorageUtils.verifyAndUpload(task, ExternalPayloadStorage.PayloadType.TASK_INPUT)); + return tasks; } private boolean isTaskSkipped(WorkflowTask taskToSchedule, Workflow workflow) { @@ -550,11 +554,13 @@ private boolean isTaskSkipped(WorkflowTask taskToSchedule, Workflow workflow) { } catch (Exception e) { throw new TerminateWorkflowException(e.getMessage()); } - } + private void populateTaskData(Workflow workflow) { - public static class DeciderOutcome { + } + + static class DeciderOutcome { List tasksToBeScheduled = new LinkedList<>(); @@ -566,6 +572,5 @@ public static class DeciderOutcome { private DeciderOutcome() { } - } } diff --git a/core/src/main/java/com/netflix/conductor/core/execution/TerminateWorkflowException.java b/core/src/main/java/com/netflix/conductor/core/execution/TerminateWorkflowException.java index 70b5cc971b..4af554747f 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/TerminateWorkflowException.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/TerminateWorkflowException.java @@ -44,5 +44,4 @@ public TerminateWorkflowException(String reason, WorkflowStatus workflowStatus, this.workflowStatus = workflowStatus; this.task = task; } - } \ No newline at end of file diff --git a/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java b/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java index 96bf49013d..49d602892e 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java @@ -1,4 +1,4 @@ -/** +/* * Copyright 2016 Netflix, Inc. *

    * Licensed under the Apache License, Version 2.0 (the "License"); @@ -13,9 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -/** - * - */ + package com.netflix.conductor.core.execution; import com.google.common.annotations.VisibleForTesting; @@ -366,6 +364,7 @@ void completeWorkflow(Workflow wf) { workflow.setStatus(WorkflowStatus.COMPLETED); workflow.setOutput(wf.getOutput()); + workflow.setExternalOutputPayloadStoragePath(wf.getExternalOutputPayloadStoragePath()); executionDAO.updateWorkflow(workflow); logger.debug("Completed workflow execution for {}", wf.getWorkflowId()); executionDAO.updateTasks(wf.getTasks()); @@ -524,11 +523,7 @@ public void updateTask(TaskResult taskResult) { taskByRefName.setWorkerId(task.getWorkerId()); taskByRefName.setCallbackAfterSeconds(task.getCallbackAfterSeconds()); WorkflowDef workflowDef = metadataDAO.get(workflowInstance.getWorkflowType(), workflowInstance.getVersion()); - Map outputData = task.getOutputData(); - if (!workflowDef.getOutputParameters().isEmpty()) { - outputData = parametersUtils.getTaskInput(workflowDef.getOutputParameters(), workflowInstance, null, null); - } - workflowInstance.setOutput(outputData); + deciderService.updateOutput(workflowDef, workflowInstance, task); } executionDAO.updateWorkflow(workflowInstance); logger.debug("Task: {} has a {} status and the Workflow has been updated with failed task reference", task, task.getStatus()); diff --git a/core/src/main/java/com/netflix/conductor/core/execution/mapper/ForkJoinDynamicTaskMapper.java b/core/src/main/java/com/netflix/conductor/core/execution/mapper/ForkJoinDynamicTaskMapper.java index cccb760644..ae5b2f539f 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/mapper/ForkJoinDynamicTaskMapper.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/mapper/ForkJoinDynamicTaskMapper.java @@ -225,6 +225,7 @@ Task createJoinTask(Workflow workflowInstance, WorkflowTask joinWorkflowTask, Ha * @throws TerminateWorkflowException : In case of input parameters of the dynamic fork tasks not represented as {@link Map} * @return: a {@link Pair} representing the list of dynamic fork tasks in {@link Pair#getLeft()} and the input for the dynamic fork tasks in {@link Pair#getRight()} */ + @SuppressWarnings("unchecked") @VisibleForTesting Pair, Map>> getDynamicForkTasksAndInput(WorkflowTask taskToSchedule, Workflow workflowInstance, String dynamicForkTaskParam) throws TerminateWorkflowException { diff --git a/core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java b/core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java index debf546da3..2f28522f79 100644 --- a/core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java +++ b/core/src/main/java/com/netflix/conductor/core/utils/DummyPayloadStorage.java @@ -27,7 +27,7 @@ public class DummyPayloadStorage implements ExternalPayloadStorage { @Override - public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType) { + public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType, String path) { return null; } diff --git a/core/src/main/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtils.java b/core/src/main/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtils.java new file mode 100644 index 0000000000..1dc87fc90f --- /dev/null +++ b/core/src/main/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtils.java @@ -0,0 +1,179 @@ +/* + * Copyright 2016 Netflix, Inc. + *

    + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.netflix.conductor.core.utils; + +import com.amazonaws.util.IOUtils; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.netflix.conductor.common.metadata.tasks.Task; +import com.netflix.conductor.common.run.ExternalStorageLocation; +import com.netflix.conductor.common.run.Workflow; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; +import com.netflix.conductor.common.utils.ExternalPayloadStorage.PayloadType; +import com.netflix.conductor.core.config.Configuration; +import com.netflix.conductor.core.execution.ApplicationException; +import com.netflix.conductor.core.execution.TerminateWorkflowException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.inject.Inject; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.HashMap; +import java.util.Map; + +/** + * Provides utility functions to upload and download payloads to {@link ExternalPayloadStorage} + */ +public class ExternalPayloadStorageUtils { + private static final Logger logger = LoggerFactory.getLogger(ExternalPayloadStorageUtils.class); + + private final ExternalPayloadStorage externalPayloadStorage; + private final Configuration configuration; + + private ObjectMapper objectMapper = new ObjectMapper(); + + @Inject + public ExternalPayloadStorageUtils(ExternalPayloadStorage externalPayloadStorage, Configuration configuration) { + this.externalPayloadStorage = externalPayloadStorage; + this.configuration = configuration; + } + + /** + * Download the payload from the given path + * + * @param path the relative path of the payload in the {@link ExternalPayloadStorage} + * @return the payload object + * @throws ApplicationException in case of JSON parsing errors or download errors + */ + @SuppressWarnings("unchecked") + public Map downloadPayload(String path) { + try (InputStream inputStream = externalPayloadStorage.download(path)) { + return objectMapper.readValue(IOUtils.toString(inputStream), Map.class); + } catch (IOException e) { + logger.error("Unable to download payload from external storage path: {}", path, e); + throw new ApplicationException(ApplicationException.Code.INTERNAL_ERROR, e); + } + } + + /** + * Verify the payload size and upload to external storage if necessary. + * + * @param entity the task or workflow for which the payload is to be verified and uploaded + * @param payloadType the {@link PayloadType} of the payload + * @param {@link Task} or {@link Workflow} + * @throws ApplicationException in case of JSON parsing errors or upload errors + * @throws TerminateWorkflowException if the payload size is bigger than permissible limit as per {@link Configuration} + */ + public void verifyAndUpload(T entity, PayloadType payloadType) { + long threshold = 0L; + long maxThreshold = 0L; + Map payload = new HashMap<>(); + String workflowId = ""; + switch (payloadType) { + case TASK_INPUT: + threshold = configuration.getTaskInputPayloadSizeThresholdKB(); + maxThreshold = configuration.getMaxTaskInputPayloadSizeThresholdKB(); + payload = ((Task) entity).getInputData(); + workflowId = ((Task) entity).getWorkflowInstanceId(); + break; + case TASK_OUTPUT: + threshold = configuration.getTaskOutputPayloadSizeThresholdKB(); + maxThreshold = configuration.getMaxTaskOutputPayloadSizeThresholdKB(); + payload = ((Task) entity).getOutputData(); + workflowId = ((Task) entity).getWorkflowInstanceId(); + break; + case WORKFLOW_INPUT: + threshold = configuration.getWorkflowInputPayloadSizeThresholdKB(); + maxThreshold = configuration.getMaxWorkflowInputPayloadSizeThresholdKB(); + payload = ((Workflow) entity).getInput(); + workflowId = ((Workflow) entity).getWorkflowId(); + break; + case WORKFLOW_OUTPUT: + threshold = configuration.getWorkflowOutputPayloadSizeThresholdKB(); + maxThreshold = configuration.getMaxWorkflowOutputPayloadSizeThresholdKB(); + payload = ((Workflow) entity).getOutput(); + workflowId = ((Workflow) entity).getWorkflowId(); + break; + } + + try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream()) { + objectMapper.writeValue(byteArrayOutputStream, payload); + byte[] payloadBytes = byteArrayOutputStream.toByteArray(); + long payloadSize = payloadBytes.length / 1024; + if (payloadSize > threshold) { + if (payloadSize > maxThreshold) { + if (entity instanceof Task) { + String errorMsg = String.format("The payload size: %dKB of task: %s in workflow: %s is greater than the permissible limit: %dKB", payloadSize, ((Task) entity).getTaskId(), ((Task) entity).getWorkflowInstanceId(), maxThreshold); + failTask(((Task) entity), payloadType, errorMsg); + } else { + String errorMsg = String.format("The output payload size: %dKB of workflow: %s is greater than the permissible limit: %dKB", payloadSize, ((Workflow) entity).getWorkflowId(), maxThreshold); + failWorkflow(errorMsg); + } + } + + switch (payloadType) { + case TASK_INPUT: + ((Task) entity).setInputData(null); + ((Task) entity).setExternalInputPayloadStoragePath(uploadHelper(payloadBytes, payloadSize, PayloadType.TASK_INPUT)); + break; + case TASK_OUTPUT: + ((Task) entity).setOutputData(null); + ((Task) entity).setExternalOutputPayloadStoragePath(uploadHelper(payloadBytes, payloadSize, PayloadType.TASK_OUTPUT)); + break; + case WORKFLOW_INPUT: + ((Workflow) entity).setInput(null); + ((Workflow) entity).setExternalInputPayloadStoragePath(uploadHelper(payloadBytes, payloadSize, PayloadType.WORKFLOW_INPUT)); + break; + case WORKFLOW_OUTPUT: + ((Workflow) entity).setOutput(null); + ((Workflow) entity).setExternalOutputPayloadStoragePath(uploadHelper(payloadBytes, payloadSize, PayloadType.WORKFLOW_OUTPUT)); + break; + } + } + } catch (IOException e) { + logger.error("Unable to upload payload to external storage for workflow: {}", workflowId, e); + throw new ApplicationException(ApplicationException.Code.INTERNAL_ERROR, e); + } + } + + @VisibleForTesting + String uploadHelper(byte[] payloadBytes, long payloadSize, ExternalPayloadStorage.PayloadType payloadType) { + ExternalStorageLocation location = externalPayloadStorage.getLocation(ExternalPayloadStorage.Operation.WRITE, payloadType, ""); + externalPayloadStorage.upload(location.getPath(), new ByteArrayInputStream(payloadBytes), payloadSize); + return location.getPath(); + } + + @VisibleForTesting + void failTask(Task task, PayloadType payloadType, String errorMsg) { + logger.error(errorMsg); + task.setReasonForIncompletion(errorMsg); + task.setStatus(Task.Status.FAILED_WITH_TERMINAL_ERROR); + if (payloadType == PayloadType.TASK_INPUT) { + task.setInputData(null); + } else { + task.setOutputData(null); + } + throw new TerminateWorkflowException(errorMsg, Workflow.WorkflowStatus.FAILED, task); + } + + private void failWorkflow(String errorMsg) { + logger.error(errorMsg); + throw new TerminateWorkflowException(errorMsg); + } +} diff --git a/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java b/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java index 8b00c4a46b..4c8eb3e297 100644 --- a/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java +++ b/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java @@ -29,6 +29,7 @@ import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.core.config.Configuration; import com.netflix.conductor.core.execution.ApplicationException; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,7 +66,7 @@ public S3PayloadStorage(Configuration config) { * @return a {@link ExternalStorageLocation} object which contains the pre-signed URL and the s3 object key for the json payload */ @Override - public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType) { + public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType, String path) { try { ExternalStorageLocation externalStorageLocation = new ExternalStorageLocation(); @@ -78,7 +79,12 @@ public ExternalStorageLocation getLocation(Operation operation, PayloadType payl httpMethod = HttpMethod.PUT; } - String objectKey = getObjectKey(payloadType); + String objectKey; + if (StringUtils.isNotBlank(path)) { + objectKey = path; + } else { + objectKey = getObjectKey(payloadType); + } externalStorageLocation.setPath(objectKey); GeneratePresignedUrlRequest generatePresignedUrlRequest = new GeneratePresignedUrlRequest(bucketName, objectKey) diff --git a/core/src/main/java/com/netflix/conductor/service/AdminService.java b/core/src/main/java/com/netflix/conductor/service/AdminService.java index c85492e23b..4c3a13b337 100644 --- a/core/src/main/java/com/netflix/conductor/service/AdminService.java +++ b/core/src/main/java/com/netflix/conductor/service/AdminService.java @@ -1,3 +1,18 @@ +/* + * Copyright 2016 Netflix, Inc. + *

    + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.conductor.service; import com.netflix.conductor.annotations.Trace; diff --git a/core/src/main/java/com/netflix/conductor/service/EventService.java b/core/src/main/java/com/netflix/conductor/service/EventService.java index 761e3900ef..fae132d7ff 100644 --- a/core/src/main/java/com/netflix/conductor/service/EventService.java +++ b/core/src/main/java/com/netflix/conductor/service/EventService.java @@ -1,3 +1,18 @@ +/* + * Copyright 2016 Netflix, Inc. + *

    + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.conductor.service; import com.netflix.conductor.annotations.Trace; diff --git a/core/src/main/java/com/netflix/conductor/service/ExecutionService.java b/core/src/main/java/com/netflix/conductor/service/ExecutionService.java index dacdc36edb..821159a844 100644 --- a/core/src/main/java/com/netflix/conductor/service/ExecutionService.java +++ b/core/src/main/java/com/netflix/conductor/service/ExecutionService.java @@ -455,9 +455,10 @@ public List getTaskLogs(String taskId) { * * @param operation the type of {@link Operation} to be performed * @param payloadType the {@link PayloadType} at the external uri + * @param path the path for which the external storage location is to be populated * @return the external uri at which the payload is stored/to be stored */ - public ExternalStorageLocation getExternalStorageLocation(Operation operation, PayloadType payloadType) { - return externalPayloadStorage.getLocation(operation, payloadType); + public ExternalStorageLocation getExternalStorageLocation(Operation operation, PayloadType payloadType, String path) { + return externalPayloadStorage.getLocation(operation, payloadType, path); } } diff --git a/core/src/main/java/com/netflix/conductor/service/TaskService.java b/core/src/main/java/com/netflix/conductor/service/TaskService.java index fc2b87b4b7..5f0d3a59bb 100644 --- a/core/src/main/java/com/netflix/conductor/service/TaskService.java +++ b/core/src/main/java/com/netflix/conductor/service/TaskService.java @@ -1,4 +1,4 @@ -/** +/* * Copyright 2016 Netflix, Inc. *

    * Licensed under the Apache License, Version 2.0 (the "License"); @@ -290,10 +290,10 @@ public SearchResult search(int start, int size, String sort, String /** * Get the external storage location where the task output payload is stored/to be stored * + * @param path the path for which the external storage location is to be populated * @return {@link ExternalStorageLocation} containing the uri and the path to the payload is stored in external storage */ - public ExternalStorageLocation getExternalStorageLocation() { - return executionService.getExternalStorageLocation(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.TASK_OUTPUT); + public ExternalStorageLocation getExternalStorageLocation(String path) { + return executionService.getExternalStorageLocation(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.TASK_OUTPUT, path); } } - diff --git a/core/src/main/java/com/netflix/conductor/service/WorkflowBulkService.java b/core/src/main/java/com/netflix/conductor/service/WorkflowBulkService.java index fad9293348..26d5057283 100644 --- a/core/src/main/java/com/netflix/conductor/service/WorkflowBulkService.java +++ b/core/src/main/java/com/netflix/conductor/service/WorkflowBulkService.java @@ -1,3 +1,18 @@ +/* + * Copyright 2016 Netflix, Inc. + *

    + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.conductor.service; import com.google.inject.Singleton; diff --git a/core/src/main/java/com/netflix/conductor/service/WorkflowService.java b/core/src/main/java/com/netflix/conductor/service/WorkflowService.java index 212fd0cc1a..40f1b20986 100644 --- a/core/src/main/java/com/netflix/conductor/service/WorkflowService.java +++ b/core/src/main/java/com/netflix/conductor/service/WorkflowService.java @@ -1,3 +1,18 @@ +/* + * Copyright 2016 Netflix, Inc. + *

    + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.conductor.service; import com.netflix.conductor.annotations.Trace; @@ -284,9 +299,10 @@ public SearchResult searchWorkflowsByTasks(int start, int size, /** * Get the external storage location where the workflow input payload is stored/to be stored * + * @param path the path for which the external storage location is to be populated * @return {@link ExternalStorageLocation} containing the uri and the path to the payload is stored in external storage */ - public ExternalStorageLocation getExternalStorageLocation() { - return executionService.getExternalStorageLocation(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.WORKFLOW_INPUT); + public ExternalStorageLocation getExternalStorageLocation(String path) { + return executionService.getExternalStorageLocation(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.WORKFLOW_INPUT, path); } } diff --git a/core/src/main/java/com/netflix/conductor/service/utils/ServiceUtils.java b/core/src/main/java/com/netflix/conductor/service/utils/ServiceUtils.java index cb3c0bfba2..34ccb041a6 100644 --- a/core/src/main/java/com/netflix/conductor/service/utils/ServiceUtils.java +++ b/core/src/main/java/com/netflix/conductor/service/utils/ServiceUtils.java @@ -1,3 +1,18 @@ +/* + * Copyright 2016 Netflix, Inc. + *

    + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.netflix.conductor.service.utils; import com.google.common.base.Preconditions; diff --git a/core/src/test/java/com/netflix/conductor/core/execution/TestConfiguration.java b/core/src/test/java/com/netflix/conductor/core/execution/TestConfiguration.java index a3a0cc0a29..6eb5c1a6b1 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/TestConfiguration.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/TestConfiguration.java @@ -14,15 +14,11 @@ * limitations under the License. */ package com.netflix.conductor.core.execution; -/** - * - */ +import com.netflix.conductor.core.config.Configuration; import java.util.Map; -import com.netflix.conductor.core.config.Configuration; - /** * @author Viren * @@ -83,6 +79,46 @@ public int getIntProperty(String string, int def) { public String getRegion() { return "us-east-1"; } + + @Override + public Long getWorkflowInputPayloadSizeThresholdKB() { + return 10L; + } + + @Override + public Long getMaxWorkflowInputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getWorkflowOutputPayloadSizeThresholdKB() { + return 10L; + } + + @Override + public Long getMaxWorkflowOutputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getTaskInputPayloadSizeThresholdKB() { + return 10L; + } + + @Override + public Long getMaxTaskInputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getTaskOutputPayloadSizeThresholdKB() { + return 10L; + } + + @Override + public Long getMaxTaskOutputPayloadSizeThresholdKB() { + return 10240L; + } @Override public Map getAll() { @@ -91,6 +127,6 @@ public Map getAll() { @Override public long getLongProperty(String name, long defaultValue) { - return 1000000l; + return 1000000L; } } diff --git a/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderOutcomes.java b/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderOutcomes.java index 2f3ff4d943..246dbbb580 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderOutcomes.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderOutcomes.java @@ -28,7 +28,7 @@ import com.netflix.conductor.common.metadata.workflow.WorkflowTask; import com.netflix.conductor.common.metadata.workflow.WorkflowTask.Type; import com.netflix.conductor.common.run.Workflow; -import com.netflix.conductor.common.utils.ExternalPayloadStorage; +import com.netflix.conductor.core.config.Configuration; import com.netflix.conductor.core.execution.DeciderService.DeciderOutcome; import com.netflix.conductor.core.execution.mapper.DecisionTaskMapper; import com.netflix.conductor.core.execution.mapper.DynamicTaskMapper; @@ -42,9 +42,11 @@ import com.netflix.conductor.core.execution.mapper.UserDefinedTaskMapper; import com.netflix.conductor.core.execution.mapper.WaitTaskMapper; import com.netflix.conductor.core.execution.tasks.Join; +import com.netflix.conductor.core.utils.ExternalPayloadStorageUtils; import com.netflix.conductor.dao.MetadataDAO; import com.netflix.conductor.dao.QueueDAO; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import java.io.InputStream; @@ -84,10 +86,14 @@ public class TestDeciderOutcomes { @Before public void init() { - + MetadataDAO metadataDAO = mock(MetadataDAO.class); QueueDAO queueDAO = mock(QueueDAO.class); - ExternalPayloadStorage externalPayloadStorage = mock(ExternalPayloadStorage.class); + ExternalPayloadStorageUtils externalPayloadStorageUtils = mock(ExternalPayloadStorageUtils.class); + Configuration configuration = mock(Configuration.class); + when(configuration.getTaskInputPayloadSizeThresholdKB()).thenReturn(10L); + when(configuration.getMaxTaskInputPayloadSizeThresholdKB()).thenReturn(10240L); + TaskDef taskDef = new TaskDef(); taskDef.setRetryCount(1); taskDef.setName("mockTaskDef"); @@ -106,7 +112,7 @@ public void init() { taskMappers.put("EVENT", new EventTaskMapper(parametersUtils)); taskMappers.put("WAIT", new WaitTaskMapper(parametersUtils)); - this.deciderService = new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorage, taskMappers); + this.deciderService = new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorageUtils, taskMappers); } @Test @@ -141,9 +147,6 @@ public void testWorkflowWithNoTasks() throws Exception { @Test public void testRetries() { - WorkflowDef def = new WorkflowDef(); - def.setName("test"); - WorkflowTask task = new WorkflowTask(); task.setName("test_task"); task.setType("USER_TASK"); @@ -151,15 +154,17 @@ public void testRetries() { task.getInputParameters().put("taskId", "${CPEWF_TASK_ID}"); task.getInputParameters().put("requestId", "${workflow.input.requestId}"); + WorkflowDef def = new WorkflowDef(); + def.setName("test"); def.getTasks().add(task); def.setSchemaVersion(2); Workflow workflow = new Workflow(); workflow.getInput().put("requestId", 123); workflow.setStartTime(System.currentTimeMillis()); + DeciderOutcome outcome = deciderService.decide(workflow, def); assertNotNull(outcome); - assertEquals(1, outcome.tasksToBeScheduled.size()); assertEquals(task.getTaskReferenceName(), outcome.tasksToBeScheduled.get(0).getReferenceTaskName()); @@ -172,7 +177,6 @@ public void testRetries() { outcome = deciderService.decide(workflow, def); assertNotNull(outcome); - assertEquals(1, outcome.tasksToBeUpdated.size()); assertEquals(1, outcome.tasksToBeScheduled.size()); assertEquals(task1Id, outcome.tasksToBeUpdated.get(0).getTaskId()); @@ -181,7 +185,6 @@ public void testRetries() { assertEquals(task1Id, outcome.tasksToBeScheduled.get(0).getRetriedTaskId()); assertEquals(123, outcome.tasksToBeScheduled.get(0).getInputData().get("requestId")); - WorkflowTask fork = new WorkflowTask(); fork.setName("fork0"); fork.setWorkflowTaskType(Type.FORK_JOIN_DYNAMIC); @@ -219,7 +222,6 @@ public void testRetries() { workflow = new Workflow(); workflow.getInput().put("requestId", 123); workflow.setStartTime(System.currentTimeMillis()); - workflow.getInput().put("forks", forks); workflow.getInput().put("forkedInputs", forkedInputs); @@ -232,13 +234,15 @@ public void testRetries() { assertEquals(1, outcome.tasksToBeScheduled.get(1).getInputData().get("k1")); assertEquals(outcome.tasksToBeScheduled.get(1).getTaskId(), outcome.tasksToBeScheduled.get(1).getInputData().get("taskId")); System.out.println(outcome.tasksToBeScheduled.get(1).getInputData()); - task1Id = outcome.tasksToBeScheduled.get(1).getTaskId(); + task1Id = outcome.tasksToBeScheduled.get(1).getTaskId(); outcome.tasksToBeScheduled.get(1).setStatus(Status.FAILED); workflow.getTasks().addAll(outcome.tasksToBeScheduled); outcome = deciderService.decide(workflow, def); assertTrue(outcome.tasksToBeScheduled.stream().anyMatch(task1 -> task1.getReferenceTaskName().equals("f0"))); + + //noinspection ConstantConditions Task task1 = outcome.tasksToBeScheduled.stream().filter(t -> t.getReferenceTaskName().equals("f0")).findFirst().get(); assertEquals("v", task1.getInputData().get("k")); assertEquals(1, task1.getInputData().get("k1")); @@ -246,7 +250,6 @@ public void testRetries() { assertNotSame(task1Id, task1.getTaskId()); assertEquals(task1Id, task1.getRetriedTaskId()); System.out.println(task1.getInputData()); - } @Test @@ -302,7 +305,7 @@ public void testOptional() { } @Test - public void testOptionalWithDynamicFork() throws Exception { + public void testOptionalWithDynamicFork() { WorkflowDef workflowDef = new WorkflowDef(); workflowDef.setName("test"); diff --git a/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java b/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java index c07c1ee381..d5929a613d 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializationFeature; import com.netflix.conductor.common.metadata.tasks.Task; import com.netflix.conductor.common.metadata.tasks.Task.Status; import com.netflix.conductor.common.metadata.tasks.TaskDef; @@ -32,7 +31,6 @@ import com.netflix.conductor.common.metadata.workflow.WorkflowTask.Type; import com.netflix.conductor.common.run.Workflow; import com.netflix.conductor.common.run.Workflow.WorkflowStatus; -import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.core.execution.DeciderService.DeciderOutcome; import com.netflix.conductor.core.execution.mapper.DecisionTaskMapper; import com.netflix.conductor.core.execution.mapper.DynamicTaskMapper; @@ -45,6 +43,7 @@ import com.netflix.conductor.core.execution.mapper.TaskMapper; import com.netflix.conductor.core.execution.mapper.UserDefinedTaskMapper; import com.netflix.conductor.core.execution.mapper.WaitTaskMapper; +import com.netflix.conductor.core.utils.ExternalPayloadStorageUtils; import com.netflix.conductor.dao.MetadataDAO; import com.netflix.conductor.dao.QueueDAO; import com.netflix.spectator.api.Counter; @@ -76,13 +75,13 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; /** * @author Viren - * */ @SuppressWarnings("Duplicates") public class TestDeciderService { @@ -95,6 +94,8 @@ public class TestDeciderService { private static Registry registry; + private ExternalPayloadStorageUtils externalPayloadStorageUtils; + private static ObjectMapper objectMapper = new ObjectMapper(); static { @@ -115,7 +116,8 @@ public static void init() { public void setup() { MetadataDAO metadataDAO = mock(MetadataDAO.class); QueueDAO queueDAO = mock(QueueDAO.class); - ExternalPayloadStorage externalPayloadStorage = mock(ExternalPayloadStorage.class); + externalPayloadStorageUtils = mock(ExternalPayloadStorageUtils.class); + TaskDef taskDef = new TaskDef(); WorkflowDef workflowDef = new WorkflowDef(); when(metadataDAO.getTaskDef(any())).thenReturn(taskDef); @@ -133,7 +135,7 @@ public void setup() { taskMappers.put("EVENT", new EventTaskMapper(parametersUtils)); taskMappers.put("WAIT", new WaitTaskMapper(parametersUtils)); - deciderService = new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorage, taskMappers); + deciderService = new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorageUtils, taskMappers); workflow = new Workflow(); workflow.getInput().put("requestId", "request id 001"); @@ -198,7 +200,7 @@ public void testGetTaskInputV2() { assertNull(taskInput.get("taskOutputParam3")); assertNull(taskInput.get("nullValue")); assertEquals(workflow.getTasks().get(0).getStatus().name(), taskInput.get("task2Status")); //task2 and task3 are the tasks respectively - System.out.println(taskInput); +// System.out.println(taskInput); workflow.setSchemaVersion(1); } @@ -264,7 +266,7 @@ public void testGetTaskInputV2Partial() throws Exception { assertEquals("The Doors", taskInput.get("secondName")); assertEquals("The Band is: The Doors-\ti-123abcdef990", taskInput.get("concatenatedName")); - System.out.println(new ObjectMapper().enable(SerializationFeature.INDENT_OUTPUT).writeValueAsString(taskInput)); +// System.out.println(new ObjectMapper().enable(SerializationFeature.INDENT_OUTPUT).writeValueAsString(taskInput)); assertEquals("request id 001", taskInput.get("workflowInputParam")); assertEquals("http://location", taskInput.get("taskOutputParam")); @@ -305,7 +307,7 @@ public void testGetTaskInput() { workflow.getTasks().add(task); workflow.setSchemaVersion(2); Map taskInput = parametersUtils.getTaskInput(ip, workflow, null, null); - System.out.println(taskInput.get("complexJson")); +// System.out.println(taskInput.get("complexJson")); assertNotNull(taskInput); assertTrue(taskInput.containsKey("workflowInputParam")); assertTrue(taskInput.containsKey("taskOutputParam")); @@ -553,7 +555,7 @@ public void testConcurrentTaskInputCalc() throws InterruptedException { Object cmObj = reqInput.get(0).get("channelMapping"); assertNotNull(cmObj); if (!(cmObj instanceof Number)) { - System.out.println("Not a number @ " + x + ", found: " + cmObj.getClass()); +// System.out.println("Not a number @ " + x + ", found: " + cmObj.getClass()); result[x] = -1; } else { Number channelMapping = (Number) cmObj; @@ -576,8 +578,7 @@ public void testConcurrentTaskInputCalc() throws InterruptedException { for (int i = 0; i < result.length; i++) { assertEquals(i, result[i]); } - System.out.println("Done"); - +// System.out.println("Done"); } @@ -612,8 +613,8 @@ public void testTaskRetry() { workflowTask.getInputParameters().put("env", env); Task task2 = deciderService.retry(taskDef, workflowTask, task, workflow); - System.out.println(task.getTaskId() + ":\n" + task.getInputData()); - System.out.println(task2.getTaskId() + ":\n" + task2.getInputData()); +// System.out.println(task.getTaskId() + ":\n" + task.getInputData()); +// System.out.println(task2.getTaskId() + ":\n" + task2.getInputData()); assertEquals("t1", task.getInputData().get("task_id")); assertEquals("t1", ((Map) task.getInputData().get("env")).get("env_task_id")); @@ -626,10 +627,10 @@ public void testTaskRetry() { @Test public void testFork() throws Exception { - InputStream stream = TestDeciderOutcomes.class.getResourceAsStream("/test.json"); + InputStream stream = TestDeciderService.class.getResourceAsStream("/test.json"); Workflow workflow = objectMapper.readValue(stream, Workflow.class); - InputStream defs = TestDeciderOutcomes.class.getResourceAsStream("/def.json"); + InputStream defs = TestDeciderService.class.getResourceAsStream("/def.json"); WorkflowDef def = objectMapper.readValue(defs, WorkflowDef.class); DeciderOutcome outcome = deciderService.decide(workflow, def); @@ -747,7 +748,7 @@ public void testGetTasksToBeScheduled() { } @Test - public void testIsResponsedTimeOut() { + public void testIsResponseTimedOut() { TaskDef taskDef = new TaskDef(); taskDef.setName("test_rt"); taskDef.setResponseTimeoutSeconds(10); @@ -763,6 +764,49 @@ public void testIsResponsedTimeOut() { assertTrue(flag); } + @Test + public void testPopulateWorkflowAndTaskData() { + String workflowInputPath = "workflow/input/test.json"; + String taskInputPath = "task/input/test.json"; + String taskOutputPath = "task/output/test.json"; + + Map workflowParams = new HashMap<>(); + workflowParams.put("key1", "value1"); + workflowParams.put("key2", 100); + when(externalPayloadStorageUtils.downloadPayload(workflowInputPath)).thenReturn(workflowParams); + + Map taskInputParams = new HashMap<>(); + taskInputParams.put("key", "taskInput"); + when(externalPayloadStorageUtils.downloadPayload(taskInputPath)).thenReturn(taskInputParams); + + Map taskOutputParams = new HashMap<>(); + taskOutputParams.put("key", "taskOutput"); + when(externalPayloadStorageUtils.downloadPayload(taskOutputPath)).thenReturn(taskOutputParams); + + Task task = new Task(); + task.setExternalInputPayloadStoragePath(taskInputPath); + task.setExternalOutputPayloadStoragePath(taskOutputPath); + + Workflow workflow = new Workflow(); + workflow.setExternalInputPayloadStoragePath(workflowInputPath); + workflow.getTasks().add(task); + + Workflow workflowInstance = deciderService.populateWorkflowAndTaskData(workflow); + assertNotNull(workflowInstance); + + assertTrue(workflow.getInput().isEmpty()); + assertNotNull(workflowInstance.getInput()); + assertEquals(workflowParams, workflowInstance.getInput()); + + assertTrue(workflow.getTasks().get(0).getInputData().isEmpty()); + assertNotNull(workflowInstance.getTasks().get(0).getInputData()); + assertEquals(taskInputParams, workflowInstance.getTasks().get(0).getInputData()); + + assertTrue(workflow.getTasks().get(0).getOutputData().isEmpty()); + assertNotNull(workflowInstance.getTasks().get(0).getOutputData()); + assertEquals(taskOutputParams, workflowInstance.getTasks().get(0).getOutputData()); + } + private WorkflowDef createConditionalWF() { WorkflowTask workflowTask1 = new WorkflowTask(); diff --git a/core/src/test/java/com/netflix/conductor/core/execution/TestWorkflowExecutor.java b/core/src/test/java/com/netflix/conductor/core/execution/TestWorkflowExecutor.java index 13e62f7303..87a4e13c25 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/TestWorkflowExecutor.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/TestWorkflowExecutor.java @@ -25,7 +25,6 @@ import com.netflix.conductor.common.metadata.workflow.WorkflowTask; import com.netflix.conductor.common.metadata.workflow.WorkflowTask.Type; import com.netflix.conductor.common.run.Workflow; -import com.netflix.conductor.common.utils.ExternalPayloadStorage; import com.netflix.conductor.core.execution.mapper.DecisionTaskMapper; import com.netflix.conductor.core.execution.mapper.DynamicTaskMapper; import com.netflix.conductor.core.execution.mapper.EventTaskMapper; @@ -39,6 +38,7 @@ import com.netflix.conductor.core.execution.mapper.WaitTaskMapper; import com.netflix.conductor.core.execution.tasks.Wait; import com.netflix.conductor.core.execution.tasks.WorkflowSystemTask; +import com.netflix.conductor.core.utils.ExternalPayloadStorageUtils; import com.netflix.conductor.core.utils.IDGenerator; import com.netflix.conductor.dao.ExecutionDAO; import com.netflix.conductor.dao.MetadataDAO; @@ -84,7 +84,7 @@ public void init() { executionDAO = mock(ExecutionDAO.class); metadataDAO = mock(MetadataDAO.class); queueDAO = mock(QueueDAO.class); - ExternalPayloadStorage externalPayloadStorage = mock(ExternalPayloadStorage.class); + ExternalPayloadStorageUtils externalPayloadStorageUtils = mock(ExternalPayloadStorageUtils.class); ObjectMapper objectMapper = new ObjectMapper(); ParametersUtils parametersUtils = new ParametersUtils(); Map taskMappers = new HashMap<>(); @@ -98,7 +98,7 @@ public void init() { taskMappers.put("SUB_WORKFLOW", new SubWorkflowTaskMapper(parametersUtils, metadataDAO)); taskMappers.put("EVENT", new EventTaskMapper(parametersUtils)); taskMappers.put("WAIT", new WaitTaskMapper(parametersUtils)); - DeciderService deciderService = new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorage, taskMappers); + DeciderService deciderService = new DeciderService(metadataDAO, parametersUtils, queueDAO, externalPayloadStorageUtils, taskMappers); workflowExecutor = new WorkflowExecutor(deciderService, metadataDAO, executionDAO, queueDAO, parametersUtils, config); } diff --git a/core/src/test/java/com/netflix/conductor/core/execution/mapper/ForkJoinDynamicTaskMapperTest.java b/core/src/test/java/com/netflix/conductor/core/execution/mapper/ForkJoinDynamicTaskMapperTest.java index 26c31c457b..ff9ce3eaf9 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/mapper/ForkJoinDynamicTaskMapperTest.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/mapper/ForkJoinDynamicTaskMapperTest.java @@ -46,7 +46,7 @@ public class ForkJoinDynamicTaskMapperTest { @Before - public void setUp() throws Exception { + public void setUp() { parametersUtils = Mockito.mock(ParametersUtils.class); objectMapper = Mockito.mock(ObjectMapper.class); deciderService = Mockito.mock(DeciderService.class); @@ -56,7 +56,7 @@ public void setUp() throws Exception { } @Test - public void getMappedTasksException() throws Exception { + public void getMappedTasksException() { WorkflowDef def = new WorkflowDef(); def.setName("DYNAMIC_FORK_JOIN_WF"); @@ -134,7 +134,7 @@ public void getMappedTasksException() throws Exception { } @Test - public void getMappedTasks() throws Exception { + public void getMappedTasks() { WorkflowDef def = new WorkflowDef(); def.setName("DYNAMIC_FORK_JOIN_WF"); @@ -219,7 +219,7 @@ public void getMappedTasks() throws Exception { @Test - public void getDynamicForkJoinTasksAndInput() throws Exception { + public void getDynamicForkJoinTasksAndInput() { //Given WorkflowTask dynamicForkJoinToSchedule = new WorkflowTask(); dynamicForkJoinToSchedule.setType(WorkflowTask.Type.FORK_JOIN_DYNAMIC.name()); @@ -257,7 +257,7 @@ public void getDynamicForkJoinTasksAndInput() throws Exception { } @Test - public void getDynamicForkJoinTasksAndInputException() throws Exception { + public void getDynamicForkJoinTasksAndInputException() { //Given WorkflowTask dynamicForkJoinToSchedule = new WorkflowTask(); dynamicForkJoinToSchedule.setType(WorkflowTask.Type.FORK_JOIN_DYNAMIC.name()); @@ -294,7 +294,7 @@ public void getDynamicForkJoinTasksAndInputException() throws Exception { } @Test - public void getDynamicForkTasksAndInput() throws Exception { + public void getDynamicForkTasksAndInput() { //Given WorkflowTask dynamicForkJoinToSchedule = new WorkflowTask(); dynamicForkJoinToSchedule.setType(WorkflowTask.Type.FORK_JOIN_DYNAMIC.name()); @@ -336,7 +336,7 @@ public void getDynamicForkTasksAndInput() throws Exception { } @Test - public void getDynamicForkTasksAndInputException() throws Exception { + public void getDynamicForkTasksAndInputException() { //Given WorkflowTask dynamicForkJoinToSchedule = new WorkflowTask(); diff --git a/core/src/test/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtilsTest.java b/core/src/test/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtilsTest.java new file mode 100644 index 0000000000..8f30e18cb2 --- /dev/null +++ b/core/src/test/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtilsTest.java @@ -0,0 +1,153 @@ +package com.netflix.conductor.core.utils; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.netflix.conductor.common.metadata.tasks.Task; +import com.netflix.conductor.common.run.ExternalStorageLocation; +import com.netflix.conductor.common.run.Workflow; +import com.netflix.conductor.common.utils.ExternalPayloadStorage; +import com.netflix.conductor.core.config.Configuration; +import com.netflix.conductor.core.execution.TerminateWorkflowException; +import com.netflix.conductor.core.execution.TestConfiguration; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ExternalPayloadStorageUtilsTest { + + private ExternalPayloadStorage externalPayloadStorage; + private ObjectMapper objectMapper; + private ExternalStorageLocation location; + + // Subject + private ExternalPayloadStorageUtils externalPayloadStorageUtils; + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Before + public void setup() { + externalPayloadStorage = mock(ExternalPayloadStorage.class); + Configuration configuration = new TestConfiguration(); + objectMapper = new ObjectMapper(); + location = new ExternalStorageLocation(); + location.setPath("some/test/path"); + + externalPayloadStorageUtils = new ExternalPayloadStorageUtils(externalPayloadStorage, configuration); + } + + @Test + public void testDownloadPayload() throws IOException { + String path = "test/payload"; + + Map payload = new HashMap<>(); + payload.put("key1", "value1"); + payload.put("key2", 200); + byte[] payloadBytes = objectMapper.writeValueAsString(payload).getBytes(); + when(externalPayloadStorage.download(path)).thenReturn(new ByteArrayInputStream(payloadBytes)); + + Map result = externalPayloadStorageUtils.downloadPayload(path); + assertNotNull(result); + assertEquals(payload, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testUploadTaskPayload() throws IOException { + AtomicInteger uploadCount = new AtomicInteger(0); + + InputStream stream = ExternalPayloadStorageUtilsTest.class.getResourceAsStream("/payload.json"); + Map payload = objectMapper.readValue(stream, Map.class); + + when(externalPayloadStorage.getLocation(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.TASK_INPUT, "")).thenReturn(location); + doAnswer(invocation -> { + uploadCount.incrementAndGet(); + return null; + }).when(externalPayloadStorage).upload(anyString(), any(), anyLong()); + + Task task = new Task(); + task.setInputData(payload); + externalPayloadStorageUtils.verifyAndUpload(task, ExternalPayloadStorage.PayloadType.TASK_INPUT); + assertNull(task.getInputData()); + assertEquals(1, uploadCount.get()); + assertNotNull(task.getExternalInputPayloadStoragePath()); + } + + @SuppressWarnings("unchecked") + @Test + public void testUploadWorkflowPayload() throws IOException { + AtomicInteger uploadCount = new AtomicInteger(0); + + InputStream stream = ExternalPayloadStorageUtilsTest.class.getResourceAsStream("/payload.json"); + Map payload = objectMapper.readValue(stream, Map.class); + + when(externalPayloadStorage.getLocation(ExternalPayloadStorage.Operation.WRITE, ExternalPayloadStorage.PayloadType.WORKFLOW_OUTPUT, "")).thenReturn(location); + doAnswer(invocation -> { + uploadCount.incrementAndGet(); + return null; + }).when(externalPayloadStorage).upload(anyString(), any(), anyLong()); + + Workflow workflow = new Workflow(); + workflow.setOutput(payload); + externalPayloadStorageUtils.verifyAndUpload(workflow, ExternalPayloadStorage.PayloadType.WORKFLOW_OUTPUT); + assertNull(workflow.getOutput()); + assertEquals(1, uploadCount.get()); + assertNotNull(workflow.getExternalOutputPayloadStoragePath()); + } + + @Test + public void testUploadHelper() { + AtomicInteger uploadCount = new AtomicInteger(0); + String path = "some/test/path.json"; + ExternalStorageLocation location = new ExternalStorageLocation(); + location.setPath(path); + + when(externalPayloadStorage.getLocation(any(), any(), any())).thenReturn(location); + doAnswer(invocation -> { + uploadCount.incrementAndGet(); + return null; + }).when(externalPayloadStorage).upload(anyString(), any(), anyLong()); + + assertEquals(path, externalPayloadStorageUtils.uploadHelper(new byte[]{}, 10L, ExternalPayloadStorage.PayloadType.TASK_OUTPUT)); + assertEquals(1, uploadCount.get()); + } + + @Test + public void testFailTaskWithInputPayload() { + Task task = new Task(); + task.setInputData(new HashMap<>()); + + expectedException.expect(TerminateWorkflowException.class); + externalPayloadStorageUtils.failTask(task, ExternalPayloadStorage.PayloadType.TASK_INPUT, "error"); + assertNotNull(task); + assertNull(task.getInputData()); + } + + @Test + public void testFailTaskWithOutputPayload() { + Task task = new Task(); + task.setOutputData(new HashMap<>()); + + expectedException.expect(TerminateWorkflowException.class); + externalPayloadStorageUtils.failTask(task, ExternalPayloadStorage.PayloadType.TASK_OUTPUT, "error"); + assertNotNull(task); + assertNull(task.getOutputData()); + } +} \ No newline at end of file diff --git a/core/src/test/resources/payload.json b/core/src/test/resources/payload.json new file mode 100644 index 0000000000..c13bc5d2b0 --- /dev/null +++ b/core/src/test/resources/payload.json @@ -0,0 +1,423 @@ +{ + "imageType": "TEST_SAMPLE", + "filteredSourceList": { + "TEST_SAMPLE": [ + { + "sourceId": "1413900_10830", + "url": "file/location/a0bdc4d0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_50241", + "url": "file/location/cd4e00a0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "generated-55ee8663-85c2-42d3-aca2-4076707e6d4e", + "url": "file/sample/location/e008d018-63d7-44b2-b07e-c7435430ac71" + }, + { + "sourceId": "generated-14056154-1544-4350-81db-b3751fe44777", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-0b0ae5ea-d5c5-410c-adc9-bf16d2909c2e", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-08869779-614d-417c-bfea-36a3f8f199da", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "generated-e117db45-1c48-45d0-b751-89386eb2d81d", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "f0221421-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/4a009209-002f-4b58-8b96-cb2198f8ba3c" + }, + { + "sourceId": "f0252161-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/55b56298-5e7a-4949-b919-88c5c9557e8e" + }, + { + "sourceId": "f038d070-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/3c4804f4-e826-436f-90c9-52b8d9266d52" + }, + { + "sourceId": "f04e0621-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/689283a1-1816-48ef-83da-7f9ac874bf45" + }, + { + "sourceId": "f04ddf10-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/586666ae-7321-445a-80b6-323c8c241ecd" + }, + { + "sourceId": "f05950c0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/31795cc4-2590-4b20-a617-deaa18301f99" + }, + { + "sourceId": "1413900_46819", + "url": "file/location/c74497a0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_11177", + "url": "file/location/a231c730-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_48713", + "url": "file/location/ca638ae0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_48525", + "url": "file/location/ca0c9140-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_73303", + "url": "file/location/d5943a40-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_55202", + "url": "file/location/d1a4d7a0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "generated-61413adf-3c10-4484-b25d-e238df898f45", + "url": "file/sample/location/e008d018-63d7-44b2-b07e-c7435430ac71" + }, + { + "sourceId": "generated-addca397-f050-4339-ae86-9ba8c4e1b0d5", + "url": "file/sample/location/838a0ddb-a315-453a-8b8a-fa795f9d7691" + }, + { + "sourceId": "generated-e4de9810-0f69-4593-8926-01ed82cbebcb", + "url": "file/sample/location/838a0ddb-a315-453a-8b8a-fa795f9d7691" + }, + { + "sourceId": "generated-e16e2074-7af6-4700-ab05-ca41ba9c9ab4", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-341c86f8-57a5-40e1-8842-3eb41dd9f528", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-88c2ea9b-cef7-4120-8043-b92713d8fade", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-3f6a731f-3c92-4677-9923-f80b8a6be632", + "url": "file/sample/location/3881aea9-a731-4e22-9ead-2d6eccc51140" + }, + { + "sourceId": "generated-1508b871-64de-47ce-8b07-76c5cb3f3e1e", + "url": "file/sample/location/a2e4195f-3900-45b4-9335-45f85fca6467" + }, + { + "sourceId": "generated-1406dce8-7b9c-4956-a7e8-78721c476ce9", + "url": "file/sample/location/a2e4195f-3900-45b4-9335-45f85fca6467" + }, + { + "sourceId": "f0206671-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/35ebee36-3072-44c5-abb5-702a5a3b1a91" + }, + { + "sourceId": "f01f5501-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/d3a9133d-c681-4910-a769-8195526ae634" + }, + { + "sourceId": "f022b060-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/8fc1413d-170e-4644-a554-5e0c596b225c" + }, + { + "sourceId": "f02fa8b1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/35bed0a2-7def-457b-bded-4f4d7d94f76e" + }, + { + "sourceId": "f031f2a0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/a5a2ea1f-8d13-429c-a44d-3057d21f608a" + }, + { + "sourceId": "f0424650-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/1c599ffc-4f10-4c0b-8d9a-ae41c7256113" + }, + { + "sourceId": "f04ec970-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/8404a421-e1a6-41cf-af63-a35ccb474457" + }, + { + "sourceId": "1413900_47197", + "url": "file/location/c81b6fa0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "generated-2a63c0c8-62ea-44a4-a33b-f0b3047e8b00", + "url": "file/sample/location/e008d018-63d7-44b2-b07e-c7435430ac71" + }, + { + "sourceId": "generated-b27face7-3589-4209-944a-5153b20c5996", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-144675b3-9321-48d2-8b5b-e19a40d30ef2", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-8cbe821e-b1fb-48ce-beb5-735319af4db6", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-ecc4ea47-9bad-4b91-97c7-35f4ea6fb479", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-c1eb9ed0-8560-4e09-a748-f926edb7cdc2", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "generated-6bed81fd-c777-4c61-8da1-0bb7f7cf0082", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "generated-852e5510-dd5d-4900-a614-854148fcc716", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "generated-f4dedcb7-37c9-4ba9-ab37-64ec9be7c882", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "f0259691-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/721bc0de-e75f-4386-8b2e-ca84eb653596" + }, + { + "sourceId": "f02b3be1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/d2043b17-8ce5-42ee-a5e4-81c68f0c4838" + }, + { + "sourceId": "f02b62f0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/63931561-3b5b-4ffe-af47-da2c9de94684" + }, + { + "sourceId": "f0315660-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/d99ed629-2885-4e4a-8a1b-22e487b875fa" + }, + { + "sourceId": "f0306c00-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/6f8e673a-7003-44aa-96b9-e2ed8a4654ff" + }, + { + "sourceId": "f033c760-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/627c00f9-14b3-4057-b6e2-0f962ad0308e" + }, + { + "sourceId": "f03526f1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/fafabaf9-fe58-4a9a-b555-026521aeb2fe" + }, + { + "sourceId": "f03acc41-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/6c9fed2c-558a-4db3-8360-659b5e8c46e4" + }, + { + "sourceId": "f0463df1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/e9fb83d2-5f14-4442-92b5-67e613f2e35f" + }, + { + "sourceId": "f04fb3d0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/e7a0f82f-be8d-4ada-a4b1-13e8165e08be" + }, + { + "sourceId": "f05272f0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/9aba488a-22b3-4932-85a7-52c461203541" + }, + { + "sourceId": "f0581841-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/457415f6-6d0c-4304-8533-0d5b43fac564" + }, + { + "sourceId": "generated-8fefb48c-6fde-4fd6-8f33-a1f3f3b62105", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-30c61aa5-f5bd-4077-8c32-336b87acbe96", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-d5da37db-d486-46d4-8f7d-1e0710a77eb5", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-77af26fe-9e22-48af-99e3-f63f10fbe6de", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-2e807016-3d11-4b60-bec7-c380a608b67d", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-615d02e9-62c2-43ab-9df7-753b6b8e2c22", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-3e1600fd-a626-4ee6-972b-5f0187e96c38", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "generated-1dcb208c-6a58-4334-a60c-6fb54c8a2af5", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "f024ac30-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/0af2107b-4231-4d23-bef3-4e417ac6c5d3" + }, + { + "sourceId": "f0282ea1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/0f592681-fd23-4194-ae43-42f61c664485" + }, + { + "sourceId": "f02c4d50-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/ec46b9a3-99af-410a-af7d-726f8854909f" + }, + { + "sourceId": "f02b8a00-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/aed7e5da-b524-4d41-b264-28ce615ec826" + }, + { + "sourceId": "f02b14d1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/b88c9055-ab0d-4d27-a405-265ba2a15f0c" + }, + { + "sourceId": "f03044f1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/fb8c4df9-d59e-4ac3-880e-4ea94cd880a4" + }, + { + "sourceId": "f034ffe1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/59f3fbe8-b300-4861-9b2f-dac7b15aea7d" + }, + { + "sourceId": "f03c2bd0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/19a06d54-41ed-419d-9947-f10cd5f0d85c" + }, + { + "sourceId": "f03fae41-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/a9a48a62-7d62-4f67-b281-cc6fdc1e722c" + }, + { + "sourceId": "f0455390-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/0aeffc0a-a5ad-46ff-abab-1b3bc6a5840a" + }, + { + "sourceId": "f04b1ff1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/9a08aaed-c125-48f7-9d1d-fd11266c2b12" + }, + { + "sourceId": "f04cf4b1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/17a6e0f9-aa64-411f-9af7-837c84f7443f" + }, + { + "sourceId": "f0511360-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/fb633c73-cb33-4806-bc08-049024644856" + }, + { + "sourceId": "f0538460-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/a7012248-6769-42da-a6c8-d4b831f6efce" + }, + { + "sourceId": "f058db91-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/bcf71522-6168-48c4-86c9-995bca60ae51" + }, + { + "sourceId": "generated-adf005c4-95c1-4904-9968-09cc19a26bfe", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-c4d367a4-4cdc-412e-af79-09b227f2e3ba", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-48dba018-f884-49db-b87e-67274e244c8f", + "url": "file/sample/location/4bce4154-fb4b-4f0a-887d-a0cd12d4d214" + }, + { + "sourceId": "generated-26700b83-4892-420e-8b46-1ee21eba75fb", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "generated-632f3198-c0dc-4348-974f-51684d4e443e", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "generated-86e2dd1d-1aa4-4dbe-b37b-b488f5dd1c70", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "f04134e0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/ff8f59bf-7757-4d51-a7e4-619f3e8ffaf2" + }, + { + "sourceId": "f04f65b0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/d66467d1-3ac6-4041-8d15-e722ee07231f" + }, + { + "sourceId": "1413900_15255", + "url": "file/location/a9e20260-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "generated-e953493b-cbe3-4319-885e-00c82089c76c", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-65c54676-3adb-4ef0-b65e-8e2a49533cbf", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "f02ac6b0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/21568877-07a5-411f-9715-5e92806c4448" + }, + { + "sourceId": "f02fcfc1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/f3b1f1a2-48d3-475d-a607-2e5a1fe532e7" + }, + { + "sourceId": "f03526f0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/84a40c66-d925-4a4a-ba62-8491d26e29e9" + }, + { + "sourceId": "f03e75c1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/e84c00e8-a148-46cf-9a0b-431c4c2aeb08" + }, + { + "sourceId": "f0429471-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/178de9fa-7cc8-457a-8fb6-5c080e6163ea" + }, + { + "sourceId": "f047eba0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/18d153aa-e13b-4264-ae03-f3da75eb425b" + }, + { + "sourceId": "f04fdae0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/7c843e53-8d87-47cf-bca5-1a02e7f5e33f" + }, + { + "sourceId": "f0553210-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/26bacd65-9082-4d83-9506-90e5f1ccd16a" + }, + { + "sourceId": "1413900_84904", + "url": "file/location/d8f7b090-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "generated-84adc784-8d7d-4088-ba51-16fde57fbc21", + "url": "file/sample/location/3881aea9-a731-4e22-9ead-2d6eccc51140" + }, + { + "sourceId": "generated-9e49c58b-0b33-4daf-a39a-8fc91e302328", + "url": "file/sample/location/4bce4154-fb4b-4f0a-887d-a0cd12d4d214" + }, + { + "sourceId": "f02dd3f1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/8937b328-8f0d-4762-8d1f-7d7bc80c3d2e" + }, + { + "sourceId": "f03240c0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/aab6e386-4d59-4b40-b257-9aed12a45446" + } + ] + } +} \ No newline at end of file diff --git a/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java b/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java index e0b8d5753f..d811c6da00 100644 --- a/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java +++ b/jersey/src/main/java/com/netflix/conductor/server/resources/TaskResource.java @@ -226,7 +226,7 @@ public SearchResult search(@QueryParam("start") @DefaultValue("0") @ApiOperation("Get the external uri where the task output payload is to be stored") @Consumes(MediaType.WILDCARD) @Path("/externalstoragelocation") - public ExternalStorageLocation getExternalStorageLocation() { - return taskService.getExternalStorageLocation(); + public ExternalStorageLocation getExternalStorageLocation(@QueryParam("path") String path) { + return taskService.getExternalStorageLocation(path); } } diff --git a/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java b/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java index 5898f8d351..0d7ed64c4b 100644 --- a/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java +++ b/jersey/src/main/java/com/netflix/conductor/server/resources/WorkflowResource.java @@ -247,7 +247,7 @@ public SearchResult searchWorkflowsByTasks(@QueryParam("start") @ApiOperation("Get the uri and path of the external storage where the workflow input payload is to be stored") @Consumes(MediaType.WILDCARD) @Path("/externalstoragelocation") - public ExternalStorageLocation getExternalStorageLocation() { - return workflowService.getExternalStorageLocation(); + public ExternalStorageLocation getExternalStorageLocation(@QueryParam("path") String path) { + return workflowService.getExternalStorageLocation(path); } } diff --git a/mysql-persistence/src/test/java/com/netflix/conductor/config/TestConfiguration.java b/mysql-persistence/src/test/java/com/netflix/conductor/config/TestConfiguration.java index 2b4d1f831e..747fb85d3e 100644 --- a/mysql-persistence/src/test/java/com/netflix/conductor/config/TestConfiguration.java +++ b/mysql-persistence/src/test/java/com/netflix/conductor/config/TestConfiguration.java @@ -15,12 +15,12 @@ */ package com.netflix.conductor.config; -import java.util.Map; - import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.netflix.conductor.core.config.Configuration; +import java.util.Map; + /** * @author Viren * @@ -64,6 +64,46 @@ public String getAppId() { return "workflow"; } + @Override + public Long getWorkflowInputPayloadSizeThresholdKB() { + return 5120L; + } + + @Override + public Long getMaxWorkflowInputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getWorkflowOutputPayloadSizeThresholdKB() { + return 5120L; + } + + @Override + public Long getMaxWorkflowOutputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getTaskInputPayloadSizeThresholdKB() { + return 3072L; + } + + @Override + public Long getMaxTaskInputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getTaskOutputPayloadSizeThresholdKB() { + return 3072L; + } + + @Override + public Long getMaxTaskOutputPayloadSizeThresholdKB() { + return 10240L; + } + @Override public String getProperty(String string, String def) { String val = testProperties.get(string); diff --git a/redis-persistence/src/test/java/com/netflix/conductor/config/TestConfiguration.java b/redis-persistence/src/test/java/com/netflix/conductor/config/TestConfiguration.java index 294f8bc77d..72b929859a 100644 --- a/redis-persistence/src/test/java/com/netflix/conductor/config/TestConfiguration.java +++ b/redis-persistence/src/test/java/com/netflix/conductor/config/TestConfiguration.java @@ -84,6 +84,46 @@ public String getRegion() { return "us-east-1"; } + @Override + public Long getWorkflowInputPayloadSizeThresholdKB() { + return 5120L; + } + + @Override + public Long getMaxWorkflowInputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getWorkflowOutputPayloadSizeThresholdKB() { + return 5120L; + } + + @Override + public Long getMaxWorkflowOutputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getTaskInputPayloadSizeThresholdKB() { + return 3072L; + } + + @Override + public Long getMaxTaskInputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getTaskOutputPayloadSizeThresholdKB() { + return 3072L; + } + + @Override + public Long getMaxTaskOutputPayloadSizeThresholdKB() { + return 10240L; + } + @Override public long getLongProperty(String name, long defaultValue) { return 1000000l; diff --git a/server/src/main/java/com/netflix/conductor/server/ConductorConfig.java b/server/src/main/java/com/netflix/conductor/server/ConductorConfig.java index ad9f09a55b..674361c567 100644 --- a/server/src/main/java/com/netflix/conductor/server/ConductorConfig.java +++ b/server/src/main/java/com/netflix/conductor/server/ConductorConfig.java @@ -92,6 +92,46 @@ public String getAvailabilityZone() { return getProperty("EC2_AVAILABILITY_ZONE", "us-east-1c"); } + @Override + public Long getWorkflowInputPayloadSizeThresholdKB() { + return getLongProperty("conductor.workflow.input.payload.threshold.kb", 5120L); + } + + @Override + public Long getMaxWorkflowInputPayloadSizeThresholdKB() { + return getLongProperty("conductor.max.workflow.input.payload.threshold.kb", 10240L); + } + + @Override + public Long getWorkflowOutputPayloadSizeThresholdKB() { + return getLongProperty("conductor.workflow.output.payload.threshold.kb", 5120L); + } + + @Override + public Long getMaxWorkflowOutputPayloadSizeThresholdKB() { + return getLongProperty("conductor.max.workflow.output.payload.threshold.kb", 10240L); + } + + @Override + public Long getTaskInputPayloadSizeThresholdKB() { + return getLongProperty("conductor.task.input.payload.threshold.kb", 3072L); + } + + @Override + public Long getMaxTaskInputPayloadSizeThresholdKB() { + return getLongProperty("conductor.max.task.input.payload.threshold.kb", 10240L); + } + + @Override + public Long getTaskOutputPayloadSizeThresholdKB() { + return getLongProperty("conductor.task.output.payload.threshold.kb", 3072L); + } + + @Override + public Long getMaxTaskOutputPayloadSizeThresholdKB() { + return getLongProperty("conductor.max.task.output.payload.threshold.kb", 10240L); + } + @Override public int getIntProperty(String key, int defaultValue) { String val = getProperty(key, Integer.toString(defaultValue)); @@ -124,7 +164,7 @@ public String getProperty(String key, String defaultValue) { val = Optional.ofNullable(System.getProperty(key)).orElse(defaultValue); } }catch(Exception e){ - logger.error(e.getMessage(), e); + logger.error("Error reading property: {}", key, e); } return val; } @@ -156,7 +196,7 @@ public List getAdditionalModules() { } return modules; }catch(Exception e) { - logger.warn(e.getMessage(), e); + logger.warn("Error loading additional modules", e); } } return null; diff --git a/test-harness/src/test/java/com/netflix/conductor/tests/integration/End2EndTests.java b/test-harness/src/test/java/com/netflix/conductor/tests/integration/End2EndTests.java index 584a19f669..76e5c5f7b7 100644 --- a/test-harness/src/test/java/com/netflix/conductor/tests/integration/End2EndTests.java +++ b/test-harness/src/test/java/com/netflix/conductor/tests/integration/End2EndTests.java @@ -14,7 +14,7 @@ * limitations under the License. */ /** - * + * */ package com.netflix.conductor.tests.integration; @@ -66,50 +66,50 @@ public class End2EndTests { System.setProperty("db", "memory"); System.setProperty("workflow.elasticsearch.version", "5"); } - - private static TaskClient tc; - - private static WorkflowClient wc; - private static MetadataClient mc; - + private static TaskClient taskClient; + + private static WorkflowClient workflowClient; + + private static MetadataClient metadataClient; + @BeforeClass public static void setup() throws Exception { ConductorServer server = new ConductorServer(new ConductorConfig()); server.start(8080, false); - - tc = new TaskClient(); - tc.setRootURI("http://localhost:8080/api/"); - - wc = new WorkflowClient(); - wc.setRootURI("http://localhost:8080/api/"); - - mc = new MetadataClient(); - mc.setRootURI("http://localhost:8080/api/"); + + taskClient = new TaskClient(); + taskClient.setRootURI("http://localhost:8080/api/"); + + workflowClient = new WorkflowClient(); + workflowClient.setRootURI("http://localhost:8080/api/"); + + metadataClient = new MetadataClient(); + metadataClient.setRootURI("http://localhost:8080/api/"); } - + @Test public void testAll() throws Exception { - assertNotNull(tc); + assertNotNull(taskClient); List defs = new LinkedList<>(); for(int i = 0; i < 5; i++) { TaskDef def = new TaskDef("t" + i, "task " + i); def.setTimeoutPolicy(TimeoutPolicy.RETRY); defs.add(def); } - tc.registerTaskDefs(defs); - List found = tc.getTaskDef(); + metadataClient.registerTaskDefs(defs); + TaskDef found = metadataClient.getTaskDef(defs.get(0).getName()); assertNotNull(found); - assertEquals(defs.size(), found.size()); - + assertEquals(defs.get(0), found); + WorkflowDef def = new WorkflowDef(); def.setName("test"); WorkflowTask t0 = new WorkflowTask(); t0.setName("t0"); t0.setWorkflowTaskType(Type.SIMPLE); t0.setTaskReferenceName("t0"); - + WorkflowTask t1 = new WorkflowTask(); t1.setName("t1"); t1.setWorkflowTaskType(Type.SIMPLE); @@ -117,61 +117,65 @@ public void testAll() throws Exception { def.getTasks().add(t0); def.getTasks().add(t1); - - wc.registerWorkflow(def); - WorkflowDef foundd = wc.getWorkflowDef(def.getName(), null); + + metadataClient.registerWorkflowDef(def); + WorkflowDef foundd = metadataClient.getWorkflowDef(def.getName(), null); assertNotNull(foundd); assertEquals(def.getName(), foundd.getName()); assertEquals(def.getVersion(), foundd.getVersion()); - + String correlationId = "test_corr_id"; - String workflowId = wc.startWorkflow(def.getName(), null, correlationId, new HashMap<>()); + StartWorkflowRequest startWorkflowRequest = new StartWorkflowRequest() + .withName(def.getName()) + .withCorrelationId(correlationId) + .withInput(new HashMap<>()); + String workflowId = workflowClient.startWorkflow(startWorkflowRequest); assertNotNull(workflowId); System.out.println(workflowId); - - Workflow wf = wc.getWorkflow(workflowId, false); + + Workflow wf = workflowClient.getWorkflow(workflowId, false); assertEquals(0, wf.getTasks().size()); assertEquals(workflowId, wf.getWorkflowId()); - List workflowList = wc.getWorkflows(def.getName(), correlationId, false, false); + List workflowList = workflowClient.getWorkflows(def.getName(), correlationId, false, false); assertEquals(1, workflowList.size()); assertEquals(workflowId, workflowList.get(0).getWorkflowId()); - wf = wc.getWorkflow(workflowId, true); + wf = workflowClient.getWorkflow(workflowId, true); assertNotNull(wf); assertEquals(WorkflowStatus.RUNNING, wf.getStatus()); assertEquals(1, wf.getTasks().size()); assertEquals(t0.getTaskReferenceName(), wf.getTasks().get(0).getReferenceTaskName()); assertEquals(workflowId, wf.getWorkflowId()); - - List runningIds = wc.getRunningWorkflow(def.getName(), def.getVersion()); + + List runningIds = workflowClient.getRunningWorkflow(def.getName(), def.getVersion()); assertNotNull(runningIds); assertEquals(1, runningIds.size()); assertEquals(workflowId, runningIds.get(0)); - - List polled = tc.batchPollTasksByTaskType("non existing task", "test", 1, 100); + + List polled = taskClient.batchPollTasksByTaskType("non existing task", "test", 1, 100); assertNotNull(polled); assertEquals(0, polled.size()); - - polled = tc.batchPollTasksByTaskType(t0.getName(), "test", 1, 100); + + polled = taskClient.batchPollTasksByTaskType(t0.getName(), "test", 1, 100); assertNotNull(polled); assertEquals(1, polled.size()); assertEquals(t0.getName(), polled.get(0).getTaskDefName()); Task task = polled.get(0); - - Boolean acked = tc.ack(task.getTaskId(), "test"); + + Boolean acked = taskClient.ack(task.getTaskId(), "test"); assertNotNull(acked); - assertTrue(acked.booleanValue()); - + assertTrue(acked); + task.getOutputData().put("key1", "value1"); task.setStatus(Status.COMPLETED); - tc.updateTask(new TaskResult(task), task.getTaskType()); - - polled = tc.batchPollTasksByTaskType(t0.getName(), "test", 1, 100); + taskClient.updateTask(new TaskResult(task), task.getTaskType()); + + polled = taskClient.batchPollTasksByTaskType(t0.getName(), "test", 1, 100); assertNotNull(polled); assertTrue(polled.toString(), polled.isEmpty()); - - wf = wc.getWorkflow(workflowId, true); + + wf = workflowClient.getWorkflow(workflowId, true); assertNotNull(wf); assertEquals(WorkflowStatus.RUNNING, wf.getStatus()); assertEquals(2, wf.getTasks().size()); @@ -179,38 +183,35 @@ public void testAll() throws Exception { assertEquals(t1.getTaskReferenceName(), wf.getTasks().get(1).getReferenceTaskName()); assertEquals(Task.Status.COMPLETED, wf.getTasks().get(0).getStatus()); assertEquals(Task.Status.SCHEDULED, wf.getTasks().get(1).getStatus()); - - Task taskById = tc.getTaskDetails(task.getTaskId()); + + Task taskById = taskClient.getTaskDetails(task.getTaskId()); assertNotNull(taskById); assertEquals(task.getTaskId(), taskById.getTaskId()); - - - List getTasks = tc.getPendingTasksByType(t0.getName(), null, 1); + + List getTasks = taskClient.getPendingTasksByType(t0.getName(), null, 1); assertNotNull(getTasks); assertEquals(0, getTasks.size()); //getTasks only gives pending tasks - - - getTasks = tc.getPendingTasksByType(t1.getName(), null, 1); + getTasks = taskClient.getPendingTasksByType(t1.getName(), null, 1); assertNotNull(getTasks); assertEquals(1, getTasks.size()); - Task pending = tc.getPendingTaskForWorkflow(workflowId, t1.getTaskReferenceName()); + Task pending = taskClient.getPendingTaskForWorkflow(workflowId, t1.getTaskReferenceName()); assertNotNull(pending); assertEquals(t1.getTaskReferenceName(), pending.getReferenceTaskName()); assertEquals(workflowId, pending.getWorkflowInstanceId()); - + Thread.sleep(1000); - SearchResult searchResult = wc.search("workflowType='" + def.getName() + "'"); + SearchResult searchResult = workflowClient.search("workflowType='" + def.getName() + "'"); assertNotNull(searchResult); assertEquals(1, searchResult.getTotalHits()); - - wc.terminateWorkflow(workflowId, "terminate reason"); - wf = wc.getWorkflow(workflowId, true); + + workflowClient.terminateWorkflow(workflowId, "terminate reason"); + wf = workflowClient.getWorkflow(workflowId, true); assertNotNull(wf); assertEquals(WorkflowStatus.TERMINATED, wf.getStatus()); - - wc.restart(workflowId); - wf = wc.getWorkflow(workflowId, true); + + workflowClient.restart(workflowId); + wf = workflowClient.getWorkflow(workflowId, true); assertNotNull(wf); assertEquals(WorkflowStatus.RUNNING, wf.getStatus()); assertEquals(1, wf.getTasks().size()); @@ -234,11 +235,11 @@ public void testMetadataWorkflowDefinition() { def.getTasks().add(t0); def.getTasks().add(t1); - mc.registerWorkflowDef(def); - mc.unregisterWorkflowDef("testWorkflowDel", 1); + metadataClient.registerWorkflowDef(def); + metadataClient.unregisterWorkflowDef("testWorkflowDel", 1); try { - WorkflowDef getDef = mc.getWorkflowDef("testWorkflowDel", 1); + metadataClient.getWorkflowDef("testWorkflowDel", 1); } catch (ConductorClientException e) { int statusCode = e.getStatus(); String errorMessage = e.getMessage(); @@ -273,16 +274,16 @@ public void testUpdateWorkflow() { WorkflowDef def = new WorkflowDef(); def.setName("testWorkflowDel"); def.setVersion(1); - mc.registerWorkflowDef(def); + metadataClient.registerWorkflowDef(def); def.setVersion(2); List workflowList = new ArrayList<>(); workflowList.add(def); - mc.updateWorkflowDefs(workflowList); - WorkflowDef def1 = mc.getWorkflowDef(def.getName(), 2); + metadataClient.updateWorkflowDefs(workflowList); + WorkflowDef def1 = metadataClient.getWorkflowDef(def.getName(), 2); assertNotNull(def1); try{ - mc.getTaskDef("test"); + metadataClient.getTaskDef("test"); } catch (ConductorClientException e) { int statuCode = e.getStatus(); assertEquals(404, statuCode); @@ -297,7 +298,7 @@ public void testUpdateWorkflow() { public void testStartWorkflow() { StartWorkflowRequest startWorkflowRequest = new StartWorkflowRequest(); try{ - wc.startWorkflow(startWorkflowRequest); + workflowClient.startWorkflow(startWorkflowRequest); } catch (IllegalArgumentException e) { assertEquals("Workflow name cannot be null or empty", e.getMessage()); } @@ -307,7 +308,7 @@ public void testStartWorkflow() { public void testUpdateTask() { TaskResult taskResult = new TaskResult(); try{ - tc.updateTask(taskResult, "taskTest"); + taskClient.updateTask(taskResult, "taskTest"); } catch (ConductorClientException e){ int statuCode = e.getStatus(); assertEquals(400, statuCode); @@ -319,7 +320,7 @@ public void testUpdateTask() { @Test public void testGetWorfklowNotFound() { try{ - wc.getWorkflow("w123", true); + workflowClient.getWorkflow("w123", true); } catch (ConductorClientException e) { assertEquals(404, e.getStatus()); assertEquals("No such workflow found by id: w123", e.getMessage()); @@ -331,7 +332,7 @@ public void testGetWorfklowNotFound() { public void testEmptyCreateWorkflowDef() { try{ WorkflowDef workflowDef = new WorkflowDef(); - mc.registerWorkflowDef(workflowDef); + metadataClient.registerWorkflowDef(workflowDef); } catch (ConductorClientException e){ assertEquals(400, e.getStatus()); assertEquals("Workflow name cannot be null or empty", e.getMessage()); @@ -345,7 +346,7 @@ public void testUpdateWorkflowDef() { WorkflowDef workflowDef = new WorkflowDef(); List workflowDefList = new ArrayList<>(); workflowDefList.add(workflowDef); - mc.updateWorkflowDefs(workflowDefList); + metadataClient.updateWorkflowDefs(workflowDefList); } catch (ConductorClientException e){ assertEquals(400, e.getStatus()); assertEquals("WorkflowDef name cannot be null", e.getMessage()); @@ -355,13 +356,13 @@ public void testUpdateWorkflowDef() { @Test(expected = Test.None.class /* no exception expected */) public void testGetTaskInProgress() { - tc.getPendingTaskForWorkflow("test", "t1"); + taskClient.getPendingTaskForWorkflow("test", "t1"); } @Test public void testRemoveTaskFromTaskQueue() { try { - tc.removeTaskFromQueue("test", "fakeQueue"); + taskClient.removeTaskFromQueue("test", "fakeQueue"); } catch (ConductorClientException e) { assertEquals(404, e.getStatus()); } @@ -370,7 +371,7 @@ public void testRemoveTaskFromTaskQueue() { @Test public void testTaskByTaskId() { try { - tc.getTaskDetails("test123"); + taskClient.getTaskDetails("test123"); } catch (ConductorClientException e) { assertEquals(404, e.getStatus()); assertEquals("No such task found by taskId: test123", e.getMessage()); @@ -379,6 +380,6 @@ public void testTaskByTaskId() { @Test(expected = Test.None.class /* no exception expected */) public void testListworkflowsByCorrelationId() { - wc.getWorkflows("test", "test12", false, false); + workflowClient.getWorkflows("test", "test12", false, false); } } diff --git a/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java b/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java index 617744951d..dbfe360273 100644 --- a/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java +++ b/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java @@ -252,69 +252,6 @@ public void init() { registered = true; } - @Test - public void testWorkflowInputWithExternalPayload() { - WorkflowDef found = metadataService.getWorkflowDef(LINEAR_WORKFLOW_T1_T2, 1); - assertNotNull(found); - - String workflowInputPath = "workflow/input"; - String correlationId = "wf_external_storage"; - String workflowId = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2, 1, correlationId, null, workflowInputPath, null, null); - assertNotNull(workflowId); - - Workflow workflow = workflowExecutionService.getExecutionStatus(workflowId, true); - assertNotNull(workflow); - assertNull("The workflow input should not be persisted", workflow.getInput()); - assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); - assertEquals(workflow.getReasonForIncompletion(), WorkflowStatus.RUNNING, workflow.getStatus()); - assertEquals(1, workflow.getTasks().size()); - - // Polling for the first task - Task task = workflowExecutionService.poll("junit_task_1", "task1.junit.worker"); - assertNotNull(task); - assertEquals("junit_task_1", task.getTaskType()); - assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); - assertEquals(workflowId, task.getWorkflowInstanceId()); - - // update first task with COMPLETED - String taskOutputPath = "task/output"; - task.setOutputData(null); - task.setExternalOutputPayloadStoragePath(taskOutputPath); - task.setStatus(COMPLETED); - workflowExecutionService.updateTask(task); - - workflow = workflowExecutionService.getExecutionStatus(workflowId, true); - assertNotNull(workflow); - assertNull("The workflow input should not be persisted", workflow.getInput()); - assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); - assertEquals(workflow.getReasonForIncompletion(), WorkflowStatus.RUNNING, workflow.getStatus()); - assertEquals(2, workflow.getTasks().size()); - assertNull("The task output should not be persisted", workflow.getTasks().get(0).getOutputData()); - assertEquals(taskOutputPath, workflow.getTasks().get(0).getExternalOutputPayloadStoragePath()); - - // Polling for the second task - task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker"); - assertNotNull(task); - assertEquals("junit_task_2", task.getTaskType()); - assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); - assertEquals(workflowId, task.getWorkflowInstanceId()); - - // update first task with COMPLETED - task.getOutputData().put("op", "success_task2"); - task.setStatus(COMPLETED); - workflowExecutionService.updateTask(task); - - workflow = workflowExecutionService.getExecutionStatus(workflowId, true); - assertNotNull(workflow); - assertNull("The workflow input should not be persisted", workflow.getInput()); - assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); - assertEquals(WorkflowStatus.COMPLETED, workflow.getStatus()); - assertEquals(2, workflow.getTasks().size()); - assertNull("The task output should not be persisted", workflow.getTasks().get(0).getOutputData()); - assertEquals(taskOutputPath, workflow.getTasks().get(0).getExternalOutputPayloadStoragePath()); - } - - @Test public void testWorkflowWithNoTasks() { @@ -3910,6 +3847,171 @@ public void testRateLimiting() { } } + @Test + public void testWorkflowUsingExternalPayloadStorage() { + WorkflowDef found = metadataService.getWorkflowDef(LINEAR_WORKFLOW_T1_T2, 1); + assertNotNull(found); + Map outputParameters = found.getOutputParameters(); + outputParameters.put("workflow_output", "${t1.output.op}"); + metadataService.updateWorkflowDef(found); + + String workflowInputPath = "workflow/input"; + String correlationId = "wf_external_storage"; + String workflowId = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2, 1, correlationId, null, workflowInputPath, null, null); + assertNotNull(workflowId); + + Workflow workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertNull("The workflow input should not be persisted", workflow.getInput()); + assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); + assertEquals(workflow.getReasonForIncompletion(), WorkflowStatus.RUNNING, workflow.getStatus()); + assertEquals(1, workflow.getTasks().size()); + + // Polling for the first task + Task task = workflowExecutionService.poll("junit_task_1", "task1.junit.worker"); + assertNotNull(task); + assertEquals("junit_task_1", task.getTaskType()); + assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); + assertEquals(workflowId, task.getWorkflowInstanceId()); + + // update first task with COMPLETED + String taskOutputPath = "task/output"; + task.setOutputData(null); + task.setExternalOutputPayloadStoragePath(taskOutputPath); + task.setStatus(COMPLETED); + workflowExecutionService.updateTask(task); + + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertNull("The workflow input should not be persisted", workflow.getInput()); + assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); + assertEquals(workflow.getReasonForIncompletion(), WorkflowStatus.RUNNING, workflow.getStatus()); + assertEquals(2, workflow.getTasks().size()); + assertNull("The first task output should not be persisted", workflow.getTasks().get(0).getOutputData()); + assertNull("The second task input should not be persisted", workflow.getTasks().get(1).getInputData()); + assertEquals(taskOutputPath, workflow.getTasks().get(0).getExternalOutputPayloadStoragePath()); + assertEquals("task/input", workflow.getTasks().get(1).getExternalInputPayloadStoragePath()); + + // Polling for the second task + task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker"); + assertNotNull(task); + assertEquals("junit_task_2", task.getTaskType()); + assertNull(task.getInputData()); + assertNotNull(task.getExternalInputPayloadStoragePath()); + assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); + assertEquals(workflowId, task.getWorkflowInstanceId()); + + // update second task with COMPLETED + task.getOutputData().put("op", "success_task2"); + task.setStatus(COMPLETED); + workflowExecutionService.updateTask(task); + + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertNull("The workflow input should not be persisted", workflow.getInput()); + assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); + assertEquals(WorkflowStatus.COMPLETED, workflow.getStatus()); + assertEquals(2, workflow.getTasks().size()); + assertNull("The first task output should not be persisted", workflow.getTasks().get(0).getOutputData()); + assertNull("The second task input should not be persisted", workflow.getTasks().get(1).getInputData()); + assertEquals(taskOutputPath, workflow.getTasks().get(0).getExternalOutputPayloadStoragePath()); + assertEquals("task/input", workflow.getTasks().get(1).getExternalInputPayloadStoragePath()); + assertNull(workflow.getOutput()); + assertNotNull(workflow.getExternalOutputPayloadStoragePath()); + assertEquals("workflow/output", workflow.getExternalOutputPayloadStoragePath()); + } + + @Test + public void testRetryWorkflowUsingExternalPayloadStorage() { + WorkflowDef found = metadataService.getWorkflowDef(LINEAR_WORKFLOW_T1_T2, 1); + assertNotNull(found); + Map outputParameters = found.getOutputParameters(); + outputParameters.put("workflow_output", "${t1.output.op}"); + metadataService.updateWorkflowDef(found); + + String taskName = "junit_task_2"; + TaskDef taskDef = metadataService.getTaskDef(taskName); + taskDef.setRetryCount(2); + taskDef.setRetryDelaySeconds(0); + metadataService.updateTaskDef(taskDef); + + String workflowInputPath = "workflow/input"; + String correlationId = "wf_external_storage"; + String workflowId = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2, 1, correlationId, null, workflowInputPath, null, null); + assertNotNull(workflowId); + + Workflow workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertNull("The workflow input should not be persisted", workflow.getInput()); + assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); + assertEquals(workflow.getReasonForIncompletion(), WorkflowStatus.RUNNING, workflow.getStatus()); + assertEquals(1, workflow.getTasks().size()); + + // Polling for the first task + Task task = workflowExecutionService.poll("junit_task_1", "task1.junit.worker"); + assertNotNull(task); + assertEquals("junit_task_1", task.getTaskType()); + assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); + assertEquals(workflowId, task.getWorkflowInstanceId()); + + // update first task with COMPLETED + String taskOutputPath = "task/output"; + task.setOutputData(null); + task.setExternalOutputPayloadStoragePath(taskOutputPath); + task.setStatus(COMPLETED); + workflowExecutionService.updateTask(task); + + // Polling for the second task + task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker"); + assertNotNull(task); + assertEquals("junit_task_2", task.getTaskType()); + assertNull(task.getInputData()); + assertNotNull(task.getExternalInputPayloadStoragePath()); + assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); + assertEquals(workflowId, task.getWorkflowInstanceId()); + + // update second task with FAILED + task.getOutputData().put("op", "failed_task2"); + task.setStatus(FAILED); + workflowExecutionService.updateTask(task); + + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertNull("The workflow input should not be persisted", workflow.getInput()); + assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); + assertEquals(WorkflowStatus.RUNNING, workflow.getStatus()); + + // Polling again for the second task + task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker"); + assertNotNull(task); + assertEquals("junit_task_2", task.getTaskType()); + assertNull(task.getInputData()); + assertNotNull(task.getExternalInputPayloadStoragePath()); + assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); + assertEquals(workflowId, task.getWorkflowInstanceId()); + + // update second task with COMPLETED + task.getOutputData().put("op", "success_task2"); + task.setStatus(COMPLETED); + workflowExecutionService.updateTask(task); + + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertNull("The workflow input should not be persisted", workflow.getInput()); + assertEquals(workflowInputPath, workflow.getExternalInputPayloadStoragePath()); + assertEquals(WorkflowStatus.COMPLETED, workflow.getStatus()); + assertEquals(3, workflow.getTasks().size()); + assertNull("The first task output should not be persisted", workflow.getTasks().get(0).getOutputData()); + assertNull("The second task input should not be persisted", workflow.getTasks().get(1).getInputData()); + assertNull("The second task input should not be persisted", workflow.getTasks().get(2).getInputData()); + assertEquals(taskOutputPath, workflow.getTasks().get(0).getExternalOutputPayloadStoragePath()); + assertEquals("task/input", workflow.getTasks().get(1).getExternalInputPayloadStoragePath()); + assertEquals("task/input", workflow.getTasks().get(2).getExternalInputPayloadStoragePath()); + assertNull(workflow.getOutput()); + assertNotNull(workflow.getExternalOutputPayloadStoragePath()); + assertEquals("workflow/output", workflow.getExternalOutputPayloadStoragePath()); + } + private void createSubWorkflow() { WorkflowTask wft1 = new WorkflowTask(); diff --git a/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockConfiguration.java b/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockConfiguration.java new file mode 100644 index 0000000000..354f5ad2d2 --- /dev/null +++ b/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockConfiguration.java @@ -0,0 +1,135 @@ +/* + * Copyright 2016 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.netflix.conductor.tests.utils; + +import com.netflix.conductor.core.config.Configuration; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.Map; + +public class MockConfiguration implements Configuration { + + @Override + public int getSweepFrequency() { + return 30; + } + + @Override + public boolean disableSweep() { + return false; + } + + @Override + public boolean disableAsyncWorkers() { + return false; + } + + @Override + public String getServerId() { + try { + return InetAddress.getLocalHost().getHostName(); + } catch (UnknownHostException e) { + return "unknown"; + } + } + + @Override + public String getEnvironment() { + return "test"; + } + + @Override + public String getStack() { + return "test"; + } + + @Override + public String getAppId() { + return "conductor"; + } + + @Override + public String getProperty(String string, String def) { + return "dummy"; + } + + @Override + public String getAvailabilityZone() { + return "us-east-1c"; + } + + @Override + public int getIntProperty(String string, int def) { + return 100; + } + + @Override + public String getRegion() { + return "us-east-1"; + } + + @Override + public Long getWorkflowInputPayloadSizeThresholdKB() { + return 10L; + } + + @Override + public Long getMaxWorkflowInputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getWorkflowOutputPayloadSizeThresholdKB() { + return 10L; + } + + @Override + public Long getMaxWorkflowOutputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getTaskInputPayloadSizeThresholdKB() { + return 1L; + } + + @Override + public Long getMaxTaskInputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Long getTaskOutputPayloadSizeThresholdKB() { + return 10L; + } + + @Override + public Long getMaxTaskOutputPayloadSizeThresholdKB() { + return 10240L; + } + + @Override + public Map getAll() { + return null; + } + + @Override + public long getLongProperty(String name, long defaultValue) { + return 1000000L; + } +} diff --git a/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java b/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java index 78f7d24463..694b54eddf 100644 --- a/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java +++ b/test-harness/src/test/java/com/netflix/conductor/tests/utils/MockExternalPayloadStorage.java @@ -30,8 +30,18 @@ public class MockExternalPayloadStorage implements ExternalPayloadStorage { private ObjectMapper objectMapper = new ObjectMapper(); @Override - public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType) { - return null; + public ExternalStorageLocation getLocation(Operation operation, PayloadType payloadType, String path) { + ExternalStorageLocation location = new ExternalStorageLocation(); + location.setUri("http://some/uri"); + switch (payloadType) { + case TASK_INPUT: + location.setPath("task/input"); + break; + case WORKFLOW_OUTPUT: + location.setPath("workflow/output"); + break; + } + return location; } @Override @@ -41,7 +51,6 @@ public void upload(String path, InputStream payload, long payloadSize) { @Override public InputStream download(String path) { try { - Map payload = getPayload(path); String jsonString = objectMapper.writeValueAsString(payload); return new ByteArrayInputStream(jsonString.getBytes()); @@ -50,16 +59,22 @@ public InputStream download(String path) { } } + @SuppressWarnings("unchecked") private Map getPayload(String path) { Map stringObjectMap = new HashMap<>(); - switch (path) { - case "workflow/input": - stringObjectMap.put("param1", "p1 value"); - stringObjectMap.put("param2", "p2 value"); - break; - case "task/output": - stringObjectMap.put("op", "success_task1"); - break; + try { + switch (path) { + case "workflow/input": + stringObjectMap.put("param1", "p1 value"); + stringObjectMap.put("param2", "p2 value"); + return stringObjectMap; + case "task/input": + case "task/output": + InputStream stream = MockExternalPayloadStorage.class.getResourceAsStream("/payload.json"); + return objectMapper.readValue(stream, Map.class); + } + } catch (IOException e) { + // just handle this exception here and return empty map so that test will fail in case this exception is thrown } return stringObjectMap; } diff --git a/test-harness/src/test/java/com/netflix/conductor/tests/utils/TestModule.java b/test-harness/src/test/java/com/netflix/conductor/tests/utils/TestModule.java index c49d06f146..83643cfbf1 100644 --- a/test-harness/src/test/java/com/netflix/conductor/tests/utils/TestModule.java +++ b/test-harness/src/test/java/com/netflix/conductor/tests/utils/TestModule.java @@ -14,7 +14,7 @@ * limitations under the License. */ /** - * + * */ package com.netflix.conductor.tests.utils; @@ -32,7 +32,6 @@ import com.netflix.conductor.dao.dynomite.RedisMetadataDAO; import com.netflix.conductor.dao.dynomite.queue.DynoQueueDAO; import com.netflix.conductor.redis.utils.JedisMock; -import com.netflix.conductor.server.ConductorConfig; import com.netflix.dyno.queues.ShardSupplier; import redis.clients.jedis.JedisCommands; @@ -47,11 +46,11 @@ * */ public class TestModule extends AbstractModule { - + private int maxThreads = 50; - + private ExecutorService executorService; - + @Override protected void configure() { @@ -61,40 +60,40 @@ protected void configure() { configureExecutorService(); - ConductorConfig config = new ConductorConfig(); + MockConfiguration config = new MockConfiguration(); bind(Configuration.class).toInstance(config); JedisCommands jedisMock = new JedisMock(); DynoQueueDAO queueDao = new DynoQueueDAO(jedisMock, jedisMock, new ShardSupplier() { - + @Override public Set getQueueShards() { return new HashSet<>(Collections.singletonList("a")); } - + @Override public String getCurrentShard() { return "a"; } }, config); - + bind(MetadataDAO.class).to(RedisMetadataDAO.class); bind(ExecutionDAO.class).to(RedisExecutionDAO.class); bind(DynoQueueDAO.class).toInstance(queueDao); bind(QueueDAO.class).to(DynoQueueDAO.class); - bind(IndexDAO.class).to(MockIndexDAO.class); + bind(IndexDAO.class).to(MockIndexDAO.class); DynoProxy proxy = new DynoProxy(jedisMock); bind(DynoProxy.class).toInstance(proxy); install(new CoreModule()); bind(UserTask.class).asEagerSingleton(); bind(ExternalPayloadStorage.class).to(MockExternalPayloadStorage.class); } - + @Provides public ExecutorService getExecutorService(){ return this.executorService; } - + private void configureExecutorService(){ AtomicInteger count = new AtomicInteger(0); this.executorService = java.util.concurrent.Executors.newFixedThreadPool(maxThreads, runnable -> { diff --git a/test-harness/src/test/resources/payload.json b/test-harness/src/test/resources/payload.json new file mode 100644 index 0000000000..7311a79bb5 --- /dev/null +++ b/test-harness/src/test/resources/payload.json @@ -0,0 +1,423 @@ +{ + "imageType": "TEST_SAMPLE", + "op": { + "TEST_SAMPLE": [ + { + "sourceId": "1413900_10830", + "url": "file/location/a0bdc4d0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_50241", + "url": "file/location/cd4e00a0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "generated-55ee8663-85c2-42d3-aca2-4076707e6d4e", + "url": "file/sample/location/e008d018-63d7-44b2-b07e-c7435430ac71" + }, + { + "sourceId": "generated-14056154-1544-4350-81db-b3751fe44777", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-0b0ae5ea-d5c5-410c-adc9-bf16d2909c2e", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-08869779-614d-417c-bfea-36a3f8f199da", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "generated-e117db45-1c48-45d0-b751-89386eb2d81d", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "f0221421-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/4a009209-002f-4b58-8b96-cb2198f8ba3c" + }, + { + "sourceId": "f0252161-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/55b56298-5e7a-4949-b919-88c5c9557e8e" + }, + { + "sourceId": "f038d070-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/3c4804f4-e826-436f-90c9-52b8d9266d52" + }, + { + "sourceId": "f04e0621-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/689283a1-1816-48ef-83da-7f9ac874bf45" + }, + { + "sourceId": "f04ddf10-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/586666ae-7321-445a-80b6-323c8c241ecd" + }, + { + "sourceId": "f05950c0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/31795cc4-2590-4b20-a617-deaa18301f99" + }, + { + "sourceId": "1413900_46819", + "url": "file/location/c74497a0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_11177", + "url": "file/location/a231c730-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_48713", + "url": "file/location/ca638ae0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_48525", + "url": "file/location/ca0c9140-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_73303", + "url": "file/location/d5943a40-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "1413900_55202", + "url": "file/location/d1a4d7a0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "generated-61413adf-3c10-4484-b25d-e238df898f45", + "url": "file/sample/location/e008d018-63d7-44b2-b07e-c7435430ac71" + }, + { + "sourceId": "generated-addca397-f050-4339-ae86-9ba8c4e1b0d5", + "url": "file/sample/location/838a0ddb-a315-453a-8b8a-fa795f9d7691" + }, + { + "sourceId": "generated-e4de9810-0f69-4593-8926-01ed82cbebcb", + "url": "file/sample/location/838a0ddb-a315-453a-8b8a-fa795f9d7691" + }, + { + "sourceId": "generated-e16e2074-7af6-4700-ab05-ca41ba9c9ab4", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-341c86f8-57a5-40e1-8842-3eb41dd9f528", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-88c2ea9b-cef7-4120-8043-b92713d8fade", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-3f6a731f-3c92-4677-9923-f80b8a6be632", + "url": "file/sample/location/3881aea9-a731-4e22-9ead-2d6eccc51140" + }, + { + "sourceId": "generated-1508b871-64de-47ce-8b07-76c5cb3f3e1e", + "url": "file/sample/location/a2e4195f-3900-45b4-9335-45f85fca6467" + }, + { + "sourceId": "generated-1406dce8-7b9c-4956-a7e8-78721c476ce9", + "url": "file/sample/location/a2e4195f-3900-45b4-9335-45f85fca6467" + }, + { + "sourceId": "f0206671-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/35ebee36-3072-44c5-abb5-702a5a3b1a91" + }, + { + "sourceId": "f01f5501-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/d3a9133d-c681-4910-a769-8195526ae634" + }, + { + "sourceId": "f022b060-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/8fc1413d-170e-4644-a554-5e0c596b225c" + }, + { + "sourceId": "f02fa8b1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/35bed0a2-7def-457b-bded-4f4d7d94f76e" + }, + { + "sourceId": "f031f2a0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/a5a2ea1f-8d13-429c-a44d-3057d21f608a" + }, + { + "sourceId": "f0424650-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/1c599ffc-4f10-4c0b-8d9a-ae41c7256113" + }, + { + "sourceId": "f04ec970-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/8404a421-e1a6-41cf-af63-a35ccb474457" + }, + { + "sourceId": "1413900_47197", + "url": "file/location/c81b6fa0-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "generated-2a63c0c8-62ea-44a4-a33b-f0b3047e8b00", + "url": "file/sample/location/e008d018-63d7-44b2-b07e-c7435430ac71" + }, + { + "sourceId": "generated-b27face7-3589-4209-944a-5153b20c5996", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-144675b3-9321-48d2-8b5b-e19a40d30ef2", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-8cbe821e-b1fb-48ce-beb5-735319af4db6", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-ecc4ea47-9bad-4b91-97c7-35f4ea6fb479", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-c1eb9ed0-8560-4e09-a748-f926edb7cdc2", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "generated-6bed81fd-c777-4c61-8da1-0bb7f7cf0082", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "generated-852e5510-dd5d-4900-a614-854148fcc716", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "generated-f4dedcb7-37c9-4ba9-ab37-64ec9be7c882", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "f0259691-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/721bc0de-e75f-4386-8b2e-ca84eb653596" + }, + { + "sourceId": "f02b3be1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/d2043b17-8ce5-42ee-a5e4-81c68f0c4838" + }, + { + "sourceId": "f02b62f0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/63931561-3b5b-4ffe-af47-da2c9de94684" + }, + { + "sourceId": "f0315660-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/d99ed629-2885-4e4a-8a1b-22e487b875fa" + }, + { + "sourceId": "f0306c00-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/6f8e673a-7003-44aa-96b9-e2ed8a4654ff" + }, + { + "sourceId": "f033c760-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/627c00f9-14b3-4057-b6e2-0f962ad0308e" + }, + { + "sourceId": "f03526f1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/fafabaf9-fe58-4a9a-b555-026521aeb2fe" + }, + { + "sourceId": "f03acc41-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/6c9fed2c-558a-4db3-8360-659b5e8c46e4" + }, + { + "sourceId": "f0463df1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/e9fb83d2-5f14-4442-92b5-67e613f2e35f" + }, + { + "sourceId": "f04fb3d0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/e7a0f82f-be8d-4ada-a4b1-13e8165e08be" + }, + { + "sourceId": "f05272f0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/9aba488a-22b3-4932-85a7-52c461203541" + }, + { + "sourceId": "f0581841-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/457415f6-6d0c-4304-8533-0d5b43fac564" + }, + { + "sourceId": "generated-8fefb48c-6fde-4fd6-8f33-a1f3f3b62105", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-30c61aa5-f5bd-4077-8c32-336b87acbe96", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-d5da37db-d486-46d4-8f7d-1e0710a77eb5", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-77af26fe-9e22-48af-99e3-f63f10fbe6de", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-2e807016-3d11-4b60-bec7-c380a608b67d", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-615d02e9-62c2-43ab-9df7-753b6b8e2c22", + "url": "file/sample/location/519f6c80-96ef-440f-9d37-ccf36c7d1e5d" + }, + { + "sourceId": "generated-3e1600fd-a626-4ee6-972b-5f0187e96c38", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "generated-1dcb208c-6a58-4334-a60c-6fb54c8a2af5", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "f024ac30-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/0af2107b-4231-4d23-bef3-4e417ac6c5d3" + }, + { + "sourceId": "f0282ea1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/0f592681-fd23-4194-ae43-42f61c664485" + }, + { + "sourceId": "f02c4d50-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/ec46b9a3-99af-410a-af7d-726f8854909f" + }, + { + "sourceId": "f02b8a00-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/aed7e5da-b524-4d41-b264-28ce615ec826" + }, + { + "sourceId": "f02b14d1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/b88c9055-ab0d-4d27-a405-265ba2a15f0c" + }, + { + "sourceId": "f03044f1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/fb8c4df9-d59e-4ac3-880e-4ea94cd880a4" + }, + { + "sourceId": "f034ffe1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/59f3fbe8-b300-4861-9b2f-dac7b15aea7d" + }, + { + "sourceId": "f03c2bd0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/19a06d54-41ed-419d-9947-f10cd5f0d85c" + }, + { + "sourceId": "f03fae41-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/a9a48a62-7d62-4f67-b281-cc6fdc1e722c" + }, + { + "sourceId": "f0455390-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/0aeffc0a-a5ad-46ff-abab-1b3bc6a5840a" + }, + { + "sourceId": "f04b1ff1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/9a08aaed-c125-48f7-9d1d-fd11266c2b12" + }, + { + "sourceId": "f04cf4b1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/17a6e0f9-aa64-411f-9af7-837c84f7443f" + }, + { + "sourceId": "f0511360-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/fb633c73-cb33-4806-bc08-049024644856" + }, + { + "sourceId": "f0538460-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/a7012248-6769-42da-a6c8-d4b831f6efce" + }, + { + "sourceId": "f058db91-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/bcf71522-6168-48c4-86c9-995bca60ae51" + }, + { + "sourceId": "generated-adf005c4-95c1-4904-9968-09cc19a26bfe", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-c4d367a4-4cdc-412e-af79-09b227f2e3ba", + "url": "file/sample/location/3d927190-1c4d-4af2-91cf-2968d3ccfe70" + }, + { + "sourceId": "generated-48dba018-f884-49db-b87e-67274e244c8f", + "url": "file/sample/location/4bce4154-fb4b-4f0a-887d-a0cd12d4d214" + }, + { + "sourceId": "generated-26700b83-4892-420e-8b46-1ee21eba75fb", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "generated-632f3198-c0dc-4348-974f-51684d4e443e", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "generated-86e2dd1d-1aa4-4dbe-b37b-b488f5dd1c70", + "url": "file/sample/location/e87da4d1-72da-47a3-801d-43e01c050c89" + }, + { + "sourceId": "f04134e0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/ff8f59bf-7757-4d51-a7e4-619f3e8ffaf2" + }, + { + "sourceId": "f04f65b0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/d66467d1-3ac6-4041-8d15-e722ee07231f" + }, + { + "sourceId": "1413900_15255", + "url": "file/location/a9e20260-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "generated-e953493b-cbe3-4319-885e-00c82089c76c", + "url": "file/sample/location/ec16facd-86e3-4c3f-8dfb-7a2ad3a4e18c" + }, + { + "sourceId": "generated-65c54676-3adb-4ef0-b65e-8e2a49533cbf", + "url": "file/sample/location/07ec28a1-189e-4f2a-9dd5-f3ca68ce977d" + }, + { + "sourceId": "f02ac6b0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/21568877-07a5-411f-9715-5e92806c4448" + }, + { + "sourceId": "f02fcfc1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/f3b1f1a2-48d3-475d-a607-2e5a1fe532e7" + }, + { + "sourceId": "f03526f0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/84a40c66-d925-4a4a-ba62-8491d26e29e9" + }, + { + "sourceId": "f03e75c1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/e84c00e8-a148-46cf-9a0b-431c4c2aeb08" + }, + { + "sourceId": "f0429471-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/178de9fa-7cc8-457a-8fb6-5c080e6163ea" + }, + { + "sourceId": "f047eba0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/18d153aa-e13b-4264-ae03-f3da75eb425b" + }, + { + "sourceId": "f04fdae0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/7c843e53-8d87-47cf-bca5-1a02e7f5e33f" + }, + { + "sourceId": "f0553210-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/26bacd65-9082-4d83-9506-90e5f1ccd16a" + }, + { + "sourceId": "1413900_84904", + "url": "file/location/d8f7b090-5315-11e8-bf88-0efd527701fc" + }, + { + "sourceId": "generated-84adc784-8d7d-4088-ba51-16fde57fbc21", + "url": "file/sample/location/3881aea9-a731-4e22-9ead-2d6eccc51140" + }, + { + "sourceId": "generated-9e49c58b-0b33-4daf-a39a-8fc91e302328", + "url": "file/sample/location/4bce4154-fb4b-4f0a-887d-a0cd12d4d214" + }, + { + "sourceId": "f02dd3f1-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/8937b328-8f0d-4762-8d1f-7d7bc80c3d2e" + }, + { + "sourceId": "f03240c0-86e8-11e8-af77-0a2ba4eae3ec", + "url": "file/test/location/aab6e386-4d59-4b40-b257-9aed12a45446" + } + ] + } +} \ No newline at end of file From b1db3b57871de208013171cca9439a20bebabe56 Mon Sep 17 00:00:00 2001 From: Anoop Panicker Date: Sat, 8 Sep 2018 00:47:16 -0700 Subject: [PATCH 4/4] metrics, javadoc, refactor --- .../conductor/client/http/TaskClient.java | 4 +- .../conductor/client/http/WorkflowClient.java | 14 +- .../conductor/common/run/Workflow.java | 5 +- .../core/execution/DeciderService.java | 35 ++-- .../core/execution/WorkflowExecutor.java | 7 +- .../utils/ExternalPayloadStorageUtils.java | 26 ++- .../core/utils/S3PayloadStorage.java | 14 +- .../netflix/conductor/metrics/Monitors.java | 4 + .../core/execution/TestDeciderService.java | 30 ++- .../dao/dynomite/RedisExecutionDAO.java | 24 +-- .../integration/WorkflowServiceTest.java | 192 +++++++----------- 11 files changed, 180 insertions(+), 175 deletions(-) diff --git a/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java b/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java index 438d9cde6c..970de846c1 100644 --- a/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java +++ b/client/src/main/java/com/netflix/conductor/client/http/TaskClient.java @@ -242,6 +242,8 @@ public Task getPendingTaskForWorkflow(String workflowId, String taskReferenceNam /** * Updates the result of a task execution. + * If the size of the task output payload is bigger than {@link ConductorClientConfiguration#getTaskOutputPayloadThresholdKB()}, + * it is uploaded to {@link ExternalPayloadStorage}, if enabled, else the task is marked as FAILED_WITH_TERMINAL_ERROR. * * @param taskResult the {@link TaskResult} of the executed task to be updated. * @param taskType the type of the task @@ -259,7 +261,7 @@ public void updateTask(TaskResult taskResult, String taskType) { long payloadSizeThreshold = conductorClientConfiguration.getTaskOutputPayloadThresholdKB() * 1024; if (taskResultSize > payloadSizeThreshold) { if (!conductorClientConfiguration.isExternalPayloadStorageEnabled() - || taskResultSize > (conductorClientConfiguration.getTaskOutputMaxPayloadThresholdKB() * 1024)) { + || taskResultSize > conductorClientConfiguration.getTaskOutputMaxPayloadThresholdKB() * 1024) { taskResult.setReasonForIncompletion(String.format("The TaskResult payload size: %d is greater than the permissible %d MB", taskResultSize, payloadSizeThreshold)); taskResult.setStatus(TaskResult.Status.FAILED_WITH_TERMINAL_ERROR); taskResult.setOutputData(null); diff --git a/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java b/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java index 1fb9ca6b49..096d901c91 100644 --- a/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java +++ b/client/src/main/java/com/netflix/conductor/client/http/WorkflowClient.java @@ -157,10 +157,13 @@ public String startWorkflow(String name, Integer version, String correlationId, } /** - * Starts a workflow + * Starts a workflow. + * If the size of the workflow input payload is bigger than {@link ConductorClientConfiguration#getWorkflowInputPayloadThresholdKB()}, + * it is uploaded to {@link ExternalPayloadStorage}, if enabled, else the workflow is rejected. * * @param startWorkflowRequest the {@link StartWorkflowRequest} object to start the workflow * @return the id of the workflow instance that can be used for tracking + * @throws ConductorClientException if {@link ExternalPayloadStorage} is disabled or if the payload size is greater than {@link ConductorClientConfiguration#getWorkflowInputMaxPayloadThresholdKB()} */ public String startWorkflow(StartWorkflowRequest startWorkflowRequest) { Preconditions.checkNotNull(startWorkflowRequest, "StartWorkflowRequest cannot be null"); @@ -174,14 +177,15 @@ public String startWorkflow(StartWorkflowRequest startWorkflowRequest) { long workflowInputSize = workflowInputBytes.length; WorkflowTaskMetrics.recordWorkflowInputPayloadSize(startWorkflowRequest.getName(), version, workflowInputSize); if (workflowInputSize > conductorClientConfiguration.getWorkflowInputPayloadThresholdKB() * 1024) { - if (conductorClientConfiguration.isExternalPayloadStorageEnabled()) { + if (!conductorClientConfiguration.isExternalPayloadStorageEnabled() || + (workflowInputSize > conductorClientConfiguration.getWorkflowInputMaxPayloadThresholdKB() * 1024)) { + String errorMsg = String.format("Input payload larger than the allowed threshold of: %d KB", conductorClientConfiguration.getWorkflowInputPayloadThresholdKB()); + throw new ConductorClientException(errorMsg); + } else { WorkflowTaskMetrics.incrementExternalPayloadUsedCount(startWorkflowRequest.getName(), ExternalPayloadStorage.Operation.WRITE.name(), ExternalPayloadStorage.PayloadType.WORKFLOW_INPUT.name()); String externalStoragePath = uploadToExternalPayloadStorage(ExternalPayloadStorage.PayloadType.WORKFLOW_INPUT, workflowInputBytes, workflowInputSize); startWorkflowRequest.setExternalInputPayloadStoragePath(externalStoragePath); startWorkflowRequest.setInput(null); - } else { - String errorMsg = String.format("Input payload larger than the allowed threshold of: %d KB", conductorClientConfiguration.getWorkflowInputPayloadThresholdKB()); - throw new ConductorClientException(errorMsg); } } } catch (IOException e) { diff --git a/common/src/main/java/com/netflix/conductor/common/run/Workflow.java b/common/src/main/java/com/netflix/conductor/common/run/Workflow.java index c65214eb35..e70624ccc3 100644 --- a/common/src/main/java/com/netflix/conductor/common/run/Workflow.java +++ b/common/src/main/java/com/netflix/conductor/common/run/Workflow.java @@ -1,4 +1,4 @@ -/** +/* * Copyright 2016 Netflix, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); @@ -26,7 +26,6 @@ import java.util.Set; import java.util.stream.Collectors; - public class Workflow extends Auditable{ public enum WorkflowStatus { @@ -402,4 +401,4 @@ public Workflow copy() { public String toString() { return workflowType + "." + version + "/" + workflowId + "." + status; } -} \ No newline at end of file +} diff --git a/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java b/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java index 5ccc8c0860..b6bfbc9a61 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/DeciderService.java @@ -206,7 +206,6 @@ private DeciderOutcome decide(final WorkflowDef workflowDef, final Workflow work .collect(Collectors.toList()), workflow.getWorkflowId()); outcome.tasksToBeScheduled.addAll(unScheduledTasks); } - updateOutput(workflowDef, workflow, null); if (outcome.tasksToBeScheduled.isEmpty() && checkForWorkflowCompletion(workflowDef, workflow)) { logger.debug("Marking workflow as complete. workflow=" + workflow.getWorkflowId() + ", tasks=" + workflow.getTasks()); outcome.isComplete = true; @@ -256,25 +255,28 @@ private List startWorkflow(Workflow workflow, WorkflowDef workflowDef) thr return Collections.singletonList(rerunFromTask); } - void updateOutput(final WorkflowDef def, final Workflow workflow, @Nullable Task task) { + /** + * Updates the workflow output. + * + * @param workflow the workflow instance + * @param task if not null, the output of this task will be copied to workflow output if no output parameters are specified in the workflow defintion + * if null, the output of the last task in the workflow will be copied to workflow output of no output parameters are specified in the workflow definition + */ + void updateWorkflowOutput(final Workflow workflow, @Nullable Task task) { List allTasks = workflow.getTasks(); if (allTasks.isEmpty()) { return; } + Task last = Optional.ofNullable(task).orElse(allTasks.get(allTasks.size() - 1)); + WorkflowDef def = metadataDAO.get(workflow.getWorkflowType(), workflow.getVersion()); Map output; - Task last; - if (task != null) { - last = task; - } else { - last = allTasks.get(allTasks.size() - 1); - } - if (!def.getOutputParameters().isEmpty()) { Workflow workflowInstance = populateWorkflowAndTaskData(workflow); output = parametersUtils.getTaskInput(def.getOutputParameters(), workflowInstance, null, null); } else if (StringUtils.isNotBlank(last.getExternalOutputPayloadStoragePath())) { output = externalPayloadStorageUtils.downloadPayload(last.getExternalOutputPayloadStoragePath()); + Monitors.recordExternalPayloadStorageUsage(last.getTaskDefName(), ExternalPayloadStorage.Operation.READ.toString(), ExternalPayloadStorage.PayloadType.TASK_OUTPUT.toString()); } else { output = last.getOutputData(); } @@ -389,6 +391,13 @@ Task retry(TaskDef taskDefinition, WorkflowTask workflowTask, Task task, Workflo return rescheduled; } + /** + * Populates the workflow input data and the tasks input/output data if stored in external payload storage. + * This method creates a deep copy of the workflow instance where the payloads will be stored after downloading from external payload storage. + * + * @param workflow the workflow for which the data needs to be populated + * @return a copy of the workflow with the payload data populated + */ @VisibleForTesting Workflow populateWorkflowAndTaskData(Workflow workflow) { Workflow workflowInstance = workflow.copy(); @@ -396,7 +405,9 @@ Workflow populateWorkflowAndTaskData(Workflow workflow) { if (StringUtils.isNotBlank(workflow.getExternalInputPayloadStoragePath())) { // download the workflow input from external storage here and plug it into the workflow Map workflowInputParams = externalPayloadStorageUtils.downloadPayload(workflow.getExternalInputPayloadStoragePath()); + Monitors.recordExternalPayloadStorageUsage(workflow.getWorkflowType(), ExternalPayloadStorage.Operation.READ.toString(), ExternalPayloadStorage.PayloadType.WORKFLOW_INPUT.toString()); workflowInstance.setInput(workflowInputParams); + workflowInstance.setExternalInputPayloadStoragePath(null); } workflowInstance.getTasks().stream() @@ -404,10 +415,12 @@ Workflow populateWorkflowAndTaskData(Workflow workflow) { .forEach(task -> { if (StringUtils.isNotBlank(task.getExternalOutputPayloadStoragePath())) { task.setOutputData(externalPayloadStorageUtils.downloadPayload(task.getExternalOutputPayloadStoragePath())); + Monitors.recordExternalPayloadStorageUsage(task.getTaskDefName(), ExternalPayloadStorage.Operation.READ.toString(), ExternalPayloadStorage.PayloadType.TASK_OUTPUT.toString()); task.setExternalOutputPayloadStoragePath(null); } if (StringUtils.isNotBlank(task.getExternalInputPayloadStoragePath())) { task.setInputData(externalPayloadStorageUtils.downloadPayload(task.getExternalInputPayloadStoragePath())); + Monitors.recordExternalPayloadStorageUsage(task.getTaskDefName(), ExternalPayloadStorage.Operation.READ.toString(), ExternalPayloadStorage.PayloadType.TASK_INPUT.toString()); task.setExternalInputPayloadStoragePath(null); } }); @@ -556,10 +569,6 @@ private boolean isTaskSkipped(WorkflowTask taskToSchedule, Workflow workflow) { } } - private void populateTaskData(Workflow workflow) { - - } - static class DeciderOutcome { List tasksToBeScheduled = new LinkedList<>(); diff --git a/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java b/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java index 49d602892e..03de08e996 100644 --- a/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java +++ b/core/src/main/java/com/netflix/conductor/core/execution/WorkflowExecutor.java @@ -362,6 +362,7 @@ void completeWorkflow(Workflow wf) { throw new ApplicationException(CONFLICT, msg); } + deciderService.updateWorkflowOutput(wf, null); workflow.setStatus(WorkflowStatus.COMPLETED); workflow.setOutput(wf.getOutput()); workflow.setExternalOutputPayloadStoragePath(wf.getExternalOutputPayloadStoragePath()); @@ -402,6 +403,8 @@ public void terminateWorkflow(Workflow workflow, String reason, String failureWo workflow.setStatus(WorkflowStatus.TERMINATED); } + deciderService.updateWorkflowOutput(workflow, null); + String workflowId = workflow.getWorkflowId(); workflow.setReasonForIncompletion(reason); executionDAO.updateWorkflow(workflow); @@ -522,8 +525,8 @@ public void updateTask(TaskResult taskResult) { taskByRefName.setReasonForIncompletion(task.getReasonForIncompletion()); taskByRefName.setWorkerId(task.getWorkerId()); taskByRefName.setCallbackAfterSeconds(task.getCallbackAfterSeconds()); - WorkflowDef workflowDef = metadataDAO.get(workflowInstance.getWorkflowType(), workflowInstance.getVersion()); - deciderService.updateOutput(workflowDef, workflowInstance, task); + //WorkflowDef workflowDef = metadataDAO.get(workflowInstance.getWorkflowType(), workflowInstance.getVersion()); + deciderService.updateWorkflowOutput(workflowInstance, task); } executionDAO.updateWorkflow(workflowInstance); logger.debug("Task: {} has a {} status and the Workflow has been updated with failed task reference", task, task.getStatus()); diff --git a/core/src/main/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtils.java b/core/src/main/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtils.java index 1dc87fc90f..09f2ac3f98 100644 --- a/core/src/main/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtils.java +++ b/core/src/main/java/com/netflix/conductor/core/utils/ExternalPayloadStorageUtils.java @@ -26,6 +26,7 @@ import com.netflix.conductor.core.config.Configuration; import com.netflix.conductor.core.execution.ApplicationException; import com.netflix.conductor.core.execution.TerminateWorkflowException; +import com.netflix.conductor.metrics.Monitors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -115,34 +116,37 @@ public void verifyAndUpload(T entity, PayloadType payloadType) { try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream()) { objectMapper.writeValue(byteArrayOutputStream, payload); byte[] payloadBytes = byteArrayOutputStream.toByteArray(); - long payloadSize = payloadBytes.length / 1024; - if (payloadSize > threshold) { - if (payloadSize > maxThreshold) { - if (entity instanceof Task) { - String errorMsg = String.format("The payload size: %dKB of task: %s in workflow: %s is greater than the permissible limit: %dKB", payloadSize, ((Task) entity).getTaskId(), ((Task) entity).getWorkflowInstanceId(), maxThreshold); - failTask(((Task) entity), payloadType, errorMsg); - } else { - String errorMsg = String.format("The output payload size: %dKB of workflow: %s is greater than the permissible limit: %dKB", payloadSize, ((Workflow) entity).getWorkflowId(), maxThreshold); - failWorkflow(errorMsg); - } - } + long payloadSize = payloadBytes.length; + if (payloadSize > maxThreshold * 1024) { + if (entity instanceof Task) { + String errorMsg = String.format("The payload size: %dKB of task: %s in workflow: %s is greater than the permissible limit: %dKB", payloadSize, ((Task) entity).getTaskId(), ((Task) entity).getWorkflowInstanceId(), maxThreshold); + failTask(((Task) entity), payloadType, errorMsg); + } else { + String errorMsg = String.format("The output payload size: %dKB of workflow: %s is greater than the permissible limit: %dKB", payloadSize, ((Workflow) entity).getWorkflowId(), maxThreshold); + failWorkflow(errorMsg); + } + } else if (payloadSize > threshold * 1024) { switch (payloadType) { case TASK_INPUT: ((Task) entity).setInputData(null); ((Task) entity).setExternalInputPayloadStoragePath(uploadHelper(payloadBytes, payloadSize, PayloadType.TASK_INPUT)); + Monitors.recordExternalPayloadStorageUsage(((Task) entity).getTaskDefName(), ExternalPayloadStorage.Operation.WRITE.toString(), PayloadType.TASK_INPUT.toString()); break; case TASK_OUTPUT: ((Task) entity).setOutputData(null); ((Task) entity).setExternalOutputPayloadStoragePath(uploadHelper(payloadBytes, payloadSize, PayloadType.TASK_OUTPUT)); + Monitors.recordExternalPayloadStorageUsage(((Task) entity).getTaskDefName(), ExternalPayloadStorage.Operation.WRITE.toString(), PayloadType.TASK_OUTPUT.toString()); break; case WORKFLOW_INPUT: ((Workflow) entity).setInput(null); ((Workflow) entity).setExternalInputPayloadStoragePath(uploadHelper(payloadBytes, payloadSize, PayloadType.WORKFLOW_INPUT)); + Monitors.recordExternalPayloadStorageUsage(((Workflow) entity).getWorkflowType(), ExternalPayloadStorage.Operation.WRITE.toString(), PayloadType.WORKFLOW_INPUT.toString()); break; case WORKFLOW_OUTPUT: ((Workflow) entity).setOutput(null); ((Workflow) entity).setExternalOutputPayloadStoragePath(uploadHelper(payloadBytes, payloadSize, PayloadType.WORKFLOW_OUTPUT)); + Monitors.recordExternalPayloadStorageUsage(((Workflow) entity).getWorkflowType(), ExternalPayloadStorage.Operation.WRITE.toString(), PayloadType.WORKFLOW_OUTPUT.toString()); break; } } diff --git a/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java b/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java index 4c8eb3e297..eb9322d3e9 100644 --- a/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java +++ b/core/src/main/java/com/netflix/conductor/core/utils/S3PayloadStorage.java @@ -56,8 +56,8 @@ public class S3PayloadStorage implements ExternalPayloadStorage { @Inject public S3PayloadStorage(Configuration config) { s3Client = AmazonS3ClientBuilder.standard().withRegion("us-east-1").build(); - bucketName = config.getProperty("s3bucket", ""); - expirationSec = config.getIntProperty("s3signedurlexpirationseconds", 5); + bucketName = config.getProperty("workflow.external.payload.storage.s3.bucket", ""); + expirationSec = config.getIntProperty("workflow.external.payload.storage.s3.signedurlexpirationseconds", 5); } /** @@ -104,6 +104,14 @@ public ExternalStorageLocation getLocation(Operation operation, PayloadType payl } } + /** + * Uploads the payload to the given s3 object key. + * It is expected that the caller retrieves the object key using {@link #getLocation(Operation, PayloadType, String)} before making this call. + * + * @param path the s3 key of the object to be uploaded + * @param payload an {@link InputStream} containing the json payload which is to be uploaded + * @param payloadSize the size of the json payload in bytes + */ @Override public void upload(String path, InputStream payload, long payloadSize) { try { @@ -120,6 +128,8 @@ public void upload(String path, InputStream payload, long payloadSize) { } /** + * Downloads the payload stored in the s3 object. + * * @param path the S3 key of the object * @return an input stream containing the contents of the object * Caller is expected to close the input stream. diff --git a/core/src/main/java/com/netflix/conductor/metrics/Monitors.java b/core/src/main/java/com/netflix/conductor/metrics/Monitors.java index 34f8fc6327..71aec0f3cf 100644 --- a/core/src/main/java/com/netflix/conductor/metrics/Monitors.java +++ b/core/src/main/java/com/netflix/conductor/metrics/Monitors.java @@ -247,4 +247,8 @@ public static void recordDaoPayloadSize(String dao, String action, int size) { public static void recordDaoPayloadSize(String dao, String action, String taskType, String workflowType, int size) { gauge(classQualifier, "dao_payload_size", size, "dao", dao, "action", action, "taskType", taskType, "workflowType", workflowType); } + + public static void recordExternalPayloadStorageUsage(String name, String operation, String payloadType) { + counter(classQualifier, "external_payload_storage_usage", "name", name, "operation", operation, "payloadType", payloadType); + } } \ No newline at end of file diff --git a/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java b/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java index d5929a613d..597ee0f779 100644 --- a/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java +++ b/core/src/test/java/com/netflix/conductor/core/execution/TestDeciderService.java @@ -75,6 +75,8 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyMap; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -94,6 +96,8 @@ public class TestDeciderService { private static Registry registry; + private MetadataDAO metadataDAO; + private ExternalPayloadStorageUtils externalPayloadStorageUtils; private static ObjectMapper objectMapper = new ObjectMapper(); @@ -114,7 +118,7 @@ public static void init() { @Before public void setup() { - MetadataDAO metadataDAO = mock(MetadataDAO.class); + metadataDAO = mock(MetadataDAO.class); QueueDAO queueDAO = mock(QueueDAO.class); externalPayloadStorageUtils = mock(ExternalPayloadStorageUtils.class); @@ -805,6 +809,30 @@ public void testPopulateWorkflowAndTaskData() { assertTrue(workflow.getTasks().get(0).getOutputData().isEmpty()); assertNotNull(workflowInstance.getTasks().get(0).getOutputData()); assertEquals(taskOutputParams, workflowInstance.getTasks().get(0).getOutputData()); + + assertNull(workflowInstance.getExternalInputPayloadStoragePath()); + assertNull(workflowInstance.getTasks().get(0).getExternalInputPayloadStoragePath()); + assertNull(workflowInstance.getTasks().get(0).getExternalOutputPayloadStoragePath()); + } + + @SuppressWarnings("unchecked") + @Test + public void testUpdateWorkflowOutput() { + Workflow workflow = new Workflow(); + deciderService.updateWorkflowOutput(workflow, null); + assertNotNull(workflow.getOutput()); + assertTrue(workflow.getOutput().isEmpty()); + + Task task = new Task(); + Map taskOutput = new HashMap<>(); + taskOutput.put("taskKey", "taskValue"); + task.setOutputData(taskOutput); + workflow.getTasks().add(task); + WorkflowDef workflowDef = new WorkflowDef(); + when(metadataDAO.get(anyString(), anyInt())).thenReturn(workflowDef); + deciderService.updateWorkflowOutput(workflow, null); + assertNotNull(workflow.getOutput()); + assertEquals("taskValue", workflow.getOutput().get("taskKey")); } private WorkflowDef createConditionalWF() { diff --git a/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisExecutionDAO.java b/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisExecutionDAO.java index 20f3ca049f..dab409d7d8 100644 --- a/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisExecutionDAO.java +++ b/redis-persistence/src/main/java/com/netflix/conductor/dao/dynomite/RedisExecutionDAO.java @@ -1,4 +1,4 @@ -/** +/* * Copyright 2016 Netflix, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); @@ -82,25 +82,16 @@ public class RedisExecutionDAO extends BaseDynoDAO implements ExecutionDAO { private final static String EVENT_EXECUTION = "EVENT_EXECUTION"; - private static final String WORKFLOW_DYNOMITE_TASK_PAYLOAD_THRESHOLD = "workflow.dynomite.task.payload.threshold"; - private static final String WORKFLOW_DYNOMITE_WORKFLOW_INPUT_THRESHOLD = "workflow.dynomite.workflow.input.threshold"; - private IndexDAO indexDAO; private MetadataDAO metadataDA0; - private long taskPayloadThreshold; - - private long workflowInputPayloadThreshold; - @Inject public RedisExecutionDAO(DynoProxy dynoClient, ObjectMapper objectMapper, IndexDAO indexDAO, MetadataDAO metadataDA0, Configuration config) { super(dynoClient, objectMapper, config); this.indexDAO = indexDAO; this.metadataDA0 = metadataDA0; - this.taskPayloadThreshold = config.getLongProperty(WORKFLOW_DYNOMITE_TASK_PAYLOAD_THRESHOLD,5 * FileUtils.ONE_MB); - this.workflowInputPayloadThreshold = config.getLongProperty(WORKFLOW_DYNOMITE_WORKFLOW_INPUT_THRESHOLD,5 * FileUtils.ONE_MB); } @Override @@ -144,7 +135,7 @@ public List getTasks(String taskDefName, String startKey, int count) { @Override public List createTasks(List tasks) { - List created = new LinkedList(); + List created = new LinkedList<>(); for (Task task : tasks) { @@ -219,14 +210,6 @@ public void updateTask(Task task) { recordRedisDaoPayloadSize("updateTask", payload.length(), Optional.ofNullable(taskDef) .map(TaskDef::getName) .orElse("n/a"), task.getWorkflowType()); - //The payload is verified and - if(payload.length() > taskPayloadThreshold) { - task.setReasonForIncompletion(String.format("Payload of the task: %s larger than the permissible %s bytes", - FileUtils.byteCountToDisplaySize(payload.length()), FileUtils.byteCountToDisplaySize(taskPayloadThreshold))); - task.setOutputData(null); - task.setStatus(Status.FAILED_WITH_TERMINAL_ERROR); - payload = toJson(task); - } recordRedisDaoRequests("updateTask", task.getTaskType(), task.getWorkflowType()); dynoClient.set(nsKey(TASK, task.getTaskId()), payload); logger.debug("Workflow task payload saved to TASK with taskKey: {}, workflowId: {}, taskId: {}, taskType: {} during updateTask", @@ -577,9 +560,6 @@ private String insertOrUpdateWorkflow(Workflow workflow, boolean update) { workflow.setTasks(new LinkedList<>()); String payload = toJson(workflow); - if(payload.length() > workflowInputPayloadThreshold) { - throw new ApplicationException(Code.INVALID_INPUT, String.format("Input payload larger than the allowed threshold of: %d", workflowInputPayloadThreshold)); - } // Store the workflow object dynoClient.set(nsKey(WORKFLOW, workflow.getWorkflowId()), payload); recordRedisDaoRequests("storeWorkflow", "n/a", workflow.getWorkflowType()); diff --git a/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java b/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java index dbfe360273..35ddd5e5ca 100644 --- a/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java +++ b/test-harness/src/test/java/com/netflix/conductor/tests/integration/WorkflowServiceTest.java @@ -1272,8 +1272,6 @@ public void testSimpleWorkflow() { workflow = workflowExecutionService.getExecutionStatus(workflowInstanceId, false); assertNotNull(workflow); assertNotNull(workflow.getOutput()); - assertTrue("Found " + workflow.getOutput().toString(), workflow.getOutput().containsKey("o3")); - assertEquals("task1.Done", workflow.getOutput().get("o3")); task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker"); assertNotNull(task); @@ -1501,11 +1499,11 @@ public void testSimpleWorkflowWithTaskSpecificDomain() { assertNotNull(found); String correlationId = "unit_test_sw"; - Map input = new HashMap(); + Map input = new HashMap<>(); String inputParam1 = "p1 value"; input.put("param1", inputParam1); input.put("param2", "p2 value"); - Map taskToDomain = new HashMap(); + Map taskToDomain = new HashMap<>(); taskToDomain.put("junit_task_3", "domain1"); taskToDomain.put("junit_task_2", "domain1"); @@ -1515,22 +1513,15 @@ public void testSimpleWorkflowWithTaskSpecificDomain() { task = workflowExecutionService.poll("junit_task_2", "task1.junit.worker", "domain1"); assertNull(task); - String wfid = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2_SW, 1, correlationId, input, null, null, taskToDomain); - System.out.println("testSimpleWorkflow.wfid=" + wfid); - assertNotNull(wfid); - Workflow wf = workflowExecutor.getWorkflow(wfid, false); - assertNotNull(wf); - - Workflow es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(es.getReasonForIncompletion(), WorkflowStatus.RUNNING, es.getStatus()); - - - es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(WorkflowStatus.RUNNING, es.getStatus()); - assertEquals(1, es.getTasks().size()); //The very first task is the one that should be scheduled. + String workflowId = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2_SW, 1, correlationId, input, null, null, taskToDomain); + //System.out.println("testSimpleWorkflow.wfid=" + workflowId); + assertNotNull(workflowId); + Workflow workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertEquals(workflow.getReasonForIncompletion(), WorkflowStatus.RUNNING, workflow.getStatus()); + assertEquals(WorkflowStatus.RUNNING, workflow.getStatus()); + assertEquals(1, workflow.getTasks().size()); //The very first task is the one that should be scheduled. // Check Size Map sizes = workflowExecutionService.getTaskQueueSizes(Arrays.asList("domain1:junit_task_3", "junit_task_3")); @@ -1544,40 +1535,37 @@ public void testSimpleWorkflowWithTaskSpecificDomain() { assertNotNull(task); assertEquals("junit_task_3", task.getTaskType()); assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); - assertEquals(wfid, task.getWorkflowInstanceId()); + assertEquals(workflowId, task.getWorkflowInstanceId()); - String task1Op = "task1.Done"; List tasks = workflowExecutionService.getTasks(task.getTaskType(), null, 1); assertNotNull(tasks); assertEquals(1, tasks.size()); task = tasks.get(0); + assertEquals(workflowId, task.getWorkflowInstanceId()); - Workflow workflow = workflowExecutionService.getExecutionStatus(task.getWorkflowInstanceId(), false); - System.out.println("task workflow = " + workflow.getWorkflowType() + "," + workflow.getInput()); - assertEquals(wfid, task.getWorkflowInstanceId()); + String task1Op = "task1.Done"; task.getOutputData().put("op", task1Op); task.setStatus(COMPLETED); workflowExecutionService.updateTask(task); - es = workflowExecutionService.getExecutionStatus(wfid, false); - assertNotNull(es); - assertNotNull(es.getOutput()); - assertTrue("Found " + es.getOutput().toString(), es.getOutput().containsKey("o3")); - assertEquals("task1.Done", es.getOutput().get("o3")); + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertEquals(WorkflowStatus.RUNNING, workflow.getStatus()); + assertEquals(2, workflow.getTasks().size()); task = workflowExecutionService.poll("junit_task_1", "task1.junit.worker"); assertNotNull(task); assertEquals("junit_task_1", task.getTaskType()); - Workflow essw = workflowExecutionService.getExecutionStatus(task.getWorkflowInstanceId(), false); + + workflow = workflowExecutionService.getExecutionStatus(task.getWorkflowInstanceId(), false); assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); - assertNotNull(essw.getTaskToDomain()); - assertEquals(essw.getTaskToDomain().size(), 2); + assertNotNull(workflow.getTaskToDomain()); + assertEquals(workflow.getTaskToDomain().size(), 2); task.setStatus(COMPLETED); task.setReasonForIncompletion("unit test failure"); workflowExecutionService.updateTask(task); - task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker", "domain1"); assertNotNull(task); assertEquals("junit_task_2", task.getTaskType()); @@ -1587,20 +1575,18 @@ public void testSimpleWorkflowWithTaskSpecificDomain() { task.setReasonForIncompletion("unit test failure"); workflowExecutionService.updateTask(task); - - es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(WorkflowStatus.COMPLETED, es.getStatus()); - tasks = es.getTasks(); + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertEquals(WorkflowStatus.COMPLETED, workflow.getStatus()); + tasks = workflow.getTasks(); assertNotNull(tasks); assertEquals(2, tasks.size()); + assertTrue("Found " + workflow.getOutput().toString(), workflow.getOutput().containsKey("o3")); + assertEquals("task1.Done", workflow.getOutput().get("o3")); - assertTrue("Found " + es.getOutput().toString(), es.getOutput().containsKey("o3")); - assertEquals("task1.Done", es.getOutput().get("o3")); - - List pddata = workflowExecutionService.getPollData("junit_task_3"); - assertTrue(pddata.size() == 2); - for (PollData pd : pddata) { + List pollData = workflowExecutionService.getPollData("junit_task_3"); + assertEquals(2, pollData.size()); + for (PollData pd : pollData) { assertEquals(pd.getQueueName(), "junit_task_3"); assertEquals(pd.getWorkerId(), "task1.junit.worker"); assertTrue(pd.getLastPollTime() != 0); @@ -1609,7 +1595,6 @@ public void testSimpleWorkflowWithTaskSpecificDomain() { } } - List pdList = workflowExecutionService.getAllPollData(); int count = 0; for (PollData pd : pdList) { @@ -1617,8 +1602,7 @@ public void testSimpleWorkflowWithTaskSpecificDomain() { count++; } } - assertTrue(count == 2); - + assertEquals(2, count); } @Test @@ -1644,22 +1628,14 @@ public void testSimpleWorkflowWithAllTaskInOneDomain() { task = workflowExecutionService.poll("junit_task_2", "task1.junit.worker", "domain12"); assertNull(task); - String wfid = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2_SW, 1, correlationId, input, null, null, taskToDomain); - System.out.println("testSimpleWorkflow.wfid=" + wfid); - assertNotNull(wfid); - Workflow wf = workflowExecutor.getWorkflow(wfid, false); - assertNotNull(wf); - - Workflow es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(es.getReasonForIncompletion(), WorkflowStatus.RUNNING, es.getStatus()); - - - es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(WorkflowStatus.RUNNING, es.getStatus()); - assertEquals(1, es.getTasks().size()); //The very first task is the one that should be scheduled. + String workflowId = workflowExecutor.startWorkflow(LINEAR_WORKFLOW_T1_T2_SW, 1, correlationId, input, null, null, taskToDomain); + //System.out.println("testSimpleWorkflow.wfid=" + workflowId); + assertNotNull(workflowId); + Workflow workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertEquals(workflow.getReasonForIncompletion(), WorkflowStatus.RUNNING, workflow.getStatus()); + assertEquals(1, workflow.getTasks().size()); //The very first task is the one that should be scheduled. // Check Size Map sizes = workflowExecutionService.getTaskQueueSizes(Arrays.asList("domain11:junit_task_3", "junit_task_3")); @@ -1674,40 +1650,36 @@ public void testSimpleWorkflowWithAllTaskInOneDomain() { assertEquals("junit_task_3", task.getTaskType()); assertEquals("domain11", task.getDomain()); assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); - assertEquals(wfid, task.getWorkflowInstanceId()); + assertEquals(workflowId, task.getWorkflowInstanceId()); - String task1Op = "task1.Done"; List tasks = workflowExecutionService.getTasks(task.getTaskType(), null, 1); assertNotNull(tasks); assertEquals(1, tasks.size()); task = tasks.get(0); - Workflow workflow = workflowExecutionService.getExecutionStatus(task.getWorkflowInstanceId(), false); - System.out.println("task workflow = " + workflow.getWorkflowType() + "," + workflow.getInput()); - assertEquals(wfid, task.getWorkflowInstanceId()); + String task1Op = "task1.Done"; + assertEquals(workflowId, task.getWorkflowInstanceId()); task.getOutputData().put("op", task1Op); task.setStatus(COMPLETED); workflowExecutionService.updateTask(task); - es = workflowExecutionService.getExecutionStatus(wfid, false); - assertNotNull(es); - assertNotNull(es.getOutput()); - assertTrue("Found " + es.getOutput().toString(), es.getOutput().containsKey("o3")); - assertEquals("task1.Done", es.getOutput().get("o3")); + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertEquals(2, workflow.getTasks().size()); task = workflowExecutionService.poll("junit_task_1", "task1.junit.worker"); assertNotNull(task); assertEquals("junit_task_1", task.getTaskType()); - Workflow essw = workflowExecutionService.getExecutionStatus(task.getWorkflowInstanceId(), false); + + workflow = workflowExecutionService.getExecutionStatus(task.getWorkflowInstanceId(), false); assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); - assertNotNull(essw.getTaskToDomain()); - assertEquals(essw.getTaskToDomain().size(), 1); + assertNotNull(workflow.getTaskToDomain()); + assertEquals(workflow.getTaskToDomain().size(), 1); task.setStatus(COMPLETED); task.setReasonForIncompletion("unit test failure"); workflowExecutionService.updateTask(task); - task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker", "domain11"); assertNull(task); task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker", "domain12"); @@ -1720,17 +1692,15 @@ public void testSimpleWorkflowWithAllTaskInOneDomain() { task.setReasonForIncompletion("unit test failure"); workflowExecutionService.updateTask(task); + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertEquals(WorkflowStatus.COMPLETED, workflow.getStatus()); - es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(WorkflowStatus.COMPLETED, es.getStatus()); - tasks = es.getTasks(); + tasks = workflow.getTasks(); assertNotNull(tasks); assertEquals(2, tasks.size()); - - assertTrue("Found " + es.getOutput().toString(), es.getOutput().containsKey("o3")); - assertEquals("task1.Done", es.getOutput().get("o3")); - + assertTrue("Found " + workflow.getOutput().toString(), workflow.getOutput().containsKey("o3")); + assertEquals("task1.Done", workflow.getOutput().get("o3")); } @After @@ -2785,63 +2755,57 @@ public void testNonRestartartableWorkflows() { assertFalse(StringUtils.isBlank(found.getFailureWorkflow())); String correlationId = "unit_test_1" + UUID.randomUUID().toString(); - Map input = new HashMap(); + Map input = new HashMap<>(); String inputParam1 = "p1 value"; input.put("param1", inputParam1); input.put("param2", "p2 value"); - String wfid = workflowExecutor.startWorkflow(JUNIT_TEST_WF_NON_RESTARTABLE, 1, correlationId, input); - assertNotNull(wfid); + String workflowId = workflowExecutor.startWorkflow(JUNIT_TEST_WF_NON_RESTARTABLE, 1, correlationId, input); + assertNotNull(workflowId); Task task = getTask("junit_task_1"); task.setStatus(FAILED); workflowExecutionService.updateTask(task); // If we get the full workflow here then, last task should be completed and the next task should be scheduled - Workflow es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(WorkflowStatus.FAILED, es.getStatus()); - - workflowExecutor.rewind(es.getWorkflowId()); + Workflow workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertEquals(WorkflowStatus.FAILED, workflow.getStatus()); - workflowExecutor.decide(wfid); + workflowExecutor.rewind(workflow.getWorkflowId()); // Polling for the first task should return the same task as before task = workflowExecutionService.poll("junit_task_1", "task1.junit.worker"); assertNotNull(task); assertEquals("junit_task_1", task.getTaskType()); assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); - assertEquals(wfid, task.getWorkflowInstanceId()); - - workflowExecutor.decide(wfid); + assertEquals(workflowId, task.getWorkflowInstanceId()); - String task1Op = "task1.Done"; List tasks = workflowExecutionService.getTasks(task.getTaskType(), null, 1); assertNotNull(tasks); assertEquals(1, tasks.size()); + task = tasks.get(0); + assertEquals(workflowId, task.getWorkflowInstanceId()); - Workflow workflow = workflowExecutionService.getExecutionStatus(task.getWorkflowInstanceId(), false); - System.out.println("task workflow = " + workflow.getWorkflowType() + "," + workflow.getInput()); - assertEquals(wfid, task.getWorkflowInstanceId()); + String task1Op = "task1.Done"; task.getOutputData().put("op", task1Op); task.setStatus(COMPLETED); workflowExecutionService.updateTask(task); - es = workflowExecutionService.getExecutionStatus(wfid, false); - assertNotNull(es); - assertNotNull(es.getOutput()); - assertTrue("Found " + es.getOutput().toString(), es.getOutput().containsKey("o3")); - assertEquals("task1.Done", es.getOutput().get("o3")); + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertNotNull(workflow.getOutput()); + assertEquals(2, workflow.getTasks().size()); task = workflowExecutionService.poll("junit_task_2", "task2.junit.worker"); assertNotNull(task); assertEquals("junit_task_2", task.getTaskType()); assertTrue(workflowExecutionService.ackTaskReceived(task.getTaskId())); + String task2Input = (String) task.getInputData().get("tp2"); assertNotNull("Found=" + task.getInputData(), task2Input); assertEquals(task1Op, task2Input); - task2Input = (String) task.getInputData().get("tp1"); assertNotNull(task2Input); assertEquals(inputParam1, task2Input); @@ -2850,21 +2814,19 @@ public void testNonRestartartableWorkflows() { task.setReasonForIncompletion("unit test failure"); workflowExecutionService.updateTask(task); + workflow = workflowExecutionService.getExecutionStatus(workflowId, true); + assertNotNull(workflow); + assertEquals(WorkflowStatus.COMPLETED, workflow.getStatus()); - es = workflowExecutionService.getExecutionStatus(wfid, true); - assertNotNull(es); - assertEquals(WorkflowStatus.COMPLETED, es.getStatus()); - tasks = es.getTasks(); + tasks = workflow.getTasks(); assertNotNull(tasks); assertEquals(2, tasks.size()); - - assertTrue("Found " + es.getOutput().toString(), es.getOutput().containsKey("o3")); - assertEquals("task1.Done", es.getOutput().get("o3")); - + assertTrue("Found " + workflow.getOutput().toString(), workflow.getOutput().containsKey("o3")); + assertEquals("task1.Done", workflow.getOutput().get("o3")); expectedException.expect(ApplicationException.class); expectedException.expectMessage(String.format("is an instance of WorkflowDef: %s and version: %d and is non restartable", JUNIT_TEST_WF_NON_RESTARTABLE, 1)); - workflowExecutor.rewind(es.getWorkflowId()); + workflowExecutor.rewind(workflow.getWorkflowId()); }