You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@brooklyn.apache.org by GitBox <gi...@apache.org> on 2022/09/07 13:42:51 UTC

[GitHub] [brooklyn-server] ahgittin opened a new pull request, #1358: Initial workflow implementation

ahgittin opened a new pull request, #1358:
URL: https://github.com/apache/brooklyn-server/pull/1358

   Essential structure and sashimi slice of new declarative YAML workflow, per discussion on ML and proposal https://docs.google.com/document/d/1u02Bi6sS8Fkf1s7UzRRMnvLhA477bqcyxGa0nJesqkI
   
   Includes the following:
   * no-op and sleep steps
   * workflow-effector initializer
   * creation and execution of tasks from step definitions
   * java and camp yaml tests
   
   many more step types and workflow functionality to be added, but this provides a base where the essential definitions work and subsequent tasks can be done in parallel


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brooklyn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [brooklyn-server] ahgittin commented on a diff in pull request #1358: Initial workflow implementation

Posted by GitBox <gi...@apache.org>.
ahgittin commented on code in PR #1358:
URL: https://github.com/apache/brooklyn-server/pull/1358#discussion_r974266182


##########
core/src/main/java/org/apache/brooklyn/core/workflow/steps/SetSensorWorkflowStep.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.brooklyn.core.workflow.steps;
+
+import com.google.common.reflect.TypeToken;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
+import org.apache.brooklyn.core.sensor.Sensors;
+import org.apache.brooklyn.core.typereg.RegisteredTypes;
+import org.apache.brooklyn.core.workflow.WorkflowExecutionContext;
+import org.apache.brooklyn.core.workflow.WorkflowStepDefinition;
+import org.apache.brooklyn.util.core.flags.BrooklynTypeNameResolution;
+import org.apache.brooklyn.util.core.task.Tasks;
+import org.apache.brooklyn.util.text.Strings;
+import org.apache.brooklyn.util.time.Duration;
+import org.apache.brooklyn.util.time.Time;
+
+public class SetSensorWorkflowStep extends WorkflowStepDefinition {
+
+    EntityValueToSet sensor;
+    Object value;
+
+    public EntityValueToSet getSensor() {
+        return sensor;
+    }
+
+    public Object getValue() {
+        return value;
+    }
+
+    public void setValue(Object value) {
+        this.value = value;
+    }
+
+    @Override
+    public void setShorthand(String expression) {
+        this.sensor = EntityValueToSet.parseFromShorthand(expression, this::setValue);
+    }
+
+    @Override
+    protected Task<?> newTask(String name, WorkflowExecutionContext workflowExecutionContext) {
+        return Tasks.create(getDefaultTaskName(workflowExecutionContext), () -> {

Review Comment:
   good spot, needs tidying.  the `name` of the current step is now available in the `workflowExecutionContext`, and included in some cases, so i think not needed on this api.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brooklyn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [brooklyn-server] ahgittin commented on a diff in pull request #1358: Initial workflow implementation

Posted by GitBox <gi...@apache.org>.
ahgittin commented on code in PR #1358:
URL: https://github.com/apache/brooklyn-server/pull/1358#discussion_r974269951


##########
core/src/main/java/org/apache/brooklyn/core/workflow/steps/NoOpWorkflowStep.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.brooklyn.core.workflow.steps;
+
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.core.workflow.WorkflowExecutionContext;
+import org.apache.brooklyn.core.workflow.WorkflowStepDefinition;
+import org.apache.brooklyn.util.core.task.Tasks;
+import org.apache.brooklyn.util.text.Strings;
+
+public class NoOpWorkflowStep extends WorkflowStepDefinition {
+
+    @Override
+    public void setShorthand(String value) {
+        if (Strings.isBlank(value)) return;
+        // no arguments siupported
+        throw new IllegalStateException("Value for shorthand syntax not supported for no-op");
+    }
+
+    @Override
+    protected Task<?> newTask(String name, WorkflowExecutionContext workflowExecutionContext) {
+        return Tasks.create(getDefaultTaskName(workflowExecutionContext), () -> {});

Review Comment:
   `WorkflowExecutionContext` already does logging for each step, so specific steps only need to log specific behaviour; i think that is enough



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brooklyn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [brooklyn-server] algairim merged pull request #1358: Initial workflow implementation

Posted by GitBox <gi...@apache.org>.
algairim merged PR #1358:
URL: https://github.com/apache/brooklyn-server/pull/1358


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brooklyn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [brooklyn-server] ahgittin commented on a diff in pull request #1358: Initial workflow implementation

Posted by GitBox <gi...@apache.org>.
ahgittin commented on code in PR #1358:
URL: https://github.com/apache/brooklyn-server/pull/1358#discussion_r974268265


##########
core/src/main/java/org/apache/brooklyn/core/workflow/steps/SetSensorWorkflowStep.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.brooklyn.core.workflow.steps;
+
+import com.google.common.reflect.TypeToken;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
+import org.apache.brooklyn.core.sensor.Sensors;
+import org.apache.brooklyn.core.typereg.RegisteredTypes;
+import org.apache.brooklyn.core.workflow.WorkflowExecutionContext;
+import org.apache.brooklyn.core.workflow.WorkflowStepDefinition;
+import org.apache.brooklyn.util.core.flags.BrooklynTypeNameResolution;
+import org.apache.brooklyn.util.core.task.Tasks;
+import org.apache.brooklyn.util.text.Strings;
+import org.apache.brooklyn.util.time.Duration;
+import org.apache.brooklyn.util.time.Time;
+
+public class SetSensorWorkflowStep extends WorkflowStepDefinition {
+
+    EntityValueToSet sensor;
+    Object value;
+
+    public EntityValueToSet getSensor() {
+        return sensor;
+    }
+
+    public Object getValue() {
+        return value;
+    }
+
+    public void setValue(Object value) {
+        this.value = value;
+    }
+
+    @Override
+    public void setShorthand(String expression) {
+        this.sensor = EntityValueToSet.parseFromShorthand(expression, this::setValue);
+    }
+
+    @Override
+    protected Task<?> newTask(String name, WorkflowExecutionContext workflowExecutionContext) {
+        return Tasks.create(getDefaultTaskName(workflowExecutionContext), () -> {
+            if (sensor==null) throw new IllegalArgumentException("Sensor name is required");
+            String sensorName = workflowExecutionContext.resolve(sensor.name);
+            if (Strings.isBlank(sensorName)) throw new IllegalArgumentException("Sensor name is required");
+            TypeToken<?> type = workflowExecutionContext.lookupType(sensor.type, () -> TypeToken.of(Object.class));
+            Object resolvedValue = workflowExecutionContext.resolve(value, type);
+            workflowExecutionContext.getEntity().sensors().set(Sensors.newSensor(Object.class, sensorName), resolvedValue);

Review Comment:
   the value will be replaced.  the declared type is not changed if it was already known.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brooklyn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [brooklyn-server] geomacy commented on a diff in pull request #1358: Initial workflow implementation

Posted by GitBox <gi...@apache.org>.
geomacy commented on code in PR #1358:
URL: https://github.com/apache/brooklyn-server/pull/1358#discussion_r973596668


##########
core/src/main/java/org/apache/brooklyn/core/workflow/steps/SetSensorWorkflowStep.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.brooklyn.core.workflow.steps;
+
+import com.google.common.reflect.TypeToken;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
+import org.apache.brooklyn.core.sensor.Sensors;
+import org.apache.brooklyn.core.typereg.RegisteredTypes;
+import org.apache.brooklyn.core.workflow.WorkflowExecutionContext;
+import org.apache.brooklyn.core.workflow.WorkflowStepDefinition;
+import org.apache.brooklyn.util.core.flags.BrooklynTypeNameResolution;
+import org.apache.brooklyn.util.core.task.Tasks;
+import org.apache.brooklyn.util.text.Strings;
+import org.apache.brooklyn.util.time.Duration;
+import org.apache.brooklyn.util.time.Time;
+
+public class SetSensorWorkflowStep extends WorkflowStepDefinition {
+
+    EntityValueToSet sensor;
+    Object value;
+
+    public EntityValueToSet getSensor() {
+        return sensor;
+    }
+
+    public Object getValue() {
+        return value;
+    }
+
+    public void setValue(Object value) {
+        this.value = value;
+    }
+
+    @Override
+    public void setShorthand(String expression) {
+        this.sensor = EntityValueToSet.parseFromShorthand(expression, this::setValue);
+    }
+
+    @Override
+    protected Task<?> newTask(String name, WorkflowExecutionContext workflowExecutionContext) {
+        return Tasks.create(getDefaultTaskName(workflowExecutionContext), () -> {

Review Comment:
   Shouldn't the name of the task involve the `name` parameter passed in `newTask` signature?



##########
core/src/main/java/org/apache/brooklyn/core/workflow/steps/NoOpWorkflowStep.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.brooklyn.core.workflow.steps;
+
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.core.workflow.WorkflowExecutionContext;
+import org.apache.brooklyn.core.workflow.WorkflowStepDefinition;
+import org.apache.brooklyn.util.core.task.Tasks;
+import org.apache.brooklyn.util.text.Strings;
+
+public class NoOpWorkflowStep extends WorkflowStepDefinition {
+
+    @Override
+    public void setShorthand(String value) {
+        if (Strings.isBlank(value)) return;
+        // no arguments siupported
+        throw new IllegalStateException("Value for shorthand syntax not supported for no-op");
+    }
+
+    @Override
+    protected Task<?> newTask(String name, WorkflowExecutionContext workflowExecutionContext) {
+        return Tasks.create(getDefaultTaskName(workflowExecutionContext), () -> {});

Review Comment:
   for debug purposes it would be worth logging the task execution



##########
core/src/main/java/org/apache/brooklyn/core/workflow/steps/SetSensorWorkflowStep.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.brooklyn.core.workflow.steps;
+
+import com.google.common.reflect.TypeToken;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
+import org.apache.brooklyn.core.sensor.Sensors;
+import org.apache.brooklyn.core.typereg.RegisteredTypes;
+import org.apache.brooklyn.core.workflow.WorkflowExecutionContext;
+import org.apache.brooklyn.core.workflow.WorkflowStepDefinition;
+import org.apache.brooklyn.util.core.flags.BrooklynTypeNameResolution;
+import org.apache.brooklyn.util.core.task.Tasks;
+import org.apache.brooklyn.util.text.Strings;
+import org.apache.brooklyn.util.time.Duration;
+import org.apache.brooklyn.util.time.Time;
+
+public class SetSensorWorkflowStep extends WorkflowStepDefinition {
+
+    EntityValueToSet sensor;
+    Object value;
+
+    public EntityValueToSet getSensor() {
+        return sensor;
+    }
+
+    public Object getValue() {
+        return value;
+    }
+
+    public void setValue(Object value) {
+        this.value = value;
+    }
+
+    @Override
+    public void setShorthand(String expression) {
+        this.sensor = EntityValueToSet.parseFromShorthand(expression, this::setValue);
+    }
+
+    @Override
+    protected Task<?> newTask(String name, WorkflowExecutionContext workflowExecutionContext) {
+        return Tasks.create(getDefaultTaskName(workflowExecutionContext), () -> {
+            if (sensor==null) throw new IllegalArgumentException("Sensor name is required");
+            String sensorName = workflowExecutionContext.resolve(sensor.name);
+            if (Strings.isBlank(sensorName)) throw new IllegalArgumentException("Sensor name is required");
+            TypeToken<?> type = workflowExecutionContext.lookupType(sensor.type, () -> TypeToken.of(Object.class));
+            Object resolvedValue = workflowExecutionContext.resolve(value, type);
+            workflowExecutionContext.getEntity().sensors().set(Sensors.newSensor(Object.class, sensorName), resolvedValue);

Review Comment:
   Doing a `newSensor` here will overwrite any pre-existing sensor ("regular" or from a previous step), does that matter? Or is it fine because if we are doing a "set" then we are replacing the previous value anyway?



##########
camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/WorkflowYamlTest.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.brooklyn.camp.brooklyn;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Iterables;
+import com.google.common.reflect.TypeToken;
+import org.apache.brooklyn.api.effector.Effector;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.entity.EntityLocal;
+import org.apache.brooklyn.api.entity.EntitySpec;
+import org.apache.brooklyn.api.mgmt.ManagementContext;
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
+import org.apache.brooklyn.api.typereg.RegisteredType;
+import org.apache.brooklyn.core.entity.Dumper;
+import org.apache.brooklyn.core.entity.EntityAsserts;
+import org.apache.brooklyn.core.resolve.jackson.BeanWithTypeUtils;
+import org.apache.brooklyn.core.sensor.Sensors;
+import org.apache.brooklyn.core.test.BrooklynMgmtUnitTestSupport;
+import org.apache.brooklyn.core.typereg.BasicBrooklynTypeRegistry;
+import org.apache.brooklyn.core.typereg.BasicTypeImplementationPlan;
+import org.apache.brooklyn.core.typereg.JavaClassNameTypePlanTransformer;
+import org.apache.brooklyn.core.typereg.RegisteredTypes;
+import org.apache.brooklyn.core.workflow.WorkflowDefinition;
+import org.apache.brooklyn.core.workflow.WorkflowEffector;
+import org.apache.brooklyn.core.workflow.WorkflowStepDefinition;
+import org.apache.brooklyn.core.workflow.WorkflowStepResolution;
+import org.apache.brooklyn.core.workflow.steps.NoOpWorkflowStep;
+import org.apache.brooklyn.core.workflow.steps.SetSensorWorkflowStep;
+import org.apache.brooklyn.core.workflow.steps.SleepWorkflowStep;
+import org.apache.brooklyn.entity.software.base.VanillaSoftwareProcess;
+import org.apache.brooklyn.entity.stock.BasicApplication;
+import org.apache.brooklyn.entity.stock.BasicEntity;
+import org.apache.brooklyn.test.Asserts;
+import org.apache.brooklyn.util.collections.MutableMap;
+import org.apache.brooklyn.util.core.config.ConfigBag;
+import org.apache.brooklyn.util.core.internal.ssh.RecordingSshTool;
+import org.apache.brooklyn.util.exceptions.Exceptions;
+import org.apache.brooklyn.util.time.Duration;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.BeforeTest;
+import org.testng.annotations.Test;
+
+import java.util.Map;
+
+public class WorkflowYamlTest extends AbstractYamlTest {
+
+    static final String VERSION = "0.1.0-SNAPSHOT";
+
+    @SuppressWarnings("deprecation")
+    static RegisteredType addRegisteredTypeBean(ManagementContext mgmt, String symName, Class<?> clazz) {
+        RegisteredType rt = RegisteredTypes.bean(symName, VERSION,
+                new BasicTypeImplementationPlan(JavaClassNameTypePlanTransformer.FORMAT, clazz.getName()));
+        ((BasicBrooklynTypeRegistry)mgmt.getTypeRegistry()).addToLocalUnpersistedTypeRegistry(rt, false);
+        return rt;
+    }
+
+    @BeforeMethod(alwaysRun = true)
+    @Override
+    public void setUp() throws Exception {
+        super.setUp();
+        addRegisteredTypeBean(mgmt(), "sleep", SleepWorkflowStep.class);
+        addRegisteredTypeBean(mgmt(), "no-op", NoOpWorkflowStep.class);
+        addRegisteredTypeBean(mgmt(), "set-sensor", SetSensorWorkflowStep.class);
+        addRegisteredTypeBean(mgmt(), "workflow-effector", WorkflowEffector.class);
+    }
+
+    @Test
+    public void testWorkflowEffector() throws Exception {
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: " + BasicEntity.class.getName(),
+                "  brooklyn.initializers:",
+                "  - type: workflow-effector",
+                "    brooklyn.config:",
+                "      name: myWorkflow",
+                "      steps:",
+                "        step1:",
+                "          type: no-op",
+                "        step1:",

Review Comment:
   Duplicate `step1`. This overwrites the first one, so the output from the dump below is just
   ```shell
   Task[myWorkflow]@BD0CgQnH: Completed, no return value (null)
     Task[Workflow for effector myWorkflow]@UsCAtiUb: Completed, no return value (null)
       Task[step1 - SetSensor]@TDbBaFAD: Completed, no return value (null)
       Task[step2 - SetSensor]@qSQZLEGr: Completed, no return value (null)
   ```
   rather than having three steps.
   
   It would be worth detecting duplicate labels and rejecting the workflow.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brooklyn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [brooklyn-server] ahgittin commented on a diff in pull request #1358: Initial workflow implementation

Posted by GitBox <gi...@apache.org>.
ahgittin commented on code in PR #1358:
URL: https://github.com/apache/brooklyn-server/pull/1358#discussion_r974296799


##########
core/src/main/java/org/apache/brooklyn/core/workflow/steps/SetSensorWorkflowStep.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.brooklyn.core.workflow.steps;
+
+import com.google.common.reflect.TypeToken;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
+import org.apache.brooklyn.core.sensor.Sensors;
+import org.apache.brooklyn.core.typereg.RegisteredTypes;
+import org.apache.brooklyn.core.workflow.WorkflowExecutionContext;
+import org.apache.brooklyn.core.workflow.WorkflowStepDefinition;
+import org.apache.brooklyn.util.core.flags.BrooklynTypeNameResolution;
+import org.apache.brooklyn.util.core.task.Tasks;
+import org.apache.brooklyn.util.text.Strings;
+import org.apache.brooklyn.util.time.Duration;
+import org.apache.brooklyn.util.time.Time;
+
+public class SetSensorWorkflowStep extends WorkflowStepDefinition {
+
+    EntityValueToSet sensor;
+    Object value;
+
+    public EntityValueToSet getSensor() {
+        return sensor;
+    }
+
+    public Object getValue() {
+        return value;
+    }
+
+    public void setValue(Object value) {
+        this.value = value;
+    }
+
+    @Override
+    public void setShorthand(String expression) {
+        this.sensor = EntityValueToSet.parseFromShorthand(expression, this::setValue);
+    }
+
+    @Override
+    protected Task<?> newTask(String name, WorkflowExecutionContext workflowExecutionContext) {
+        return Tasks.create(getDefaultTaskName(workflowExecutionContext), () -> {

Review Comment:
   signature change in #1361 at https://github.com/apache/brooklyn-server/pull/1361/files#diff-061e4e878f0722d9f76ae09ddde336a8ead2d9982e7459125033ab1c594ba987R92



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brooklyn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [brooklyn-server] ahgittin commented on a diff in pull request #1358: Initial workflow implementation

Posted by GitBox <gi...@apache.org>.
ahgittin commented on code in PR #1358:
URL: https://github.com/apache/brooklyn-server/pull/1358#discussion_r974270863


##########
camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/WorkflowYamlTest.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.brooklyn.camp.brooklyn;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Iterables;
+import com.google.common.reflect.TypeToken;
+import org.apache.brooklyn.api.effector.Effector;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.entity.EntityLocal;
+import org.apache.brooklyn.api.entity.EntitySpec;
+import org.apache.brooklyn.api.mgmt.ManagementContext;
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.api.mgmt.classloading.BrooklynClassLoadingContext;
+import org.apache.brooklyn.api.typereg.RegisteredType;
+import org.apache.brooklyn.core.entity.Dumper;
+import org.apache.brooklyn.core.entity.EntityAsserts;
+import org.apache.brooklyn.core.resolve.jackson.BeanWithTypeUtils;
+import org.apache.brooklyn.core.sensor.Sensors;
+import org.apache.brooklyn.core.test.BrooklynMgmtUnitTestSupport;
+import org.apache.brooklyn.core.typereg.BasicBrooklynTypeRegistry;
+import org.apache.brooklyn.core.typereg.BasicTypeImplementationPlan;
+import org.apache.brooklyn.core.typereg.JavaClassNameTypePlanTransformer;
+import org.apache.brooklyn.core.typereg.RegisteredTypes;
+import org.apache.brooklyn.core.workflow.WorkflowDefinition;
+import org.apache.brooklyn.core.workflow.WorkflowEffector;
+import org.apache.brooklyn.core.workflow.WorkflowStepDefinition;
+import org.apache.brooklyn.core.workflow.WorkflowStepResolution;
+import org.apache.brooklyn.core.workflow.steps.NoOpWorkflowStep;
+import org.apache.brooklyn.core.workflow.steps.SetSensorWorkflowStep;
+import org.apache.brooklyn.core.workflow.steps.SleepWorkflowStep;
+import org.apache.brooklyn.entity.software.base.VanillaSoftwareProcess;
+import org.apache.brooklyn.entity.stock.BasicApplication;
+import org.apache.brooklyn.entity.stock.BasicEntity;
+import org.apache.brooklyn.test.Asserts;
+import org.apache.brooklyn.util.collections.MutableMap;
+import org.apache.brooklyn.util.core.config.ConfigBag;
+import org.apache.brooklyn.util.core.internal.ssh.RecordingSshTool;
+import org.apache.brooklyn.util.exceptions.Exceptions;
+import org.apache.brooklyn.util.time.Duration;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.BeforeTest;
+import org.testng.annotations.Test;
+
+import java.util.Map;
+
+public class WorkflowYamlTest extends AbstractYamlTest {
+
+    static final String VERSION = "0.1.0-SNAPSHOT";
+
+    @SuppressWarnings("deprecation")
+    static RegisteredType addRegisteredTypeBean(ManagementContext mgmt, String symName, Class<?> clazz) {
+        RegisteredType rt = RegisteredTypes.bean(symName, VERSION,
+                new BasicTypeImplementationPlan(JavaClassNameTypePlanTransformer.FORMAT, clazz.getName()));
+        ((BasicBrooklynTypeRegistry)mgmt.getTypeRegistry()).addToLocalUnpersistedTypeRegistry(rt, false);
+        return rt;
+    }
+
+    @BeforeMethod(alwaysRun = true)
+    @Override
+    public void setUp() throws Exception {
+        super.setUp();
+        addRegisteredTypeBean(mgmt(), "sleep", SleepWorkflowStep.class);
+        addRegisteredTypeBean(mgmt(), "no-op", NoOpWorkflowStep.class);
+        addRegisteredTypeBean(mgmt(), "set-sensor", SetSensorWorkflowStep.class);
+        addRegisteredTypeBean(mgmt(), "workflow-effector", WorkflowEffector.class);
+    }
+
+    @Test
+    public void testWorkflowEffector() throws Exception {
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: " + BasicEntity.class.getName(),
+                "  brooklyn.initializers:",
+                "  - type: workflow-effector",
+                "    brooklyn.config:",
+                "      name: myWorkflow",
+                "      steps:",
+                "        step1:",
+                "          type: no-op",
+                "        step1:",

Review Comment:
   yaml removes the duplicates so it is quite low level.  actually a good argument for switching to a list-based definition.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brooklyn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [brooklyn-server] algairim commented on pull request #1358: Initial workflow implementation

Posted by GitBox <gi...@apache.org>.
algairim commented on PR #1358:
URL: https://github.com/apache/brooklyn-server/pull/1358#issuecomment-1240775775

   Good start. Merging it.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@brooklyn.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org