You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@streampipes.apache.org by wi...@apache.org on 2020/02/07 14:16:49 UTC

[incubator-streampipes-extensions] 01/02: STREAMPIPES-72

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

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

commit 9078591e59d7c9e23d90ee9036b79788fc2ca815
Author: Patrick Wiener <wi...@fzi.de>
AuthorDate: Fri Feb 7 15:14:28 2020 +0100

    STREAMPIPES-72
---
 .../pe/jvm/AllPipelineElementsInit.java            |   5 +-
 .../processors/filters/jvm/FiltersJvmInit.java     |   4 +-
 .../numericaltextfilter/NumericalOperator.java     |  24 +++++
 .../numericaltextfilter/NumericalTextFilter.java   |  81 +++++++++++++++
 .../NumericalTextFilterController.java             | 115 +++++++++++++++++++++
 .../NumericalTextFilterParameters.java             |  67 ++++++++++++
 .../numericaltextfilter/StringOperator.java        |  23 +++++
 .../documentation.md                               |  61 +++++++++++
 .../icon.png                                       | Bin 0 -> 40429 bytes
 .../strings.en                                     |  20 ++++
 10 files changed, 398 insertions(+), 2 deletions(-)

diff --git a/streampipes-pipeline-elements-all-jvm/src/main/java/org/apache/streampipes/pe/jvm/AllPipelineElementsInit.java b/streampipes-pipeline-elements-all-jvm/src/main/java/org/apache/streampipes/pe/jvm/AllPipelineElementsInit.java
index 6321b29..3841105 100644
--- a/streampipes-pipeline-elements-all-jvm/src/main/java/org/apache/streampipes/pe/jvm/AllPipelineElementsInit.java
+++ b/streampipes-pipeline-elements-all-jvm/src/main/java/org/apache/streampipes/pe/jvm/AllPipelineElementsInit.java
@@ -28,6 +28,7 @@ import org.apache.streampipes.messaging.kafka.SpKafkaProtocolFactory;
 import org.apache.streampipes.pe.jvm.config.AllPipelineElementsConfig;
 import org.apache.streampipes.processors.filters.jvm.processor.compose.ComposeController;
 import org.apache.streampipes.processors.filters.jvm.processor.numericalfilter.NumericalFilterController;
+import org.apache.streampipes.processors.filters.jvm.processor.numericaltextfilter.NumericalTextFilterController;
 import org.apache.streampipes.processors.filters.jvm.processor.projection.ProjectionController;
 import org.apache.streampipes.processors.filters.jvm.processor.textfilter.TextFilterController;
 import org.apache.streampipes.processors.filters.jvm.processor.threshold.ThresholdDetectionController;
@@ -43,6 +44,7 @@ import org.apache.streampipes.processors.transformation.jvm.processor.booloperat
 import org.apache.streampipes.processors.transformation.jvm.processor.booloperator.inverter.BooleanInverterController;
 import org.apache.streampipes.processors.transformation.jvm.processor.booloperator.timekeeping.BooleanTimekeepingController;
 import org.apache.streampipes.processors.transformation.jvm.processor.booloperator.timer.BooleanTimerController;
+import org.apache.streampipes.processors.transformation.jvm.processor.statemonitor.StateMonitorController;
 import org.apache.streampipes.processors.transformation.jvm.processor.csvmetadata.CsvMetadataEnrichmentController;
 import org.apache.streampipes.processors.transformation.jvm.processor.task.TaskDurationController;
 import org.apache.streampipes.processors.transformation.jvm.processor.timestampextractor.TimestampExtractorController;
