Browse Source

Msg sending improvements (#640)

* init

* Messages sending fixes & tests

* Messages sending fixes & tests

* Messages sending json test added

* PR comments fixes

* Setting getTopicMessages & sendTopicMessage key value type to string.

* admin-client-timeout default removed

* sonar fix

* messages filtering changes to String.contains

Co-authored-by: Ilya Kuramshin <ikuramshin@provectus.com>
Ilya Kuramshin 4 years ago
parent
commit
dd19cc2eec
29 changed files with 599 additions and 316 deletions
  1. 6 2
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/DeserializationService.java
  2. 32 25
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/ProtobufFileRecordSerDe.java
  3. 13 5
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/RecordSerDe.java
  4. 15 11
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/SimpleRecordSerDe.java
  5. 5 20
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/AvroMessageFormatter.java
  6. 8 12
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/AvroMessageReader.java
  7. 0 29
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/JsonMessageFormatter.java
  8. 0 39
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/JsonMessageReader.java
  9. 1 1
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/MessageFormatter.java
  10. 2 2
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/MessageReader.java
  11. 4 8
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/ProtobufMessageFormatter.java
  12. 8 4
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/ProtobufMessageReader.java
  13. 91 81
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/SchemaRegistryAwareRecordSerDe.java
  14. 2 9
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/StringMessageFormatter.java
  15. 8 0
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/service/ClusterService.java
  16. 5 3
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/service/ClustersMetricsScheduler.java
  17. 2 27
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/service/ConsumingService.java
  18. 8 4
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/service/KafkaService.java
  19. 3 3
      kafka-ui-api/src/main/java/com/provectus/kafka/ui/util/ClusterUtil.java
  20. 2 2
      kafka-ui-api/src/test/java/com/provectus/kafka/ui/AbstractBaseTest.java
  21. 8 1
      kafka-ui-api/src/test/java/com/provectus/kafka/ui/container/SchemaRegistryContainer.java
  22. 7 9
      kafka-ui-api/src/test/java/com/provectus/kafka/ui/serde/SchemaRegistryRecordDeserializerTest.java
  23. 360 0
      kafka-ui-api/src/test/java/com/provectus/kafka/ui/service/SendAndReadTests.java
  24. 2 4
      kafka-ui-contract/src/main/resources/swagger/kafka-ui-api.yaml
  25. 1 1
      kafka-ui-react-app/src/components/Topics/Topic/Details/Messages/MessageItem.tsx
  26. 2 2
      kafka-ui-react-app/src/components/Topics/Topic/Details/Messages/__test__/Messages.spec.tsx
  27. 1 4
      kafka-ui-react-app/src/components/Topics/Topic/Details/Messages/__test__/__snapshots__/MessageItem.spec.tsx.snap
  28. 1 6
      kafka-ui-react-app/src/components/Topics/Topic/Details/Messages/__test__/__snapshots__/MessagesTable.spec.tsx.snap
  29. 2 2
      kafka-ui-react-app/src/components/Topics/Topic/Details/Messages/__test__/fixtures.ts

+ 6 - 2
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/DeserializationService.java

@@ -2,14 +2,16 @@ package com.provectus.kafka.ui.serde;
 
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.provectus.kafka.ui.model.KafkaCluster;
 import com.provectus.kafka.ui.model.KafkaCluster;
-import com.provectus.kafka.ui.serde.schemaregistry.SchemaRegistryRecordSerDe;
+import com.provectus.kafka.ui.serde.schemaregistry.SchemaRegistryAwareRecordSerDe;
 import com.provectus.kafka.ui.service.ClustersStorage;
 import com.provectus.kafka.ui.service.ClustersStorage;
 import java.util.Map;
 import java.util.Map;
 import java.util.stream.Collectors;
 import java.util.stream.Collectors;
 import javax.annotation.PostConstruct;
 import javax.annotation.PostConstruct;
 import lombok.RequiredArgsConstructor;
 import lombok.RequiredArgsConstructor;
+import lombok.extern.log4j.Log4j2;
 import org.springframework.stereotype.Component;
 import org.springframework.stereotype.Component;
 
 
+@Log4j2
 @Component
 @Component
 @RequiredArgsConstructor
 @RequiredArgsConstructor
 public class DeserializationService {
 public class DeserializationService {
@@ -31,10 +33,12 @@ public class DeserializationService {
   private RecordSerDe createRecordDeserializerForCluster(KafkaCluster cluster) {
   private RecordSerDe createRecordDeserializerForCluster(KafkaCluster cluster) {
     try {
     try {
       if (cluster.getProtobufFile() != null) {
       if (cluster.getProtobufFile() != null) {
+        log.info("Using ProtobufFileRecordSerDe for cluster '{}'", cluster.getName());
         return new ProtobufFileRecordSerDe(cluster.getProtobufFile(),
         return new ProtobufFileRecordSerDe(cluster.getProtobufFile(),
             cluster.getProtobufMessageName(), objectMapper);
             cluster.getProtobufMessageName(), objectMapper);
       } else {
       } else {
-        return new SchemaRegistryRecordSerDe(cluster, objectMapper);
+        log.info("Using SchemaRegistryAwareRecordSerDe for cluster '{}'", cluster.getName());
+        return new SchemaRegistryAwareRecordSerDe(cluster);
       }
       }
     } catch (Throwable e) {
     } catch (Throwable e) {
       throw new RuntimeException("Can't init deserializer", e);
       throw new RuntimeException("Can't init deserializer", e);

+ 32 - 25
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/ProtobufFileRecordSerDe.java

@@ -1,6 +1,5 @@
 package com.provectus.kafka.ui.serde;
 package com.provectus.kafka.ui.serde;
 
 
-import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.protobuf.DynamicMessage;
 import com.google.protobuf.DynamicMessage;
 import com.google.protobuf.util.JsonFormat;
 import com.google.protobuf.util.JsonFormat;
@@ -14,16 +13,17 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Path;
-import java.util.Map;
+import java.util.Objects;
 import java.util.Optional;
 import java.util.Optional;
 import java.util.stream.Collectors;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import lombok.SneakyThrows;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Bytes;
-import reactor.util.function.Tuple2;
-import reactor.util.function.Tuples;
 
 
+//TODO: currently we assume that keys for this serde are always string - need to discuss if it is ok
 public class ProtobufFileRecordSerDe implements RecordSerDe {
 public class ProtobufFileRecordSerDe implements RecordSerDe {
   private final ProtobufSchema protobufSchema;
   private final ProtobufSchema protobufSchema;
   private final ObjectMapper objectMapper;
   private final ObjectMapper objectMapper;
@@ -42,33 +42,45 @@ public class ProtobufFileRecordSerDe implements RecordSerDe {
   }
   }
 
 
   @Override
   @Override
-  public Tuple2<String, Object> deserialize(ConsumerRecord<Bytes, Bytes> msg) {
+  public DeserializedKeyValue deserialize(ConsumerRecord<Bytes, Bytes> msg) {
     try {
     try {
-      final var message = DynamicMessage.parseFrom(
-          protobufSchema.toDescriptor(),
-          new ByteArrayInputStream(msg.value().get())
-      );
-      byte[] bytes = ProtobufSchemaUtils.toJson(message);
-      return Tuples.of(
-          msg.key() != null ? new String(msg.key().get()) : "",
-          parseJson(bytes)
+      return new DeserializedKeyValue(
+          msg.key() != null ? new String(msg.key().get()) : null,
+          msg.value() != null ? parse(msg.value().get()) : null
       );
       );
     } catch (Throwable e) {
     } catch (Throwable e) {
       throw new RuntimeException("Failed to parse record from topic " + msg.topic(), e);
       throw new RuntimeException("Failed to parse record from topic " + msg.topic(), e);
     }
     }
   }
   }
 
 
+  @SneakyThrows
+  private String parse(byte[] value) {
+    DynamicMessage protoMsg = DynamicMessage.parseFrom(
+        protobufSchema.toDescriptor(),
+        new ByteArrayInputStream(value)
+    );
+    byte[] jsonFromProto = ProtobufSchemaUtils.toJson(protoMsg);
+    return new String(jsonFromProto);
+  }
+
   @Override
   @Override
-  public ProducerRecord<byte[], byte[]> serialize(String topic, byte[] key, byte[] data,
-                                                  Optional<Integer> partition) {
+  public ProducerRecord<byte[], byte[]> serialize(String topic,
+                                                  @Nullable String key,
+                                                  @Nullable String data,
+                                                  @Nullable Integer partition) {
+    if (data == null) {
+      return new ProducerRecord<>(topic, partition, Objects.requireNonNull(key).getBytes(), null);
+    }
     DynamicMessage.Builder builder = protobufSchema.newMessageBuilder();
     DynamicMessage.Builder builder = protobufSchema.newMessageBuilder();
     try {
     try {
-      JsonFormat.parser().merge(new String(data), builder);
+      JsonFormat.parser().merge(data, builder);
       final DynamicMessage message = builder.build();
       final DynamicMessage message = builder.build();
-      return partition
-          .map(p -> new ProducerRecord<>(topic, p, key, message.toByteArray()))
-          .orElseGet(() -> new ProducerRecord<>(topic, key, message.toByteArray()));
-
+      return new ProducerRecord<>(
+          topic,
+          partition,
+          Optional.ofNullable(key).map(String::getBytes).orElse(null),
+          message.toByteArray()
+      );
     } catch (Throwable e) {
     } catch (Throwable e) {
       throw new RuntimeException("Failed to merge record for topic " + topic, e);
       throw new RuntimeException("Failed to merge record for topic " + topic, e);
     }
     }
@@ -95,9 +107,4 @@ public class ProtobufFileRecordSerDe implements RecordSerDe {
         .key(keySchema)
         .key(keySchema)
         .value(valueSchema);
         .value(valueSchema);
   }
   }
-
-  private Object parseJson(byte[] bytes) throws IOException {
-    return objectMapper.readValue(bytes, new TypeReference<Map<String, Object>>() {
-    });
-  }
 }
 }

+ 13 - 5
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/RecordSerDe.java

@@ -1,18 +1,26 @@
 package com.provectus.kafka.ui.serde;
 package com.provectus.kafka.ui.serde;
 
 
 import com.provectus.kafka.ui.model.TopicMessageSchema;
 import com.provectus.kafka.ui.model.TopicMessageSchema;
-import java.util.Optional;
+import javax.annotation.Nullable;
+import lombok.Value;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Bytes;
-import reactor.util.function.Tuple2;
 
 
 public interface RecordSerDe {
 public interface RecordSerDe {
 
 
-  Tuple2<String, Object> deserialize(ConsumerRecord<Bytes, Bytes> msg);
+  @Value
+  class DeserializedKeyValue {
+    @Nullable String key;
+    @Nullable String value;
+  }
 
 
-  ProducerRecord<byte[], byte[]> serialize(String topic, byte[] key, byte[] data,
-                                           Optional<Integer> partition);
+  DeserializedKeyValue deserialize(ConsumerRecord<Bytes, Bytes> msg);
+
+  ProducerRecord<byte[], byte[]> serialize(String topic,
+                                           @Nullable String key,
+                                           @Nullable String data,
+                                           @Nullable Integer partition);
 
 
   TopicMessageSchema getTopicSchema(String topic);
   TopicMessageSchema getTopicSchema(String topic);
 }
 }

+ 15 - 11
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/SimpleRecordSerDe.java

@@ -4,28 +4,32 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.provectus.kafka.ui.model.MessageSchema;
 import com.provectus.kafka.ui.model.MessageSchema;
 import com.provectus.kafka.ui.model.TopicMessageSchema;
 import com.provectus.kafka.ui.model.TopicMessageSchema;
 import com.provectus.kafka.ui.util.jsonschema.JsonSchema;
 import com.provectus.kafka.ui.util.jsonschema.JsonSchema;
-import java.util.Optional;
+import javax.annotation.Nullable;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Bytes;
-import reactor.util.function.Tuple2;
-import reactor.util.function.Tuples;
 
 
 public class SimpleRecordSerDe implements RecordSerDe {
 public class SimpleRecordSerDe implements RecordSerDe {
 
 
   @Override
   @Override
-  public Tuple2<String, Object> deserialize(ConsumerRecord<Bytes, Bytes> msg) {
-    return Tuples.of(
-        msg.key() != null ? new String(msg.key().get()) : "",
-        msg.value() != null ? new String(msg.value().get()) : ""
+  public DeserializedKeyValue deserialize(ConsumerRecord<Bytes, Bytes> msg) {
+    return new DeserializedKeyValue(
+        msg.key() != null ? new String(msg.key().get()) : null,
+        msg.value() != null ? new String(msg.value().get()) : null
     );
     );
   }
   }
 
 
   @Override
   @Override
-  public ProducerRecord<byte[], byte[]> serialize(String topic, byte[] key, byte[] data,
-                                                  Optional<Integer> partition) {
-    return partition.map(p -> new ProducerRecord<>(topic, p, key, data))
-        .orElseGet(() -> new ProducerRecord<>(topic, key, data));
+  public ProducerRecord<byte[], byte[]> serialize(String topic,
+                                                  @Nullable String key,
+                                                  @Nullable String data,
+                                                  @Nullable Integer partition) {
+    return new ProducerRecord<>(
+        topic,
+        partition,
+        key != null ? key.getBytes() : null,
+        data != null ? data.getBytes() : null
+    );
   }
   }
 
 
   @Override
   @Override

+ 5 - 20
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/AvroMessageFormatter.java

@@ -1,38 +1,23 @@
 package com.provectus.kafka.ui.serde.schemaregistry;
 package com.provectus.kafka.ui.serde.schemaregistry;
 
 
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import io.confluent.kafka.schemaregistry.avro.AvroSchemaUtils;
 import io.confluent.kafka.schemaregistry.avro.AvroSchemaUtils;
 import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
 import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
 import io.confluent.kafka.serializers.KafkaAvroDeserializer;
 import io.confluent.kafka.serializers.KafkaAvroDeserializer;
-import java.io.IOException;
-import java.util.Map;
 import lombok.SneakyThrows;
 import lombok.SneakyThrows;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.GenericRecord;
 
 
 public class AvroMessageFormatter implements MessageFormatter {
 public class AvroMessageFormatter implements MessageFormatter {
   private final KafkaAvroDeserializer avroDeserializer;
   private final KafkaAvroDeserializer avroDeserializer;
-  private final ObjectMapper objectMapper;
 
 
-  public AvroMessageFormatter(SchemaRegistryClient client, ObjectMapper objectMapper) {
+  public AvroMessageFormatter(SchemaRegistryClient client) {
     this.avroDeserializer = new KafkaAvroDeserializer(client);
     this.avroDeserializer = new KafkaAvroDeserializer(client);
-    this.objectMapper = objectMapper;
   }
   }
 
 
   @Override
   @Override
   @SneakyThrows
   @SneakyThrows
-  public Object format(String topic, byte[] value) {
-    if (value != null) {
-      GenericRecord avroRecord = (GenericRecord) avroDeserializer.deserialize(topic, value);
-      byte[] bytes = AvroSchemaUtils.toJson(avroRecord);
-      return parseJson(bytes);
-    } else {
-      return Map.of();
-    }
-  }
-
-  private Object parseJson(byte[] bytes) throws IOException {
-    return objectMapper.readValue(bytes, new TypeReference<Map<String, Object>>() {
-    });
+  public String format(String topic, byte[] value) {
+    GenericRecord avroRecord = (GenericRecord) avroDeserializer.deserialize(topic, value);
+    byte[] jsonBytes = AvroSchemaUtils.toJson(avroRecord);
+    return new String(jsonBytes);
   }
   }
 }
 }

+ 8 - 12
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/AvroMessageReader.java

@@ -8,8 +8,7 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
 import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
 import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
 import io.confluent.kafka.serializers.KafkaAvroSerializer;
 import io.confluent.kafka.serializers.KafkaAvroSerializer;
 import java.io.IOException;
 import java.io.IOException;
-import org.apache.avro.Schema;
-import org.apache.avro.util.Utf8;
+import java.util.Map;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.serialization.Serializer;
 
 
 public class AvroMessageReader extends MessageReader<Object> {
 public class AvroMessageReader extends MessageReader<Object> {
@@ -23,21 +22,18 @@ public class AvroMessageReader extends MessageReader<Object> {
 
 
   @Override
   @Override
   protected Serializer<Object> createSerializer(SchemaRegistryClient client) {
   protected Serializer<Object> createSerializer(SchemaRegistryClient client) {
-    return new KafkaAvroSerializer(client);
+    var serializer = new KafkaAvroSerializer(client);
+    // need to call configure to set isKey property
+    serializer.configure(Map.of("schema.registry.url", "wontbeused"), isKey);
+    return serializer;
   }
   }
 
 
   @Override
   @Override
-  protected Object read(byte[] value, ParsedSchema schema) {
-    Schema rawSchema = ((AvroSchema) schema).rawSchema();
-
+  protected Object read(String value, ParsedSchema schema) {
     try {
     try {
-      Object object = AvroSchemaUtils.toObject(new String(value), (AvroSchema) schema);
-      if (rawSchema.getType().equals(Schema.Type.STRING)) {
-        object = ((Utf8) object).toString();
-      }
-      return object;
+      return AvroSchemaUtils.toObject(value, (AvroSchema) schema);
     } catch (Throwable e) {
     } catch (Throwable e) {
-      throw new RuntimeException("Failed to merge record for topic " + topic, e);
+      throw new RuntimeException("Failed to serialize record for topic " + topic, e);
     }
     }
 
 
   }
   }

+ 0 - 29
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/JsonMessageFormatter.java

@@ -1,29 +0,0 @@
-package com.provectus.kafka.ui.serde.schemaregistry;
-
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import java.io.IOException;
-import java.util.Map;
-import lombok.SneakyThrows;
-
-public class JsonMessageFormatter implements MessageFormatter {
-  private final ObjectMapper objectMapper;
-
-  public JsonMessageFormatter(ObjectMapper objectMapper) {
-    this.objectMapper = objectMapper;
-  }
-
-  @Override
-  @SneakyThrows
-  public Object format(String topic, byte[] value) {
-    if (value == null) {
-      return Map.of();
-    }
-    return parseJson(value);
-  }
-
-  private Object parseJson(byte[] bytes) throws IOException {
-    return objectMapper.readValue(bytes, new TypeReference<Map<String, Object>>() {
-    });
-  }
-}

+ 0 - 39
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/JsonMessageReader.java

@@ -1,39 +0,0 @@
-package com.provectus.kafka.ui.serde.schemaregistry;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import io.confluent.kafka.schemaregistry.ParsedSchema;
-import io.confluent.kafka.schemaregistry.client.SchemaMetadata;
-import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
-import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
-import java.io.IOException;
-import lombok.SneakyThrows;
-import org.apache.kafka.common.serialization.Serializer;
-
-public class JsonMessageReader extends MessageReader<JsonNode> {
-  private static final ObjectMapper mapper = new ObjectMapper();
-
-  public JsonMessageReader(String topic, boolean isKey,
-                           SchemaRegistryClient client, SchemaMetadata schema) throws IOException,
-      RestClientException {
-    super(topic, isKey, client, schema);
-  }
-
-  @Override
-  protected Serializer<JsonNode> createSerializer(SchemaRegistryClient client) {
-    return new JsonNodeSerializer();
-  }
-
-  @Override
-  @SneakyThrows
-  protected JsonNode read(byte[] value, ParsedSchema schema) {
-    return mapper.readTree(new String(value));
-  }
-
-  private static class JsonNodeSerializer implements Serializer<JsonNode> {
-    @Override
-    public byte[] serialize(String topic, JsonNode data) {
-      return data.toString().getBytes();
-    }
-  }
-}

+ 1 - 1
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/MessageFormatter.java

@@ -1,5 +1,5 @@
 package com.provectus.kafka.ui.serde.schemaregistry;
 package com.provectus.kafka.ui.serde.schemaregistry;
 
 
 public interface MessageFormatter {
 public interface MessageFormatter {
-  Object format(String topic, byte[] value);
+  String format(String topic, byte[] value);
 }
 }

+ 2 - 2
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/MessageReader.java

@@ -23,10 +23,10 @@ public abstract class MessageReader<T> {
 
 
   protected abstract Serializer<T> createSerializer(SchemaRegistryClient client);
   protected abstract Serializer<T> createSerializer(SchemaRegistryClient client);
 
 
-  public byte[] read(byte[] value) {
+  public byte[] read(String value) {
     final T read = this.read(value, schema);
     final T read = this.read(value, schema);
     return this.serializer.serialize(topic, read);
     return this.serializer.serialize(topic, read);
   }
   }
 
 
-  protected abstract T read(byte[] value, ParsedSchema schema);
+  protected abstract T read(String value, ParsedSchema schema);
 }
 }

+ 4 - 8
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/ProtobufMessageFormatter.java

@@ -4,7 +4,6 @@ import com.google.protobuf.Message;
 import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
 import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
 import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchemaUtils;
 import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchemaUtils;
 import io.confluent.kafka.serializers.protobuf.KafkaProtobufDeserializer;
 import io.confluent.kafka.serializers.protobuf.KafkaProtobufDeserializer;
-import java.util.Map;
 import lombok.SneakyThrows;
 import lombok.SneakyThrows;
 
 
 public class ProtobufMessageFormatter implements MessageFormatter {
 public class ProtobufMessageFormatter implements MessageFormatter {
@@ -16,12 +15,9 @@ public class ProtobufMessageFormatter implements MessageFormatter {
 
 
   @Override
   @Override
   @SneakyThrows
   @SneakyThrows
-  public Object format(String topic, byte[] value) {
-    if (value != null) {
-      final Message message = protobufDeserializer.deserialize(topic, value);
-      return ProtobufSchemaUtils.toJson(message);
-    } else {
-      return Map.of();
-    }
+  public String format(String topic, byte[] value) {
+    final Message message = protobufDeserializer.deserialize(topic, value);
+    byte[] jsonBytes = ProtobufSchemaUtils.toJson(message);
+    return new String(jsonBytes);
   }
   }
 }
 }

+ 8 - 4
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/ProtobufMessageReader.java

@@ -10,6 +10,7 @@ import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientExcept
 import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema;
 import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema;
 import io.confluent.kafka.serializers.protobuf.KafkaProtobufSerializer;
 import io.confluent.kafka.serializers.protobuf.KafkaProtobufSerializer;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.Map;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.serialization.Serializer;
 
 
 public class ProtobufMessageReader extends MessageReader<Message> {
 public class ProtobufMessageReader extends MessageReader<Message> {
@@ -22,18 +23,21 @@ public class ProtobufMessageReader extends MessageReader<Message> {
 
 
   @Override
   @Override
   protected Serializer<Message> createSerializer(SchemaRegistryClient client) {
   protected Serializer<Message> createSerializer(SchemaRegistryClient client) {
-    return new KafkaProtobufSerializer<>(client);
+    var serializer = new KafkaProtobufSerializer<>(client);
+    // need to call configure to set isKey property
+    serializer.configure(Map.of("schema.registry.url", "wontbeused"), isKey);
+    return serializer;
   }
   }
 
 
   @Override
   @Override
-  protected Message read(byte[] value, ParsedSchema schema) {
+  protected Message read(String value, ParsedSchema schema) {
     ProtobufSchema protobufSchema = (ProtobufSchema) schema;
     ProtobufSchema protobufSchema = (ProtobufSchema) schema;
     DynamicMessage.Builder builder = protobufSchema.newMessageBuilder();
     DynamicMessage.Builder builder = protobufSchema.newMessageBuilder();
     try {
     try {
-      JsonFormat.parser().merge(new String(value), builder);
+      JsonFormat.parser().merge(value, builder);
       return builder.build();
       return builder.build();
     } catch (Throwable e) {
     } catch (Throwable e) {
-      throw new RuntimeException("Failed to merge record for topic " + topic, e);
+      throw new RuntimeException("Failed to serialize record for topic " + topic, e);
     }
     }
   }
   }
 
 

+ 91 - 81
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/SchemaRegistryRecordSerDe.java → kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/SchemaRegistryAwareRecordSerDe.java

@@ -15,6 +15,7 @@ import io.confluent.kafka.schemaregistry.avro.AvroSchemaProvider;
 import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient;
 import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient;
 import io.confluent.kafka.schemaregistry.client.SchemaMetadata;
 import io.confluent.kafka.schemaregistry.client.SchemaMetadata;
 import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
 import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
+import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
 import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema;
 import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema;
 import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchemaProvider;
 import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchemaProvider;
 import java.net.URI;
 import java.net.URI;
@@ -22,70 +23,76 @@ import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Optional;
 import java.util.Optional;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
+import javax.annotation.Nullable;
 import lombok.SneakyThrows;
 import lombok.SneakyThrows;
 import lombok.extern.log4j.Log4j2;
 import lombok.extern.log4j.Log4j2;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Bytes;
-import reactor.util.function.Tuple2;
-import reactor.util.function.Tuples;
 
 
 @Log4j2
 @Log4j2
-public class SchemaRegistryRecordSerDe implements RecordSerDe {
+public class SchemaRegistryAwareRecordSerDe implements RecordSerDe {
 
 
   private static final int CLIENT_IDENTITY_MAP_CAPACITY = 100;
   private static final int CLIENT_IDENTITY_MAP_CAPACITY = 100;
 
 
   private final KafkaCluster cluster;
   private final KafkaCluster cluster;
-  private final SchemaRegistryClient schemaRegistryClient;
   private final Map<String, MessageFormatter> valueFormatMap = new ConcurrentHashMap<>();
   private final Map<String, MessageFormatter> valueFormatMap = new ConcurrentHashMap<>();
   private final Map<String, MessageFormatter> keyFormatMap = new ConcurrentHashMap<>();
   private final Map<String, MessageFormatter> keyFormatMap = new ConcurrentHashMap<>();
 
 
-  private AvroMessageFormatter avroFormatter;
-  private ProtobufMessageFormatter protobufFormatter;
-  private final JsonMessageFormatter jsonFormatter;
+  @Nullable
+  private final SchemaRegistryClient schemaRegistryClient;
+
+  @Nullable
+  private final AvroMessageFormatter avroFormatter;
+
+  @Nullable
+  private final ProtobufMessageFormatter protobufFormatter;
+
   private final StringMessageFormatter stringFormatter = new StringMessageFormatter();
   private final StringMessageFormatter stringFormatter = new StringMessageFormatter();
   private final ProtobufSchemaConverter protoSchemaConverter = new ProtobufSchemaConverter();
   private final ProtobufSchemaConverter protoSchemaConverter = new ProtobufSchemaConverter();
   private final AvroJsonSchemaConverter avroSchemaConverter = new AvroJsonSchemaConverter();
   private final AvroJsonSchemaConverter avroSchemaConverter = new AvroJsonSchemaConverter();
   private final ObjectMapper objectMapper = new ObjectMapper();
   private final ObjectMapper objectMapper = new ObjectMapper();
 
 
-  public SchemaRegistryRecordSerDe(KafkaCluster cluster, ObjectMapper objectMapper) {
-    this.cluster = cluster;
-
-    this.schemaRegistryClient = Optional.ofNullable(cluster.getSchemaRegistry())
-        .map(schemaRegistryUrl -> {
-              List<SchemaProvider> schemaProviders =
-                  List.of(new AvroSchemaProvider(), new ProtobufSchemaProvider());
-              return new CachedSchemaRegistryClient(
-                  Collections.singletonList(schemaRegistryUrl),
-                  CLIENT_IDENTITY_MAP_CAPACITY,
-                  schemaProviders,
-                  Collections.emptyMap()
-              );
-            }
-        ).orElse(null);
-
-    this.jsonFormatter = new JsonMessageFormatter(objectMapper);
+  private static SchemaRegistryClient createSchemaRegistryClient(KafkaCluster cluster) {
+    Objects.requireNonNull(cluster.getSchemaRegistry());
+    List<SchemaProvider> schemaProviders =
+        List.of(new AvroSchemaProvider(), new ProtobufSchemaProvider());
+    //TODO add auth
+    return new CachedSchemaRegistryClient(
+        Collections.singletonList(cluster.getSchemaRegistry()),
+        CLIENT_IDENTITY_MAP_CAPACITY,
+        schemaProviders,
+        Collections.emptyMap()
+    );
+  }
 
 
+  public SchemaRegistryAwareRecordSerDe(KafkaCluster cluster) {
+    this.cluster = cluster;
+    this.schemaRegistryClient = cluster.getSchemaRegistry() != null
+        ? createSchemaRegistryClient(cluster)
+        : null;
     if (schemaRegistryClient != null) {
     if (schemaRegistryClient != null) {
-      this.avroFormatter = new AvroMessageFormatter(schemaRegistryClient, objectMapper);
+      this.avroFormatter = new AvroMessageFormatter(schemaRegistryClient);
       this.protobufFormatter = new ProtobufMessageFormatter(schemaRegistryClient);
       this.protobufFormatter = new ProtobufMessageFormatter(schemaRegistryClient);
+    } else {
+      this.avroFormatter = null;
+      this.protobufFormatter = null;
     }
     }
   }
   }
 
 
-  public Tuple2<String, Object> deserialize(ConsumerRecord<Bytes, Bytes> msg) {
-    MessageFormatter valueFormatter = getMessageFormatter(msg, false);
-    MessageFormatter keyFormatter = getMessageFormatter(msg, true);
+  public DeserializedKeyValue deserialize(ConsumerRecord<Bytes, Bytes> msg) {
     try {
     try {
-      return Tuples.of(
+      return new DeserializedKeyValue(
           msg.key() != null
           msg.key() != null
-              ? keyFormatter.format(msg.topic(), msg.key().get()).toString()
-              : "",
-          valueFormatter.format(
-              msg.topic(),
-              msg.value() != null ? msg.value().get() : null
-          )
+              ? getMessageFormatter(msg, true).format(msg.topic(), msg.key().get())
+              : null,
+          msg.value() != null
+              ? getMessageFormatter(msg, false).format(msg.topic(), msg.value().get())
+              : null
       );
       );
     } catch (Throwable e) {
     } catch (Throwable e) {
       throw new RuntimeException("Failed to parse record from topic " + msg.topic(), e);
       throw new RuntimeException("Failed to parse record from topic " + msg.topic(), e);
@@ -93,47 +100,43 @@ public class SchemaRegistryRecordSerDe implements RecordSerDe {
   }
   }
 
 
   @Override
   @Override
-  @SneakyThrows
-  public ProducerRecord<byte[], byte[]> serialize(String topic, byte[] key, byte[] data,
-                                                  Optional<Integer> partition) {
+  public ProducerRecord<byte[], byte[]> serialize(String topic,
+                                                  @Nullable String key,
+                                                  @Nullable String data,
+                                                  @Nullable Integer partition) {
     final Optional<SchemaMetadata> maybeValueSchema = getSchemaBySubject(topic, false);
     final Optional<SchemaMetadata> maybeValueSchema = getSchemaBySubject(topic, false);
     final Optional<SchemaMetadata> maybeKeySchema = getSchemaBySubject(topic, true);
     final Optional<SchemaMetadata> maybeKeySchema = getSchemaBySubject(topic, true);
 
 
-    final Optional<byte[]> serializedValue = serialize(maybeValueSchema, topic, data);
-    final Optional<byte[]> serializedKey = serialize(maybeKeySchema, topic, key);
+    final byte[] serializedValue = data != null
+        ? serialize(maybeValueSchema, topic, data, false)
+        : null;
+    final byte[] serializedKey = key != null
+        ? serialize(maybeKeySchema, topic, key, true)
+        : null;
 
 
-    if (serializedValue.isPresent()) {
-      return partition
-          .map(p ->
-              new ProducerRecord<>(topic, p, serializedKey.orElse(key), serializedValue.get())
-          ).orElseGet(() ->
-              new ProducerRecord<>(topic, serializedKey.orElse(key), serializedValue.get())
-          );
-    } else {
-      throw new RuntimeException("Subject was not found for topic " + topic);
-    }
+    return new ProducerRecord<>(topic, partition, serializedKey, serializedValue);
   }
   }
 
 
   @SneakyThrows
   @SneakyThrows
-  private Optional<byte[]> serialize(
-      Optional<SchemaMetadata> maybeSchema, String topic, byte[] value) {
+  private byte[] serialize(
+      Optional<SchemaMetadata> maybeSchema, String topic, String value, boolean isKey) {
     if (maybeSchema.isPresent()) {
     if (maybeSchema.isPresent()) {
       final SchemaMetadata schema = maybeSchema.get();
       final SchemaMetadata schema = maybeSchema.get();
 
 
       MessageReader<?> reader;
       MessageReader<?> reader;
       if (schema.getSchemaType().equals(MessageFormat.PROTOBUF.name())) {
       if (schema.getSchemaType().equals(MessageFormat.PROTOBUF.name())) {
-        reader = new ProtobufMessageReader(topic, false, schemaRegistryClient, schema);
+        reader = new ProtobufMessageReader(topic, isKey, schemaRegistryClient, schema);
       } else if (schema.getSchemaType().equals(MessageFormat.AVRO.name())) {
       } else if (schema.getSchemaType().equals(MessageFormat.AVRO.name())) {
-        reader = new AvroMessageReader(topic, false, schemaRegistryClient, schema);
+        reader = new AvroMessageReader(topic, isKey, schemaRegistryClient, schema);
       } else {
       } else {
-        reader = new JsonMessageReader(topic, false, schemaRegistryClient, schema);
+        throw new IllegalStateException("Unsupported schema type: " + schema.getSchemaType());
       }
       }
 
 
-      return Optional.of(reader.read(value));
+      return reader.read(value);
     } else {
     } else {
-      return Optional.empty();
+      // if no schema provided serialize input as raw string
+      return value.getBytes();
     }
     }
-
   }
   }
 
 
   @Override
   @Override
@@ -171,7 +174,8 @@ public class SchemaRegistryRecordSerDe implements RecordSerDe {
 
 
     String jsonSchema;
     String jsonSchema;
     URI basePath = new URI(cluster.getSchemaRegistry()).resolve(Integer.toString(schema.getId()));
     URI basePath = new URI(cluster.getSchemaRegistry()).resolve(Integer.toString(schema.getId()));
-    final ParsedSchema schemaById = schemaRegistryClient.getSchemaById(schema.getId());
+    final ParsedSchema schemaById = Objects.requireNonNull(schemaRegistryClient)
+        .getSchemaById(schema.getId());
 
 
     if (schema.getSchemaType().equals(MessageFormat.PROTOBUF.name())) {
     if (schema.getSchemaType().equals(MessageFormat.PROTOBUF.name())) {
       final ProtobufSchema protobufSchema = (ProtobufSchema) schemaById;
       final ProtobufSchema protobufSchema = (ProtobufSchema) schemaById;
@@ -207,38 +211,31 @@ public class SchemaRegistryRecordSerDe implements RecordSerDe {
             .or(() -> getSchemaBySubject(msg.topic(), isKey).map(SchemaMetadata::getSchemaType));
             .or(() -> getSchemaBySubject(msg.topic(), isKey).map(SchemaMetadata::getSchemaType));
         if (type.isPresent()) {
         if (type.isPresent()) {
           if (type.get().equals(MessageFormat.PROTOBUF.name())) {
           if (type.get().equals(MessageFormat.PROTOBUF.name())) {
-            if (tryFormatter(protobufFormatter, msg).isPresent()) {
+            if (tryFormatter(protobufFormatter, msg, isKey).isPresent()) {
               return protobufFormatter;
               return protobufFormatter;
             }
             }
           } else if (type.get().equals(MessageFormat.AVRO.name())) {
           } else if (type.get().equals(MessageFormat.AVRO.name())) {
-            if (tryFormatter(avroFormatter, msg).isPresent()) {
+            if (tryFormatter(avroFormatter, msg, isKey).isPresent()) {
               return avroFormatter;
               return avroFormatter;
             }
             }
-          } else if (type.get().equals(MessageFormat.JSON.name())) {
-            if (tryFormatter(jsonFormatter, msg).isPresent()) {
-              return jsonFormatter;
-            }
+          } else {
+            throw new IllegalStateException("Unsupported schema type: " + type.get());
           }
           }
         }
         }
       } catch (Exception e) {
       } catch (Exception e) {
         log.warn("Failed to get Schema for topic {}", msg.topic(), e);
         log.warn("Failed to get Schema for topic {}", msg.topic(), e);
       }
       }
     }
     }
-
-    if (tryFormatter(jsonFormatter, msg).isPresent()) {
-      return jsonFormatter;
-    }
-
     return stringFormatter;
     return stringFormatter;
   }
   }
 
 
   private Optional<MessageFormatter> tryFormatter(
   private Optional<MessageFormatter> tryFormatter(
-      MessageFormatter formatter, ConsumerRecord<Bytes, Bytes> msg) {
+      MessageFormatter formatter, ConsumerRecord<Bytes, Bytes> msg, boolean isKey) {
     try {
     try {
-      formatter.format(msg.topic(), msg.value().get());
+      formatter.format(msg.topic(), isKey ? msg.key().get() : msg.value().get());
       return Optional.of(formatter);
       return Optional.of(formatter);
     } catch (Throwable e) {
     } catch (Throwable e) {
-      log.info("Failed to parse by {} from topic {}", formatter.getClass(), msg.topic());
+      log.warn("Failed to parse by {} from topic {}", formatter.getClass(), msg.topic(), e);
     }
     }
 
 
     return Optional.empty();
     return Optional.empty();
@@ -252,9 +249,10 @@ public class SchemaRegistryRecordSerDe implements RecordSerDe {
       ByteBuffer buffer = ByteBuffer.wrap(value.get());
       ByteBuffer buffer = ByteBuffer.wrap(value.get());
       if (buffer.get() == 0) {
       if (buffer.get() == 0) {
         int id = buffer.getInt();
         int id = buffer.getInt();
-        result = Optional.ofNullable(
-            schemaRegistryClient.getSchemaById(id)
-        ).map(ParsedSchema::schemaType);
+        result =
+            Optional.ofNullable(schemaRegistryClient)
+                .flatMap(client -> wrapClientCall(() -> client.getSchemaById(id)))
+                .map(ParsedSchema::schemaType);
       }
       }
     }
     }
     return result;
     return result;
@@ -262,11 +260,23 @@ public class SchemaRegistryRecordSerDe implements RecordSerDe {
 
 
   @SneakyThrows
   @SneakyThrows
   private Optional<SchemaMetadata> getSchemaBySubject(String topic, boolean isKey) {
   private Optional<SchemaMetadata> getSchemaBySubject(String topic, boolean isKey) {
-    return Optional.ofNullable(
-        schemaRegistryClient.getLatestSchemaMetadata(
-            schemaSubject(topic, isKey)
-        )
-    );
+    return Optional.ofNullable(schemaRegistryClient)
+        .flatMap(client ->
+            wrapClientCall(() ->
+                client.getLatestSchemaMetadata(schemaSubject(topic, isKey))));
+  }
+
+  @SneakyThrows
+  private <T> Optional<T> wrapClientCall(Callable<T> call) {
+    try {
+      return Optional.ofNullable(call.call());
+    } catch (RestClientException restClientException) {
+      if (restClientException.getStatus() == 404) {
+        return Optional.empty();
+      } else {
+        throw new RuntimeException("Error calling SchemaRegistryClient", restClientException);
+      }
+    }
   }
   }
 
 
   private String schemaSubject(String topic, boolean isKey) {
   private String schemaSubject(String topic, boolean isKey) {

+ 2 - 9
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serde/schemaregistry/StringMessageFormatter.java

@@ -1,18 +1,11 @@
 package com.provectus.kafka.ui.serde.schemaregistry;
 package com.provectus.kafka.ui.serde.schemaregistry;
 
 
 import java.nio.charset.StandardCharsets;
 import java.nio.charset.StandardCharsets;
-import java.util.Map;
-import lombok.SneakyThrows;
 
 
 public class StringMessageFormatter implements MessageFormatter {
 public class StringMessageFormatter implements MessageFormatter {
 
 
   @Override
   @Override
-  @SneakyThrows
-  public Object format(String topic, byte[] value) {
-    if (value != null) {
-      return new String(value, StandardCharsets.UTF_8);
-    } else {
-      return Map.of();
-    }
+  public String format(String topic, byte[] value) {
+    return new String(value, StandardCharsets.UTF_8);
   }
   }
 }
 }

+ 8 - 0
kafka-ui-api/src/main/java/com/provectus/kafka/ui/service/ClusterService.java

@@ -4,6 +4,7 @@ import com.provectus.kafka.ui.exception.ClusterNotFoundException;
 import com.provectus.kafka.ui.exception.IllegalEntityStateException;
 import com.provectus.kafka.ui.exception.IllegalEntityStateException;
 import com.provectus.kafka.ui.exception.NotFoundException;
 import com.provectus.kafka.ui.exception.NotFoundException;
 import com.provectus.kafka.ui.exception.TopicNotFoundException;
 import com.provectus.kafka.ui.exception.TopicNotFoundException;
+import com.provectus.kafka.ui.exception.ValidationException;
 import com.provectus.kafka.ui.mapper.ClusterMapper;
 import com.provectus.kafka.ui.mapper.ClusterMapper;
 import com.provectus.kafka.ui.model.Broker;
 import com.provectus.kafka.ui.model.Broker;
 import com.provectus.kafka.ui.model.BrokerMetrics;
 import com.provectus.kafka.ui.model.BrokerMetrics;
@@ -351,6 +352,13 @@ public class ClusterService {
     if (!cluster.getTopics().containsKey(topicName)) {
     if (!cluster.getTopics().containsKey(topicName)) {
       throw new TopicNotFoundException();
       throw new TopicNotFoundException();
     }
     }
+    if (msg.getKey() == null && msg.getContent() == null) {
+      throw new ValidationException("Invalid message: both key and value can't be null");
+    }
+    if (msg.getPartition() != null
+        && msg.getPartition() > cluster.getTopics().get(topicName).getPartitionCount() - 1) {
+      throw new ValidationException("Invalid partition");
+    }
     return kafkaService.sendMessage(cluster, topicName, msg).then();
     return kafkaService.sendMessage(cluster, topicName, msg).then();
   }
   }
 
 

+ 5 - 3
kafka-ui-api/src/main/java/com/provectus/kafka/ui/service/ClustersMetricsScheduler.java

@@ -17,13 +17,15 @@ public class ClustersMetricsScheduler {
 
 
   private final MetricsUpdateService metricsUpdateService;
   private final MetricsUpdateService metricsUpdateService;
 
 
-  @Scheduled(fixedRate = 30000)
+  @Scheduled(fixedRateString = "${kafka.update-metrics-rate-millis:30000}")
   public void updateMetrics() {
   public void updateMetrics() {
     Flux.fromIterable(clustersStorage.getKafkaClustersMap().entrySet())
     Flux.fromIterable(clustersStorage.getKafkaClustersMap().entrySet())
-        .subscribeOn(Schedulers.parallel())
+        .parallel()
+        .runOn(Schedulers.parallel())
         .map(Map.Entry::getValue)
         .map(Map.Entry::getValue)
         .flatMap(metricsUpdateService::updateMetrics)
         .flatMap(metricsUpdateService::updateMetrics)
         .doOnNext(s -> clustersStorage.setKafkaCluster(s.getName(), s))
         .doOnNext(s -> clustersStorage.setKafkaCluster(s.getName(), s))
-        .subscribe();
+        .then()
+        .block();
   }
   }
 }
 }

+ 2 - 27
kafka-ui-api/src/main/java/com/provectus/kafka/ui/service/ConsumingService.java

@@ -1,6 +1,5 @@
 package com.provectus.kafka.ui.service;
 package com.provectus.kafka.ui.service;
 
 
-import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.provectus.kafka.ui.emitter.BackwardRecordEmitter;
 import com.provectus.kafka.ui.emitter.BackwardRecordEmitter;
 import com.provectus.kafka.ui.emitter.ForwardRecordEmitter;
 import com.provectus.kafka.ui.emitter.ForwardRecordEmitter;
@@ -14,7 +13,6 @@ import com.provectus.kafka.ui.util.ClusterUtil;
 import com.provectus.kafka.ui.util.OffsetsSeekBackward;
 import com.provectus.kafka.ui.util.OffsetsSeekBackward;
 import com.provectus.kafka.ui.util.OffsetsSeekForward;
 import com.provectus.kafka.ui.util.OffsetsSeekForward;
 import java.util.Collection;
 import java.util.Collection;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Optional;
@@ -108,31 +106,8 @@ public class ConsumingService {
     if (StringUtils.isEmpty(query)) {
     if (StringUtils.isEmpty(query)) {
       return true;
       return true;
     }
     }
-
-    Object content = message.getContent();
-    JsonNode tree = objectMapper.valueToTree(content);
-    return treeContainsValue(tree, query);
-  }
-
-  private boolean treeContainsValue(JsonNode tree, String query) {
-    LinkedList<JsonNode> nodesForSearch = new LinkedList<>();
-    nodesForSearch.add(tree);
-
-    while (!nodesForSearch.isEmpty()) {
-      JsonNode node = nodesForSearch.removeFirst();
-
-      if (node.isContainerNode()) {
-        node.elements().forEachRemaining(nodesForSearch::add);
-        continue;
-      }
-
-      String nodeValue = node.asText();
-      if (nodeValue.contains(query)) {
-        return true;
-      }
-    }
-
-    return false;
+    return (StringUtils.isNotEmpty(message.getKey()) && message.getKey().contains(query))
+         || (StringUtils.isNotEmpty(message.getContent()) && message.getContent().contains(query));
   }
   }
 
 
 }
 }

+ 8 - 4
kafka-ui-api/src/main/java/com/provectus/kafka/ui/service/KafkaService.java

@@ -68,6 +68,7 @@ import org.apache.kafka.clients.producer.RecordMetadata;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.config.ConfigResource;
 import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.kafka.common.serialization.BytesDeserializer;
 import org.apache.kafka.common.serialization.BytesDeserializer;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Bytes;
 import org.springframework.beans.factory.annotation.Value;
 import org.springframework.beans.factory.annotation.Value;
@@ -672,11 +673,14 @@ public class KafkaService {
     Properties properties = new Properties();
     Properties properties = new Properties();
     properties.putAll(cluster.getProperties());
     properties.putAll(cluster.getProperties());
     properties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers());
     properties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers());
+    properties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class);
+    properties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class);
     try (KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(properties)) {
     try (KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(properties)) {
-      final ProducerRecord<byte[], byte[]> producerRecord = serde.serialize(topic,
-          msg.getKey() != null ? msg.getKey().getBytes() : null,
-          msg.getContent().toString().getBytes(),
-          Optional.ofNullable(msg.getPartition())
+      final ProducerRecord<byte[], byte[]> producerRecord = serde.serialize(
+          topic,
+          msg.getKey(),
+          msg.getContent(),
+          msg.getPartition()
       );
       );
 
 
       CompletableFuture<RecordMetadata> cf = new CompletableFuture<>();
       CompletableFuture<RecordMetadata> cf = new CompletableFuture<>();

+ 3 - 3
kafka-ui-api/src/main/java/com/provectus/kafka/ui/util/ClusterUtil.java

@@ -215,9 +215,9 @@ public class ClusterUtil {
     topicMessage.setTimestampType(timestampType);
     topicMessage.setTimestampType(timestampType);
 
 
     topicMessage.setHeaders(headers);
     topicMessage.setHeaders(headers);
-    Tuple2<String, Object> parsed = recordDeserializer.deserialize(consumerRecord);
-    topicMessage.setKey(parsed.getT1());
-    topicMessage.setContent(parsed.getT2());
+    var parsed = recordDeserializer.deserialize(consumerRecord);
+    topicMessage.setKey(parsed.getKey());
+    topicMessage.setContent(parsed.getValue());
 
 
     return topicMessage;
     return topicMessage;
   }
   }

+ 2 - 2
kafka-ui-api/src/test/java/com/provectus/kafka/ui/AbstractBaseTest.java

@@ -56,14 +56,14 @@ public abstract class AbstractBaseTest {
     public void initialize(@NotNull ConfigurableApplicationContext context) {
     public void initialize(@NotNull ConfigurableApplicationContext context) {
       System.setProperty("kafka.clusters.0.name", LOCAL);
       System.setProperty("kafka.clusters.0.name", LOCAL);
       System.setProperty("kafka.clusters.0.bootstrapServers", kafka.getBootstrapServers());
       System.setProperty("kafka.clusters.0.bootstrapServers", kafka.getBootstrapServers());
-      System.setProperty("kafka.clusters.0.schemaRegistry", schemaRegistry.getTarget());
+      System.setProperty("kafka.clusters.0.schemaRegistry", schemaRegistry.getUrl());
       System.setProperty("kafka.clusters.0.kafkaConnect.0.name", "kafka-connect");
       System.setProperty("kafka.clusters.0.kafkaConnect.0.name", "kafka-connect");
       System.setProperty("kafka.clusters.0.kafkaConnect.0.address", kafkaConnect.getTarget());
       System.setProperty("kafka.clusters.0.kafkaConnect.0.address", kafkaConnect.getTarget());
 
 
       System.setProperty("kafka.clusters.1.name", SECOND_LOCAL);
       System.setProperty("kafka.clusters.1.name", SECOND_LOCAL);
       System.setProperty("kafka.clusters.1.readOnly", "true");
       System.setProperty("kafka.clusters.1.readOnly", "true");
       System.setProperty("kafka.clusters.1.bootstrapServers", kafka.getBootstrapServers());
       System.setProperty("kafka.clusters.1.bootstrapServers", kafka.getBootstrapServers());
-      System.setProperty("kafka.clusters.1.schemaRegistry", schemaRegistry.getTarget());
+      System.setProperty("kafka.clusters.1.schemaRegistry", schemaRegistry.getUrl());
       System.setProperty("kafka.clusters.1.kafkaConnect.0.name", "kafka-connect");
       System.setProperty("kafka.clusters.1.kafkaConnect.0.name", "kafka-connect");
       System.setProperty("kafka.clusters.1.kafkaConnect.0.address", kafkaConnect.getTarget());
       System.setProperty("kafka.clusters.1.kafkaConnect.0.address", kafkaConnect.getTarget());
     }
     }

+ 8 - 1
kafka-ui-api/src/test/java/com/provectus/kafka/ui/container/SchemaRegistryContainer.java

@@ -1,5 +1,7 @@
 package com.provectus.kafka.ui.container;
 package com.provectus.kafka.ui.container;
 
 
+import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient;
+import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
 import org.testcontainers.containers.GenericContainer;
 import org.testcontainers.containers.GenericContainer;
 import org.testcontainers.containers.KafkaContainer;
 import org.testcontainers.containers.KafkaContainer;
 import org.testcontainers.containers.Network;
 import org.testcontainers.containers.Network;
@@ -25,7 +27,12 @@ public class SchemaRegistryContainer extends GenericContainer<SchemaRegistryCont
     return self();
     return self();
   }
   }
 
 
-  public String getTarget() {
+  public String getUrl() {
     return "http://" + getContainerIpAddress() + ":" + getMappedPort(SCHEMA_PORT);
     return "http://" + getContainerIpAddress() + ":" + getMappedPort(SCHEMA_PORT);
   }
   }
+
+  public SchemaRegistryClient schemaRegistryClient() {
+    return new CachedSchemaRegistryClient(getUrl(), 1000);
+  }
+
 }
 }

+ 7 - 9
kafka-ui-api/src/test/java/com/provectus/kafka/ui/serde/SchemaRegistryRecordDeserializerTest.java

@@ -1,24 +1,22 @@
 package com.provectus.kafka.ui.serde;
 package com.provectus.kafka.ui.serde;
 
 
+import static com.provectus.kafka.ui.serde.RecordSerDe.DeserializedKeyValue;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.provectus.kafka.ui.model.KafkaCluster;
 import com.provectus.kafka.ui.model.KafkaCluster;
-import com.provectus.kafka.ui.serde.schemaregistry.SchemaRegistryRecordSerDe;
-import java.util.Map;
+import com.provectus.kafka.ui.serde.schemaregistry.SchemaRegistryAwareRecordSerDe;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Bytes;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.Test;
-import reactor.util.function.Tuples;
 
 
 class SchemaRegistryRecordDeserializerTest {
 class SchemaRegistryRecordDeserializerTest {
 
 
-  private final SchemaRegistryRecordSerDe deserializer =
-      new SchemaRegistryRecordSerDe(
+  private final SchemaRegistryAwareRecordSerDe deserializer =
+      new SchemaRegistryAwareRecordSerDe(
           KafkaCluster.builder()
           KafkaCluster.builder()
               .schemaNameTemplate("%s-value")
               .schemaNameTemplate("%s-value")
-              .build(),
-          new ObjectMapper()
+              .build()
       );
       );
 
 
   @Test
   @Test
@@ -27,13 +25,13 @@ class SchemaRegistryRecordDeserializerTest {
     var deserializedRecord = deserializer.deserialize(
     var deserializedRecord = deserializer.deserialize(
         new ConsumerRecord<>("topic", 1, 0, Bytes.wrap("key".getBytes()),
         new ConsumerRecord<>("topic", 1, 0, Bytes.wrap("key".getBytes()),
             Bytes.wrap(value.getBytes())));
             Bytes.wrap(value.getBytes())));
-    assertEquals(Tuples.of("key", value), deserializedRecord);
+    assertEquals(new DeserializedKeyValue("key", value), deserializedRecord);
   }
   }
 
 
   @Test
   @Test
   public void shouldDeserializeNullValueRecordToEmptyMap() {
   public void shouldDeserializeNullValueRecordToEmptyMap() {
     var deserializedRecord = deserializer
     var deserializedRecord = deserializer
         .deserialize(new ConsumerRecord<>("topic", 1, 0, Bytes.wrap("key".getBytes()), null));
         .deserialize(new ConsumerRecord<>("topic", 1, 0, Bytes.wrap("key".getBytes()), null));
-    assertEquals(Tuples.of("key", Map.of()), deserializedRecord);
+    assertEquals(new DeserializedKeyValue("key", null), deserializedRecord);
   }
   }
 }
 }

+ 360 - 0
kafka-ui-api/src/test/java/com/provectus/kafka/ui/service/SendAndReadTests.java

@@ -0,0 +1,360 @@
+package com.provectus.kafka.ui.service;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.provectus.kafka.ui.AbstractBaseTest;
+import com.provectus.kafka.ui.model.ConsumerPosition;
+import com.provectus.kafka.ui.model.CreateTopicMessage;
+import com.provectus.kafka.ui.model.SeekDirection;
+import com.provectus.kafka.ui.model.SeekType;
+import com.provectus.kafka.ui.model.TopicMessage;
+import io.confluent.kafka.schemaregistry.ParsedSchema;
+import io.confluent.kafka.schemaregistry.avro.AvroSchema;
+import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema;
+import java.time.Duration;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.function.Consumer;
+import lombok.SneakyThrows;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.common.TopicPartition;
+import org.junit.Assert;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+public class SendAndReadTests extends AbstractBaseTest {
+
+  private static final AvroSchema AVRO_SCHEMA_1 = new AvroSchema(
+      "{"
+          + "  \"type\": \"record\","
+          + "  \"name\": \"TestAvroRecord1\","
+          + "  \"fields\": ["
+          + "    {"
+          + "      \"name\": \"field1\","
+          + "      \"type\": \"string\""
+          + "    },"
+          + "    {"
+          + "      \"name\": \"field2\","
+          + "      \"type\": \"int\""
+          + "    }"
+          + "  ]"
+          + "}"
+  );
+
+  private static final AvroSchema AVRO_SCHEMA_2 = new AvroSchema(
+      "{"
+          + "  \"type\": \"record\","
+          + "  \"name\": \"TestAvroRecord2\","
+          + "  \"fields\": ["
+          + "    {"
+          + "      \"name\": \"f1\","
+          + "      \"type\": \"int\""
+          + "    },"
+          + "    {"
+          + "      \"name\": \"f2\","
+          + "      \"type\": \"string\""
+          + "    }"
+          + "  ]"
+          + "}"
+  );
+
+  private static final String AVRO_SCHEMA_1_JSON_RECORD
+      = "{ \"field1\":\"testStr\", \"field2\": 123 }";
+
+  private static final String AVRO_SCHEMA_2_JSON_RECORD = "{ \"f1\": 111, \"f2\": \"testStr\" }";
+
+  private static final ProtobufSchema PROTOBUF_SCHEMA = new ProtobufSchema(
+      "syntax = \"proto3\";\n"
+          + "package com.provectus;\n"
+          + "\n"
+          + "message TestProtoRecord {\n"
+          + "  string f1 = 1;\n"
+          + "  int32 f2 = 2;\n"
+          + "}\n"
+          + "\n"
+  );
+
+  private static final String PROTOBUF_SCHEMA_JSON_RECORD
+      = "{ \"f1\" : \"test str\", \"f2\" : 123 }";
+
+  @Autowired
+  private ClusterService clusterService;
+
+  @Autowired
+  private ClustersMetricsScheduler clustersMetricsScheduler;
+
+  @Test
+  void noSchemaStringKeyStringValue() {
+    new SendAndReadSpec()
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key("testKey")
+                .content("testValue")
+        )
+        .doAssert(polled -> {
+          assertThat(polled.getKey()).isEqualTo("testKey");
+          assertThat(polled.getContent()).isEqualTo("testValue");
+        });
+  }
+
+  @Test
+  void noSchemaJsonKeyJsonValue() {
+    new SendAndReadSpec()
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key("{ \"f1\": 111, \"f2\": \"testStr1\" }")
+                .content("{ \"f1\": 222, \"f2\": \"testStr2\" }")
+        )
+        .doAssert(polled -> {
+          assertThat(polled.getKey()).isEqualTo("{ \"f1\": 111, \"f2\": \"testStr1\" }");
+          assertThat(polled.getContent()).isEqualTo("{ \"f1\": 222, \"f2\": \"testStr2\" }");
+        });
+  }
+
+  @Test
+  void keyIsIntValueIsDoubleShouldBeSerializedAsStrings() {
+    new SendAndReadSpec()
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key("123")
+                .content("234.56")
+        )
+        .doAssert(polled -> {
+          assertThat(polled.getKey()).isEqualTo("123");
+          assertThat(polled.getContent()).isEqualTo("234.56");
+        });
+  }
+
+  @Test
+  void noSchemaKeyIsNull() {
+    new SendAndReadSpec()
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key(null)
+                .content("testValue")
+        )
+        .doAssert(polled -> {
+          assertThat(polled.getKey()).isNull();
+          assertThat(polled.getContent()).isEqualTo("testValue");
+        });
+  }
+
+  @Test
+  void noSchemaValueIsNull() {
+    new SendAndReadSpec()
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key("testKey")
+                .content(null)
+        )
+        .doAssert(polled -> {
+          assertThat(polled.getKey()).isEqualTo("testKey");
+          assertThat(polled.getContent()).isNull();
+        });
+  }
+
+  @Test
+  void nonNullableKvWithAvroSchema() {
+    new SendAndReadSpec()
+        .withKeySchema(AVRO_SCHEMA_1)
+        .withValueSchema(AVRO_SCHEMA_2)
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key(AVRO_SCHEMA_1_JSON_RECORD)
+                .content(AVRO_SCHEMA_2_JSON_RECORD)
+        )
+        .doAssert(polled -> {
+          assertJsonEqual(polled.getKey(), AVRO_SCHEMA_1_JSON_RECORD);
+          assertJsonEqual(polled.getContent(), AVRO_SCHEMA_2_JSON_RECORD);
+        });
+  }
+
+  @Test
+  void keyWithNoSchemaValueWithAvroSchema() {
+    new SendAndReadSpec()
+        .withValueSchema(AVRO_SCHEMA_1)
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key("testKey")
+                .content(AVRO_SCHEMA_1_JSON_RECORD)
+        )
+        .doAssert(polled -> {
+          assertThat(polled.getKey()).isEqualTo("testKey");
+          assertJsonEqual(polled.getContent(), AVRO_SCHEMA_1_JSON_RECORD);
+        });
+  }
+
+  @Test
+  void keyWithAvroSchemaValueWithNoSchema() {
+    new SendAndReadSpec()
+        .withKeySchema(AVRO_SCHEMA_1)
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key(AVRO_SCHEMA_1_JSON_RECORD)
+                .content("testVal")
+        )
+        .doAssert(polled -> {
+          assertJsonEqual(polled.getKey(), AVRO_SCHEMA_1_JSON_RECORD);
+          assertThat(polled.getContent()).isEqualTo("testVal");
+        });
+  }
+
+  @Test
+  void keyWithNoSchemaValueWithProtoSchema() {
+    new SendAndReadSpec()
+        .withValueSchema(PROTOBUF_SCHEMA)
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key("testKey")
+                .content(PROTOBUF_SCHEMA_JSON_RECORD)
+        )
+        .doAssert(polled -> {
+          assertThat(polled.getKey()).isEqualTo("testKey");
+          assertJsonEqual(polled.getContent(), PROTOBUF_SCHEMA_JSON_RECORD);
+        });
+  }
+
+  @Test
+  void keyWithAvroSchemaValueWithAvroSchemaKeyIsNull() {
+    new SendAndReadSpec()
+        .withKeySchema(AVRO_SCHEMA_1)
+        .withValueSchema(AVRO_SCHEMA_2)
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key(null)
+                .content(AVRO_SCHEMA_2_JSON_RECORD)
+        )
+        .doAssert(polled -> {
+          assertThat(polled.getKey()).isNull();
+          assertJsonEqual(polled.getContent(), AVRO_SCHEMA_2_JSON_RECORD);
+        });
+  }
+
+  @Test
+  void valueWithAvroSchemaShouldThrowExceptionArgIsNotValidJsonObject() {
+    assertThatThrownBy(() -> {
+      new SendAndReadSpec()
+          .withValueSchema(AVRO_SCHEMA_2)
+          .withMsgToSend(
+              new CreateTopicMessage()
+                  .content("not a json object")
+          )
+          .doAssert(polled -> Assertions.fail());
+    }).hasMessageContaining("Failed to serialize record");
+  }
+
+  @Test
+  void keyWithAvroSchemaValueWithAvroSchemaValueIsNull() {
+    new SendAndReadSpec()
+        .withKeySchema(AVRO_SCHEMA_1)
+        .withValueSchema(AVRO_SCHEMA_2)
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key(AVRO_SCHEMA_1_JSON_RECORD)
+                .content(null)
+        )
+        .doAssert(polled -> {
+          assertJsonEqual(polled.getKey(), AVRO_SCHEMA_1_JSON_RECORD);
+          assertThat(polled.getContent()).isNull();
+        });
+  }
+
+  @Test
+  void keyWithAvroSchemaValueWithProtoSchema() {
+    new SendAndReadSpec()
+        .withKeySchema(AVRO_SCHEMA_1)
+        .withValueSchema(PROTOBUF_SCHEMA)
+        .withMsgToSend(
+            new CreateTopicMessage()
+                .key(AVRO_SCHEMA_1_JSON_RECORD)
+                .content(PROTOBUF_SCHEMA_JSON_RECORD)
+        )
+        .doAssert(polled -> {
+          assertJsonEqual(polled.getKey(), AVRO_SCHEMA_1_JSON_RECORD);
+          assertJsonEqual(polled.getContent(), PROTOBUF_SCHEMA_JSON_RECORD);
+        });
+  }
+
+  @Test
+  void valueWithProtoSchemaShouldThrowExceptionArgIsNotValidJsonObject() {
+    assertThatThrownBy(() -> {
+      new SendAndReadSpec()
+          .withValueSchema(PROTOBUF_SCHEMA)
+          .withMsgToSend(
+              new CreateTopicMessage()
+                  .content("not a json object")
+          )
+          .doAssert(polled -> Assertions.fail());
+    }).hasMessageContaining("Failed to serialize record");
+  }
+
+
+  @SneakyThrows
+  private void assertJsonEqual(String actual, String expected) {
+    var mapper = new ObjectMapper();
+    assertThat(mapper.readTree(actual)).isEqualTo(mapper.readTree(expected));
+  }
+
+  class SendAndReadSpec {
+    CreateTopicMessage msgToSend;
+    ParsedSchema keySchema;
+    ParsedSchema valueSchema;
+
+    public SendAndReadSpec withMsgToSend(CreateTopicMessage msg) {
+      this.msgToSend = msg;
+      return this;
+    }
+
+    public SendAndReadSpec withKeySchema(ParsedSchema keyScheam) {
+      this.keySchema = keyScheam;
+      return this;
+    }
+
+    public SendAndReadSpec withValueSchema(ParsedSchema valueSchema) {
+      this.valueSchema = valueSchema;
+      return this;
+    }
+
+    @SneakyThrows
+    public void doAssert(Consumer<TopicMessage> msgAssert) {
+      Objects.requireNonNull(msgToSend);
+      String topic = UUID.randomUUID().toString();
+      createTopic(new NewTopic(topic, 1, (short) 1));
+      if (keySchema != null) {
+        schemaRegistry.schemaRegistryClient().register(topic + "-key", keySchema);
+      }
+      if (valueSchema != null) {
+        schemaRegistry.schemaRegistryClient().register(topic + "-value", valueSchema);
+      }
+
+      // need to update to see new topic & schemas
+      clustersMetricsScheduler.updateMetrics();
+      try {
+        clusterService.sendMessage(LOCAL, topic, msgToSend).block();
+        TopicMessage polled = clusterService.getMessages(
+            LOCAL,
+            topic,
+            new ConsumerPosition(
+                SeekType.BEGINNING,
+                Map.of(new TopicPartition(topic, 0), 0L),
+                SeekDirection.FORWARD
+            ),
+            null,
+            1
+        ).blockLast(Duration.ofSeconds(5));
+
+        assertThat(polled).isNotNull();
+        assertThat(polled.getPartition()).isEqualTo(0);
+        assertThat(polled.getOffset()).isNotNull();
+        msgAssert.accept(polled);
+      } finally {
+        deleteTopic(topic);
+      }
+    }
+  }
+
+}

+ 2 - 4
kafka-ui-contract/src/main/resources/swagger/kafka-ui-api.yaml

@@ -1615,9 +1615,7 @@ components:
           additionalProperties:
           additionalProperties:
             type: string
             type: string
         content:
         content:
-          type: object
-      required:
-        - content
+          type: string
 
 
     TopicMessageSchema:
     TopicMessageSchema:
       type: object
       type: object
@@ -1672,7 +1670,7 @@ components:
           additionalProperties:
           additionalProperties:
             type: string
             type: string
         content:
         content:
-          type: object
+          type: string
       required:
       required:
         - partition
         - partition
         - offset
         - offset

+ 1 - 1
kafka-ui-react-app/src/components/Topics/Topic/Details/Messages/MessageItem.tsx

@@ -24,7 +24,7 @@ const MessageItem: React.FC<MessageItemProp> = ({
 }) => {
 }) => {
   const { copyToClipboard, saveFile } = useDataSaver(
   const { copyToClipboard, saveFile } = useDataSaver(
     'topic-message',
     'topic-message',
-    (content as Record<string, string>) || ''
+    content || ''
   );
   );
   return (
   return (
     <tr>
     <tr>

+ 2 - 2
kafka-ui-react-app/src/components/Topics/Topic/Details/Messages/__test__/Messages.spec.tsx

@@ -57,7 +57,7 @@ describe('Messages', () => {
                 partition: 1,
                 partition: 1,
                 offset: 2,
                 offset: 2,
                 timestamp: new Date('05-05-1994'),
                 timestamp: new Date('05-05-1994'),
-                content: [1, 2, 3],
+                content: '[1, 2, 3]',
               },
               },
             ],
             ],
           })
           })
