diff --git a/streampipes-extensions-management/pom.xml b/streampipes-extensions-management/pom.xml
index 3c0655c595..9c0427023b 100644
--- a/streampipes-extensions-management/pom.xml
+++ b/streampipes-extensions-management/pom.xml
@@ -94,6 +94,11 @@
com.opencsv
opencsv
+
+ org.apache.avro
+ avro
+ 1.11.4
+
diff --git a/streampipes-extensions-management/src/main/java/org/apache/streampipes/extensions/management/connect/adapter/BrokerEventProcessor.java b/streampipes-extensions-management/src/main/java/org/apache/streampipes/extensions/management/connect/adapter/BrokerEventProcessor.java
index 2ad113e8a7..b3ed8c6fd1 100644
--- a/streampipes-extensions-management/src/main/java/org/apache/streampipes/extensions/management/connect/adapter/BrokerEventProcessor.java
+++ b/streampipes-extensions-management/src/main/java/org/apache/streampipes/extensions/management/connect/adapter/BrokerEventProcessor.java
@@ -23,11 +23,10 @@
import org.apache.streampipes.extensions.api.connect.IParser;
import org.apache.streampipes.messaging.InternalEventProcessor;
-import org.apache.commons.io.IOUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.nio.charset.StandardCharsets;
+import java.io.ByteArrayInputStream;
public record BrokerEventProcessor(
IParser parser,
@@ -39,7 +38,7 @@ public record BrokerEventProcessor(
@Override
public void onEvent(byte[] payload) {
try {
- parser.parse(IOUtils.toInputStream(new String(payload), StandardCharsets.UTF_8), collector::collect);
+ parser.parse(new ByteArrayInputStream(payload), collector::collect);
} catch (ParseException e) {
LOG.error("Error while parsing: " + e.getMessage());
}
diff --git a/streampipes-extensions-management/src/main/java/org/apache/streampipes/extensions/management/connect/adapter/parser/AvroParser.java b/streampipes-extensions-management/src/main/java/org/apache/streampipes/extensions/management/connect/adapter/parser/AvroParser.java
new file mode 100644
index 0000000000..32bf26f489
--- /dev/null
+++ b/streampipes-extensions-management/src/main/java/org/apache/streampipes/extensions/management/connect/adapter/parser/AvroParser.java
@@ -0,0 +1,209 @@
+/*
+ * 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.extensions.management.connect.adapter.parser;
+
+import org.apache.streampipes.commons.exceptions.connect.ParseException;
+import org.apache.streampipes.extensions.api.connect.IParser;
+import org.apache.streampipes.extensions.api.connect.IParserEventHandler;
+import org.apache.streampipes.model.connect.grounding.ParserDescription;
+import org.apache.streampipes.model.connect.guess.GuessSchema;
+import org.apache.streampipes.model.staticproperty.StaticProperty;
+import org.apache.streampipes.sdk.builder.adapter.ParserDescriptionBuilder;
+import org.apache.streampipes.sdk.extractor.StaticPropertyExtractor;
+import org.apache.streampipes.sdk.helpers.Labels;
+import org.apache.streampipes.sdk.helpers.Options;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.util.Utf8;
+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.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+public class AvroParser implements IParser {
+
+ private static final Logger LOG = LoggerFactory.getLogger(AvroParser.class);
+
+ public static final String ID = "org.apache.streampipes.extensions.management.connect.adapter.parser.avro";
+ public static final String LABEL = "Avro";
+ public static final String DESCRIPTION = "Can be used to read avro records";
+
+ public static final String SCHEMA = "schema";
+ public static final String SCHEMA_REGISTRY = "schemaRegistry";
+ public static final String FLATTEN_RECORDS = "flattenRecord";
+
+ private final ParserUtils parserUtils;
+ private DatumReader datumReader;
+ private boolean schemaRegistry;
+ private boolean flattenRecord;
+
+
+ public AvroParser() {
+ parserUtils = new ParserUtils();
+ }
+
+ public AvroParser(String schemaString, boolean schemaRegistry, boolean flattenRecord) {
+ this();
+ Schema schema = new Schema.Parser().parse(schemaString);
+ this.datumReader = new GenericDatumReader<>(schema);
+ this.schemaRegistry = schemaRegistry;
+ this.flattenRecord = flattenRecord;
+ }
+
+ @Override
+ public ParserDescription declareDescription() {
+ return ParserDescriptionBuilder.create(ID, LABEL, DESCRIPTION)
+ .requiredSingleValueSelection(
+ Labels.from(SCHEMA_REGISTRY, "Schema Registry",
+ "Does the messages include the schema registry header?"),
+ Options.from("yes", "no")
+ )
+ .requiredSingleValueSelection(
+ Labels.from(FLATTEN_RECORDS, "Flatten Records",
+ "Should nested records be flattened?"),
+ Options.from("no", "yes")
+ )
+ .requiredCodeblock(Labels.from(SCHEMA, "Schema",
+ "The schema of the avro record"),
+ "{\n"
+ + " \"namespace\": \"example.avro\",\n"
+ + " \"type\": \"record\",\n"
+ + " \"name\": \"Test\",\n"
+ + " \"fields\": [\n"
+ + " {\"name\": \"id\", \"type\": \"string\"},\n"
+ + " {\"name\": \"value\", \"type\": \"double\"}\n"
+ + " ]\n"
+ + "}")
+ .build();
+ }
+
+
+ @Override
+ public GuessSchema getGuessSchema(InputStream inputStream) throws ParseException {
+ GenericRecord avroRecord = getRecord(inputStream);
+ var event = toMap(avroRecord);
+ return parserUtils.getGuessSchema(event);
+ }
+
+ @Override
+ public void parse(InputStream inputStream, IParserEventHandler handler) throws ParseException {
+ GenericRecord avroRecord = getRecord(inputStream);
+ var event = toMap(avroRecord);
+ handler.handle(event);
+ }
+
+ @Override
+ public IParser fromDescription(List configuration) {
+ var extractor = StaticPropertyExtractor.from(configuration);
+ String schema = extractor.codeblockValue(SCHEMA);
+ boolean schemaRegistry = extractor
+ .selectedSingleValue(SCHEMA_REGISTRY, String.class)
+ .equals("yes");
+ boolean flattenRecords = extractor
+ .selectedSingleValue(FLATTEN_RECORDS, String.class)
+ .equals("yes");
+
+ return new AvroParser(schema, schemaRegistry, flattenRecords);
+ }
+
+ private GenericRecord getRecord(InputStream inputStream) throws ParseException {
+ try {
+ if (schemaRegistry) {
+ inputStream.skipNBytes(5);
+ }
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ return datumReader.read(null, decoder);
+ } catch (IOException e) {
+ throw new ParseException(
+ "Error decoding the avro message. Please check the schema."
+ );
+ }
+ }
+
+
+ private Map toMap(GenericRecord avroRecord) {
+ Map resultMap = new LinkedHashMap<>();
+ avroRecord.getSchema().getFields().forEach(field -> {
+ String fieldName = field.name();
+ Object fieldValue = avroRecord.get(fieldName);
+ if (flattenRecord && fieldValue instanceof GenericRecord){
+ Map flatMap = unwrapNestedRecord((GenericRecord) fieldValue, fieldName);
+ resultMap.putAll(flatMap);
+ } else {
+ resultMap.put(fieldName, toMapHelper(fieldValue));
+ }
+ });
+
+ return resultMap;
+ }
+
+ private Object toMapHelper(Object fieldValue) {
+ if (fieldValue instanceof GenericRecord){
+ return toMap((GenericRecord) fieldValue);
+ }
+ if (fieldValue instanceof GenericData.Array>){
+ List
+
+ org.apache.avro
+ avro
+ 1.11.4
+
org.apache.streampipes
streampipes-connectors-mqtt
diff --git a/streampipes-integration-tests/src/test/java/org/apache/streampipes/integration/adapters/KafkaAdapterTester.java b/streampipes-integration-tests/src/test/java/org/apache/streampipes/integration/adapters/KafkaAdapterTester.java
index 75883a244a..b5e242eb50 100644
--- a/streampipes-integration-tests/src/test/java/org/apache/streampipes/integration/adapters/KafkaAdapterTester.java
+++ b/streampipes-integration-tests/src/test/java/org/apache/streampipes/integration/adapters/KafkaAdapterTester.java
@@ -66,7 +66,7 @@ public IAdapterConfiguration prepareAdapter() throws AdapterException {
list.add(new Option(TOPIC));
((RuntimeResolvableOneOfStaticProperty) configuration.getAdapterDescription()
.getConfig()
- .get(4))
+ .get(5))
.setOptions(list);
List