You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@streampipes.apache.org by ze...@apache.org on 2020/09/14 21:55:11 UTC

[incubator-streampipes-extensions] branch rel/0.67.0 updated: Add processor string to state

This is an automated email from the ASF dual-hosted git repository.

zehnder pushed a commit to branch rel/0.67.0
in repository https://gitbox.apache.org/repos/asf/incubator-streampipes-extensions.git


The following commit(s) were added to refs/heads/rel/0.67.0 by this push:
     new 3f7e332  Add processor string to state
     new 8216b3e  Merge branch 'rel/0.67.0' of github.com:apache/incubator-streampipes-extensions into rel/0.67.0
3f7e332 is described below

commit 3f7e3321d2fb0b287b8cc7f944bc227ddbd47ccc
Author: Philipp Zehnder <ze...@fzi.de>
AuthorDate: Mon Sep 14 23:45:21 2020 +0200

    Add processor string to state
---
 .../transformation/jvm/TransformationJvmInit.java  |   2 +
 .../state/BooleanToStateController.java            |  12 -----
 .../stringoperator/state/StringToState.java        |  59 +++++++++++++++++++++
 .../state/StringToStateController.java}            |  34 ++++--------
 .../state/StringToStateParameters.java             |  42 +++++++++++++++
 .../strings.en                                     |  11 +---
 .../strings.en                                     |   9 ++++
 .../documentation.md                               |  45 ++++++++++++++++
 .../icon.png                                       | Bin 0 -> 12174 bytes
 .../strings.en                                     |  11 ++++
 10 files changed, 178 insertions(+), 47 deletions(-)

diff --git a/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/TransformationJvmInit.java b/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/TransformationJvmInit.java
index f8045e0..c1ef992 100644
--- a/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/TransformationJvmInit.java
+++ b/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/TransformationJvmInit.java
@@ -40,6 +40,7 @@ import org.apache.streampipes.processors.transformation.jvm.processor.state.buff
 import org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.buffer.StateBufferLabelerController;
 import org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.number.NumberLabelerController;
 import org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.counter.StringCounterController;
+import org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state.StringToStateController;
 import org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.timer.StringTimerController;
 import org.apache.streampipes.processors.transformation.jvm.processor.task.TaskDurationController;
 import org.apache.streampipes.processors.transformation.jvm.processor.timestampextractor.TimestampExtractorController;
