You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@streampipes.apache.org by ri...@apache.org on 2022/08/12 13:58:50 UTC

[incubator-streampipes] branch dev updated: [STREAMPIPES-576] Add transformation processors to transformation-jvm module

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

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


The following commit(s) were added to refs/heads/dev by this push:
     new 64f79c4d7 [STREAMPIPES-576] Add transformation processors to transformation-jvm module
64f79c4d7 is described below

commit 64f79c4d700ea3fdfbd6b88889fd1eda4ba23bf7
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Fri Aug 12 15:58:42 2022 +0200

    [STREAMPIPES-576] Add transformation processors to transformation-jvm module
---
 .../transformation/jvm/TransformationJvmInit.java  |   6 +
 .../jvm/processor/hasher/FieldHasherProcessor.java |  81 ++++++++++++
 .../processor/hasher/algorithm/HashAlgorithm.java  |  25 +---
 .../hasher/algorithm/HashAlgorithmType.java        |  34 ++---
 .../hasher/algorithm/Md5HashAlgorithm.java         |  27 ++--
 .../hasher/algorithm/Sha1HashAlgorithm.java        |  27 ++--
 .../hasher/algorithm/Sha2HashAlgorithm.java        |  27 ++--
 .../jvm/processor/mapper/FieldMapperProcessor.java | 117 ++++++++++++++++
 .../MeasurementUnitConverterProcessor.java         | 147 +++++++++++++++++++++
 .../documentation.md                               |  63 +++++++++
 .../icon.png                                       | Bin 0 -> 11398 bytes
 .../strings.en                                     |   8 ++
 .../documentation.md                               |  48 +++++++
 .../icon.png                                       | Bin 0 -> 19799 bytes
 .../strings.en                                     |   8 ++
 .../documentation.md                               |  47 +++++++
 .../icon.png                                       | Bin 0 -> 10904 bytes
 .../strings.en                                     |   8 ++
 .../fieldHasher1/description.json                  |   7 +
 .../transformation-jvm/fieldHasher1/expected.csv   |   3 +
 .../transformation-jvm/fieldHasher1/input.csv      |   3 +
 .../fieldHasher2/description.json                  |   7 +
 .../transformation-jvm/fieldHasher2/expected.csv   |   3 +
 .../transformation-jvm/fieldHasher2/input.csv      |   5 +
 .../fieldMapper1/description.json                  |   8 ++
 .../transformation-jvm/fieldMapper1/expected.csv   |   3 +
 .../transformation-jvm/fieldMapper1/input.csv      |   5 +
 .../tests/pipelineElement/SinglePipelineElement.ts |   2 +-
 28 files changed, 623 insertions(+), 96 deletions(-)

diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/TransformationJvmInit.java b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/TransformationJvmInit.java
index a7d637d82..f50421bf8 100644
--- a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/TransformationJvmInit.java
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/TransformationJvmInit.java
@@ -39,6 +39,9 @@ import org.apache.streampipes.processors.transformation.jvm.processor.booloperat
 import org.apache.streampipes.processors.transformation.jvm.processor.booloperator.timer.BooleanTimerController;
 import org.apache.streampipes.processors.transformation.jvm.processor.csvmetadata.CsvMetadataEnrichmentController;
 import org.apache.streampipes.processors.transformation.jvm.processor.fieldrename.FiledRenameProcessor;
+import org.apache.streampipes.processors.transformation.jvm.processor.hasher.FieldHasherProcessor;
+import org.apache.streampipes.processors.transformation.jvm.processor.mapper.FieldMapperProcessor;
+import org.apache.streampipes.processors.transformation.jvm.processor.measurementconverter.MeasurementUnitConverterProcessor;
 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;
