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>
This commit is contained in:
parent
63ba8890b5
commit
dd19cc2eec
29 changed files with 599 additions and 316 deletions
|
@ -2,14 +2,16 @@ package com.provectus.kafka.ui.serde;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
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 java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import javax.annotation.PostConstruct;
|
||||
import lombok.RequiredArgsConstructor;
|
||||
import lombok.extern.log4j.Log4j2;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Log4j2
|
||||
@Component
|
||||
@RequiredArgsConstructor
|
||||
public class DeserializationService {
|
||||
|
@ -31,10 +33,12 @@ public class DeserializationService {
|
|||
private RecordSerDe createRecordDeserializerForCluster(KafkaCluster cluster) {
|
||||
try {
|
||||
if (cluster.getProtobufFile() != null) {
|
||||
log.info("Using ProtobufFileRecordSerDe for cluster '{}'", cluster.getName());
|
||||
return new ProtobufFileRecordSerDe(cluster.getProtobufFile(),
|
||||
cluster.getProtobufMessageName(), objectMapper);
|
||||
} else {
|
||||
return new SchemaRegistryRecordSerDe(cluster, objectMapper);
|
||||
log.info("Using SchemaRegistryAwareRecordSerDe for cluster '{}'", cluster.getName());
|
||||
return new SchemaRegistryAwareRecordSerDe(cluster);
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
throw new RuntimeException("Can't init deserializer", e);
|
||||
|
|
|
@ -1,6 +1,5 @@
|
|||
package com.provectus.kafka.ui.serde;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.protobuf.DynamicMessage;
|
||||
import com.google.protobuf.util.JsonFormat;
|
||||
|
@ -14,16 +13,17 @@ import java.io.ByteArrayInputStream;
|
|||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import javax.annotation.Nullable;
|
||||
import lombok.SneakyThrows;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
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 {
|
||||
private final ProtobufSchema protobufSchema;
|
||||
private final ObjectMapper objectMapper;
|
||||
|
@ -42,33 +42,45 @@ public class ProtobufFileRecordSerDe implements RecordSerDe {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Tuple2<String, Object> deserialize(ConsumerRecord<Bytes, Bytes> msg) {
|
||||
public DeserializedKeyValue deserialize(ConsumerRecord<Bytes, Bytes> msg) {
|
||||
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) {
|
||||
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
|
||||
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();
|
||||
try {
|
||||
JsonFormat.parser().merge(new String(data), builder);
|
||||
JsonFormat.parser().merge(data, builder);
|
||||
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) {
|
||||
throw new RuntimeException("Failed to merge record for topic " + topic, e);
|
||||
}
|
||||
|
@ -95,9 +107,4 @@ public class ProtobufFileRecordSerDe implements RecordSerDe {
|
|||
.key(keySchema)
|
||||
.value(valueSchema);
|
||||
}
|
||||
|
||||
private Object parseJson(byte[] bytes) throws IOException {
|
||||
return objectMapper.readValue(bytes, new TypeReference<Map<String, Object>>() {
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,18 +1,26 @@
|
|||
package com.provectus.kafka.ui.serde;
|
||||
|
||||
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.producer.ProducerRecord;
|
||||
import org.apache.kafka.common.utils.Bytes;
|
||||
import reactor.util.function.Tuple2;
|
||||
|
||||
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);
|
||||
}
|
||||
|
|
|
@ -4,28 +4,32 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.provectus.kafka.ui.model.MessageSchema;
|
||||
import com.provectus.kafka.ui.model.TopicMessageSchema;
|
||||
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.producer.ProducerRecord;
|
||||
import org.apache.kafka.common.utils.Bytes;
|
||||
import reactor.util.function.Tuple2;
|
||||
import reactor.util.function.Tuples;
|
||||
|
||||
public class SimpleRecordSerDe implements RecordSerDe {
|
||||
|
||||
@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
|
||||
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
|
||||
|
|
|
@ -1,38 +1,23 @@
|
|||
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.client.SchemaRegistryClient;
|
||||
import io.confluent.kafka.serializers.KafkaAvroDeserializer;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import lombok.SneakyThrows;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
public class AvroMessageFormatter implements MessageFormatter {
|
||||
private final KafkaAvroDeserializer avroDeserializer;
|
||||
private final ObjectMapper objectMapper;
|
||||
|
||||
public AvroMessageFormatter(SchemaRegistryClient client, ObjectMapper objectMapper) {
|
||||
public AvroMessageFormatter(SchemaRegistryClient client) {
|
||||
this.avroDeserializer = new KafkaAvroDeserializer(client);
|
||||
this.objectMapper = objectMapper;
|
||||
}
|
||||
|
||||
@Override
|
||||
@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,8 +8,7 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
|
|||
import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
|
||||
import io.confluent.kafka.serializers.KafkaAvroSerializer;
|
||||
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;
|
||||
|
||||
public class AvroMessageReader extends MessageReader<Object> {
|
||||
|
@ -23,21 +22,18 @@ public class AvroMessageReader extends MessageReader<Object> {
|
|||
|
||||
@Override
|
||||
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
|
||||
protected Object read(byte[] value, ParsedSchema schema) {
|
||||
Schema rawSchema = ((AvroSchema) schema).rawSchema();
|
||||
|
||||
protected Object read(String value, ParsedSchema schema) {
|
||||
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) {
|
||||
throw new RuntimeException("Failed to merge record for topic " + topic, e);
|
||||
throw new RuntimeException("Failed to serialize record for topic " + topic, e);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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>>() {
|
||||
});
|
||||
}
|
||||
}
|
|
@ -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,5 +1,5 @@
|
|||
package com.provectus.kafka.ui.serde.schemaregistry;
|
||||
|
||||
public interface MessageFormatter {
|
||||
Object format(String topic, byte[] value);
|
||||
String format(String topic, byte[] value);
|
||||
}
|
||||
|
|
|
@ -23,10 +23,10 @@ public abstract class MessageReader<T> {
|
|||
|
||||
protected abstract Serializer<T> createSerializer(SchemaRegistryClient client);
|
||||
|
||||
public byte[] read(byte[] value) {
|
||||
public byte[] read(String value) {
|
||||
final T read = this.read(value, schema);
|
||||
return this.serializer.serialize(topic, read);
|
||||
}
|
||||
|
||||
protected abstract T read(byte[] value, ParsedSchema schema);
|
||||
protected abstract T read(String value, ParsedSchema schema);
|
||||
}
|
||||
|
|
|
@ -4,7 +4,6 @@ import com.google.protobuf.Message;
|
|||
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
|
||||
import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchemaUtils;
|
||||
import io.confluent.kafka.serializers.protobuf.KafkaProtobufDeserializer;
|
||||
import java.util.Map;
|
||||
import lombok.SneakyThrows;
|
||||
|
||||
public class ProtobufMessageFormatter implements MessageFormatter {
|
||||
|
@ -16,12 +15,9 @@ public class ProtobufMessageFormatter implements MessageFormatter {
|
|||
|
||||
@Override
|
||||
@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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -10,6 +10,7 @@ import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientExcept
|
|||
import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema;
|
||||
import io.confluent.kafka.serializers.protobuf.KafkaProtobufSerializer;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import org.apache.kafka.common.serialization.Serializer;
|
||||
|
||||
public class ProtobufMessageReader extends MessageReader<Message> {
|
||||
|
@ -22,18 +23,21 @@ public class ProtobufMessageReader extends MessageReader<Message> {
|
|||
|
||||
@Override
|
||||
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
|
||||
protected Message read(byte[] value, ParsedSchema schema) {
|
||||
protected Message read(String value, ParsedSchema schema) {
|
||||
ProtobufSchema protobufSchema = (ProtobufSchema) schema;
|
||||
DynamicMessage.Builder builder = protobufSchema.newMessageBuilder();
|
||||
try {
|
||||
JsonFormat.parser().merge(new String(value), builder);
|
||||
JsonFormat.parser().merge(value, builder);
|
||||
return builder.build();
|
||||
} 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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -15,6 +15,7 @@ import io.confluent.kafka.schemaregistry.avro.AvroSchemaProvider;
|
|||
import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient;
|
||||
import io.confluent.kafka.schemaregistry.client.SchemaMetadata;
|
||||
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.ProtobufSchemaProvider;
|
||||
import java.net.URI;
|
||||
|
@ -22,70 +23,76 @@ import java.nio.ByteBuffer;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import javax.annotation.Nullable;
|
||||
import lombok.SneakyThrows;
|
||||
import lombok.extern.log4j.Log4j2;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
import org.apache.kafka.common.utils.Bytes;
|
||||
import reactor.util.function.Tuple2;
|
||||
import reactor.util.function.Tuples;
|
||||
|
||||
@Log4j2
|
||||
public class SchemaRegistryRecordSerDe implements RecordSerDe {
|
||||
public class SchemaRegistryAwareRecordSerDe implements RecordSerDe {
|
||||
|
||||
private static final int CLIENT_IDENTITY_MAP_CAPACITY = 100;
|
||||
|
||||
private final KafkaCluster cluster;
|
||||
private final SchemaRegistryClient schemaRegistryClient;
|
||||
private final Map<String, MessageFormatter> valueFormatMap = 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 ProtobufSchemaConverter protoSchemaConverter = new ProtobufSchemaConverter();
|
||||
private final AvroJsonSchemaConverter avroSchemaConverter = new AvroJsonSchemaConverter();
|
||||
private final ObjectMapper objectMapper = new ObjectMapper();
|
||||
|
||||
public SchemaRegistryRecordSerDe(KafkaCluster cluster, ObjectMapper 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 = 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);
|
||||
|
||||
this.schemaRegistryClient = cluster.getSchemaRegistry() != null
|
||||
? createSchemaRegistryClient(cluster)
|
||||
: null;
|
||||
if (schemaRegistryClient != null) {
|
||||
this.avroFormatter = new AvroMessageFormatter(schemaRegistryClient, objectMapper);
|
||||
this.avroFormatter = new AvroMessageFormatter(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 {
|
||||
return Tuples.of(
|
||||
return new DeserializedKeyValue(
|
||||
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) {
|
||||
throw new RuntimeException("Failed to parse record from topic " + msg.topic(), e);
|
||||
|
@ -93,47 +100,43 @@ public class SchemaRegistryRecordSerDe implements RecordSerDe {
|
|||
}
|
||||
|
||||
@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> 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
|
||||
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()) {
|
||||
final SchemaMetadata schema = maybeSchema.get();
|
||||
|
||||
MessageReader<?> reader;
|
||||
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())) {
|
||||
reader = new AvroMessageReader(topic, false, schemaRegistryClient, schema);
|
||||
reader = new AvroMessageReader(topic, isKey, schemaRegistryClient, schema);
|
||||
} 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 {
|
||||
return Optional.empty();
|
||||
// if no schema provided serialize input as raw string
|
||||
return value.getBytes();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -171,7 +174,8 @@ public class SchemaRegistryRecordSerDe implements RecordSerDe {
|
|||
|
||||
String jsonSchema;
|
||||
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())) {
|
||||
final ProtobufSchema protobufSchema = (ProtobufSchema) schemaById;
|
||||
|
@ -207,38 +211,31 @@ public class SchemaRegistryRecordSerDe implements RecordSerDe {
|
|||
.or(() -> getSchemaBySubject(msg.topic(), isKey).map(SchemaMetadata::getSchemaType));
|
||||
if (type.isPresent()) {
|
||||
if (type.get().equals(MessageFormat.PROTOBUF.name())) {
|
||||
if (tryFormatter(protobufFormatter, msg).isPresent()) {
|
||||
if (tryFormatter(protobufFormatter, msg, isKey).isPresent()) {
|
||||
return protobufFormatter;
|
||||
}
|
||||
} else if (type.get().equals(MessageFormat.AVRO.name())) {
|
||||
if (tryFormatter(avroFormatter, msg).isPresent()) {
|
||||
if (tryFormatter(avroFormatter, msg, isKey).isPresent()) {
|
||||
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) {
|
||||
log.warn("Failed to get Schema for topic {}", msg.topic(), e);
|
||||
}
|
||||
}
|
||||
|
||||
if (tryFormatter(jsonFormatter, msg).isPresent()) {
|
||||
return jsonFormatter;
|
||||
}
|
||||
|
||||
return stringFormatter;
|
||||
}
|
||||
|
||||
private Optional<MessageFormatter> tryFormatter(
|
||||
MessageFormatter formatter, ConsumerRecord<Bytes, Bytes> msg) {
|
||||
MessageFormatter formatter, ConsumerRecord<Bytes, Bytes> msg, boolean isKey) {
|
||||
try {
|
||||
formatter.format(msg.topic(), msg.value().get());
|
||||
formatter.format(msg.topic(), isKey ? msg.key().get() : msg.value().get());
|
||||
return Optional.of(formatter);
|
||||
} 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();
|
||||
|
@ -252,9 +249,10 @@ public class SchemaRegistryRecordSerDe implements RecordSerDe {
|
|||
ByteBuffer buffer = ByteBuffer.wrap(value.get());
|
||||
if (buffer.get() == 0) {
|
||||
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;
|
||||
|
@ -262,11 +260,23 @@ public class SchemaRegistryRecordSerDe implements RecordSerDe {
|
|||
|
||||
@SneakyThrows
|
||||
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) {
|
|
@ -1,18 +1,11 @@
|
|||
package com.provectus.kafka.ui.serde.schemaregistry;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Map;
|
||||
import lombok.SneakyThrows;
|
||||
|
||||
public class StringMessageFormatter implements MessageFormatter {
|
||||
|
||||
@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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -4,6 +4,7 @@ import com.provectus.kafka.ui.exception.ClusterNotFoundException;
|
|||
import com.provectus.kafka.ui.exception.IllegalEntityStateException;
|
||||
import com.provectus.kafka.ui.exception.NotFoundException;
|
||||
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.model.Broker;
|
||||
import com.provectus.kafka.ui.model.BrokerMetrics;
|
||||
|
@ -351,6 +352,13 @@ public class ClusterService {
|
|||
if (!cluster.getTopics().containsKey(topicName)) {
|
||||
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();
|
||||
}
|
||||
|
||||
|
|
|
@ -17,13 +17,15 @@ public class ClustersMetricsScheduler {
|
|||
|
||||
private final MetricsUpdateService metricsUpdateService;
|
||||
|
||||
@Scheduled(fixedRate = 30000)
|
||||
@Scheduled(fixedRateString = "${kafka.update-metrics-rate-millis:30000}")
|
||||
public void updateMetrics() {
|
||||
Flux.fromIterable(clustersStorage.getKafkaClustersMap().entrySet())
|
||||
.subscribeOn(Schedulers.parallel())
|
||||
.parallel()
|
||||
.runOn(Schedulers.parallel())
|
||||
.map(Map.Entry::getValue)
|
||||
.flatMap(metricsUpdateService::updateMetrics)
|
||||
.doOnNext(s -> clustersStorage.setKafkaCluster(s.getName(), s))
|
||||
.subscribe();
|
||||
.then()
|
||||
.block();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,6 +1,5 @@
|
|||
package com.provectus.kafka.ui.service;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.provectus.kafka.ui.emitter.BackwardRecordEmitter;
|
||||
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.OffsetsSeekForward;
|
||||
import java.util.Collection;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
@ -108,31 +106,8 @@ public class ConsumingService {
|
|||
if (StringUtils.isEmpty(query)) {
|
||||
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));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -68,6 +68,7 @@ import org.apache.kafka.clients.producer.RecordMetadata;
|
|||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
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.utils.Bytes;
|
||||
import org.springframework.beans.factory.annotation.Value;
|
||||
|
@ -672,11 +673,14 @@ public class KafkaService {
|
|||
Properties properties = new Properties();
|
||||
properties.putAll(cluster.getProperties());
|
||||
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)) {
|
||||
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<>();
|
||||
|
|
|
@ -215,9 +215,9 @@ public class ClusterUtil {
|
|||
topicMessage.setTimestampType(timestampType);
|
||||
|
||||
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;
|
||||
}
|
||||
|
|
|
@ -56,14 +56,14 @@ public abstract class AbstractBaseTest {
|
|||
public void initialize(@NotNull ConfigurableApplicationContext context) {
|
||||
System.setProperty("kafka.clusters.0.name", LOCAL);
|
||||
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.address", kafkaConnect.getTarget());
|
||||
|
||||
System.setProperty("kafka.clusters.1.name", SECOND_LOCAL);
|
||||
System.setProperty("kafka.clusters.1.readOnly", "true");
|
||||
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.address", kafkaConnect.getTarget());
|
||||
}
|
||||
|
|
|
@ -1,5 +1,7 @@
|
|||
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.KafkaContainer;
|
||||
import org.testcontainers.containers.Network;
|
||||
|
@ -25,7 +27,12 @@ public class SchemaRegistryContainer extends GenericContainer<SchemaRegistryCont
|
|||
return self();
|
||||
}
|
||||
|
||||
public String getTarget() {
|
||||
public String getUrl() {
|
||||
return "http://" + getContainerIpAddress() + ":" + getMappedPort(SCHEMA_PORT);
|
||||
}
|
||||
|
||||
public SchemaRegistryClient schemaRegistryClient() {
|
||||
return new CachedSchemaRegistryClient(getUrl(), 1000);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,24 +1,22 @@
|
|||
package com.provectus.kafka.ui.serde;
|
||||
|
||||
import static com.provectus.kafka.ui.serde.RecordSerDe.DeserializedKeyValue;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
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.common.utils.Bytes;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import reactor.util.function.Tuples;
|
||||
|
||||
class SchemaRegistryRecordDeserializerTest {
|
||||
|
||||
private final SchemaRegistryRecordSerDe deserializer =
|
||||
new SchemaRegistryRecordSerDe(
|
||||
private final SchemaRegistryAwareRecordSerDe deserializer =
|
||||
new SchemaRegistryAwareRecordSerDe(
|
||||
KafkaCluster.builder()
|
||||
.schemaNameTemplate("%s-value")
|
||||
.build(),
|
||||
new ObjectMapper()
|
||||
.build()
|
||||
);
|
||||
|
||||
@Test
|
||||
|
@ -27,13 +25,13 @@ class SchemaRegistryRecordDeserializerTest {
|
|||
var deserializedRecord = deserializer.deserialize(
|
||||
new ConsumerRecord<>("topic", 1, 0, Bytes.wrap("key".getBytes()),
|
||||
Bytes.wrap(value.getBytes())));
|
||||
assertEquals(Tuples.of("key", value), deserializedRecord);
|
||||
assertEquals(new DeserializedKeyValue("key", value), deserializedRecord);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldDeserializeNullValueRecordToEmptyMap() {
|
||||
var deserializedRecord = deserializer
|
||||
.deserialize(new ConsumerRecord<>("topic", 1, 0, Bytes.wrap("key".getBytes()), null));
|
||||
assertEquals(Tuples.of("key", Map.of()), deserializedRecord);
|
||||
assertEquals(new DeserializedKeyValue("key", null), deserializedRecord);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -1615,9 +1615,7 @@ components:
|
|||
additionalProperties:
|
||||
type: string
|
||||
content:
|
||||
type: object
|
||||
required:
|
||||
- content
|
||||
type: string
|
||||
|
||||
TopicMessageSchema:
|
||||
type: object
|
||||
|
@ -1672,7 +1670,7 @@ components:
|
|||
additionalProperties:
|
||||
type: string
|
||||
content:
|
||||
type: object
|
||||
type: string
|
||||
required:
|
||||
- partition
|
||||
- offset
|
||||
|
|
|
@ -24,7 +24,7 @@ const MessageItem: React.FC<MessageItemProp> = ({
|
|||
}) => {
|
||||
const { copyToClipboard, saveFile } = useDataSaver(
|
||||
'topic-message',
|
||||
(content as Record<string, string>) || ''
|
||||
content || ''
|
||||
);
|
||||
return (
|
||||
<tr>
|
||||
|
|
|
@ -57,7 +57,7 @@ describe('Messages', () => {
|
|||
partition: 1,
|
||||
offset: 2,
|
||||
timestamp: new Date('05-05-1994'),
|
||||
content: [1, 2, 3],
|
||||
content: '[1, 2, 3]',
|
||||
},
|
||||
],
|
||||
})
|
||||
|
@ -74,7 +74,7 @@ describe('Messages', () => {
|
|||
partition: 1,
|
||||
offset: 2,
|
||||
timestamp: new Date('05-05-1994'),
|
||||
content: [1, 2, 3],
|
||||
content: '[1, 2, 3]',
|
||||
},
|
||||
];
|
||||
const content = JSON.stringify(messages[0].content);
|
||||
|
|
|
@ -38,10 +38,7 @@ exports[`MessageItem when content is defined matches snapshot 1`] = `
|
|||
}
|
||||
>
|
||||
<MessageContent
|
||||
message="{
|
||||
\\"foo\\": \\"bar\\",
|
||||
\\"key\\": \\"val\\"
|
||||
}"
|
||||
message="\\"{\\\\\\"foo\\\\\\":\\\\\\"bar\\\\\\",\\\\\\"key\\\\\\":\\\\\\"val\\\\\\"}\\""
|
||||
/>
|
||||
</td>
|
||||
<td
|
||||
|
|
|
@ -29,12 +29,7 @@ exports[`MessagesTable when topic contains messages matches snapshot 1`] = `
|
|||
</thead>
|
||||
<tbody>
|
||||
<MessageItem
|
||||
content={
|
||||
Object {
|
||||
"foo": "bar",
|
||||
"key": "val",
|
||||
}
|
||||
}
|
||||
content="{\\"foo\\":\\"bar\\",\\"key\\":\\"val\\"}"
|
||||
key="message-802310400000-2"
|
||||
messageKey="1"
|
||||
offset={2}
|
||||
|
|
|
@ -5,10 +5,10 @@ export const messages: TopicMessage[] = [
|
|||
partition: 1,
|
||||
offset: 2,
|
||||
timestamp: new Date(Date.UTC(1995, 5, 5)),
|
||||
content: {
|
||||
content: JSON.stringify({
|
||||
foo: 'bar',
|
||||
key: 'val',
|
||||
},
|
||||
}),
|
||||
key: '1',
|
||||
},
|
||||
{
|
||||
|
|
Loading…
Add table
Reference in a new issue