@@ -71,6 +72,7 @@ public class TransformationJvmInit extends StandaloneModelSubmitter {
             .add(new StateBufferController())
             .add(new StateBufferLabelerController())
             .add(new NumberLabelerController())
+            .add(new StringToStateController())
             .add(new StringCounterController());
 
     DeclarersSingleton.getInstance().registerDataFormats(new JsonDataFormatFactory(),
diff --git a/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/booloperator/state/BooleanToStateController.java b/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/booloperator/state/BooleanToStateController.java
index 8112731..c715b50 100644
--- a/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/booloperator/state/BooleanToStateController.java
+++ b/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/booloperator/state/BooleanToStateController.java
@@ -36,16 +36,11 @@ import java.util.List;
 
 public class BooleanToStateController extends StandaloneEventProcessingDeclarer<BooleanToStateParameters> {
 
-  public static final String COLLECTION_ID = "collection-id";
-  public static final String GROUP_ID = "group-id";
-  public static final String STATE_NAME_ID = "state-name-id";
-  public static final String STATE_MAPPING_ID = "state-mapping-id";
   public static final String BOOLEAN_STATE_FIELD = "boolean_state_field";
   public static final String DEFAULT_STATE_ID = "default-state-id";
 
   public static final String CURRENT_STATE = "current_state";
 
-
   @Override
   public DataProcessorDescription declareModel() {
     return ProcessingElementBuilder.create("org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state")
@@ -54,13 +49,6 @@ public class BooleanToStateController extends StandaloneEventProcessingDeclarer<
             .requiredStream(StreamRequirementsBuilder.create()
                     .requiredPropertyWithNaryMapping(EpRequirements.booleanReq(), Labels.withId(BOOLEAN_STATE_FIELD), PropertyScope.NONE)
                     .build())
-            // TODO UI does not render collections with group
-//            .requiredParameterAsCollection(
-//                    Labels.withId(COLLECTION_ID),
-//                    StaticProperties.group(
-//                            Labels.withId(GROUP_ID),
-//                            StaticProperties.stringFreeTextProperty(Labels.withId(STATE_NAME_ID)),
-//                            StaticProperties.mappingPropertyUnary(Labels.withId(STATE_MAPPING_ID), PropertyScope.NONE)))
             .requiredTextParameter(Labels.withId(DEFAULT_STATE_ID))
             .outputStrategy(OutputStrategies.append(
                     EpProperties.listStringEp(Labels.withId(CURRENT_STATE), CURRENT_STATE, SPSensor.STATE)
diff --git a/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/stringoperator/state/StringToState.java b/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/stringoperator/state/StringToState.java
new file mode 100644
index 0000000..8730c9c
--- /dev/null
+++ b/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/stringoperator/state/StringToState.java
@@ -0,0 +1,59 @@
+/*
+ * 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.streampipes.processors.transformation.jvm.processor.stringoperator.state;
+
+import org.apache.streampipes.logging.api.Logger;
+import org.apache.streampipes.model.runtime.Event;
+import org.apache.streampipes.wrapper.context.EventProcessorRuntimeContext;
+import org.apache.streampipes.wrapper.routing.SpOutputCollector;
+import org.apache.streampipes.wrapper.runtime.EventProcessor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class StringToState implements EventProcessor<StringToStateParameters> {
+
+  private static Logger LOG;
+
+  private List<String> stateFields;
+
+  @Override
+  public void onInvocation(StringToStateParameters booleanInverterParameters,
+                           SpOutputCollector spOutputCollector,
+                           EventProcessorRuntimeContext runtimeContext) {
+    LOG = booleanInverterParameters.getGraph().getLogger(StringToState.class);
+    this.stateFields = booleanInverterParameters.getStateFields();
+  }
+
+  @Override
+  public void onEvent(Event inputEvent, SpOutputCollector out) {
+    List<String> states = new ArrayList<>();
+
+    for (String stateField : stateFields) {
+      states.add(inputEvent.getFieldBySelector(stateField).getAsPrimitive().getAsString());
+    }
+
+    inputEvent.addField(StringToStateController.CURRENT_STATE, states.toArray());
+    out.collect(inputEvent);
+  }
+
+  @Override
+  public void onDetach() {
+  }
+}
diff --git a/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/booloperator/state/BooleanToStateController.java b/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/stringoperator/state/StringToStateController.java
similarity index 55%
copy from streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/booloperator/state/BooleanToStateController.java
copy to streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/stringoperator/state/StringToStateController.java
index 8112731..4aedbfe 100644
--- a/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/booloperator/state/BooleanToStateController.java
+++ b/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/stringoperator/state/StringToStateController.java
@@ -16,13 +16,11 @@
  *
  */
 
-package org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state;
+package org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state;
 
 import org.apache.streampipes.model.graph.DataProcessorDescription;
 import org.apache.streampipes.model.graph.DataProcessorInvocation;
 import org.apache.streampipes.model.schema.PropertyScope;
-import org.apache.streampipes.model.staticproperty.MappingPropertyUnary;
-import org.apache.streampipes.sdk.StaticProperties;
 import org.apache.streampipes.sdk.builder.ProcessingElementBuilder;
 import org.apache.streampipes.sdk.builder.StreamRequirementsBuilder;
 import org.apache.streampipes.sdk.extractor.ProcessingElementParameterExtractor;
@@ -34,34 +32,21 @@ import org.apache.streampipes.wrapper.standalone.declarer.StandaloneEventProcess
 
 import java.util.List;
 
-public class BooleanToStateController extends StandaloneEventProcessingDeclarer<BooleanToStateParameters> {
+public class StringToStateController extends StandaloneEventProcessingDeclarer<StringToStateParameters> {
 
-  public static final String COLLECTION_ID = "collection-id";
-  public static final String GROUP_ID = "group-id";
-  public static final String STATE_NAME_ID = "state-name-id";
-  public static final String STATE_MAPPING_ID = "state-mapping-id";
-  public static final String BOOLEAN_STATE_FIELD = "boolean_state_field";
-  public static final String DEFAULT_STATE_ID = "default-state-id";
+  public static final String STRING_STATE_FIELD = "string_state_field";
 
   public static final String CURRENT_STATE = "current_state";
 
 
   @Override
   public DataProcessorDescription declareModel() {
-    return ProcessingElementBuilder.create("org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state")
+    return ProcessingElementBuilder.create("org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state")
             .withLocales(Locales.EN)
             .withAssets(Assets.DOCUMENTATION, Assets.ICON)
             .requiredStream(StreamRequirementsBuilder.create()
-                    .requiredPropertyWithNaryMapping(EpRequirements.booleanReq(), Labels.withId(BOOLEAN_STATE_FIELD), PropertyScope.NONE)
+                    .requiredPropertyWithNaryMapping(EpRequirements.stringReq(), Labels.withId(STRING_STATE_FIELD), PropertyScope.NONE)
                     .build())
-            // TODO UI does not render collections with group
-//            .requiredParameterAsCollection(
-//                    Labels.withId(COLLECTION_ID),
-//                    StaticProperties.group(
-//                            Labels.withId(GROUP_ID),
-//                            StaticProperties.stringFreeTextProperty(Labels.withId(STATE_NAME_ID)),
-//                            StaticProperties.mappingPropertyUnary(Labels.withId(STATE_MAPPING_ID), PropertyScope.NONE)))
-            .requiredTextParameter(Labels.withId(DEFAULT_STATE_ID))
             .outputStrategy(OutputStrategies.append(
                     EpProperties.listStringEp(Labels.withId(CURRENT_STATE), CURRENT_STATE, SPSensor.STATE)
             ))
@@ -69,13 +54,12 @@ public class BooleanToStateController extends StandaloneEventProcessingDeclarer<
   }
 
   @Override
-  public ConfiguredEventProcessor<BooleanToStateParameters> onInvocation(DataProcessorInvocation graph, ProcessingElementParameterExtractor extractor) {
+  public ConfiguredEventProcessor<StringToStateParameters> onInvocation(DataProcessorInvocation graph, ProcessingElementParameterExtractor extractor) {
 
-    List<String> stateFields = extractor.mappingPropertyValues(BOOLEAN_STATE_FIELD);
-    String defaultState = extractor.singleValueParameter(DEFAULT_STATE_ID, String.class);
+    List<String> stateFields = extractor.mappingPropertyValues(STRING_STATE_FIELD);
 
-    BooleanToStateParameters params = new BooleanToStateParameters(graph, stateFields, defaultState);
+    StringToStateParameters params = new StringToStateParameters(graph, stateFields);
 
-    return new ConfiguredEventProcessor<>(params, BooleanToState::new);
+    return new ConfiguredEventProcessor<>(params, StringToState::new);
   }
 }
diff --git a/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/stringoperator/state/StringToStateParameters.java b/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/stringoperator/state/StringToStateParameters.java
new file mode 100644
index 0000000..08bb155
--- /dev/null
+++ b/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/stringoperator/state/StringToStateParameters.java
@@ -0,0 +1,42 @@
+/*
+ * 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.streampipes.processors.transformation.jvm.processor.stringoperator.state;
+
+import org.apache.streampipes.model.graph.DataProcessorInvocation;
+import org.apache.streampipes.wrapper.params.binding.EventProcessorBindingParams;
+
+import java.util.List;
+
+public class StringToStateParameters extends EventProcessorBindingParams {
+    private List<String> stateFields;
+
+    public StringToStateParameters(DataProcessorInvocation graph, List<String> stateFields) {
+        super(graph);
+        this.stateFields = stateFields;
+    }
+
+    public List<String> getStateFields() {
+        return stateFields;
+    }
+
+    public void setStateFields(List<String> stateFields) {
+        this.stateFields = stateFields;
+    }
+
+}
diff --git a/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state/strings.en b/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state/strings.en
index b5ebeeb..f95156d 100644
--- a/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state/strings.en
+++ b/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.booloperator.state/strings.en
@@ -7,17 +7,8 @@ state-field.description=Test
 collection-id.title=Collection Field
 collection-id.description=Test
 
-group-id.title=Group Field
-group-id.description=Test
-
-state-name-id.title=State Name
-state-name-id.description=Enter the name of the state
-
-state-mapping-id.title=State Mapping
-state-mapping-id.description=When boolean variable is true, the state is set to active
-
 default-state-id.title=Default State
 default-state-id.description=Define a default state, when all values are false
 
-current_state.title=Current State
+boolean_state.title=Current State
 boolean_state_field.description=
diff --git a/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.number/strings.en b/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.number/strings.en
index 4d568d0..94ef97a 100644
--- a/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.number/strings.en
+++ b/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.state.labeler.number/strings.en
@@ -7,6 +7,15 @@ sensorValueId.description=The value used for the labeling
 labelCollectionId.title=Condition
 labelCollectionId.description=Add a condition to provide a label for the data
 
+stateFilterId.title=Select a specific state
+stateFilterId.description=Add a filter to define which states to evaluate. Add '*' to select all states.
+
+stateFieldId.title=State field
+stateFieldId.description=Select the field containing the state of the event
+
+operationsId.title=Operation
+operationsId.description=Define the operation that should be applied on sensor values
+
 label.title=Label
 label.description=User defined label for data
 
diff --git a/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/documentation.md b/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/documentation.md
new file mode 100644
index 0000000..961c54b
--- /dev/null
+++ b/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/documentation.md
@@ -0,0 +1,45 @@
+<!--
+  ~ 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.
+  ~
+  -->
+
+## String To State
+
+<p align="center"> 
+    <img src="icon.png" width="150px;" class="pe-image-documentation"/>
+</p>
+
+***
+
+## Description
+
+Convert string fields to a state representing the current state of the system.
+This processor requires one or multiple string values in the data stream.
+For each of the selected values is added to the states field.
+***
+
+## Required input
+
+### String Fields
+String fields that are added to the state array
+
+***
+
+## Configuration
+No further configuration required
+
+## Output
+The output contains a new value with the string values of the state
diff --git a/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/icon.png b/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/icon.png
new file mode 100644
index 0000000..61a7699
Binary files /dev/null and b/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/icon.png differ
diff --git a/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/strings.en b/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/strings.en
new file mode 100644
index 0000000..dac85b1
--- /dev/null
+++ b/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state/strings.en
@@ -0,0 +1,11 @@
+org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state.title=String To State
+org.apache.streampipes.processors.transformation.jvm.processor.stringoperator.state.description=Transforms string properties into a string state property
+
+state-field.title=State field
+state-field.description=Test
+
+collection-id.title=Collection Field
+collection-id.description=Test
+
+string_state.title=Current State
+string_state_field.description=