@@ -72,6 +75,9 @@ public class TransformationJvmInit extends StandaloneModelSubmitter {
                     new BooleanTimekeepingController(),
                     new BooleanTimerController(),
                     new CsvMetadataEnrichmentController(),
+                    new FieldHasherProcessor(),
+                    new FieldMapperProcessor(),
+                    new MeasurementUnitConverterProcessor(),
                     new TaskDurationController(),
                     new TransformToBooleanController(),
                     new StringTimerController(),
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/FieldHasherProcessor.java b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/FieldHasherProcessor.java
new file mode 100644
index 000000000..e13decd84
--- /dev/null
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/FieldHasherProcessor.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.transformation.jvm.processor.hasher;
+
+import org.apache.streampipes.commons.exceptions.SpRuntimeException;
+import org.apache.streampipes.model.graph.DataProcessorDescription;
+import org.apache.streampipes.model.runtime.Event;
+import org.apache.streampipes.model.schema.PropertyScope;
+import org.apache.streampipes.processors.transformation.jvm.processor.hasher.algorithm.HashAlgorithm;
+import org.apache.streampipes.processors.transformation.jvm.processor.hasher.algorithm.HashAlgorithmType;
+import org.apache.streampipes.sdk.builder.ProcessingElementBuilder;
+import org.apache.streampipes.sdk.builder.StreamRequirementsBuilder;
+import org.apache.streampipes.sdk.helpers.*;
+import org.apache.streampipes.sdk.utils.Assets;
+import org.apache.streampipes.wrapper.context.EventProcessorRuntimeContext;
+import org.apache.streampipes.wrapper.routing.SpOutputCollector;
+import org.apache.streampipes.wrapper.standalone.ProcessorParams;
+import org.apache.streampipes.wrapper.standalone.StreamPipesDataProcessor;
+
+public class FieldHasherProcessor extends StreamPipesDataProcessor {
+
+  private static final String HASH_PROPERTIES = "property-mapping";
+  private static final String HASH_ALGORITHM = "hash-algorithm";
+
+  private String propertyName;
+  private HashAlgorithm hashAlgorithm;
+
+  @Override
+  public DataProcessorDescription declareModel() {
+    return ProcessingElementBuilder.create("org.apache.streampipes.processors.transformation.jvm.fieldhasher")
+      .withLocales(Locales.EN)
+      .withAssets(Assets.DOCUMENTATION, Assets.ICON)
+      .requiredStream(StreamRequirementsBuilder
+        .create()
+        .requiredPropertyWithUnaryMapping(EpRequirements.stringReq(), Labels.withId
+          (HASH_PROPERTIES), PropertyScope.NONE)
+        .build())
+      .requiredSingleValueSelection(Labels.withId(HASH_ALGORITHM),
+        Options.from("SHA1", "SHA2", "MD5"))
+      .outputStrategy(OutputStrategies.keep())
+      .build();
+  }
+
+  @Override
+  public void onInvocation(ProcessorParams parameters,
+                           SpOutputCollector spOutputCollector,
+                           EventProcessorRuntimeContext runtimeContext) throws SpRuntimeException {
+    var extractor = parameters.extractor();
+    this.propertyName = extractor.mappingPropertyValue(HASH_PROPERTIES);
+
+    this.hashAlgorithm = HashAlgorithmType.valueOf(extractor.selectedSingleValue(HASH_ALGORITHM, String.class)).hashAlgorithm();
+  }
+
+  @Override
+  public void onEvent(Event in, SpOutputCollector out) throws SpRuntimeException {
+    in.updateFieldBySelector(propertyName,
+      hashAlgorithm.toHashValue(in.getFieldBySelector(propertyName).getAsPrimitive().getAsString()));
+    out.collect(in);
+  }
+
+  @Override
+  public void onDetach() throws SpRuntimeException {
+
+  }
+}
diff --git a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/HashAlgorithm.java
similarity index 55%
copy from ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
copy to streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/HashAlgorithm.java
index 167d76786..5ee116243 100644
--- a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/HashAlgorithm.java
@@ -16,26 +16,11 @@
  *
  */
 
-import { ProcessingElementTestUtils } from '../../support/utils/ProcessingElementTestUtils';
-import { ProcessorTest } from '../../support/model/ProcessorTest';
+package org.apache.streampipes.processors.transformation.jvm.processor.hasher.algorithm;
 
-const allTests = Cypress.env('processingElements');
+import java.io.Serializable;
 
-allTests.forEach(test => {
-  const testNames = ['trigonometry2'];
+public interface HashAlgorithm extends Serializable {
 
-  const processorTest = test as ProcessorTest;
-
-  if (testNames.includes(processorTest.name)) {
-
-    describe('Test Processor ' + test.dir, () => {
-      beforeEach('Setup Test', () => {
-        cy.initStreamPipesTest();
-      });
-
-      it('Initialize Test', () => {
-        ProcessingElementTestUtils.testElement(processorTest);
-      });
-    });
-  }
-});
+	String toHashValue(Object value);
+}
diff --git a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/HashAlgorithmType.java
similarity index 55%
copy from ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
copy to streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/HashAlgorithmType.java
index 167d76786..943aa40c3 100644
--- a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/HashAlgorithmType.java
@@ -16,26 +16,18 @@
  *
  */
 
-import { ProcessingElementTestUtils } from '../../support/utils/ProcessingElementTestUtils';
-import { ProcessorTest } from '../../support/model/ProcessorTest';
+package org.apache.streampipes.processors.transformation.jvm.processor.hasher.algorithm;
 
-const allTests = Cypress.env('processingElements');
+public enum HashAlgorithmType {
+	MD5(new Md5HashAlgorithm()), SHA1(new Sha1HashAlgorithm()), SHA2(new Sha2HashAlgorithm());
 
-allTests.forEach(test => {
-  const testNames = ['trigonometry2'];
-
-  const processorTest = test as ProcessorTest;
-
-  if (testNames.includes(processorTest.name)) {
-
-    describe('Test Processor ' + test.dir, () => {
-      beforeEach('Setup Test', () => {
-        cy.initStreamPipesTest();
-      });
-
-      it('Initialize Test', () => {
-        ProcessingElementTestUtils.testElement(processorTest);
-      });
-    });
-  }
-});
+	private HashAlgorithm hashAlgorithm;
+	
+	HashAlgorithmType(HashAlgorithm hashAlgorithm) {
+		this.hashAlgorithm = hashAlgorithm;
+	}
+	
+	public HashAlgorithm hashAlgorithm() {
+		return hashAlgorithm;
+	}
+}
diff --git a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/Md5HashAlgorithm.java
similarity index 56%
copy from ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
copy to streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/Md5HashAlgorithm.java
index 167d76786..96accec30 100644
--- a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/Md5HashAlgorithm.java
@@ -16,26 +16,17 @@
  *
  */
 
-import { ProcessingElementTestUtils } from '../../support/utils/ProcessingElementTestUtils';
-import { ProcessorTest } from '../../support/model/ProcessorTest';
+package org.apache.streampipes.processors.transformation.jvm.processor.hasher.algorithm;
 
-const allTests = Cypress.env('processingElements');
+import org.apache.commons.codec.digest.DigestUtils;
 
-allTests.forEach(test => {
-  const testNames = ['trigonometry2'];
+public class Md5HashAlgorithm implements HashAlgorithm {
 
-  const processorTest = test as ProcessorTest;
+	private static final long serialVersionUID = 1L;
 
-  if (testNames.includes(processorTest.name)) {
+	@Override
+	public String toHashValue(Object value) {
+		return DigestUtils.md5Hex(String.valueOf(value));
+	}
 
-    describe('Test Processor ' + test.dir, () => {
-      beforeEach('Setup Test', () => {
-        cy.initStreamPipesTest();
-      });
-
-      it('Initialize Test', () => {
-        ProcessingElementTestUtils.testElement(processorTest);
-      });
-    });
-  }
-});
+}
diff --git a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/Sha1HashAlgorithm.java
similarity index 56%
copy from ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
copy to streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/Sha1HashAlgorithm.java
index 167d76786..2eb3997ac 100644
--- a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/Sha1HashAlgorithm.java
@@ -16,26 +16,17 @@
  *
  */
 
-import { ProcessingElementTestUtils } from '../../support/utils/ProcessingElementTestUtils';
-import { ProcessorTest } from '../../support/model/ProcessorTest';
+package org.apache.streampipes.processors.transformation.jvm.processor.hasher.algorithm;
 
-const allTests = Cypress.env('processingElements');
+import org.apache.commons.codec.digest.DigestUtils;
 
-allTests.forEach(test => {
-  const testNames = ['trigonometry2'];
+public class Sha1HashAlgorithm implements HashAlgorithm {
 
-  const processorTest = test as ProcessorTest;
+	private static final long serialVersionUID = 1L;
 
-  if (testNames.includes(processorTest.name)) {
+	@Override
+	public String toHashValue(Object value) {
+		return DigestUtils.shaHex(String.valueOf(value));
+	}
 
-    describe('Test Processor ' + test.dir, () => {
-      beforeEach('Setup Test', () => {
-        cy.initStreamPipesTest();
-      });
-
-      it('Initialize Test', () => {
-        ProcessingElementTestUtils.testElement(processorTest);
-      });
-    });
-  }
-});
+}
diff --git a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/Sha2HashAlgorithm.java
similarity index 56%
copy from ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
copy to streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/Sha2HashAlgorithm.java
index 167d76786..7cfef6e5a 100644
--- a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/hasher/algorithm/Sha2HashAlgorithm.java
@@ -16,26 +16,17 @@
  *
  */
 
-import { ProcessingElementTestUtils } from '../../support/utils/ProcessingElementTestUtils';
-import { ProcessorTest } from '../../support/model/ProcessorTest';
+package org.apache.streampipes.processors.transformation.jvm.processor.hasher.algorithm;
 
-const allTests = Cypress.env('processingElements');
+import org.apache.commons.codec.digest.DigestUtils;
 
-allTests.forEach(test => {
-  const testNames = ['trigonometry2'];
+public class Sha2HashAlgorithm implements HashAlgorithm {
 
-  const processorTest = test as ProcessorTest;
+	private static final long serialVersionUID = 1L;
 
-  if (testNames.includes(processorTest.name)) {
+	@Override
+	public String toHashValue(Object value) {
+		return DigestUtils.sha256Hex(String.valueOf(value));
+	}
 
-    describe('Test Processor ' + test.dir, () => {
-      beforeEach('Setup Test', () => {
-        cy.initStreamPipesTest();
-      });
-
-      it('Initialize Test', () => {
-        ProcessingElementTestUtils.testElement(processorTest);
-      });
-    });
-  }
-});
+}
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/mapper/FieldMapperProcessor.java b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/mapper/FieldMapperProcessor.java
new file mode 100644
index 000000000..48c49c44d
--- /dev/null
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/mapper/FieldMapperProcessor.java
@@ -0,0 +1,117 @@
+/*
+ * 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.mapper;
+
+import org.apache.streampipes.commons.exceptions.SpRuntimeException;
+import org.apache.streampipes.container.api.ResolvesContainerProvidedOutputStrategy;
+import org.apache.streampipes.model.graph.DataProcessorDescription;
+import org.apache.streampipes.model.graph.DataProcessorInvocation;
+import org.apache.streampipes.model.runtime.Event;
+import org.apache.streampipes.model.schema.EventProperty;
+import org.apache.streampipes.model.schema.EventPropertyPrimitive;
+import org.apache.streampipes.model.schema.EventSchema;
+import org.apache.streampipes.model.schema.PropertyScope;
+import org.apache.streampipes.processors.transformation.jvm.processor.hasher.algorithm.HashAlgorithmType;
+import org.apache.streampipes.sdk.builder.PrimitivePropertyBuilder;
+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.helpers.EpRequirements;
+import org.apache.streampipes.sdk.helpers.Labels;
+import org.apache.streampipes.sdk.helpers.Locales;
+import org.apache.streampipes.sdk.helpers.OutputStrategies;
+import org.apache.streampipes.sdk.utils.Assets;
+import org.apache.streampipes.sdk.utils.Datatypes;
+import org.apache.streampipes.wrapper.context.EventProcessorRuntimeContext;
+import org.apache.streampipes.wrapper.routing.SpOutputCollector;
+import org.apache.streampipes.wrapper.standalone.ProcessorParams;
+import org.apache.streampipes.wrapper.standalone.StreamPipesDataProcessor;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class FieldMapperProcessor extends StreamPipesDataProcessor implements ResolvesContainerProvidedOutputStrategy<DataProcessorInvocation, ProcessingElementParameterExtractor> {
+
+  private static final String REPLACE_PROPERTIES = "replaceProperties";
+  private static final String FIELD_NAME = "fieldName";
+
+  private List<String> replacePropertyNames;
+  private String newFieldName;
+
+  @Override
+  public DataProcessorDescription declareModel() {
+    return ProcessingElementBuilder.create("org.apache.streampipes.processors.transformation.jvm.field-mapper")
+      .withLocales(Locales.EN)
+      .withAssets(Assets.DOCUMENTATION, Assets.ICON)
+      .requiredStream(StreamRequirementsBuilder
+        .create()
+        .requiredPropertyWithNaryMapping(EpRequirements.anyProperty(), Labels.withId
+          (REPLACE_PROPERTIES), PropertyScope.NONE)
+        .build())
+      .requiredTextParameter(Labels.withId(FIELD_NAME))
+      .outputStrategy(OutputStrategies.customTransformation())
+      .build();
+  }
+
+  @Override
+  public void onInvocation(ProcessorParams parameters,
+                           SpOutputCollector spOutputCollector,
+                           EventProcessorRuntimeContext runtimeContext) throws SpRuntimeException {
+    var extractor = parameters.extractor();
+    this.replacePropertyNames = extractor.mappingPropertyValues(REPLACE_PROPERTIES);
+    this.newFieldName = extractor.singleValueParameter(FIELD_NAME, String.class);
+  }
+
+  @Override
+  public void onEvent(Event in, SpOutputCollector out) throws SpRuntimeException {
+    Event event = new Event();
+    StringBuilder hashValue = new StringBuilder();
+    List<String> keys = in.getFields().keySet().stream().sorted().collect(Collectors.toList());
+    for (String key : keys) {
+      if (replacePropertyNames.stream().noneMatch(r -> r.equals(key))) {
+        event.addField(in.getFieldBySelector(key));
+      } else {
+        hashValue.append(in.getFieldBySelector((key)).getAsPrimitive().getAsString());
+      }
+    }
+
+    event.addField(newFieldName, HashAlgorithmType.MD5.hashAlgorithm().toHashValue(hashValue
+      .toString
+        ()));
+    out.collect(event);
+  }
+
+  @Override
+  public void onDetach() throws SpRuntimeException {
+
+  }
+
+  @Override
+  public EventSchema resolveOutputStrategy(DataProcessorInvocation processingElement,
+                                           ProcessingElementParameterExtractor extractor) throws SpRuntimeException {
+    List<String> replacePropertyNames = extractor.mappingPropertyValues(REPLACE_PROPERTIES);
+    String newFieldName = extractor.singleValueParameter(FIELD_NAME, String.class);
+
+    List<EventProperty> outProperties = extractor.getNoneInputStreamEventPropertySubset(replacePropertyNames);
+
+    EventPropertyPrimitive newProperty = PrimitivePropertyBuilder.create(Datatypes.String, newFieldName).build();
+    outProperties.add(newProperty);
+    return new EventSchema(outProperties);
+  }
+}
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/measurementconverter/MeasurementUnitConverterProcessor.java b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/measurementconverter/MeasurementUnitConverterProcessor.java
new file mode 100644
index 000000000..7090b6821
--- /dev/null
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/java/org/apache/streampipes/processors/transformation/jvm/processor/measurementconverter/MeasurementUnitConverterProcessor.java
@@ -0,0 +1,147 @@
+/*
+ * 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.measurementconverter;
+
+import com.github.jqudt.Quantity;
+import com.github.jqudt.Unit;
+import org.apache.streampipes.commons.exceptions.SpRuntimeException;
+import org.apache.streampipes.container.api.ResolvesContainerProvidedOptions;
+import org.apache.streampipes.model.graph.DataProcessorDescription;
+import org.apache.streampipes.model.runtime.Event;
+import org.apache.streampipes.model.schema.EventProperty;
+import org.apache.streampipes.model.schema.EventPropertyPrimitive;
+import org.apache.streampipes.model.schema.PropertyScope;
+import org.apache.streampipes.model.staticproperty.Option;
+import org.apache.streampipes.model.staticproperty.RuntimeResolvableOneOfStaticProperty;
+import org.apache.streampipes.sdk.builder.ProcessingElementBuilder;
+import org.apache.streampipes.sdk.builder.PropertyRequirementsBuilder;
+import org.apache.streampipes.sdk.builder.StreamRequirementsBuilder;
+import org.apache.streampipes.sdk.extractor.StaticPropertyExtractor;
+import org.apache.streampipes.sdk.helpers.Labels;
+import org.apache.streampipes.sdk.helpers.Locales;
+import org.apache.streampipes.sdk.helpers.OutputStrategies;
+import org.apache.streampipes.sdk.helpers.TransformOperations;
+import org.apache.streampipes.sdk.utils.Assets;
+import org.apache.streampipes.units.UnitProvider;
+import org.apache.streampipes.wrapper.context.EventProcessorRuntimeContext;
+import org.apache.streampipes.wrapper.routing.SpOutputCollector;
+import org.apache.streampipes.wrapper.standalone.ProcessorParams;
+import org.apache.streampipes.wrapper.standalone.StreamPipesDataProcessor;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class MeasurementUnitConverterProcessor extends StreamPipesDataProcessor implements ResolvesContainerProvidedOptions {
+
+  private static final String CONVERT_PROPERTY = "convert-property";
+  private static final String OUTPUT_UNIT = "output-unit";
+
+  private Unit inputUnit;
+  private Unit outputUnit;
+
+  private String convertProperty;
+
+  @Override
+  public DataProcessorDescription declareModel() {
+    return ProcessingElementBuilder.create("org.apache.streampipes.processors.transformation.jvm.measurementunitconverter")
+      .withLocales(Locales.EN)
+      .withAssets(Assets.DOCUMENTATION, Assets.ICON)
+      .requiredStream(StreamRequirementsBuilder
+        .create()
+        .requiredPropertyWithUnaryMapping(PropertyRequirementsBuilder
+            .create()
+            .measurementUnitPresence()
+            .build(),
+          Labels.withId(CONVERT_PROPERTY),
+          PropertyScope.MEASUREMENT_PROPERTY)
+        .build())
+      .requiredSingleValueSelectionFromContainer(Labels.withId(OUTPUT_UNIT))
+      .outputStrategy(OutputStrategies.transform(TransformOperations
+        .dynamicMeasurementUnitTransformation(CONVERT_PROPERTY, OUTPUT_UNIT)))
+      .build();
+  }
+
+  @Override
+  public void onInvocation(ProcessorParams parameters, SpOutputCollector spOutputCollector, EventProcessorRuntimeContext runtimeContext) throws SpRuntimeException {
+    var extractor = parameters.extractor();
+
+    this.convertProperty = extractor.mappingPropertyValue(CONVERT_PROPERTY);
+    String runtimeName = extractor.getEventPropertyBySelector(this.convertProperty).getRuntimeName();
+    String inputUnitId = extractor.measurementUnit(runtimeName, 0);
+    String outputUnitId = parameters.getGraph().getStaticProperties().stream().filter(sp -> sp
+        .getInternalName().equals(OUTPUT_UNIT))
+      .map(sp ->
+        (RuntimeResolvableOneOfStaticProperty) sp)
+      .findFirst()
+      .get().getOptions()
+      .stream
+        ().filter(Option::isSelected).map(Option::getInternalName).findFirst().get();
+
+    this.inputUnit = UnitProvider.INSTANCE.getUnit(inputUnitId);
+    this.outputUnit = UnitProvider.INSTANCE.getUnit(outputUnitId);
+  }
+
+  @Override
+  public void onEvent(Event in, SpOutputCollector out) throws SpRuntimeException {
+    double value = in.getFieldBySelector(convertProperty).getAsPrimitive().getAsDouble();
+
+    // transform old value to new unit
+    Quantity obs = new Quantity(value, inputUnit);
+    try {
+      Double newValue = obs.convertTo(outputUnit).getValue();
+      in.updateFieldBySelector(convertProperty, newValue);
+      out.collect(in);
+    } catch (IllegalAccessException e) {
+      throw new SpRuntimeException("Could not convert measurement", e);
+    }
+  }
+
+  @Override
+  public void onDetach() throws SpRuntimeException {
+
+  }
+
+  @Override
+  public List<Option> resolveOptions(String staticPropertyInternalName, StaticPropertyExtractor parameterExtractor) {
+    try {
+      String selector = parameterExtractor.mappingPropertyValue(CONVERT_PROPERTY);
+      EventProperty linkedEventProperty = parameterExtractor.getEventPropertyBySelector(selector);
+      if (linkedEventProperty instanceof EventPropertyPrimitive && ((EventPropertyPrimitive) linkedEventProperty)
+        .getMeasurementUnit() != null) {
+        Unit measurementUnit = UnitProvider.INSTANCE.getUnit(((EventPropertyPrimitive) linkedEventProperty)
+          .getMeasurementUnit().toString());
+        URI type = measurementUnit.getType();
+        List<Unit> availableUnits = UnitProvider.INSTANCE.getUnitsByType(type);
+        return availableUnits
+          .stream()
+          .filter(unit -> !(unit.getResource().toString().equals(measurementUnit.getResource().toString())))
+          .map(unit -> new Option(unit.getLabel(), unit.getResource().toString()))
+          .collect(Collectors.toList());
+      } else {
+        return new ArrayList<>();
+      }
+    } catch (SpRuntimeException e) {
+      e.printStackTrace();
+      return new ArrayList<>();
+    }
+  }
+}
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.field-mapper/documentation.md b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.field-mapper/documentation.md
new file mode 100644
index 000000000..8babeefe3
--- /dev/null
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.field-mapper/documentation.md
@@ -0,0 +1,63 @@
+<!--
+  ~ 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.
+  ~
+  -->
+
+## Field Mapper
+
+<p align="center"> 
+    <img src="icon.png" width="150px;" class="pe-image-documentation"/>
+</p>
+
+***
+
+## Description
+
+Replaces one or more fields with a new field and computes a hash value of these fields
+
+***
+
+## Configuration
+
+* Fields: Fields that will be mapped into a property
+* Name of the new field
+
+***
+
+## Example
+Merge two fields into a hash value
+### Input  event
+```
+{
+  "timestamp":1586380104915,
+  "mass_flow":4.3167,
+  "temperature":40.05,
+  "sensorId":"flowrate01"
+}
+```
+
+### Configuration
+* Fields: mass_flow, temperature
+* Name of new field: demo
+
+### Output event 
+```
+{
+  "timestamp":1586380104915,
+  "sensorId":"flowrate01"
+  "demo":"8ae11f5c83610104408d485b73120832",
+}
+```
\ No newline at end of file
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.field-mapper/icon.png b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.field-mapper/icon.png
new file mode 100644
index 000000000..2eafc5f45
Binary files /dev/null and b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.field-mapper/icon.png differ
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.field-mapper/strings.en b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.field-mapper/strings.en
new file mode 100644
index 000000000..cf570b572
--- /dev/null
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.field-mapper/strings.en
@@ -0,0 +1,8 @@
+org.apache.streampipes.processors.transformation.jvm.field-mapper.title=Field Mapper
+org.apache.streampipes.processors.transformation.jvm.field-mapper.description=Replaces one or more field with a new field and computes a hash value of these fields
+
+replaceProperties.title=Fields
+replaceProperties.description=The fields to replace
+
+fieldName.title=New Field Name
+fieldName.description=The name of the new field
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.fieldhasher/documentation.md b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.fieldhasher/documentation.md
new file mode 100644
index 000000000..4049b6da1
--- /dev/null
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.fieldhasher/documentation.md
@@ -0,0 +1,48 @@
+<!--
+  ~ 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.
+  ~
+  -->
+
+## Field Hasher
+
+<p align="center"> 
+    <img src="icon.png" width="150px;" class="pe-image-documentation"/>
+</p>
+
+***
+
+## Description
+
+The Field Hasher uses an algorithm to encode values in a field. The Field Hasher can use MD5, SHA1 or SHA2 to hash field values.
+
+***
+
+## Required input
+This processor requires at least one field of type string.
+
+***
+
+## Configuration
+
+### Field
+Specifies the string field that will be encoded.
+
+### Hash Algorithm
+Specifies the algorithm used to encode the string field. The following algorithms
+are available: SHA2, MD5 or SHA1.
+
+## Output
+The encoded string field.
\ No newline at end of file
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.fieldhasher/icon.png b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.fieldhasher/icon.png
new file mode 100644
index 000000000..f9fa3a2b3
Binary files /dev/null and b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.fieldhasher/icon.png differ
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.fieldhasher/strings.en b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.fieldhasher/strings.en
new file mode 100644
index 000000000..561f434cc
--- /dev/null
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.fieldhasher/strings.en
@@ -0,0 +1,8 @@
+org.apache.streampipes.processors.transformation.jvm.fieldhasher.title=Field Hasher
+org.apache.streampipes.processors.transformation.jvm.fieldhasher.description=Hashes the value of a field using various hash functions.
+
+property-mapping.title=Field
+property-mapping.description=The field the hash function should be applied on
+
+hash-algorithm.title=Hash Algorithm
+hash-algorithm.description=The hash algorithm that should be used.
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.measurementunitconverter/documentation.md b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.measurementunitconverter/documentation.md
new file mode 100644
index 000000000..f9b8e8889
--- /dev/null
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.measurementunitconverter/documentation.md
@@ -0,0 +1,47 @@
+<!--
+  ~ 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.
+  ~
+  -->
+
+## Measurement Unit Converter
+
+<p align="center"> 
+    <img src="icon.png" width="150px;" class="pe-image-documentation"/>
+</p>
+
+***
+
+## Description
+
+Converts a unit of measurement to another one.
+
+***
+
+## Required input
+
+
+***
+
+## Configuration
+
+Describe the configuration parameters here
+
+### 1st parameter
+
+
+### 2nd parameter
+
+## Output
\ No newline at end of file
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.measurementunitconverter/icon.png b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.measurementunitconverter/icon.png
new file mode 100644
index 000000000..ebe9f3a3e
Binary files /dev/null and b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.measurementunitconverter/icon.png differ
diff --git a/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.measurementunitconverter/strings.en b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.measurementunitconverter/strings.en
new file mode 100644
index 000000000..754f04b67
--- /dev/null
+++ b/streampipes-extensions/streampipes-processors-transformation-jvm/src/main/resources/org.apache.streampipes.processors.transformation.jvm.measurementunitconverter/strings.en
@@ -0,0 +1,8 @@
+org.apache.streampipes.processors.transformation.jvm.measurementunitconverter.title=Measurement Unit Converter
+org.apache.streampipes.processors.transformation.jvm.measurementunitconverter.description=Converts a unit of measurement to another one
+
+convert-property.title=Field
+convert-property.description=The field to convert
+
+output-unit.title=Output Unit
+output-unit.description=The output type unit of measurement
diff --git a/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher1/description.json b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher1/description.json
new file mode 100644
index 000000000..147ee238b
--- /dev/null
+++ b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher1/description.json
@@ -0,0 +1,7 @@
+{
+ "name": "field_hasher",
+ "config": [
+   {"type":  "drop-down", "selector": "property-mapping", "value":  "sensorId"},
+   {"type":  "radio", "selector": "hash-algorithm", "value":  "SHA1"}
+ ]
+}
diff --git a/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher1/expected.csv b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher1/expected.csv
new file mode 100644
index 000000000..27830ff66
--- /dev/null
+++ b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher1/expected.csv
@@ -0,0 +1,3 @@
+time;sensorId
+1623871499055;a9993e364706816aba3e25717850c26c9cd0d89d
+1623871500059;40bd001563085fc35165329ea1ff5c5ecbdbbeef
diff --git a/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher1/input.csv b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher1/input.csv
new file mode 100644
index 000000000..bcbd5414d
--- /dev/null
+++ b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher1/input.csv
@@ -0,0 +1,3 @@
+timestamp;sensorId
+1623871499055;abc
+1623871500059;123
diff --git a/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher2/description.json b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher2/description.json
new file mode 100644
index 000000000..75aff6957
--- /dev/null
+++ b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher2/description.json
@@ -0,0 +1,7 @@
+{
+ "name": "field_hasher",
+ "config": [
+   {"type":  "drop-down", "selector": "property-mapping", "value":  "sensorId"},
+   {"type":  "radio", "selector": "hash-algorithm", "value":  "MD5"}
+ ]
+}
diff --git a/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher2/expected.csv b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher2/expected.csv
new file mode 100644
index 000000000..8f61632da
--- /dev/null
+++ b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher2/expected.csv
@@ -0,0 +1,3 @@
+time;sensorId
+1623871499055;900150983cd24fb0d6963f7d28e17f72
+1623871500059;202cb962ac59075b964b07152d234b70
diff --git a/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher2/input.csv b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher2/input.csv
new file mode 100644
index 000000000..3f2fd7638
--- /dev/null
+++ b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldHasher2/input.csv
@@ -0,0 +1,5 @@
+timestamp;sensorId
+1623871499055;abc
+1623871500059;123
+
+
diff --git a/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldMapper1/description.json b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldMapper1/description.json
new file mode 100644
index 000000000..40a5298a9
--- /dev/null
+++ b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldMapper1/description.json
@@ -0,0 +1,8 @@
+{
+ "name": "field_mapper",
+ "config": [
+   {"type":  "checkbox", "selector": "value1", "value":  "check"},
+   {"type":  "checkbox", "selector": "value2", "value":  "check"},
+   {"type":  "input", "selector": "fieldName", "value":  "hashedField"}
+ ]
+}
diff --git a/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldMapper1/expected.csv b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldMapper1/expected.csv
new file mode 100644
index 000000000..cc037c0d1
--- /dev/null
+++ b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldMapper1/expected.csv
@@ -0,0 +1,3 @@
+time;hashedField
+1623871499055;e80b5017098950fc58aad83c8c14978e
+1623871500059;e10adc3949ba59abbe56e057f20f883e
diff --git a/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldMapper1/input.csv b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldMapper1/input.csv
new file mode 100644
index 000000000..38082515b
--- /dev/null
+++ b/ui/cypress/fixtures/pipelineElement/transformation-jvm/fieldMapper1/input.csv
@@ -0,0 +1,5 @@
+timestamp;value1;value2
+1623871499055;abc;def
+1623871500059;123;456
+
+
diff --git a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts b/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
index 167d76786..e3053a68f 100644
--- a/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
+++ b/ui/cypress/tests/pipelineElement/SinglePipelineElement.ts
@@ -22,7 +22,7 @@ import { ProcessorTest } from '../../support/model/ProcessorTest';
 const allTests = Cypress.env('processingElements');
 
 allTests.forEach(test => {
-  const testNames = ['trigonometry2'];
+  const testNames = ['fieldHasher1'];
 
   const processorTest = test as ProcessorTest;