@@ -109,7 +111,8 @@ public class AllPipelineElementsInit extends StandaloneModelSubmitter {
             .add(new DashboardController())
             .add(new EmailController())
             .add(new OneSignalController())
-            .add(new SlackNotificationController());
+            .add(new SlackNotificationController())
+            .add(new NumericalTextFilterController());
 
 
     DeclarersSingleton.getInstance().registerDataFormats(new JsonDataFormatFactory(),
diff --git a/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/FiltersJvmInit.java b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/FiltersJvmInit.java
index 5cb6743..61ef759 100644
--- a/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/FiltersJvmInit.java
+++ b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/FiltersJvmInit.java
@@ -31,6 +31,7 @@ import org.apache.streampipes.processors.filters.jvm.processor.compose.ComposeCo
 import org.apache.streampipes.processors.filters.jvm.processor.enrich.MergeByEnrichController;
 import org.apache.streampipes.processors.filters.jvm.processor.merge.MergeByTimeController;
 import org.apache.streampipes.processors.filters.jvm.processor.numericalfilter.NumericalFilterController;
+import org.apache.streampipes.processors.filters.jvm.processor.numericaltextfilter.NumericalTextFilterController;
 import org.apache.streampipes.processors.filters.jvm.processor.projection.ProjectionController;
 import org.apache.streampipes.processors.filters.jvm.processor.textfilter.TextFilterController;
 import org.apache.streampipes.processors.filters.jvm.processor.threshold.ThresholdDetectionController;
@@ -46,7 +47,8 @@ public class FiltersJvmInit extends StandaloneModelSubmitter {
             .add(new ProjectionController())
             .add(new MergeByEnrichController())
             .add(new MergeByTimeController())
-            .add(new ComposeController());
+            .add(new ComposeController())
+            .add(new NumericalTextFilterController());
 
     DeclarersSingleton.getInstance().registerDataFormats(new JsonDataFormatFactory(),
             new CborDataFormatFactory(),
diff --git a/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalOperator.java b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalOperator.java
new file mode 100644
index 0000000..1864795
--- /dev/null
+++ b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalOperator.java
@@ -0,0 +1,24 @@
+/*
+ * 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.filters.jvm.processor.numericaltextfilter;
+
+public enum NumericalOperator {
+  // Greater/Equal, Greater than, Less/Equal, Less than, Equal, Inequal
+  GE, GT, LE, LT, EQ, IE
+}
diff --git a/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalTextFilter.java b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalTextFilter.java
new file mode 100644
index 0000000..27c3950
--- /dev/null
+++ b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalTextFilter.java
@@ -0,0 +1,81 @@
+/*
+ * 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.filters.jvm.processor.numericaltextfilter;
+
+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;
+
+public class NumericalTextFilter implements EventProcessor<NumericalTextFilterParameters> {
+
+  private NumericalTextFilterParameters params;
+
+  @Override
+  public void onInvocation(NumericalTextFilterParameters numericalTextFilterParameters, SpOutputCollector spOutputCollector, EventProcessorRuntimeContext
+          runtimeContext) {
+    this.params = numericalTextFilterParameters;
+  }
+
+  @Override
+  public void onEvent(Event event, SpOutputCollector out) {
+    boolean satisfiesNumberFilter = false;
+    boolean satisfiesTextFilter = false;
+
+    Double numbervalue = event.getFieldBySelector(params.getNumberProperty())
+            .getAsPrimitive()
+            .getAsDouble();
+
+    String value = event.getFieldBySelector(params.getTextProperty())
+            .getAsPrimitive()
+            .getAsString();
+
+    //Double value = Double.parseDouble(String.valueOf(in.get(params.getFilterProperty())));
+    Double threshold = params.getNumberThreshold();
+
+    if (params.getNumericalOperator() == NumericalOperator.EQ) {
+      satisfiesNumberFilter = (Math.abs(numbervalue - threshold) < 0.000001);
+    } else if (params.getNumericalOperator() == NumericalOperator.GE) {
+      satisfiesNumberFilter = (numbervalue >= threshold);
+    } else if (params.getNumericalOperator() == NumericalOperator.GT) {
+      satisfiesNumberFilter = numbervalue > threshold;
+    } else if (params.getNumericalOperator() == NumericalOperator.LE) {
+      satisfiesNumberFilter = (numbervalue <= threshold);
+    } else if (params.getNumericalOperator() == NumericalOperator.LT) {
+      satisfiesNumberFilter = (numbervalue < threshold);
+    } else if (params.getNumericalOperator() == NumericalOperator.IE) {
+      satisfiesNumberFilter = (Math.abs(numbervalue - threshold) > 0.000001);
+    }
+
+    if (params.getTextOperator() == StringOperator.MATCHES) {
+      satisfiesTextFilter = (value.equals(params.getTextKeyword()));
+    } else if (params.getTextOperator() == StringOperator.CONTAINS) {
+      satisfiesTextFilter = (value.contains(params.getTextKeyword()));
+    }
+
+    if (satisfiesNumberFilter && satisfiesTextFilter) {
+      out.collect(event);
+    }
+  }
+
+  @Override
+  public void onDetach() {
+
+  }
+}
diff --git a/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalTextFilterController.java b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalTextFilterController.java
new file mode 100644
index 0000000..32882d5
--- /dev/null
+++ b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalTextFilterController.java
@@ -0,0 +1,115 @@
+/*
+ * 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.filters.jvm.processor.numericaltextfilter;
+
+import org.apache.streampipes.container.api.ResolvesContainerProvidedOptions;
+import org.apache.streampipes.model.DataProcessorType;
+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.Option;
+import org.apache.streampipes.sdk.builder.ProcessingElementBuilder;
+import org.apache.streampipes.sdk.builder.StreamRequirementsBuilder;
+import org.apache.streampipes.sdk.extractor.ProcessingElementParameterExtractor;
+import org.apache.streampipes.sdk.extractor.StaticPropertyExtractor;
+import org.apache.streampipes.sdk.helpers.*;
+import org.apache.streampipes.sdk.utils.Assets;
+import org.apache.streampipes.wrapper.standalone.ConfiguredEventProcessor;
+import org.apache.streampipes.wrapper.standalone.declarer.StandaloneEventProcessingDeclarer;
+
+import java.util.List;
+
+public class NumericalTextFilterController extends StandaloneEventProcessingDeclarer<NumericalTextFilterParameters> {
+
+  // number
+  private static final String NUMBER_MAPPING = "number-mapping";
+  private static final String NUMBER_OPERATION = "number-operation";
+  private static final String NUMBER_VALUE = "number-value";
+  // text
+  private static final String TEXT_MAPPING = "text-mapping";
+  private static final String TEXT_OPERATION = "text-operation";
+  private static final String TEXT_KEYWORD = "text-keyword";
+
+  @Override
+  public DataProcessorDescription declareModel() {
+    return ProcessingElementBuilder.create("org.apache.streampipes.processors.filters.jvm.numericaltextfilter")
+            .category(DataProcessorType.FILTER)
+            .withAssets(Assets.DOCUMENTATION, Assets.ICON)
+            .withLocales(Locales.EN)
+            .requiredStream(StreamRequirementsBuilder
+                    .create()
+                    .requiredPropertyWithUnaryMapping(EpRequirements.numberReq(),
+                            Labels.withId(NUMBER_MAPPING),
+                            PropertyScope.MEASUREMENT_PROPERTY)
+                    .requiredPropertyWithUnaryMapping(EpRequirements.stringReq(),
+                            Labels.withId(TEXT_MAPPING), PropertyScope.NONE)
+                    .build())
+            .requiredSingleValueSelection(Labels.withId(NUMBER_OPERATION), Options.from("<", "<=", ">",
+                    ">=", "==", "!="))
+            .requiredFloatParameter(Labels.withId(NUMBER_VALUE), NUMBER_MAPPING)
+            .requiredSingleValueSelection(Labels.withId(TEXT_OPERATION), Options.from("MATCHES",
+                    "CONTAINS"))
+            .requiredTextParameter(Labels.withId(TEXT_KEYWORD), "text")
+            .outputStrategy(OutputStrategies.keep())
+            .build();
+
+  }
+
+  @Override
+  public ConfiguredEventProcessor<NumericalTextFilterParameters> onInvocation
+          (DataProcessorInvocation graph, ProcessingElementParameterExtractor extractor) {
+
+    // number
+    String numberProperty = extractor.mappingPropertyValue(NUMBER_MAPPING);
+    Double numberThreshold = extractor.singleValueParameter(NUMBER_VALUE, Double.class);
+    String numberOperation = extractor.selectedSingleValue(NUMBER_OPERATION, String.class);
+
+    // text
+    String textProperty = extractor.mappingPropertyValue(TEXT_MAPPING);
+    String textKeyword = extractor.singleValueParameter(TEXT_KEYWORD, String.class);
+    String textOperation = extractor.selectedSingleValue(TEXT_OPERATION, String.class);
+
+    String numOperation = "GT";
+
+    if (numberOperation.equals("<=")) {
+      numOperation = "LE";
+    } else if (numberOperation.equals("<")) {
+      numOperation = "LT";
+    } else if (numberOperation.equals(">=")) {
+      numOperation = "GE";
+    } else if (numberOperation.equals("==")) {
+      numOperation = "EQ";
+    } else if (numberOperation.equals("!=")) {
+      numOperation = "IE";
+    }
+
+
+    NumericalTextFilterParameters staticParam = new NumericalTextFilterParameters(
+            graph,
+            numberThreshold,
+            NumericalOperator.valueOf(numOperation),
+            numberProperty,
+            textKeyword,
+            StringOperator.valueOf(textOperation),
+            textProperty);
+
+    return new ConfiguredEventProcessor<>(staticParam, NumericalTextFilter::new);
+  }
+
+}
diff --git a/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalTextFilterParameters.java b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalTextFilterParameters.java
new file mode 100644
index 0000000..c669430
--- /dev/null
+++ b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/NumericalTextFilterParameters.java
@@ -0,0 +1,67 @@
+/*
+ * 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.filters.jvm.processor.numericaltextfilter;
+
+import org.apache.streampipes.model.graph.DataProcessorInvocation;
+import org.apache.streampipes.wrapper.params.binding.EventProcessorBindingParams;
+
+public class NumericalTextFilterParameters extends EventProcessorBindingParams {
+
+  private double numberThreshold;
+  private NumericalOperator numericalOperator;
+  private String numberProperty;
+  private String textKeyword;
+  private StringOperator textOperator;
+  private String textProperty;
+
+  public NumericalTextFilterParameters(DataProcessorInvocation graph, Double numberThreshold, NumericalOperator
+          NumericalOperator, String numberProperty, String textKeyword, StringOperator textOperator, String textProperty) {
+    super(graph);
+    this.numberThreshold = numberThreshold;
+    this.numericalOperator = NumericalOperator;
+    this.numberProperty = numberProperty;
+    this.textKeyword = textKeyword;
+    this.textOperator = textOperator;
+    this.textProperty = textProperty;
+  }
+
+  public double getNumberThreshold() {
+    return numberThreshold;
+  }
+
+  public NumericalOperator getNumericalOperator() {
+    return numericalOperator;
+  }
+
+  public String getNumberProperty() {
+    return numberProperty;
+  }
+
+  public String getTextKeyword() {
+    return textKeyword;
+  }
+
+  public StringOperator getTextOperator() {
+    return textOperator;
+  }
+
+  public String getTextProperty() {
+    return textProperty;
+  }
+}
diff --git a/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/StringOperator.java b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/StringOperator.java
new file mode 100644
index 0000000..3d53118
--- /dev/null
+++ b/streampipes-processors-filters-jvm/src/main/java/org/apache/streampipes/processors/filters/jvm/processor/numericaltextfilter/StringOperator.java
@@ -0,0 +1,23 @@
+/*
+ * 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.filters.jvm.processor.numericaltextfilter;
+
+public enum StringOperator {
+	CONTAINS, MATCHES
+}
diff --git a/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/documentation.md b/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/documentation.md
new file mode 100644
index 0000000..88147ad
--- /dev/null
+++ b/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/documentation.md
@@ -0,0 +1,61 @@
+<!--
+  ~ 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.
+  ~
+  -->
+
+## Numerical Text Filter
+
+<p align="center"> 
+    <img src="icon.png" width="150px;" class="pe-image-documentation"/>
+</p>
+
+
+***
+
+## Description
+The Numerical Text Filter processor filters numerical values based on a given threshold and text values 
+based on a given string. It only forwards events in case both criteria are satisfied.
+
+***
+
+## Required input
+The processor works with any input event that has one field containing a numerical value and one field 
+containing a text.
+
+***
+
+## Configuration
+
+### Number Field
+Specifies the field name where the filter operation should be applied on.
+
+### Number Operation
+Specifies the filter operation that should be applied on the field.
+
+### Number Threshold
+Specifies the threshold value.
+
+### Text Field
+The field containing the text that should be filtered.
+
+### Text Operation
+The operation used by the filter processor (equals or matches).
+
+### Text Keyword
+Specifies the keyword to filter the text field.
+
+## Output
+The processor outputs the input event if it satisfies the filter expression.
diff --git a/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/icon.png b/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/icon.png
new file mode 100644
index 0000000..35052bf
Binary files /dev/null and b/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/icon.png differ
diff --git a/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/strings.en b/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/strings.en
new file mode 100644
index 0000000..d5ccfdd
--- /dev/null
+++ b/streampipes-processors-filters-jvm/src/main/resources/org.apache.streampipes.processors.filters.jvm.numericaltextfilter/strings.en
@@ -0,0 +1,20 @@
+org.apache.streampipes.processors.filters.jvm.numericaltextfilter.title=Numerical Text Filter
+org.apache.streampipes.processors.filters.jvm.numericaltextfilter.description=Filters numerical and text values.
+
+number-mapping.title=Number Field
+number-mapping.description=Specifies the field name where the filter operation should be applied on.
+
+number-operation.title=Number Filter Operation
+number-operation.description=Specifies the filter operation that should be applied on the field
+
+number-value.title=Number Threshold
+number-value.description=Specifies a threshold value
+
+text-mapping.title=Text Field
+text.description=
+
+text-operation.title=Text Operation
+text-operation.description=
+
+text-keyword.title=Text keyword
+text-keyword.description=
\ No newline at end of file