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/18 20:33:06 UTC

[incubator-streampipes] branch rel/0.70.0 updated (1bddfc66f -> fbe88ba3f)

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

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


    from 1bddfc66f [maven-release-plugin] prepare branch @{releaseLabel}
     new 5dd99908f [hotfix] Improve x axis representation of line chart
     new 7a1f96bfb [STREAMPIPES-577] Improve error handling in StreamPipes Connect
     new d995362d3 [STREAMPIPES-577] Show live preview for OPC-UA adapter
     new 95e40b02d [STREAMPIPES-577] Add check and warning message for bad readings
     new 8ee998066 [STREAMPIPES-577] Add event preview to schema editor
     new dcdbe8abb [STREAMPIPES-577] Let JSON parser support adapter preview
     new 0ce9fa824 [STREAMPIPES-577] More adapters support preview, improve preview appearance
     new df0984812 [STREAMPIPES-577] Improve error handling of adapters
     new f5b30b5e3 [STREAMPIPES-577] Let PLC adapter support preview
     new 370962efb [STREAMPIPES-577] Add missing header
     new aea54f13d [STREAMPIPES-577] Improve parsing and preview of datatypes
     new bb121bbcb [STREAMPIPES-577] Fix wrong assignment of data type
     new 9d0feb50e [hotfix] Remove comment
     new aa10f7f67 [hotfix] Add missing header
     new fbe88ba3f [STREAMPIPES-577] Improve handling of unknown data types

The 15 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 ...xception.java => SpConfigurationException.java} |  10 +-
 .../apache/streampipes/connect/api/IParser.java    |   9 +
 .../connect/api/exception/ParseException.java      |   6 +
 .../master/management/GuessManagement.java         |  19 +-
 .../master/management/WorkerRestClient.java        |  23 +-
 .../worker/management/GuessManagement.java         |   3 +-
 .../container/worker/rest/GuessResource.java       |  10 +-
 .../worker/rest/RuntimeResolvableResource.java     |  26 +-
 .../streampipes/connect/adapter/Adapter.java       | 112 +------
 .../connect/adapter/AdapterPipelineGenerator.java  | 150 ++++++++++
 .../connect/adapter/format/csv/CsvParser.java      |  39 +--
 .../adapter/format/json/AbstractJsonFormat.java    |   3 -
 .../format/json/object/JsonObjectParser.java       | 135 ++-------
 .../adapter/format/util/JsonEventProperty.java     |   2 +-
 .../connect/adapter/guess/SchemaGuesser.java       |  10 +-
 .../pipeline/AdapterEventPreviewPipeline.java      |  81 +++++
 .../TransformValueAdapterPipelineElement.java      |  14 +-
 .../value/DatatypeTransformationRule.java          |  52 ++++
 .../transform/value/ValueEventTransformer.java     |  46 +--
 .../connect/adapter/util/DatatypeUtils.java        | 118 ++++++++
 .../connect/adapter/util/PollingSettings.java      |   1 +
 .../api/InvocablePipelineElementResource.java      |  27 +-
 .../api/ResolvesContainerProvidedOptions.java      |   3 +-
 .../ResolvesContainerProvidedOutputStrategy.java   |   4 +-
 .../api/RuntimeResolvableRequestHandler.java       |   5 +-
 .../container/api/SupportsRuntimeConfig.java       |   3 +-
 .../dataexplorer/commons/influx/InfluxStore.java   |  12 +-
 .../opcua/MiloOpcUaConfigurationProvider.java      |   8 +-
 .../connect/iiot/adapters/opcua/OpcUaAdapter.java  |  26 +-
 .../iiot/adapters/opcua/OpcUaNodeBrowser.java      |  10 +
 .../connect/iiot/adapters/opcua/SpOpcUaClient.java |   8 +-
 .../opcua/utils/ExceptionMessageExtractor.java}    |  24 +-
 .../iiot/adapters/opcua/utils/OpcUaUtil.java       | 327 ++++++++++++---------
 .../iiot/adapters/plc4x/s7/Plc4xS7Adapter.java     | 148 ++++++----
 .../adapters/plc4x/s7/PlcReadResponseHandler.java  |  11 +-
 .../machine/MachineDataSimulatorUtils.java         | 240 ++++++++-------
 .../connect/iiot/protocol/set/FileProtocol.java    |  11 +-
 .../connect/iiot/protocol/set/HttpProtocol.java    |   2 +-
 .../iiot/protocol/stream/BrokerProtocol.java       |   8 +-
 .../iiot/protocol/stream/FileStreamProtocol.java   |  11 +-
 .../iiot/protocol/stream/HttpStreamProtocol.java   |   2 +-
 .../iiot/protocol/stream/KafkaProtocol.java        |  41 ++-
 .../apache/streampipes/model/ErrorMessageLd.java   |  40 ---
 .../org/apache/streampipes/model/MessageLd.java    | 117 --------
 .../apache/streampipes/model/NotificationLd.java   |  95 ------
 .../streampipes/model/StreamPipesErrorMessage.java | 103 +++++++
 .../{GuessSchema.java => AdapterEventPreview.java} |  37 +--
 .../guess/AdapterGuessInfo.java}                   |  35 ++-
 .../model/connect/guess/FieldStatus.java           |   8 +-
 .../model/connect/guess/FieldStatusInfo.java       |  69 +++++
 .../model/connect/guess/GuessSchema.java           |  30 ++
 .../model/connect/guess/GuessTypeInfo.java         |  36 +--
 .../rules/TransformationRuleDescription.java       |   1 +
 ...angeDatatypeTransformationRuleDescription.java} |  34 ++-
 .../rest/impl/connect/GuessResource.java           |  25 +-
 .../impl/connect/RuntimeResolvableResource.java    |  19 +-
 .../sdk/builder/adapter/GuessSchemaBuilder.java    |  14 +-
 .../storage/couchdb/serializer/GsonSerializer.java |   1 +
 ui/package.json                                    |   4 +-
 .../src/lib/model/gen/streampipes-model.ts         | 123 +++++++-
 ui/projects/streampipes/shared-ui/package.json     |   1 +
 .../exception-details-dialog.component.html}       |  22 +-
 .../exception-details-dialog.component.scss}       |  24 +-
 .../exception-details-dialog.component.ts}         |  22 +-
 .../sp-exception-message.component.html}           |  31 +-
 .../sp-exception-message.component.scss}           |  12 +-
 .../sp-exception-message.component.ts}             |  39 +--
 .../shared-ui/src/lib/shared-ui.module.ts          |  10 +-
 .../streampipes/shared-ui/src/public-api.ts        |   2 +
 .../new-adapter/new-adapter.component.ts           |   1 -
 .../error-message/error-message.component.html     |  18 +-
 .../error-message/error-message.component.ts       |   3 +-
 .../event-property-row.component.html              | 110 ++++---
 .../event-property-row.component.scss              |  66 ++++-
 .../event-property-row.component.ts                |  49 ++-
 .../event-schema-preview.component.html            |  24 +-
 .../event-schema-preview.component.scss            |  11 +
 .../event-schema-preview.component.ts              |  51 +++-
 .../event-schema/event-schema.component.html       |  49 +--
 .../event-schema/event-schema.component.scss       |  11 +-
 .../event-schema/event-schema.component.ts         |  48 ++-
 .../schema-editor-header.component.ts              |   6 +-
 ui/src/app/connect/connect.module.ts               |   5 +-
 .../filter/json-pretty-print.pipe.ts}              |  18 +-
 ui/src/app/connect/services/rest.service.ts        |  18 +-
 .../services/transformation-rule.service.ts        |  70 +++--
 ui/src/app/core-model/base/UserErrorMessage.ts     |   4 +-
 .../static-property.component.html                 |   2 +
 ...tatic-runtime-resolvable-any-input.component.ts |   3 +
 .../base-runtime-resolvable-input.ts               |  13 +
 ...c-runtime-resolvable-oneof-input.component.html |  24 +-
 ...tic-runtime-resolvable-oneof-input.component.ts |  18 ++
 .../static-tree-input.component.html               |   3 +
 .../static-tree-input.component.ts                 |  31 ++
 .../widgets/base/base-ngx-line-charts-widget.ts    |   3 +-
 95 files changed, 2127 insertions(+), 1286 deletions(-)
 copy streampipes-commons/src/main/java/org/apache/streampipes/commons/exceptions/{SpException.java => SpConfigurationException.java} (84%)
 create mode 100644 streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java
 create mode 100644 streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/model/pipeline/AdapterEventPreviewPipeline.java
 create mode 100644 streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/transform/value/DatatypeTransformationRule.java
 create mode 100644 streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/DatatypeUtils.java
 copy streampipes-extensions/{streampipes-processors-pattern-detection-flink/src/main/java/org/apache/streampipes/processors/pattern/detection/flink/processor/and/TimeUnitConverter.java => streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/ExceptionMessageExtractor.java} (64%)
 copy streampipes-client/src/main/java/org/apache/streampipes/client/credentials/CredentialsProvider.java => streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/PlcReadResponseHandler.java (76%)
 delete mode 100644 streampipes-model/src/main/java/org/apache/streampipes/model/ErrorMessageLd.java
 delete mode 100644 streampipes-model/src/main/java/org/apache/streampipes/model/MessageLd.java
 delete mode 100644 streampipes-model/src/main/java/org/apache/streampipes/model/NotificationLd.java
 create mode 100644 streampipes-model/src/main/java/org/apache/streampipes/model/StreamPipesErrorMessage.java
 copy streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/{GuessSchema.java => AdapterEventPreview.java} (59%)
 copy streampipes-model/src/main/java/org/apache/streampipes/model/{runtime/SchemaInfo.java => connect/guess/AdapterGuessInfo.java} (59%)
 copy streampipes-commons/src/main/java/org/apache/streampipes/commons/exceptions/NoValidSepaStructureException.java => streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/FieldStatus.java (88%)
 create mode 100644 streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/FieldStatusInfo.java
 copy streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/PollingSettings.java => streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/GuessTypeInfo.java (60%)
 copy streampipes-model/src/main/java/org/apache/streampipes/model/connect/rules/{schema/CreateNestedRuleDescription.java => value/ChangeDatatypeTransformationRuleDescription.java} (53%)
 copy ui/{src/app/editor/dialog/missing-elements-for-tutorial/missing-elements-for-tutorial.component.html => projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.html} (62%)
 copy ui/{src/app/pipelines/dialog/pipeline-notifications/pipeline-notifications.component.scss => projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.scss} (75%)
 copy ui/{src/app/editor/dialog/matching-error/matching-error.component.ts => projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.ts} (63%)
 copy ui/{src/app/editor/dialog/missing-elements-for-tutorial/missing-elements-for-tutorial.component.html => projects/streampipes/shared-ui/src/lib/components/sp-exception-message/sp-exception-message.component.html} (53%)
 copy ui/{src/app/connect/components/runtime-info/pipeline-element-runtime-info.component.scss => projects/streampipes/shared-ui/src/lib/components/sp-exception-message/sp-exception-message.component.scss} (86%)
 copy ui/projects/streampipes/shared-ui/src/lib/components/{basic-nav-tabs/basic-nav-tabs.component.ts => sp-exception-message/sp-exception-message.component.ts} (50%)
 copy ui/src/app/{data-explorer/components/widgets/utils/load-data-spinner/load-data-spinner.component.ts => connect/filter/json-pretty-print.pipe.ts} (72%)


[incubator-streampipes] 07/15: [STREAMPIPES-577] More adapters support preview, improve preview appearance

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 0ce9fa82437c8f466a1e3278b8d66d4d5511e430
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Wed Aug 17 09:38:12 2022 +0200

    [STREAMPIPES-577] More adapters support preview, improve preview appearance
---
 .../connect/adapter/format/csv/CsvParser.java      | 40 ++++++++++++++++++----
 .../adapter/format/json/AbstractJsonFormat.java    |  3 --
 .../connect/iiot/protocol/set/FileProtocol.java    | 11 +++---
 .../iiot/protocol/stream/FileStreamProtocol.java   | 11 +++---
 .../event-property-row.component.html              | 10 +++---
 .../event-property-row.component.scss              | 12 +++++--
 .../event-schema-preview.component.ts              | 10 +++++-
 7 files changed, 71 insertions(+), 26 deletions(-)

diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java
index 4f6b87323..a5e4f3841 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java
@@ -19,11 +19,13 @@
 package org.apache.streampipes.connect.adapter.format.csv;
 
 
-import org.apache.streampipes.connect.api.EmitBinaryEvent;
 import org.apache.streampipes.connect.adapter.model.generic.Parser;
 import org.apache.streampipes.connect.adapter.sdk.ParameterExtractor;
+import org.apache.streampipes.connect.api.EmitBinaryEvent;
 import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.model.connect.grounding.FormatDescription;
+import org.apache.streampipes.model.connect.guess.AdapterGuessInfo;
+import org.apache.streampipes.model.connect.guess.GuessTypeInfo;
 import org.apache.streampipes.model.schema.EventPropertyPrimitive;
 import org.apache.streampipes.model.schema.EventSchema;
 import org.apache.streampipes.vocabulary.XSD;
@@ -33,6 +35,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 
 
@@ -81,7 +84,13 @@ public class CsvParser extends Parser {
     }
 
     @Override
