浏览代码

Proto to JsonSchema well-known types support (#3088)

1. Add support for predefined (well-known) protobuf types.
2. `preservingProtoFieldNames` mode is enabled for proto messages formatting to fit proto file's message's field names
3. Invalid oneof-processing logic removed
4. Protobuf -> JsonSchema self-reference issue fixed
5. format added to Timestamp protobuf type
Ilya Kuramshin 2 年之前
父节点
当前提交
7c6d04bca0

+ 6 - 3
kafka-ui-api/src/main/java/com/provectus/kafka/ui/serdes/builtin/sr/MessageFormatter.java

@@ -2,9 +2,9 @@ package com.provectus.kafka.ui.serdes.builtin.sr;
 
 import com.fasterxml.jackson.databind.JsonNode;
 import com.google.protobuf.Message;
+import com.google.protobuf.util.JsonFormat;
 import io.confluent.kafka.schemaregistry.avro.AvroSchemaUtils;
 import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
-import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchemaUtils;
 import io.confluent.kafka.serializers.KafkaAvroDeserializer;
 import io.confluent.kafka.serializers.json.KafkaJsonSchemaDeserializer;
 import io.confluent.kafka.serializers.protobuf.KafkaProtobufDeserializer;
@@ -52,8 +52,11 @@ interface MessageFormatter {
     @SneakyThrows
     public String format(String topic, byte[] value) {
       final Message message = protobufDeserializer.deserialize(topic, value);
-      byte[] jsonBytes = ProtobufSchemaUtils.toJson(message);
-      return new String(jsonBytes);
+      return JsonFormat.printer()
+          .includingDefaultValueFields()
+          .omittingInsignificantWhitespace()
+          .preservingProtoFieldNames()
+          .print(message);
     }
   }
 

+ 27 - 0
kafka-ui-api/src/main/java/com/provectus/kafka/ui/util/jsonschema/AnyFieldSchema.java

@@ -0,0 +1,27 @@
+package com.provectus.kafka.ui.util.jsonschema;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+// Specifies field that can contain any kind of value - primitive, complex and nulls
+public class AnyFieldSchema implements FieldSchema {
+
+  public static AnyFieldSchema get() {
+    return new AnyFieldSchema();
+  }
+
+  private AnyFieldSchema() {
+  }
+
+  @Override
+  public JsonNode toJsonNode(ObjectMapper mapper) {
+    var arr = mapper.createArrayNode();
+    arr.add("number");
+    arr.add("string");
+    arr.add("object");
+    arr.add("array");
+    arr.add("boolean");
+    arr.add("null");
+    return mapper.createObjectNode().set("type", arr);
+  }
+}

+ 1 - 1
kafka-ui-api/src/main/java/com/provectus/kafka/ui/util/jsonschema/AvroJsonSchemaConverter.java

@@ -110,7 +110,7 @@ public class AvroJsonSchemaConverter implements JsonSchemaConverter<Schema> {
       return createRefField(definitionName);
     }
     // adding stub record, need to avoid infinite recursion
-    definitions.put(definitionName, new ObjectFieldSchema(Map.of(), List.of()));
+    definitions.put(definitionName, ObjectFieldSchema.EMPTY);
 
     final Map<String, FieldSchema> fields = schema.getFields().stream()
         .map(f -> Tuples.of(f.name(), convertField(f, definitions)))

+ 4 - 0
kafka-ui-api/src/main/java/com/provectus/kafka/ui/util/jsonschema/JsonSchema.java

@@ -23,6 +23,7 @@ public class JsonSchema {
   private final Map<String, FieldSchema> properties;
   private final Map<String, FieldSchema> definitions;
   private final List<String> required;
+  private final String rootRef;
 
   public String toJson() {
     final ObjectMapper mapper = new ObjectMapper();
@@ -53,6 +54,9 @@ public class JsonSchema {
               ))
       ));
     }
+    if (rootRef != null) {
+      objectNode.set("$ref", new TextNode(rootRef));
+    }
     return objectNode.toString();
   }
 

+ 3 - 0
kafka-ui-api/src/main/java/com/provectus/kafka/ui/util/jsonschema/ObjectFieldSchema.java