@@ -74,7 +74,7 @@ describe('Messages', () => {
               partition: 1,
               partition: 1,
               offset: 2,
               offset: 2,
               timestamp: new Date('05-05-1994'),
               timestamp: new Date('05-05-1994'),
-              content: [1, 2, 3],
+              content: '[1, 2, 3]',
             },
             },
           ];
           ];
           const content = JSON.stringify(messages[0].content);
           const content = JSON.stringify(messages[0].content);

+ 1 - 4
kafka-ui-react-app/src/components/Topics/Topic/Details/Messages/__test__/__snapshots__/MessageItem.spec.tsx.snap

@@ -38,10 +38,7 @@ exports[`MessageItem when content is defined matches snapshot 1`] = `
     }
     }
   >
   >
     <MessageContent
     <MessageContent
-      message="{
-	\\"foo\\": \\"bar\\",
-	\\"key\\": \\"val\\"
-}"
+      message="\\"{\\\\\\"foo\\\\\\":\\\\\\"bar\\\\\\",\\\\\\"key\\\\\\":\\\\\\"val\\\\\\"}\\""
     />
     />
   </td>
   </td>
   <td
   <td

+ 1 - 6
kafka-ui-react-app/src/components/Topics/Topic/Details/Messages/__test__/__snapshots__/MessagesTable.spec.tsx.snap

@@ -29,12 +29,7 @@ exports[`MessagesTable when topic contains messages matches snapshot 1`] = `
     </thead>
     </thead>
     <tbody>
     <tbody>
       <MessageItem
       <MessageItem
-        content={
-          Object {
-            "foo": "bar",
-            "key": "val",
-          }
-        }
+        content="{\\"foo\\":\\"bar\\",\\"key\\":\\"val\\"}"
         key="message-802310400000-2"
         key="message-802310400000-2"
         messageKey="1"
         messageKey="1"
         offset={2}
         offset={2}

+ 2 - 2
kafka-ui-react-app/src/components/Topics/Topic/Details/Messages/__test__/fixtures.ts

@@ -5,10 +5,10 @@ export const messages: TopicMessage[] = [
     partition: 1,
     partition: 1,
     offset: 2,
     offset: 2,
     timestamp: new Date(Date.UTC(1995, 5, 5)),
     timestamp: new Date(Date.UTC(1995, 5, 5)),
-    content: {
+    content: JSON.stringify({
       foo: 'bar',
       foo: 'bar',
       key: 'val',
       key: 'val',
-    },
+    }),
     key: '1',
     key: '1',
   },
   },
   {
   {