-    public EventSchema getEventSchema(List<byte[]> oneEvent) {
+    public boolean supportsPreview() {
+        return true;
+    }
+
+    @Override
+    public AdapterGuessInfo getSchemaAndSample(List<byte[]> oneEvent) {
+        var sample = new HashMap<String, GuessTypeInfo>();
         String[] keys;
         String[] data;
 
@@ -101,26 +110,45 @@ public class CsvParser extends Parser {
             EventPropertyPrimitive p = new EventPropertyPrimitive();
             p.setRuntimeName(keys[i]);
             p.setRuntimeType(getTypeString(data[i]));
+            sample.put(keys[i], new GuessTypeInfo(getTypeString(data[i]), data[i]));
             resultSchema.addEventProperty(p);
         }
 
-        return resultSchema;
+        return new AdapterGuessInfo(resultSchema, sample);
+    }
+
+    @Override
+    public EventSchema getEventSchema(List<byte[]> oneEvent) {
+        return getSchemaAndSample(oneEvent).getEventSchema();
     }
 
     private String getTypeString(String o) {
 
+        String typeClass = getTypeClass(o);
+
+        if (Float.class.getCanonicalName().equals(typeClass)) {
+            return XSD._float.toString();
+        } else if (Boolean.class.getCanonicalName().equals(typeClass)) {
+            return XSD._boolean.toString();
+        } else {
+            return XSD._string.toString();
+        }
+    }
+
+    private String getTypeClass(String o) {
+
         try {
             Double.parseDouble(o);
-            return XSD._float.toString();
+            return Float.class.getCanonicalName();
         } catch (NumberFormatException e) {
 
         }
 
         if (o.equalsIgnoreCase("true") || o.equalsIgnoreCase("false")) {
-            return XSD._boolean.toString();
+            return Boolean.class.getCanonicalName();
         }
 
-        return XSD._string.toString();
+        return String.class.getCanonicalName();
     }
 
 
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/json/AbstractJsonFormat.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/json/AbstractJsonFormat.java
index a7116aecd..1da9b1696 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/json/AbstractJsonFormat.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/json/AbstractJsonFormat.java
@@ -21,7 +21,6 @@ import org.apache.streampipes.commons.exceptions.SpRuntimeException;
 import org.apache.streampipes.connect.api.IFormat;
 import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.dataformat.json.JsonDataFormatDefinition;
-import org.apache.streampipes.model.schema.EventSchema;
 
 import java.util.Map;
 
@@ -30,8 +29,6 @@ public abstract class AbstractJsonFormat implements IFormat {
 
   @Override
   public Map<String, Object> parse(byte[] object) throws ParseException {
-    EventSchema resultSchema = new EventSchema();
-
     JsonDataFormatDefinition jsonDefinition = new JsonDataFormatDefinition();
 
     Map<String, Object> result = null;
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/FileProtocol.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/FileProtocol.java
index 31d4fc5bf..8dcb342c4 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/FileProtocol.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/FileProtocol.java
@@ -121,11 +121,12 @@ public class FileProtocol extends Protocol {
             InputStream targetStream = FileProtocolUtils.getFileInputStream(this.selectedFilename);
             List<byte[]> dataByte = parser.parseNEvents(targetStream, 20);
 
-            EventSchema eventSchema = parser.getEventSchema(dataByte);
-
-            GuessSchema result = SchemaGuesser.guessSchema(eventSchema);
-
-            return result;
+            if (parser.supportsPreview()) {
+                return SchemaGuesser.guessSchema(parser.getSchemaAndSample(dataByte));
+            } else {
+                EventSchema eventSchema = parser.getEventSchema(dataByte);
+                return SchemaGuesser.guessSchema(eventSchema);
+            }
         } catch (FileNotFoundException e) {
             throw new ParseException("Could not read local file");
         }
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/FileStreamProtocol.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/FileStreamProtocol.java
index d82d28ce9..f88f5c538 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/FileStreamProtocol.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/FileStreamProtocol.java
@@ -211,11 +211,12 @@ public class FileStreamProtocol extends Protocol {
 
     List<byte[]> dataByte = parser.parseNEvents(dataInputStream, 2);
 
-    EventSchema eventSchema = parser.getEventSchema(dataByte);
-
-    GuessSchema result = SchemaGuesser.guessSchema(eventSchema);
-
-    return result;
+    if (parser.supportsPreview()) {
+      return SchemaGuesser.guessSchema(parser.getSchemaAndSample(dataByte));
+    } else {
+      EventSchema eventSchema = parser.getEventSchema(dataByte);
+      return SchemaGuesser.guessSchema(eventSchema);
+    }
   }
 
   @Override
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html
index 88a49c3e4..cf07a44ba 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html
@@ -43,12 +43,14 @@
             </mat-icon>
             <span style="margin-left: -5px;">marked as timestamp</span>
         </div>
-        <div fxLayoutAlign="end center" *ngIf="runtimeType" class="runtime-type-info-outer">
+        <div fxLayoutAlign="end center" class="runtime-type-info-outer">
+            <div fxLayout="row" fxLayoutAlign="start center" fxFlex="100" *ngIf="runtimeType" >
             <span class="runtime-info runtime-type-info">{{originalRuntimeType}}</span>
-            <span fxLayoutAlign="center center"><i class="material-icons">arrow_right</i></span>
-            <span class="runtime-info runtime-type-info">{{runtimeType}}</span>
+            <span fxLayoutAlign="center center" *ngIf="originalRuntimeType !== runtimeType"><i class="material-icons">arrow_right</i></span>
+            <span class="runtime-info runtime-type-info" *ngIf="originalRuntimeType !== runtimeType">{{runtimeType}}</span>
+            </div>
         </div>
-        <div fxLayoutAlign="end center">
+        <div fxLayoutAlign="end center" class="status-outer">
         <span *ngIf="showFieldStatus"
               [ngClass]="'status status-' +fieldStatusInfo[originalRuntimeName].fieldStatus.toLowerCase()"
               [matTooltip]="fieldStatusInfo[originalRuntimeName].additionalInfo">{{fieldStatusInfo[originalRuntimeName].fieldStatus}}</span>
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
index 9e4191930..ec2466197 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
@@ -60,6 +60,11 @@
   border-radius: 5px;
 }
 
+.status-outer {
+  width: 70px;
+  min-width: 70px;
+}
+
 .runtime-info {
   border-radius: 5px;
   color: var(--color-default-text);
@@ -70,15 +75,18 @@
 .runtime-type-info {
   border-radius: 5px;
   color: var(--color-default-text);
-  width: 60px;
   font-size: 10pt;
+  width: 70px;
+  border: 1px solid var(--color-bg-3);
   text-align: center;
 }
 
 .runtime-type-info-outer {
   padding: 3px;
   border-radius: 5px;
-  background: var(--color-bg-3);
+  width: 170px;
+  text-align: center;
+
 }
 
 
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.ts b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.ts
index 11823a586..d7dc13cc1 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.ts
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.ts
@@ -43,11 +43,19 @@ export class EventSchemaPreviewComponent implements OnInit {
   }
 
   toSimpleMap(event: Record<string, GuessTypeInfo>): Record<string, any> {
-    const result = {};
+    let result: Record<string, any> = {};
+
     for (const key in event) {
       result[key] = event[key].value;
     }
 
+   result = Object.keys(result).sort().reduce(
+      (obj, key) => {
+        obj[key] = result[key];
+        return obj;
+      },
+      {}
+    );
 
     return result;
   }


[incubator-streampipes] 04/15: [STREAMPIPES-577] Add check and warning message for bad readings

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 95e40b02d8900cb43f6746b8fdfa0dce25693108
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Tue Aug 16 15:03:10 2022 +0200

    [STREAMPIPES-577] Add check and warning message for bad readings
---
 .../event-schema/event-schema.component.html       | 22 ++++++++++++----------
 .../event-schema/event-schema.component.scss       | 11 ++++++++++-
 .../event-schema/event-schema.component.ts         |  5 +++++
 ui/src/app/core-model/base/UserErrorMessage.ts     |  4 +++-
 4 files changed, 30 insertions(+), 12 deletions(-)

diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
index 8344671e5..f7474a0e7 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
@@ -22,24 +22,26 @@
         <div fxLayout="column"
              fxFlex="100"
              fxLayoutAlign="start center"
-             [ngClass]="schemaErrorHints.length === 0 ? 'schema-validation schema-validation-ok' : 'schema-validation schema-validation-error'" *ngIf="!isLoading && !isError">
+            *ngIf="!isLoading && !isError && schemaErrorHints.length === 0" class="schema-validation schema-validation-ok">
             <div fxFlex="100"
                  fxLayout="row"
                  fxLayoutAlign="start center"
-                 class="schema-validation-text-ok"
-                 *ngIf="schemaErrorHints.length === 0">
+                 class="schema-validation-text-ok">
                 <i class="material-icons">check_circle</i>&nbsp;<b>Schema ok</b>
             </div>
+        </div>
+
+        <div fxLayout="column" fxFlex="100" *ngIf="!isLoading && schemaErrorHints.length > 0">
             <div fxFlex="100"
                  fxLayout="column"
-                 fxLayoutAlign="start center"
-                 *ngIf="schemaErrorHints.length > 0">
-                <div fxFlex="100" fxLayout="row" fxLayoutAlign="center start" class="schema-validation-text-error">
-                <i class="material-icons">warning</i>&nbsp;
-                    <b>{{schemaErrorHints[0].title}}</b>
+                 [ngClass]="schemaErrorHint.level === 'error' ? 'schema-validation schema-validation-error' : 'schema-validation schema-validation-warning'"
+                 fxLayoutAlign="start center" *ngFor="let schemaErrorHint of schemaErrorHints">
+                <div fxFlex="100" fxLayout="row" fxLayoutAlign="center start" [ngClass]="schemaErrorHint.level === 'error' ? 'schema-validation-text-error' : 'schema-validation-text-warning'">
+                    <i class="material-icons">warning</i>&nbsp;
+                    <b>{{schemaErrorHint.title}}</b>
                 </div>
-                <div *ngFor="let errorMessage of schemaErrorHints">
-                    <span>{{errorMessage.content}}</span>
+                <div>
+                    <span>{{schemaErrorHint.content}}</span>
                 </div>
             </div>
         </div>
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.scss b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.scss
index 741fc7628..228a3ae77 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.scss
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.scss
@@ -179,8 +179,9 @@ opacity:0.5;
 .schema-validation {
   margin-bottom: 10px;
   border: 2px solid var(--color-bg-3);
-  padding: 8px;
+  padding: 10px;
   min-height: 50px;
+  text-align: center;
 }
 
 .schema-validation-ok {
@@ -191,6 +192,10 @@ opacity:0.5;
   border: 2px solid #963e3e;
 }
 
+.schema-validation-warning {
+  border: 2px solid #d09836;
+}
+
 .schema-validation-text-ok {
   color: #629f62;
 }
@@ -198,3 +203,7 @@ opacity:0.5;
 .schema-validation-text-error {
   color: #963e3e;
 }
+
+.schema-validation-text-warning {
+  color: #d09836;
+}
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
index d70a53048..d6e19e214 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
@@ -238,6 +238,11 @@ export class EventSchemaComponent implements OnChanges {
       this.schemaErrorHints.push(new UserErrorMessage('Missing Timestamp', 'The timestamp must be a UNIX timestamp in milliseconds. Edit the timestamp field or add an ingestion timestamp.'));
     }
 
+    const badFields = eventSchema.eventProperties.map(ep => this.fieldStatusInfo[ep.runtimeName]).find(field => field.fieldStatus !== 'GOOD');
+    if (badFields !== undefined) {
+      this.schemaErrorHints.push(new UserErrorMessage('Bad reading', 'At least one field could not be properly read. If this is a permanent problem, consider removing it - keeping this field might cause the adapter to fail or to omit sending events.', 'warning'));
+    }
+
     return hasTimestamp;
   }
 }
diff --git a/ui/src/app/core-model/base/UserErrorMessage.ts b/ui/src/app/core-model/base/UserErrorMessage.ts
index 85322e0cb..2fc7f5c72 100644
--- a/ui/src/app/core-model/base/UserErrorMessage.ts
+++ b/ui/src/app/core-model/base/UserErrorMessage.ts
@@ -19,9 +19,11 @@
 export class UserErrorMessage {
   public title: string;
   public content: string;
+  public level: string;
 
-  constructor(title: string, content: string) {
+  constructor(title: string, content: string, level = 'error') {
     this.title = title;
     this.content = content;
+    this.level = level;
   }
 }


[incubator-streampipes] 12/15: [STREAMPIPES-577] Fix wrong assignment of data type

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit bb121bbcb7f2e3347f48a248a7893271263c3da4
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Wed Aug 17 17:16:01 2022 +0200

    [STREAMPIPES-577] Fix wrong assignment of data type
---
 .../streampipes/connect/adapter/format/util/JsonEventProperty.java  | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/util/JsonEventProperty.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/util/JsonEventProperty.java
index 672163863..b42711bc4 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/util/JsonEventProperty.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/util/JsonEventProperty.java
@@ -55,7 +55,11 @@ public class JsonEventProperty {
             resultProperty = new EventPropertyPrimitive();
             resultProperty.setRuntimeName(key);
             ((EventPropertyPrimitive) resultProperty).setRuntimeType(XSD._string.toString());
-        } else if (o.getClass().equals(Integer.class) || o.getClass().equals(Double.class)|| o.getClass().equals(Long.class)) {
+        } else if (o.getClass().equals(Long.class)) {
+            resultProperty = new EventPropertyPrimitive();
+            resultProperty.setRuntimeName(key);
+            ((EventPropertyPrimitive) resultProperty).setRuntimeType(XSD._long.toString());
+        } else if (o.getClass().equals(Integer.class) || o.getClass().equals(Double.class) || o.getClass().equals(Float.class)) {
             resultProperty = new EventPropertyPrimitive();
             resultProperty.setRuntimeName(key);
             ((EventPropertyPrimitive) resultProperty).setRuntimeType(XSD._float.toString());


[incubator-streampipes] 11/15: [STREAMPIPES-577] Improve parsing and preview of datatypes

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit aea54f13d641c60d897e57466ff8f90e085eed57
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Wed Aug 17 17:03:18 2022 +0200

    [STREAMPIPES-577] Improve parsing and preview of datatypes
---
 .../connect/adapter/AdapterPipelineGenerator.java  |  22 ++---
 .../connect/adapter/format/csv/CsvParser.java      |  37 +------
 .../TransformValueAdapterPipelineElement.java      |  14 ++-
 .../value/DatatypeTransformationRule.java          |  52 ++++++++++
 .../transform/value/ValueEventTransformer.java     |  46 +++------
 .../connect/adapter/util/DatatypeUtils.java        | 108 +++++++++++++++++++++
 .../connect/adapter/util/PollingSettings.java      |   1 +
 .../rules/TransformationRuleDescription.java       |   1 +
 ...hangeDatatypeTransformationRuleDescription.java |  60 ++++++++++++
 .../src/lib/model/gen/streampipes-model.ts         |  39 ++++++--
 .../services/transformation-rule.service.ts        |  70 ++++++++-----
 11 files changed, 328 insertions(+), 122 deletions(-)

diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java
index 31ce0e9a1..dfe84a105 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java
@@ -19,7 +19,6 @@
 package org.apache.streampipes.connect.adapter;
 
 import org.apache.streampipes.config.backend.BackendConfig;
-import org.apache.streampipes.config.backend.SpProtocol;
 import org.apache.streampipes.connect.adapter.model.pipeline.AdapterPipeline;
 import org.apache.streampipes.connect.adapter.preprocessing.elements.*;
 import org.apache.streampipes.connect.adapter.preprocessing.transform.stream.DuplicateFilterPipelineElement;
@@ -29,10 +28,7 @@ import org.apache.streampipes.model.connect.rules.TransformationRuleDescription;
 import org.apache.streampipes.model.connect.rules.schema.SchemaTransformationRuleDescription;
 import org.apache.streampipes.model.connect.rules.stream.EventRateTransformationRuleDescription;
 import org.apache.streampipes.model.connect.rules.stream.RemoveDuplicatesTransformationRuleDescription;
-import org.apache.streampipes.model.connect.rules.value.AddTimestampRuleDescription;
-import org.apache.streampipes.model.connect.rules.value.AddValueTransformationRuleDescription;
-import org.apache.streampipes.model.connect.rules.value.CorrectionValueTransformationRuleDescription;
-import org.apache.streampipes.model.connect.rules.value.ValueTransformationRuleDescription;
+import org.apache.streampipes.model.connect.rules.value.*;
 import org.apache.streampipes.model.grounding.JmsTransportProtocol;
 import org.apache.streampipes.model.grounding.KafkaTransportProtocol;
 
@@ -47,13 +43,13 @@ public class AdapterPipelineGenerator {
 
     var pipelineElements = makeAdapterPipelineElements(adapterDescription.getRules());
 
-    RemoveDuplicatesTransformationRuleDescription duplicatesTransformationRuleDescription = getRemoveDuplicateRule(adapterDescription.getRules());
+    var duplicatesTransformationRuleDescription = getRemoveDuplicateRule(adapterDescription.getRules());
     if (duplicatesTransformationRuleDescription != null) {
       pipelineElements.add(new DuplicateFilterPipelineElement(duplicatesTransformationRuleDescription.getFilterTimeWindow()));
     }
 
-    TransformStreamAdapterElement transformStreamAdapterElement = new TransformStreamAdapterElement();
-    EventRateTransformationRuleDescription eventRateTransformationRuleDescription = getEventRateTransformationRule(adapterDescription.getRules());
+    var transformStreamAdapterElement = new TransformStreamAdapterElement();
+    var eventRateTransformationRuleDescription = getEventRateTransformationRule(adapterDescription.getRules());
     if (eventRateTransformationRuleDescription != null) {
       transformStreamAdapterElement.addStreamTransformationRuleDescription(eventRateTransformationRuleDescription);
     }
@@ -73,13 +69,13 @@ public class AdapterPipelineGenerator {
     List<IAdapterPipelineElement> pipelineElements = new ArrayList<>();
 
     // Must be before the schema transformations to ensure that user can move this event property
-    AddTimestampRuleDescription timestampTransformationRuleDescription = getTimestampRule(rules);
+    var timestampTransformationRuleDescription = getTimestampRule(rules);
     if (timestampTransformationRuleDescription != null) {
       pipelineElements.add(new AddTimestampPipelineElement(
         timestampTransformationRuleDescription.getRuntimeKey()));
     }
 
-    AddValueTransformationRuleDescription valueTransformationRuleDescription = getAddValueRule(rules);
+    var valueTransformationRuleDescription = getAddValueRule(rules);
     if (valueTransformationRuleDescription != null) {
       pipelineElements.add(new AddValuePipelineElement(
         valueTransformationRuleDescription.getRuntimeKey(),
@@ -95,7 +91,7 @@ public class AdapterPipelineGenerator {
   }
 
   private SendToBrokerAdapterSink<?> getAdapterSink(AdapterDescription adapterDescription) {
-    SpProtocol prioritizedProtocol =
+    var prioritizedProtocol =
       BackendConfig.INSTANCE.getMessagingSettings().getPrioritizedProtocols().get(0);
 
     if (GroundingService.isPrioritized(prioritizedProtocol, JmsTransportProtocol.class)) {
@@ -125,10 +121,6 @@ public class AdapterPipelineGenerator {
     return getRule(rules, AddValueTransformationRuleDescription.class);
   }
 
-  private CorrectionValueTransformationRuleDescription getCorrectionValueRule(List<TransformationRuleDescription> rules) {
-    return getRule(rules, CorrectionValueTransformationRuleDescription.class);
-  }
-
   private <G extends TransformationRuleDescription> G getRule(List<TransformationRuleDescription> rules,
                                                               Class<G> type) {
 
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java
index a5e4f3841..812c67b7f 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java
@@ -21,6 +21,7 @@ package org.apache.streampipes.connect.adapter.format.csv;
 
 import org.apache.streampipes.connect.adapter.model.generic.Parser;
 import org.apache.streampipes.connect.adapter.sdk.ParameterExtractor;
+import org.apache.streampipes.connect.adapter.util.DatatypeUtils;
 import org.apache.streampipes.connect.api.EmitBinaryEvent;
 import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.model.connect.grounding.FormatDescription;
@@ -28,7 +29,6 @@ import org.apache.streampipes.model.connect.guess.AdapterGuessInfo;
 import org.apache.streampipes.model.connect.guess.GuessTypeInfo;
 import org.apache.streampipes.model.schema.EventPropertyPrimitive;
 import org.apache.streampipes.model.schema.EventSchema;
-import org.apache.streampipes.vocabulary.XSD;
 
 import java.io.BufferedReader;
 import java.io.IOException;
@@ -108,9 +108,11 @@ public class CsvParser extends Parser {
         EventSchema resultSchema = new EventSchema();
         for (int i = 0; i < keys.length; i++) {
             EventPropertyPrimitive p = new EventPropertyPrimitive();
+            var runtimeType = DatatypeUtils.getXsdDatatype(data[i]);
+            var convertedValue = DatatypeUtils.convertValue(data[i], runtimeType);
             p.setRuntimeName(keys[i]);
-            p.setRuntimeType(getTypeString(data[i]));
-            sample.put(keys[i], new GuessTypeInfo(getTypeString(data[i]), data[i]));
+            p.setRuntimeType(runtimeType);
+            sample.put(keys[i], new GuessTypeInfo(DatatypeUtils.getCanonicalTypeClassName(data[i]), convertedValue));
             resultSchema.addEventProperty(p);
         }
 
@@ -122,35 +124,6 @@ public class CsvParser extends Parser {
         return getSchemaAndSample(oneEvent).getEventSchema();
     }
 
-    private String getTypeString(String o) {
-
-        String typeClass = getTypeClass(o);
-
-        if (Float.class.getCanonicalName().equals(typeClass)) {
-            return XSD._float.toString();
-        } else if (Boolean.class.getCanonicalName().equals(typeClass)) {
-            return XSD._boolean.toString();
-        } else {
-            return XSD._string.toString();
-        }
-    }
-
-    private String getTypeClass(String o) {
-
-        try {
-            Double.parseDouble(o);
-            return Float.class.getCanonicalName();
-        } catch (NumberFormatException e) {
-
-        }
-
-        if (o.equalsIgnoreCase("true") || o.equalsIgnoreCase("false")) {
-            return Boolean.class.getCanonicalName();
-        }
-
-        return String.class.getCanonicalName();
-    }
-
 
     public static String[] parseLine(String cvsLine, String separatorString) {
 
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/elements/TransformValueAdapterPipelineElement.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/elements/TransformValueAdapterPipelineElement.java
index 25d357518..abb1bd27a 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/elements/TransformValueAdapterPipelineElement.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/elements/TransformValueAdapterPipelineElement.java
@@ -22,6 +22,7 @@ import org.apache.streampipes.connect.adapter.preprocessing.Util;
 import org.apache.streampipes.connect.adapter.preprocessing.transform.value.*;
 import org.apache.streampipes.connect.api.IAdapterPipelineElement;
 import org.apache.streampipes.model.connect.rules.TransformationRuleDescription;
+import org.apache.streampipes.model.connect.rules.value.ChangeDatatypeTransformationRuleDescription;
 import org.apache.streampipes.model.connect.rules.value.CorrectionValueTransformationRuleDescription;
 import org.apache.streampipes.model.connect.rules.value.TimestampTranfsformationRuleDescription;
 import org.apache.streampipes.model.connect.rules.value.UnitTransformRuleDescription;
@@ -43,11 +44,11 @@ public class TransformValueAdapterPipelineElement implements IAdapterPipelineEle
         // transforms description to actual rules
         for (TransformationRuleDescription ruleDescription : transformationRuleDescriptions) {
             if (ruleDescription instanceof UnitTransformRuleDescription) {
-                UnitTransformRuleDescription tmp = (UnitTransformRuleDescription) ruleDescription;
+                var tmp = (UnitTransformRuleDescription) ruleDescription;
                 rules.add(new UnitTransformationRule(Util.toKeyArray(tmp.getRuntimeKey()),
                         tmp.getFromUnitRessourceURL(), tmp.getToUnitRessourceURL()));
-            } else if(ruleDescription instanceof TimestampTranfsformationRuleDescription) {
-                TimestampTranfsformationRuleDescription tmp = (TimestampTranfsformationRuleDescription) ruleDescription;
+            } else if (ruleDescription instanceof TimestampTranfsformationRuleDescription) {
+                var tmp = (TimestampTranfsformationRuleDescription) ruleDescription;
                 TimestampTranformationRuleMode mode = null;
                 switch (tmp.getMode()) {
                     case "formatString": mode = TimestampTranformationRuleMode.FORMAT_STRING;
@@ -57,9 +58,12 @@ public class TransformValueAdapterPipelineElement implements IAdapterPipelineEle
                 rules.add(new TimestampTranformationRule(Util.toKeyArray(tmp.getRuntimeKey()), mode,
                         tmp.getFormatString(), tmp.getMultiplier()));
             }
-            else if(ruleDescription instanceof CorrectionValueTransformationRuleDescription) {
-                CorrectionValueTransformationRuleDescription tmp = (CorrectionValueTransformationRuleDescription) ruleDescription;
+            else if (ruleDescription instanceof CorrectionValueTransformationRuleDescription) {
+                var tmp = (CorrectionValueTransformationRuleDescription) ruleDescription;
                 rules.add(new CorrectionValueTransformationRule(Util.toKeyArray(tmp.getRuntimeKey()), tmp.getCorrectionValue(), tmp.getOperator()));
+            } else if (ruleDescription instanceof ChangeDatatypeTransformationRuleDescription) {
+                var tmp = (ChangeDatatypeTransformationRuleDescription) ruleDescription;
+                rules.add(new DatatypeTransformationRule(tmp.getRuntimeKey(), tmp.getOriginalDatatypeXsd(), tmp.getTargetDatatypeXsd()));
             }
 
             else {
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/transform/value/DatatypeTransformationRule.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/transform/value/DatatypeTransformationRule.java
new file mode 100644
index 000000000..d8c6d8aac
--- /dev/null
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/transform/value/DatatypeTransformationRule.java
@@ -0,0 +1,52 @@
+/*
+ * 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.connect.adapter.preprocessing.transform.value;
+
+import org.apache.streampipes.connect.adapter.util.DatatypeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class DatatypeTransformationRule implements ValueTransformationRule {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DatatypeTransformationRule.class);
+
+  private String eventKey;
+  private String originalDatatypeXsd;
+  private String targetDatatypeXsd;
+
+  public DatatypeTransformationRule(String eventKey, String originalDatatypeXsd, String targetDatatypeXsd) {
+    this.eventKey = eventKey;
+    this.originalDatatypeXsd = originalDatatypeXsd;
+    this.targetDatatypeXsd = targetDatatypeXsd;
+  }
+
+  @Override
+  public Map<String, Object> transform(Map<String, Object> event) {
+    Object value = event.get(eventKey);
+    Object transformedValue = transformDatatype(value);
+    event.put(eventKey, transformedValue);
+    return event;
+  }
+
+  public Object transformDatatype(Object value) {
+    return DatatypeUtils.convertValue(value, targetDatatypeXsd);
+  }
+}
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/transform/value/ValueEventTransformer.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/transform/value/ValueEventTransformer.java
index 939cbad78..c965c894d 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/transform/value/ValueEventTransformer.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/preprocessing/transform/value/ValueEventTransformer.java
@@ -26,14 +26,16 @@ import java.util.Map;
 
 public class ValueEventTransformer implements ValueTransformationRule {
 
-    private List<UnitTransformationRule> unitTransformationRules;
-    private List<TimestampTranformationRule> timestampTransformationRules;
-    private List<CorrectionValueTransformationRule> correctionValueTransformationRules;
+    private final List<UnitTransformationRule> unitTransformationRules;
+    private final List<TimestampTranformationRule> timestampTransformationRules;
+    private final List<CorrectionValueTransformationRule> correctionValueTransformationRules;
+    private final List<DatatypeTransformationRule> datatypeTransformationRules;
 
     public ValueEventTransformer(List<ValueTransformationRule> rules) {
         this.unitTransformationRules = new ArrayList<>();
         this.timestampTransformationRules = new ArrayList<>();
         this.correctionValueTransformationRules = new ArrayList<>();
+        this.datatypeTransformationRules = new ArrayList<>();
 
         for (TransformationRule rule : rules) {
             if (rule instanceof UnitTransformationRule) {
@@ -42,16 +44,12 @@ public class ValueEventTransformer implements ValueTransformationRule {
                 this.timestampTransformationRules.add((TimestampTranformationRule) rule);
             } else if (rule instanceof CorrectionValueTransformationRule) {
                 this.correctionValueTransformationRules.add((CorrectionValueTransformationRule) rule);
+            } else if (rule instanceof DatatypeTransformationRule) {
+                this.datatypeTransformationRules.add((DatatypeTransformationRule) rule);
             }
         }
     }
 
-/*
-    public ValueEventTransformer(List<UnitTransformationRule> unitTransformationRule) {
-        this.unitTransformationRules = new ArrayList<>();
-    }
-*/
-
     @Override
     public Map<String, Object> transform(Map<String, Object> event) {
 
@@ -63,36 +61,14 @@ public class ValueEventTransformer implements ValueTransformationRule {
             event = rule.transform(event);
         }
 
-        for (CorrectionValueTransformationRule rule : correctionValueTransformationRules) {
+        for (var rule: datatypeTransformationRules) {
             event = rule.transform(event);
         }
 
+        for (CorrectionValueTransformationRule rule : correctionValueTransformationRules) {
+            event = rule.transform(event);
+        }
 
         return event;
     }
-
-
-    public List<UnitTransformationRule> getUnitTransformationRules() {
-        return unitTransformationRules;
-    }
-
-    public void setUnitTransformationRules(List<UnitTransformationRule> unitTransformationRules) {
-        this.unitTransformationRules = unitTransformationRules;
-    }
-
-    public List<TimestampTranformationRule> getTimestampTransformationRules() {
-        return timestampTransformationRules;
-    }
-
-    public void setTimestampTransformationRules(List<TimestampTranformationRule> timestampTransformationRules) {
-        this.timestampTransformationRules = timestampTransformationRules;
-    }
-
-    public List<CorrectionValueTransformationRule> getCorrectionValueTransformationRules() {
-        return correctionValueTransformationRules;
-    }
-
-    public void setCorrectionValueTransformationRules(List<CorrectionValueTransformationRule> correctionValueTransformationRules) {
-        this.correctionValueTransformationRules = correctionValueTransformationRules;
-    }
 }
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/DatatypeUtils.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/DatatypeUtils.java
new file mode 100644
index 000000000..9261beee3
--- /dev/null
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/DatatypeUtils.java
@@ -0,0 +1,108 @@
+/*
+ * 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.connect.adapter.util;
+
+import org.apache.commons.lang3.math.NumberUtils;
+import org.apache.streampipes.vocabulary.XSD;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DatatypeUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DatatypeUtils.class);
+
+  public static Object convertValue(Object value,
+                                    String targetDatatypeXsd) {
+    var stringValue = String.valueOf(value);
+    if (XSD._string.toString().equals(targetDatatypeXsd)) {
+      return stringValue;
+    } else {
+      try {
+        if (XSD._double.toString().equals(targetDatatypeXsd)) {
+          return Double.parseDouble(stringValue);
+        } else if (XSD._float.toString().equals(targetDatatypeXsd)) {
+          return Float.parseFloat(stringValue);
+        } else if (XSD._boolean.toString().equals(targetDatatypeXsd)) {
+          return Boolean.parseBoolean(stringValue);
+        } else if (XSD._integer.toString().equals(targetDatatypeXsd)) {
+          var floatingNumber = Float.parseFloat(stringValue);
+          return Math.round(floatingNumber);
+        } else if (XSD._long.toString().equals(targetDatatypeXsd)) {
+          var floatingNumber = Double.parseDouble(stringValue);
+          return Math.round(floatingNumber);
+        }
+      } catch (NumberFormatException e) {
+        LOG.error("Number format exception {}", value);
+        return value;
+      }
+    }
+
+    return value;
+  }
+
+  public static String getCanonicalTypeClassName(String value) {
+    return getTypeClass(value).getCanonicalName();
+  }
+
+  public static String getXsdDatatype(String value) {
+    var clazz = getTypeClass(value);
+    if (clazz.equals(Integer.class)) {
+      return XSD._integer.toString();
+    } else if (clazz.equals(Long.class)) {
+      return XSD._long.toString();
+    } else if (clazz.equals(Float.class)) {
+      return XSD._float.toString();
+    } else if (clazz.equals(Double.class)) {
+      return XSD._double.toString();
+    } else if (clazz.equals(Boolean.class)) {
+      return XSD._boolean.toString();
+    } else {
+      return XSD._string.toString();
+    }
+  }
+
+  public static Class<?> getTypeClass(String value) {
+    if (NumberUtils.isParsable(value)) {
+      try {
+        Integer.parseInt(value);
+        return Integer.class;
+      } catch (NumberFormatException ignored) {
+      }
+
+      try {
+        Long.parseLong(value);
+        return Long.class;
+      } catch (NumberFormatException ignored) {
+      }
+
+      try {
+        Double.parseDouble(value);
+        return Float.class;
+      } catch (NumberFormatException ignored) {
+      }
+
+    }
+
+    if (value.equalsIgnoreCase("true") || value.equalsIgnoreCase("false")) {
+      return Boolean.class;
+    }
+
+    return String.class;
+  }
+}
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/PollingSettings.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/PollingSettings.java
index 79dea5b0e..177702c82 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/PollingSettings.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/PollingSettings.java
@@ -15,6 +15,7 @@
  * limitations under the License.
  *
  */
+
 package org.apache.streampipes.connect.adapter.util;
 
 import java.util.concurrent.TimeUnit;
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/connect/rules/TransformationRuleDescription.java b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/rules/TransformationRuleDescription.java
index bd1b85b5e..590b4161e 100644
--- a/streampipes-model/src/main/java/org/apache/streampipes/model/connect/rules/TransformationRuleDescription.java
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/rules/TransformationRuleDescription.java
@@ -41,6 +41,7 @@ import org.apache.streampipes.model.shared.annotation.TsModel;
         @JsonSubTypes.Type(DeleteRuleDescription.class),
         @JsonSubTypes.Type(RenameRuleDescription.class),
         @JsonSubTypes.Type(MoveRuleDescription.class),
+        @JsonSubTypes.Type(ChangeDatatypeTransformationRuleDescription.class),
         @JsonSubTypes.Type(CorrectionValueTransformationRuleDescription.class),
 })
 public abstract class TransformationRuleDescription extends UnnamedStreamPipesEntity {
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/connect/rules/value/ChangeDatatypeTransformationRuleDescription.java b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/rules/value/ChangeDatatypeTransformationRuleDescription.java
new file mode 100644
index 000000000..aa64e647b
--- /dev/null
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/rules/value/ChangeDatatypeTransformationRuleDescription.java
@@ -0,0 +1,60 @@
+/*
+ * 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.model.connect.rules.value;
+
+public class ChangeDatatypeTransformationRuleDescription extends ValueTransformationRuleDescription {
+
+  private String runtimeKey;
+  private String originalDatatypeXsd;
+  private String targetDatatypeXsd;
+
+  public ChangeDatatypeTransformationRuleDescription() {
+  }
+
+  public ChangeDatatypeTransformationRuleDescription(ChangeDatatypeTransformationRuleDescription other) {
+    super(other);
+    this.runtimeKey = other.getRuntimeKey();
+    this.originalDatatypeXsd = other.getOriginalDatatypeXsd();
+    this.targetDatatypeXsd = other.getTargetDatatypeXsd();
+  }
+
+  public String getRuntimeKey() {
+    return runtimeKey;
+  }
+
+  public void setRuntimeKey(String runtimeKey) {
+    this.runtimeKey = runtimeKey;
+  }
+
+  public String getOriginalDatatypeXsd() {
+    return originalDatatypeXsd;
+  }
+
+  public void setOriginalDatatypeXsd(String originalDatatypeXsd) {
+    this.originalDatatypeXsd = originalDatatypeXsd;
+  }
+
+  public String getTargetDatatypeXsd() {
+    return targetDatatypeXsd;
+  }
+
+  public void setTargetDatatypeXsd(String targetDatatypeXsd) {
+    this.targetDatatypeXsd = targetDatatypeXsd;
+  }
+}
diff --git a/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts b/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
index d2712a990..88c0b73ca 100644
--- a/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
+++ b/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
@@ -18,10 +18,10 @@
 /* tslint:disable */
 /* eslint-disable */
 // @ts-nocheck
-// Generated using typescript-generator version 2.27.744 on 2022-08-16 16:51:03.
+// Generated using typescript-generator version 2.27.744 on 2022-08-17 14:48:34.
 
 export class AbstractStreamPipesEntity {
-    "@class": "org.apache.streampipes.model.base.AbstractStreamPipesEntity" | "org.apache.streampipes.model.base.NamedStreamPipesEntity" | "org.apache.streampipes.model.connect.adapter.AdapterDescription" | "org.apache.streampipes.model.connect.adapter.AdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.GenericAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.SpecificAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.AdapterStre [...]
+    "@class": "org.apache.streampipes.model.base.AbstractStreamPipesEntity" | "org.apache.streampipes.model.base.NamedStreamPipesEntity" | "org.apache.streampipes.model.connect.adapter.AdapterDescription" | "org.apache.streampipes.model.connect.adapter.AdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.GenericAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.SpecificAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.AdapterStre [...]
     elementId: string;
 
     static fromData(data: AbstractStreamPipesEntity, target?: AbstractStreamPipesEntity): AbstractStreamPipesEntity {
@@ -36,7 +36,7 @@ export class AbstractStreamPipesEntity {
 }
 
 export class UnnamedStreamPipesEntity extends AbstractStreamPipesEntity {
-    "@class": "org.apache.streampipes.model.base.UnnamedStreamPipesEntity" | "org.apache.streampipes.model.connect.guess.GuessSchema" | "org.apache.streampipes.model.connect.rules.TransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.ValueTransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddTimestampRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddValueTransformationRuleDescription" | "org.apache.streamp [...]
+    "@class": "org.apache.streampipes.model.base.UnnamedStreamPipesEntity" | "org.apache.streampipes.model.connect.guess.GuessSchema" | "org.apache.streampipes.model.connect.rules.TransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.ValueTransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddTimestampRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddValueTransformationRuleDescription" | "org.apache.streamp [...]
 
     static fromData(data: UnnamedStreamPipesEntity, target?: UnnamedStreamPipesEntity): UnnamedStreamPipesEntity {
         if (!data) {
@@ -151,8 +151,8 @@ export class NamedStreamPipesEntity extends AbstractStreamPipesEntity {
         instance.applicationLinks = __getCopyArrayFn(ApplicationLink.fromData)(data.applicationLinks);
         instance.internallyManaged = data.internallyManaged;
         instance.connectedTo = __getCopyArrayFn(__identity<string>())(data.connectedTo);
-        instance.dom = data.dom;
         instance.uri = data.uri;
+        instance.dom = data.dom;
         instance._rev = data._rev;
         return instance;
     }
@@ -192,9 +192,9 @@ export class AdapterDescription extends NamedStreamPipesEntity {
         instance.selectedEndpointUrl = data.selectedEndpointUrl;
         instance.correspondingServiceGroup = data.correspondingServiceGroup;
         instance.correspondingDataStreamElementId = data.correspondingDataStreamElementId;
-        instance.schemaRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.schemaRules);
         instance.streamRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.streamRules);
         instance.valueRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.valueRules);
+        instance.schemaRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.schemaRules);
         return instance;
     }
 
@@ -304,7 +304,7 @@ export class AdapterType {
 }
 
 export class TransformationRuleDescription extends UnnamedStreamPipesEntity {
-    "@class": "org.apache.streampipes.model.connect.rules.TransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.ValueTransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddTimestampRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddValueTransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.TimestampTranfsformationRuleDescription" | "org.apache.streampipes.model.connect.rules.valu [...]
+    "@class": "org.apache.streampipes.model.connect.rules.TransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.ValueTransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddTimestampRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddValueTransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.TimestampTranfsformationRuleDescription" | "org.apache.streampipes.model.connect.rules.valu [...]
 
     static fromData(data: TransformationRuleDescription, target?: TransformationRuleDescription): TransformationRuleDescription {
         if (!data) {
@@ -340,6 +340,8 @@ export class TransformationRuleDescription extends UnnamedStreamPipesEntity {
                 return RenameRuleDescription.fromData(data);
             case "org.apache.streampipes.model.connect.rules.schema.MoveRuleDescription":
                 return MoveRuleDescription.fromData(data);
+            case "org.apache.streampipes.model.connect.rules.value.ChangeDatatypeTransformationRuleDescription":
+                return ChangeDatatypeTransformationRuleDescription.fromData(data);
             case "org.apache.streampipes.model.connect.rules.value.CorrectionValueTransformationRuleDescription":
                 return CorrectionValueTransformationRuleDescription.fromData(data);
         }
@@ -347,7 +349,7 @@ export class TransformationRuleDescription extends UnnamedStreamPipesEntity {
 }
 
 export class ValueTransformationRuleDescription extends TransformationRuleDescription {
-    "@class": "org.apache.streampipes.model.connect.rules.value.ValueTransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddTimestampRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddValueTransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.TimestampTranfsformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.UnitTransformRuleDescription" | "org.apache.streampipes.model.connect.rules [...]
+    "@class": "org.apache.streampipes.model.connect.rules.value.ValueTransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddTimestampRuleDescription" | "org.apache.streampipes.model.connect.rules.value.AddValueTransformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.TimestampTranfsformationRuleDescription" | "org.apache.streampipes.model.connect.rules.value.UnitTransformRuleDescription" | "org.apache.streampipes.model.connect.rules [...]
 
     static fromData(data: ValueTransformationRuleDescription, target?: ValueTransformationRuleDescription): ValueTransformationRuleDescription {
         if (!data) {
@@ -689,6 +691,25 @@ export class Category {
     }
 }
 
+export class ChangeDatatypeTransformationRuleDescription extends ValueTransformationRuleDescription {
+    "@class": "org.apache.streampipes.model.connect.rules.value.ChangeDatatypeTransformationRuleDescription";
+    originalDatatypeXsd: string;
+    runtimeKey: string;
+    targetDatatypeXsd: string;
+
+    static fromData(data: ChangeDatatypeTransformationRuleDescription, target?: ChangeDatatypeTransformationRuleDescription): ChangeDatatypeTransformationRuleDescription {
+        if (!data) {
+            return data;
+        }
+        const instance = target || new ChangeDatatypeTransformationRuleDescription();
+        super.fromData(data, instance);
+        instance.runtimeKey = data.runtimeKey;
+        instance.originalDatatypeXsd = data.originalDatatypeXsd;
+        instance.targetDatatypeXsd = data.targetDatatypeXsd;
+        return instance;
+    }
+}
+
 export class CodeInputStaticProperty extends StaticProperty {
     "@class": "org.apache.streampipes.model.staticproperty.CodeInputStaticProperty";
     codeTemplate: string;
@@ -2618,9 +2639,9 @@ export class PipelineTemplateDescription extends NamedStreamPipesEntity {
         const instance = target || new PipelineTemplateDescription();
         super.fromData(data, instance);
         instance.boundTo = __getCopyArrayFn(BoundPipelineElement.fromData)(data.boundTo);
-        instance.pipelineTemplateName = data.pipelineTemplateName;
         instance.pipelineTemplateId = data.pipelineTemplateId;
         instance.pipelineTemplateDescription = data.pipelineTemplateDescription;
+        instance.pipelineTemplateName = data.pipelineTemplateName;
         return instance;
     }
 }
@@ -3461,7 +3482,7 @@ export type StreamTransformationRuleDescriptionUnion = EventRateTransformationRu
 
 export type TopicDefinitionUnion = SimpleTopicDefinition | WildcardTopicDefinition;
 
-export type TransformationRuleDescriptionUnion = AddTimestampRuleDescription | AddValueTransformationRuleDescription | TimestampTranfsformationRuleDescription | UnitTransformRuleDescription | EventRateTransformationRuleDescription | RemoveDuplicatesTransformationRuleDescription | CreateNestedRuleDescription | DeleteRuleDescription | RenameRuleDescription | MoveRuleDescription | CorrectionValueTransformationRuleDescription;
+export type TransformationRuleDescriptionUnion = AddTimestampRuleDescription | AddValueTransformationRuleDescription | TimestampTranfsformationRuleDescription | UnitTransformRuleDescription | EventRateTransformationRuleDescription | RemoveDuplicatesTransformationRuleDescription | CreateNestedRuleDescription | DeleteRuleDescription | RenameRuleDescription | MoveRuleDescription | ChangeDatatypeTransformationRuleDescription | CorrectionValueTransformationRuleDescription;
 
 export type TransportProtocolUnion = JmsTransportProtocol | KafkaTransportProtocol | MqttTransportProtocol;
 
diff --git a/ui/src/app/connect/services/transformation-rule.service.ts b/ui/src/app/connect/services/transformation-rule.service.ts
index 87819561a..478669698 100644
--- a/ui/src/app/connect/services/transformation-rule.service.ts
+++ b/ui/src/app/connect/services/transformation-rule.service.ts
@@ -30,6 +30,7 @@ import {
   EventSchema,
   MoveRuleDescription,
   RenameRuleDescription,
+  ChangeDatatypeTransformationRuleDescription,
   TimestampTranfsformationRuleDescription,
   TransformationRuleDescriptionUnion,
   UnitTransformRuleDescription
@@ -82,33 +83,16 @@ export class TransformationRuleService {
       }
 
       // Scale
-      transformationRuleDescription = transformationRuleDescription.concat(this.getCorrectionValueRules(
-        this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema));
+      transformationRuleDescription = transformationRuleDescription
+        .concat(this.getCorrectionValueRules(this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema))
+        .concat(this.getRenameRules(this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema))
+        .concat(this.getCreateNestedRules(this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema))
+        .concat(this.getMoveRules(this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema))
+        .concat(this.getDeleteRules(this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema))
+        .concat(this.getUnitTransformRules(this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema))
+        .concat(this.getTimestampTransformRules(this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema))
+        .concat(this.getDatatypeTransformRules(this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema));
 
-      // Rename
-      transformationRuleDescription = transformationRuleDescription.concat(this.getRenameRules(
-        this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema));
-
-
-      // Create Nested
-      transformationRuleDescription = transformationRuleDescription.concat(this.getCreateNestedRules(
-        this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema));
-
-      // Move
-      transformationRuleDescription = transformationRuleDescription.concat(this.getMoveRules(
-        this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema));
-
-      // Delete
-      transformationRuleDescription = transformationRuleDescription.concat(this.getDeleteRules(
-        this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema));
-
-      // Unit
-      transformationRuleDescription = transformationRuleDescription.concat(this.getUnitTransformRules(
-        this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema));
-
-      // Timestmap
-      transformationRuleDescription = transformationRuleDescription.concat(this.getTimestampTransformRules(
-        this.newEventSchema.eventProperties, this.oldEventSchema, this.newEventSchema));
 
       return transformationRuleDescription;
     }
@@ -454,6 +438,40 @@ export class TransformationRuleService {
     return property.domainProperties.some(dp => dp === 'http://schema.org/DateTime');
   }
 
+  private getDatatypeTransformRules(eventProperties: EventPropertyUnion[],
+                                    oldEventSchema: EventSchema,
+                                    newEventSchema: EventSchema): ChangeDatatypeTransformationRuleDescription[] {
+
+    let result: ChangeDatatypeTransformationRuleDescription[] = [];
+
+    eventProperties.forEach(ep => {
+      if (ep instanceof EventPropertyPrimitive) {
+        const eventPropertyPrimitive = ep as EventPropertyPrimitive;
+        const newRuntimeType = ep.runtimeType;
+        const keyNew = this.getCompleteRuntimeNameKey(newEventSchema.eventProperties, eventPropertyPrimitive.elementId);
+        const oldProperty = this.getEventProperty(oldEventSchema.eventProperties, ep.elementId);
+        if (oldProperty) {
+          const oldRuntimeType = (oldProperty as EventPropertyPrimitive).runtimeType;
+          if (newRuntimeType !== oldRuntimeType) {
+            const rule: ChangeDatatypeTransformationRuleDescription = new ChangeDatatypeTransformationRuleDescription();
+            rule['@class'] = 'org.apache.streampipes.model.connect.rules.value.ChangeDatatypeTransformationRuleDescription';
+            rule.runtimeKey = keyNew;
+            rule.originalDatatypeXsd = oldRuntimeType;
+            rule.targetDatatypeXsd = newRuntimeType;
+
+            result.push(rule);
+          }
+        }
+
+      } else if (ep instanceof EventPropertyNested) {
+        const nestedResults = this.getDatatypeTransformRules((ep as EventPropertyNested).eventProperties, oldEventSchema, newEventSchema);
+        result = result.concat(nestedResults);
+      }
+    });
+
+    return result;
+  }
+
   private getCorrectionValueRules(eventProperties: EventPropertyUnion[],
                                   oldEventSchema: EventSchema,
                                   newEventSchema: EventSchema) {


[incubator-streampipes] 14/15: [hotfix] Add missing header

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit aa10f7f67b5d330210913fa8ab65f45c10729fa3
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Wed Aug 17 22:14:58 2022 +0200

    [hotfix] Add missing header
---
 .../commons/exceptions/SpConfigurationException.java   | 18 ++++++++++++++++++
 1 file changed, 18 insertions(+)

diff --git a/streampipes-commons/src/main/java/org/apache/streampipes/commons/exceptions/SpConfigurationException.java b/streampipes-commons/src/main/java/org/apache/streampipes/commons/exceptions/SpConfigurationException.java
index a1436f4f1..321330b51 100644
--- a/streampipes-commons/src/main/java/org/apache/streampipes/commons/exceptions/SpConfigurationException.java
+++ b/streampipes-commons/src/main/java/org/apache/streampipes/commons/exceptions/SpConfigurationException.java
@@ -1,3 +1,21 @@
+/*
+ * 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.commons.exceptions;
 
 public class SpConfigurationException extends Exception {


[incubator-streampipes] 01/15: [hotfix] Improve x axis representation of line chart

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 5dd99908f7996d9e405652db996d35bdd13a3052
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Mon Aug 15 13:09:19 2022 +0200

    [hotfix] Improve x axis representation of line chart
---
 .../dashboard/components/widgets/base/base-ngx-line-charts-widget.ts   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/ui/src/app/dashboard/components/widgets/base/base-ngx-line-charts-widget.ts b/ui/src/app/dashboard/components/widgets/base/base-ngx-line-charts-widget.ts
index 94fef221b..55d608560 100644
--- a/ui/src/app/dashboard/components/widgets/base/base-ngx-line-charts-widget.ts
+++ b/ui/src/app/dashboard/components/widgets/base/base-ngx-line-charts-widget.ts
@@ -63,8 +63,9 @@ export abstract class BaseNgxLineChartsStreamPipesWidget extends BaseNgxChartsSt
     }
 
     timestampTickFormatting(timestamp: any): string {
+        const padL = (nr, len = 2, chr = `0`) => `${nr}`.padStart(2, chr);
         const date = new Date(timestamp);
-        return date.getHours() + ':' + date.getMinutes().toString().substr(-2) + ':' + date.getSeconds().toString().substr(-2);
+        return date.getHours() + ':' + `${padL(date.getMinutes())}` + ':' + `${padL(date.getSeconds())}`;
     }
 
     protected getQueryLimit(extractor: StaticPropertyExtractor): number {


[incubator-streampipes] 03/15: [STREAMPIPES-577] Show live preview for OPC-UA adapter

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d995362d3d637e88f15db9a1235fe56f5b1a3563
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Tue Aug 16 13:20:26 2022 +0200

    [STREAMPIPES-577] Show live preview for OPC-UA adapter
---
 .../connect/iiot/adapters/opcua/OpcUaAdapter.java  | 22 ++++---
 .../iiot/adapters/opcua/OpcUaNodeBrowser.java      | 10 ++++
 .../iiot/adapters/opcua/utils/OpcUaUtil.java       | 50 ++++++++++++++--
 .../streampipes/model/StreamPipesErrorMessage.java | 11 +++-
 .../model/connect/guess/FieldStatus.java           | 31 ++--------
 .../model/connect/guess/FieldStatusInfo.java       | 69 ++++++++++++++++++++++
 .../model/connect/guess/GuessSchema.java           | 30 ++++++++++
 .../model/connect/guess/GuessTypeInfo.java         | 47 ++++++++-------
 .../src/lib/model/gen/streampipes-model.ts         | 48 +++++++++++++--
 .../event-property-row.component.html              | 27 ++++++---
 .../event-property-row.component.scss              | 25 ++++++++
 .../event-property-row.component.ts                | 35 ++++++++---
 .../event-schema/event-schema.component.html       |  8 ++-
 .../event-schema/event-schema.component.ts         | 10 +++-
 14 files changed, 337 insertions(+), 86 deletions(-)

diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaAdapter.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaAdapter.java
index 76b3df7e2..9b00f84d5 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaAdapter.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaAdapter.java
@@ -31,6 +31,7 @@ import org.apache.streampipes.container.api.SupportsRuntimeConfig;
 import org.apache.streampipes.model.AdapterType;
 import org.apache.streampipes.model.connect.adapter.SpecificAdapterStreamDescription;
 import org.apache.streampipes.model.connect.guess.GuessSchema;
+import org.apache.streampipes.model.connect.rules.schema.DeleteRuleDescription;
 import org.apache.streampipes.model.staticproperty.StaticProperty;
 import org.apache.streampipes.sdk.StaticProperties;
 import org.apache.streampipes.sdk.builder.adapter.SpecificDataStreamAdapterBuilder;
@@ -47,14 +48,10 @@ import org.eclipse.milo.opcua.stack.core.types.enumerated.TimestampsToReturn;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
+import java.util.*;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 public class OpcUaAdapter extends PullAdapter implements SupportsRuntimeConfig {
 
@@ -91,11 +88,18 @@ public class OpcUaAdapter extends PullAdapter implements SupportsRuntimeConfig {
     protected void before() throws AdapterException {
 
         this.allNodeIds = new ArrayList<>();
+        List<String> deleteKeys = this.adapterDescription
+          .getSchemaRules()
+          .stream()
+          .filter(rule -> rule instanceof DeleteRuleDescription)
+          .map(rule -> ((DeleteRuleDescription) rule).getRuntimeKey())
+          .collect(Collectors.toList());
+
         try {
             this.spOpcUaClient.connect();
             OpcUaNodeBrowser browserClient =
                     new OpcUaNodeBrowser(this.spOpcUaClient.getClient(), this.spOpcUaClient.getSpOpcConfig());
-            this.allNodes = browserClient.findNodes();
+            this.allNodes = browserClient.findNodes(deleteKeys);
 
 
             for (OpcNode node : this.allNodes) {
@@ -141,8 +145,8 @@ public class OpcUaAdapter extends PullAdapter implements SupportsRuntimeConfig {
 
     @Override
     protected void pullData() {
-        CompletableFuture<List<DataValue>> response =
-                this.spOpcUaClient.getClient().readValues(0, TimestampsToReturn.Both, this.allNodeIds);
+        var response =
+          this.spOpcUaClient.getClient().readValues(0, TimestampsToReturn.Both, this.allNodeIds);
         boolean badStatusCodeReceived = false;
         boolean emptyValueReceived = false;
         try {
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaNodeBrowser.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaNodeBrowser.java
index 2e8e3dbc5..c1cbc202e 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaNodeBrowser.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaNodeBrowser.java
@@ -37,6 +37,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 public class OpcUaNodeBrowser {
@@ -61,6 +62,15 @@ public class OpcUaNodeBrowser {
     return opcNodes;
   }
 
+  public List<OpcNode> findNodes(List<String> runtimeNameFilters) throws UaException {
+    return findNodes()
+      .stream()
+      .filter(node -> runtimeNameFilters
+        .stream()
+        .noneMatch(f -> f.equals(node.getLabel())))
+      .collect(Collectors.toList());
+  }
+
   public List<TreeInputNode> buildNodeTreeFromOrigin() throws UaException, ExecutionException, InterruptedException {
     NodeId origin = spOpcConfig.getOriginNodeId();
 
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/OpcUaUtil.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/OpcUaUtil.java
index d9f95872f..766433a24 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/OpcUaUtil.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/OpcUaUtil.java
@@ -26,7 +26,9 @@ import org.apache.streampipes.connect.iiot.adapters.opcua.OpcUaNodeBrowser;
 import org.apache.streampipes.connect.iiot.adapters.opcua.SpOpcUaClient;
 import org.apache.streampipes.connect.iiot.adapters.opcua.configuration.SpOpcUaConfigBuilder;
 import org.apache.streampipes.model.connect.adapter.SpecificAdapterStreamDescription;
+import org.apache.streampipes.model.connect.guess.FieldStatusInfo;
 import org.apache.streampipes.model.connect.guess.GuessSchema;
+import org.apache.streampipes.model.connect.guess.GuessTypeInfo;
 import org.apache.streampipes.model.schema.EventProperty;
 import org.apache.streampipes.model.schema.EventSchema;
 import org.apache.streampipes.model.staticproperty.RuntimeResolvableTreeInputStaticProperty;
@@ -34,14 +36,20 @@ import org.apache.streampipes.sdk.builder.PrimitivePropertyBuilder;
 import org.apache.streampipes.sdk.extractor.StaticPropertyExtractor;
 import org.eclipse.milo.opcua.sdk.client.OpcUaClient;
 import org.eclipse.milo.opcua.stack.core.UaException;
+import org.eclipse.milo.opcua.stack.core.types.builtin.DataValue;
 import org.eclipse.milo.opcua.stack.core.types.builtin.NodeId;
+import org.eclipse.milo.opcua.stack.core.types.builtin.StatusCode;
 import org.eclipse.milo.opcua.stack.core.types.builtin.unsigned.UInteger;
+import org.eclipse.milo.opcua.stack.core.types.enumerated.TimestampsToReturn;
 
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
 
 /***
  * Collection of several utility functions in context of OPC UA
@@ -73,6 +81,8 @@ public class OpcUaUtil {
     throws AdapterException, ParseException {
     GuessSchema guessSchema = new GuessSchema();
     EventSchema eventSchema = new EventSchema();
+    List<Map<String, GuessTypeInfo>> eventPreview = new ArrayList<>();
+    Map<String, FieldStatusInfo> fieldStatusInfos = new HashMap<>();
     List<EventProperty> allProperties = new ArrayList<>();
 
     SpOpcUaClient spOpcUaClient = new SpOpcUaClient(SpOpcUaConfigBuilder.from(adapterStreamDescription));
@@ -97,22 +107,53 @@ public class OpcUaUtil {
               .label(opcNode.getLabel())
               .build());
           }
-
         }
       }
 
-      spOpcUaClient.disconnect();
+      var nodeIds = selectedNodes.stream().map(OpcNode::getNodeId).collect(Collectors.toList());
+      var response = spOpcUaClient.getClient().readValues(0, TimestampsToReturn.Both, nodeIds);
+
+      var returnValues = response.get();
+
+      makeEventPreview(selectedNodes, eventPreview, fieldStatusInfos, returnValues);
+
 
     } catch (Exception e) {
-      throw new AdapterException("Could not guess schema for opc node:  " + e.getMessage(), e.getCause());
+      throw new AdapterException("Could not guess schema for opc node:  " + e.getMessage(), e);
+    } finally {
+      spOpcUaClient.disconnect();
     }
 
     eventSchema.setEventProperties(allProperties);
     guessSchema.setEventSchema(eventSchema);
+    guessSchema.setEventPreview(eventPreview);
+    guessSchema.setFieldStatusInfo(fieldStatusInfos);
 
     return guessSchema;
   }
 
+  private static void makeEventPreview(List<OpcNode> selectedNodes,
+                                       List<Map<String, GuessTypeInfo>> eventPreview,
+                                       Map<String, FieldStatusInfo> fieldStatusInfos,
+                                       List<DataValue> dataValues) {
+    var singlePreview = new HashMap<String, GuessTypeInfo>();
+
+    for (int i = 0; i < dataValues.size(); i++) {
+      var dv = dataValues.get(i);
+      String label = selectedNodes.get(i).getLabel();
+      if (StatusCode.GOOD.equals(dv.getStatusCode())) {
+        var value = dv.getValue().getValue();
+        singlePreview.put(label, new GuessTypeInfo(value.getClass().getCanonicalName(), value));
+        fieldStatusInfos.put(label, FieldStatusInfo.good());
+      } else {
+        String additionalInfo = dv.getStatusCode() != null ? dv.getStatusCode().toString() : "Status code is null";
+        fieldStatusInfos.put(label, FieldStatusInfo.bad(additionalInfo, false));
+      }
+    }
+
+    eventPreview.add(singlePreview);
+  }
+
 
   /***
    * OPC UA specific implementation of {@link
@@ -142,13 +183,14 @@ public class OpcUaUtil {
       OpcUaNodeBrowser nodeBrowser =
         new OpcUaNodeBrowser(spOpcUaClient.getClient(), spOpcUaClient.getSpOpcConfig());
       config.setNodes(nodeBrowser.buildNodeTreeFromOrigin());
-      spOpcUaClient.disconnect();
 
       return config;
     } catch (UaException e) {
       throw new SpConfigurationException(ExceptionMessageExtractor.getDescription(e), e);
     } catch (ExecutionException | InterruptedException | URISyntaxException e) {
       throw new SpConfigurationException("Could not connect to the OPC UA server with the provided settings", e);
+    } finally {
+      spOpcUaClient.disconnect();
     }
   }
 
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/StreamPipesErrorMessage.java b/streampipes-model/src/main/java/org/apache/streampipes/model/StreamPipesErrorMessage.java
index d2d964090..c6bf711a6 100644
--- a/streampipes-model/src/main/java/org/apache/streampipes/model/StreamPipesErrorMessage.java
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/StreamPipesErrorMessage.java
@@ -31,9 +31,6 @@ public class StreamPipesErrorMessage {
   private String cause;
   private String fullStackTrace;
 
-  public StreamPipesErrorMessage() {
-  }
-
   public static StreamPipesErrorMessage from(Exception exception) {
     String cause = exception.getCause() != null ? exception.getCause().getMessage() : exception.getMessage();
     return new StreamPipesErrorMessage(
@@ -44,6 +41,14 @@ public class StreamPipesErrorMessage {
       cause);
   }
 
+  public StreamPipesErrorMessage(String level,
+                                 String title,
+                                 String detail) {
+    this.level = level;
+    this.title = title;
+    this.detail = detail;
+  }
+
   public StreamPipesErrorMessage(String level,
                                  String title,
                                  String detail,
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/FieldStatus.java
similarity index 56%
copy from ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
copy to streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/FieldStatus.java
index aa6f60f17..eec688b79 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/FieldStatus.java
@@ -16,31 +16,10 @@
  *
  */
 
+package org.apache.streampipes.model.connect.guess;
 
-::ng-deep .mat-checkbox-checked.mat-accent .mat-checkbox-background, .mat-checkbox-indeterminate.mat-accent .mat-checkbox-background {
-    background-color: var(--color-accent) !important;
+public enum FieldStatus {
+  GOOD,
+  BAD,
+  ATTENTION
 }
-
-::ng-deep .mat-checkbox:not(.mat-checkbox-disabled).mat-accent .mat-checkbox-ripple .mat-ripple-element {
-    background-color: var(--color-accent) !important;
-}
-
-.checkbox-selected {
-    opacity: 1 !important;
-}
-
-.timestamp-property {
-    margin-left: 15px;
-    border-radius: 10px;
-    background: var(--color-processor);
-    padding: 0px 10px;
-    font-size: 12px;
-    color: #FFFFFF;
-}
-
-.timestamp-icon {
-    font-size: 12px;
-}
-
-
-
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/FieldStatusInfo.java b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/FieldStatusInfo.java
new file mode 100644
index 000000000..a66b83ddb
--- /dev/null
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/FieldStatusInfo.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.streampipes.model.connect.guess;
+
+public class FieldStatusInfo {
+
+  private FieldStatus fieldStatus;
+  private String additionalInfo;
+  private boolean changesRequired;
+
+  public static FieldStatusInfo good() {
+    var info = new FieldStatusInfo();
+    info.setFieldStatus(FieldStatus.GOOD);
+    return info;
+  }
+
+  public static FieldStatusInfo bad(String additionalInfo,
+                                    boolean changesRequired) {
+    var info = new FieldStatusInfo();
+    info.setFieldStatus(FieldStatus.BAD);
+    info.setAdditionalInfo(additionalInfo);
+    info.setChangesRequired(changesRequired);
+
+    return info;
+  }
+
+  public FieldStatusInfo() {
+  }
+
+  public FieldStatus getFieldStatus() {
+    return fieldStatus;
+  }
+
+  public void setFieldStatus(FieldStatus fieldStatus) {
+    this.fieldStatus = fieldStatus;
+  }
+
+  public String getAdditionalInfo() {
+    return additionalInfo;
+  }
+
+  public void setAdditionalInfo(String additionalInfo) {
+    this.additionalInfo = additionalInfo;
+  }
+
+  public boolean isChangesRequired() {
+    return changesRequired;
+  }
+
+  public void setChangesRequired(boolean changesRequired) {
+    this.changesRequired = changesRequired;
+  }
+}
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/GuessSchema.java b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/GuessSchema.java
index 4bda9da85..ba39f8925 100644
--- a/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/GuessSchema.java
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/GuessSchema.java
@@ -22,18 +22,31 @@ import org.apache.streampipes.model.base.UnnamedStreamPipesEntity;
 import org.apache.streampipes.model.schema.EventSchema;
 import org.apache.streampipes.model.shared.annotation.TsModel;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 @TsModel
 public class GuessSchema extends UnnamedStreamPipesEntity {
 
     public EventSchema eventSchema;
 
+    public List<Map<String, GuessTypeInfo>> eventPreview;
+
+    public Map<String, FieldStatusInfo> fieldStatusInfo;
+
     public GuessSchema() {
         super();
+        this.eventPreview = new ArrayList<>();
+        this.fieldStatusInfo = new HashMap<>();
     }
 
     public GuessSchema(GuessSchema other) {
         super(other);
         this.eventSchema = other.getEventSchema() != null ? new EventSchema(other.getEventSchema()) : null;
+        this.eventPreview = other.getEventPreview();
+        this.fieldStatusInfo = other.getFieldStatusInfo();
     }
     public EventSchema getEventSchema() {
         return eventSchema;
@@ -43,4 +56,21 @@ public class GuessSchema extends UnnamedStreamPipesEntity {
         this.eventSchema = eventSchema;
     }
 
+    public List<Map<String, GuessTypeInfo>> getEventPreview() {
+        return eventPreview;
+    }
+
+    public void setEventPreview(List<Map<String, GuessTypeInfo>> eventPreview) {
+        this.eventPreview = eventPreview;
+    }
+
+    public Map<String, FieldStatusInfo> getFieldStatusInfo() {
+        return fieldStatusInfo;
+    }
+
+    public void setFieldStatusInfo(Map<String, FieldStatusInfo> fieldStatusInfo) {
+        this.fieldStatusInfo = fieldStatusInfo;
+    }
+
+
 }
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/GuessTypeInfo.java
similarity index 56%
copy from ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
copy to streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/GuessTypeInfo.java
index aa6f60f17..7404451d6 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/GuessTypeInfo.java
@@ -16,31 +16,38 @@
  *
  */
 
+package org.apache.streampipes.model.connect.guess;
 
-::ng-deep .mat-checkbox-checked.mat-accent .mat-checkbox-background, .mat-checkbox-indeterminate.mat-accent .mat-checkbox-background {
-    background-color: var(--color-accent) !important;
-}
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
 
-::ng-deep .mat-checkbox:not(.mat-checkbox-disabled).mat-accent .mat-checkbox-ripple .mat-ripple-element {
-    background-color: var(--color-accent) !important;
-}
+public class GuessTypeInfo {
 
-.checkbox-selected {
-    opacity: 1 !important;
-}
+  private String type;
 
-.timestamp-property {
-    margin-left: 15px;
-    border-radius: 10px;
-    background: var(--color-processor);
-    padding: 0px 10px;
-    font-size: 12px;
-    color: #FFFFFF;
-}
+  @JsonTypeInfo(use = JsonTypeInfo.Id.NONE, property = "type")
+  private Object value;
 
-.timestamp-icon {
-    font-size: 12px;
-}
+  public GuessTypeInfo() {
+  }
+
+  public GuessTypeInfo(String type, Object value) {
+    this.type = type;
+    this.value = value;
+  }
 
+  public String getType() {
+    return type;
+  }
 
+  public void setType(String type) {
+    this.type = type;
+  }
 
+  public Object getValue() {
+    return value;
+  }
+
+  public void setValue(Object value) {
+    this.value = value;
+  }
+}
diff --git a/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts b/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
index 24ebb69e4..7d282d6de 100644
--- a/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
+++ b/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
@@ -18,7 +18,7 @@
 /* tslint:disable */
 /* eslint-disable */
 // @ts-nocheck
-// Generated using typescript-generator version 2.27.744 on 2022-08-14 22:39:25.
+// Generated using typescript-generator version 2.27.744 on 2022-08-16 11:39:37.
 
 export class AbstractStreamPipesEntity {
     "@class": "org.apache.streampipes.model.base.AbstractStreamPipesEntity" | "org.apache.streampipes.model.base.NamedStreamPipesEntity" | "org.apache.streampipes.model.connect.adapter.AdapterDescription" | "org.apache.streampipes.model.connect.adapter.AdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.GenericAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.SpecificAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.AdapterStre [...]
@@ -192,9 +192,9 @@ export class AdapterDescription extends NamedStreamPipesEntity {
         instance.selectedEndpointUrl = data.selectedEndpointUrl;
         instance.correspondingServiceGroup = data.correspondingServiceGroup;
         instance.correspondingDataStreamElementId = data.correspondingDataStreamElementId;
+        instance.valueRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.valueRules);
         instance.streamRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.streamRules);
         instance.schemaRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.schemaRules);
-        instance.valueRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.valueRules);
         return instance;
     }
 
@@ -1592,6 +1592,23 @@ export class ExportItem {
     }
 }
 
+export class FieldStatusInfo {
+    additionalInfo: string;
+    changesRequired: boolean;
+    fieldStatus: FieldStatus;
+
+    static fromData(data: FieldStatusInfo, target?: FieldStatusInfo): FieldStatusInfo {
+        if (!data) {
+            return data;
+        }
+        const instance = target || new FieldStatusInfo();
+        instance.fieldStatus = data.fieldStatus;
+        instance.additionalInfo = data.additionalInfo;
+        instance.changesRequired = data.changesRequired;
+        return instance;
+    }
+}
+
 export class FileMetadata {
     createdAt: number;
     createdByUser: string;
@@ -1736,8 +1753,8 @@ export class GenericAdapterSetDescription extends AdapterSetDescription implemen
         const instance = target || new GenericAdapterSetDescription();
         super.fromData(data, instance);
         instance.eventSchema = EventSchema.fromData(data.eventSchema);
-        instance.formatDescription = FormatDescription.fromData(data.formatDescription);
         instance.protocolDescription = ProtocolDescription.fromData(data.protocolDescription);
+        instance.formatDescription = FormatDescription.fromData(data.formatDescription);
         return instance;
     }
 }
@@ -1755,15 +1772,17 @@ export class GenericAdapterStreamDescription extends AdapterStreamDescription im
         const instance = target || new GenericAdapterStreamDescription();
         super.fromData(data, instance);
         instance.eventSchema = EventSchema.fromData(data.eventSchema);
-        instance.formatDescription = FormatDescription.fromData(data.formatDescription);
         instance.protocolDescription = ProtocolDescription.fromData(data.protocolDescription);
+        instance.formatDescription = FormatDescription.fromData(data.formatDescription);
         return instance;
     }
 }
 
 export class GuessSchema extends UnnamedStreamPipesEntity {
     "@class": "org.apache.streampipes.model.connect.guess.GuessSchema";
+    eventPreview: { [index: string]: GuessTypeInfo }[];
     eventSchema: EventSchema;
+    fieldStatusInfo: { [index: string]: FieldStatusInfo };
 
     static fromData(data: GuessSchema, target?: GuessSchema): GuessSchema {
         if (!data) {
@@ -1772,6 +1791,23 @@ export class GuessSchema extends UnnamedStreamPipesEntity {
         const instance = target || new GuessSchema();
         super.fromData(data, instance);
         instance.eventSchema = EventSchema.fromData(data.eventSchema);
+        instance.eventPreview = __getCopyArrayFn(__getCopyObjectFn(GuessTypeInfo.fromData))(data.eventPreview);
+        instance.fieldStatusInfo = __getCopyObjectFn(FieldStatusInfo.fromData)(data.fieldStatusInfo);
+        return instance;
+    }
+}
+
+export class GuessTypeInfo {
+    type: string;
+    value: any;
+
+    static fromData(data: GuessTypeInfo, target?: GuessTypeInfo): GuessTypeInfo {
+        if (!data) {
+            return data;
+        }
+        const instance = target || new GuessTypeInfo();
+        instance.type = data.type;
+        instance.value = data.value;
         return instance;
     }
 }
@@ -2567,8 +2603,8 @@ export class PipelineTemplateDescription extends NamedStreamPipesEntity {
         const instance = target || new PipelineTemplateDescription();
         super.fromData(data, instance);
         instance.boundTo = __getCopyArrayFn(BoundPipelineElement.fromData)(data.boundTo);
-        instance.pipelineTemplateId = data.pipelineTemplateId;
         instance.pipelineTemplateName = data.pipelineTemplateName;
+        instance.pipelineTemplateId = data.pipelineTemplateId;
         instance.pipelineTemplateDescription = data.pipelineTemplateDescription;
         return instance;
     }
@@ -3384,6 +3420,8 @@ export type EventPropertyUnion = EventPropertyList | EventPropertyNested | Event
 
 export type EventStreamQualityDefinitionUnion = Frequency | Latency;
 
+export type FieldStatus = "GOOD" | "BAD" | "ATTENTION";
+
 export type MappingPropertyUnion = MappingPropertyNary | MappingPropertyUnary;
 
 export type MeasurementPropertyUnion = EventPropertyQualityDefinition | EventStreamQualityDefinition;
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html
index 47ac53898..be87eb7e2 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html
@@ -16,8 +16,8 @@
   ~
   -->
 
-<div fxLayout="row">
-    <div fxFlex="1 1 auto" fxLayout="row" fxLayoutAlign="start center">
+<div fxLayout="row" fxFlex="100" fxLayoutGap="10px">
+    <div fxFlex fxLayout="row" fxLayoutAlign="start center">
         <b>
             {{ label }}
         </b>
@@ -29,8 +29,20 @@
         </div>
         <p style="margin: 0px 10px 10px;" *ngIf="isList">[List]</p>
     </div>
+    <div fxFlex="15" fxLayoutAlign="start center" *ngIf="runtimeType">
+        <span class="runtime-type-info">{{runtimeType}}</span>
+    </div>
+    <div fxFlex="15" fxLayoutAlign="start center">
+        <span *ngIf="showEventPreview">{{eventPreview[0][node.data.runtimeName].value || 'n/a'}}</span>
+    </div>
+    <div fxFlex="15" fxLayoutAlign="start center">
+        <span *ngIf="showFieldStatus"
+              [ngClass]="'status status-' +fieldStatusInfo[node.data.runtimeName].fieldStatus.toLowerCase()"
+        [matTooltip]="fieldStatusInfo[node.data.runtimeName].additionalInfo">{{fieldStatusInfo[node.data.runtimeName].fieldStatus}}</span>
+    </div>
 
-    <div fxFlex="15" *ngIf="isPrimitive" fxLayoutAlign="center center">
+    <div fxLayout="row" fxLayoutAlign="end center">
+    <div fxFlex="15" *ngIf="isPrimitive" fxLayoutAlign="end center">
         <mat-form-field class="small-select" color="accent">
         <mat-select [(ngModel)]="node.data.propertyScope" panelClass="small-select"
                     [attr.data-cy]="'property-scope-' + label">
@@ -41,14 +53,14 @@
         </mat-form-field>
     </div>
 
-    <div fxFlex="0 1 auto" fxLayoutAlign="center center"
+    <div fxLayoutAlign="end center"
          *ngIf="isNested">
         <button [disabled]="!isEditable" color="accent" mat-button (click)=addNestedProperty(node.data)>
             <mat-icon matTooltip="Add a Nested Property">queue</mat-icon>
         </button>
     </div>
-    <div fxFlex="0 1 auto"
-         fxLayoutAlign="center center"
+    <div
+         fxLayoutAlign="end center"
          class="ml-5 mr-5"
          *ngIf="isNested || isPrimitive || isList">
         <button [disabled]="!isEditable" color="accent" mat-button
@@ -57,7 +69,7 @@
             <mat-icon>edit</mat-icon>&nbsp;Edit field
         </button>
     </div>
-    <div fxFlex="0 1 auto" fxLayoutAlign="center center">
+    <div fxLayoutAlign="end center">
         <mat-checkbox
                 *ngIf="isNested || isPrimitive || isList"
                 (click)="selectProperty(node.data.id, undefined)"
@@ -67,4 +79,5 @@
                 [attr.data-cy]="'delete-property-' + label.toLowerCase()">
         </mat-checkbox>
     </div>
+    </div>
 </div>
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
index aa6f60f17..c20307e9b 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
@@ -42,5 +42,30 @@
     font-size: 12px;
 }
 
+.status-good {
+    background: #75c575;
+    color: white;
+}
+
+.status-bad {
+    background: #d2a169;
+    color: white;
+}
+
+.status {
+    width: 60px;
+    text-align: center;
+    padding: 5px;
+    border-radius: 3px;
+}
+
+.runtime-type-info {
+    padding: 5px;
+    border-radius: 3px;
+    background: var(--color-bg-3);
+    color: var(--color-default-text);
+    width: 70px;
+    text-align: center;
+}
 
 
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.ts b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.ts
index c4fb32923..f2be21368 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.ts
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.ts
@@ -21,15 +21,18 @@ import { UUID } from 'angular2-uuid';
 import { TreeNode } from '@circlon/angular-tree-component';
 import { MatDialog } from '@angular/material/dialog';
 import {
-    EventProperty,
-    EventPropertyList,
-    EventPropertyNested,
-    EventPropertyPrimitive,
-    EventPropertyUnion,
-    EventSchema
+  EventProperty,
+  EventPropertyList,
+  EventPropertyNested,
+  EventPropertyPrimitive,
+  EventPropertyUnion,
+  EventSchema,
+  FieldStatusInfo,
+
 } from '@streampipes/platform-services';
 import { EditEventPropertyComponent } from '../../../../dialog/edit-event-property/edit-event-property.component';
 import { DialogService, PanelType } from '@streampipes/shared-ui';
+import { GuessTypeInfo } from '../../../../../../../projects/streampipes/platform-services/src/lib/model/gen/streampipes-model';
 
 @Component({
   selector: 'event-property-row',
@@ -42,6 +45,8 @@ export class EventPropertyRowComponent implements OnInit, OnChanges {
   @Input() isEditable = true;
   @Input() eventSchema: EventSchema = new EventSchema();
   @Input() countSelected: number;
+  @Input() eventPreview: Record<string, GuessTypeInfo>[];
+  @Input() fieldStatusInfo: Record<string, FieldStatusInfo>;
 
   @Output() isEditableChange = new EventEmitter<boolean>();
   @Output() eventSchemaChange = new EventEmitter<EventSchema>();
@@ -53,6 +58,10 @@ export class EventPropertyRowComponent implements OnInit, OnChanges {
   isNested = false;
   isList = false;
   timestampProperty = false;
+  showEventPreview = false;
+  showFieldStatus = false;
+
+  runtimeType: string;
 
   constructor(private dialog: MatDialog,
               private dialogService: DialogService) {
@@ -60,12 +69,18 @@ export class EventPropertyRowComponent implements OnInit, OnChanges {
   }
 
   ngOnInit() {
+    this.showFieldStatus = this.fieldStatusInfo && this.fieldStatusInfo[this.node.data.runtimeName] !== undefined;
+    this.showEventPreview = this.eventPreview && this.eventPreview.length > 0 && this.eventPreview[0][this.node.data.runtimeName] !== undefined;
     this.label = this.getLabel(this.node.data);
     this.isPrimitive = this.isEventPropertyPrimitive(this.node.data);
     this.isList = this.isEventPropertyList(this.node.data);
     this.isNested = this.isEventPropertyNested(this.node.data);
     this.timestampProperty = this.isTimestampProperty(this.node.data);
 
+    if (this.isPrimitive) {
+      this.runtimeType = this.parseType(this.node.data.runtimeType);
+    }
+
     if (!this.node.data.propertyScope) {
       this.node.data.propertyScope = 'MEASUREMENT_PROPERTY';
     }
@@ -74,6 +89,10 @@ export class EventPropertyRowComponent implements OnInit, OnChanges {
   ngOnChanges(changes: SimpleChanges): void {
   }
 
+  private parseType(runtimeType: string) {
+    return runtimeType.split('#')[1].toUpperCase();
+  }
+
   private isEventPropertyPrimitive(instance: EventProperty): boolean {
     return instance instanceof EventPropertyPrimitive;
   }
@@ -122,8 +141,8 @@ export class EventPropertyRowComponent implements OnInit, OnChanges {
     });
 
     dialogRef.afterClosed().subscribe(refresh => {
-        this.timestampProperty = this.isTimestampProperty(this.node.data);
-        this.refreshTreeEmitter.emit();
+      this.timestampProperty = this.isTimestampProperty(this.node.data);
+      this.refreshTreeEmitter.emit();
     });
   }
 
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
index 6e85d424f..8344671e5 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
@@ -18,7 +18,7 @@
 
 
 <div fxLayout="row" fxLayoutAlign="center" class="mt-20">
-    <div fxFlex="80" fxLayout="column">
+    <div fxFlex="90" fxLayout="column">
         <div fxLayout="column"
              fxFlex="100"
              fxLayoutAlign="start center"
@@ -73,10 +73,10 @@
 
                 <sp-error-message
                         [errorMessage]="errorMessage"
-                        *ngIf="isError">
+                        *ngIf="isError && !isLoading">
                 </sp-error-message>
 
-                <div *ngIf="!isError && !isLoading"
+                <div *ngIf="!isError && !isLoading && eventSchema"
                      fxLayout="column"
                      fxLayoutAlign="space-evenly stretched"
                      class="drag-drop-tree"
@@ -90,6 +90,8 @@
                                     [node]="node"
                                     [(isEditable)]="isEditable"
                                     [(eventSchema)]="eventSchema"
+                                    [eventPreview]="eventPreview"
+                                    [fieldStatusInfo]="fieldStatusInfo"
                                     (refreshTreeEmitter)="refreshTree()"
                                     [(countSelected)]="countSelected"></event-property-row>
                         </ng-template>
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
index 5ca2b4e58..d70a53048 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
@@ -32,7 +32,10 @@ import {
 } from '@streampipes/platform-services';
 import { MatStepper } from '@angular/material/stepper';
 import { UserErrorMessage } from '../../../../../core-model/base/UserErrorMessage';
-import { StreamPipesErrorMessage } from '../../../../../../../projects/streampipes/platform-services/src/lib/model/gen/streampipes-model';
+import {
+  FieldStatusInfo, GuessTypeInfo,
+  StreamPipesErrorMessage
+} from '../../../../../../../projects/streampipes/platform-services/src/lib/model/gen/streampipes-model';
 
 @Component({
   selector: 'sp-event-schema',
@@ -79,6 +82,9 @@ export class EventSchemaComponent implements OnChanges {
   validEventSchema = false;
   schemaErrorHints: UserErrorMessage[] = [];
 
+  eventPreview: Record<string, GuessTypeInfo>[];
+  fieldStatusInfo: Record<string, FieldStatusInfo>;
+
   options: ITreeOptions = {
     childrenField: 'eventProperties',
     allowDrag: () => {
@@ -99,6 +105,8 @@ export class EventSchemaComponent implements OnChanges {
     this.isLoading = true;
     this.isError = false;
     this.restService.getGuessSchema(this.adapterDescription).subscribe(guessSchema => {
+        this.eventPreview = guessSchema.eventPreview;
+        this.fieldStatusInfo = guessSchema.fieldStatusInfo;
         this.eventSchema = guessSchema.eventSchema;
         this.eventSchema.eventProperties.sort((a, b) => {
           return a.runtimeName < b.runtimeName ? -1 : 1;


[incubator-streampipes] 13/15: [hotfix] Remove comment

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 9d0feb50eb06359ef8522aecf990c691ac138345
Author: Philipp Zehnder <te...@users.noreply.github.com>
AuthorDate: Wed Aug 17 21:26:03 2022 +0200

    [hotfix] Remove comment
---
 .../org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java | 1 -
 1 file changed, 1 deletion(-)

diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java
index dfe84a105..76ecd9fbc 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java
@@ -38,7 +38,6 @@ import java.util.stream.Collectors;
 
 public class AdapterPipelineGenerator {
 
-  // TODO improve this code
   public AdapterPipeline generatePipeline(AdapterDescription adapterDescription) {
 
     var pipelineElements = makeAdapterPipelineElements(adapterDescription.getRules());


[incubator-streampipes] 06/15: [STREAMPIPES-577] Let JSON parser support adapter preview

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit dcdbe8abb21d56971f2e835edaff3c597ea79c68
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Wed Aug 17 08:23:54 2022 +0200

    [STREAMPIPES-577] Let JSON parser support adapter preview
---
 .../apache/streampipes/connect/api/IParser.java    |   9 +
 .../connect/api/exception/ParseException.java      |   6 +
 .../format/json/object/JsonObjectParser.java       | 135 +++---------
 .../connect/adapter/guess/SchemaGuesser.java       |  10 +-
 .../machine/MachineDataSimulatorUtils.java         | 240 +++++++++++----------
 .../connect/iiot/protocol/set/FileProtocol.java    |   2 +-
 .../connect/iiot/protocol/set/HttpProtocol.java    |   2 +-
 .../iiot/protocol/stream/BrokerProtocol.java       |   8 +-
 .../iiot/protocol/stream/FileStreamProtocol.java   |   2 +-
 .../iiot/protocol/stream/HttpStreamProtocol.java   |   2 +-
 .../model/connect/guess/AdapterGuessInfo.java      |  43 ++--
 .../sdk/builder/adapter/GuessSchemaBuilder.java    |  14 +-
 12 files changed, 230 insertions(+), 243 deletions(-)

diff --git a/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/IParser.java b/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/IParser.java
index 482bf9fa3..017adb658 100644
--- a/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/IParser.java
+++ b/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/IParser.java
@@ -19,6 +19,7 @@ package org.apache.streampipes.connect.api;
 
 import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.model.connect.grounding.FormatDescription;
+import org.apache.streampipes.model.connect.guess.AdapterGuessInfo;
 import org.apache.streampipes.model.schema.EventSchema;
 
 import java.io.InputStream;
@@ -39,4 +40,12 @@ public interface IParser {
    * @return
    */
   EventSchema getEventSchema(List<byte[]> oneEvent);
+
+  default boolean supportsPreview() {
+    return false;
+  }
+
+  default AdapterGuessInfo getSchemaAndSample(List<byte[]> eventSample) throws ParseException {
+    throw new RuntimeException("Not yet implemented!");
+  }
 }
diff --git a/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/exception/ParseException.java b/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/exception/ParseException.java
index 2064f2903..37c3ed376 100644
--- a/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/exception/ParseException.java
+++ b/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/exception/ParseException.java
@@ -19,10 +19,16 @@
 package org.apache.streampipes.connect.api.exception;
 
 public class ParseException extends RuntimeException {
+
     public ParseException() {}
 
     public ParseException(String message)
     {
         super(message);
     }
+
+    public ParseException(String message,
+                          Throwable throwable) {
+        super(message, throwable);
+    }
 }
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/json/object/JsonObjectParser.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/json/object/JsonObjectParser.java
index 3419bc90f..06c5cf5f2 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/json/object/JsonObjectParser.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/json/object/JsonObjectParser.java
@@ -20,24 +20,26 @@ package org.apache.streampipes.connect.adapter.format.json.object;
 
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.streampipes.commons.exceptions.SpRuntimeException;
-import org.apache.streampipes.connect.api.EmitBinaryEvent;
-import org.apache.streampipes.connect.adapter.model.generic.Parser;
 import org.apache.streampipes.connect.adapter.format.util.JsonEventProperty;
+import org.apache.streampipes.connect.adapter.model.generic.Parser;
+import org.apache.streampipes.connect.api.EmitBinaryEvent;
 import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.dataformat.json.JsonDataFormatDefinition;
 import org.apache.streampipes.model.connect.grounding.FormatDescription;
+import org.apache.streampipes.model.connect.guess.AdapterGuessInfo;
+import org.apache.streampipes.model.connect.guess.GuessTypeInfo;
 import org.apache.streampipes.model.schema.EventProperty;
 import org.apache.streampipes.model.schema.EventSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 
 public class JsonObjectParser extends Parser {
 
@@ -75,9 +77,17 @@ public class JsonObjectParser extends Parser {
 
     @Override
     public EventSchema getEventSchema(List<byte[]> oneEvent) {
-        EventSchema resultSchema = new EventSchema();
+        return getSchemaAndSample(oneEvent).getEventSchema();
+    }
+
+    @Override
+    public boolean supportsPreview() {
+        return true;
+    }
 
-//        resultSchema.setEventProperties(Arrays.asList(EpProperties.timestampProperty("timestamp")));
+    @Override
+    public AdapterGuessInfo getSchemaAndSample(List<byte[]> eventSample) throws ParseException {
+        EventSchema resultSchema = new EventSchema();
 
         JsonDataFormatDefinition jsonDefinition = new JsonDataFormatDefinition();
 
@@ -85,107 +95,22 @@ public class JsonObjectParser extends Parser {
         Map<String, Object> exampleEvent = null;
 
         try {
-            exampleEvent = jsonDefinition.toMap(oneEvent.get(0));
-        } catch (SpRuntimeException e) {
-            e.printStackTrace();
-        }
-
-        for (Map.Entry<String, Object> entry : exampleEvent.entrySet())
-        {
-//            System.out.println(entry.getKey() + "/" + entry.getValue());
-            EventProperty p = JsonEventProperty.getEventProperty(entry.getKey(), entry.getValue());
+            exampleEvent = jsonDefinition.toMap(eventSample.get(0));
+            var sample = exampleEvent
+              .entrySet()
+              .stream()
+              .collect(Collectors.toMap(Map.Entry::getKey, e ->
+                new GuessTypeInfo(e.getValue().getClass().getCanonicalName(), e.getValue())));
 
-            resultSchema.addEventProperty(p);
+            for (Map.Entry<String, Object> entry : exampleEvent.entrySet()) {
+                EventProperty p = JsonEventProperty.getEventProperty(entry.getKey(), entry.getValue());
 
-        }
-
-        return resultSchema;
-    }
-
-    public Map<String, Object> parseObject(javax.json.stream.JsonParser jsonParser, boolean root, int start) {
-        // this variable is needed to skip the first object start
-        String mapKey = "";
-        Map<String, Object> result = new HashMap<>();
-        List<Object> arr = null;
-
-        while (jsonParser.hasNext()) {
-            javax.json.stream.JsonParser.Event event = jsonParser.next();
-            switch (event) {
-                case KEY_NAME:
-                    mapKey = jsonParser.getString();
-                    logger.debug("key: " + mapKey );
-                    break;
-                case START_OBJECT:
-                    if (start == 0) {
-                        Map<String, Object> ob = parseObject(jsonParser, false, 0);
-                        if (arr == null) {
-                            result.put(mapKey, ob);
-                        } else {
-                            arr.add(ob);
-                        }
-                    } else {
-                        start--;
-                    }
-                    logger.debug("start object");
-                    break;
-                case END_OBJECT:
-
-                    logger.debug("end object");
-                    return result;
-                case START_ARRAY:
-                    arr = new ArrayList<>();
-                    logger.debug("start array");
-                    break;
-                case END_ARRAY:
-                    // Check if just the end of array is entered
-                    if (result.keySet().size() == 0 && mapKey.equals("")) {
-                        return null;
-                    }
-                    result.put(mapKey, arr);
-                    arr = null;
-                    logger.debug("end array");
-                    break;
-                case VALUE_TRUE:
-                    if (arr == null) {
-                        result.put(mapKey, true);
-                    } else {
-                        arr.add(true);
-                    }
-                    logger.debug("value: true");
-                    break;
-                case VALUE_FALSE:
-                    if (arr == null) {
-                        result.put(mapKey, false);
-                    } else {
-                        arr.add(false);
-                    }
-                    logger.debug("value: false");
-                    break;
-                case VALUE_STRING:
-                    if (arr == null) {
-                        result.put(mapKey, jsonParser.getString());
-                    } else {
-                        arr.add(jsonParser.getString());
-                    }
-                    logger.debug("value string: " + jsonParser.getString());
-                    break;
-                case VALUE_NUMBER:
-                    if (arr == null) {
-                        result.put(mapKey, jsonParser.getBigDecimal());
-                    } else {
-                        arr.add(jsonParser.getBigDecimal());
-                    }
-                    logger.debug("value number: " + jsonParser.getBigDecimal());
-                    break;
-                case VALUE_NULL:
-                    logger.debug("value null");
-                    break;
-                default:
-                    logger.error("Error: " + event + " event is not handled in the JSON parser");
-                    break;
+                resultSchema.addEventProperty(p);
             }
-        }
 
-        return result;
+            return new AdapterGuessInfo(resultSchema, sample);
+        } catch (SpRuntimeException e) {
+            throw new ParseException("Could not serialize event, did you choose the correct format?", e);
+        }
     }
 }
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/guess/SchemaGuesser.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/guess/SchemaGuesser.java
index 53be8abb2..29f534e97 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/guess/SchemaGuesser.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/guess/SchemaGuesser.java
@@ -18,16 +18,24 @@
 
 package org.apache.streampipes.connect.adapter.guess;
 
+import org.apache.streampipes.model.connect.guess.AdapterGuessInfo;
 import org.apache.streampipes.model.connect.guess.GuessSchema;
 import org.apache.streampipes.model.schema.EventSchema;
 
 public class SchemaGuesser {
 
-    public static GuessSchema guessSchma(EventSchema eventSchema) {
+    public static GuessSchema guessSchema(EventSchema eventSchema) {
         GuessSchema result = new GuessSchema();
 
         result.setEventSchema(eventSchema);
 
         return result;
     }
+
+    public static GuessSchema guessSchema(AdapterGuessInfo guessInfo) {
+        var result = guessSchema(guessInfo.getEventSchema());
+        result.setEventPreview(guessInfo.getEventPreview());
+
+        return result;
+    }
 }
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/simulator/machine/MachineDataSimulatorUtils.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/simulator/machine/MachineDataSimulatorUtils.java
index 0d7b2af3d..f157babca 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/simulator/machine/MachineDataSimulatorUtils.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/simulator/machine/MachineDataSimulatorUtils.java
@@ -30,123 +30,137 @@ import static org.apache.streampipes.sdk.helpers.EpProperties.*;
 
 public class MachineDataSimulatorUtils {
 
-    // Vocabulary
-    public static final String NS = "https://streampipes.org/vocabulary/examples/watertank/v1/";
-    public static final String HAS_SENSOR_ID = NS + "hasSensorId";
+  // Vocabulary
+  public static final String NS = "https://streampipes.org/vocabulary/examples/watertank/v1/";
+  public static final String HAS_SENSOR_ID = NS + "hasSensorId";
 
-    private static final String TIMESTAMP = "timestamp";
-    private static final String SENSOR_ID = "sensorId";
-    private static final String MASS_FLOW = "mass_flow";
-    private static final String TEMPERATURE = "temperature";
+  private static final String TIMESTAMP = "timestamp";
+  private static final String SENSOR_ID = "sensorId";
+  private static final String MASS_FLOW = "mass_flow";
+  private static final String TEMPERATURE = "temperature";
 
-    public static GuessSchema getSchema(String selectedSimulatorOption) throws AdapterException {
-        switch(selectedSimulatorOption) {
-            case "flowrate":
-                return getFlowrateSchema();
-            case "pressure":
-                return getPressureSchema();
-            case "waterlevel":
-                return getWaterlevelSchema();
-            default:
-                throw new AdapterException("resource not found");
-        }
+  public static GuessSchema getSchema(String selectedSimulatorOption) throws AdapterException {
+    switch (selectedSimulatorOption) {
+      case "flowrate":
+        return getFlowrateSchema();
+      case "pressure":
+        return getPressureSchema();
+      case "waterlevel":
+        return getWaterlevelSchema();
+      default:
+        throw new AdapterException("resource not found");
     }
+  }
 
-    private static GuessSchema getWaterlevelSchema() {
-        return GuessSchemaBuilder.create()
-                .property(timestampProperty(TIMESTAMP))
-                .property(PrimitivePropertyBuilder
-                        .create(Datatypes.String, "sensorId")
-                        .label("Sensor ID")
-                        .description("The ID of the sensor")
-                        .domainProperty(HAS_SENSOR_ID)
-                        .scope(PropertyScope.DIMENSION_PROPERTY)
-                        .build())
-                .property(PrimitivePropertyBuilder
-                        .create(Datatypes.Float, "level")
-                        .label("Water Level")
-                        .description("Denotes the current water level in the container")
-                        .domainProperty(SO.Number)
-                        .scope(PropertyScope.MEASUREMENT_PROPERTY)
-                        .build())
-                .property(PrimitivePropertyBuilder
-                        .create(Datatypes.Boolean, "overflow")
-                        .label("Overflow")
-                        .description("Indicates whether the tank overflows")
-                        .domainProperty(SO.Number)
-                        .scope(PropertyScope.MEASUREMENT_PROPERTY)
-                        .build())
-                .build();
-    }
+  private static GuessSchema getWaterlevelSchema() {
+    return GuessSchemaBuilder.create()
+      .property(timestampProperty(TIMESTAMP))
+      .sample(TIMESTAMP, System.currentTimeMillis())
+      .property(PrimitivePropertyBuilder
+        .create(Datatypes.String, "sensorId")
+        .label("Sensor ID")
+        .description("The ID of the sensor")
+        .domainProperty(HAS_SENSOR_ID)
+        .scope(PropertyScope.DIMENSION_PROPERTY)
+        .build())
+      .sample("sensorId", "sensor01")
+      .property(PrimitivePropertyBuilder
+        .create(Datatypes.Float, "level")
+        .label("Water Level")
+        .description("Denotes the current water level in the container")
+        .domainProperty(SO.Number)
+        .scope(PropertyScope.MEASUREMENT_PROPERTY)
+        .build())
+      .sample("level", 5.25f)
+      .property(PrimitivePropertyBuilder
+        .create(Datatypes.Boolean, "overflow")
+        .label("Overflow")
+        .description("Indicates whether the tank overflows")
+        .domainProperty(SO.Number)
+        .scope(PropertyScope.MEASUREMENT_PROPERTY)
+        .build())
+      .sample("overflow", true)
+      .build();
+  }
 
-    private static GuessSchema getPressureSchema() {
-        return GuessSchemaBuilder.create()
-                .property(timestampProperty(TIMESTAMP))
-                .property(PrimitivePropertyBuilder
-                        .create(Datatypes.String, "sensorId")
-                        .label("Sensor ID")
-                        .description("The ID of the sensor")
-                        .domainProperty(HAS_SENSOR_ID)
-                        .scope(PropertyScope.DIMENSION_PROPERTY)
-                        .build())
-                .property(PrimitivePropertyBuilder
-                        .create(Datatypes.Float, "pressure")
-                        .label("Pressure")
-                        .description("Denotes the current pressure in the pressure tank")
-                        .domainProperty(SO.Number)
-                        .valueSpecification(0.0f, 100.0f, 0.5f)
-                        .scope(PropertyScope.MEASUREMENT_PROPERTY)
-                        .build())
-                .build();
-    }
+  private static GuessSchema getPressureSchema() {
+    return GuessSchemaBuilder.create()
+      .property(timestampProperty(TIMESTAMP))
+      .sample(TIMESTAMP, System.currentTimeMillis())
+      .property(PrimitivePropertyBuilder
+        .create(Datatypes.String, "sensorId")
+        .label("Sensor ID")
+        .description("The ID of the sensor")
+        .domainProperty(HAS_SENSOR_ID)
+        .scope(PropertyScope.DIMENSION_PROPERTY)
+        .build())
+      .sample("sensorId", "sensor01")
+      .property(PrimitivePropertyBuilder
+        .create(Datatypes.Float, "pressure")
+        .label("Pressure")
+        .description("Denotes the current pressure in the pressure tank")
+        .domainProperty(SO.Number)
+        .valueSpecification(0.0f, 100.0f, 0.5f)
+        .scope(PropertyScope.MEASUREMENT_PROPERTY)
+        .build())
+      .sample("pressure", 85.22f)
+      .build();
+  }
 
-    public static GuessSchema getFlowrateSchema() {
-        return GuessSchemaBuilder.create()
-                .property(timestampProperty(TIMESTAMP))
-                .property(PrimitivePropertyBuilder
-                        .create(Datatypes.String, SENSOR_ID)
-                        .label("Sensor ID")
-                        .description("The ID of the sensor")
-                        .domainProperty(HAS_SENSOR_ID)
-                        .scope(PropertyScope.DIMENSION_PROPERTY)
-                        .build())
-                .property(PrimitivePropertyBuilder
-                        .create(Datatypes.Float, MASS_FLOW)
-                        .label("Mass Flow")
-                        .description("Denotes the current mass flow in the sensor")
-                        .domainProperty(SO.Number)
-                        .scope(PropertyScope.MEASUREMENT_PROPERTY)
-                        .build())
-                .property(PrimitivePropertyBuilder
-                        .create(Datatypes.Float, "volume_flow")
-                        .label("Volume Flow")
-                        .description("Denotes the current volume flow")
-                        .domainProperty(SO.Number)
-                        .scope(PropertyScope.MEASUREMENT_PROPERTY)
-                        .build())
-                .property(PrimitivePropertyBuilder
-                        .create(Datatypes.Float, TEMPERATURE)
-                        .label("Temperature")
-                        .description("Denotes the current temperature in degrees celsius")
-                        .domainProperty(SO.Number)
-                        .scope(PropertyScope.MEASUREMENT_PROPERTY)
-                        .measurementUnit(URI.create("http://codes.wmo.int/common/unit/degC"))
-                        .valueSpecification(0.0f, 100.0f, 0.1f)
-                        .build())
-                .property(PrimitivePropertyBuilder
-                        .create(Datatypes.Float, "density")
-                        .label("Density")
-                        .description("Denotes the current density of the fluid")
-                        .domainProperty(SO.Number)
-                        .scope(PropertyScope.MEASUREMENT_PROPERTY)
-                        .build())
-                .property(PrimitivePropertyBuilder
-                        .create(Datatypes.Boolean, "sensor_fault_flags")
-                        .label("Sensor Fault Flags")
-                        .description("Any fault flags of the sensors")
-                        .domainProperty(SO.Boolean)
-                        .scope(PropertyScope.MEASUREMENT_PROPERTY)
-                        .build())
-                .build();
-    }
+  public static GuessSchema getFlowrateSchema() {
+    return GuessSchemaBuilder.create()
+      .property(timestampProperty(TIMESTAMP))
+      .sample(TIMESTAMP, System.currentTimeMillis())
+      .property(PrimitivePropertyBuilder
+        .create(Datatypes.String, SENSOR_ID)
+        .label("Sensor ID")
+        .description("The ID of the sensor")
+        .domainProperty(HAS_SENSOR_ID)
+        .scope(PropertyScope.DIMENSION_PROPERTY)
+        .build())
+      .sample("sensorId", "sensor01")
+      .property(PrimitivePropertyBuilder
+        .create(Datatypes.Float, MASS_FLOW)
+        .label("Mass Flow")
+        .description("Denotes the current mass flow in the sensor")
+        .domainProperty(SO.Number)
+        .scope(PropertyScope.MEASUREMENT_PROPERTY)
+        .build())
+      .sample(MASS_FLOW, 5.76f)
+      .property(PrimitivePropertyBuilder
+        .create(Datatypes.Float, "volume_flow")
+        .label("Volume Flow")
+        .description("Denotes the current volume flow")
+        .domainProperty(SO.Number)
+        .scope(PropertyScope.MEASUREMENT_PROPERTY)
+        .build())
+      .sample("volume_flow", 3.34f)
+      .property(PrimitivePropertyBuilder
+        .create(Datatypes.Float, TEMPERATURE)
+        .label("Temperature")
+        .description("Denotes the current temperature in degrees celsius")
+        .domainProperty(SO.Number)
+        .scope(PropertyScope.MEASUREMENT_PROPERTY)
+        .measurementUnit(URI.create("http://codes.wmo.int/common/unit/degC"))
+        .valueSpecification(0.0f, 100.0f, 0.1f)
+        .build())
+      .sample(TEMPERATURE, 33.221f)
+      .property(PrimitivePropertyBuilder
+        .create(Datatypes.Float, "density")
+        .label("Density")
+        .description("Denotes the current density of the fluid")
+        .domainProperty(SO.Number)
+        .scope(PropertyScope.MEASUREMENT_PROPERTY)
+        .build())
+      .sample("density", 5.0f)
+      .property(PrimitivePropertyBuilder
+        .create(Datatypes.Boolean, "sensor_fault_flags")
+        .label("Sensor Fault Flags")
+        .description("Any fault flags of the sensors")
+        .domainProperty(SO.Boolean)
+        .scope(PropertyScope.MEASUREMENT_PROPERTY)
+        .build())
+      .sample("sensor_fault_flags", true)
+      .build();
+  }
 }
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/FileProtocol.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/FileProtocol.java
index 25976de44..31d4fc5bf 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/FileProtocol.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/FileProtocol.java
@@ -123,7 +123,7 @@ public class FileProtocol extends Protocol {
 
             EventSchema eventSchema = parser.getEventSchema(dataByte);
 
-            GuessSchema result = SchemaGuesser.guessSchma(eventSchema);
+            GuessSchema result = SchemaGuesser.guessSchema(eventSchema);
 
             return result;
         } catch (FileNotFoundException e) {
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/HttpProtocol.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/HttpProtocol.java
index 1feb4cceb..eca0f25f7 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/HttpProtocol.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/set/HttpProtocol.java
@@ -116,7 +116,7 @@ public class HttpProtocol extends Protocol {
 
         EventSchema eventSchema= parser.getEventSchema(dataByte);
 
-        GuessSchema result = SchemaGuesser.guessSchma(eventSchema);
+        GuessSchema result = SchemaGuesser.guessSchema(eventSchema);
 
         return result;
     }
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/BrokerProtocol.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/BrokerProtocol.java
index 835fffc72..e34d7f941 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/BrokerProtocol.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/BrokerProtocol.java
@@ -23,7 +23,6 @@ import org.apache.streampipes.connect.api.IFormat;
 import org.apache.streampipes.connect.api.IParser;
 import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.model.connect.guess.GuessSchema;
-import org.apache.streampipes.model.schema.EventSchema;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -48,9 +47,12 @@ public abstract class BrokerProtocol extends Protocol {
   public GuessSchema getGuessSchema() throws ParseException {
 
     List<byte[]> eventByte = getNByteElements(1);
-    EventSchema eventSchema = parser.getEventSchema(eventByte);
 
-    return SchemaGuesser.guessSchma(eventSchema);
+    if (parser.supportsPreview()) {
+      return SchemaGuesser.guessSchema(parser.getSchemaAndSample(eventByte));
+    } else {
+      return SchemaGuesser.guessSchema(parser.getEventSchema(eventByte));
+    }
   }
 
   @Override
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/FileStreamProtocol.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/FileStreamProtocol.java
index 4b9368b0b..d82d28ce9 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/FileStreamProtocol.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/FileStreamProtocol.java
@@ -213,7 +213,7 @@ public class FileStreamProtocol extends Protocol {
 
     EventSchema eventSchema = parser.getEventSchema(dataByte);
 
-    GuessSchema result = SchemaGuesser.guessSchma(eventSchema);
+    GuessSchema result = SchemaGuesser.guessSchema(eventSchema);
 
     return result;
   }
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/HttpStreamProtocol.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/HttpStreamProtocol.java
index 39a5dad26..be6882957 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/HttpStreamProtocol.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/HttpStreamProtocol.java
@@ -112,7 +112,7 @@ public class HttpStreamProtocol extends PullProtocol {
             dataByte.addAll(dataByte);
         }
         EventSchema eventSchema= parser.getEventSchema(dataByte);
-        GuessSchema result = SchemaGuesser.guessSchma(eventSchema);
+        GuessSchema result = SchemaGuesser.guessSchema(eventSchema);
 
         return result;
     }
diff --git a/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/IParser.java b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/AdapterGuessInfo.java
similarity index 51%
copy from streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/IParser.java
copy to streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/AdapterGuessInfo.java
index 482bf9fa3..6aa137fbd 100644
--- a/streampipes-connect-api/src/main/java/org/apache/streampipes/connect/api/IParser.java
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/AdapterGuessInfo.java
@@ -15,28 +15,41 @@
  * limitations under the License.
  *
  */
-package org.apache.streampipes.connect.api;
 
-import org.apache.streampipes.connect.api.exception.ParseException;
-import org.apache.streampipes.model.connect.grounding.FormatDescription;
+package org.apache.streampipes.model.connect.guess;
+
 import org.apache.streampipes.model.schema.EventSchema;
 
-import java.io.InputStream;
 import java.util.List;
+import java.util.Map;
+
+public class AdapterGuessInfo {
+
+  private EventSchema eventSchema;
+  private List<Map<String, GuessTypeInfo>> eventPreview;
+
+  public AdapterGuessInfo() {
+  }
 
-public interface IParser {
+  public AdapterGuessInfo(EventSchema eventSchema,
+                          Map<String, GuessTypeInfo> singleEventSample) {
+    this.eventSchema = eventSchema;
+    this.eventPreview = List.of(singleEventSample);
+  }
 
-  IParser getInstance(FormatDescription formatDescription);
+  public EventSchema getEventSchema() {
+    return eventSchema;
+  }
 
-  void parse(InputStream data, EmitBinaryEvent emitBinaryEvent) throws ParseException;
+  public void setEventSchema(EventSchema eventSchema) {
+    this.eventSchema = eventSchema;
+  }
 
-  List<byte[]> parseNEvents(InputStream data, int n) throws ParseException;
+  public List<Map<String, GuessTypeInfo>> getEventPreview() {
+    return eventPreview;
+  }
 
-  /**
-   * Pass one event to Parser to get the event schema
-   *
-   * @param oneEvent
-   * @return
-   */
-  EventSchema getEventSchema(List<byte[]> oneEvent);
+  public void setEventPreview(List<Map<String, GuessTypeInfo>> eventPreview) {
+    this.eventPreview = eventPreview;
+  }
 }
diff --git a/streampipes-sdk/src/main/java/org/apache/streampipes/sdk/builder/adapter/GuessSchemaBuilder.java b/streampipes-sdk/src/main/java/org/apache/streampipes/sdk/builder/adapter/GuessSchemaBuilder.java
index 91e0da129..8f88a54eb 100644
--- a/streampipes-sdk/src/main/java/org/apache/streampipes/sdk/builder/adapter/GuessSchemaBuilder.java
+++ b/streampipes-sdk/src/main/java/org/apache/streampipes/sdk/builder/adapter/GuessSchemaBuilder.java
@@ -18,18 +18,20 @@
 package org.apache.streampipes.sdk.builder.adapter;
 
 import org.apache.streampipes.model.connect.guess.GuessSchema;
+import org.apache.streampipes.model.connect.guess.GuessTypeInfo;
 import org.apache.streampipes.model.schema.EventProperty;
 import org.apache.streampipes.model.schema.EventSchema;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.util.*;
 
 public class GuessSchemaBuilder {
 
   private List<EventProperty> eventProperties;
+  private Map<String, GuessTypeInfo> samples;
 
   private GuessSchemaBuilder() {
     this.eventProperties = new ArrayList<>();
+    this.samples = new HashMap<>();
   }
 
   /**
@@ -39,6 +41,13 @@ public class GuessSchemaBuilder {
     return new GuessSchemaBuilder();
   }
 
+  public GuessSchemaBuilder sample(String runtimeName,
+                                   Object sampleValue) {
+    this.samples.put(runtimeName, new GuessTypeInfo(sampleValue.getClass().getCanonicalName(), sampleValue));
+
+    return this;
+  }
+
   public GuessSchemaBuilder property(EventProperty property) {
     this.eventProperties.add(property);
 
@@ -56,6 +65,7 @@ public class GuessSchemaBuilder {
     eventSchema.setEventProperties(eventProperties);
 
     guessSchema.setEventSchema(eventSchema);
+    guessSchema.setEventPreview(List.of(this.samples));
 
     return guessSchema;
   }


[incubator-streampipes] 08/15: [STREAMPIPES-577] Improve error handling of adapters

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit df098481207c92101e1deeb0dee6c834ec568836
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Wed Aug 17 11:50:15 2022 +0200

    [STREAMPIPES-577] Improve error handling of adapters
---
 .../master/management/GuessManagement.java         | 12 +++----
 .../worker/management/GuessManagement.java         |  2 +-
 .../container/worker/rest/GuessResource.java       | 10 +++---
 .../iiot/adapters/plc4x/s7/Plc4xS7Adapter.java     |  2 +-
 .../iiot/protocol/stream/KafkaProtocol.java        | 41 ++++++++++++++--------
 .../event-schema/event-schema.component.ts         |  2 +-
 .../static-property.component.html                 |  2 ++
 ...c-runtime-resolvable-oneof-input.component.html | 24 +++----------
 ...tic-runtime-resolvable-oneof-input.component.ts | 15 ++++++++
 9 files changed, 62 insertions(+), 48 deletions(-)

diff --git a/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java b/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java
index ce380864c..de66c5449 100644
--- a/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java
+++ b/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java
@@ -26,6 +26,7 @@ import org.apache.http.client.fluent.Response;
 import org.apache.http.entity.ContentType;
 import org.apache.http.util.EntityUtils;
 import org.apache.streampipes.commons.exceptions.NoServiceEndpointsAvailableException;
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.connect.adapter.model.pipeline.AdapterEventPreviewPipeline;
 import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.connect.api.exception.WorkerAdapterException;
@@ -34,7 +35,6 @@ import org.apache.streampipes.model.connect.adapter.AdapterDescription;
 import org.apache.streampipes.model.connect.guess.AdapterEventPreview;
 import org.apache.streampipes.model.connect.guess.GuessSchema;
 import org.apache.streampipes.model.connect.guess.GuessTypeInfo;
-import org.apache.streampipes.model.message.ErrorMessage;
 import org.apache.streampipes.serializers.json.JacksonSerializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -69,12 +69,10 @@ public class GuessManagement {
             String responseString = EntityUtils.toString(httpResponse.getEntity());
 
             if (httpResponse.getStatusLine().getStatusCode() == HttpStatus.SC_OK) {
-                return mapper.readValue(responseString, GuessSchema.class);
-            }  else {
-                ErrorMessage errorMessage = mapper.readValue(responseString, ErrorMessage.class);
-
-                LOG.error(errorMessage.getElementName());
-                throw new WorkerAdapterException(errorMessage);
+              return mapper.readValue(responseString, GuessSchema.class);
+            } else {
+              var exception = mapper.readValue(responseString, SpConfigurationException.class);
+              throw new WorkerAdapterException(exception.getMessage(), exception.getCause());
             }
     }
 
diff --git a/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/management/GuessManagement.java b/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/management/GuessManagement.java
index 41f81def3..9bc96f9b2 100644
--- a/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/management/GuessManagement.java
+++ b/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/management/GuessManagement.java
@@ -61,7 +61,7 @@ public class GuessManagement {
             throw new ParseException(errorClass + e.getMessage());
         } catch (Exception e) {
             LOG.error("Unknown Error: " + e.toString());
-            throw new AdapterException(e.toString());
+            throw new AdapterException(e.getMessage(), e);
         }
 
         return guessSchema;
diff --git a/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/rest/GuessResource.java b/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/rest/GuessResource.java
index 39d87840c..c357a5493 100644
--- a/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/rest/GuessResource.java
+++ b/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/rest/GuessResource.java
@@ -18,11 +18,11 @@
 
 package org.apache.streampipes.connect.container.worker.rest;
 
+import org.apache.streampipes.connect.api.exception.AdapterException;
 import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.connect.container.worker.management.GuessManagement;
 import org.apache.streampipes.model.connect.adapter.AdapterDescription;
 import org.apache.streampipes.model.connect.guess.GuessSchema;
-import org.apache.streampipes.model.message.Notifications;
 import org.apache.streampipes.rest.shared.annotation.JacksonSerialized;
 import org.apache.streampipes.rest.shared.impl.AbstractSharedRestInterface;
 import org.slf4j.Logger;
@@ -62,10 +62,10 @@ public class GuessResource extends AbstractSharedRestInterface {
           return ok(result);
       } catch (ParseException e) {
           logger.error("Error while parsing events: ", e);
-          return serverError(Notifications.error(e.getMessage()));
-      } catch (Exception e) {
-          logger.error("Error while guess schema for AdapterDescription: " + adapterDescription.getElementId(), e);
-          return serverError(Notifications.error(e.getMessage()));
+          return serverError(e);
+      } catch (AdapterException e) {
+          logger.error("Error while guessing schema for AdapterDescription: {}, {}", adapterDescription.getElementId(), e.getMessage());
+          return serverError(e);
       }
 
   }
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/Plc4xS7Adapter.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/Plc4xS7Adapter.java
index 04ece8aaa..2ee301ab1 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/Plc4xS7Adapter.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/Plc4xS7Adapter.java
@@ -136,7 +136,7 @@ public class Plc4xS7Adapter extends PullAdapter {
         getConfigurations(adapterDescription);
 
         if (this.pollingInterval < 10) {
-            throw new AdapterException("Polling interval must be higher then 10. Current value: " + this.pollingInterval);
+            throw new AdapterException("Polling interval must be higher than 10. Current value: " + this.pollingInterval);
         }
 
         GuessSchema guessSchema = new GuessSchema();
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/KafkaProtocol.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/KafkaProtocol.java
index 502aa7ce7..cdc8cb492 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/KafkaProtocol.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/protocol/stream/KafkaProtocol.java
@@ -20,9 +20,11 @@ package org.apache.streampipes.connect.iiot.protocol.stream;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.kafka.clients.consumer.*;
+import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
 import org.apache.streampipes.commons.constants.GlobalStreamPipesConstants;
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.commons.exceptions.SpRuntimeException;
 import org.apache.streampipes.connect.SendToPipeline;
 import org.apache.streampipes.connect.adapter.model.generic.Protocol;
@@ -30,7 +32,7 @@ import org.apache.streampipes.connect.api.IAdapterPipeline;
 import org.apache.streampipes.connect.api.IFormat;
 import org.apache.streampipes.connect.api.IParser;
 import org.apache.streampipes.connect.api.exception.ParseException;
-import org.apache.streampipes.container.api.ResolvesContainerProvidedOptions;
+import org.apache.streampipes.container.api.SupportsRuntimeConfig;
 import org.apache.streampipes.messaging.InternalEventProcessor;
 import org.apache.streampipes.messaging.kafka.SpKafkaConsumer;
 import org.apache.streampipes.model.AdapterType;
@@ -38,6 +40,8 @@ import org.apache.streampipes.model.connect.grounding.ProtocolDescription;
 import org.apache.streampipes.model.grounding.KafkaTransportProtocol;
 import org.apache.streampipes.model.grounding.SimpleTopicDefinition;
 import org.apache.streampipes.model.staticproperty.Option;
+import org.apache.streampipes.model.staticproperty.RuntimeResolvableOneOfStaticProperty;
+import org.apache.streampipes.model.staticproperty.StaticProperty;
 import org.apache.streampipes.pe.shared.config.kafka.KafkaConfig;
 import org.apache.streampipes.pe.shared.config.kafka.KafkaConnectUtils;
 import org.apache.streampipes.sdk.builder.adapter.ProtocolDescriptionBuilder;
@@ -45,7 +49,6 @@ import org.apache.streampipes.sdk.extractor.StaticPropertyExtractor;
 import org.apache.streampipes.sdk.helpers.AdapterSourceType;
 import org.apache.streampipes.sdk.helpers.Labels;
 import org.apache.streampipes.sdk.helpers.Locales;
-import org.apache.streampipes.sdk.helpers.Options;
 import org.apache.streampipes.sdk.utils.Assets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -55,7 +58,7 @@ import java.io.InputStream;
 import java.util.*;
 import java.util.stream.Collectors;
 
-public class KafkaProtocol extends BrokerProtocol implements ResolvesContainerProvidedOptions {
+public class KafkaProtocol extends BrokerProtocol implements SupportsRuntimeConfig {
 
     Logger logger = LoggerFactory.getLogger(KafkaProtocol.class);
     KafkaConfig config;
@@ -154,7 +157,7 @@ public class KafkaProtocol extends BrokerProtocol implements ResolvesContainerPr
         return resultEventsByte;
     }
 
-    private Consumer<byte[], byte[]> createConsumer(KafkaConfig kafkaConfig) {
+    private Consumer<byte[], byte[]> createConsumer(KafkaConfig kafkaConfig) throws KafkaException {
         final Properties props = new Properties();
 
         kafkaConfig.getSecurityConfig().appendConfig(props);
@@ -165,6 +168,8 @@ public class KafkaProtocol extends BrokerProtocol implements ResolvesContainerPr
         props.put(ConsumerConfig.GROUP_ID_CONFIG,
                 "KafkaExampleConsumer" + System.currentTimeMillis());
 
+        props.put(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, 6000);
+
         props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
                 ByteArrayDeserializer.class.getName());
         props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
@@ -214,22 +219,30 @@ public class KafkaProtocol extends BrokerProtocol implements ResolvesContainerPr
     }
 
     @Override
-    public List<Option> resolveOptions(String requestId, StaticPropertyExtractor extractor) {
+    public StaticProperty resolveConfiguration(String staticPropertyInternalName, StaticPropertyExtractor extractor) throws SpConfigurationException {
+        RuntimeResolvableOneOfStaticProperty config = extractor
+          .getStaticPropertyByName(KafkaConnectUtils.TOPIC_KEY, RuntimeResolvableOneOfStaticProperty.class);
         KafkaConfig kafkaConfig = KafkaConnectUtils.getConfig(extractor, false);
         boolean hideInternalTopics = extractor.slideToggleValue(KafkaConnectUtils.getHideInternalTopicsKey());
 
-        Consumer<byte[], byte[]> consumer = createConsumer(kafkaConfig);
+        try {
+            Consumer<byte[], byte[]> consumer = createConsumer(kafkaConfig);
+            Set<String> topics = consumer.listTopics().keySet();
+            consumer.close();
+
+            if (hideInternalTopics) {
+                topics = topics
+                  .stream()
+                  .filter(t -> !t.startsWith(GlobalStreamPipesConstants.INTERNAL_TOPIC_PREFIX))
+                  .collect(Collectors.toSet());
+            }
 
-        Set<String> topics = consumer.listTopics().keySet();
-        consumer.close();
+            config.setOptions(topics.stream().map(Option::new).collect(Collectors.toList()));
 
-        if (hideInternalTopics) {
-            topics = topics
-                    .stream()
-                    .filter(t -> !t.startsWith(GlobalStreamPipesConstants.INTERNAL_TOPIC_PREFIX))
-                    .collect(Collectors.toSet());
+            return config;
+        } catch (KafkaException e) {
+            throw new SpConfigurationException(e.getMessage(), e);
         }
-        return topics.stream().map(Option::new).collect(Collectors.toList());
     }
 
 
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
index e7867d91f..76d9bdaf1 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
@@ -129,7 +129,7 @@ export class EventSchemaComponent implements OnChanges {
         this.isEditableChange.emit(true);
         this.isLoading = false;
 
-        if (guessSchema.eventPreview) {
+        if (guessSchema.eventPreview && guessSchema.eventPreview.length > 0) {
           this.updatePreview();
         }
       },
diff --git a/ui/src/app/core-ui/static-properties/static-property.component.html b/ui/src/app/core-ui/static-properties/static-property.component.html
index 308ce518d..c7d89676f 100644
--- a/ui/src/app/core-ui/static-properties/static-property.component.html
+++ b/ui/src/app/core-ui/static-properties/static-property.component.html
@@ -68,6 +68,7 @@
                                                      [staticProperties]="staticProperties"
                                                      [eventSchemas]="eventSchemas"
                                                      [pipelineElement]="pipelineElement"
+                                                     [parentForm]="parentForm"
                                                      [completedStaticProperty]="completedStaticProperty"
                                                      [adapterId]="adapterId">
             </app-static-runtime-resolvable-any-input>
@@ -77,6 +78,7 @@
                                                        [pipelineElement]="pipelineElement"
                                                        [eventSchemas]="eventSchemas"
                                                        [staticProperty]="staticProperty"
+                                                       [parentForm]="parentForm"
                                                        [staticProperties]="staticProperties"
                                                        [adapterId]="adapterId"></app-static-runtime-resolvable-oneof-input>
 
diff --git a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.html b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.html
index 265866cf3..3f1ff1076 100644
--- a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.html
+++ b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.html
@@ -16,7 +16,7 @@
   ~
   -->
 
-<div id="formWrapper" fxFlex="100" fxLayout="column">
+<div [formGroup]="parentForm" id="formWrapper" fxFlex="100" fxLayout="column">
     <div>
         <button mat-button mat-raised-button color="accent" class="small-button"
                 data-cy="sp-reload"
@@ -26,7 +26,10 @@
             <span>Reload</span>
         </button>
     </div>
-    <div *ngIf="!staticProperty.horizontalRendering" fxLayout="column">
+    <div fxLayout="column" *ngIf="error" class="mt-10">
+        <sp-exception-message [message]="errorMessage"></sp-exception-message>
+    </div>
+    <div *ngIf="!loading" fxLayout="column">
         <div fxFlex fxLayout="row">
             <div fxLayout="column" *ngIf="showOptions || staticProperty.options" style="margin-left: 10px">
                 <mat-radio-button *ngFor="let option of staticProperty.options"
@@ -49,21 +52,4 @@
             </div>
         </div>
     </div>
-
-    <div *ngIf="staticProperty.horizontalRendering">
-        <mat-form-field style="width: 100%">
-            <mat-label>{{staticProperty.label}}</mat-label>
-            <span matPrefix *ngIf="loading"><mat-spinner style="top:5px" [diameter]="20"></mat-spinner></span>
-            <mat-select>
-                <mat-option *ngFor="let option of staticProperty.options" [value]="option.elementId"
-                            (click)="select(option.elementId)">
-                    <label style="font-weight: normal">
-                        {{option.name}}
-                    </label>
-                </mat-option>
-            </mat-select>
-            <mat-hint>{{staticProperty.description}}</mat-hint>
-        </mat-form-field>
-    </div>
-
 </div>
diff --git a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.ts b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.ts
index cae01d7c1..f5b973c11 100644
--- a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.ts
+++ b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.ts
@@ -20,6 +20,7 @@ import { Component, OnChanges, OnInit } from '@angular/core';
 import { RuntimeResolvableOneOfStaticProperty, StaticPropertyUnion } from '@streampipes/platform-services';
 import { RuntimeResolvableService } from '../static-runtime-resolvable-input/runtime-resolvable.service';
 import { BaseRuntimeResolvableSelectionInput } from '../static-runtime-resolvable-input/base-runtime-resolvable-selection-input';
+import { FormControl } from '@angular/forms';
 
 @Component({
     selector: 'app-static-runtime-resolvable-oneof-input',
@@ -35,6 +36,8 @@ export class StaticRuntimeResolvableOneOfInputComponent
 
     ngOnInit() {
         super.onInit();
+        this.parentForm.addControl(this.staticProperty.internalName, new FormControl(this.staticProperty.options, []));
+        this.performValidation();
     }
 
     afterOptionsLoaded(staticProperty: RuntimeResolvableOneOfStaticProperty) {
@@ -49,6 +52,7 @@ export class StaticRuntimeResolvableOneOfInputComponent
             option.selected = false;
         }
         this.staticProperty.options.find(option => option.elementId === id).selected = true;
+        this.performValidation();
     }
 
     parse(staticProperty: StaticPropertyUnion): RuntimeResolvableOneOfStaticProperty {
@@ -56,5 +60,16 @@ export class StaticRuntimeResolvableOneOfInputComponent
     }
 
     afterErrorReceived() {
+        this.staticProperty.options = [];
+        this.performValidation();
+    }
+
+    performValidation() {
+        let error = {error: true};
+        if (this.staticProperty.options && this.staticProperty.options.find(o => o.selected) !== undefined) {
+            error = undefined;
+        }
+        console.log(error);
+        this.parentForm.controls[this.staticProperty.internalName].setErrors(error);
     }
 }


[incubator-streampipes] 10/15: [STREAMPIPES-577] Add missing header

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 370962efb177a8b082d4159ae61dc29601eb9341
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Wed Aug 17 13:29:30 2022 +0200

    [STREAMPIPES-577] Add missing header
---
 .../iiot/adapters/plc4x/s7/PlcReadResponseHandler.java | 18 ++++++++++++++++++
 1 file changed, 18 insertions(+)

diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/PlcReadResponseHandler.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/PlcReadResponseHandler.java
index 071f27c5c..b9b8f1bfc 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/PlcReadResponseHandler.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/PlcReadResponseHandler.java
@@ -1,3 +1,21 @@
+/*
+ * 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.connect.iiot.adapters.plc4x.s7;
 
 import org.apache.plc4x.java.api.messages.PlcReadResponse;


[incubator-streampipes] 15/15: [STREAMPIPES-577] Improve handling of unknown data types

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit fbe88ba3fdfabcf2f5e75b7add7e6463056b1031
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Thu Aug 18 10:25:19 2022 +0200

    [STREAMPIPES-577] Improve handling of unknown data types
---
 .../connect/adapter/format/csv/CsvParser.java      |  4 ++--
 .../adapter/format/util/JsonEventProperty.java     |  6 +----
 .../connect/adapter/util/DatatypeUtils.java        | 28 +++++++++++++++-------
 .../dataexplorer/commons/influx/InfluxStore.java   | 12 ++++++++--
 .../storage/couchdb/serializer/GsonSerializer.java |  1 +
 5 files changed, 33 insertions(+), 18 deletions(-)

diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java
index 812c67b7f..0b3e6be6d 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/csv/CsvParser.java
@@ -108,11 +108,11 @@ public class CsvParser extends Parser {
         EventSchema resultSchema = new EventSchema();
         for (int i = 0; i < keys.length; i++) {
             EventPropertyPrimitive p = new EventPropertyPrimitive();
-            var runtimeType = DatatypeUtils.getXsdDatatype(data[i]);
+            var runtimeType = DatatypeUtils.getXsdDatatype(data[i], true);
             var convertedValue = DatatypeUtils.convertValue(data[i], runtimeType);
             p.setRuntimeName(keys[i]);
             p.setRuntimeType(runtimeType);
-            sample.put(keys[i], new GuessTypeInfo(DatatypeUtils.getCanonicalTypeClassName(data[i]), convertedValue));
+            sample.put(keys[i], new GuessTypeInfo(DatatypeUtils.getCanonicalTypeClassName(data[i], true), convertedValue));
             resultSchema.addEventProperty(p);
         }
 
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/util/JsonEventProperty.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/util/JsonEventProperty.java
index b42711bc4..a20319075 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/util/JsonEventProperty.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/format/util/JsonEventProperty.java
@@ -55,11 +55,7 @@ public class JsonEventProperty {
             resultProperty = new EventPropertyPrimitive();
             resultProperty.setRuntimeName(key);
             ((EventPropertyPrimitive) resultProperty).setRuntimeType(XSD._string.toString());
-        } else if (o.getClass().equals(Long.class)) {
-            resultProperty = new EventPropertyPrimitive();
-            resultProperty.setRuntimeName(key);
-            ((EventPropertyPrimitive) resultProperty).setRuntimeType(XSD._long.toString());
-        } else if (o.getClass().equals(Integer.class) || o.getClass().equals(Double.class) || o.getClass().equals(Float.class)) {
+        } else if (o.getClass().equals(Integer.class) || o.getClass().equals(Double.class) || o.getClass().equals(Float.class) || o.getClass().equals(Long.class)) {
             resultProperty = new EventPropertyPrimitive();
             resultProperty.setRuntimeName(key);
             ((EventPropertyPrimitive) resultProperty).setRuntimeType(XSD._float.toString());
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/DatatypeUtils.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/DatatypeUtils.java
index 9261beee3..f87bab36e 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/DatatypeUtils.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/util/DatatypeUtils.java
@@ -42,10 +42,10 @@ public class DatatypeUtils {
           return Boolean.parseBoolean(stringValue);
         } else if (XSD._integer.toString().equals(targetDatatypeXsd)) {
           var floatingNumber = Float.parseFloat(stringValue);
-          return Math.round(floatingNumber);
+          return Integer.parseInt(String.valueOf(Math.round(floatingNumber)));
         } else if (XSD._long.toString().equals(targetDatatypeXsd)) {
           var floatingNumber = Double.parseDouble(stringValue);
-          return Math.round(floatingNumber);
+          return Long.parseLong(String.valueOf(Math.round(floatingNumber)));
         }
       } catch (NumberFormatException e) {
         LOG.error("Number format exception {}", value);
@@ -56,12 +56,14 @@ public class DatatypeUtils {
     return value;
   }
 
-  public static String getCanonicalTypeClassName(String value) {
-    return getTypeClass(value).getCanonicalName();
+  public static String getCanonicalTypeClassName(String value,
+                                                 boolean preferFloat) {
+    return getTypeClass(value, preferFloat).getCanonicalName();
   }
 
-  public static String getXsdDatatype(String value) {
-    var clazz = getTypeClass(value);
+  public static String getXsdDatatype(String value,
+                                      boolean preferFloat) {
+    var clazz = getTypeClass(value, preferFloat);
     if (clazz.equals(Integer.class)) {
       return XSD._integer.toString();
     } else if (clazz.equals(Long.class)) {
@@ -77,17 +79,18 @@ public class DatatypeUtils {
     }
   }
 
-  public static Class<?> getTypeClass(String value) {
+  public static Class<?> getTypeClass(String value,
+                                      boolean preferFloat) {
     if (NumberUtils.isParsable(value)) {
       try {
         Integer.parseInt(value);
-        return Integer.class;
+        return preferFloat ? Float.class : Integer.class;
       } catch (NumberFormatException ignored) {
       }
 
       try {
         Long.parseLong(value);
-        return Long.class;
+        return preferFloat ? Float.class : Long.class;
       } catch (NumberFormatException ignored) {
       }
 
@@ -105,4 +108,11 @@ public class DatatypeUtils {
 
     return String.class;
   }
+
+  public static void main(String[] args) {
+    long max = Long.MAX_VALUE;
+    String className = getCanonicalTypeClassName(String.valueOf(max), true);
+    System.out.println(className);
+    System.out.println(convertValue(max, className));
+  }
 }
diff --git a/streampipes-data-explorer-commons/src/main/java/org/apache/streampipes/dataexplorer/commons/influx/InfluxStore.java b/streampipes-data-explorer-commons/src/main/java/org/apache/streampipes/dataexplorer/commons/influx/InfluxStore.java
index cb429fee8..d0bfb7928 100644
--- a/streampipes-data-explorer-commons/src/main/java/org/apache/streampipes/dataexplorer/commons/influx/InfluxStore.java
+++ b/streampipes-data-explorer-commons/src/main/java/org/apache/streampipes/dataexplorer/commons/influx/InfluxStore.java
@@ -152,7 +152,11 @@ public class InfluxStore {
                             // Store property according to property type
                             String runtimeType = ((EventPropertyPrimitive) ep).getRuntimeType();
                             if (XSD._integer.toString().equals(runtimeType)) {
-                                p.addField(preparedRuntimeName, eventPropertyPrimitiveField.getAsInt());
+                                try {
+                                    p.addField(preparedRuntimeName, eventPropertyPrimitiveField.getAsInt());
+                                } catch (NumberFormatException ef) {
+                                    p.addField(preparedRuntimeName, eventPropertyPrimitiveField.getAsFloat());
+                                }
                             } else if (XSD._float.toString().equals(runtimeType)) {
                                 p.addField(preparedRuntimeName, eventPropertyPrimitiveField.getAsFloat());
                             } else if (XSD._double.toString().equals(runtimeType)) {
@@ -160,7 +164,11 @@ public class InfluxStore {
                             } else if (XSD._boolean.toString().equals(runtimeType)) {
                                 p.addField(preparedRuntimeName, eventPropertyPrimitiveField.getAsBoolean());
                             } else if (XSD._long.toString().equals(runtimeType)) {
-                                p.addField(preparedRuntimeName, eventPropertyPrimitiveField.getAsLong());
+                                try {
+                                    p.addField(preparedRuntimeName, eventPropertyPrimitiveField.getAsLong());
+                                } catch (NumberFormatException ef) {
+                                    p.addField(preparedRuntimeName, eventPropertyPrimitiveField.getAsFloat());
+                                }
                             } else {
                                 p.addField(preparedRuntimeName, eventPropertyPrimitiveField.getAsString());
                             }
diff --git a/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/serializer/GsonSerializer.java b/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/serializer/GsonSerializer.java
index af208ac96..311c55fd0 100644
--- a/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/serializer/GsonSerializer.java
+++ b/streampipes-storage-couchdb/src/main/java/org/apache/streampipes/storage/couchdb/serializer/GsonSerializer.java
@@ -108,6 +108,7 @@ public class GsonSerializer {
             .registerSubtype(UnitTransformRuleDescription.class, "org.apache.streampipes.model.UnitTransformRuleDescription")
             .registerSubtype(TimestampTranfsformationRuleDescription.class, "org.apache.streampipes.model.TimestampTranfsformationRuleDescription")
             .registerSubtype(EventRateTransformationRuleDescription.class, "org.apache.streampipes.model.EventRateTransformationRuleDescription")
+            .registerSubtype(ChangeDatatypeTransformationRuleDescription.class, "org.apache.streampipes.model.ChangeDatatypeTransformationRuleDescription")
             .registerSubtype(CorrectionValueTransformationRuleDescription.class, "org.apache.streampipes.model.CorrectionValueTransformationRuleDescription"));
 
     builder.registerTypeAdapterFactory(RuntimeTypeAdapterFactory.of(AdapterDescription.class, "type")


[incubator-streampipes] 05/15: [STREAMPIPES-577] Add event preview to schema editor

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8ee998066fe8f546d54c257123d26d71f3dd2082
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Tue Aug 16 21:58:21 2022 +0200

    [STREAMPIPES-577] Add event preview to schema editor
---
 .../master/management/GuessManagement.java         |   7 +
 .../streampipes/connect/adapter/Adapter.java       | 112 +--------------
 .../connect/adapter/AdapterPipelineGenerator.java  | 159 +++++++++++++++++++++
 .../pipeline/AdapterEventPreviewPipeline.java      |  81 +++++++++++
 .../model/connect/guess/AdapterEventPreview.java   |  40 ++++--
 .../rest/impl/connect/GuessResource.java           |  11 ++
 .../src/lib/model/gen/streampipes-model.ts         |  25 +++-
 .../event-property-row.component.html              | 117 ++++++++-------
 .../event-property-row.component.scss              |  61 ++++----
 .../event-property-row.component.ts                |  24 +++-
 .../event-schema-preview.component.html            |  24 +++-
 .../event-schema-preview.component.scss            |  11 ++
 .../event-schema-preview.component.ts              |  43 +++++-
 .../event-schema/event-schema.component.html       |  23 +--
 .../event-schema/event-schema.component.ts         |  36 ++++-
 .../schema-editor-header.component.ts              |   6 +-
 ui/src/app/connect/connect.module.ts               |   2 +
 .../json-pretty-print.pipe.ts}                     |  15 ++
 ui/src/app/connect/services/rest.service.ts        |  18 ++-
 19 files changed, 575 insertions(+), 240 deletions(-)

diff --git a/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java b/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java
index e54ade7a4..ce380864c 100644
--- a/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java
+++ b/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java
@@ -26,17 +26,21 @@ import org.apache.http.client.fluent.Response;
 import org.apache.http.entity.ContentType;
 import org.apache.http.util.EntityUtils;
 import org.apache.streampipes.commons.exceptions.NoServiceEndpointsAvailableException;
+import org.apache.streampipes.connect.adapter.model.pipeline.AdapterEventPreviewPipeline;
 import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.connect.api.exception.WorkerAdapterException;
 import org.apache.streampipes.connect.container.master.util.WorkerPaths;
 import org.apache.streampipes.model.connect.adapter.AdapterDescription;
+import org.apache.streampipes.model.connect.guess.AdapterEventPreview;
 import org.apache.streampipes.model.connect.guess.GuessSchema;
+import org.apache.streampipes.model.connect.guess.GuessTypeInfo;
 import org.apache.streampipes.model.message.ErrorMessage;
 import org.apache.streampipes.serializers.json.JacksonSerializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.Map;
 
 public class GuessManagement {
 
@@ -74,4 +78,7 @@ public class GuessManagement {
             }
     }
 
+  public Map<String, GuessTypeInfo> performAdapterEventPreview(AdapterEventPreview previewRequest) {
+      return new AdapterEventPreviewPipeline(previewRequest).makePreview();
+  }
 }
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/Adapter.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/Adapter.java
index 52344b1bc..86fcf9bec 100644
--- a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/Adapter.java
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/Adapter.java
@@ -18,32 +18,18 @@
 
 package org.apache.streampipes.connect.adapter;
 
-import org.apache.streampipes.config.backend.BackendConfig;
-import org.apache.streampipes.config.backend.SpProtocol;
 import org.apache.streampipes.connect.adapter.model.pipeline.AdapterPipeline;
-import org.apache.streampipes.connect.adapter.preprocessing.transform.stream.DuplicateFilterPipelineElement;
-import org.apache.streampipes.connect.api.IAdapterPipelineElement;
-import org.apache.streampipes.connect.adapter.preprocessing.elements.*;
+import org.apache.streampipes.connect.adapter.preprocessing.elements.SendToJmsAdapterSink;
+import org.apache.streampipes.connect.adapter.preprocessing.elements.SendToKafkaAdapterSink;
+import org.apache.streampipes.connect.adapter.preprocessing.elements.SendToMqttAdapterSink;
 import org.apache.streampipes.connect.api.IAdapter;
 import org.apache.streampipes.model.connect.adapter.AdapterDescription;
-import org.apache.streampipes.model.connect.rules.TransformationRuleDescription;
-import org.apache.streampipes.model.connect.rules.stream.EventRateTransformationRuleDescription;
-import org.apache.streampipes.model.connect.rules.stream.RemoveDuplicatesTransformationRuleDescription;
-import org.apache.streampipes.model.connect.rules.value.AddTimestampRuleDescription;
-import org.apache.streampipes.model.connect.rules.value.AddValueTransformationRuleDescription;
-import org.apache.streampipes.model.connect.rules.value.CorrectionValueTransformationRuleDescription;
 import org.apache.streampipes.model.grounding.JmsTransportProtocol;
 import org.apache.streampipes.model.grounding.KafkaTransportProtocol;
 import org.apache.streampipes.model.grounding.MqttTransportProtocol;
 import org.apache.streampipes.model.grounding.TransportProtocol;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
 
 public abstract class Adapter<T extends AdapterDescription> implements IAdapter<T> {
-    Logger logger = LoggerFactory.getLogger(Adapter.class);
 
     private boolean debug;
 
@@ -99,97 +85,7 @@ public abstract class Adapter<T extends AdapterDescription> implements IAdapter<
     }
 
     private AdapterPipeline getAdapterPipeline(T adapterDescription) {
-
-        List<IAdapterPipelineElement> pipelineElements = new ArrayList<>();
-
-        // Must be before the schema transformations to ensure that user can move this event property
-        AddTimestampRuleDescription timestampTransformationRuleDescription = getTimestampRule(adapterDescription);
-        if (timestampTransformationRuleDescription != null) {
-            pipelineElements.add(new AddTimestampPipelineElement(
-                    timestampTransformationRuleDescription.getRuntimeKey()));
-        }
-
-        AddValueTransformationRuleDescription valueTransformationRuleDescription = getAddValueRule(adapterDescription);
-        if (valueTransformationRuleDescription != null) {
-            pipelineElements.add(new AddValuePipelineElement(
-                    valueTransformationRuleDescription.getRuntimeKey(),
-                    valueTransformationRuleDescription.getStaticValue()));
-        }
-
-
-        // first transform schema before transforming vales
-        // value rules should use unique keys for of new schema
-        pipelineElements.add(new TransformSchemaAdapterPipelineElement(adapterDescription.getSchemaRules()));
-        pipelineElements.add(new TransformValueAdapterPipelineElement(adapterDescription.getValueRules()));
-
-
-        RemoveDuplicatesTransformationRuleDescription duplicatesTransformationRuleDescription = getRemoveDuplicateRule(adapterDescription);
-        if (duplicatesTransformationRuleDescription != null) {
-            pipelineElements.add(new DuplicateFilterPipelineElement(duplicatesTransformationRuleDescription.getFilterTimeWindow()));
-        }
-
-        TransformStreamAdapterElement transformStreamAdapterElement = new TransformStreamAdapterElement();
-        EventRateTransformationRuleDescription eventRateTransformationRuleDescription = getEventRateTransformationRule(adapterDescription);
-        if (eventRateTransformationRuleDescription != null) {
-            transformStreamAdapterElement.addStreamTransformationRuleDescription(eventRateTransformationRuleDescription);
-        }
-        pipelineElements.add(transformStreamAdapterElement);
-
-        // Needed when adapter is (
-        if (adapterDescription.getEventGrounding() != null && adapterDescription.getEventGrounding().getTransportProtocol() != null
-                && adapterDescription.getEventGrounding().getTransportProtocol().getBrokerHostname() != null) {
-            return new AdapterPipeline(pipelineElements, getAdapterSink(adapterDescription));
-        }
-
-        return new AdapterPipeline(pipelineElements);
-    }
-
-    private SendToBrokerAdapterSink<?> getAdapterSink(AdapterDescription adapterDescription) {
-        SpProtocol prioritizedProtocol =
-                BackendConfig.INSTANCE.getMessagingSettings().getPrioritizedProtocols().get(0);
-
-        if (GroundingService.isPrioritized(prioritizedProtocol, JmsTransportProtocol.class)) {
-            return new SendToJmsAdapterSink(adapterDescription);
-        }
-        else if (GroundingService.isPrioritized(prioritizedProtocol, KafkaTransportProtocol.class)) {
-            return new SendToKafkaAdapterSink(adapterDescription);
-        }
-        else {
-            return new SendToMqttAdapterSink(adapterDescription);
-        }
-    }
-
-    private RemoveDuplicatesTransformationRuleDescription getRemoveDuplicateRule(T adapterDescription) {
-        return getRule(adapterDescription, RemoveDuplicatesTransformationRuleDescription.class);
-    }
-
-    private EventRateTransformationRuleDescription getEventRateTransformationRule(T adapterDescription) {
-        return getRule(adapterDescription, EventRateTransformationRuleDescription.class);
-    }
-
-    private AddTimestampRuleDescription getTimestampRule(T adapterDescription) {
-        return getRule(adapterDescription, AddTimestampRuleDescription.class);
-    }
-
-    private AddValueTransformationRuleDescription getAddValueRule(T adapterDescription) {
-        return getRule(adapterDescription, AddValueTransformationRuleDescription.class);
-    }
-
-    private CorrectionValueTransformationRuleDescription getCorrectionValueRule(T adapterDescription) {
-        return getRule(adapterDescription, CorrectionValueTransformationRuleDescription.class);
-    }
-
-    private <G extends TransformationRuleDescription> G getRule(T adapterDescription, Class<G> type) {
-
-        if (adapterDescription != null) {
-            for (TransformationRuleDescription tr : adapterDescription.getRules()) {
-                if (type.isInstance(tr)) {
-                    return type.cast(tr);
-                }
-            }
-        }
-
-        return null;
+        return new AdapterPipelineGenerator().generatePipeline(adapterDescription);
     }
 
     @Override
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java
new file mode 100644
index 000000000..31ce0e9a1
--- /dev/null
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/AdapterPipelineGenerator.java
@@ -0,0 +1,159 @@
+/*
+ * 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.connect.adapter;
+
+import org.apache.streampipes.config.backend.BackendConfig;
+import org.apache.streampipes.config.backend.SpProtocol;
+import org.apache.streampipes.connect.adapter.model.pipeline.AdapterPipeline;
+import org.apache.streampipes.connect.adapter.preprocessing.elements.*;
+import org.apache.streampipes.connect.adapter.preprocessing.transform.stream.DuplicateFilterPipelineElement;
+import org.apache.streampipes.connect.api.IAdapterPipelineElement;
+import org.apache.streampipes.model.connect.adapter.AdapterDescription;
+import org.apache.streampipes.model.connect.rules.TransformationRuleDescription;
+import org.apache.streampipes.model.connect.rules.schema.SchemaTransformationRuleDescription;
+import org.apache.streampipes.model.connect.rules.stream.EventRateTransformationRuleDescription;
+import org.apache.streampipes.model.connect.rules.stream.RemoveDuplicatesTransformationRuleDescription;
+import org.apache.streampipes.model.connect.rules.value.AddTimestampRuleDescription;
+import org.apache.streampipes.model.connect.rules.value.AddValueTransformationRuleDescription;
+import org.apache.streampipes.model.connect.rules.value.CorrectionValueTransformationRuleDescription;
+import org.apache.streampipes.model.connect.rules.value.ValueTransformationRuleDescription;
+import org.apache.streampipes.model.grounding.JmsTransportProtocol;
+import org.apache.streampipes.model.grounding.KafkaTransportProtocol;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class AdapterPipelineGenerator {
+
+  // TODO improve this code
+  public AdapterPipeline generatePipeline(AdapterDescription adapterDescription) {
+
+    var pipelineElements = makeAdapterPipelineElements(adapterDescription.getRules());
+
+    RemoveDuplicatesTransformationRuleDescription duplicatesTransformationRuleDescription = getRemoveDuplicateRule(adapterDescription.getRules());
+    if (duplicatesTransformationRuleDescription != null) {
+      pipelineElements.add(new DuplicateFilterPipelineElement(duplicatesTransformationRuleDescription.getFilterTimeWindow()));
+    }
+
+    TransformStreamAdapterElement transformStreamAdapterElement = new TransformStreamAdapterElement();
+    EventRateTransformationRuleDescription eventRateTransformationRuleDescription = getEventRateTransformationRule(adapterDescription.getRules());
+    if (eventRateTransformationRuleDescription != null) {
+      transformStreamAdapterElement.addStreamTransformationRuleDescription(eventRateTransformationRuleDescription);
+    }
+    pipelineElements.add(transformStreamAdapterElement);
+
+    // TODO decide what was meant with this comment
+    // Needed when adapter is (
+    if (adapterDescription.getEventGrounding() != null && adapterDescription.getEventGrounding().getTransportProtocol() != null
+      && adapterDescription.getEventGrounding().getTransportProtocol().getBrokerHostname() != null) {
+      return new AdapterPipeline(pipelineElements, getAdapterSink(adapterDescription));
+    }
+
+    return new AdapterPipeline(pipelineElements);
+  }
+
+  public List<IAdapterPipelineElement> makeAdapterPipelineElements(List<TransformationRuleDescription> rules) {
+    List<IAdapterPipelineElement> pipelineElements = new ArrayList<>();
+
+    // Must be before the schema transformations to ensure that user can move this event property
+    AddTimestampRuleDescription timestampTransformationRuleDescription = getTimestampRule(rules);
+    if (timestampTransformationRuleDescription != null) {
+      pipelineElements.add(new AddTimestampPipelineElement(
+        timestampTransformationRuleDescription.getRuntimeKey()));
+    }
+
+    AddValueTransformationRuleDescription valueTransformationRuleDescription = getAddValueRule(rules);
+    if (valueTransformationRuleDescription != null) {
+      pipelineElements.add(new AddValuePipelineElement(
+        valueTransformationRuleDescription.getRuntimeKey(),
+        valueTransformationRuleDescription.getStaticValue()));
+    }
+
+    // first transform schema before transforming vales
+    // value rules should use unique keys for of new schema
+    pipelineElements.add(new TransformSchemaAdapterPipelineElement(getSchemaRules(rules)));
+    pipelineElements.add(new TransformValueAdapterPipelineElement(getValueRules(rules)));
+
+    return pipelineElements;
+  }
+
+  private SendToBrokerAdapterSink<?> getAdapterSink(AdapterDescription adapterDescription) {
+    SpProtocol prioritizedProtocol =
+      BackendConfig.INSTANCE.getMessagingSettings().getPrioritizedProtocols().get(0);
+
+    if (GroundingService.isPrioritized(prioritizedProtocol, JmsTransportProtocol.class)) {
+      return new SendToJmsAdapterSink(adapterDescription);
+    }
+    else if (GroundingService.isPrioritized(prioritizedProtocol, KafkaTransportProtocol.class)) {
+      return new SendToKafkaAdapterSink(adapterDescription);
+    }
+    else {
+      return new SendToMqttAdapterSink(adapterDescription);
+    }
+  }
+
+  private RemoveDuplicatesTransformationRuleDescription getRemoveDuplicateRule(List<TransformationRuleDescription> rules) {
+    return getRule(rules, RemoveDuplicatesTransformationRuleDescription.class);
+  }
+
+  private EventRateTransformationRuleDescription getEventRateTransformationRule(List<TransformationRuleDescription> rules) {
+    return getRule(rules, EventRateTransformationRuleDescription.class);
+  }
+
+  private AddTimestampRuleDescription getTimestampRule(List<TransformationRuleDescription> rules) {
+    return getRule(rules, AddTimestampRuleDescription.class);
+  }
+
+  private AddValueTransformationRuleDescription getAddValueRule(List<TransformationRuleDescription> rules) {
+    return getRule(rules, AddValueTransformationRuleDescription.class);
+  }
+
+  private CorrectionValueTransformationRuleDescription getCorrectionValueRule(List<TransformationRuleDescription> rules) {
+    return getRule(rules, CorrectionValueTransformationRuleDescription.class);
+  }
+
+  private <G extends TransformationRuleDescription> G getRule(List<TransformationRuleDescription> rules,
+                                                              Class<G> type) {
+
+    if (rules != null) {
+      for (TransformationRuleDescription tr : rules) {
+        if (type.isInstance(tr)) {
+          return type.cast(tr);
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private List<TransformationRuleDescription> getValueRules(List<TransformationRuleDescription> rules) {
+    return rules
+      .stream()
+      .filter(r -> r instanceof ValueTransformationRuleDescription && !(r instanceof AddTimestampRuleDescription))
+      .collect(Collectors.toList());
+  }
+
+  private List<TransformationRuleDescription> getSchemaRules(List<TransformationRuleDescription> rules) {
+    return rules
+      .stream()
+      .filter(r -> r instanceof SchemaTransformationRuleDescription)
+      .collect(Collectors.toList());
+  }
+}
diff --git a/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/model/pipeline/AdapterEventPreviewPipeline.java b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/model/pipeline/AdapterEventPreviewPipeline.java
new file mode 100644
index 000000000..3e36ad5b4
--- /dev/null
+++ b/streampipes-connect/src/main/java/org/apache/streampipes/connect/adapter/model/pipeline/AdapterEventPreviewPipeline.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.connect.adapter.model.pipeline;
+
+import org.apache.streampipes.connect.adapter.AdapterPipelineGenerator;
+import org.apache.streampipes.connect.api.IAdapterPipeline;
+import org.apache.streampipes.connect.api.IAdapterPipelineElement;
+import org.apache.streampipes.model.connect.guess.AdapterEventPreview;
+import org.apache.streampipes.model.connect.guess.GuessTypeInfo;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class AdapterEventPreviewPipeline implements IAdapterPipeline {
+
+  private List<IAdapterPipelineElement> pipelineElements;
+  private Map<String, GuessTypeInfo> event;
+
+  public AdapterEventPreviewPipeline(AdapterEventPreview previewRequest) {
+    this.pipelineElements = new AdapterPipelineGenerator().makeAdapterPipelineElements(previewRequest.getRules());
+    this.event = previewRequest.getInputData();
+  }
+
+  @Override
+  public void process(Map<String, Object> event) {
+    for (IAdapterPipelineElement pe : this.pipelineElements) {
+      event = pe.process(event);
+    }
+  }
+
+  @Override
+  public List<IAdapterPipelineElement> getPipelineElements() {
+    return null;
+  }
+
+  @Override
+  public void setPipelineElements(List<IAdapterPipelineElement> pipelineElements) {
+
+  }
+
+  @Override
+  public void changePipelineSink(IAdapterPipelineElement pipelineSink) {
+
+  }
+
+  @Override
+  public IAdapterPipelineElement getPipelineSink() {
+    return null;
+  }
+
+  public Map<String, GuessTypeInfo> makePreview() {
+    Map<String, Object> ev = this.event
+      .entrySet()
+      .stream()
+      .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().getValue()));
+    this.process(ev);
+
+    return ev
+      .entrySet()
+      .stream()
+      .collect(Collectors.toMap(Map.Entry::getKey, e-> new GuessTypeInfo(e.getValue().getClass().getCanonicalName(), e.getValue())));
+  }
+}
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.ts b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/AdapterEventPreview.java
similarity index 51%
copy from ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.ts
copy to streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/AdapterEventPreview.java
index f80de67d9..d70af4f3f 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.ts
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/connect/guess/AdapterEventPreview.java
@@ -16,14 +16,34 @@
  *
  */
 
-import { Component, Input } from '@angular/core';
-import { EventSchema } from '@streampipes/platform-services';
-
-@Component({
-    selector: 'sp-event-schema-preview',
-    templateUrl: './event-schema-preview.component.html',
-    styleUrls: ['./event-schema-preview.component.scss']
-})
-export class EventSchemaPreviewComponent {
-    @Input() eventSchema: EventSchema;
+package org.apache.streampipes.model.connect.guess;
+
+import org.apache.streampipes.model.connect.rules.TransformationRuleDescription;
+import org.apache.streampipes.model.shared.annotation.TsModel;
+
+import java.util.List;
+import java.util.Map;
+
+@TsModel
+public class AdapterEventPreview {
+
+  private List<TransformationRuleDescription> rules;
+
+  private Map<String, GuessTypeInfo> inputData;
+
+  public List<TransformationRuleDescription> getRules() {
+    return rules;
+  }
+
+  public void setRules(List<TransformationRuleDescription> rules) {
+    this.rules = rules;
+  }
+
+  public Map<String, GuessTypeInfo> getInputData() {
+    return inputData;
+  }
+
+  public void setInputData(Map<String, GuessTypeInfo> inputData) {
+    this.inputData = inputData;
+  }
 }
diff --git a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/GuessResource.java b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/GuessResource.java
index 7cdd81281..b678b9eab 100644
--- a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/GuessResource.java
+++ b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/GuessResource.java
@@ -24,11 +24,13 @@ import org.apache.streampipes.connect.api.exception.WorkerAdapterException;
 import org.apache.streampipes.connect.container.master.management.GuessManagement;
 import org.apache.streampipes.model.StreamPipesErrorMessage;
 import org.apache.streampipes.model.connect.adapter.AdapterDescription;
+import org.apache.streampipes.model.connect.guess.AdapterEventPreview;
 import org.apache.streampipes.model.connect.guess.GuessSchema;
 import org.apache.streampipes.rest.shared.annotation.JacksonSerialized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.ws.rs.Consumes;
 import javax.ws.rs.POST;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
@@ -66,5 +68,14 @@ public class GuessResource extends AbstractAdapterResource<GuessManagement> {
         return serverError(StreamPipesErrorMessage.from(e));
       }
   }
+
+  @POST
+  @JacksonSerialized
+  @Path("/schema/preview")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Consumes(MediaType.APPLICATION_JSON)
+  public Response getAdapterEventPreview(AdapterEventPreview previewRequest) {
+    return ok(managementService.performAdapterEventPreview(previewRequest));
+  }
 }
 
diff --git a/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts b/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
index 7d282d6de..d2712a990 100644
--- a/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
+++ b/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
@@ -18,7 +18,7 @@
 /* tslint:disable */
 /* eslint-disable */
 // @ts-nocheck
-// Generated using typescript-generator version 2.27.744 on 2022-08-16 11:39:37.
+// Generated using typescript-generator version 2.27.744 on 2022-08-16 16:51:03.
 
 export class AbstractStreamPipesEntity {
     "@class": "org.apache.streampipes.model.base.AbstractStreamPipesEntity" | "org.apache.streampipes.model.base.NamedStreamPipesEntity" | "org.apache.streampipes.model.connect.adapter.AdapterDescription" | "org.apache.streampipes.model.connect.adapter.AdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.GenericAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.SpecificAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.AdapterStre [...]
@@ -192,9 +192,9 @@ export class AdapterDescription extends NamedStreamPipesEntity {
         instance.selectedEndpointUrl = data.selectedEndpointUrl;
         instance.correspondingServiceGroup = data.correspondingServiceGroup;
         instance.correspondingDataStreamElementId = data.correspondingDataStreamElementId;
-        instance.valueRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.valueRules);
-        instance.streamRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.streamRules);
         instance.schemaRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.schemaRules);
+        instance.streamRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.streamRules);
+        instance.valueRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.valueRules);
         return instance;
     }
 
@@ -215,6 +215,21 @@ export class AdapterDescription extends NamedStreamPipesEntity {
     }
 }
 
+export class AdapterEventPreview {
+    inputData: { [index: string]: GuessTypeInfo };
+    rules: TransformationRuleDescriptionUnion[];
+
+    static fromData(data: AdapterEventPreview, target?: AdapterEventPreview): AdapterEventPreview {
+        if (!data) {
+            return data;
+        }
+        const instance = target || new AdapterEventPreview();
+        instance.rules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.rules);
+        instance.inputData = __getCopyObjectFn(GuessTypeInfo.fromData)(data.inputData);
+        return instance;
+    }
+}
+
 export class AdapterSetDescription extends AdapterDescription {
     "@class": "org.apache.streampipes.model.connect.adapter.AdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.GenericAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.SpecificAdapterSetDescription";
     dataSet: SpDataSet;
@@ -1753,8 +1768,8 @@ export class GenericAdapterSetDescription extends AdapterSetDescription implemen
         const instance = target || new GenericAdapterSetDescription();
         super.fromData(data, instance);
         instance.eventSchema = EventSchema.fromData(data.eventSchema);
-        instance.protocolDescription = ProtocolDescription.fromData(data.protocolDescription);
         instance.formatDescription = FormatDescription.fromData(data.formatDescription);
+        instance.protocolDescription = ProtocolDescription.fromData(data.protocolDescription);
         return instance;
     }
 }
@@ -1772,8 +1787,8 @@ export class GenericAdapterStreamDescription extends AdapterStreamDescription im
         const instance = target || new GenericAdapterStreamDescription();
         super.fromData(data, instance);
         instance.eventSchema = EventSchema.fromData(data.eventSchema);
-        instance.protocolDescription = ProtocolDescription.fromData(data.protocolDescription);
         instance.formatDescription = FormatDescription.fromData(data.formatDescription);
+        instance.protocolDescription = ProtocolDescription.fromData(data.protocolDescription);
         return instance;
     }
 }
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html
index be87eb7e2..88a49c3e4 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.html
@@ -17,67 +17,78 @@
   -->
 
 <div fxLayout="row" fxFlex="100" fxLayoutGap="10px">
-    <div fxFlex fxLayout="row" fxLayoutAlign="start center">
-        <b>
+    <div fxLayout="row" fxLayoutAlign="start center">
+        <div>
+        <b style="white-space: nowrap">
             {{ label }}
         </b>
-        <div class="timestamp-property" *ngIf="timestampProperty" fxLayoutAlign="start center" fxLayout="row">
+        </div>
+        <div fxLayoutAlign="start center" *ngIf="originalRuntimeName" class="ml-5">
+            <span class="runtime-info">{{originalRuntimeName}}</span>
+            <span fxLayoutAlign="center center"
+                  *ngIf="originalRuntimeName !== node.data.runtimeName">
+                <i class="material-icons">arrow_right</i>
+            </span>
+            <span class="runtime-info"
+                  *ngIf="originalRuntimeName !== node.data.runtimeName">{{node.data.runtimeName}}
+            </span>
+        </div>
+        <p style="margin: 0px 10px 10px;" *ngIf="isList">[List]</p>
+    </div>
+
+    <div fxLayout="row" fxLayoutAlign="end center" fxFlex="100" fxLayoutGap="20px">
+        <div class="timestamp-property" *ngIf="timestampProperty" fxLayoutAlign="end center" fxLayout="row">
             <mat-icon *ngIf="timestampProperty" class="timestamp-icon" fxLayout="row" fxLayoutAlign="start center">
                 access_time
             </mat-icon>
             <span style="margin-left: -5px;">marked as timestamp</span>
         </div>
-        <p style="margin: 0px 10px 10px;" *ngIf="isList">[List]</p>
-    </div>
-    <div fxFlex="15" fxLayoutAlign="start center" *ngIf="runtimeType">
-        <span class="runtime-type-info">{{runtimeType}}</span>
-    </div>
-    <div fxFlex="15" fxLayoutAlign="start center">
-        <span *ngIf="showEventPreview">{{eventPreview[0][node.data.runtimeName].value || 'n/a'}}</span>
-    </div>
-    <div fxFlex="15" fxLayoutAlign="start center">
+        <div fxLayoutAlign="end center" *ngIf="runtimeType" class="runtime-type-info-outer">
+            <span class="runtime-info runtime-type-info">{{originalRuntimeType}}</span>
+            <span fxLayoutAlign="center center"><i class="material-icons">arrow_right</i></span>
+            <span class="runtime-info runtime-type-info">{{runtimeType}}</span>
+        </div>
+        <div fxLayoutAlign="end center">
         <span *ngIf="showFieldStatus"
-              [ngClass]="'status status-' +fieldStatusInfo[node.data.runtimeName].fieldStatus.toLowerCase()"
-        [matTooltip]="fieldStatusInfo[node.data.runtimeName].additionalInfo">{{fieldStatusInfo[node.data.runtimeName].fieldStatus}}</span>
-    </div>
-
-    <div fxLayout="row" fxLayoutAlign="end center">
-    <div fxFlex="15" *ngIf="isPrimitive" fxLayoutAlign="end center">
-        <mat-form-field class="small-select" color="accent">
-        <mat-select [(ngModel)]="node.data.propertyScope" panelClass="small-select"
-                    [attr.data-cy]="'property-scope-' + label">
-            <mat-option value="MEASUREMENT_PROPERTY">Measurement</mat-option>
-            <mat-option value="DIMENSION_PROPERTY">Dimension</mat-option>
-            <mat-option value="HEADER_PROPERTY">Header</mat-option>
-        </mat-select>
-        </mat-form-field>
-    </div>
+              [ngClass]="'status status-' +fieldStatusInfo[originalRuntimeName].fieldStatus.toLowerCase()"
+              [matTooltip]="fieldStatusInfo[originalRuntimeName].additionalInfo">{{fieldStatusInfo[originalRuntimeName].fieldStatus}}</span>
+        </div>
+        <div *ngIf="isPrimitive" fxLayoutAlign="end center">
+            <mat-form-field class="small-select" color="accent">
+                <mat-select [(ngModel)]="node.data.propertyScope" panelClass="small-select"
+                            [attr.data-cy]="'property-scope-' + label">
+                    <mat-option value="MEASUREMENT_PROPERTY">Measurement</mat-option>
+                    <mat-option value="DIMENSION_PROPERTY">Dimension</mat-option>
+                    <mat-option value="HEADER_PROPERTY">Header</mat-option>
+                </mat-select>
+            </mat-form-field>
+        </div>
 
-    <div fxLayoutAlign="end center"
-         *ngIf="isNested">
-        <button [disabled]="!isEditable" color="accent" mat-button (click)=addNestedProperty(node.data)>
-            <mat-icon matTooltip="Add a Nested Property">queue</mat-icon>
-        </button>
-    </div>
-    <div
-         fxLayoutAlign="end center"
-         class="ml-5 mr-5"
-         *ngIf="isNested || isPrimitive || isList">
-        <button [disabled]="!isEditable" color="accent" mat-button
-                (click)="openEditDialog(node.data)"
-                [attr.data-cy]="'edit-' + label.toLowerCase()">
-            <mat-icon>edit</mat-icon>&nbsp;Edit field
-        </button>
-    </div>
-    <div fxLayoutAlign="end center">
-        <mat-checkbox
-                *ngIf="isNested || isPrimitive || isList"
-                (click)="selectProperty(node.data.id, undefined)"
-                [disabled]="!isEditable"
-                [class.checkbox-selected]="node.data.selected"
-                [checked]="node.data.selected"
-                [attr.data-cy]="'delete-property-' + label.toLowerCase()">
-        </mat-checkbox>
-    </div>
+        <div fxLayoutAlign="end center"
+             *ngIf="isNested">
+            <button [disabled]="!isEditable" color="accent" mat-button (click)=addNestedProperty(node.data)>
+                <mat-icon matTooltip="Add a Nested Property">queue</mat-icon>
+            </button>
+        </div>
+        <div
+                fxLayoutAlign="end center"
+                class="ml-5 mr-5"
+                *ngIf="isNested || isPrimitive || isList">
+            <button [disabled]="!isEditable" color="accent" mat-button
+                    (click)="openEditDialog(node.data)"
+                    [attr.data-cy]="'edit-' + label.toLowerCase()">
+                <mat-icon>edit</mat-icon>&nbsp;Edit field
+            </button>
+        </div>
+        <div fxLayoutAlign="end center">
+            <mat-checkbox
+                    *ngIf="isNested || isPrimitive || isList"
+                    (click)="selectProperty(node.data.id, undefined)"
+                    [disabled]="!isEditable"
+                    [class.checkbox-selected]="node.data.selected"
+                    [checked]="node.data.selected"
+                    [attr.data-cy]="'delete-property-' + label.toLowerCase()">
+            </mat-checkbox>
+        </div>
     </div>
 </div>
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
index c20307e9b..9e4191930 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.scss
@@ -18,54 +18,67 @@
 
 
 ::ng-deep .mat-checkbox-checked.mat-accent .mat-checkbox-background, .mat-checkbox-indeterminate.mat-accent .mat-checkbox-background {
-    background-color: var(--color-accent) !important;
+  background-color: var(--color-accent) !important;
 }
 
 ::ng-deep .mat-checkbox:not(.mat-checkbox-disabled).mat-accent .mat-checkbox-ripple .mat-ripple-element {
-    background-color: var(--color-accent) !important;
+  background-color: var(--color-accent) !important;
 }
 
 .checkbox-selected {
-    opacity: 1 !important;
+  opacity: 1 !important;
 }
 
 .timestamp-property {
-    margin-left: 15px;
-    border-radius: 10px;
-    background: var(--color-processor);
-    padding: 0px 10px;
-    font-size: 12px;
-    color: #FFFFFF;
+  margin-left: 15px;
+  border-radius: 10px;
+  background: var(--color-processor);
+  padding: 0px 10px;
+  font-size: 12px;
+  color: #FFFFFF;
 }
 
 .timestamp-icon {
-    font-size: 12px;
+  font-size: 12px;
 }
 
 .status-good {
-    background: #75c575;
-    color: white;
+  background: #75c575;
+  color: white;
 }
 
 .status-bad {
-    background: #d2a169;
-    color: white;
+  background: #d2a169;
+  color: white;
 }
 
 .status {
-    width: 60px;
-    text-align: center;
-    padding: 5px;
-    border-radius: 3px;
+  width: 70px;
+  min-width: 70px;
+  text-align: center;
+  padding: 5px;
+  border-radius: 5px;
+}
+
+.runtime-info {
+  border-radius: 5px;
+  color: var(--color-default-text);
+  font-size: 10pt;
+  text-align: center;
 }
 
 .runtime-type-info {
-    padding: 5px;
-    border-radius: 3px;
-    background: var(--color-bg-3);
-    color: var(--color-default-text);
-    width: 70px;
-    text-align: center;
+  border-radius: 5px;
+  color: var(--color-default-text);
+  width: 60px;
+  font-size: 10pt;
+  text-align: center;
+}
+
+.runtime-type-info-outer {
+  padding: 3px;
+  border-radius: 5px;
+  background: var(--color-bg-3);
 }
 
 
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.ts b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.ts
index f2be21368..393066c93 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.ts
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-property-row/event-property-row.component.ts
@@ -28,28 +28,30 @@ import {
   EventPropertyUnion,
   EventSchema,
   FieldStatusInfo,
+  GuessTypeInfo
 
 } from '@streampipes/platform-services';
 import { EditEventPropertyComponent } from '../../../../dialog/edit-event-property/edit-event-property.component';
 import { DialogService, PanelType } from '@streampipes/shared-ui';
-import { GuessTypeInfo } from '../../../../../../../projects/streampipes/platform-services/src/lib/model/gen/streampipes-model';
 
 @Component({
   selector: 'event-property-row',
   templateUrl: './event-property-row.component.html',
   styleUrls: ['./event-property-row.component.scss']
 })
-export class EventPropertyRowComponent implements OnInit, OnChanges {
+export class EventPropertyRowComponent implements OnInit {
 
   @Input() node: TreeNode;
   @Input() isEditable = true;
   @Input() eventSchema: EventSchema = new EventSchema();
+  @Input() originalEventSchema: EventSchema;
   @Input() countSelected: number;
   @Input() eventPreview: Record<string, GuessTypeInfo>[];
   @Input() fieldStatusInfo: Record<string, FieldStatusInfo>;
 
   @Output() isEditableChange = new EventEmitter<boolean>();
   @Output() eventSchemaChange = new EventEmitter<EventSchema>();
+  @Output() originalEventSchemaChange = new EventEmitter<EventSchema>();
   @Output() refreshTreeEmitter = new EventEmitter<void>();
   @Output() countSelectedChange = new EventEmitter<number>();
 
@@ -62,6 +64,8 @@ export class EventPropertyRowComponent implements OnInit, OnChanges {
   showFieldStatus = false;
 
   runtimeType: string;
+  originalRuntimeType: string;
+  originalRuntimeName: string;
 
   constructor(private dialog: MatDialog,
               private dialogService: DialogService) {
@@ -69,16 +73,20 @@ export class EventPropertyRowComponent implements OnInit, OnChanges {
   }
 
   ngOnInit() {
-    this.showFieldStatus = this.fieldStatusInfo && this.fieldStatusInfo[this.node.data.runtimeName] !== undefined;
-    this.showEventPreview = this.eventPreview && this.eventPreview.length > 0 && this.eventPreview[0][this.node.data.runtimeName] !== undefined;
     this.label = this.getLabel(this.node.data);
     this.isPrimitive = this.isEventPropertyPrimitive(this.node.data);
     this.isList = this.isEventPropertyList(this.node.data);
     this.isNested = this.isEventPropertyNested(this.node.data);
     this.timestampProperty = this.isTimestampProperty(this.node.data);
 
-    if (this.isPrimitive) {
-      this.runtimeType = this.parseType(this.node.data.runtimeType);
+    if (this.node.data instanceof EventProperty) {
+      this.originalRuntimeName = this.findOriginalProperty().runtimeName;
+      this.showFieldStatus = this.fieldStatusInfo && this.fieldStatusInfo[this.originalRuntimeName] !== undefined;
+      this.showEventPreview = this.eventPreview && this.eventPreview.length > 0 && this.eventPreview[0][this.originalRuntimeName] !== undefined;
+      if (this.isPrimitive) {
+        this.originalRuntimeType = this.parseType(this.findOriginalProperty().runtimeType);
+        this.runtimeType = this.parseType((this.node.data as EventPropertyPrimitive).runtimeType);
+      }
     }
 
     if (!this.node.data.propertyScope) {
@@ -86,7 +94,9 @@ export class EventPropertyRowComponent implements OnInit, OnChanges {
     }
   }
 
-  ngOnChanges(changes: SimpleChanges): void {
+  private findOriginalProperty(): any {
+    return this.originalEventSchema.eventProperties
+      .find(ep => ep.elementId === this.node.data.elementId);
   }
 
   private parseType(runtimeType: string) {
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.html b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.html
index 79bf5a41d..d3cad53f6 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.html
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.html
@@ -16,4 +16,26 @@
   ~
   -->
 
-<pre>{{ eventSchema | json }}</pre>
\ No newline at end of file
+<div fxFlex="100" fxLayout="row" fxLayoutGap="15px">
+    <div fxFlex="50" fxLayout="column">
+        <sp-basic-inner-panel [showTitle]="true" panelTitle="Original (Parsed)">
+            <pre [innerHTML]="originalField | jsonpretty" class="preview-text"></pre>
+        </sp-basic-inner-panel>
+    </div>
+
+    <div fxFlex="50" fxLayout="column">
+        <sp-basic-inner-panel [showTitle]="true" panelTitle="Result" fxFlex="100">
+            <div header fxLayoutAlign="end center" fxFlex="100">
+                <button color="accent"
+                        mat-button
+                        data-cy="connect-schema-update-preview-btn"
+                        matTooltip="Update event preview"
+                        (click)="updateEventPreview()">
+                    <mat-icon>refresh</mat-icon>
+                    <span>&nbsp;Update result preview</span>
+                </button>
+            </div>
+            <pre [innerHTML]="desiredField | jsonpretty" class="preview-text"></pre>
+        </sp-basic-inner-panel>
+    </div>
+</div>
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.scss b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.scss
index 58ba04bdd..eddad5e07 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.scss
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.scss
@@ -16,3 +16,14 @@
  *
  */
 
+.preview-text {
+  background-color: black;
+  font: 9pt Inconsolata, monospace;
+  text-shadow: 0 0 5px #C8C8C8;
+  color: white;
+  padding: 10px;
+  max-width: 100%;
+  max-height: 300px;
+  overflow-y: scroll;
+  white-space: pre-wrap;
+}
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.ts b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.ts
index f80de67d9..11823a586 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.ts
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.ts
@@ -16,14 +16,43 @@
  *
  */
 
-import { Component, Input } from '@angular/core';
-import { EventSchema } from '@streampipes/platform-services';
+import { Component, EventEmitter, Input, OnInit, Output } from '@angular/core';
+import { EventSchema, GuessTypeInfo } from '@streampipes/platform-services';
 
 @Component({
-    selector: 'sp-event-schema-preview',
-    templateUrl: './event-schema-preview.component.html',
-    styleUrls: ['./event-schema-preview.component.scss']
+  selector: 'sp-event-schema-preview',
+  templateUrl: './event-schema-preview.component.html',
+  styleUrls: ['./event-schema-preview.component.scss']
 })
-export class EventSchemaPreviewComponent {
-    @Input() eventSchema: EventSchema;
+export class EventSchemaPreviewComponent implements OnInit {
+
+  @Input() originalEventSchema: EventSchema;
+  @Input() desiredEventSchema: EventSchema;
+
+  @Input() originalPreview: Record<string, GuessTypeInfo>;
+  @Input() desiredPreview: Record<string, GuessTypeInfo>;
+
+  @Output() updatePreviewEmitter = new EventEmitter();
+
+  originalField: Record<string, any>;
+  desiredField: Record<string, any>;
+
+  ngOnInit(): void {
+    this.originalField = this.toSimpleMap(this.originalPreview);
+    this.desiredField = this.toSimpleMap(this.desiredPreview);
+  }
+
+  toSimpleMap(event: Record<string, GuessTypeInfo>): Record<string, any> {
+    const result = {};
+    for (const key in event) {
+      result[key] = event[key].value;
+    }
+
+
+    return result;
+  }
+
+  public updateEventPreview() {
+    this.updatePreviewEmitter.emit();
+  }
 }
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
index f7474a0e7..6709dad75 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
@@ -17,12 +17,13 @@
   -->
 
 
-<div fxLayout="row" fxLayoutAlign="center" class="mt-20">
+<div fxLayout="column" fxLayoutAlign="center" class="mt-20">
     <div fxFlex="90" fxLayout="column">
         <div fxLayout="column"
              fxFlex="100"
              fxLayoutAlign="start center"
-            *ngIf="!isLoading && !isError && schemaErrorHints.length === 0" class="schema-validation schema-validation-ok">
+             *ngIf="!isLoading && !isError && schemaErrorHints.length === 0"
+             class="schema-validation schema-validation-ok">
             <div fxFlex="100"
                  fxLayout="row"
                  fxLayoutAlign="start center"
@@ -36,7 +37,8 @@
                  fxLayout="column"
                  [ngClass]="schemaErrorHint.level === 'error' ? 'schema-validation schema-validation-error' : 'schema-validation schema-validation-warning'"
                  fxLayoutAlign="start center" *ngFor="let schemaErrorHint of schemaErrorHints">
-                <div fxFlex="100" fxLayout="row" fxLayoutAlign="center start" [ngClass]="schemaErrorHint.level === 'error' ? 'schema-validation-text-error' : 'schema-validation-text-warning'">
+                <div fxFlex="100" fxLayout="row" fxLayoutAlign="center start"
+                     [ngClass]="schemaErrorHint.level === 'error' ? 'schema-validation-text-error' : 'schema-validation-text-warning'">
                     <i class="material-icons">warning</i>&nbsp;
                     <b>{{schemaErrorHint.title}}</b>
                 </div>
@@ -53,7 +55,7 @@
                                          (addStaticValuePropertyEmitter)="addStaticValueProperty()"
                                          (addTimestampPropertyEmitter)="addTimestampProperty()"
                                          (guessSchemaEmitter)="guessSchema()"
-                                         (togglePreviewEmitter)="togglePreview()"
+                                         (updatePreviewEmitter)="updatePreview()"
                                          (removeSelectedPropertiesEmitter)="removeSelectedProperties()">
                 </sp-schema-editor-header>
             </div>
@@ -78,7 +80,7 @@
                         *ngIf="isError && !isLoading">
                 </sp-error-message>
 
-                <div *ngIf="!isError && !isLoading && eventSchema"
+                <div *ngIf="!isError && !isLoading && eventSchema && oldEventSchema && nodes"
                      fxLayout="column"
                      fxLayoutAlign="space-evenly stretched"
                      class="drag-drop-tree"
@@ -92,6 +94,7 @@
                                     [node]="node"
                                     [(isEditable)]="isEditable"
                                     [(eventSchema)]="eventSchema"
+                                    [(originalEventSchema)]="oldEventSchema"
                                     [eventPreview]="eventPreview"
                                     [fieldStatusInfo]="fieldStatusInfo"
                                     (refreshTreeEmitter)="refreshTree()"
@@ -102,13 +105,17 @@
             </div>
         </sp-basic-inner-panel>
     </div>
-    <div fxFlex="0 1 50%" *ngIf="isPreviewEnabled">
-        <sp-event-schema-preview [eventSchema]="eventSchema"></sp-event-schema-preview>
+    <div fxFlex="100" *ngIf="desiredPreview && isPreviewEnabled && !isLoading && !isError">
+        <sp-event-schema-preview [originalEventSchema]="oldEventSchema"
+                                 [desiredEventSchema]="eventSchema"
+                                 [originalPreview]="eventPreview[0]"
+                                 [desiredPreview]="desiredPreview"
+                                 (updatePreviewEmitter)="updatePreview()"></sp-event-schema-preview>
     </div>
 </div>
 
 
-<div fxLayoutAlign="end">
+<div fxLayoutAlign="end" class="mt-10">
     <button class="mat-basic" mat-raised-button (click)="removeSelection()">Cancel</button>
     <button class="mat-basic stepper-button" mat-raised-button (click)="goBack()">Back</button>
     <button class="stepper-button"
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
index d6e19e214..e7867d91f 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
@@ -36,6 +36,7 @@ import {
   FieldStatusInfo, GuessTypeInfo,
   StreamPipesErrorMessage
 } from '../../../../../../../projects/streampipes/platform-services/src/lib/model/gen/streampipes-model';
+import { TransformationRuleService } from '../../../../services/transformation-rule.service';
 
 @Component({
   selector: 'sp-event-schema',
@@ -44,7 +45,9 @@ import {
 })
 export class EventSchemaComponent implements OnChanges {
 
-  constructor(private restService: RestService, private dataTypesService: DataTypesService) {
+  constructor(private restService: RestService,
+              private dataTypesService: DataTypesService,
+              private transformationRuleService: TransformationRuleService ) {
   }
 
   @Input() adapterDescription: AdapterDescription;
@@ -83,6 +86,7 @@ export class EventSchemaComponent implements OnChanges {
   schemaErrorHints: UserErrorMessage[] = [];
 
   eventPreview: Record<string, GuessTypeInfo>[];
+  desiredPreview: Record<string, GuessTypeInfo>;
   fieldStatusInfo: Record<string, FieldStatusInfo>;
 
   options: ITreeOptions = {
@@ -124,6 +128,10 @@ export class EventSchemaComponent implements OnChanges {
         this.isEditable = true;
         this.isEditableChange.emit(true);
         this.isLoading = false;
+
+        if (guessSchema.eventPreview) {
+          this.updatePreview();
+        }
       },
       errorMessage => {
         this.errorMessage = errorMessage.error;
@@ -138,7 +146,6 @@ export class EventSchemaComponent implements OnChanges {
     this.nodes = new Array<EventProperty>();
     this.nodes.push(this.eventSchema as unknown as EventProperty);
     this.validEventSchema = this.checkIfValid(this.eventSchema);
-    // this.tree.treeModel.update();
   }
 
   public addNestedProperty(eventProperty?: EventPropertyNested): void {
@@ -201,8 +208,15 @@ export class EventSchemaComponent implements OnChanges {
     this.refreshTree();
   }
 
-  public togglePreview(): void {
-    this.isPreviewEnabled = !this.isPreviewEnabled;
+  public updatePreview(): void {
+    this.isPreviewEnabled = false;
+    this.transformationRuleService.setOldEventSchema(this.oldEventSchema);
+    this.transformationRuleService.setNewEventSchema(this.eventSchema);
+    const ruleDescriptions = this.transformationRuleService.getTransformationRuleDescriptions();
+    this.restService.getAdapterEventPreview({rules: ruleDescriptions, inputData: this.eventPreview[0]}).subscribe(preview => {
+      this.desiredPreview = preview;
+      this.isPreviewEnabled = true;
+    });
   }
 
   ngOnChanges(changes: SimpleChanges) {
@@ -238,9 +252,17 @@ export class EventSchemaComponent implements OnChanges {
       this.schemaErrorHints.push(new UserErrorMessage('Missing Timestamp', 'The timestamp must be a UNIX timestamp in milliseconds. Edit the timestamp field or add an ingestion timestamp.'));
     }
 
-    const badFields = eventSchema.eventProperties.map(ep => this.fieldStatusInfo[ep.runtimeName]).find(field => field.fieldStatus !== 'GOOD');
-    if (badFields !== undefined) {
-      this.schemaErrorHints.push(new UserErrorMessage('Bad reading', 'At least one field could not be properly read. If this is a permanent problem, consider removing it - keeping this field might cause the adapter to fail or to omit sending events.', 'warning'));
+    if (this.fieldStatusInfo) {
+      const badFields = eventSchema.eventProperties
+        .filter(ep => this.fieldStatusInfo[ep.runtimeName] !== undefined)
+        .map(ep => this.fieldStatusInfo[ep.runtimeName])
+        .find(field => field.fieldStatus !== 'GOOD');
+      if (badFields !== undefined) {
+        this.schemaErrorHints.push(new UserErrorMessage(
+          'Bad reading',
+          'At least one field could not be properly read. If this is a permanent problem, consider removing it - keeping this field might cause the adapter to fail or to omit sending events.',
+          'warning'));
+      }
     }
 
     return hasTimestamp;
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/schema-editor-header/schema-editor-header.component.ts b/ui/src/app/connect/components/new-adapter/schema-editor/schema-editor-header/schema-editor-header.component.ts
index 27db921bc..7bcd8cc3c 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/schema-editor-header/schema-editor-header.component.ts
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/schema-editor-header/schema-editor-header.component.ts
@@ -33,7 +33,7 @@ export class SchemaEditorHeaderComponent implements OnInit {
   @Output() addStaticValuePropertyEmitter = new EventEmitter();
   @Output() addTimestampPropertyEmitter = new EventEmitter();
   @Output() guessSchemaEmitter = new EventEmitter();
-  @Output() togglePreviewEmitter = new EventEmitter();
+  @Output() updatePreviewEmitter = new EventEmitter();
   @Output() removeSelectedPropertiesEmitter = new EventEmitter();
 
   constructor() { }
@@ -57,10 +57,6 @@ export class SchemaEditorHeaderComponent implements OnInit {
     this.guessSchemaEmitter.emit();
   }
 
-  public togglePreview() {
-    this.togglePreviewEmitter.emit();
-  }
-
   public removeSelectedProperties() {
     this.removeSelectedPropertiesEmitter.emit();
   }
diff --git a/ui/src/app/connect/connect.module.ts b/ui/src/app/connect/connect.module.ts
index 1b23c969c..1bd77b851 100644
--- a/ui/src/app/connect/connect.module.ts
+++ b/ui/src/app/connect/connect.module.ts
@@ -84,6 +84,7 @@ import { EditValueTransformationComponent } from './dialog/edit-event-property/c
 import { SpEpSettingsSectionComponent } from './dialog/edit-event-property/components/ep-settings-section/ep-settings-section.component';
 import { SpAdapterOptionsPanelComponent } from './components/new-adapter/start-adapter-configuration/adapter-options-panel/adapter-options-panel.component';
 import { SpAdapterTemplateDialogComponent } from './dialog/adapter-template/adapter-template-dialog.component';
+import { JsonPrettyPrintPipe } from './filter/json-pretty-print.pipe';
 
 @NgModule({
   imports: [
@@ -146,6 +147,7 @@ import { SpAdapterTemplateDialogComponent } from './dialog/adapter-template/adap
     AdapterFilterPipe,
     FormatItemComponent,
     FormatListComponent,
+    JsonPrettyPrintPipe,
     NewAdapterComponent,
     SpAdapterTemplateDialogComponent,
     PipelineElementRuntimeInfoComponent,
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.scss b/ui/src/app/connect/filter/json-pretty-print.pipe.ts
similarity index 70%
copy from ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.scss
copy to ui/src/app/connect/filter/json-pretty-print.pipe.ts
index 58ba04bdd..5383916e4 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema-preview/event-schema-preview.component.scss
+++ b/ui/src/app/connect/filter/json-pretty-print.pipe.ts
@@ -16,3 +16,18 @@
  *
  */
 
+import { Injectable, Pipe, PipeTransform } from '@angular/core';
+
+@Pipe({
+  name: 'jsonpretty'
+})
+@Injectable({providedIn: 'root'})
+export class JsonPrettyPrintPipe implements PipeTransform {
+
+  transform(json) {
+    return JSON.stringify(json, undefined, 4)
+      .replace(/ /g, '&nbsp;')
+      .replace(/\n/g, '<br/>');
+  }
+
+}
diff --git a/ui/src/app/connect/services/rest.service.ts b/ui/src/app/connect/services/rest.service.ts
index 5d33cdc3c..17459d539 100644
--- a/ui/src/app/connect/services/rest.service.ts
+++ b/ui/src/app/connect/services/rest.service.ts
@@ -23,16 +23,20 @@ import { HttpClient } from '@angular/common/http';
 import { from, Observable } from 'rxjs';
 import { map } from 'rxjs/operators';
 import { UnitDescription } from '../model/UnitDescription';
-import { AdapterDescription, FormatDescription, GuessSchema, Message, SpDataStream, PlatformServicesCommons } from '@streampipes/platform-services';
+import {
+  AdapterDescription, FormatDescription, GuessSchema, Message, SpDataStream, PlatformServicesCommons,
+  AdapterEventPreview,
+  GuessTypeInfo
+} from '@streampipes/platform-services';
 import { AuthService } from '../../services/auth.service';
 
 @Injectable()
 export class RestService {
 
   constructor(
-      private http: HttpClient,
-      private platformServicesCommons: PlatformServicesCommons,
-      private authService: AuthService) {
+    private http: HttpClient,
+    private platformServicesCommons: PlatformServicesCommons,
+    private authService: AuthService) {
   }
 
   get connectPath() {
@@ -66,7 +70,11 @@ export class RestService {
       .pipe(map(response => {
         return GuessSchema.fromData(response as GuessSchema);
       }));
+  }
 
+  getAdapterEventPreview(adapterEventPreview: AdapterEventPreview): Observable<Record<string, GuessTypeInfo>> {
+    return this.http.post(`${this.connectPath}/master/guess/schema/preview`, adapterEventPreview)
+      .pipe(map(response => response as Record<string, GuessTypeInfo>));
   }
 
   getSourceDetails(sourceElementId): Observable<SpDataStream> {
@@ -78,7 +86,7 @@ export class RestService {
 
   getRuntimeInfo(sourceDescription): Observable<any> {
     return this.http.post(`${this.platformServicesCommons.apiBasePath}/pipeline-element/runtime`, sourceDescription, {
-      headers: { ignoreLoadingBar: '' }
+      headers: {ignoreLoadingBar: ''}
     });
   }
 


[incubator-streampipes] 09/15: [STREAMPIPES-577] Let PLC adapter support preview

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit f5b30b5e39098b373f9e16fe3b00b758c699168f
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Wed Aug 17 13:28:51 2022 +0200

    [STREAMPIPES-577] Let PLC adapter support preview
---
 .../worker/management/GuessManagement.java         |   1 -
 .../iiot/adapters/plc4x/s7/Plc4xS7Adapter.java     | 148 +++++++++++++--------
 .../adapters/plc4x/s7/PlcReadResponseHandler.java  |   9 ++
 .../event-schema/event-schema.component.html       |   2 +-
 4 files changed, 99 insertions(+), 61 deletions(-)

diff --git a/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/management/GuessManagement.java b/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/management/GuessManagement.java
index 9bc96f9b2..362c63a42 100644
--- a/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/management/GuessManagement.java
+++ b/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/management/GuessManagement.java
@@ -60,7 +60,6 @@ public class GuessManagement {
 
             throw new ParseException(errorClass + e.getMessage());
         } catch (Exception e) {
-            LOG.error("Unknown Error: " + e.toString());
             throw new AdapterException(e.getMessage(), e);
         }
 
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/Plc4xS7Adapter.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/Plc4xS7Adapter.java
index 2ee301ab1..f3e778a32 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/Plc4xS7Adapter.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/Plc4xS7Adapter.java
@@ -27,11 +27,12 @@ import org.apache.plc4x.java.api.types.PlcResponseCode;
 import org.apache.plc4x.java.utils.connectionpool.PooledPlcDriverManager;
 import org.apache.streampipes.connect.adapter.Adapter;
 import org.apache.streampipes.connect.adapter.util.PollingSettings;
-import org.apache.streampipes.connect.iiot.adapters.PullAdapter;
 import org.apache.streampipes.connect.api.exception.AdapterException;
+import org.apache.streampipes.connect.iiot.adapters.PullAdapter;
 import org.apache.streampipes.model.AdapterType;
 import org.apache.streampipes.model.connect.adapter.SpecificAdapterStreamDescription;
 import org.apache.streampipes.model.connect.guess.GuessSchema;
+import org.apache.streampipes.model.connect.guess.GuessTypeInfo;
 import org.apache.streampipes.model.schema.EventProperty;
 import org.apache.streampipes.model.schema.EventSchema;
 import org.apache.streampipes.model.staticproperty.CollectionStaticProperty;
@@ -56,8 +57,10 @@ import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
 
-public class Plc4xS7Adapter extends PullAdapter {
+public class Plc4xS7Adapter extends PullAdapter implements PlcReadResponseHandler {
 
     /**
      * A unique id to identify the Plc4xS7Adapter
@@ -133,31 +136,44 @@ public class Plc4xS7Adapter extends PullAdapter {
     public GuessSchema getSchema(SpecificAdapterStreamDescription adapterDescription) throws AdapterException {
 
         // Extract user input
-        getConfigurations(adapterDescription);
+        try {
+            getConfigurations(adapterDescription);
 
-        if (this.pollingInterval < 10) {
-            throw new AdapterException("Polling interval must be higher than 10. Current value: " + this.pollingInterval);
-        }
+            if (this.pollingInterval < 10) {
+                throw new AdapterException("Polling interval must be higher than 10. Current value: " + this.pollingInterval);
+            }
 
-        GuessSchema guessSchema = new GuessSchema();
+            GuessSchema guessSchema = new GuessSchema();
 
-        EventSchema eventSchema = new EventSchema();
-        List<EventProperty> allProperties = new ArrayList<>();
+            EventSchema eventSchema = new EventSchema();
+            List<EventProperty> allProperties = new ArrayList<>();
 
-        for (Map<String, String> node : this.nodes) {
-            Datatypes datatype = getStreamPipesDataType(node.get(PLC_NODE_TYPE).toUpperCase().replaceAll(" ", "_"));
-
-            allProperties.add(
-                    PrimitivePropertyBuilder
-                            .create(datatype, node.get(PLC_NODE_RUNTIME_NAME))
-                            .label(node.get(PLC_NODE_RUNTIME_NAME))
-                            .description("")
-                            .build());
-        }
+            for (Map<String, String> node : this.nodes) {
+                Datatypes datatype = getStreamPipesDataType(node.get(PLC_NODE_TYPE).toUpperCase().replaceAll(" ", "_"));
+
+                allProperties.add(
+                  PrimitivePropertyBuilder
+                    .create(datatype, node.get(PLC_NODE_RUNTIME_NAME))
+                    .label(node.get(PLC_NODE_RUNTIME_NAME))
+                    .description("")
+                    .build());
+            }
 
-        eventSchema.setEventProperties(allProperties);
-        guessSchema.setEventSchema(eventSchema);
-        return guessSchema;
+            var event = readPlcDataSynchronized();
+            var preview = event
+              .entrySet()
+              .stream()
+              .collect(Collectors.toMap(Map.Entry::getKey, e ->
+                new GuessTypeInfo(e.getValue().getClass().getCanonicalName(), e.getValue())));
+
+            eventSchema.setEventProperties(allProperties);
+            guessSchema.setEventSchema(eventSchema);
+            guessSchema.setEventPreview(List.of(preview));
+
+            return guessSchema;
+        } catch (PlcConnectionException | ExecutionException | InterruptedException | TimeoutException e) {
+            throw new AdapterException(e.getMessage(), e);
+        }
     }
 
     /**
@@ -171,7 +187,6 @@ public class Plc4xS7Adapter extends PullAdapter {
 
         this.driverManager = new PooledPlcDriverManager();
         try (PlcConnection plcConnection = this.driverManager.getConnection("s7://" + this.ip)) {
-
             if (!plcConnection.getMetadata().canRead()) {
                 this.LOG.error("The S7 on IP: " + this.ip + " does not support reading data");
             }
@@ -188,49 +203,40 @@ public class Plc4xS7Adapter extends PullAdapter {
      */
     @Override
     protected void pullData() {
-
         // Create PLC read request
-        try (PlcConnection plcConnection = this.driverManager.getConnection("s7://" + this.ip)) {
-            PlcReadRequest.Builder builder = plcConnection.readRequestBuilder();
-            for (Map<String, String> node : this.nodes) {
-                builder.addItem(node.get(PLC_NODE_NAME), node.get(PLC_NODE_NAME) + ":" + node.get(PLC_NODE_TYPE).toUpperCase().replaceAll(" ", "_"));
-            }
-            PlcReadRequest readRequest = builder.build();
-
-            // Execute the request
-            CompletableFuture<? extends PlcReadResponse> asyncResponse = readRequest.execute();
-
-            asyncResponse.whenComplete((response, throwable) -> {
-                // Create an event containing the value of the PLC
-                if (throwable != null) {
-                    throwable.printStackTrace();
-                    this.LOG.error(throwable.getMessage());
-                } else {
-                    Map<String, Object> event = new HashMap<>();
-                    for (Map<String, String> node : this.nodes) {
-                        if (response.getResponseCode(node.get(PLC_NODE_NAME)) == PlcResponseCode.OK) {
-                            event.put(node.get(PLC_NODE_RUNTIME_NAME), response.getObject(node.get(PLC_NODE_NAME)));
-                        } else {
-                            this.LOG.error("Error[" + node.get(PLC_NODE_NAME) + "]: " +
-                                    response.getResponseCode(node.get(PLC_NODE_NAME)).name());
-                        }
-                    }
-
-                    // publish the final event
-                    adapterPipeline.process(event);
-                }
-            });
-
-        } catch (InterruptedException | ExecutionException e) {
-            this.LOG.error(e.getMessage());
-            e.printStackTrace();
-        } catch (Exception e) {
+        try {
+            readPlcData(this);
+        } catch (PlcConnectionException e) {
             this.LOG.error("Could not establish connection to S7 with ip " + this.ip, e);
             e.printStackTrace();
         }
 
     }
 
+    private PlcReadRequest makeReadRequest() throws PlcConnectionException {
+        PlcConnection plcConnection = this.driverManager.getConnection("s7://" + this.ip);
+        PlcReadRequest.Builder builder = plcConnection.readRequestBuilder();
+        for (Map<String, String> node : this.nodes) {
+            builder.addItem(node.get(PLC_NODE_NAME), node.get(PLC_NODE_NAME) + ":" + node.get(PLC_NODE_TYPE).toUpperCase().replaceAll(" ", "_"));
+        }
+        return builder.build();
+    }
+
+    private void readPlcData(PlcReadResponseHandler handler) throws PlcConnectionException {
+        var readRequest = makeReadRequest();
+        // Execute the request
+        CompletableFuture<? extends PlcReadResponse> asyncResponse = readRequest.execute();
+        asyncResponse.whenComplete(handler::onReadResult);
+    }
+
+    private Map<String, Object> readPlcDataSynchronized() throws PlcConnectionException, ExecutionException, InterruptedException, TimeoutException, AdapterException {
+        this.before();
+        var readRequest = makeReadRequest();
+        // Execute the request
+        var readResponse = readRequest.execute().get(5000, TimeUnit.MILLISECONDS);
+        return makeEvent(readResponse);
+    }
+
     /**
      * Define the polling interval of this adapter. Default is to poll every second
      * @return
@@ -315,4 +321,28 @@ public class Plc4xS7Adapter extends PullAdapter {
         }
     }
 
+    @Override
+    public void onReadResult(PlcReadResponse response, Throwable throwable) {
+        if (throwable != null) {
+            throwable.printStackTrace();
+            this.LOG.error(throwable.getMessage());
+        } else {
+            var event = makeEvent(response);
+            // publish the final event
+            adapterPipeline.process(event);
+        }
+    }
+
+    private Map<String, Object> makeEvent(PlcReadResponse response) {
+        Map<String, Object> event = new HashMap<>();
+        for (Map<String, String> node : this.nodes) {
+            if (response.getResponseCode(node.get(PLC_NODE_NAME)) == PlcResponseCode.OK) {
+                event.put(node.get(PLC_NODE_RUNTIME_NAME), response.getObject(node.get(PLC_NODE_NAME)));
+            } else {
+                this.LOG.error("Error[" + node.get(PLC_NODE_NAME) + "]: " +
+                  response.getResponseCode(node.get(PLC_NODE_NAME)).name());
+            }
+        }
+        return event;
+    }
 }
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/PlcReadResponseHandler.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/PlcReadResponseHandler.java
new file mode 100644
index 000000000..071f27c5c
--- /dev/null
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/plc4x/s7/PlcReadResponseHandler.java
@@ -0,0 +1,9 @@
+package org.apache.streampipes.connect.iiot.adapters.plc4x.s7;
+
+import org.apache.plc4x.java.api.messages.PlcReadResponse;
+
+public interface PlcReadResponseHandler {
+
+  void onReadResult(PlcReadResponse response,
+                    Throwable throwable);
+}
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
index 6709dad75..a478d6fbb 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
@@ -32,7 +32,7 @@
             </div>
         </div>
 
-        <div fxLayout="column" fxFlex="100" *ngIf="!isLoading && schemaErrorHints.length > 0">
+        <div fxLayout="column" fxFlex="100" *ngIf="!isLoading && !isError && schemaErrorHints.length > 0">
             <div fxFlex="100"
                  fxLayout="column"
                  [ngClass]="schemaErrorHint.level === 'error' ? 'schema-validation schema-validation-error' : 'schema-validation schema-validation-warning'"


[incubator-streampipes] 02/15: [STREAMPIPES-577] Improve error handling in StreamPipes Connect

Posted by ri...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 7a1f96bfb31ceb81367c68f0ecae7176afab89eb
Author: Dominik Riemer <do...@gmail.com>
AuthorDate: Mon Aug 15 14:48:11 2022 +0200

    [STREAMPIPES-577] Improve error handling in StreamPipes Connect
---
 .../exceptions/SpConfigurationException.java       |  32 +++
 .../master/management/GuessManagement.java         |   6 +-
 .../master/management/WorkerRestClient.java        |  23 +-
 .../worker/rest/RuntimeResolvableResource.java     |  26 +-
 .../api/InvocablePipelineElementResource.java      |  27 +-
 .../api/ResolvesContainerProvidedOptions.java      |   3 +-
 .../ResolvesContainerProvidedOutputStrategy.java   |   4 +-
 .../api/RuntimeResolvableRequestHandler.java       |   5 +-
 .../container/api/SupportsRuntimeConfig.java       |   3 +-
 .../opcua/MiloOpcUaConfigurationProvider.java      |   8 +-
 .../connect/iiot/adapters/opcua/OpcUaAdapter.java  |   4 +-
 .../connect/iiot/adapters/opcua/SpOpcUaClient.java |   8 +-
 .../opcua/utils/ExceptionMessageExtractor.java     |  34 ++-
 .../iiot/adapters/opcua/utils/OpcUaUtil.java       | 287 +++++++++++----------
 .../org/apache/streampipes/model/MessageLd.java    | 117 ---------
 .../apache/streampipes/model/NotificationLd.java   |  95 -------
 .../streampipes/model/StreamPipesErrorMessage.java |  98 +++++++
 .../rest/impl/connect/GuessResource.java           |  14 +-
 .../impl/connect/RuntimeResolvableResource.java    |  19 +-
 ui/package.json                                    |   4 +-
 .../src/lib/model/gen/streampipes-model.ts         |  35 ++-
 ui/projects/streampipes/shared-ui/package.json     |   1 +
 .../exception-details-dialog.component.html        |  43 +++
 .../exception-details-dialog.component.scss}       |  39 +--
 .../exception-details-dialog.component.ts}         |  25 +-
 .../sp-exception-message.component.html            |  38 +++
 .../sp-exception-message.component.scss            |  17 +-
 .../sp-exception-message.component.ts              |  56 ++++
 .../shared-ui/src/lib/shared-ui.module.ts          |  10 +-
 .../streampipes/shared-ui/src/public-api.ts        |   2 +
 .../new-adapter/new-adapter.component.ts           |   1 -
 .../error-message/error-message.component.html     |  18 +-
 .../error-message/error-message.component.ts       |   3 +-
 .../event-schema/event-schema.component.html       |   2 +-
 .../event-schema/event-schema.component.ts         |   5 +-
 ui/src/app/connect/connect.module.ts               |   3 +-
 ...tatic-runtime-resolvable-any-input.component.ts |   3 +
 .../base-runtime-resolvable-input.ts               |  13 +
 ...tic-runtime-resolvable-oneof-input.component.ts |   3 +
 .../static-tree-input.component.html               |   3 +
 .../static-tree-input.component.ts                 |  31 +++
 41 files changed, 687 insertions(+), 481 deletions(-)

diff --git a/streampipes-commons/src/main/java/org/apache/streampipes/commons/exceptions/SpConfigurationException.java b/streampipes-commons/src/main/java/org/apache/streampipes/commons/exceptions/SpConfigurationException.java
new file mode 100644
index 000000000..a1436f4f1
--- /dev/null
+++ b/streampipes-commons/src/main/java/org/apache/streampipes/commons/exceptions/SpConfigurationException.java
@@ -0,0 +1,32 @@
+package org.apache.streampipes.commons.exceptions;
+
+public class SpConfigurationException extends Exception {
+
+  /**
+   * Creates a new Exception with the given message and null as the cause.
+   *
+   * @param message The exception message
+   */
+  public SpConfigurationException(String message) {
+    super(message);
+  }
+
+  /**
+   * Creates a new exception with a null message and the given cause.
+   *
+   * @param cause The exception that caused this exception
+   */
+  public SpConfigurationException(Throwable cause) {
+    super(cause);
+  }
+
+  /**
+   * Creates a new exception with the given message and cause
+   *
+   * @param message The exception message
+   * @param cause The exception that caused this exception
+   */
+  public SpConfigurationException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}
diff --git a/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java b/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java
index f5385e317..e54ade7a4 100644
--- a/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java
+++ b/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/GuessManagement.java
@@ -67,10 +67,10 @@ public class GuessManagement {
             if (httpResponse.getStatusLine().getStatusCode() == HttpStatus.SC_OK) {
                 return mapper.readValue(responseString, GuessSchema.class);
             }  else {
-                    ErrorMessage errorMessage = mapper.readValue(responseString, ErrorMessage.class);
+                ErrorMessage errorMessage = mapper.readValue(responseString, ErrorMessage.class);
 
-                    LOG.error(errorMessage.getElementName());
-                    throw new WorkerAdapterException(errorMessage);
+                LOG.error(errorMessage.getElementName());
+                throw new WorkerAdapterException(errorMessage);
             }
     }
 
diff --git a/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/WorkerRestClient.java b/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/WorkerRestClient.java
index 8c9f1074c..dab1e4bd7 100644
--- a/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/WorkerRestClient.java
+++ b/streampipes-connect-container-master/src/main/java/org/apache/streampipes/connect/container/master/management/WorkerRestClient.java
@@ -18,8 +18,11 @@
 
 package org.apache.streampipes.connect.container.master.management;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.commons.io.IOUtils;
 import org.apache.http.client.fluent.Request;
 import org.apache.http.entity.ContentType;
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.connect.api.exception.AdapterException;
 import org.apache.streampipes.connect.container.master.util.WorkerPaths;
 import org.apache.streampipes.model.connect.adapter.AdapterDescription;
@@ -37,6 +40,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.List;
 
 /**
@@ -146,19 +150,26 @@ public class WorkerRestClient {
 
     public static RuntimeOptionsResponse getConfiguration(String workerEndpoint,
                                                           String appId,
-                                                          RuntimeOptionsRequest runtimeOptionsRequest) throws AdapterException {
+                                                          RuntimeOptionsRequest runtimeOptionsRequest) throws AdapterException, SpConfigurationException {
         String url = workerEndpoint + WorkerPaths.getRuntimeResolvablePath(appId);
 
         try {
             String payload = JacksonSerializer.getObjectMapper().writeValueAsString(runtimeOptionsRequest);
-            String responseString = Request.Post(url)
+            var response = Request.Post(url)
                        .bodyString(payload, ContentType.APPLICATION_JSON)
                        .connectTimeout(1000)
                        .socketTimeout(100000)
-                       .execute().returnContent().asString();
+                       .execute()
+                        .returnResponse();
 
-            return JacksonSerializer.getObjectMapper().readValue(responseString, RuntimeOptionsResponse.class);
+            String responseString = IOUtils.toString(response.getEntity().getContent(), StandardCharsets.UTF_8);
 
+            if (response.getStatusLine().getStatusCode() == 200) {
+                return getSerializer().readValue(responseString, RuntimeOptionsResponse.class);
+            } else {
+                var exception = getSerializer().readValue(responseString, SpConfigurationException.class);
+                throw new SpConfigurationException(exception.getMessage(), exception.getCause());
+            }
         } catch (IOException e) {
             e.printStackTrace();
             throw new AdapterException("Could not resolve runtime configurations from " + url);
@@ -245,5 +256,9 @@ public class WorkerRestClient {
     private static IAdapterStorage getAdapterStorage() {
         return StorageDispatcher.INSTANCE.getNoSqlStore().getAdapterInstanceStorage();
     }
+
+    private static ObjectMapper getSerializer() {
+        return JacksonSerializer.getObjectMapper();
+    }
 }
 
diff --git a/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/rest/RuntimeResolvableResource.java b/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/rest/RuntimeResolvableResource.java
index 92a81fdec..72a209462 100644
--- a/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/rest/RuntimeResolvableResource.java
+++ b/streampipes-connect-container-worker/src/main/java/org/apache/streampipes/connect/container/worker/rest/RuntimeResolvableResource.java
@@ -18,6 +18,8 @@
 
 package org.apache.streampipes.connect.container.worker.rest;
 
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
+import org.apache.streampipes.commons.exceptions.SpRuntimeException;
 import org.apache.streampipes.connect.api.Connector;
 import org.apache.streampipes.connect.container.worker.management.RuntimeResovable;
 import org.apache.streampipes.container.api.ResolvesContainerProvidedOptions;
@@ -47,15 +49,21 @@ public class RuntimeResolvableResource extends AbstractSharedRestInterface {
         RuntimeOptionsResponse response;
         RuntimeResolvableRequestHandler handler = new RuntimeResolvableRequestHandler();
 
-        if (connector instanceof ResolvesContainerProvidedOptions) {
-            response = handler.handleRuntimeResponse((ResolvesContainerProvidedOptions) connector, runtimeOptionsRequest);
-        } else if (connector instanceof SupportsRuntimeConfig) {
-            response = handler.handleRuntimeResponse((SupportsRuntimeConfig) connector, runtimeOptionsRequest);
-        } else {
-            throw new WebApplicationException(javax.ws.rs.core.Response.Status.BAD_REQUEST);
+        try {
+            if (connector instanceof ResolvesContainerProvidedOptions) {
+                response = handler.handleRuntimeResponse((ResolvesContainerProvidedOptions) connector, runtimeOptionsRequest);
+                return ok(response);
+            } else if (connector instanceof SupportsRuntimeConfig) {
+                response = handler.handleRuntimeResponse((SupportsRuntimeConfig) connector, runtimeOptionsRequest);
+                return ok(response);
+            } else {
+                throw new SpRuntimeException("This element does not support dynamic options - is the pipeline element description up to date?");
+            }
+        } catch (SpConfigurationException e) {
+            return javax.ws.rs.core.Response
+              .status(400)
+              .entity(e)
+              .build();
         }
-
-        return ok(response);
     }
-
 }
diff --git a/streampipes-container/src/main/java/org/apache/streampipes/container/api/InvocablePipelineElementResource.java b/streampipes-container/src/main/java/org/apache/streampipes/container/api/InvocablePipelineElementResource.java
index 9936bd2bf..780b70434 100644
--- a/streampipes-container/src/main/java/org/apache/streampipes/container/api/InvocablePipelineElementResource.java
+++ b/streampipes-container/src/main/java/org/apache/streampipes/container/api/InvocablePipelineElementResource.java
@@ -19,6 +19,7 @@
 package org.apache.streampipes.container.api;
 
 import org.apache.streampipes.commons.constants.Envs;
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.commons.exceptions.SpRuntimeException;
 import org.apache.streampipes.container.declarer.Declarer;
 import org.apache.streampipes.container.declarer.InvocableDeclarer;
@@ -106,14 +107,21 @@ public abstract class InvocablePipelineElementResource<I extends InvocableStream
     D declarer = getDeclarerById(elementId);
     RuntimeOptionsResponse responseOptions;
 
-    if (declarer instanceof ResolvesContainerProvidedOptions) {
-      responseOptions = new RuntimeResolvableRequestHandler().handleRuntimeResponse((ResolvesContainerProvidedOptions) declarer, req);
-      return ok(responseOptions);
-    } else if (declarer instanceof SupportsRuntimeConfig) {
-      responseOptions = new RuntimeResolvableRequestHandler().handleRuntimeResponse((SupportsRuntimeConfig) declarer, req);
-      return ok(responseOptions);
-    } else {
-      throw new WebApplicationException(javax.ws.rs.core.Response.Status.BAD_REQUEST);
+    try {
+      if (declarer instanceof ResolvesContainerProvidedOptions) {
+        responseOptions = new RuntimeResolvableRequestHandler().handleRuntimeResponse((ResolvesContainerProvidedOptions) declarer, req);
+        return ok(responseOptions);
+      } else if (declarer instanceof SupportsRuntimeConfig) {
+          responseOptions = new RuntimeResolvableRequestHandler().handleRuntimeResponse((SupportsRuntimeConfig) declarer, req);
+          return ok(responseOptions);
+      } else {
+        return javax.ws.rs.core.Response.status(500).build();
+      }
+    } catch (SpConfigurationException e) {
+      return javax.ws.rs.core.Response
+        .status(400)
+        .entity(e)
+        .build();
     }
   }
 
@@ -131,8 +139,7 @@ public abstract class InvocablePipelineElementResource<I extends InvocableStream
                               (elementId);
       return ok(resolvesOutput.resolveOutputStrategy
               (runtimeOptionsRequest, getExtractor(runtimeOptionsRequest)));
-    } catch (SpRuntimeException e) {
-      e.printStackTrace();
+    } catch (SpRuntimeException | SpConfigurationException e) {
       return ok(new Response(elementId, false));
     }
   }
diff --git a/streampipes-container/src/main/java/org/apache/streampipes/container/api/ResolvesContainerProvidedOptions.java b/streampipes-container/src/main/java/org/apache/streampipes/container/api/ResolvesContainerProvidedOptions.java
index a32a1decb..110d95eaa 100644
--- a/streampipes-container/src/main/java/org/apache/streampipes/container/api/ResolvesContainerProvidedOptions.java
+++ b/streampipes-container/src/main/java/org/apache/streampipes/container/api/ResolvesContainerProvidedOptions.java
@@ -17,6 +17,7 @@
  */
 package org.apache.streampipes.container.api;
 
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.model.staticproperty.Option;
 import org.apache.streampipes.sdk.extractor.StaticPropertyExtractor;
 
@@ -29,5 +30,5 @@ import java.util.List;
 public interface ResolvesContainerProvidedOptions {
 
   List<Option> resolveOptions(String staticPropertyInternalName,
-                              StaticPropertyExtractor parameterExtractor);
+                              StaticPropertyExtractor parameterExtractor) throws SpConfigurationException;
 }
diff --git a/streampipes-container/src/main/java/org/apache/streampipes/container/api/ResolvesContainerProvidedOutputStrategy.java b/streampipes-container/src/main/java/org/apache/streampipes/container/api/ResolvesContainerProvidedOutputStrategy.java
index 83ed622b0..ab44539ae 100644
--- a/streampipes-container/src/main/java/org/apache/streampipes/container/api/ResolvesContainerProvidedOutputStrategy.java
+++ b/streampipes-container/src/main/java/org/apache/streampipes/container/api/ResolvesContainerProvidedOutputStrategy.java
@@ -17,7 +17,7 @@
  */
 package org.apache.streampipes.container.api;
 
-import org.apache.streampipes.commons.exceptions.SpRuntimeException;
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.model.base.InvocableStreamPipesEntity;
 import org.apache.streampipes.model.schema.EventSchema;
 import org.apache.streampipes.sdk.extractor.AbstractParameterExtractor;
@@ -25,5 +25,5 @@ import org.apache.streampipes.sdk.extractor.AbstractParameterExtractor;
 public interface ResolvesContainerProvidedOutputStrategy<T extends InvocableStreamPipesEntity, P
         extends AbstractParameterExtractor<T>> {
 
-  EventSchema resolveOutputStrategy(T processingElement, P parameterExtractor) throws SpRuntimeException;
+  EventSchema resolveOutputStrategy(T processingElement, P parameterExtractor) throws SpConfigurationException;
 }
diff --git a/streampipes-container/src/main/java/org/apache/streampipes/container/api/RuntimeResolvableRequestHandler.java b/streampipes-container/src/main/java/org/apache/streampipes/container/api/RuntimeResolvableRequestHandler.java
index aba13222a..91e18f97f 100644
--- a/streampipes-container/src/main/java/org/apache/streampipes/container/api/RuntimeResolvableRequestHandler.java
+++ b/streampipes-container/src/main/java/org/apache/streampipes/container/api/RuntimeResolvableRequestHandler.java
@@ -18,6 +18,7 @@
 
 package org.apache.streampipes.container.api;
 
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.model.runtime.RuntimeOptionsRequest;
 import org.apache.streampipes.model.runtime.RuntimeOptionsResponse;
 import org.apache.streampipes.model.staticproperty.Option;
@@ -31,7 +32,7 @@ public class RuntimeResolvableRequestHandler {
 
   // for backwards compatibility
   public RuntimeOptionsResponse handleRuntimeResponse(ResolvesContainerProvidedOptions resolvesOptions,
-                                                      RuntimeOptionsRequest req) {
+                                                      RuntimeOptionsRequest req) throws SpConfigurationException {
     List<Option> availableOptions =
             resolvesOptions.resolveOptions(req.getRequestId(),
                     makeExtractor(req));
@@ -43,7 +44,7 @@ public class RuntimeResolvableRequestHandler {
   }
 
   public RuntimeOptionsResponse handleRuntimeResponse(SupportsRuntimeConfig declarer,
-                                                      RuntimeOptionsRequest req) {
+                                                      RuntimeOptionsRequest req) throws SpConfigurationException {
     StaticProperty result = declarer.resolveConfiguration(
             req.getRequestId(),
             makeExtractor(req));
diff --git a/streampipes-container/src/main/java/org/apache/streampipes/container/api/SupportsRuntimeConfig.java b/streampipes-container/src/main/java/org/apache/streampipes/container/api/SupportsRuntimeConfig.java
index 10cacd125..4ebd76854 100644
--- a/streampipes-container/src/main/java/org/apache/streampipes/container/api/SupportsRuntimeConfig.java
+++ b/streampipes-container/src/main/java/org/apache/streampipes/container/api/SupportsRuntimeConfig.java
@@ -18,12 +18,13 @@
 
 package org.apache.streampipes.container.api;
 
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.model.staticproperty.StaticProperty;
 import org.apache.streampipes.sdk.extractor.StaticPropertyExtractor;
 
 public interface SupportsRuntimeConfig {
 
   StaticProperty resolveConfiguration(String staticPropertyInternalName,
-                                      StaticPropertyExtractor extractor);
+                                      StaticPropertyExtractor extractor) throws SpConfigurationException;
 
 }
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/MiloOpcUaConfigurationProvider.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/MiloOpcUaConfigurationProvider.java
index e4336d842..7145858ea 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/MiloOpcUaConfigurationProvider.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/MiloOpcUaConfigurationProvider.java
@@ -18,6 +18,7 @@
 
 package org.apache.streampipes.connect.iiot.adapters.opcua;
 
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.connect.iiot.adapters.opcua.configuration.SpOpcUaConfig;
 import org.eclipse.milo.opcua.sdk.client.api.config.OpcUaClientConfig;
 import org.eclipse.milo.opcua.sdk.client.api.config.OpcUaClientConfigBuilder;
@@ -31,10 +32,11 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
 
 public class MiloOpcUaConfigurationProvider {
 
-  public OpcUaClientConfig makeClientConfig(SpOpcUaConfig spOpcConfig) throws Exception {
+  public OpcUaClientConfig makeClientConfig(SpOpcUaConfig spOpcConfig) throws ExecutionException, InterruptedException, SpConfigurationException, URISyntaxException {
     String opcServerUrl = spOpcConfig.getOpcServerURL();
     List<EndpointDescription> endpoints = DiscoveryClient.getEndpoints(opcServerUrl).get();
     String host = opcServerUrl.split("://")[1].split(":")[0];
@@ -43,7 +45,7 @@ public class MiloOpcUaConfigurationProvider {
             .stream()
             .filter(e -> e.getSecurityPolicyUri().equals(SecurityPolicy.None.getUri()))
             .findFirst()
-            .orElseThrow(() -> new Exception("No endpoint with security policy none"));
+            .orElseThrow(() -> new SpConfigurationException("No endpoint with security policy none"));
 
     tmpEndpoint = updateEndpointUrl(tmpEndpoint, host);
     endpoints = Collections.singletonList(tmpEndpoint);
@@ -51,7 +53,7 @@ public class MiloOpcUaConfigurationProvider {
     EndpointDescription endpoint = endpoints
             .stream()
             .filter(e -> e.getSecurityPolicyUri().equals(SecurityPolicy.None.getUri()))
-            .findFirst().orElseThrow(() -> new Exception("no desired endpoints returned"));
+            .findFirst().orElseThrow(() -> new SpConfigurationException("no desired endpoints returned"));
 
     return buildConfig(endpoint, spOpcConfig);
   }
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaAdapter.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaAdapter.java
index 57039d1ee..76b3df7e2 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaAdapter.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/OpcUaAdapter.java
@@ -18,6 +18,7 @@
 
 package org.apache.streampipes.connect.iiot.adapters.opcua;
 
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.connect.adapter.Adapter;
 import org.apache.streampipes.connect.adapter.util.PollingSettings;
 import org.apache.streampipes.connect.api.exception.AdapterException;
@@ -270,7 +271,8 @@ public class OpcUaAdapter extends PullAdapter implements SupportsRuntimeConfig {
     }
 
     @Override
-    public StaticProperty resolveConfiguration(String staticPropertyInternalName, StaticPropertyExtractor extractor) {
+    public StaticProperty resolveConfiguration(String staticPropertyInternalName,
+                                               StaticPropertyExtractor extractor) throws SpConfigurationException {
         return OpcUaUtil.resolveConfiguration(staticPropertyInternalName, extractor);
     }
 }
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/SpOpcUaClient.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/SpOpcUaClient.java
index da00cbf1b..6e7b9af8f 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/SpOpcUaClient.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/SpOpcUaClient.java
@@ -19,12 +19,14 @@
 package org.apache.streampipes.connect.iiot.adapters.opcua;
 
 
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.connect.iiot.adapters.opcua.configuration.SpOpcUaConfig;
 import org.eclipse.milo.opcua.sdk.client.OpcUaClient;
 import org.eclipse.milo.opcua.sdk.client.api.config.OpcUaClientConfig;
 import org.eclipse.milo.opcua.sdk.client.api.subscriptions.UaMonitoredItem;
 import org.eclipse.milo.opcua.sdk.client.api.subscriptions.UaSubscription;
 import org.eclipse.milo.opcua.stack.core.AttributeId;
+import org.eclipse.milo.opcua.stack.core.UaException;
 import org.eclipse.milo.opcua.stack.core.types.builtin.DataValue;
 import org.eclipse.milo.opcua.stack.core.types.builtin.NodeId;
 import org.eclipse.milo.opcua.stack.core.types.builtin.QualifiedName;
@@ -37,9 +39,11 @@ import org.eclipse.milo.opcua.stack.core.types.structured.ReadValueId;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicLong;
 
 import static org.eclipse.milo.opcua.stack.core.types.builtin.unsigned.Unsigned.uint;
@@ -71,9 +75,9 @@ public class SpOpcUaClient {
     /***
      * Establishes appropriate connection to OPC UA endpoint depending on the {@link SpOpcUaClient} instance
      *
-     * @throws Exception An exception occurring during OPC connection
+     * @throws UaException An exception occurring during OPC connection
      */
-    public void connect() throws Exception {
+    public void connect() throws UaException, ExecutionException, InterruptedException, SpConfigurationException, URISyntaxException {
         OpcUaClientConfig clientConfig = new MiloOpcUaConfigurationProvider().makeClientConfig(spOpcConfig);
         this.client = OpcUaClient.create(clientConfig);
         client.connect().get();
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/ErrorMessageLd.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/ExceptionMessageExtractor.java
similarity index 61%
rename from streampipes-model/src/main/java/org/apache/streampipes/model/ErrorMessageLd.java
rename to streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/ExceptionMessageExtractor.java
index 091f35b1d..7c1b45396 100644
--- a/streampipes-model/src/main/java/org/apache/streampipes/model/ErrorMessageLd.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/ExceptionMessageExtractor.java
@@ -16,25 +16,23 @@
  *
  */
 
-package org.apache.streampipes.model;
+package org.apache.streampipes.connect.iiot.adapters.opcua.utils;
 
-import java.util.List;
+import org.eclipse.milo.opcua.stack.core.UaException;
 
-public class ErrorMessageLd extends MessageLd {
+public class ExceptionMessageExtractor {
 
-	public ErrorMessageLd() {
-		super(false);
-	}
-
-	public ErrorMessageLd(NotificationLd...notifications) {
-		super(false, notifications);
-	}
-
-	public ErrorMessageLd(List<NotificationLd> notifications) {
-		super(false, notifications.toArray(new NotificationLd[0]));
-	}
-
-	public ErrorMessageLd(String elementName, List<NotificationLd> notifications) {
-		super(false, notifications, elementName);
-	}
+  public static String getDescription(UaException e) {
+    String[] parts = e.getMessage().split(", ");
+    if (parts.length > 1) {
+      String[] kv = parts[1].split("=");
+      if (kv.length > 1) {
+        return kv[1];
+      } else {
+        return parts[1];
+      }
+    } else {
+      return e.getMessage();
+    }
+  }
 }
diff --git a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/OpcUaUtil.java b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/OpcUaUtil.java
index 3bf5430c8..d9f95872f 100644
--- a/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/OpcUaUtil.java
+++ b/streampipes-extensions/streampipes-connect-adapters-iiot/src/main/java/org/apache/streampipes/connect/iiot/adapters/opcua/utils/OpcUaUtil.java
@@ -18,6 +18,7 @@
 
 package org.apache.streampipes.connect.iiot.adapters.opcua.utils;
 
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.connect.api.exception.AdapterException;
 import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.connect.iiot.adapters.opcua.OpcNode;
@@ -37,170 +38,174 @@ import org.eclipse.milo.opcua.stack.core.types.builtin.NodeId;
 import org.eclipse.milo.opcua.stack.core.types.builtin.unsigned.UInteger;
 
 import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
 
 /***
  * Collection of several utility functions in context of OPC UA
  */
 public class OpcUaUtil {
 
-    /***
-     * Ensures server address starts with {@code opc.tcp://}
-     * @param serverAddress server address as given by user
-     * @return correctly formated server address
-     */
-    public static String formatServerAddress(String serverAddress) {
+  /***
+   * Ensures server address starts with {@code opc.tcp://}
+   * @param serverAddress server address as given by user
+   * @return correctly formated server address
+   */
+  public static String formatServerAddress(String serverAddress) {
 
-        if (!serverAddress.startsWith("opc.tcp://")) {
-            serverAddress = "opc.tcp://" + serverAddress;
-        }
-
-        return serverAddress;
+    if (!serverAddress.startsWith("opc.tcp://")) {
+      serverAddress = "opc.tcp://" + serverAddress;
     }
 
-    /***
-     * OPC UA specific implementation of {@link org.apache.streampipes.connect.adapter.Adapter}
-     * @param adapterStreamDescription
-     * @return guess schema
-     * @throws AdapterException
-     * @throws ParseException
-     */
-    public static GuessSchema getSchema(SpecificAdapterStreamDescription adapterStreamDescription)
-            throws AdapterException, ParseException {
-        GuessSchema guessSchema = new GuessSchema();
-        EventSchema eventSchema = new EventSchema();
-        List<EventProperty> allProperties = new ArrayList<>();
-
-        SpOpcUaClient spOpcUaClient = new SpOpcUaClient(SpOpcUaConfigBuilder.from(adapterStreamDescription));
-
-        try {
-            spOpcUaClient.connect();
-            OpcUaNodeBrowser nodeBrowser =
-                    new OpcUaNodeBrowser(spOpcUaClient.getClient(), spOpcUaClient.getSpOpcConfig());
-            List<OpcNode> selectedNodes = nodeBrowser.findNodes();
-
-            if (!selectedNodes.isEmpty()) {
-                for (OpcNode opcNode : selectedNodes) {
-                    if (opcNode.hasUnitId()) {
-                        allProperties.add(PrimitivePropertyBuilder
-                                .create(opcNode.getType(), opcNode.getLabel())
-                                .label(opcNode.getLabel())
-                                .measurementUnit(new URI(opcNode.getQudtURI()))
-                                .build());
-                    } else {
-                        allProperties.add(PrimitivePropertyBuilder
-                                .create(opcNode.getType(), opcNode.getLabel())
-                                .label(opcNode.getLabel())
-                                .build());
-                    }
-
-                }
-            }
-
-            spOpcUaClient.disconnect();
-
-        } catch (Exception e) {
-            throw new AdapterException("Could not guess schema for opc node:  " + e.getMessage(), e.getCause());
+    return serverAddress;
+  }
+
+  /***
+   * OPC UA specific implementation of {@link org.apache.streampipes.connect.adapter.Adapter}
+   * @param adapterStreamDescription
+   * @return guess schema
+   * @throws AdapterException
+   * @throws ParseException
+   */
+  public static GuessSchema getSchema(SpecificAdapterStreamDescription adapterStreamDescription)
+    throws AdapterException, ParseException {
+    GuessSchema guessSchema = new GuessSchema();
+    EventSchema eventSchema = new EventSchema();
+    List<EventProperty> allProperties = new ArrayList<>();
+
+    SpOpcUaClient spOpcUaClient = new SpOpcUaClient(SpOpcUaConfigBuilder.from(adapterStreamDescription));
+
+    try {
+      spOpcUaClient.connect();
+      OpcUaNodeBrowser nodeBrowser =
+        new OpcUaNodeBrowser(spOpcUaClient.getClient(), spOpcUaClient.getSpOpcConfig());
+      List<OpcNode> selectedNodes = nodeBrowser.findNodes();
+
+      if (!selectedNodes.isEmpty()) {
+        for (OpcNode opcNode : selectedNodes) {
+          if (opcNode.hasUnitId()) {
+            allProperties.add(PrimitivePropertyBuilder
+              .create(opcNode.getType(), opcNode.getLabel())
+              .label(opcNode.getLabel())
+              .measurementUnit(new URI(opcNode.getQudtURI()))
+              .build());
+          } else {
+            allProperties.add(PrimitivePropertyBuilder
+              .create(opcNode.getType(), opcNode.getLabel())
+              .label(opcNode.getLabel())
+              .build());
+          }
+
         }
+      }
 
-        eventSchema.setEventProperties(allProperties);
-        guessSchema.setEventSchema(eventSchema);
+      spOpcUaClient.disconnect();
 
-        return guessSchema;
+    } catch (Exception e) {
+      throw new AdapterException("Could not guess schema for opc node:  " + e.getMessage(), e.getCause());
     }
 
-
-    /***
-     * OPC UA specific implementation of {@link
-     * org.apache.streampipes.container.api.ResolvesContainerProvidedOptions#
-     * resolveOptions(String, StaticPropertyExtractor)}.
-     * @param internalName The internal name of the Static Property
-     * @param parameterExtractor
-     * @return {@code List<Option>} with available node names for the given OPC UA configuration
-     */
-    public static RuntimeResolvableTreeInputStaticProperty
-    resolveConfiguration(String internalName,
-                            StaticPropertyExtractor parameterExtractor) {
-
-        RuntimeResolvableTreeInputStaticProperty config = parameterExtractor
-                .getStaticPropertyByName(internalName, RuntimeResolvableTreeInputStaticProperty.class);
-        // access mode and host/url have to be selected
-        try {
-            parameterExtractor.selectedAlternativeInternalId(OpcUaLabels.OPC_HOST_OR_URL.name());
-            parameterExtractor.selectedAlternativeInternalId(OpcUaLabels.ACCESS_MODE.name());
-        } catch (NullPointerException nullPointerException) {
-            return config;
-        }
-
-        SpOpcUaClient spOpcUaClient = new SpOpcUaClient(SpOpcUaConfigBuilder.from(parameterExtractor));
-        try {
-            spOpcUaClient.connect();
-            OpcUaNodeBrowser nodeBrowser =
-                    new OpcUaNodeBrowser(spOpcUaClient.getClient(), spOpcUaClient.getSpOpcConfig());
-            config.setNodes(nodeBrowser.buildNodeTreeFromOrigin());
-            spOpcUaClient.disconnect();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-
-        return config;
+    eventSchema.setEventProperties(allProperties);
+    guessSchema.setEventSchema(eventSchema);
+
+    return guessSchema;
+  }
+
+
+  /***
+   * OPC UA specific implementation of {@link
+   * org.apache.streampipes.container.api.ResolvesContainerProvidedOptions#
+   * resolveOptions(String, StaticPropertyExtractor)}.
+   * @param internalName The internal name of the Static Property
+   * @param parameterExtractor to extract parameters from the OPC UA config
+   * @return {@code List<Option>} with available node names for the given OPC UA configuration
+   */
+  public static RuntimeResolvableTreeInputStaticProperty resolveConfiguration(String internalName,
+                                                                              StaticPropertyExtractor parameterExtractor) throws SpConfigurationException {
+
+    RuntimeResolvableTreeInputStaticProperty config = parameterExtractor
+      .getStaticPropertyByName(internalName, RuntimeResolvableTreeInputStaticProperty.class);
+    // access mode and host/url have to be selected
+    try {
+      parameterExtractor.selectedAlternativeInternalId(OpcUaLabels.OPC_HOST_OR_URL.name());
+      parameterExtractor.selectedAlternativeInternalId(OpcUaLabels.ACCESS_MODE.name());
+    } catch (NullPointerException nullPointerException) {
+      return config;
     }
 
-    public static String getRuntimeNameOfNode(NodeId nodeId) {
-        String[] keys = nodeId.getIdentifier().toString().split("\\.");
-        String key;
+    SpOpcUaClient spOpcUaClient = new SpOpcUaClient(SpOpcUaConfigBuilder.from(parameterExtractor));
 
-        if (keys.length > 0) {
-            key = keys[keys.length - 1];
-        } else {
-            key = nodeId.getIdentifier().toString();
-        }
+    try {
+      spOpcUaClient.connect();
+      OpcUaNodeBrowser nodeBrowser =
+        new OpcUaNodeBrowser(spOpcUaClient.getClient(), spOpcUaClient.getSpOpcConfig());
+      config.setNodes(nodeBrowser.buildNodeTreeFromOrigin());
+      spOpcUaClient.disconnect();
 
-        return key;
+      return config;
+    } catch (UaException e) {
+      throw new SpConfigurationException(ExceptionMessageExtractor.getDescription(e), e);
+    } catch (ExecutionException | InterruptedException | URISyntaxException e) {
+      throw new SpConfigurationException("Could not connect to the OPC UA server with the provided settings", e);
     }
+  }
 
-    /**
-     * connects to each node individually and updates the data type in accordance to the data from the server.
-     *
-     * @param opcNodes List of opcNodes where the data type is not determined appropriately
-     */
-    public static void retrieveDataTypesFromServer(OpcUaClient client, List<OpcNode> opcNodes) throws AdapterException {
-
-        for (OpcNode opcNode : opcNodes) {
-            try {
-                UInteger dataTypeId =
-                        (UInteger) client.getAddressSpace().getVariableNode(opcNode.getNodeId()).getDataType()
-                                .getIdentifier();
-                OpcUaTypes.getType(dataTypeId);
-                opcNode.setType(OpcUaTypes.getType(dataTypeId));
-            } catch (UaException e) {
-                throw new AdapterException("Could not guess schema for opc node! " + e.getMessage());
-            }
-        }
+  public static String getRuntimeNameOfNode(NodeId nodeId) {
+    String[] keys = nodeId.getIdentifier().toString().split("\\.");
+    String key;
+
+    if (keys.length > 0) {
+      key = keys[keys.length - 1];
+    } else {
+      key = nodeId.getIdentifier().toString();
     }
 
-    /***
-     * Enum for all possible labels in the context of OPC UA adapters
-     */
-    public enum OpcUaLabels {
-        OPC_HOST_OR_URL,
-        OPC_URL,
-        OPC_HOST,
-        OPC_SERVER_URL,
-        OPC_SERVER_HOST,
-        OPC_SERVER_PORT,
-        NAMESPACE_INDEX,
-        NODE_ID,
-        ACCESS_MODE,
-        USERNAME_GROUP,
-        USERNAME,
-        PASSWORD,
-        UNAUTHENTICATED,
-        AVAILABLE_NODES,
-        PULLING_INTERVAL,
-        ADAPTER_TYPE,
-        PULL_MODE,
-        SUBSCRIPTION_MODE;
+    return key;
+  }
+
+  /**
+   * connects to each node individually and updates the data type in accordance to the data from the server.
+   *
+   * @param opcNodes List of opcNodes where the data type is not determined appropriately
+   */
+  public static void retrieveDataTypesFromServer(OpcUaClient client, List<OpcNode> opcNodes) throws AdapterException {
+
+    for (OpcNode opcNode : opcNodes) {
+      try {
+        UInteger dataTypeId =
+          (UInteger) client.getAddressSpace().getVariableNode(opcNode.getNodeId()).getDataType()
+            .getIdentifier();
+        OpcUaTypes.getType(dataTypeId);
+        opcNode.setType(OpcUaTypes.getType(dataTypeId));
+      } catch (UaException e) {
+        throw new AdapterException("Could not guess schema for opc node! " + e.getMessage());
+      }
     }
+  }
+
+  /***
+   * Enum for all possible labels in the context of OPC UA adapters
+   */
+  public enum OpcUaLabels {
+    OPC_HOST_OR_URL,
+    OPC_URL,
+    OPC_HOST,
+    OPC_SERVER_URL,
+    OPC_SERVER_HOST,
+    OPC_SERVER_PORT,
+    NAMESPACE_INDEX,
+    NODE_ID,
+    ACCESS_MODE,
+    USERNAME_GROUP,
+    USERNAME,
+    PASSWORD,
+    UNAUTHENTICATED,
+    AVAILABLE_NODES,
+    PULLING_INTERVAL,
+    ADAPTER_TYPE,
+    PULL_MODE,
+    SUBSCRIPTION_MODE;
+  }
 }
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/MessageLd.java b/streampipes-model/src/main/java/org/apache/streampipes/model/MessageLd.java
deleted file mode 100644
index 61b3cd87c..000000000
--- a/streampipes-model/src/main/java/org/apache/streampipes/model/MessageLd.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * 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.model;
-
-import org.apache.commons.lang3.RandomStringUtils;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-public class MessageLd {
-
-	private static final String prefix = "urn:streampipes.org:spi:";
-
-	private String elementId;
-
-	private boolean success;
-
-	private String elementName;
-
-	private List<NotificationLd> notifications;
-
-	public MessageLd() {
-		this.elementId = prefix
-				+ this.getClass().getSimpleName().toLowerCase()
-				+ ":"
-				+ RandomStringUtils.randomAlphabetic(6);
-		this.elementName = "";
-	}
-
-	public MessageLd(MessageLd other) {
-		this();
-		this.success = other.isSuccess();
-		this.elementName = other.getElementName();
-		this.notifications = other.getNotifications();
-	}
-
-	public MessageLd(boolean success){
-		this();
-		this.success = success;
-		this.notifications = null;
-	}
-
-	public MessageLd(boolean success, List<NotificationLd> notifications) {
-		this();
-		this.success = success;
-		this.notifications = notifications;
-	}
-
-	public MessageLd(boolean success, List<NotificationLd> notifications, String elementName) {
-		this(success, notifications);
-		this.elementName = elementName;
-	}
-
-
-	public MessageLd(boolean success, NotificationLd...notifications) {
-		this();
-		this.success = success;
-		this.notifications = new ArrayList<>();
-		this.notifications.addAll(Arrays.asList(notifications));
-	}
-
-	public boolean isSuccess() {
-		return success;
-	}
-
-	public void setSuccess(boolean success) {
-		this.success = success;
-	}
-
-	public List<NotificationLd> getNotifications() {
-		return notifications;
-	}
-
-	public void setNotifications(List<NotificationLd> notifications) {
-		this.notifications = notifications;
-	}
-	
-	public boolean addNotification(NotificationLd notification)
-	{
-		return notifications.add(notification);
-	}
-
-	public String getElementName() {
-		return elementName;
-	}
-
-	public void setElementName(String elementName) {
-		this.elementName = elementName;
-	}
-
-	public String getElementId() {
-		return elementId;
-	}
-
-	public void setElementId(String elementId) {
-		this.elementId = elementId;
-	}
-	
-	
-}
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/NotificationLd.java b/streampipes-model/src/main/java/org/apache/streampipes/model/NotificationLd.java
deleted file mode 100644
index 7b6b7b89c..000000000
--- a/streampipes-model/src/main/java/org/apache/streampipes/model/NotificationLd.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * 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.model;
-
-import org.apache.commons.lang3.RandomStringUtils;
-
-public class NotificationLd {
-
-    private static final String prefix = "urn:streampipes.org:spi:";
-
-    private String elementId;
-
-    private String title;
-
-    private String description;
-
-    private String additionalInformation;
-
-    public NotificationLd() {
-        this.elementId = prefix
-                + this.getClass().getSimpleName().toLowerCase()
-                + ":"
-                + RandomStringUtils.randomAlphabetic(6);
-        this.additionalInformation = "";
-    }
-
-    public NotificationLd(NotificationLd other) {
-        this();
-        this.title = other.getTitle();
-        this.description = other.getDescription();
-        this.additionalInformation = other.getAdditionalInformation();
-    }
-
-    public NotificationLd(String title, String description) {
-        this();
-        this.title = title;
-        this.description = description;
-    }
-
-    public NotificationLd(String title, String description,
-                        String additionalInformation) {
-        this();
-        this.title = title;
-        this.description = description;
-        this.additionalInformation = additionalInformation;
-    }
-
-    public String getTitle() {
-        return title;
-    }
-
-    public void setTitle(String title) {
-        this.title = title;
-    }
-
-    public String getDescription() {
-        return description;
-    }
-
-    public void setDescription(String description) {
-        this.description = description;
-    }
-
-    public String getAdditionalInformation() {
-        return additionalInformation;
-    }
-
-    public void setAdditionalInformation(String additionalInformation) {
-        this.additionalInformation = additionalInformation;
-    }
-
-    public String getElementId() {
-        return elementId;
-    }
-
-    public void setElementId(String elementId) {
-        this.elementId = elementId;
-    }
-}
diff --git a/streampipes-model/src/main/java/org/apache/streampipes/model/StreamPipesErrorMessage.java b/streampipes-model/src/main/java/org/apache/streampipes/model/StreamPipesErrorMessage.java
new file mode 100644
index 000000000..d2d964090
--- /dev/null
+++ b/streampipes-model/src/main/java/org/apache/streampipes/model/StreamPipesErrorMessage.java
@@ -0,0 +1,98 @@
+/*
+ * 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.model;
+
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.streampipes.model.shared.annotation.TsModel;
+
+@TsModel
+public class StreamPipesErrorMessage {
+
+  private String level;
+  private String title;
+  private String detail;
+
+  private String cause;
+  private String fullStackTrace;
+
+  public StreamPipesErrorMessage() {
+  }
+
+  public static StreamPipesErrorMessage from(Exception exception) {
+    String cause = exception.getCause() != null ? exception.getCause().getMessage() : exception.getMessage();
+    return new StreamPipesErrorMessage(
+      "error",
+      exception.getMessage(),
+      "",
+      ExceptionUtils.getStackTrace(exception),
+      cause);
+  }
+
+  public StreamPipesErrorMessage(String level,
+                                 String title,
+                                 String detail,
+                                 String fullStackTrace,
+                                 String cause) {
+    this.level = level;
+    this.title = title;
+    this.detail = detail;
+    this.fullStackTrace = fullStackTrace;
+    this.cause = cause;
+  }
+
+  public String getLevel() {
+    return level;
+  }
+
+  public void setLevel(String level) {
+    this.level = level;
+  }
+
+  public String getTitle() {
+    return title;
+  }
+
+  public void setTitle(String title) {
+    this.title = title;
+  }
+
+  public String getDetail() {
+    return detail;
+  }
+
+  public void setDetail(String detail) {
+    this.detail = detail;
+  }
+
+  public String getFullStackTrace() {
+    return fullStackTrace;
+  }
+
+  public void setFullStackTrace(String fullStackTrace) {
+    this.fullStackTrace = fullStackTrace;
+  }
+
+  public String getCause() {
+    return cause;
+  }
+
+  public void setCause(String cause) {
+    this.cause = cause;
+  }
+}
diff --git a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/GuessResource.java b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/GuessResource.java
index 6593be344..7cdd81281 100644
--- a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/GuessResource.java
+++ b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/GuessResource.java
@@ -18,12 +18,13 @@
 
 package org.apache.streampipes.rest.impl.connect;
 
+import org.apache.streampipes.commons.exceptions.NoServiceEndpointsAvailableException;
 import org.apache.streampipes.connect.api.exception.ParseException;
 import org.apache.streampipes.connect.api.exception.WorkerAdapterException;
 import org.apache.streampipes.connect.container.master.management.GuessManagement;
+import org.apache.streampipes.model.StreamPipesErrorMessage;
 import org.apache.streampipes.model.connect.adapter.AdapterDescription;
 import org.apache.streampipes.model.connect.guess.GuessSchema;
-import org.apache.streampipes.model.message.Notifications;
 import org.apache.streampipes.rest.shared.annotation.JacksonSerialized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -33,6 +34,7 @@ import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import java.io.IOException;
 
 
 @Path("/v2/connect/master/guess")
@@ -56,12 +58,12 @@ public class GuessResource extends AbstractAdapterResource<GuessManagement> {
           return ok(result);
       } catch (ParseException e) {
           LOG.error("Error while parsing events: ", e);
-          return serverError(Notifications.error(e.getMessage()));
+          return badRequest(StreamPipesErrorMessage.from(e));
       } catch (WorkerAdapterException e) {
-          return serverError(e.getContent());
-      } catch (Exception e) {
-          LOG.error("Error while guessing the schema for AdapterDescription: {}", e.getMessage());
-          return serverError(Notifications.error(e.getMessage()));
+          return serverError(StreamPipesErrorMessage.from(e));
+      } catch (NoServiceEndpointsAvailableException | IOException e) {
+        LOG.error(e.getMessage());
+        return serverError(StreamPipesErrorMessage.from(e));
       }
   }
 }
diff --git a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/RuntimeResolvableResource.java b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/RuntimeResolvableResource.java
index 1db6196b6..5f3fe1f67 100644
--- a/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/RuntimeResolvableResource.java
+++ b/streampipes-rest/src/main/java/org/apache/streampipes/rest/impl/connect/RuntimeResolvableResource.java
@@ -19,13 +19,17 @@
 package org.apache.streampipes.rest.impl.connect;
 
 import org.apache.streampipes.commons.exceptions.NoServiceEndpointsAvailableException;
+import org.apache.streampipes.commons.exceptions.SpConfigurationException;
 import org.apache.streampipes.connect.api.exception.AdapterException;
 import org.apache.streampipes.connect.container.master.management.WorkerAdministrationManagement;
 import org.apache.streampipes.connect.container.master.management.WorkerRestClient;
 import org.apache.streampipes.connect.container.master.management.WorkerUrlProvider;
+import org.apache.streampipes.model.StreamPipesErrorMessage;
 import org.apache.streampipes.model.runtime.RuntimeOptionsRequest;
 import org.apache.streampipes.model.runtime.RuntimeOptionsResponse;
 import org.apache.streampipes.rest.shared.annotation.JacksonSerialized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.ws.rs.*;
 import javax.ws.rs.core.MediaType;
@@ -34,6 +38,8 @@ import javax.ws.rs.core.Response;
 @Path("/v2/connect/master/resolvable")
 public class RuntimeResolvableResource extends AbstractAdapterResource<WorkerAdministrationManagement> {
 
+    private static final Logger LOG = LoggerFactory.getLogger(RuntimeResolvableResource.class);
+
     private final WorkerUrlProvider workerUrlProvider;
 
     public RuntimeResolvableResource() {
@@ -56,11 +62,16 @@ public class RuntimeResolvableResource extends AbstractAdapterResource<WorkerAdm
             RuntimeOptionsResponse result = WorkerRestClient.getConfiguration(workerEndpoint, appId, runtimeOptionsRequest);
 
             return ok(result);
-        } catch (AdapterException | NoServiceEndpointsAvailableException e) {
-            e.printStackTrace();
-            return fail();
+        } catch (AdapterException e) {
+            LOG.error("Adapter exception occurred", e);
+            return serverError(StreamPipesErrorMessage.from(e));
+        } catch (NoServiceEndpointsAvailableException e) {
+            LOG.error("Could not find service endpoint for {} while fetching configuration", appId);
+            return serverError(StreamPipesErrorMessage.from(e));
+        } catch (SpConfigurationException e) {
+            LOG.error("Tried to fetch a runtime configuration with insufficient settings");
+            return badRequest(StreamPipesErrorMessage.from(e));
         }
-
     }
 
 }
diff --git a/ui/package.json b/ui/package.json
index 30e2878f0..5be6f1bb5 100644
--- a/ui/package.json
+++ b/ui/package.json
@@ -8,8 +8,8 @@
     "url": "https://github.com/apache/incubator-streampipes"
   },
   "scripts": {
-    "build-libs": "ng build @streampipes/shared-ui && ng build @streampipes/platform-services",
-    "install-libs": "npm install @streampipes/shared-ui@file:./dist/streampipes/shared-ui @streampipes/platform-services@file:./dist/streampipes/platform-services --no-save",
+    "build-libs": "ng build @streampipes/platform-services && ng build @streampipes/shared-ui",
+    "install-libs": "npm install @streampipes/platform-services@file:./dist/streampipes/platform-services @streampipes/shared-ui@file:./dist/streampipes/shared-ui --no-save",
     "build-libraries": "npm run build-libs && npm run install-libs",
     "start": "node ./deployment/prebuild.js && npm run build-libraries && ng serve",
     "test": "node ./deployment/prebuild.js && npm run build-libraries && ng test",
diff --git a/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts b/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
index c4f9a582e..24ebb69e4 100644
--- a/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
+++ b/ui/projects/streampipes/platform-services/src/lib/model/gen/streampipes-model.ts
@@ -18,7 +18,7 @@
 /* tslint:disable */
 /* eslint-disable */
 // @ts-nocheck
-// Generated using typescript-generator version 2.27.744 on 2022-08-01 10:55:40.
+// Generated using typescript-generator version 2.27.744 on 2022-08-14 22:39:25.
 
 export class AbstractStreamPipesEntity {
     "@class": "org.apache.streampipes.model.base.AbstractStreamPipesEntity" | "org.apache.streampipes.model.base.NamedStreamPipesEntity" | "org.apache.streampipes.model.connect.adapter.AdapterDescription" | "org.apache.streampipes.model.connect.adapter.AdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.GenericAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.SpecificAdapterSetDescription" | "org.apache.streampipes.model.connect.adapter.AdapterStre [...]
@@ -192,9 +192,9 @@ export class AdapterDescription extends NamedStreamPipesEntity {
         instance.selectedEndpointUrl = data.selectedEndpointUrl;
         instance.correspondingServiceGroup = data.correspondingServiceGroup;
         instance.correspondingDataStreamElementId = data.correspondingDataStreamElementId;
+        instance.streamRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.streamRules);
         instance.schemaRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.schemaRules);
         instance.valueRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.valueRules);
-        instance.streamRules = __getCopyArrayFn(TransformationRuleDescription.fromDataUnion)(data.streamRules);
         return instance;
     }
 
@@ -987,11 +987,14 @@ export class DataExplorerWidgetModel extends DashboardEntity {
 
 export class DataLakeMeasure extends UnnamedStreamPipesEntity {
     "@class": "org.apache.streampipes.model.datalake.DataLakeMeasure";
+    _rev: string;
     eventSchema: EventSchema;
     measureName: string;
     pipelineId: string;
     pipelineIsRunning: boolean;
     pipelineName: string;
+    schemaVersion: string;
+    timestampField: string;
 
     static fromData(data: DataLakeMeasure, target?: DataLakeMeasure): DataLakeMeasure {
         if (!data) {
@@ -1000,10 +1003,13 @@ export class DataLakeMeasure extends UnnamedStreamPipesEntity {
         const instance = target || new DataLakeMeasure();
         super.fromData(data, instance);
         instance.measureName = data.measureName;
+        instance.timestampField = data.timestampField;
         instance.eventSchema = EventSchema.fromData(data.eventSchema);
         instance.pipelineId = data.pipelineId;
         instance.pipelineName = data.pipelineName;
         instance.pipelineIsRunning = data.pipelineIsRunning;
+        instance.schemaVersion = data.schemaVersion;
+        instance._rev = data._rev;
         return instance;
     }
 }
@@ -2561,8 +2567,8 @@ export class PipelineTemplateDescription extends NamedStreamPipesEntity {
         const instance = target || new PipelineTemplateDescription();
         super.fromData(data, instance);
         instance.boundTo = __getCopyArrayFn(BoundPipelineElement.fromData)(data.boundTo);
-        instance.pipelineTemplateName = data.pipelineTemplateName;
         instance.pipelineTemplateId = data.pipelineTemplateId;
+        instance.pipelineTemplateName = data.pipelineTemplateName;
         instance.pipelineTemplateDescription = data.pipelineTemplateDescription;
         return instance;
     }
@@ -2964,8 +2970,8 @@ export class SpDataSet extends SpDataStream {
         instance.datasetInvocationId = data.datasetInvocationId;
         instance.correspondingPipeline = data.correspondingPipeline;
         instance.selectedEndpointUrl = data.selectedEndpointUrl;
-        instance.brokerHostname = data.brokerHostname;
         instance.actualTopicName = data.actualTopicName;
+        instance.brokerHostname = data.brokerHostname;
         return instance;
     }
 }
@@ -3105,6 +3111,27 @@ export class StreamPipesApplicationPackage {
     }
 }
 
+export class StreamPipesErrorMessage {
+    cause: string;
+    detail: string;
+    fullStackTrace: string;
+    level: string;
+    title: string;
+
+    static fromData(data: StreamPipesErrorMessage, target?: StreamPipesErrorMessage): StreamPipesErrorMessage {
+        if (!data) {
+            return data;
+        }
+        const instance = target || new StreamPipesErrorMessage();
+        instance.level = data.level;
+        instance.title = data.title;
+        instance.detail = data.detail;
+        instance.cause = data.cause;
+        instance.fullStackTrace = data.fullStackTrace;
+        return instance;
+    }
+}
+
 export class SuccessMessage extends Message {
 
     static fromData(data: SuccessMessage, target?: SuccessMessage): SuccessMessage {
diff --git a/ui/projects/streampipes/shared-ui/package.json b/ui/projects/streampipes/shared-ui/package.json
index a602da9ba..8a55c7fd6 100644
--- a/ui/projects/streampipes/shared-ui/package.json
+++ b/ui/projects/streampipes/shared-ui/package.json
@@ -9,6 +9,7 @@
     "@angular/flex-layout": "^13.0.0-beta.38",
     "@angular/material": "^13.3.0",
     "@angular/router": "^13.3.0",
+    "@streampipes/platform-services": "0.0.1",
     "rxjs": "^6.6.2"
   },
   "dependencies": {
diff --git a/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.html b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.html
new file mode 100644
index 000000000..3ed79a1df
--- /dev/null
+++ b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.html
@@ -0,0 +1,43 @@
+<!--
+  ~ 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.
+  ~
+  -->
+
+<div class="sp-dialog-container">
+    <div class="sp-dialog-content p-20">
+        <div fxFlex="100" fxLayout="column" class="mt-10">
+            <div class="error-details-title">Probable cause</div>
+            <div class="log-message" [innerText]="message.cause">
+            </div>
+            <div class="mt-10">
+            <button mat-button color="accent" (click)="showDetails = !showDetails">Full details</button>
+            </div>
+            <div fxFlex="100" fxLayout="column" *ngIf="showDetails" class="mt-10">
+                <div class="error-details-title">Full stack trace</div>
+                <div class="log-message">
+                    <div [innerText]="message.fullStackTrace"></div>
+                </div>
+            </div>
+        </div>
+
+    </div>
+    <mat-divider></mat-divider>
+    <div class="sp-dialog-actions actions-align-right">
+        <button mat-button mat-raised-button class="mat-basic" (click)="close()">
+            Close
+        </button>
+    </div>
+</div>
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.ts b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.scss
similarity index 63%
copy from ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.ts
copy to ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.scss
index 4e64c258c..c3e663fc4 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.ts
+++ b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.scss
@@ -16,23 +16,30 @@
  *
  */
 
-import { Component, Input, OnInit } from '@angular/core';
-import { Notification } from '@streampipes/platform-services';
-
-@Component({
-  selector: 'sp-error-message',
-  templateUrl: './error-message.component.html',
-  styleUrls: ['./error-message.component.scss']
-})
-export class ErrorMessageComponent implements OnInit {
-
-  @Input() errorMessages: Notification[];
-
-  showErrorMessage = false;
+.log-message {
+  background-color: black;
+  font: 9pt Inconsolata, monospace;
+  text-shadow: 0 0 5px #C8C8C8;
+  color: white;
+  padding: 10px;
+  max-width: 100%;
+  max-height: 300px;
+  overflow-y: scroll;
+  white-space: pre-wrap;
+}
 
-  constructor() { }
+.error-details-title {
+  font-size: 13pt;
+  font-weight: var(--color-default-text);
+  border-left: 3px solid var(--color-accent);
+  padding-left: 10px;
+  margin-bottom: 15px;
+}
 
-  ngOnInit(): void {
-  }
+.mt-10 {
+  margin-top: 10px;
+}
 
+.p-20 {
+  padding: 20px;
 }
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.ts b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.ts
similarity index 56%
copy from ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.ts
copy to ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.ts
index 4e64c258c..28eef5829 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.ts
+++ b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component.ts
@@ -16,23 +16,28 @@
  *
  */
 
-import { Component, Input, OnInit } from '@angular/core';
-import { Notification } from '@streampipes/platform-services';
+import { Component, Input } from '@angular/core';
+import { StreamPipesErrorMessage } from '@streampipes/platform-services';
+import { DialogRef } from '../../../dialog/base-dialog/dialog-ref';
 
 @Component({
-  selector: 'sp-error-message',
-  templateUrl: './error-message.component.html',
-  styleUrls: ['./error-message.component.scss']
+  selector: 'sp-exception-details-dialog',
+  templateUrl: './exception-details-dialog.component.html',
+  styleUrls: ['./exception-details-dialog.component.scss', '../../../../../../../../src/scss/sp/sp-dialog.scss']
 })
-export class ErrorMessageComponent implements OnInit {
+export class SpExceptionDetailsDialogComponent {
 
-  @Input() errorMessages: Notification[];
+  @Input()
+  message: StreamPipesErrorMessage;
 
-  showErrorMessage = false;
+  showDetails = false;
 
-  constructor() { }
+  constructor(private dialogRef: DialogRef<SpExceptionDetailsDialogComponent>) {
 
-  ngOnInit(): void {
+  }
+
+  close() {
+    this.dialogRef.close();
   }
 
 }
diff --git a/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/sp-exception-message.component.html b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/sp-exception-message.component.html
new file mode 100644
index 000000000..46238bfc4
--- /dev/null
+++ b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/sp-exception-message.component.html
@@ -0,0 +1,38 @@
+<!--
+  ~ 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.
+  ~
+  -->
+
+<div fxLayout="column" fxFlex="100" class="error-panel error">
+    <div fxLayout="row">
+        <div fxFlex="100" fxLayoutAlign="start center">
+            <div fxLayout="row" fxLayoutAlign="start center" class="p-5" fxFlex="100">
+                <div fxLayoutAlign="start center">
+                <i class="material-icons color-warn" style="margin-right: 15px;">warning</i>
+                <h5 fxFlex class="color-warn">{{message.title}}</h5>
+                </div>
+                <span fxFlex></span>
+                <div fxLayoutAlign="end center" *ngIf="showDetails">
+                    <button mat-button (click)="openDetailsDialog()">
+                        <i class="material-icons">visibility</i>&nbsp;Details
+                    </button>
+                </div>
+            </div>
+
+        </div>
+    </div>
+
+</div>
diff --git a/streampipes-container/src/main/java/org/apache/streampipes/container/api/SupportsRuntimeConfig.java b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/sp-exception-message.component.scss
similarity index 68%
copy from streampipes-container/src/main/java/org/apache/streampipes/container/api/SupportsRuntimeConfig.java
copy to ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/sp-exception-message.component.scss
index 10cacd125..15a00db00 100644
--- a/streampipes-container/src/main/java/org/apache/streampipes/container/api/SupportsRuntimeConfig.java
+++ b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/sp-exception-message.component.scss
@@ -16,14 +16,15 @@
  *
  */
 
-package org.apache.streampipes.container.api;
-
-import org.apache.streampipes.model.staticproperty.StaticProperty;
-import org.apache.streampipes.sdk.extractor.StaticPropertyExtractor;
-
-public interface SupportsRuntimeConfig {
+.error-panel {
+  background: var(--color-bg-1);
+  border-radius: 5px;
+}
 
-  StaticProperty resolveConfiguration(String staticPropertyInternalName,
-                                      StaticPropertyExtractor extractor);
+.error {
+  border: 1px solid var(--color-warn);
+}
 
+.color-warn {
+  color: var(--color-warn);
 }
diff --git a/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/sp-exception-message.component.ts b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/sp-exception-message.component.ts
new file mode 100644
index 000000000..2d78e1b02
--- /dev/null
+++ b/ui/projects/streampipes/shared-ui/src/lib/components/sp-exception-message/sp-exception-message.component.ts
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ *
+ */
+
+import { Component, Input } from '@angular/core';
+import { StreamPipesErrorMessage } from '@streampipes/platform-services';
+import { DialogService } from '../../dialog/base-dialog/base-dialog.service';
+import { PanelType } from '../../dialog/base-dialog/base-dialog.model';
+import { SpExceptionDetailsDialogComponent } from './exception-details-dialog/exception-details-dialog.component';
+
+@Component({
+  selector: 'sp-exception-message',
+  templateUrl: './sp-exception-message.component.html',
+  styleUrls: ['./sp-exception-message.component.scss']
+})
+export class SpExceptionMessageComponent {
+
+  @Input()
+  level = 'error';
+
+  @Input()
+  showDetails = true;
+
+  @Input()
+  message: StreamPipesErrorMessage;
+
+  constructor(private dialogService: DialogService) {
+
+  }
+
+  openDetailsDialog() {
+    this.dialogService.open(SpExceptionDetailsDialogComponent, {
+      panelType: PanelType.STANDARD_PANEL,
+      width: '80vw',
+      title: 'Error Details',
+      data: {
+        'message': this.message
+      }
+    });
+  }
+
+}
diff --git a/ui/projects/streampipes/shared-ui/src/lib/shared-ui.module.ts b/ui/projects/streampipes/shared-ui/src/lib/shared-ui.module.ts
index d69664dd2..8e40ee51f 100644
--- a/ui/projects/streampipes/shared-ui/src/lib/shared-ui.module.ts
+++ b/ui/projects/streampipes/shared-ui/src/lib/shared-ui.module.ts
@@ -32,6 +32,9 @@ import { SpBasicNavTabsComponent } from './components/basic-nav-tabs/basic-nav-t
 import { MatTabsModule } from '@angular/material/tabs';
 import { SpBasicInnerPanelComponent } from './components/basic-inner-panel/basic-inner-panel.component';
 import { SpBasicHeaderTitleComponent } from './components/basic-header-title/header-title.component';
+import { SpExceptionMessageComponent } from './components/sp-exception-message/sp-exception-message.component';
+import { SpExceptionDetailsDialogComponent } from './components/sp-exception-message/exception-details-dialog/exception-details-dialog.component';
+import { MatDividerModule } from '@angular/material/divider';
 
 @NgModule({
   declarations: [
@@ -41,12 +44,15 @@ import { SpBasicHeaderTitleComponent } from './components/basic-header-title/hea
     SpBasicInnerPanelComponent,
     SpBasicHeaderTitleComponent,
     SpBasicViewComponent,
-    SpBasicNavTabsComponent
+    SpBasicNavTabsComponent,
+    SpExceptionMessageComponent,
+    SpExceptionDetailsDialogComponent
   ],
   imports: [
     CommonModule,
     FlexLayoutModule,
     MatButtonModule,
+    MatDividerModule,
     MatIconModule,
     MatTabsModule,
     MatTooltipModule,
@@ -61,6 +67,8 @@ import { SpBasicHeaderTitleComponent } from './components/basic-header-title/hea
     SpBasicHeaderTitleComponent,
     SpBasicViewComponent,
     SpBasicNavTabsComponent,
+    SpExceptionMessageComponent,
+    SpExceptionDetailsDialogComponent
   ]
 })
 export class SharedUiModule {
diff --git a/ui/projects/streampipes/shared-ui/src/public-api.ts b/ui/projects/streampipes/shared-ui/src/public-api.ts
index 41e289808..a19dfd5af 100644
--- a/ui/projects/streampipes/shared-ui/src/public-api.ts
+++ b/ui/projects/streampipes/shared-ui/src/public-api.ts
@@ -30,6 +30,8 @@ export * from './lib/components/basic-header-title/header-title.component';
 export * from './lib/components/basic-inner-panel/basic-inner-panel.component';
 export * from './lib/components/basic-view/basic-view.component';
 export * from './lib/components/basic-nav-tabs/basic-nav-tabs.component';
+export * from './lib/components/sp-exception-message/sp-exception-message.component';
+export * from './lib/components/sp-exception-message/exception-details-dialog/exception-details-dialog.component';
 
 export * from './lib/models/sp-navigation.model';
 
diff --git a/ui/src/app/connect/components/new-adapter/new-adapter.component.ts b/ui/src/app/connect/components/new-adapter/new-adapter.component.ts
index 1cdee14b4..2a0566e89 100644
--- a/ui/src/app/connect/components/new-adapter/new-adapter.component.ts
+++ b/ui/src/app/connect/components/new-adapter/new-adapter.component.ts
@@ -176,7 +176,6 @@ export class NewAdapterComponent implements OnInit, AfterViewInit {
 
   clickSpecificSettingsNextButton() {
     this.shepherdService.trigger('specific-settings-next-button');
-    console.log(this.adapter);
     this.guessEventSchema();
     this.goForward();
   }
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.html b/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.html
index 8172ea552..ecfdf152a 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.html
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.html
@@ -17,22 +17,10 @@
   -->
 
 <div fxLayout="column" fxFlex="100">
-    <div fxLayout="row" fxLayoutAlign="center center" class="error-color" fxFlex="100">
-        <mat-icon fxLayoutAlign="start center">warning</mat-icon>
-        <div fxLayoutAlign="start center" class="error-text">&nbsp;Sorry, there was an error while guessing the schema of your configured data source...</div>
+    <div fxLayout="row" fxLayoutAlign="center center" fxFlex="100">
+        <div fxLayoutAlign="start center" class="error-text">&nbsp;There was an error while guessing the schema of your configured data source:</div>
     </div>
     <div fxLayout="row" fxLayoutAlign="center center" class="mt-10">
-        <button mat-button color="accent">
-            <div *ngIf="!showErrorMessage" (click)="showErrorMessage = true">Show Details</div>
-            <div *ngIf="showErrorMessage" (click)="showErrorMessage = false">Hide Details</div>
-        </button>
-    </div>
-    <div fxLayoutAlign="center center" *ngIf="showErrorMessage">
-        <div class="error-message">
-            <div *ngFor="let error of errorMessages" style="margin-bottom: 5px; margin-top: 5px">
-                <div>{{error.title}}</div>
-                <div>{{error.description}}</div>
-            </div>
-        </div>
+        <sp-exception-message [message]="errorMessage"></sp-exception-message>
     </div>
 </div>
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.ts b/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.ts
index 4e64c258c..b6dc01d70 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.ts
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/error-message/error-message.component.ts
@@ -18,6 +18,7 @@
 
 import { Component, Input, OnInit } from '@angular/core';
 import { Notification } from '@streampipes/platform-services';
+import { StreamPipesErrorMessage } from '../../../../../../../projects/streampipes/platform-services/src/lib/model/gen/streampipes-model';
 
 @Component({
   selector: 'sp-error-message',
@@ -26,7 +27,7 @@ import { Notification } from '@streampipes/platform-services';
 })
 export class ErrorMessageComponent implements OnInit {
 
-  @Input() errorMessages: Notification[];
+  @Input() errorMessage: StreamPipesErrorMessage;
 
   showErrorMessage = false;
 
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
index 163fbb730..6e85d424f 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.html
@@ -72,7 +72,7 @@
                 <sp-loading-message *ngIf="isLoading"></sp-loading-message>
 
                 <sp-error-message
-                        [errorMessages]="errorMessages"
+                        [errorMessage]="errorMessage"
                         *ngIf="isError">
                 </sp-error-message>
 
diff --git a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
index 6ad0b7299..5ca2b4e58 100644
--- a/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
+++ b/ui/src/app/connect/components/new-adapter/schema-editor/event-schema/event-schema.component.ts
@@ -32,6 +32,7 @@ import {
 } from '@streampipes/platform-services';
 import { MatStepper } from '@angular/material/stepper';
 import { UserErrorMessage } from '../../../../../core-model/base/UserErrorMessage';
+import { StreamPipesErrorMessage } from '../../../../../../../projects/streampipes/platform-services/src/lib/model/gen/streampipes-model';
 
 @Component({
   selector: 'sp-event-schema',
@@ -73,7 +74,7 @@ export class EventSchemaComponent implements OnChanges {
   isLoading = false;
   isError = false;
   isPreviewEnabled = false;
-  errorMessages: Notification[];
+  errorMessage: StreamPipesErrorMessage;
   nodes: EventProperty[] = new Array<EventProperty>();
   validEventSchema = false;
   schemaErrorHints: UserErrorMessage[] = [];
@@ -117,7 +118,7 @@ export class EventSchemaComponent implements OnChanges {
         this.isLoading = false;
       },
       errorMessage => {
-        this.errorMessages = errorMessage.error.notifications;
+        this.errorMessage = errorMessage.error;
         this.isError = true;
         this.isLoading = false;
         this.eventSchema = new EventSchema();
diff --git a/ui/src/app/connect/connect.module.ts b/ui/src/app/connect/connect.module.ts
index e7e324795..1b23c969c 100644
--- a/ui/src/app/connect/connect.module.ts
+++ b/ui/src/app/connect/connect.module.ts
@@ -123,7 +123,8 @@ import { SpAdapterTemplateDialogComponent } from './dialog/adapter-template/adap
     SharedUiModule
   ],
   exports: [
-    PipelineElementRuntimeInfoComponent
+    PipelineElementRuntimeInfoComponent,
+    ErrorMessageComponent
   ],
   declarations: [
     AdapterDescriptionComponent,
diff --git a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-any-input/static-runtime-resolvable-any-input.component.ts b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-any-input/static-runtime-resolvable-any-input.component.ts
index b24bc7afa..c1589be9e 100644
--- a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-any-input/static-runtime-resolvable-any-input.component.ts
+++ b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-any-input/static-runtime-resolvable-any-input.component.ts
@@ -53,4 +53,7 @@ export class StaticRuntimeResolvableAnyInputComponent
         return staticProperty as RuntimeResolvableAnyStaticProperty;
     }
 
+    afterErrorReceived() {
+    }
+
 }
diff --git a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-input/base-runtime-resolvable-input.ts b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-input/base-runtime-resolvable-input.ts
index 229beba9b..b2f33e393 100644
--- a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-input/base-runtime-resolvable-input.ts
+++ b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-input/base-runtime-resolvable-input.ts
@@ -30,6 +30,7 @@ import { RuntimeResolvableService } from './runtime-resolvable.service';
 import { Observable } from 'rxjs';
 import { Directive, Input, OnChanges, SimpleChanges } from '@angular/core';
 import { ConfigurationInfo } from '../../../connect/model/ConfigurationInfo';
+import { StreamPipesErrorMessage } from '../../../../../projects/streampipes/platform-services/src/lib/model/gen/streampipes-model';
 
 @Directive()
 // tslint:disable-next-line:directive-class-suffix
@@ -43,6 +44,8 @@ export abstract class BaseRuntimeResolvableInput<T
 
   showOptions = false;
   loading = false;
+  error = false;
+  errorMessage: StreamPipesErrorMessage;
   dependentStaticProperties: any = new Map();
 
   constructor(private runtimeResolvableService: RuntimeResolvableService) {
@@ -70,6 +73,8 @@ export abstract class BaseRuntimeResolvableInput<T
 
     this.showOptions = false;
     this.loading = true;
+    this.error = false;
+    this.errorMessage = undefined;
     const observable: Observable<RuntimeOptionsResponse> = this.adapterId ?
       this.runtimeResolvableService.fetchRemoteOptionsForAdapter(resolvableOptionsParameterRequest, this.adapterId) :
       this.runtimeResolvableService.fetchRemoteOptionsForPipelineElement(resolvableOptionsParameterRequest);
@@ -80,6 +85,12 @@ export abstract class BaseRuntimeResolvableInput<T
       }
       this.loading = false;
       this.showOptions = true;
+    }, errorMessage => {
+      this.loading = false;
+      this.showOptions = true;
+      this.error = true;
+      this.errorMessage = errorMessage.error as StreamPipesErrorMessage;
+      this.afterErrorReceived();
     });
   }
 
@@ -107,4 +118,6 @@ export abstract class BaseRuntimeResolvableInput<T
 
   abstract afterOptionsLoaded(staticProperty: T);
 
+  abstract afterErrorReceived();
+
 }
diff --git a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.ts b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.ts
index a6c12b1d6..cae01d7c1 100644
--- a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.ts
+++ b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-oneof-input/static-runtime-resolvable-oneof-input.component.ts
@@ -54,4 +54,7 @@ export class StaticRuntimeResolvableOneOfInputComponent
     parse(staticProperty: StaticPropertyUnion): RuntimeResolvableOneOfStaticProperty {
         return staticProperty as RuntimeResolvableOneOfStaticProperty;
     }
+
+    afterErrorReceived() {
+    }
 }
diff --git a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-tree-input/static-tree-input.component.html b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-tree-input/static-tree-input.component.html
index c9216e29c..49e036de3 100644
--- a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-tree-input/static-tree-input.component.html
+++ b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-tree-input/static-tree-input.component.html
@@ -29,6 +29,9 @@
                      [mode]="'indeterminate'"
                      [diameter]="20"></mat-spinner>
     </div>
+    <div fxLayout="column" *ngIf="error" class="mt-10">
+        <sp-exception-message [message]="errorMessage"></sp-exception-message>
+    </div>
     <mat-tree [dataSource]="dataSource" [treeControl]="treeControl" class="sp-tree">
         <mat-tree-node *matTreeNodeDef="let node" matTreeNodeToggle>
             <mat-checkbox color="accent"
diff --git a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-tree-input/static-tree-input.component.ts b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-tree-input/static-tree-input.component.ts
index 3be93789f..cb2fa1883 100644
--- a/ui/src/app/core-ui/static-properties/static-runtime-resolvable-tree-input/static-tree-input.component.ts
+++ b/ui/src/app/core-ui/static-properties/static-runtime-resolvable-tree-input/static-tree-input.component.ts
@@ -26,6 +26,7 @@ import {
 import { RuntimeResolvableService } from '../static-runtime-resolvable-input/runtime-resolvable.service';
 import { NestedTreeControl } from '@angular/cdk/tree';
 import { MatTreeNestedDataSource } from '@angular/material/tree';
+import { AbstractControl, FormControl, ValidationErrors, ValidatorFn } from '@angular/forms';
 
 @Component({
   selector: 'sp-runtime-resolvable-tree-input',
@@ -55,6 +56,7 @@ export class StaticRuntimeResolvableTreeInputComponent
       this.showOptions = true;
     }
     super.onInit();
+    this.parentForm.addControl(this.staticProperty.internalName, new FormControl(this.staticProperty.nodes, []));
   }
 
   parse(staticProperty: StaticPropertyUnion): RuntimeResolvableTreeInputStaticProperty {
@@ -64,10 +66,12 @@ export class StaticRuntimeResolvableTreeInputComponent
   afterOptionsLoaded(staticProperty: RuntimeResolvableTreeInputStaticProperty) {
     this.staticProperty.nodes = staticProperty.nodes;
     this.dataSource.data = this.staticProperty.nodes;
+    this.performValidation();
   }
 
   toggleNodeSelection(node: TreeInputNode) {
     node.selected = !node.selected;
+    this.performValidation();
   }
 
   toggleAllNodeSelection(node: any) {
@@ -75,6 +79,7 @@ export class StaticRuntimeResolvableTreeInputComponent
     const newState = !node.selected;
     node.selected = newState;
     descendants.forEach(d => d.selected = newState);
+    this.performValidation();
   }
 
   descendantsAllSelected(node: TreeInputNode) {
@@ -93,4 +98,30 @@ export class StaticRuntimeResolvableTreeInputComponent
     return result && !this.descendantsAllSelected(node);
   }
 
+  performValidation() {
+    let error = {error: true};
+    if (this.anyNodeSelected()) {
+      error = undefined;
+    }
+    this.parentForm.controls[this.staticProperty.internalName].setErrors(error);
+  }
+
+  anyNodeSelected(): boolean {
+    return this.dataSource.data.find(d => this.anySelected(d)) !== undefined;
+  }
+
+  anySelected(node: TreeInputNode): boolean {
+    if (node.selected) {
+      return true;
+    } else {
+      return node.children.find(n => this.anySelected(n)) !== undefined;
+    }
+  }
+
+  afterErrorReceived() {
+    this.staticProperty.nodes = [];
+    this.dataSource.data = [];
+    this.performValidation();
+  }
+
 }