@@ -10,6 +10,9 @@ import reactor.util.function.Tuple2;
 import reactor.util.function.Tuples;
 
 public class ObjectFieldSchema implements FieldSchema {
+
+  public static final ObjectFieldSchema EMPTY = new ObjectFieldSchema(Map.of(), List.of());
+
   private final Map<String, FieldSchema> properties;
   private final List<String> required;
 

+ 146 - 65
kafka-ui-api/src/main/java/com/provectus/kafka/ui/util/jsonschema/ProtobufSchemaConverter.java

@@ -1,90 +1,103 @@
 package com.provectus.kafka.ui.util.jsonschema;
 
+import static java.util.Objects.requireNonNull;
+
+import com.fasterxml.jackson.databind.node.BigIntegerNode;
+import com.fasterxml.jackson.databind.node.IntNode;
+import com.fasterxml.jackson.databind.node.LongNode;
+import com.fasterxml.jackson.databind.node.TextNode;
+import com.google.common.primitives.UnsignedInteger;
+import com.google.common.primitives.UnsignedLong;
+import com.google.protobuf.Any;
+import com.google.protobuf.BoolValue;
+import com.google.protobuf.BytesValue;
 import com.google.protobuf.Descriptors;
+import com.google.protobuf.DoubleValue;
+import com.google.protobuf.Duration;
+import com.google.protobuf.FieldMask;
+import com.google.protobuf.FloatValue;
+import com.google.protobuf.Int32Value;
+import com.google.protobuf.Int64Value;
+import com.google.protobuf.ListValue;
+import com.google.protobuf.StringValue;
+import com.google.protobuf.Struct;
+import com.google.protobuf.Timestamp;
+import com.google.protobuf.UInt32Value;
+import com.google.protobuf.UInt64Value;
+import com.google.protobuf.Value;
 import java.net.URI;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
 import java.util.stream.Collectors;
 import reactor.util.function.Tuple2;
 import reactor.util.function.Tuples;
 
 public class ProtobufSchemaConverter implements JsonSchemaConverter<Descriptors.Descriptor> {
-  @Override
-  public JsonSchema convert(URI basePath, Descriptors.Descriptor schema) {
-    final JsonSchema.JsonSchemaBuilder builder = JsonSchema.builder();
 
-    builder.id(basePath.resolve(schema.getFullName()));
-    builder.type(new SimpleJsonType(JsonType.Type.OBJECT));
+  private final Set<String> simpleTypesWrapperNames = Set.of(
+      BoolValue.getDescriptor().getFullName(),
+      Int32Value.getDescriptor().getFullName(),
+      UInt32Value.getDescriptor().getFullName(),
+      Int64Value.getDescriptor().getFullName(),
+      UInt64Value.getDescriptor().getFullName(),
+      StringValue.getDescriptor().getFullName(),
+      BytesValue.getDescriptor().getFullName(),
+      FloatValue.getDescriptor().getFullName(),
+      DoubleValue.getDescriptor().getFullName()
+  );
 
+  @Override
+  public JsonSchema convert(URI basePath, Descriptors.Descriptor schema) {
     Map<String, FieldSchema> definitions = new HashMap<>();
-    final ObjectFieldSchema root =
-        (ObjectFieldSchema) convertObjectSchema(schema, definitions, false);
-    builder.definitions(definitions);
-
-    builder.properties(root.getProperties());
-    builder.required(root.getRequired());
-
-    return builder.build();
+    RefFieldSchema rootRef = registerObjectAndReturnRef(schema, definitions);
+    return JsonSchema.builder()
+        .id(basePath.resolve(schema.getFullName()))
+        .type(new SimpleJsonType(JsonType.Type.OBJECT))
+        .rootRef(rootRef.getRef())
+        .definitions(definitions)
+        .build();
   }
 
-  private FieldSchema convertObjectSchema(Descriptors.Descriptor schema,
-                                          Map<String, FieldSchema> definitions, boolean ref) {
-    final Map<String, FieldSchema> fields = schema.getFields().stream()
-        .map(f -> Tuples.of(f.getName(), convertField(f, definitions)))
-        .collect(Collectors.toMap(
-            Tuple2::getT1,
-            Tuple2::getT2
-        ));
-
-    final Map<String, OneOfFieldSchema> oneOfFields = schema.getOneofs().stream().map(o ->
-        Tuples.of(
-            o.getName(),
-            new OneOfFieldSchema(
-                o.getFields().stream().map(
-                    Descriptors.FieldDescriptor::getName
-                ).map(fields::get).collect(Collectors.toList())
-            )
-        )
-    ).collect(Collectors.toMap(
-        Tuple2::getT1,
-        Tuple2::getT2
-    ));
-
-    final List<String> allOneOfFields = schema.getOneofs().stream().flatMap(o ->
-        o.getFields().stream().map(Descriptors.FieldDescriptor::getName)
-    ).collect(Collectors.toList());
+  private RefFieldSchema registerObjectAndReturnRef(Descriptors.Descriptor schema,
+                                                    Map<String, FieldSchema> definitions) {
+    var definition = schema.getFullName();
+    if (definitions.containsKey(definition)) {
+      return createRefField(definition);
+    }
+    // adding stub record, need to avoid infinite recursion
+    definitions.put(definition, ObjectFieldSchema.EMPTY);
 
-    final Map<String, FieldSchema> excludedOneOf = fields.entrySet().stream()
-        .filter(f -> !allOneOfFields.contains(f.getKey()))
-        .collect(Collectors.toMap(
-            Map.Entry::getKey,
-            Map.Entry::getValue
-        ));
+    Map<String, FieldSchema> fields = schema.getFields().stream()
+        .map(f -> Tuples.of(f.getName(), convertField(f, definitions)))
+        .collect(Collectors.toMap(Tuple2::getT1, Tuple2::getT2));
 
-    Map<String, FieldSchema> finalFields = new HashMap<>(excludedOneOf);
-    finalFields.putAll(oneOfFields);
+    List<String> required = schema.getFields().stream()
+        .filter(Descriptors.FieldDescriptor::isRequired)
+        .map(Descriptors.FieldDescriptor::getName)
+        .collect(Collectors.toList());
 
-    final List<String> required = schema.getFields().stream()
-        .filter(f -> !f.isOptional())
-        .map(Descriptors.FieldDescriptor::getName).collect(Collectors.toList());
+    // replacing stub record with actual object structure
+    definitions.put(definition, new ObjectFieldSchema(fields, required));
+    return createRefField(definition);
+  }
 
-    if (ref) {
-      String definitionName = String.format("record.%s", schema.getFullName());
-      definitions.put(definitionName, new ObjectFieldSchema(finalFields, required));
-      return new RefFieldSchema(String.format("#/definitions/%s", definitionName));
-    } else {
-      return new ObjectFieldSchema(fields, required);
-    }
+  private RefFieldSchema createRefField(String definition) {
+    return new RefFieldSchema("#/definitions/%s".formatted(definition));
   }
 
   private FieldSchema convertField(Descriptors.FieldDescriptor field,
                                    Map<String, FieldSchema> definitions) {
+    Optional<FieldSchema> wellKnownTypeSchema = convertProtoWellKnownTypes(field);
+    if (wellKnownTypeSchema.isPresent()) {
+      return wellKnownTypeSchema.get();
+    }
     final JsonType jsonType = convertType(field);
-
     FieldSchema fieldSchema;
     if (jsonType.getType().equals(JsonType.Type.OBJECT)) {
-      fieldSchema = convertObjectSchema(field.getMessageType(), definitions, true);
+      fieldSchema = registerObjectAndReturnRef(field.getMessageType(), definitions);
     } else {
       fieldSchema = new SimpleFieldSchema(jsonType);
     }
@@ -96,20 +109,88 @@ public class ProtobufSchemaConverter implements JsonSchemaConverter<Descriptors.
     }
   }
 
+  // converts Protobuf Well-known type (from google.protobuf.* packages) to Json-schema types
+  // see JsonFormat::buildWellKnownTypePrinters for impl details
+  private Optional<FieldSchema> convertProtoWellKnownTypes(Descriptors.FieldDescriptor field) {
+    // all well-known types are messages
+    if (field.getType() != Descriptors.FieldDescriptor.Type.MESSAGE) {
+      return Optional.empty();
+    }
+    String typeName = field.getMessageType().getFullName();
+    if (typeName.equals(Timestamp.getDescriptor().getFullName())) {
+      return Optional.of(
+          new SimpleFieldSchema(
+              new SimpleJsonType(JsonType.Type.STRING, Map.of("format", new TextNode("date-time")))));
+    }
+    if (typeName.equals(Duration.getDescriptor().getFullName())) {
+      return Optional.of(
+          new SimpleFieldSchema(
+              //TODO: current UI is failing when format=duration is set - need to fix this first
+              new SimpleJsonType(JsonType.Type.STRING // , Map.of("format", new TextNode("duration"))
+              )));
+    }
+    if (typeName.equals(FieldMask.getDescriptor().getFullName())) {
+      return Optional.of(new SimpleFieldSchema(new SimpleJsonType(JsonType.Type.STRING)));
+    }
+    if (typeName.equals(Any.getDescriptor().getFullName()) || typeName.equals(Struct.getDescriptor().getFullName())) {
+      return Optional.of(ObjectFieldSchema.EMPTY);
+    }
+    if (typeName.equals(Value.getDescriptor().getFullName())) {
+      return Optional.of(AnyFieldSchema.get());
+    }
+    if (typeName.equals(ListValue.getDescriptor().getFullName())) {
+      return Optional.of(new ArrayFieldSchema(AnyFieldSchema.get()));
+    }
+    if (simpleTypesWrapperNames.contains(typeName)) {
+      return Optional.of(new SimpleFieldSchema(
+          convertType(requireNonNull(field.getMessageType().findFieldByName("value")))));
+    }
+    return Optional.empty();
+  }
 
   private JsonType convertType(Descriptors.FieldDescriptor field) {
     switch (field.getType()) {
       case INT32:
-      case INT64:
+      case FIXED32:
+      case SFIXED32:
       case SINT32:
-      case SINT64:
+        return new SimpleJsonType(
+            JsonType.Type.INTEGER,
+            Map.of(
+                "maximum", IntNode.valueOf(Integer.MAX_VALUE),
+                "minimum", IntNode.valueOf(Integer.MIN_VALUE)
+            )
+        );
       case UINT32:
-      case UINT64:
-      case FIXED32:
+        return new SimpleJsonType(
+            JsonType.Type.INTEGER,
+            Map.of(
+                "maximum", LongNode.valueOf(UnsignedInteger.MAX_VALUE.longValue()),
+                "minimum", IntNode.valueOf(0)
+            )
+        );
+      //TODO: actually all *64 types will be printed with quotes (as strings),
+      // see JsonFormat::printSingleFieldValue for impl. This can cause problems when you copy-paste from messages
+      // table to `Produce` area - need to think if it is critical or not.
+      case INT64:
       case FIXED64:
-      case SFIXED32:
       case SFIXED64:
-        return new SimpleJsonType(JsonType.Type.INTEGER);
+      case SINT64:
+        return new SimpleJsonType(
+            JsonType.Type.INTEGER,
+            Map.of(
+                "maximum", LongNode.valueOf(Long.MAX_VALUE),
+                "minimum", LongNode.valueOf(Long.MIN_VALUE)
+            )
+        );
+      case UINT64:
+        return new SimpleJsonType(
+            JsonType.Type.INTEGER,
+            Map.of(
+                "maximum", new BigIntegerNode(UnsignedLong.MAX_VALUE.bigIntegerValue()),
+                "minimum", LongNode.valueOf(0)
+            )
+        );
       case MESSAGE:
       case GROUP:
         return new SimpleJsonType(JsonType.Type.OBJECT);

+ 4 - 0
kafka-ui-api/src/main/java/com/provectus/kafka/ui/util/jsonschema/RefFieldSchema.java

@@ -15,4 +15,8 @@ public class RefFieldSchema implements FieldSchema {
   public JsonNode toJsonNode(ObjectMapper mapper) {
     return mapper.createObjectNode().set("$ref", new TextNode(ref));
   }
+
+  public String getRef() {
+    return ref;
+  }
 }

+ 12 - 4
kafka-ui-api/src/main/java/com/provectus/kafka/ui/util/jsonschema/SimpleJsonType.java

@@ -3,19 +3,27 @@ package com.provectus.kafka.ui.util.jsonschema;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.TextNode;
+import com.google.common.collect.ImmutableMap;
 import java.util.Map;
 
 public class SimpleJsonType extends JsonType {
 
+  private final Map<String, JsonNode> additionalTypeProperties;
+
   public SimpleJsonType(Type type) {
+    this(type, Map.of());
+  }
+
+  public SimpleJsonType(Type type, Map<String, JsonNode> additionalTypeProperties) {
     super(type);
+    this.additionalTypeProperties = additionalTypeProperties;
   }
 
   @Override
   public Map<String, JsonNode> toJsonNode(ObjectMapper mapper) {
-    return Map.of(
-        "type",
-        new TextNode(type.getName())
-    );
+    return ImmutableMap.<String, JsonNode>builder()
+        .put("type", new TextNode(type.getName()))
+        .putAll(additionalTypeProperties)
+        .build();
   }
 }

+ 122 - 54
kafka-ui-api/src/test/java/com/provectus/kafka/ui/util/jsonschema/ProtobufSchemaConverterTest.java

@@ -1,70 +1,138 @@
 package com.provectus.kafka.ui.util.jsonschema;
 
-import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema;
 import java.net.URI;
-import java.net.URISyntaxException;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
 
-public class ProtobufSchemaConverterTest {
+class ProtobufSchemaConverterTest {
 
   @Test
-  public void testSimpleProto() throws URISyntaxException, JsonProcessingException {
-
-    String proto = "syntax = \"proto3\";\n"
-        + "package com.acme;\n"
-        + "\n"
-        + "message MyRecord {\n"
-        + "  string f1 = 1;\n"
-        + "  OtherRecord f2 = 2;\n"
-        + "  repeated OtherRecord f3 = 3;\n"
-        + "}\n"
-        + "\n"
-        + "message OtherRecord {\n"
-        + "  int32 other_id = 1;\n"
-        + "  Order order = 2;\n"
-        + "  oneof optionalField {"
-        + "    string name = 3;"
-        + "    uint64 size = 4;"
-        + "  }"
-        + "}\n"
-        + "\n"
-        + "enum Order {\n"
-        + "    FIRST = 1;\n"
-        + "    SECOND = 1;\n"
-        + "}\n";
-
-    String expected =
-        "{\"$id\":\"http://example.com/com.acme.MyRecord\","
-        + "\"$schema\":\"https://json-schema.org/draft/2020-12/schema\","
-        + "\"type\":\"object\",\"properties\":{\"f1\":{\"type\":\"string\"},"
-        + "\"f2\":{\"$ref\":\"#/definitions/record.com.acme.OtherRecord\"},"
-        + "\"f3\":{\"type\":\"array\","
-        + "\"items\":{\"$ref\":\"#/definitions/record.com.acme.OtherRecord\"}}},"
-        + "\"required\":[\"f3\"],"
-        + "\"definitions\":"
-        + "{\"record.com.acme.OtherRecord\":"
-        + "{\"type\":\"object\",\"properties\":"
-        + "{\"optionalField\":{\"oneOf\":[{\"type\":\"string\"},"
-        + "{\"type\":\"integer\"}]},\"other_id\":"
-        + "{\"type\":\"integer\"},\"order\":{\"enum\":[\"FIRST\",\"SECOND\"],"
-        + "\"type\":\"string\"}}}}}";
-
-    ProtobufSchema protobufSchema = new ProtobufSchema(proto);
-
-    final ProtobufSchemaConverter converter = new ProtobufSchemaConverter();
+  void testSchemaConvert() throws Exception {
+    String protoSchema = """
+        syntax = "proto3";
+        package test;
+
+        import "google/protobuf/timestamp.proto";
+        import "google/protobuf/duration.proto";
+        import "google/protobuf/struct.proto";
+        import "google/protobuf/wrappers.proto";
+
+        message TestMsg {
+            string string_field = 1;
+            int32 int32_field = 2;
+            bool bool_field = 3;
+            SampleEnum enum_field = 4;
+
+            enum SampleEnum {
+                ENUM_V1 = 0;
+                ENUM_V2 = 1;
+            }
+
+            google.protobuf.Timestamp ts_field = 5;
+            google.protobuf.Struct struct_field = 6;
+            google.protobuf.ListValue lst_v_field = 7;
+            google.protobuf.Duration duration_field = 8;
+
+            oneof some_oneof1 {
+                google.protobuf.Value v1 = 9;
+                google.protobuf.Value v2 = 10;
+            }
+            // wrapper fields:
+            google.protobuf.Int64Value int64_w_field = 11;
+            google.protobuf.Int32Value int32_w_field = 12;
+            google.protobuf.UInt64Value uint64_w_field = 13;
+            google.protobuf.UInt32Value uint32_w_field = 14;
+            google.protobuf.StringValue string_w_field = 15;
+            google.protobuf.BoolValue bool_w_field = 16;
+            google.protobuf.DoubleValue double_w_field = 17;
+            google.protobuf.FloatValue float_w_field = 18;
+
+            //embedded msg
+            EmbeddedMsg emb = 19;
+            repeated EmbeddedMsg emb_list = 20;
+
+            message EmbeddedMsg {
+                int32 emb_f1 = 1;
+                TestMsg outer_ref = 2;
+            }
+        }""";
+
+
+    String expectedJsonSchema = """
+        {
+            "$id": "http://example.com/test.TestMsg",
+            "$schema": "https://json-schema.org/draft/2020-12/schema",
+            "type": "object",
+            "definitions":
+            {
+                "test.TestMsg":
+                {
+                    "type": "object",
+                    "properties":
+                    {
+                        "enum_field": {
+                            "enum":
+                            [
+                                "ENUM_V1",
+                                "ENUM_V2"
+                            ],
+                            "type": "string"
+                        },
+                        "string_w_field": { "type": "string" },
+                        "ts_field": { "type": "string", "format": "date-time" },
+                        "emb_list": {
+                            "type": "array",
+                            "items": { "$ref": "#/definitions/test.TestMsg.EmbeddedMsg" }
+                        },
+                        "float_w_field": { "type": "number" },
+                        "lst_v_field": {
+                            "type": "array",
+                            "items": { "type":[ "number", "string", "object", "array", "boolean", "null" ] }
+                        },
+                        "struct_field": { "type": "object", "properties": {} },
+                        "string_field": { "type": "string" },
+                        "double_w_field": { "type": "number" },
+                        "bool_field": { "type": "boolean" },
+                        "int32_w_field": { "type": "integer", "maximum": 2147483647, "minimum": -2147483648 },
+                        "duration_field": { "type": "string" },
+                        "int32_field": { "type": "integer", "maximum": 2147483647, "minimum": -2147483648 },
+                        "int64_w_field": {
+                            "type": "integer",
+                            "maximum": 9223372036854775807, "minimum": -9223372036854775808
+                        },
+                        "v1": { "type": [ "number", "string", "object", "array", "boolean", "null" ] },
+                        "emb": { "$ref": "#/definitions/test.TestMsg.EmbeddedMsg" },
+                        "v2": { "type": [ "number", "string", "object", "array", "boolean", "null" ] },
+                        "uint32_w_field": { "type": "integer", "maximum": 4294967295, "minimum": 0 },
+                        "bool_w_field": { "type": "boolean" },
+                        "uint64_w_field": { "type": "integer", "maximum": 18446744073709551615, "minimum": 0 }
+                    }
+                },
+                "test.TestMsg.EmbeddedMsg": {
+                    "type": "object",
+                    "properties":
+                    {
+                        "emb_f1": { "type": "integer", "maximum": 2147483647, "minimum": -2147483648 },
+                        "outer_ref": { "$ref": "#/definitions/test.TestMsg" }
+                    }
+                }
+            },
+            "$ref": "#/definitions/test.TestMsg"
+        }""";
+
+    ProtobufSchemaConverter converter = new ProtobufSchemaConverter();
+    ProtobufSchema protobufSchema = new ProtobufSchema(protoSchema);
     URI basePath = new URI("http://example.com/");
 
-    final JsonSchema convert =
-        converter.convert(basePath, protobufSchema.toDescriptor("MyRecord"));
+    JsonSchema converted = converter.convert(basePath, protobufSchema.toDescriptor());
+    assertJsonEqual(expectedJsonSchema, converted.toJson());
+  }
 
+  private void assertJsonEqual(String expected, String actual) throws Exception {
     ObjectMapper om = new ObjectMapper();
-    Assertions.assertEquals(
-        om.readTree(expected),
-        om.readTree(convert.toJson())
-    );
+    Assertions.assertEquals(om.readTree(expected), om.readTree(actual));
   }
-}
+}

+ 3 - 0
kafka-ui-react-app/package.json

@@ -19,6 +19,9 @@
     "@types/testing-library__jest-dom": "^5.14.5",
     "ace-builds": "^1.7.1",
     "ajv": "^8.6.3",
+    "ajv-formats": "^2.1.1",
+    "babel-jest": "^29.0.3",
+    "classnames": "^2.2.6",
     "fetch-mock": "^9.11.0",
     "jest": "^29.0.3",
     "jest-watch-typeahead": "^2.0.0",

+ 84 - 10
kafka-ui-react-app/pnpm-lock.yaml

@@ -38,6 +38,9 @@ specifiers:
   '@vitejs/plugin-react-swc': ^3.0.0
   ace-builds: ^1.7.1
   ajv: ^8.6.3
+  ajv-formats: ^2.1.1
+  babel-jest: ^29.0.3
+  classnames: ^2.2.6
   dotenv: ^16.0.1
   eslint: ^8.3.0
   eslint-config-airbnb: ^19.0.4
@@ -106,6 +109,9 @@ dependencies:
   '@types/testing-library__jest-dom': 5.14.5
   ace-builds: 1.7.1
   ajv: 8.8.2
+  ajv-formats: 2.1.1_ajv@8.8.2
+  babel-jest: 29.0.3_@babel+core@7.18.2
+  classnames: 2.3.1
   fetch-mock: 9.11.0
   jest: 29.0.3_yqiaopbgmqcuvx27p5xxvum6wm
   jest-watch-typeahead: 2.0.0_jest@29.0.3
@@ -1084,7 +1090,6 @@ packages:
     dependencies:
       '@babel/core': 7.18.2
       '@babel/helper-plugin-utils': 7.18.6
-    dev: true
 
   /@babel/plugin-syntax-async-generators/7.8.4_@babel+core@7.18.9:
     resolution: {integrity: sha512-tycmZxkGfZaxhMRbXlPXuVFpdWlXpir2W4AMhSJgRKzk/eDlIXOhb2LHWoLpDF7TEHylV5zNhykX6KAgHJmTNw==}
@@ -1094,6 +1099,15 @@ packages:
       '@babel/core': 7.18.9
       '@babel/helper-plugin-utils': 7.18.6
 
+  /@babel/plugin-syntax-bigint/7.8.3_@babel+core@7.18.2:
+    resolution: {integrity: sha512-wnTnFlG+YxQm3vDxpGE57Pj0srRU4sHE/mDkt1qv2YJJSeUAec2ma4WLUnUPeKjyrfntVwe/N6dCXpU+zL3Npg==}
+    peerDependencies:
+      '@babel/core': ^7.0.0-0
+    dependencies:
+      '@babel/core': 7.18.2
+      '@babel/helper-plugin-utils': 7.18.6
+    dev: false
+
   /@babel/plugin-syntax-bigint/7.8.3_@babel+core@7.18.9:
     resolution: {integrity: sha512-wnTnFlG+YxQm3vDxpGE57Pj0srRU4sHE/mDkt1qv2YJJSeUAec2ma4WLUnUPeKjyrfntVwe/N6dCXpU+zL3Npg==}
     peerDependencies:
@@ -1109,7 +1123,6 @@ packages:
     dependencies:
       '@babel/core': 7.18.2
       '@babel/helper-plugin-utils': 7.18.6
-    dev: true
 
   /@babel/plugin-syntax-class-properties/7.12.13_@babel+core@7.18.9:
     resolution: {integrity: sha512-fm4idjKla0YahUNgFNLCB0qySdsoPiZP3iQE3rky0mBUtMZ23yDJ9SJdg6dXTSDnulOVqiF3Hgr9nbXvXTQZYA==}
@@ -1224,6 +1237,15 @@ packages:
       '@babel/helper-plugin-utils': 7.18.6
     dev: true
 
+  /@babel/plugin-syntax-import-meta/7.10.4_@babel+core@7.18.2:
+    resolution: {integrity: sha512-Yqfm+XDx0+Prh3VSeEQCPU81yC+JWZ2pDPFSS4ZdpfZhp4MkFMaDC1UqseovEKwSUpnIL7+vK+Clp7bfh0iD7g==}
+    peerDependencies:
+      '@babel/core': ^7.0.0-0
+    dependencies:
+      '@babel/core': 7.18.2
+      '@babel/helper-plugin-utils': 7.18.6
+    dev: false
+
   /@babel/plugin-syntax-import-meta/7.10.4_@babel+core@7.18.9:
     resolution: {integrity: sha512-Yqfm+XDx0+Prh3VSeEQCPU81yC+JWZ2pDPFSS4ZdpfZhp4MkFMaDC1UqseovEKwSUpnIL7+vK+Clp7bfh0iD7g==}
     peerDependencies:
@@ -1239,7 +1261,6 @@ packages:
     dependencies:
       '@babel/core': 7.18.2
       '@babel/helper-plugin-utils': 7.18.6
-    dev: true
 
   /@babel/plugin-syntax-json-strings/7.8.3_@babel+core@7.18.9:
     resolution: {integrity: sha512-lY6kdGpWHvjoe2vk4WrAapEuBR69EMxZl+RoGRhrFGNYVK8mOPAW8VfbT/ZgrFbXlDNiiaxQnAtgVCZ6jv30EA==}
@@ -1274,7 +1295,6 @@ packages:
     dependencies:
       '@babel/core': 7.18.2
       '@babel/helper-plugin-utils': 7.18.6
-    dev: true
 
   /@babel/plugin-syntax-logical-assignment-operators/7.10.4_@babel+core@7.18.9:
     resolution: {integrity: sha512-d8waShlpFDinQ5MtvGU9xDAOzKH47+FFoney2baFIoMr952hKOLp1HR7VszoZvOsV/4+RRszNY7D17ba0te0ig==}
@@ -1291,7 +1311,6 @@ packages:
     dependencies:
       '@babel/core': 7.18.2
       '@babel/helper-plugin-utils': 7.18.6
-    dev: true
 
   /@babel/plugin-syntax-nullish-coalescing-operator/7.8.3_@babel+core@7.18.9:
     resolution: {integrity: sha512-aSff4zPII1u2QD7y+F8oDsz19ew4IGEJg9SVW+bqwpwtfFleiQDMdzA/R+UlWDzfnHFCxxleFT0PMIrR36XLNQ==}
@@ -1308,7 +1327,6 @@ packages:
     dependencies:
       '@babel/core': 7.18.2
       '@babel/helper-plugin-utils': 7.18.6
-    dev: true
 
   /@babel/plugin-syntax-numeric-separator/7.10.4_@babel+core@7.18.9:
     resolution: {integrity: sha512-9H6YdfkcK/uOnY/K7/aA2xpzaAgkQn37yzWUMRK7OaPOqOpGS1+n0H5hxT9AUw9EsSjPW8SVyMJwYRtWs3X3ug==}
@@ -1325,7 +1343,6 @@ packages:
     dependencies:
       '@babel/core': 7.18.2
       '@babel/helper-plugin-utils': 7.18.6
-    dev: true
 
   /@babel/plugin-syntax-object-rest-spread/7.8.3_@babel+core@7.18.9:
     resolution: {integrity: sha512-XoqMijGZb9y3y2XskN+P1wUGiVwWZ5JmoDRwx5+3GmEplNyVM2s2Dg8ILFQm8rWM48orGy5YpI5Bl8U1y7ydlA==}
@@ -1342,7 +1359,6 @@ packages:
     dependencies:
       '@babel/core': 7.18.2
       '@babel/helper-plugin-utils': 7.18.6
-    dev: true
 
   /@babel/plugin-syntax-optional-catch-binding/7.8.3_@babel+core@7.18.9:
     resolution: {integrity: sha512-6VPD0Pc1lpTqw0aKoeRTMiB+kWhAoT24PA+ksWSBrFtl5SIRVpZlwN3NNPQjehA2E/91FV3RjLWoVTglWcSV3Q==}
@@ -1359,7 +1375,6 @@ packages:
     dependencies:
       '@babel/core': 7.18.2
       '@babel/helper-plugin-utils': 7.18.6
-    dev: true
 
   /@babel/plugin-syntax-optional-chaining/7.8.3_@babel+core@7.18.9:
     resolution: {integrity: sha512-KoK9ErH1MBlCPxV0VANkXW2/dw4vlbGDrFgz8bmUsBGYkFRcbRwMh6cIJubdPrkxRwuGdtCk0v/wPTKbQgBjkg==}
@@ -1397,7 +1412,6 @@ packages:
     dependencies:
       '@babel/core': 7.18.2
       '@babel/helper-plugin-utils': 7.18.6
-    dev: true
 
   /@babel/plugin-syntax-top-level-await/7.14.5_@babel+core@7.18.9:
     resolution: {integrity: sha512-hx++upLv5U1rgYfwe1xBQUhRmU41NEvpUvrp8jkrSCdvGSnM5/qdRMtylJ6PG5OFkBaHkbTAKTnd3/YyESRHFw==}
@@ -4115,6 +4129,17 @@ packages:
       indent-string: 4.0.0
     dev: true
 
+  /ajv-formats/2.1.1_ajv@8.8.2:
+    resolution: {integrity: sha512-Wx0Kx52hxE7C18hkMEggYlEifqWZtYaRgouJor+WMdPnQyEK13vgEWyVNup7SoeeoLMsr4kf5h6dOW11I15MUA==}
+    peerDependencies:
+      ajv: ^8.0.0
+    peerDependenciesMeta:
+      ajv:
+        optional: true
+    dependencies:
+      ajv: 8.8.2
+    dev: false
+
   /ajv/6.12.6:
     resolution: {integrity: sha512-j3fVLgvTo527anyYyJOGTYJbG+vnnQYvE0m5mmkc1TK+nxAppkCLMIL0aZ4dblVCNoGShhm+kzE4ZUykBoMg4g==}
     dependencies:
@@ -4297,6 +4322,24 @@ packages:
     resolution: {integrity: sha512-Td525n+iPOOyUQIeBfcASuG6uJsDOITl7Mds5gFyerkWiX7qhUTdYUBlSgNMyVqtSJqwpt1kXGLdUt6SykLMRA==}
     dev: true
 
+  /babel-jest/29.0.3_@babel+core@7.18.2:
+    resolution: {integrity: sha512-ApPyHSOhS/sVzwUOQIWJmdvDhBsMG01HX9z7ogtkp1TToHGGUWFlnXJUIzCgKPSfiYLn3ibipCYzsKSURHEwLg==}
+    engines: {node: ^14.15.0 || ^16.10.0 || >=18.0.0}
+    peerDependencies:
+      '@babel/core': ^7.8.0
+    dependencies:
+      '@babel/core': 7.18.2
+      '@jest/transform': 29.0.3
+      '@types/babel__core': 7.1.19
+      babel-plugin-istanbul: 6.1.1
+      babel-preset-jest: 29.0.2_@babel+core@7.18.2
+      chalk: 4.1.2
+      graceful-fs: 4.2.10
+      slash: 3.0.0
+    transitivePeerDependencies:
+      - supports-color
+    dev: false
+
   /babel-jest/29.0.3_@babel+core@7.18.9:
     resolution: {integrity: sha512-ApPyHSOhS/sVzwUOQIWJmdvDhBsMG01HX9z7ogtkp1TToHGGUWFlnXJUIzCgKPSfiYLn3ibipCYzsKSURHEwLg==}
     engines: {node: ^14.15.0 || ^16.10.0 || >=18.0.0}
@@ -4440,6 +4483,26 @@ packages:
     resolution: {integrity: sha512-eqj0hVcJUR57/Ug2zE1Yswsw4LhuqqHhD+8v120T1cl3kjg76QwtyBrdIk4WVwK+lAhBJVYCd/v+4nc4y+8JsA==}
     dev: true
 
+  /babel-preset-current-node-syntax/1.0.1_@babel+core@7.18.2:
+    resolution: {integrity: sha512-M7LQ0bxarkxQoN+vz5aJPsLBn77n8QgTFmo8WK0/44auK2xlCXrYcUxHFxgU7qW5Yzw/CjmLRK2uJzaCd7LvqQ==}
+    peerDependencies:
+      '@babel/core': ^7.0.0
+    dependencies:
+      '@babel/core': 7.18.2
+      '@babel/plugin-syntax-async-generators': 7.8.4_@babel+core@7.18.2
+      '@babel/plugin-syntax-bigint': 7.8.3_@babel+core@7.18.2
+      '@babel/plugin-syntax-class-properties': 7.12.13_@babel+core@7.18.2
+      '@babel/plugin-syntax-import-meta': 7.10.4_@babel+core@7.18.2
+      '@babel/plugin-syntax-json-strings': 7.8.3_@babel+core@7.18.2
+      '@babel/plugin-syntax-logical-assignment-operators': 7.10.4_@babel+core@7.18.2
+      '@babel/plugin-syntax-nullish-coalescing-operator': 7.8.3_@babel+core@7.18.2
+      '@babel/plugin-syntax-numeric-separator': 7.10.4_@babel+core@7.18.2
+      '@babel/plugin-syntax-object-rest-spread': 7.8.3_@babel+core@7.18.2
+      '@babel/plugin-syntax-optional-catch-binding': 7.8.3_@babel+core@7.18.2
+      '@babel/plugin-syntax-optional-chaining': 7.8.3_@babel+core@7.18.2
+      '@babel/plugin-syntax-top-level-await': 7.14.5_@babel+core@7.18.2
+    dev: false
+
   /babel-preset-current-node-syntax/1.0.1_@babel+core@7.18.9:
     resolution: {integrity: sha512-M7LQ0bxarkxQoN+vz5aJPsLBn77n8QgTFmo8WK0/44auK2xlCXrYcUxHFxgU7qW5Yzw/CjmLRK2uJzaCd7LvqQ==}
     peerDependencies:
@@ -4459,6 +4522,17 @@ packages:
       '@babel/plugin-syntax-optional-chaining': 7.8.3_@babel+core@7.18.9
       '@babel/plugin-syntax-top-level-await': 7.14.5_@babel+core@7.18.9
 
+  /babel-preset-jest/29.0.2_@babel+core@7.18.2:
+    resolution: {integrity: sha512-BeVXp7rH5TK96ofyEnHjznjLMQ2nAeDJ+QzxKnHAAMs0RgrQsCywjAN8m4mOm5Di0pxU//3AoEeJJrerMH5UeA==}
+    engines: {node: ^14.15.0 || ^16.10.0 || >=18.0.0}
+    peerDependencies:
+      '@babel/core': ^7.0.0
+    dependencies:
+      '@babel/core': 7.18.2
+      babel-plugin-jest-hoist: 29.0.2
+      babel-preset-current-node-syntax: 1.0.1_@babel+core@7.18.2
+    dev: false
+
   /babel-preset-jest/29.0.2_@babel+core@7.18.9:
     resolution: {integrity: sha512-BeVXp7rH5TK96ofyEnHjznjLMQ2nAeDJ+QzxKnHAAMs0RgrQsCywjAN8m4mOm5Di0pxU//3AoEeJJrerMH5UeA==}
     engines: {node: ^14.15.0 || ^16.10.0 || >=18.0.0}

+ 4 - 1
kafka-ui-react-app/src/components/Topics/Topic/SendMessage/utils.ts

@@ -6,6 +6,7 @@ import {
 import jsf from 'json-schema-faker';
 import { compact } from 'lodash';
 import Ajv, { DefinedError } from 'ajv/dist/2020';
+import addFormats from 'ajv-formats';
 import upperFirst from 'lodash/upperFirst';
 
 jsf.option('fillProperties', false);
@@ -80,7 +81,9 @@ export const validateBySchema = (
     return [`Error in parsing the "${type}" field value`];
   }
   try {
-    const validate = new Ajv().compile(parcedSchema);
+    const ajv = new Ajv();
+    addFormats(ajv);
+    const validate = ajv.compile(parcedSchema);
     validate(parsedValue);
     if (validate.errors) {
       errors = validate.errors.map(