From 35fd583080d5a8794a0ae7fe147685d1f4d636d0 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Thu, 7 Sep 2023 09:30:41 -0700 Subject: [PATCH] Kafka protobuf consumer support (#4375) This adds support for kafka serdes-protobuf deserialization via KafkaTools#protobufSpec. There is an extensive amount of unit testing, both in the form of "how does this message structure gets parsed into column types" and "how do schema changes get adapted into the same table". While this feature is primarily targetted for use via KafkaTools, the underlying extensions-protobuf library is generalized to work in other cases as well. For example, it would work just as well at parsing protobuf messages via gRPC or websockets (at which point, it could easily be adapted into a blink table). The implementation is underpinned by a set a functional interfaces for extracting primitives or Objects from a generic type; in the case of protobuf, the generic type is com.google.protobuf.Message. Coupled with functional composition, these interfaces provide useful primitives for fluently expressing and executing parsing expressions in a reusable way. See https://docs.confluent.io/platform/current/schema-registry/fundamentals/serdes-develop/serdes-protobuf.html Fixes #3832 --- extensions/kafka/TESTING.md | 20 + extensions/kafka/build.gradle | 20 +- .../java/io/deephaven/kafka/BoxTransform.java | 289 +++ .../kafka/DhNullableTypeTransform.java | 94 + .../java/io/deephaven/kafka/KafkaTools.java | 31 + .../kafka/KeyOrValueProcessorImpl.java | 44 + .../io/deephaven/kafka/NullFunctions.java | 131 ++ .../java/io/deephaven/kafka/ProtobufImpl.java | 501 +++++ .../deephaven/kafka/ToChunkTypeTransform.java | 210 ++ .../io/deephaven/kafka/UnboxTransform.java | 267 +++ .../kafka/ingest/BooleanFieldCopier.java | 33 + .../kafka/ingest/ByteFieldCopier.java | 44 + .../kafka/ingest/CharFieldCopier.java | 39 + .../io/deephaven/kafka/ingest/ChunkUtils.java | 136 ++ .../kafka/ingest/DoubleFieldCopier.java | 39 + .../kafka/ingest/FieldCopierAdapter.java | 119 + .../kafka/ingest/FloatFieldCopier.java | 39 + .../kafka/ingest/IntFieldCopier.java | 39 + .../kafka/ingest/LongFieldCopier.java | 39 + .../kafka/ingest/MultiFieldChunkAdapter.java | 21 + .../kafka/ingest/ObjectFieldCopier.java | 33 + .../kafka/ingest/ShortFieldCopier.java | 39 + .../protobuf/ProtobufConsumeOptions.java | 138 ++ .../kafka/ProtobufImplSchemaChangeTest.java | 543 +++++ .../deephaven/kafka/protobuf/gen/BoolV1.java | 518 +++++ .../deephaven/kafka/protobuf/gen/BoolV2.java | 447 ++++ .../kafka/protobuf/gen/MyMessageV1.java | 597 +++++ .../kafka/protobuf/gen/MyMessageV2.java | 667 ++++++ .../kafka/protobuf/gen/MyMessageV3.java | 1724 +++++++++++++++ .../kafka/protobuf/gen/MyMessageV4.java | 1935 +++++++++++++++++ .../kafka/protobuf/gen/MyMessageV5.java | 1877 ++++++++++++++++ .../kafka/protobuf/gen/RenameV1.java | 596 +++++ .../kafka/protobuf/gen/RenameV2.java | 746 +++++++ .../kafka/protobuf/gen/SpecialTypesV1.java | 1194 ++++++++++ .../kafka/protobuf/gen/SpecialTypesV2.java | 1195 ++++++++++ .../kafka/src/test/proto/BoolV1.proto.txt | 13 + .../kafka/src/test/proto/BoolV2.proto.txt | 13 + .../src/test/proto/MyMessageV1.proto.txt | 13 + .../src/test/proto/MyMessageV2.proto.txt | 14 + .../src/test/proto/MyMessageV3.proto.txt | 21 + .../src/test/proto/MyMessageV4.proto.txt | 24 + .../src/test/proto/MyMessageV5.proto.txt | 25 + .../kafka/src/test/proto/RenameV1.proto.txt | 13 + .../kafka/src/test/proto/RenameV2.proto.txt | 14 + .../src/test/proto/SpecialTypesV1.proto.txt | 22 + .../src/test/proto/SpecialTypesV2.proto.txt | 22 + extensions/protobuf/build.gradle | 44 + extensions/protobuf/gradle.properties | 1 + .../deephaven/functions/BooleanFunctions.java | 155 ++ .../io/deephaven/functions/ByteFunctions.java | 42 + .../io/deephaven/functions/CharFunctions.java | 42 + .../deephaven/functions/DoubleFunctions.java | 47 + .../deephaven/functions/FloatFunctions.java | 42 + .../io/deephaven/functions/IntFunctions.java | 46 + .../io/deephaven/functions/LongFunctions.java | 47 + .../deephaven/functions/ObjectFunctions.java | 256 +++ .../deephaven/functions/ShortFunctions.java | 43 + .../functions/ToBooleanFunction.java | 114 + .../deephaven/functions/ToByteFunction.java | 62 + .../deephaven/functions/ToCharFunction.java | 61 + .../deephaven/functions/ToDoubleFunction.java | 57 + .../deephaven/functions/ToFloatFunction.java | 61 + .../io/deephaven/functions/ToIntFunction.java | 57 + .../deephaven/functions/ToLongFunction.java | 56 + .../deephaven/functions/ToObjectFunction.java | 282 +++ .../functions/ToPrimitiveFunction.java | 50 + .../deephaven/functions/ToShortFunction.java | 62 + .../io/deephaven/functions/TypedFunction.java | 33 + .../main/java/io/deephaven/protobuf/Box.java | 83 + .../java/io/deephaven/protobuf/Builtin.java | 607 ++++++ .../io/deephaven/protobuf/BypassOnNull.java | 171 ++ .../deephaven/protobuf/FieldNumberPath.java | 20 + .../io/deephaven/protobuf/FieldOptions.java | 177 ++ .../java/io/deephaven/protobuf/FieldPath.java | 148 ++ .../io/deephaven/protobuf/MessageParser.java | 128 ++ .../protobuf/MessageParserSingle.java | 31 + .../protobuf/ProtobufDescriptorParser.java | 175 ++ .../ProtobufDescriptorParserImpl.java | 551 +++++ .../ProtobufDescriptorParserOptions.java | 66 + .../deephaven/protobuf/ProtobufFunction.java | 55 + .../deephaven/protobuf/ProtobufFunctions.java | 73 + .../functions/ToBooleanFunctionTest.java | 64 + .../functions/ToByteFunctionTest.java | 20 + .../functions/ToCharFunctionTest.java | 20 + .../functions/ToDoubleFunctionTest.java | 16 + .../functions/ToFloatFunctionTest.java | 16 + .../functions/ToIntFunctionTest.java | 16 + .../functions/ToLongFunctionTest.java | 16 + .../functions/ToObjectFunctionTest.java | 23 + .../functions/ToShortFunctionTest.java | 16 + .../protobuf/ByteWrapperCustomType.java | 34 + .../io/deephaven/protobuf/FieldPathTest.java | 93 + .../ProtobufDescriptorParserTest.java | 1515 +++++++++++++ .../protobuf/src/test/proto/mytest.proto | 222 ++ py/server/deephaven/stream/kafka/consumer.py | 62 +- py/server/tests/test_kafka_consumer.py | 107 + redpanda-standalone/docker-compose.yml | 2 +- settings.gradle | 3 + 98 files changed, 20779 insertions(+), 9 deletions(-) create mode 100644 extensions/kafka/TESTING.md create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/BoxTransform.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/DhNullableTypeTransform.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/KeyOrValueProcessorImpl.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/NullFunctions.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ProtobufImpl.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ToChunkTypeTransform.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/UnboxTransform.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ingest/BooleanFieldCopier.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ByteFieldCopier.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ingest/CharFieldCopier.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ChunkUtils.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ingest/DoubleFieldCopier.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ingest/FieldCopierAdapter.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ingest/FloatFieldCopier.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ingest/IntFieldCopier.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ingest/LongFieldCopier.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ObjectFieldCopier.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ShortFieldCopier.java create mode 100644 extensions/kafka/src/main/java/io/deephaven/kafka/protobuf/ProtobufConsumeOptions.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/ProtobufImplSchemaChangeTest.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/BoolV1.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/BoolV2.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV1.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV2.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV3.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV4.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV5.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/RenameV1.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/RenameV2.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/SpecialTypesV1.java create mode 100644 extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/SpecialTypesV2.java create mode 100644 extensions/kafka/src/test/proto/BoolV1.proto.txt create mode 100644 extensions/kafka/src/test/proto/BoolV2.proto.txt create mode 100644 extensions/kafka/src/test/proto/MyMessageV1.proto.txt create mode 100644 extensions/kafka/src/test/proto/MyMessageV2.proto.txt create mode 100644 extensions/kafka/src/test/proto/MyMessageV3.proto.txt create mode 100644 extensions/kafka/src/test/proto/MyMessageV4.proto.txt create mode 100644 extensions/kafka/src/test/proto/MyMessageV5.proto.txt create mode 100644 extensions/kafka/src/test/proto/RenameV1.proto.txt create mode 100644 extensions/kafka/src/test/proto/RenameV2.proto.txt create mode 100644 extensions/kafka/src/test/proto/SpecialTypesV1.proto.txt create mode 100644 extensions/kafka/src/test/proto/SpecialTypesV2.proto.txt create mode 100644 extensions/protobuf/build.gradle create mode 100644 extensions/protobuf/gradle.properties create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/BooleanFunctions.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ByteFunctions.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/CharFunctions.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/DoubleFunctions.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/FloatFunctions.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/IntFunctions.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/LongFunctions.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ObjectFunctions.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ShortFunctions.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ToBooleanFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ToByteFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ToCharFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ToDoubleFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ToFloatFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ToIntFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ToLongFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ToObjectFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ToPrimitiveFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/ToShortFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/functions/TypedFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/Box.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/Builtin.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/BypassOnNull.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldNumberPath.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldOptions.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldPath.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/MessageParser.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/MessageParserSingle.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParser.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParserImpl.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParserOptions.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufFunction.java create mode 100644 extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufFunctions.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/functions/ToBooleanFunctionTest.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/functions/ToByteFunctionTest.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/functions/ToCharFunctionTest.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/functions/ToDoubleFunctionTest.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/functions/ToFloatFunctionTest.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/functions/ToIntFunctionTest.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/functions/ToLongFunctionTest.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/functions/ToObjectFunctionTest.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/functions/ToShortFunctionTest.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/protobuf/ByteWrapperCustomType.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/protobuf/FieldPathTest.java create mode 100644 extensions/protobuf/src/test/java/io/deephaven/protobuf/ProtobufDescriptorParserTest.java create mode 100644 extensions/protobuf/src/test/proto/mytest.proto diff --git a/extensions/kafka/TESTING.md b/extensions/kafka/TESTING.md new file mode 100644 index 00000000000..d8a4c3c92a7 --- /dev/null +++ b/extensions/kafka/TESTING.md @@ -0,0 +1,20 @@ +# testing + +## ProtobufImpl schema change testing + +Most of the "simple" protobuf testing is done in the protobuf project, `project(':extensions-protobuf')`. One of +the important parts with respect to the kafka integration is how schema changes are handled, and so that is tested here, +[ProtobufImplSchemaChangeTest.java](src/test/java/io/deephaven/kafka/ProtobufImplSchemaChangeTest.java). + +The test development setup for this is a little unconventional due to the fact that protoc won't let you generate +multiple versions of the same message type, at least not within the same protoc invocation. To work around this, there +is a little bit of manual test development workflow needed to add a new message / message version. It requires: + * Uncommenting the proto plugin logic in [build.gradle](build.gradle) + * Adding a single new .proto file per version of the message you want to create; it must have `option java_multiple_files = false;` (only one .proto file with a given message name can compile at any given time) + * Generating the new test code, `./gradlew :extensions-kafka:generateTestProto` + * Moving the output from [build/generated/source/proto/test/java/io/deephaven/kafka/protobuf/gen](build/generated/source/proto/test/java/io/deephaven/kafka/protobuf/gen) into [src/test/java/io/deephaven/kafka/protobuf/gen](src/test/java/io/deephaven/kafka/protobuf/gen) + * Renaming the .proto file to .proto.txt (to ensure it doesn't get re-generated later). + * Commenting out the proto plugin logic in [build.gradle](build.gradle) + +While it's likely possible to automate the above, it would likely be a lot of bespoke work that would probably not see +much use elsewhere. diff --git a/extensions/kafka/build.gradle b/extensions/kafka/build.gradle index cfb627fa8e4..745a949797e 100644 --- a/extensions/kafka/build.gradle +++ b/extensions/kafka/build.gradle @@ -1,6 +1,8 @@ plugins { id 'java-library' id 'io.deephaven.project.register' + // See TESTING.md + // id 'com.google.protobuf' version '0.9.4' } description 'Kafka: Integrating Engine tables with Kafka' @@ -8,14 +10,16 @@ description 'Kafka: Integrating Engine tables with Kafka' dependencies { api project(':engine-table') - api 'org.apache.avro:avro:1.11.1' + api 'org.apache.avro:avro:1.11.2' // Using io.confluent dependencies requires code in the toplevel build.gradle to add their maven repository. // Note: the -ccs flavor is provided by confluent as their community edition. It is equivalent to the maven central // version, but has a different version to make it easier to keep confluent dependencies aligned. - api 'org.apache.kafka:kafka-clients:7.3.0-ccs' - api 'io.confluent:kafka-avro-serializer:7.3.0' - runtimeOnly 'io.confluent:kafka-protobuf-serializer:7.3.0' + api 'org.apache.kafka:kafka-clients:7.4.0-ccs' + api 'io.confluent:kafka-avro-serializer:7.4.0' + + api 'io.confluent:kafka-protobuf-serializer:7.4.0' + api project(':extensions-protobuf') implementation project(':Configuration') implementation project(':log-factory') @@ -37,6 +41,14 @@ spotless { java { targetExclude( '**/**FieldCopier.java', + 'src/test/java/io/deephaven/kafka/protobuf/gen/**', ) } } + +// See TESTING.md +//protobuf { +// protoc { +// artifact = 'com.google.protobuf:protoc:3.24.1' +// } +//} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/BoxTransform.java b/extensions/kafka/src/main/java/io/deephaven/kafka/BoxTransform.java new file mode 100644 index 00000000000..70ed02548c0 --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/BoxTransform.java @@ -0,0 +1,289 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka; + +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToCharFunction; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToPrimitiveFunction; +import io.deephaven.functions.ToShortFunction; +import io.deephaven.functions.TypedFunction; +import io.deephaven.qst.type.BoxedBooleanType; +import io.deephaven.qst.type.BoxedByteType; +import io.deephaven.qst.type.BoxedCharType; +import io.deephaven.qst.type.BoxedDoubleType; +import io.deephaven.qst.type.BoxedFloatType; +import io.deephaven.qst.type.BoxedIntType; +import io.deephaven.qst.type.BoxedLongType; +import io.deephaven.qst.type.BoxedShortType; +import io.deephaven.util.type.TypeUtils; + +class BoxTransform { + + /** + * Creates the function composition {@code box ∘ f}. + * + *

+ * For primitive functions {@code f}, see {@link #of(ToPrimitiveFunction)}. + * + *

+ * For object functions {@code f}, {@code box} is the identity function (and {@code f} will simply be returned). + * + * @param f the function + * @return the object function + * @param the input type + * @see #of(ToBooleanFunction) + * @see #of(ToCharFunction) + * @see #of(ToByteFunction) + * @see #of(ToShortFunction) + * @see #of(ToIntFunction) + * @see #of(ToLongFunction) + * @see #of(ToFloatFunction) + * @see #of(ToDoubleFunction) + */ + public static ToObjectFunction of(TypedFunction f) { + return BoxedVisitor.of(f); + } + + /** + * Creates the function composition {@code box ∘ f}. + * + *

+ * The returned object function will have return type {@code f.returnType().boxedType()}. + * + * @param f the primitive function + * @return the object function + * @param the input type + * @see #of(ToBooleanFunction) + * @see #of(ToCharFunction) + * @see #of(ToByteFunction) + * @see #of(ToShortFunction) + * @see #of(ToIntFunction) + * @see #of(ToLongFunction) + * @see #of(ToFloatFunction) + * @see #of(ToDoubleFunction) + */ + public static ToObjectFunction of(ToPrimitiveFunction f) { + return BoxedVisitor.of(f); + } + + /** + * Creates the function composition {@code box ∘ f}, where {@code box} is simply a cast from {@code boolean} to + * {@code Boolean}. + * + *

+ * Equivalent to {@code x -> (Boolean)f.test(x)}. + * + * @param f the boolean function + * @return the object function + * @param the input type + */ + public static ToObjectFunction of(ToBooleanFunction f) { + return ToObjectFunction.of(f::test, BoxedBooleanType.of()); + } + + /** + * Creates the function composition {@code box ∘ f}. + * + *

+ * Equivalent to {@code x -> TypeUtils.box(f.applyAsChar(x))}. + * + * @param f the char function + * @return the object function + * @param the input type + * @see TypeUtils#box(char) + */ + public static ToObjectFunction of(ToCharFunction f) { + return ToObjectFunction.of(t -> box(f, t), BoxedCharType.of()); + } + + /** + * Creates the function composition {@code box ∘ f}. + * + *

+ * Equivalent to {@code x -> TypeUtils.box(f.applyAsByte(x))}. + * + * @param f the byte function + * @return the object function + * @param the input type + * @see TypeUtils#box(byte) + */ + public static ToObjectFunction of(ToByteFunction f) { + return ToObjectFunction.of(t -> box(f, t), BoxedByteType.of()); + } + + /** + * Creates the function composition {@code box ∘ f}. + * + *

+ * Equivalent to {@code x -> TypeUtils.box(f.applyAsShort(x))}. + * + * @param f the short function + * @return the object function + * @param the input type + * @see TypeUtils#box(short) + */ + public static ToObjectFunction of(ToShortFunction f) { + return ToObjectFunction.of(t -> box(f, t), BoxedShortType.of()); + } + + /** + * Creates the function composition {@code box ∘ f}. + * + *

+ * Equivalent to {@code x -> TypeUtils.box(f.applyAsInt(x))}. + * + * @param f the int function + * @return the object function + * @param the input type + * @see TypeUtils#box(int) + */ + public static ToObjectFunction of(ToIntFunction f) { + return ToObjectFunction.of(t -> box(f, t), BoxedIntType.of()); + } + + /** + * Creates the function composition {@code box ∘ f}. + * + *

+ * Equivalent to {@code x -> TypeUtils.box(f.applyAsLong(x))}. + * + * @param f the long function + * @return the object function + * @param the input type + * @see TypeUtils#box(long) + */ + public static ToObjectFunction of(ToLongFunction f) { + return ToObjectFunction.of(t -> box(f, t), BoxedLongType.of()); + } + + /** + * Creates the function composition {@code box ∘ f}. + * + *

+ * Equivalent to {@code x -> TypeUtils.box(f.applyAsFloat(x))}. + * + * @param f the float function + * @return the object function + * @param the input type + * @see TypeUtils#box(float) + */ + public static ToObjectFunction of(ToFloatFunction f) { + return ToObjectFunction.of(t -> box(f, t), BoxedFloatType.of()); + } + + /** + * Creates the function composition {@code box ∘ f}. + * + *

+ * Equivalent to {@code x -> TypeUtils.box(f.applyAsDouble(x))}. + * + * @param f the double function + * @return the object function + * @param the input type + * @see TypeUtils#box(double) + */ + public static ToObjectFunction of(ToDoubleFunction f) { + return ToObjectFunction.of(t -> box(f, t), BoxedDoubleType.of()); + } + + private enum BoxedVisitor implements TypedFunction.Visitor>, + ToPrimitiveFunction.Visitor> { + INSTANCE; + + public static ToObjectFunction of(TypedFunction f) { + // noinspection unchecked + return f.walk((TypedFunction.Visitor>) (TypedFunction.Visitor) INSTANCE); + } + + public static ToObjectFunction of(ToPrimitiveFunction f) { + // noinspection unchecked + return f.walk( + (ToPrimitiveFunction.Visitor>) (ToPrimitiveFunction.Visitor) INSTANCE); + } + + @Override + public ToObjectFunction visit(ToPrimitiveFunction f) { + return BoxTransform.of(f); + } + + @Override + public ToObjectFunction visit(ToObjectFunction f) { + return f; + } + + @Override + public ToObjectFunction visit(ToBooleanFunction f) { + return BoxTransform.of(f); + } + + @Override + public ToObjectFunction visit(ToCharFunction f) { + return BoxTransform.of(f); + } + + @Override + public ToObjectFunction visit(ToByteFunction f) { + return BoxTransform.of(f); + } + + @Override + public ToObjectFunction visit(ToShortFunction f) { + return BoxTransform.of(f); + } + + @Override + public ToObjectFunction visit(ToIntFunction f) { + return BoxTransform.of(f); + } + + @Override + public ToObjectFunction visit(ToLongFunction f) { + return BoxTransform.of(f); + } + + @Override + public ToObjectFunction visit(ToFloatFunction f) { + return BoxTransform.of(f); + } + + @Override + public ToObjectFunction visit(ToDoubleFunction f) { + return BoxTransform.of(f); + } + } + + private static Character box(ToCharFunction f, T x) { + return TypeUtils.box(f.applyAsChar(x)); + } + + private static Byte box(ToByteFunction f, T x) { + return TypeUtils.box(f.applyAsByte(x)); + } + + private static Short box(ToShortFunction f, T x) { + return TypeUtils.box(f.applyAsShort(x)); + } + + private static Integer box(ToIntFunction f, T x) { + return TypeUtils.box(f.applyAsInt(x)); + } + + private static Long box(ToLongFunction f, T x) { + return TypeUtils.box(f.applyAsLong(x)); + } + + private static Float box(ToFloatFunction f, T x) { + return TypeUtils.box(f.applyAsFloat(x)); + } + + private static Double box(ToDoubleFunction f, T x) { + return TypeUtils.box(f.applyAsDouble(x)); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/DhNullableTypeTransform.java b/extensions/kafka/src/main/java/io/deephaven/kafka/DhNullableTypeTransform.java new file mode 100644 index 00000000000..9132913f185 --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/DhNullableTypeTransform.java @@ -0,0 +1,94 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka; + +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToCharFunction; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToPrimitiveFunction; +import io.deephaven.functions.ToShortFunction; +import io.deephaven.functions.TypedFunction; + +class DhNullableTypeTransform { + + public static TypedFunction of(TypedFunction f) { + return NullableTypeVisitor.of(f); + } + + public static TypedFunction of(ToPrimitiveFunction f) { + return NullableTypeVisitor.of(f); + } + + private enum NullableTypeVisitor implements TypedFunction.Visitor>, + ToPrimitiveFunction.Visitor> { + INSTANCE; + + public static TypedFunction of(TypedFunction f) { + // noinspection unchecked + return f.walk((TypedFunction.Visitor>) (TypedFunction.Visitor) INSTANCE); + } + + public static TypedFunction of(ToPrimitiveFunction f) { + // noinspection unchecked + return f.walk( + (ToPrimitiveFunction.Visitor>) (ToPrimitiveFunction.Visitor) INSTANCE); + } + + @Override + public TypedFunction visit(ToPrimitiveFunction f) { + return of(f); + } + + @Override + public TypedFunction visit(ToObjectFunction f) { + return f; + } + + @Override + public ToObjectFunction visit(ToBooleanFunction f) { + // BooleanFunction is the only function / primitive type that doesn't natively have a "null" type. + return BoxTransform.of(f); + } + + @Override + public ToCharFunction visit(ToCharFunction f) { + return f; + } + + @Override + public ToByteFunction visit(ToByteFunction f) { + return f; + } + + @Override + public ToShortFunction visit(ToShortFunction f) { + return f; + } + + @Override + public ToIntFunction visit(ToIntFunction f) { + return f; + } + + @Override + public ToLongFunction visit(ToLongFunction f) { + return f; + } + + @Override + public ToFloatFunction visit(ToFloatFunction f) { + return f; + } + + @Override + public ToDoubleFunction visit(ToDoubleFunction f) { + return f; + } + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/KafkaTools.java b/extensions/kafka/src/main/java/io/deephaven/kafka/KafkaTools.java index 1d337f8e697..acea6cff6a9 100644 --- a/extensions/kafka/src/main/java/io/deephaven/kafka/KafkaTools.java +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/KafkaTools.java @@ -4,6 +4,7 @@ package io.deephaven.kafka; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.protobuf.Descriptors.Descriptor; import gnu.trove.map.hash.TIntLongHashMap; import io.confluent.kafka.schemaregistry.SchemaProvider; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; @@ -53,6 +54,7 @@ import io.deephaven.kafka.KafkaTools.TableType.Blink; import io.deephaven.kafka.KafkaTools.TableType.Ring; import io.deephaven.kafka.KafkaTools.TableType.Visitor; +import io.deephaven.kafka.ProtobufImpl.ProtobufConsumeImpl; import io.deephaven.kafka.RawImpl.RawConsume; import io.deephaven.kafka.RawImpl.RawProduce; import io.deephaven.kafka.SimpleImpl.SimpleConsume; @@ -62,9 +64,11 @@ import io.deephaven.kafka.ingest.KafkaRecordConsumer; import io.deephaven.kafka.ingest.KafkaStreamPublisher; import io.deephaven.kafka.ingest.KeyOrValueProcessor; +import io.deephaven.kafka.protobuf.ProtobufConsumeOptions; import io.deephaven.kafka.publish.KafkaPublisherException; import io.deephaven.kafka.publish.KeyOrValueSerializer; import io.deephaven.kafka.publish.PublishToKafka; +import io.deephaven.protobuf.ProtobufDescriptorParserOptions; import io.deephaven.qst.column.header.ColumnHeader; import io.deephaven.stream.StreamChunkUtils; import io.deephaven.stream.StreamConsumer; @@ -513,6 +517,33 @@ public static KeyOrValueSpec avroSpec(final String schemaName) { return new AvroConsume(schemaName, AVRO_LATEST_VERSION, DIRECT_MAPPING); } + /** + * The kafka protobuf specs. This will fetch the {@link com.google.protobuf.Descriptors.Descriptor protobuf + * descriptor} for the {@link ProtobufConsumeOptions#schemaSubject() schema subject} from the schema registry + * using version {@link ProtobufConsumeOptions#schemaVersion() schema version} and create + * {@link com.google.protobuf.Message message} parsing functions according to + * {@link io.deephaven.protobuf.ProtobufDescriptorParser#parse(Descriptor, ProtobufDescriptorParserOptions)}. + * These functions will be adapted to handle schema changes. + * + *

+ * For purposes of reproducibility across restarts where schema changes may occur, it is advisable for callers + * to set a specific {@link ProtobufConsumeOptions#schemaVersion() schema version}. This will ensure the + * resulting {@link io.deephaven.engine.table.TableDefinition table definition} will not change across restarts. + * This gives the caller an explicit opportunity to update any downstream consumers when updating + * {@link ProtobufConsumeOptions#schemaVersion() schema version} if necessary. + * + * @param options the options + * @return the key or value spec + * @see io.deephaven.protobuf.ProtobufDescriptorParser#parse(Descriptor, ProtobufDescriptorParserOptions) + * parsing + * @see kafka + * protobuf serdes + */ + public static KeyOrValueSpec protobufSpec(ProtobufConsumeOptions options) { + return new ProtobufConsumeImpl(options); + } + /** * If {@code columnName} is set, that column name will be used. Otherwise, the names for the key or value * columns can be provided in the properties as {@value KEY_COLUMN_NAME_PROPERTY} or diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/KeyOrValueProcessorImpl.java b/extensions/kafka/src/main/java/io/deephaven/kafka/KeyOrValueProcessorImpl.java new file mode 100644 index 00000000000..48607660a31 --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/KeyOrValueProcessorImpl.java @@ -0,0 +1,44 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka; + +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.kafka.ingest.FieldCopier; +import io.deephaven.kafka.ingest.KafkaIngesterException; +import io.deephaven.kafka.ingest.KeyOrValueProcessor; + +import java.util.List; +import java.util.Objects; + +final class KeyOrValueProcessorImpl implements KeyOrValueProcessor { + + private final int[] chunkOffsets; + private final List copiers; + private final boolean allowNulls; + + public KeyOrValueProcessorImpl(final int[] chunkOffsets, List copiers, boolean allowNulls) { + this.chunkOffsets = Objects.requireNonNull(chunkOffsets); + this.copiers = Objects.requireNonNull(copiers); + this.allowNulls = allowNulls; + } + + @Override + public void handleChunk(ObjectChunk inputChunk, WritableChunk[] publisherChunks) { + if (!allowNulls) { + for (int ii = 0; ii < inputChunk.size(); ++ii) { + if (inputChunk.get(ii) == null) { + throw new KafkaIngesterException("Null records are not permitted"); + } + } + } + for (int i = 0; i < chunkOffsets.length; ++i) { + final WritableChunk publisherChunk = publisherChunks[chunkOffsets[i]]; + final int existingSize = publisherChunk.size(); + publisherChunk.setSize(existingSize + inputChunk.size()); + copiers.get(i).copyField(inputChunk, publisherChunk, 0, existingSize, inputChunk.size()); + } + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/NullFunctions.java b/extensions/kafka/src/main/java/io/deephaven/kafka/NullFunctions.java new file mode 100644 index 00000000000..7e71c9ddae8 --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/NullFunctions.java @@ -0,0 +1,131 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka; + +import io.deephaven.qst.type.BooleanType; +import io.deephaven.qst.type.ByteType; +import io.deephaven.qst.type.CharType; +import io.deephaven.qst.type.DoubleType; +import io.deephaven.qst.type.FloatType; +import io.deephaven.qst.type.GenericType; +import io.deephaven.qst.type.IntType; +import io.deephaven.qst.type.LongType; +import io.deephaven.qst.type.PrimitiveType; +import io.deephaven.qst.type.ShortType; +import io.deephaven.qst.type.Type; +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToCharFunction; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToPrimitiveFunction; +import io.deephaven.functions.ToShortFunction; +import io.deephaven.functions.TypedFunction; +import io.deephaven.util.QueryConstants; + +import java.util.Optional; + +class NullFunctions { + + public static Optional> of(Type returnType) { + // noinspection unchecked + return Optional.ofNullable((TypedFunction) returnType.walk(NullFunctionVisitor.INSTANCE)); + } + + public static Optional> of(PrimitiveType returnType) { + // noinspection unchecked + return Optional.ofNullable((ToPrimitiveFunction) returnType + .walk((PrimitiveType.Visitor>) NullFunctionVisitor.INSTANCE)); + } + + public static ToObjectFunction of(GenericType returnType) { + return ToObjectFunction.of(x -> null, returnType); + } + + public static ToCharFunction nullCharFunction() { + return x -> QueryConstants.NULL_CHAR; + } + + public static ToByteFunction nullByteFunction() { + return x -> QueryConstants.NULL_BYTE; + } + + public static ToShortFunction nullShortFunction() { + return x -> QueryConstants.NULL_SHORT; + } + + public static ToIntFunction nullIntFunction() { + return x -> QueryConstants.NULL_INT; + } + + public static ToLongFunction nullLongFunction() { + return x -> QueryConstants.NULL_LONG; + } + + public static ToFloatFunction nullFloatFunction() { + return x -> QueryConstants.NULL_FLOAT; + } + + public static ToDoubleFunction nullDoubleFunction() { + return x -> QueryConstants.NULL_DOUBLE; + } + + private enum NullFunctionVisitor + implements Type.Visitor>, PrimitiveType.Visitor> { + INSTANCE; + + @Override + public ToPrimitiveFunction visit(PrimitiveType primitiveType) { + return of(primitiveType).orElse(null); + } + + @Override + public TypedFunction visit(GenericType genericType) { + return of(genericType); + } + + @Override + public ToBooleanFunction visit(BooleanType booleanType) { + return null; + } + + @Override + public ToByteFunction visit(ByteType byteType) { + return nullByteFunction(); + } + + @Override + public ToPrimitiveFunction visit(CharType charType) { + return nullCharFunction(); + } + + @Override + public ToPrimitiveFunction visit(ShortType shortType) { + return nullShortFunction(); + } + + @Override + public ToPrimitiveFunction visit(IntType intType) { + return nullIntFunction(); + } + + @Override + public ToPrimitiveFunction visit(LongType longType) { + return nullLongFunction(); + } + + @Override + public ToPrimitiveFunction visit(FloatType floatType) { + return nullFloatFunction(); + } + + @Override + public ToPrimitiveFunction visit(DoubleType doubleType) { + return nullDoubleFunction(); + } + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ProtobufImpl.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ProtobufImpl.java new file mode 100644 index 00000000000..93f60275d2c --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ProtobufImpl.java @@ -0,0 +1,501 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka; + +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Descriptors.FieldDescriptor; +import com.google.protobuf.Message; +import io.confluent.kafka.schemaregistry.SchemaProvider; +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 io.confluent.kafka.serializers.protobuf.KafkaProtobufDeserializer; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.api.ColumnName; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToCharFunction; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToPrimitiveFunction; +import io.deephaven.functions.ToShortFunction; +import io.deephaven.functions.TypedFunction; +import io.deephaven.kafka.KafkaTools.Consume; +import io.deephaven.kafka.KafkaTools.KeyOrValue; +import io.deephaven.kafka.KafkaTools.KeyOrValueIngestData; +import io.deephaven.kafka.ingest.FieldCopier; +import io.deephaven.kafka.ingest.FieldCopierAdapter; +import io.deephaven.kafka.ingest.KeyOrValueProcessor; +import io.deephaven.kafka.ingest.MultiFieldChunkAdapter; +import io.deephaven.kafka.protobuf.ProtobufConsumeOptions; +import io.deephaven.kafka.protobuf.ProtobufConsumeOptions.FieldPathToColumnName; +import io.deephaven.protobuf.FieldNumberPath; +import io.deephaven.protobuf.FieldOptions; +import io.deephaven.protobuf.FieldPath; +import io.deephaven.protobuf.ProtobufDescriptorParser; +import io.deephaven.protobuf.ProtobufDescriptorParserOptions; +import io.deephaven.protobuf.ProtobufFunction; +import io.deephaven.protobuf.ProtobufFunctions; +import io.deephaven.protobuf.ProtobufFunctions.Builder; +import io.deephaven.qst.type.ArrayType; +import io.deephaven.qst.type.BooleanType; +import io.deephaven.qst.type.BoxedType; +import io.deephaven.qst.type.ByteType; +import io.deephaven.qst.type.CharType; +import io.deephaven.qst.type.CustomType; +import io.deephaven.qst.type.DoubleType; +import io.deephaven.qst.type.FloatType; +import io.deephaven.qst.type.GenericType; +import io.deephaven.qst.type.InstantType; +import io.deephaven.qst.type.IntType; +import io.deephaven.qst.type.LongType; +import io.deephaven.qst.type.PrimitiveType; +import io.deephaven.qst.type.ShortType; +import io.deephaven.qst.type.StringType; +import io.deephaven.qst.type.Type; +import io.deephaven.qst.type.Type.Visitor; +import io.deephaven.util.annotations.VisibleForTesting; +import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.kafka.common.serialization.Deserializer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.function.Function; + +/** + * This layer builds on top of {@link ProtobufDescriptorParser#parse(Descriptor, ProtobufDescriptorParserOptions)} by + * further transforming the functions according to {@link #withMostAppropriateType(TypedFunction)}, and then further + * adapting the functions to ensure they work for the expected chunk types {@link ToChunkTypeTransform}. This layer is + * also responsible for managing schema changes; in essence, ensuring that newly + * {@link ProtobufDescriptorParser#parse(Descriptor, ProtobufDescriptorParserOptions) parsed} {@link Descriptor + * descriptor} {@link TypedFunction functions} can be adapted into the original function type. + */ +class ProtobufImpl { + + @VisibleForTesting + static ProtobufFunctions schemaChangeAwareFunctions(Descriptor descriptor, + ProtobufDescriptorParserOptions options) { + return new ParsedStates(descriptor, options).functionsForSchemaChanges(); + } + + static final class ProtobufConsumeImpl extends Consume.KeyOrValueSpec { + + private static final ToObjectFunction PROTOBUF_MESSAGE_OBJ = + ToObjectFunction.identity(Type.ofCustom(Message.class)); + + private final ProtobufConsumeOptions specs; + + ProtobufConsumeImpl(ProtobufConsumeOptions specs) { + this.specs = Objects.requireNonNull(specs); + } + + @Override + public Optional getSchemaProvider() { + return Optional.of(new ProtobufSchemaProvider()); + } + + @Override + Deserializer getDeserializer(KeyOrValue keyOrValue, SchemaRegistryClient schemaRegistryClient, + Map configs) { + return new KafkaProtobufDeserializer<>(Objects.requireNonNull(schemaRegistryClient)); + } + + @Override + KeyOrValueIngestData getIngestData(KeyOrValue keyOrValue, SchemaRegistryClient schemaRegistryClient, + Map configs, MutableInt nextColumnIndexMut, List> columnDefinitionsOut) { + final Descriptor descriptor; + try { + descriptor = getDescriptor(schemaRegistryClient); + } catch (RestClientException | IOException e) { + throw new UncheckedDeephavenException(e); + } + final ProtobufFunctions functions = schemaChangeAwareFunctions(descriptor, specs.parserOptions()); + final List fieldCopiers = new ArrayList<>(functions.functions().size()); + final KeyOrValueIngestData data = new KeyOrValueIngestData(); + data.fieldPathToColumnName = new LinkedHashMap<>(); + final FieldPathToColumnName fieldPathToColumnName = specs.pathToColumnName(); + final Map indices = new HashMap<>(); + for (ProtobufFunction f : functions.functions()) { + final int ix = indices.compute(f.path(), (fieldPath, i) -> i == null ? 0 : i + 1); + final ColumnName columnName = fieldPathToColumnName.columnName(f.path(), ix); + add(columnName, f.function(), data, columnDefinitionsOut, fieldCopiers); + } + // we don't have enough info at this time to create KeyOrValueProcessorImpl + // data.extra = new KeyOrValueProcessorImpl(MultiFieldChunkAdapter.chunkOffsets(null, null), fieldCopiers, + // false); + data.extra = fieldCopiers; + return data; + } + + private void add( + ColumnName columnName, + TypedFunction function, + KeyOrValueIngestData data, + List> columnDefinitionsOut, + List fieldCopiersOut) { + data.fieldPathToColumnName.put(columnName.name(), columnName.name()); + columnDefinitionsOut.add(ColumnDefinition.of(columnName.name(), function.returnType())); + fieldCopiersOut.add(FieldCopierAdapter.of(PROTOBUF_MESSAGE_OBJ.map(ToChunkTypeTransform.of(function)))); + } + + @Override + KeyOrValueProcessor getProcessor(TableDefinition tableDef, KeyOrValueIngestData data) { + // noinspection unchecked + return new KeyOrValueProcessorImpl( + MultiFieldChunkAdapter.chunkOffsets(tableDef, data.fieldPathToColumnName), + (List) data.extra, false); + } + + private Descriptor getDescriptor(SchemaRegistryClient schemaRegistryClient) + throws RestClientException, IOException { + final SchemaMetadata metadata = specs.schemaVersion().isPresent() + ? schemaRegistryClient.getSchemaMetadata(specs.schemaSubject(), specs.schemaVersion().getAsInt()) + : schemaRegistryClient.getLatestSchemaMetadata(specs.schemaSubject()); + if (!ProtobufSchema.TYPE.equals(metadata.getSchemaType())) { + throw new IllegalStateException(String.format("Expected schema type %s but was %s", ProtobufSchema.TYPE, + metadata.getSchemaType())); + } + final ProtobufSchema protobufSchema = (ProtobufSchema) schemaRegistryClient + .getSchemaBySubjectAndId(specs.schemaSubject(), metadata.getId()); + // The potential need to set io.deephaven.kafka.protobuf.ProtobufConsumeOptions#schemaMessageName + // seems unfortunate; I'm surprised the information is not part of the kafka serdes protocol. + // Maybe it's so that a single schema can be used, and different topics with different root messages can + // all share that common schema? + return specs.schemaMessageName().isPresent() + ? protobufSchema.toDescriptor(specs.schemaMessageName().get()) + : protobufSchema.toDescriptor(); + } + } + + + private static ProtobufFunctions withMostAppropriateType(ProtobufFunctions functions) { + final Builder builder = ProtobufFunctions.builder(); + for (ProtobufFunction f : functions.functions()) { + builder.addFunctions(ProtobufFunction.of(f.path(), withMostAppropriateType(f.function()))); + } + return builder.build(); + } + + /** + * Adapts {@code f} to the most appropriate Deephaven equivalent / nullable type. + * + *

    + *
  • boolean -> Boolean
  • + *
  • Byte -> byte
  • + *
  • Character -> char
  • + *
  • Short -> short
  • + *
  • Integer -> int
  • + *
  • Long -> long
  • + *
  • Float -> float
  • + *
  • Double -> double
  • + *
+ */ + private static TypedFunction withMostAppropriateType(TypedFunction f) { + final TypedFunction f2 = DhNullableTypeTransform.of(f); + final ToPrimitiveFunction unboxed = UnboxTransform.of(f2).orElse(null); + return unboxed != null ? unboxed : f2; + } + + private static class ParsedStates { + private final Descriptor originalDescriptor; + private final ProtobufDescriptorParserOptions options; + private final Map parsed; + + private ParsedStates(Descriptor originalDescriptor, ProtobufDescriptorParserOptions options) { + this.originalDescriptor = Objects.requireNonNull(originalDescriptor); + this.options = Objects.requireNonNull(options); + this.parsed = new HashMap<>(); + getOrCreate(originalDescriptor); + } + + public ProtobufFunctions functionsForSchemaChanges() { + final Builder builder = ProtobufFunctions.builder(); + for (ProtobufFunction f : getOrCreate(originalDescriptor).functions()) { + builder.addFunctions(ProtobufFunction.of(f.path(), new ForPath(f).adaptForSchemaChanges())); + } + return builder.build(); + } + + private ProtobufFunctions getOrCreate(Descriptor descriptor) { + return parsed.computeIfAbsent(descriptor, this::create); + } + + private ProtobufFunctions create(Descriptor newDescriptor) { + if (!originalDescriptor.getFullName().equals(newDescriptor.getFullName())) { + throw new IllegalArgumentException(String.format( + "Expected descriptor names to match. expected='%s', actual='%s'. You may need to explicitly set schema_message_name.", + originalDescriptor.getFullName(), newDescriptor.getFullName())); + } + if (newDescriptor == originalDescriptor) { + return withMostAppropriateType(ProtobufDescriptorParser.parse(newDescriptor, options)); + } + final Function adaptedOptions = fieldPath -> { + final FieldPath originalFieldPath = adaptFieldPath(fieldPath).orElse(null); + if (originalFieldPath == null) { + // This must be a new field, exclude it. + return FieldOptions.exclude(); + } + return options.fieldOptions().apply(originalFieldPath); + }; + final ProtobufDescriptorParserOptions a = ProtobufDescriptorParserOptions.builder() + .parsers(options.parsers()) + .fieldOptions(adaptedOptions) + .build(); + return withMostAppropriateType(ProtobufDescriptorParser.parse(newDescriptor, a)); + } + + private Optional adaptFieldPath(FieldPath path) { + if (path.path().isEmpty()) { + return Optional.of(path); + } + final List originalFds = new ArrayList<>(path.path().size()); + Descriptor descriptor = originalDescriptor; + final Iterator it = path.path().iterator(); + while (true) { + final FieldDescriptor currentFd = it.next(); + final FieldDescriptor originalFd = descriptor.findFieldByNumber(currentFd.getNumber()); + if (originalFd == null) { + // originalFd does not exist + return Optional.empty(); + } + originalFds.add(originalFd); + if (!it.hasNext()) { + break; + } + descriptor = originalFd.getMessageType(); + } + return Optional.of(FieldPath.of(originalFds)); + } + + private class ForPath { + private final ProtobufFunction originalFunction; + private final Map> functions; + + public ForPath(ProtobufFunction originalFunction) { + this.originalFunction = Objects.requireNonNull(originalFunction); + this.functions = new HashMap<>(); + } + + public TypedFunction adaptForSchemaChanges() { + final Type originalReturnType = originalReturnType(); + final TypedFunction out = originalReturnType.walk(new AdaptForSchemaChanges()); + if (!originalReturnType.equals(out.returnType())) { + throw new IllegalStateException(String.format( + "AdaptForSchemaChanges error, mismatched types for %s. expected=%s, actual=%s", + originalFunction.path().namePath(), originalReturnType, out.returnType())); + } + return out; + } + + private Type originalReturnType() { + return originalFunction.function().returnType(); + } + + private boolean test(Message message) { + return ((ToBooleanFunction) getOrCreateForType(message)).test(message); + } + + private char applyAsChar(Message message) { + return ((ToCharFunction) getOrCreateForType(message)).applyAsChar(message); + } + + private byte applyAsByte(Message message) { + return ((ToByteFunction) getOrCreateForType(message)).applyAsByte(message); + } + + private short applyAsShort(Message message) { + return ((ToShortFunction) getOrCreateForType(message)).applyAsShort(message); + } + + private int applyAsInt(Message message) { + return ((ToIntFunction) getOrCreateForType(message)).applyAsInt(message); + } + + private long applyAsLong(Message message) { + return ((ToLongFunction) getOrCreateForType(message)).applyAsLong(message); + } + + private float applyAsFloat(Message message) { + return ((ToFloatFunction) getOrCreateForType(message)).applyAsFloat(message); + } + + private double applyAsDouble(Message message) { + return ((ToDoubleFunction) getOrCreateForType(message)).applyAsDouble(message); + } + + private T applyAsObject(Message message) { + return ((ToObjectFunction) getOrCreateForType(message)).apply(message); + } + + private TypedFunction getOrCreateForType(Message message) { + return getOrCreate(message.getDescriptorForType()); + } + + private TypedFunction getOrCreate(Descriptor descriptor) { + return functions.computeIfAbsent(descriptor, this::createFunctionFor); + } + + private TypedFunction createFunctionFor(Descriptor descriptor) { + final Type originalReturnType = originalReturnType(); + final TypedFunction newFunction = + find(ParsedStates.this.getOrCreate(descriptor), originalFunction.path().numberPath()) + .map(ProtobufFunction::function) + .orElse(null); + final TypedFunction adaptedFunction = + SchemaChangeAdaptFunction.of(newFunction, originalReturnType).orElse(null); + if (adaptedFunction == null) { + throw new UncheckedDeephavenException( + String.format("Incompatible schema change for %s, originalType=%s, newType=%s", + originalFunction.path().namePath(), originalReturnType, + newFunction == null ? null : newFunction.returnType())); + } + if (!originalReturnType.equals(adaptedFunction.returnType())) { + // If this happens, must be a logical error in SchemaChangeAdaptFunction + throw new IllegalStateException(String.format( + "Expected adapted return types to be equal for %s, originalType=%s, adaptedType=%s", + originalFunction.path().namePath(), originalReturnType, adaptedFunction.returnType())); + } + return adaptedFunction; + } + + class AdaptForSchemaChanges + implements Visitor>, PrimitiveType.Visitor> { + + @Override + public TypedFunction visit(PrimitiveType primitiveType) { + return primitiveType.walk((PrimitiveType.Visitor>) this); + } + + @Override + public ToObjectFunction visit(GenericType genericType) { + // noinspection unchecked + return ToObjectFunction.of(ForPath.this::applyAsObject, (GenericType) genericType); + } + + @Override + public ToBooleanFunction visit(BooleanType booleanType) { + return ForPath.this::test; + } + + @Override + public ToByteFunction visit(ByteType byteType) { + return ForPath.this::applyAsByte; + } + + @Override + public ToCharFunction visit(CharType charType) { + return ForPath.this::applyAsChar; + } + + @Override + public ToShortFunction visit(ShortType shortType) { + return ForPath.this::applyAsShort; + } + + @Override + public ToIntFunction visit(IntType intType) { + return ForPath.this::applyAsInt; + } + + @Override + public ToLongFunction visit(LongType longType) { + return ForPath.this::applyAsLong; + } + + @Override + public ToFloatFunction visit(FloatType floatType) { + return ForPath.this::applyAsFloat; + } + + @Override + public ToDoubleFunction visit(DoubleType doubleType) { + return ForPath.this::applyAsDouble; + } + } + } + } + + private static class SchemaChangeAdaptFunction implements TypedFunction.Visitor> { + + public static Optional> of(TypedFunction f, Type desiredReturnType) { + if (f == null) { + return NullFunctions.of(desiredReturnType); + } + if (desiredReturnType.equals(f.returnType())) { + return Optional.of(f); + } + return Optional.ofNullable(f.walk(new SchemaChangeAdaptFunction<>(desiredReturnType))); + } + + private final Type desiredReturnType; + + public SchemaChangeAdaptFunction(Type desiredReturnType) { + this.desiredReturnType = Objects.requireNonNull(desiredReturnType); + } + + @Override + public TypedFunction visit(ToPrimitiveFunction f) { + if (desiredReturnType.equals(f.returnType().boxedType())) { + return BoxTransform.of(f); + } + return null; + } + + @Override + public TypedFunction visit(ToObjectFunction f) { + return f.returnType().walk(new GenericType.Visitor<>() { + @Override + public TypedFunction visit(BoxedType boxedType) { + if (desiredReturnType.equals(boxedType.primitiveType())) { + return UnboxTransform.of(f).orElse(null); + } + return null; + } + + @Override + public TypedFunction visit(StringType stringType) { + return null; + } + + @Override + public TypedFunction visit(InstantType instantType) { + return null; + } + + @Override + public TypedFunction visit(ArrayType arrayType) { + return null; + } + + @Override + public TypedFunction visit(CustomType customType) { + return null; + } + }); + } + } + + private static Optional find(ProtobufFunctions f, FieldNumberPath numberPath) { + for (ProtobufFunction function : f.functions()) { + if (numberPath.equals(function.path().numberPath())) { + return Optional.of(function); + } + } + return Optional.empty(); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ToChunkTypeTransform.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ToChunkTypeTransform.java new file mode 100644 index 00000000000..0e03f8dfd09 --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ToChunkTypeTransform.java @@ -0,0 +1,210 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka; + +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToPrimitiveFunction; +import io.deephaven.functions.TypedFunction; +import io.deephaven.qst.type.ArrayType; +import io.deephaven.qst.type.BoxedBooleanType; +import io.deephaven.qst.type.BoxedByteType; +import io.deephaven.qst.type.BoxedCharType; +import io.deephaven.qst.type.BoxedDoubleType; +import io.deephaven.qst.type.BoxedFloatType; +import io.deephaven.qst.type.BoxedIntType; +import io.deephaven.qst.type.BoxedLongType; +import io.deephaven.qst.type.BoxedShortType; +import io.deephaven.qst.type.BoxedType; +import io.deephaven.qst.type.CustomType; +import io.deephaven.qst.type.GenericType; +import io.deephaven.qst.type.InstantType; +import io.deephaven.qst.type.StringType; +import io.deephaven.time.DateTimeUtils; +import io.deephaven.util.BooleanUtils; + +import java.time.Instant; +import java.util.Objects; + +import static io.deephaven.kafka.UnboxTransform.unboxByte; +import static io.deephaven.kafka.UnboxTransform.unboxChar; +import static io.deephaven.kafka.UnboxTransform.unboxDouble; +import static io.deephaven.kafka.UnboxTransform.unboxFloat; +import static io.deephaven.kafka.UnboxTransform.unboxInt; +import static io.deephaven.kafka.UnboxTransform.unboxLong; +import static io.deephaven.kafka.UnboxTransform.unboxShort; + +class ToChunkTypeTransform { + + private static final ToByteFunction BOOLEAN_AS_BYTE = BooleanUtils::booleanAsByte; + private static final ToLongFunction EPOCH_NANOS = DateTimeUtils::epochNanos; + + /** + * Transform the {@link TypedFunction function} {@code f} into its expected chunk type function. + * + * @param f the function + * @return the chunk type function + * @param the input type + * @see #of(ToPrimitiveFunction) + * @see #of(ToObjectFunction) + */ + public static TypedFunction of(TypedFunction f) { + return FunctionVisitor.of(f); + } + + /** + * Transform the {@link ToPrimitiveFunction function} {@code f} into its expected chunk type function. + * + * @param f the function + * @return the chunk type function + * @param the input type + */ + public static TypedFunction of(ToPrimitiveFunction f) { + return f; + } + + /** + * Transform the {@link ToPrimitiveFunction function} {@code f} into its expected chunk type function. + * + * @param f the Object function + * @return the chunk type function + * @param the input type + * @see #toEpochNanos(ToObjectFunction) + * @see #unboxBooleanAsByte(ToObjectFunction) + * @see UnboxTransform#unboxByte(ToObjectFunction) + * @see UnboxTransform#unboxChar(ToObjectFunction) + * @see UnboxTransform#unboxDouble(ToObjectFunction) + * @see UnboxTransform#unboxFloat(ToObjectFunction) + * @see UnboxTransform#unboxInt(ToObjectFunction) + * @see UnboxTransform#unboxLong(ToObjectFunction) + * @see UnboxTransform#unboxShort(ToObjectFunction) + */ + public static TypedFunction of(ToObjectFunction f) { + return ObjectFunctionVisitor.of(f); + } + + /** + * Equivalent to {@code f.mapByte(BooleanUtils::booleanAsByte)}. + * + * @param f the Boolean function + * @return the byte function + * @param the input type + * @see BooleanUtils#booleanAsByte(Boolean) + */ + public static ToByteFunction unboxBooleanAsByte(ToObjectFunction f) { + return f.mapToByte(BOOLEAN_AS_BYTE); + } + + /** + * Equivalent to {@code f.mapLong(DateTimeUtils::epochNanos)}. + * + * @param f the instant function + * @return the epoch nanos function + * @param the input type + * @see DateTimeUtils#epochNanos(Instant) + */ + public static ToLongFunction toEpochNanos(ToObjectFunction f) { + return f.mapToLong(EPOCH_NANOS); + } + + private static class FunctionVisitor implements TypedFunction.Visitor> { + + public static TypedFunction of(TypedFunction f) { + return f.walk(new FunctionVisitor<>()); + } + + private FunctionVisitor() {} + + @Override + public TypedFunction visit(ToPrimitiveFunction f) { + return ToChunkTypeTransform.of(f); + } + + @Override + public TypedFunction visit(ToObjectFunction f) { + return ToChunkTypeTransform.of(f); + } + } + + private static class ObjectFunctionVisitor implements + GenericType.Visitor>, + BoxedType.Visitor> { + + public static TypedFunction of(ToObjectFunction f) { + return f.returnType().walk(new ObjectFunctionVisitor<>(f)); + } + + private final ToObjectFunction f; + + private ObjectFunctionVisitor(ToObjectFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public TypedFunction visit(BoxedType boxedType) { + return boxedType.walk((BoxedType.Visitor>) this); + } + + @Override + public TypedFunction visit(InstantType instantType) { + return toEpochNanos(f.cast(instantType)); + } + + @Override + public TypedFunction visit(StringType stringType) { + return f; + } + + @Override + public TypedFunction visit(ArrayType arrayType) { + return f; + } + + @Override + public TypedFunction visit(CustomType customType) { + return f; + } + + @Override + public TypedFunction visit(BoxedBooleanType booleanType) { + return unboxBooleanAsByte(f.cast(booleanType)); + } + + @Override + public TypedFunction visit(BoxedByteType byteType) { + return unboxByte(f.cast(byteType)); + } + + @Override + public TypedFunction visit(BoxedCharType charType) { + return unboxChar(f.cast(charType)); + } + + @Override + public TypedFunction visit(BoxedShortType shortType) { + return unboxShort(f.cast(shortType)); + } + + @Override + public TypedFunction visit(BoxedIntType intType) { + return unboxInt(f.cast(intType)); + } + + @Override + public TypedFunction visit(BoxedLongType longType) { + return unboxLong(f.cast(longType)); + } + + @Override + public TypedFunction visit(BoxedFloatType floatType) { + return unboxFloat(f.cast(floatType)); + } + + @Override + public TypedFunction visit(BoxedDoubleType doubleType) { + return unboxDouble(f.cast(doubleType)); + } + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/UnboxTransform.java b/extensions/kafka/src/main/java/io/deephaven/kafka/UnboxTransform.java new file mode 100644 index 00000000000..9fb4122891a --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/UnboxTransform.java @@ -0,0 +1,267 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka; + +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToCharFunction; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToPrimitiveFunction; +import io.deephaven.functions.ToShortFunction; +import io.deephaven.functions.TypedFunction; +import io.deephaven.qst.type.ArrayType; +import io.deephaven.qst.type.BoxedBooleanType; +import io.deephaven.qst.type.BoxedByteType; +import io.deephaven.qst.type.BoxedCharType; +import io.deephaven.qst.type.BoxedDoubleType; +import io.deephaven.qst.type.BoxedFloatType; +import io.deephaven.qst.type.BoxedIntType; +import io.deephaven.qst.type.BoxedLongType; +import io.deephaven.qst.type.BoxedShortType; +import io.deephaven.qst.type.BoxedType; +import io.deephaven.qst.type.CustomType; +import io.deephaven.qst.type.GenericType; +import io.deephaven.qst.type.InstantType; +import io.deephaven.qst.type.StringType; +import io.deephaven.util.type.TypeUtils; + +import java.util.Objects; +import java.util.Optional; + +class UnboxTransform { + + private static final ToByteFunction UNBOX_BYTE = TypeUtils::unbox; + private static final ToCharFunction UNBOX_CHAR = TypeUtils::unbox; + private static final ToShortFunction UNBOX_SHORT = TypeUtils::unbox; + private static final ToIntFunction UNBOX_INT = TypeUtils::unbox; + private static final ToLongFunction UNBOX_LONG = TypeUtils::unbox; + private static final ToFloatFunction UNBOX_FLOAT = TypeUtils::unbox; + private static final ToDoubleFunction UNBOX_DOULE = TypeUtils::unbox; + + /** + * Returns the Deephaven unboxed equivalent of {@code f}. Relevant for all {@link BoxedType boxed types} except the + * {@link BoxedBooleanType boxed Boolean type}. All other functions will be return unchanged. + * + * @param f the function + * @return the unboxed equivalent + * @param the input type + * @see #unboxByte(ToObjectFunction) + * @see #unboxChar(ToObjectFunction) + * @see #unboxDouble(ToObjectFunction) + * @see #unboxFloat(ToObjectFunction) + * @see #unboxInt(ToObjectFunction) + * @see #unboxLong(ToObjectFunction) + * @see #unboxShort(ToObjectFunction) + */ + public static Optional> of(TypedFunction f) { + return UnboxFunctionVisitor.of(f); + } + + /** + * Returns the Deephaven unboxed equivalent of {@code f}. Relevant for all {@link BoxedType boxed types} except the + * {@link BoxedBooleanType boxed Boolean type}. All other functions will be return unchanged. + * + * @param f the object function + * @return the unboxed equivalent + * @param the input type + * @see #unboxByte(ToObjectFunction) + * @see #unboxChar(ToObjectFunction) + * @see #unboxDouble(ToObjectFunction) + * @see #unboxFloat(ToObjectFunction) + * @see #unboxInt(ToObjectFunction) + * @see #unboxLong(ToObjectFunction) + * @see #unboxShort(ToObjectFunction) + */ + public static Optional> of(ToObjectFunction f) { + return UnboxObjectFunctionVisitor.of(f); + } + + /** + * Equivalent to {@code f.mapByte(TypeUtils::unbox)}. + * + * @param f the Byte function + * @return the byte function + * @param the input type + * @see TypeUtils#unbox(Byte) + */ + public static ToByteFunction unboxByte(ToObjectFunction f) { + return f.mapToByte(UNBOX_BYTE); + } + + /** + * Equivalent to {@code f.mapChar(TypeUtils::unbox)}. + * + * @param f the Character function + * @return the char function + * @param the input type + * @see TypeUtils#unbox(Character) + */ + public static ToCharFunction unboxChar(ToObjectFunction f) { + return f.mapToChar(UNBOX_CHAR); + } + + /** + * Equivalent to {@code f.mapShort(TypeUtils::unbox)}. + * + * @param f the Short function + * @return the short function + * @param the input type + * @see TypeUtils#unbox(Short) + */ + public static ToShortFunction unboxShort(ToObjectFunction f) { + return f.mapToShort(UNBOX_SHORT); + } + + /** + * Equivalent to {@code f.mapInt(TypeUtils::unbox)}. + * + * @param f the Integer function + * @return the int function + * @param the input type + * @see TypeUtils#unbox(Integer) + */ + public static ToIntFunction unboxInt(ToObjectFunction f) { + return f.mapToInt(UNBOX_INT); + } + + /** + * Equivalent to {@code f.mapLong(TypeUtils::unbox)}. + * + * @param f the Long function + * @return the long function + * @param the input type + * @see TypeUtils#unbox(Long) + */ + public static ToLongFunction unboxLong(ToObjectFunction f) { + return f.mapToLong(UNBOX_LONG); + } + + /** + * Equivalent to {@code f.mapFloat(TypeUtils::unbox)}. + * + * @param f the Float function + * @return the float function + * @param the input type + * @see TypeUtils#unbox(Float) + */ + public static ToFloatFunction unboxFloat(ToObjectFunction f) { + return f.mapToFloat(UNBOX_FLOAT); + } + + /** + * Equivalent to {@code f.mapDouble(TypeUtils::unbox)}. + * + * @param f the Double function + * @return the double function + * @param the input type + * @see TypeUtils#unbox(Double) + */ + public static ToDoubleFunction unboxDouble(ToObjectFunction f) { + return f.mapToDouble(UNBOX_DOULE); + } + + private enum UnboxFunctionVisitor implements TypedFunction.Visitor> { + INSTANCE; + + public static Optional> of(TypedFunction f) { + // noinspection unchecked + return Optional.ofNullable( + f.walk((TypedFunction.Visitor>) (TypedFunction.Visitor) INSTANCE)); + } + + @Override + public ToPrimitiveFunction visit(ToPrimitiveFunction f) { + return f; + } + + @Override + public ToPrimitiveFunction visit(ToObjectFunction f) { + return UnboxTransform.of(f).orElse(null); + } + } + + private static class UnboxObjectFunctionVisitor + implements GenericType.Visitor>, BoxedType.Visitor> { + + public static Optional> of(ToObjectFunction f) { + return Optional.ofNullable(f.returnType().walk(new UnboxObjectFunctionVisitor<>(f))); + } + + private final ToObjectFunction f; + + public UnboxObjectFunctionVisitor(ToObjectFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public ToPrimitiveFunction visit(BoxedType boxedType) { + return boxedType.walk((BoxedType.Visitor>) this); + } + + @Override + public ToPrimitiveFunction visit(StringType stringType) { + return null; + } + + @Override + public ToPrimitiveFunction visit(InstantType instantType) { + return null; + } + + @Override + public ToPrimitiveFunction visit(ArrayType arrayType) { + return null; + } + + @Override + public ToPrimitiveFunction visit(CustomType customType) { + return null; + } + + @Override + public ToPrimitiveFunction visit(BoxedBooleanType booleanType) { + // We don't have an "unboxed boolean". + // We _can_ transform it to a byte, but that's a separate operation. + return null; + } + + @Override + public ToPrimitiveFunction visit(BoxedByteType byteType) { + return unboxByte(f.cast(byteType)); + } + + @Override + public ToPrimitiveFunction visit(BoxedCharType charType) { + return unboxChar(f.cast(charType)); + } + + @Override + public ToPrimitiveFunction visit(BoxedShortType shortType) { + return unboxShort(f.cast(shortType)); + } + + @Override + public ToPrimitiveFunction visit(BoxedIntType intType) { + return unboxInt(f.cast(intType)); + } + + @Override + public ToPrimitiveFunction visit(BoxedLongType longType) { + return unboxLong(f.cast(longType)); + } + + @Override + public ToPrimitiveFunction visit(BoxedFloatType floatType) { + return unboxFloat(f.cast(floatType)); + } + + @Override + public ToPrimitiveFunction visit(BoxedDoubleType doubleType) { + return unboxDouble(f.cast(doubleType)); + } + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/BooleanFieldCopier.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/BooleanFieldCopier.java new file mode 100644 index 00000000000..96c3b69f025 --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/BooleanFieldCopier.java @@ -0,0 +1,33 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.ingest; + +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.functions.ToBooleanFunction; + +import java.util.Objects; + +class BooleanFieldCopier implements FieldCopier { + public static BooleanFieldCopier of(ToBooleanFunction f) { + return new BooleanFieldCopier(f); + } + + private final ToBooleanFunction f; + + private BooleanFieldCopier(ToBooleanFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public void copyField( + ObjectChunk inputChunk, + WritableChunk publisherChunk, + int sourceOffset, + int destOffset, + int length) { + ChunkUtils.applyInto(f, inputChunk, sourceOffset, publisherChunk.asWritableBooleanChunk(), destOffset, length); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ByteFieldCopier.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ByteFieldCopier.java new file mode 100644 index 00000000000..077d8646c9a --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ByteFieldCopier.java @@ -0,0 +1,44 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.ingest; + +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.util.BooleanUtils; +import io.deephaven.util.type.TypeUtils; + +import java.util.Objects; + +class ByteFieldCopier implements FieldCopier { + public static ByteFieldCopier of(ToByteFunction f) { + return new ByteFieldCopier(f); + } + + public static ByteFieldCopier of(ToObjectFunction f) { + return of(f.mapToByte(TypeUtils::unbox)); + } + + public static ByteFieldCopier ofBoolean(ToObjectFunction f) { + return of(f.mapToByte(BooleanUtils::booleanAsByte)); + } + + private final ToByteFunction f; + + private ByteFieldCopier(ToByteFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public void copyField( + ObjectChunk inputChunk, + WritableChunk publisherChunk, + int sourceOffset, + int destOffset, + int length) { + ChunkUtils.applyInto(f, inputChunk, sourceOffset, publisherChunk.asWritableByteChunk(), destOffset, length); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/CharFieldCopier.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/CharFieldCopier.java new file mode 100644 index 00000000000..0eb3b5b3f81 --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/CharFieldCopier.java @@ -0,0 +1,39 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.ingest; + +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.functions.ToCharFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.util.type.TypeUtils; + +import java.util.Objects; + +class CharFieldCopier implements FieldCopier { + public static CharFieldCopier of(ToCharFunction f) { + return new CharFieldCopier(f); + } + + public static CharFieldCopier of(ToObjectFunction f) { + return of(f.mapToChar(TypeUtils::unbox)); + } + + private final ToCharFunction f; + + private CharFieldCopier(ToCharFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public void copyField( + ObjectChunk inputChunk, + WritableChunk publisherChunk, + int sourceOffset, + int destOffset, + int length) { + ChunkUtils.applyInto(f, inputChunk, sourceOffset, publisherChunk.asWritableCharChunk(), destOffset, length); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ChunkUtils.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ChunkUtils.java new file mode 100644 index 00000000000..4b08821302f --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ChunkUtils.java @@ -0,0 +1,136 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.ingest; + +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableBooleanChunk; +import io.deephaven.chunk.WritableByteChunk; +import io.deephaven.chunk.WritableCharChunk; +import io.deephaven.chunk.WritableDoubleChunk; +import io.deephaven.chunk.WritableFloatChunk; +import io.deephaven.chunk.WritableIntChunk; +import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.chunk.WritableShortChunk; +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToCharFunction; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToShortFunction; + +class ChunkUtils { + + public static void applyInto( + ToBooleanFunction booleanFunction, + ObjectChunk src, + int srcOffset, + WritableBooleanChunk dest, + int destOffset, + int length) { + for (int i = 0; i < length; ++i) { + dest.set(destOffset + i, booleanFunction.test(src.get(i + srcOffset))); + } + } + + public static void applyInto( + ToByteFunction byteFunction, + ObjectChunk src, + int srcOffset, + WritableByteChunk dest, + int destOffset, + int length) { + for (int i = 0; i < length; ++i) { + dest.set(destOffset + i, byteFunction.applyAsByte(src.get(srcOffset + i))); + } + } + + public static void applyInto( + ToCharFunction charFunction, + ObjectChunk src, + int srcOffset, + WritableCharChunk dest, + int destOffset, + int length) { + for (int i = 0; i < length; ++i) { + dest.set(destOffset + i, charFunction.applyAsChar(src.get(srcOffset + i))); + } + } + + public static void applyInto( + ToShortFunction shortFunction, + ObjectChunk src, + int srcOffset, + WritableShortChunk dest, + int destOffset, + int length) { + for (int i = 0; i < length; ++i) { + dest.set(destOffset + i, shortFunction.applyAsShort(src.get(srcOffset + i))); + } + } + + public static void applyInto( + ToIntFunction intFunction, + ObjectChunk src, + int srcOffset, + WritableIntChunk dest, + int destOffset, + int length) { + for (int i = 0; i < length; ++i) { + dest.set(destOffset + i, intFunction.applyAsInt(src.get(srcOffset + i))); + } + } + + + public static void applyInto( + ToLongFunction longFunction, + ObjectChunk src, + int srcOffset, + WritableLongChunk dest, + int destOffset, + int length) { + for (int i = 0; i < length; ++i) { + dest.set(destOffset + i, longFunction.applyAsLong(src.get(srcOffset + i))); + } + } + + public static void applyInto( + ToFloatFunction floatFunction, + ObjectChunk src, + int srcOffset, + WritableFloatChunk dest, + int destOffset, + int length) { + for (int i = 0; i < length; ++i) { + dest.set(destOffset + i, floatFunction.applyAsFloat(src.get(srcOffset + i))); + } + } + + public static void applyInto( + ToDoubleFunction doubleFunction, + ObjectChunk src, + int srcOffset, + WritableDoubleChunk dest, + int destOffset, + int length) { + for (int i = 0; i < length; ++i) { + dest.set(destOffset + i, doubleFunction.applyAsDouble(src.get(srcOffset + i))); + } + } + + public static void applyInto( + ToObjectFunction objFunction, + ObjectChunk src, + int srcOffset, + WritableObjectChunk dest, + int destOffset, + int length) { + for (int i = 0; i < length; ++i) { + dest.set(destOffset + i, objFunction.apply(src.get(srcOffset + i))); + } + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/DoubleFieldCopier.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/DoubleFieldCopier.java new file mode 100644 index 00000000000..1b82518af52 --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/DoubleFieldCopier.java @@ -0,0 +1,39 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.ingest; + +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.util.type.TypeUtils; + +import java.util.Objects; + +class DoubleFieldCopier implements FieldCopier { + public static DoubleFieldCopier of(ToDoubleFunction f) { + return new DoubleFieldCopier(f); + } + + public static DoubleFieldCopier of(ToObjectFunction f) { + return of(f.mapToDouble(TypeUtils::unbox)); + } + + private final ToDoubleFunction f; + + private DoubleFieldCopier(ToDoubleFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public void copyField( + ObjectChunk inputChunk, + WritableChunk publisherChunk, + int sourceOffset, + int destOffset, + int length) { + ChunkUtils.applyInto(f, inputChunk, sourceOffset, publisherChunk.asWritableDoubleChunk(), destOffset, length); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/FieldCopierAdapter.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/FieldCopierAdapter.java new file mode 100644 index 00000000000..896dc8749e4 --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/FieldCopierAdapter.java @@ -0,0 +1,119 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.ingest; + +import io.deephaven.qst.type.BoxedBooleanType; +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToCharFunction; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToPrimitiveFunction; +import io.deephaven.functions.ToShortFunction; +import io.deephaven.functions.TypedFunction; + +public enum FieldCopierAdapter + implements TypedFunction.Visitor, ToPrimitiveFunction.Visitor { + INSTANCE; + + public static FieldCopier of(TypedFunction f) { + return f.walk(INSTANCE); + } + + public static FieldCopier of(ToPrimitiveFunction f) { + return f.walk((ToPrimitiveFunction.Visitor) INSTANCE); + } + + public static FieldCopier of(ToBooleanFunction f) { + return BooleanFieldCopier.of(f); + } + + public static FieldCopier of(ToCharFunction f) { + return CharFieldCopier.of(f); + } + + public static FieldCopier of(ToByteFunction f) { + return ByteFieldCopier.of(f); + } + + public static FieldCopier of(ToShortFunction f) { + return ShortFieldCopier.of(f); + } + + public static FieldCopier of(ToIntFunction f) { + return IntFieldCopier.of(f); + } + + public static FieldCopier of(ToLongFunction f) { + return LongFieldCopier.of(f); + } + + public static FieldCopier of(ToFloatFunction f) { + return FloatFieldCopier.of(f); + } + + public static FieldCopier of(ToDoubleFunction f) { + return DoubleFieldCopier.of(f); + } + + public static FieldCopier of(ToObjectFunction f) { + if (f.returnType().equals(BoxedBooleanType.of())) { + return ByteFieldCopier.ofBoolean(f.cast(BoxedBooleanType.of())); + } + return ObjectFieldCopier.of(f); + } + + @Override + public FieldCopier visit(ToPrimitiveFunction f) { + return of(f); + } + + @Override + public FieldCopier visit(ToBooleanFunction f) { + return of(f); + } + + @Override + public FieldCopier visit(ToCharFunction f) { + return of(f); + } + + @Override + public FieldCopier visit(ToByteFunction f) { + return of(f); + } + + @Override + public FieldCopier visit(ToShortFunction f) { + return of(f); + } + + @Override + public FieldCopier visit(ToIntFunction f) { + return of(f); + } + + @Override + public FieldCopier visit(ToLongFunction f) { + return of(f); + } + + @Override + public FieldCopier visit(ToFloatFunction f) { + return of(f); + } + + @Override + public FieldCopier visit(ToDoubleFunction f) { + return of(f); + } + + @Override + public FieldCopier visit(ToObjectFunction f) { + return of(f); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/FloatFieldCopier.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/FloatFieldCopier.java new file mode 100644 index 00000000000..8be6811daae --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/FloatFieldCopier.java @@ -0,0 +1,39 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.ingest; + +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.util.type.TypeUtils; + +import java.util.Objects; + +class FloatFieldCopier implements FieldCopier { + public static FloatFieldCopier of(ToFloatFunction f) { + return new FloatFieldCopier(f); + } + + public static FloatFieldCopier of(ToObjectFunction f) { + return of(f.mapToFloat(TypeUtils::unbox)); + } + + private final ToFloatFunction f; + + private FloatFieldCopier(ToFloatFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public void copyField( + ObjectChunk inputChunk, + WritableChunk publisherChunk, + int sourceOffset, + int destOffset, + int length) { + ChunkUtils.applyInto(f, inputChunk, sourceOffset, publisherChunk.asWritableFloatChunk(), destOffset, length); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/IntFieldCopier.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/IntFieldCopier.java new file mode 100644 index 00000000000..6e73a805f5a --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/IntFieldCopier.java @@ -0,0 +1,39 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.ingest; + +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.util.type.TypeUtils; + +import java.util.Objects; + +class IntFieldCopier implements FieldCopier { + public static IntFieldCopier of(ToIntFunction f) { + return new IntFieldCopier(f); + } + + public static IntFieldCopier of(ToObjectFunction f) { + return of(f.mapToInt(TypeUtils::unbox)); + } + + private final ToIntFunction f; + + private IntFieldCopier(ToIntFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public void copyField( + ObjectChunk inputChunk, + WritableChunk publisherChunk, + int sourceOffset, + int destOffset, + int length) { + ChunkUtils.applyInto(f, inputChunk, sourceOffset, publisherChunk.asWritableIntChunk(), destOffset, length); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/LongFieldCopier.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/LongFieldCopier.java new file mode 100644 index 00000000000..78ccbd46051 --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/LongFieldCopier.java @@ -0,0 +1,39 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.ingest; + +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.util.type.TypeUtils; + +import java.util.Objects; + +class LongFieldCopier implements FieldCopier { + public static LongFieldCopier of(ToLongFunction f) { + return new LongFieldCopier(f); + } + + public static LongFieldCopier of(ToObjectFunction f) { + return of(f.mapToLong(TypeUtils::unbox)); + } + + private final ToLongFunction f; + + private LongFieldCopier(ToLongFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public void copyField( + ObjectChunk inputChunk, + WritableChunk publisherChunk, + int sourceOffset, + int destOffset, + int length) { + ChunkUtils.applyInto(f, inputChunk, sourceOffset, publisherChunk.asWritableLongChunk(), destOffset, length); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/MultiFieldChunkAdapter.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/MultiFieldChunkAdapter.java index 2b76006c031..62cb34b637d 100644 --- a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/MultiFieldChunkAdapter.java +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/MultiFieldChunkAdapter.java @@ -17,6 +17,27 @@ import java.util.function.IntFunction; public class MultiFieldChunkAdapter implements KeyOrValueProcessor { + + public static int[] chunkOffsets( + final TableDefinition definition, + final Map fieldNamesToColumnNames) { + final String[] columnNames = definition.getColumnNamesArray(); + final TObjectIntMap deephavenColumnNameToIndex = new TObjectIntHashMap<>(columnNames.length, 0.5f, -1); + for (int ii = 0; ii < columnNames.length; ++ii) { + deephavenColumnNameToIndex.put(columnNames[ii], ii); + } + final int[] chunkOffsets = new int[fieldNamesToColumnNames.size()]; + int col = 0; + for (String columnName : fieldNamesToColumnNames.values()) { + final int deephavenColumnIndex = deephavenColumnNameToIndex.get(columnName); + if (deephavenColumnIndex == deephavenColumnNameToIndex.getNoEntryValue()) { + throw new IllegalArgumentException("Column not found in Deephaven table: " + deephavenColumnIndex); + } + chunkOffsets[col++] = deephavenColumnIndex; + } + return chunkOffsets; + } + private final boolean allowNulls; private final int[] chunkOffsets; diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ObjectFieldCopier.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ObjectFieldCopier.java new file mode 100644 index 00000000000..7bdf39051d8 --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ObjectFieldCopier.java @@ -0,0 +1,33 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.ingest; + +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.functions.ToObjectFunction; + +import java.util.Objects; + +class ObjectFieldCopier implements FieldCopier { + public static ObjectFieldCopier of(ToObjectFunction f) { + return new ObjectFieldCopier(f); + } + + private final ToObjectFunction f; + + private ObjectFieldCopier(ToObjectFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public void copyField( + ObjectChunk inputChunk, + WritableChunk publisherChunk, + int sourceOffset, + int destOffset, + int length) { + ChunkUtils.applyInto(f, inputChunk, sourceOffset, publisherChunk.asWritableObjectChunk(), destOffset, length); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ShortFieldCopier.java b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ShortFieldCopier.java new file mode 100644 index 00000000000..a14cab19ada --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/ingest/ShortFieldCopier.java @@ -0,0 +1,39 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.ingest; + +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToShortFunction; +import io.deephaven.util.type.TypeUtils; + +import java.util.Objects; + +class ShortFieldCopier implements FieldCopier { + public static ShortFieldCopier of(ToShortFunction f) { + return new ShortFieldCopier(f); + } + + public static ShortFieldCopier of(ToObjectFunction f) { + return of(f.mapToShort(TypeUtils::unbox)); + } + + private final ToShortFunction f; + + private ShortFieldCopier(ToShortFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public void copyField( + ObjectChunk inputChunk, + WritableChunk publisherChunk, + int sourceOffset, + int destOffset, + int length) { + ChunkUtils.applyInto(f, inputChunk, sourceOffset, publisherChunk.asWritableShortChunk(), destOffset, length); + } +} diff --git a/extensions/kafka/src/main/java/io/deephaven/kafka/protobuf/ProtobufConsumeOptions.java b/extensions/kafka/src/main/java/io/deephaven/kafka/protobuf/ProtobufConsumeOptions.java new file mode 100644 index 00000000000..2066e53887a --- /dev/null +++ b/extensions/kafka/src/main/java/io/deephaven/kafka/protobuf/ProtobufConsumeOptions.java @@ -0,0 +1,138 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.kafka.protobuf; + +import com.google.protobuf.Descriptors.Descriptor; +import io.deephaven.annotations.BuildableStyle; +import io.deephaven.api.ColumnName; +import io.deephaven.kafka.KafkaTools.Consume; +import io.deephaven.protobuf.FieldPath; +import io.deephaven.protobuf.ProtobufDescriptorParserOptions; +import org.immutables.value.Value.Default; +import org.immutables.value.Value.Immutable; + +import java.util.Optional; +import java.util.OptionalInt; +import java.util.function.Function; + +/** + * The kafka protobuf options. This will fetch the {@link com.google.protobuf.Descriptors.Descriptor protobuf + * descriptor} for the {@link #schemaSubject()} from the schema registry using version {@link #schemaVersion()} and + * create {@link com.google.protobuf.Message message} parsing functions according to + * {@link io.deephaven.protobuf.ProtobufDescriptorParser#parse(Descriptor, ProtobufDescriptorParserOptions)}. These + * functions will be adapted to handle schema changes. + * + * @see Consume#protobufSpec(ProtobufConsumeOptions) + * @see kafka + * protobuf serdes + */ +@Immutable +@BuildableStyle +public abstract class ProtobufConsumeOptions { + + @FunctionalInterface + public interface FieldPathToColumnName { + /** + * Creates a unique column name from {@code fieldPath} and {@code indexOccurrence}. Implementations will need to + * take notice when {@code indexOccurrence > 0}, as that means a column name for {@code fieldPath} has already + * been generated {@code indexOccurrence} times. + * + * @param fieldPath the field path + * @param indexOccurrence the number of times a column name for fieldPath has already been generated + * @return the column name + */ + ColumnName columnName(FieldPath fieldPath, int indexOccurrence); + } + + /** + * The builder. + * + * @return the builder + */ + public static Builder builder() { + return ImmutableProtobufConsumeOptions.builder(); + } + + /** + * Joins the name paths with underscores, appending {@code indexOccurrence + 1} if {@code indexOccurrence != 0}. + * + * @param path the path + * @param indexOccurrence the number of times this field path has been used + * @return the underscore joined path names + */ + public static ColumnName joinNamePathWithUnderscore(FieldPath path, int indexOccurrence) { + final String simple = String.join("_", path.namePath()); + return ColumnName.of(indexOccurrence == 0 ? simple : simple + "_" + (indexOccurrence + 1)); + } + + /** + * The schema subject to fetch from the schema registry. + * + * @return the schema subject + */ + public abstract String schemaSubject(); + + /** + * The schema version to fetch from the schema registry. When not set, the latest schema will be fetched. + * + *

+ * For purposes of reproducibility across restarts where schema changes may occur, it is advisable for callers to + * set this. This will ensure the resulting {@link io.deephaven.engine.table.TableDefinition table definition} will + * not change across restarts. This gives the caller an explicit opportunity to update any downstream consumers + * before bumping schema versions. + * + * @return the schema version, or none for latest + */ + public abstract OptionalInt schemaVersion(); + + /** + * The fully-qualified protobuf {@link com.google.protobuf.Message} name, for example "com.example.MyMessage". This + * message's {@link Descriptor} will be used as the basis for the resulting table's + * {@link io.deephaven.engine.table.TableDefinition definition}. When not set, the first message descriptor in the + * protobuf schema will be used. + * + *

+ * It is advisable for callers to explicitly set this. + * + * @return the schema message name + */ + public abstract Optional schemaMessageName(); + + /** + * The descriptor parsing options. By default, is {@link ProtobufDescriptorParserOptions#defaults()}. + * + * @return the descriptor parsing options + */ + @Default + public ProtobufDescriptorParserOptions parserOptions() { + return ProtobufDescriptorParserOptions.defaults(); + } + + /** + * The function to turn field paths into column names. By default, is the function + * {@link #joinNamePathWithUnderscore(FieldPath, int)}}. + * + * @return the function to create column names + */ + @Default + public FieldPathToColumnName pathToColumnName() { + return ProtobufConsumeOptions::joinNamePathWithUnderscore; + } + + public interface Builder { + + Builder schemaSubject(String schemaSubject); + + Builder schemaVersion(int schemaVersion); + + Builder schemaMessageName(String schemaMessageName); + + Builder parserOptions(ProtobufDescriptorParserOptions options); + + Builder pathToColumnName(FieldPathToColumnName pathToColumnName); + + ProtobufConsumeOptions build(); + } +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/ProtobufImplSchemaChangeTest.java b/extensions/kafka/src/test/java/io/deephaven/kafka/ProtobufImplSchemaChangeTest.java new file mode 100644 index 00000000000..fe14bc65da1 --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/ProtobufImplSchemaChangeTest.java @@ -0,0 +1,543 @@ +package io.deephaven.kafka; + +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Message; +import com.google.protobuf.Timestamp; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.TypedFunction; +import io.deephaven.kafka.protobuf.gen.BoolV1; +import io.deephaven.kafka.protobuf.gen.BoolV2; +import io.deephaven.kafka.protobuf.gen.MyMessageV1; +import io.deephaven.kafka.protobuf.gen.MyMessageV2; +import io.deephaven.kafka.protobuf.gen.MyMessageV3; +import io.deephaven.kafka.protobuf.gen.MyMessageV3.MyMessage.FirstAndLast; +import io.deephaven.kafka.protobuf.gen.MyMessageV4; +import io.deephaven.kafka.protobuf.gen.MyMessageV5; +import io.deephaven.kafka.protobuf.gen.RenameV1; +import io.deephaven.kafka.protobuf.gen.RenameV2; +import io.deephaven.kafka.protobuf.gen.SpecialTypesV1; +import io.deephaven.kafka.protobuf.gen.SpecialTypesV2; +import io.deephaven.protobuf.FieldOptions; +import io.deephaven.protobuf.FieldOptions.Builder; +import io.deephaven.protobuf.FieldOptions.BytesBehavior; +import io.deephaven.protobuf.FieldOptions.MapBehavior; +import io.deephaven.protobuf.FieldOptions.WellKnownBehavior; +import io.deephaven.protobuf.FieldPath; +import io.deephaven.protobuf.ProtobufDescriptorParserOptions; +import io.deephaven.protobuf.ProtobufFunction; +import io.deephaven.protobuf.ProtobufFunctions; +import io.deephaven.util.QueryConstants; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.time.Instant; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; + +import static io.deephaven.functions.ToBooleanFunction.map; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; + + +/** + * See notes in TESTING.md for adding new tests here. + */ +public class ProtobufImplSchemaChangeTest { + + @Test + public void myMessageV1toV2() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(MyMessageV1.MyMessage.getDescriptor()); + assertThat(functions.functions()).hasSize(1); + final ToObjectFunction nameFunction = + (ToObjectFunction) get(functions, "name"); + { + final MyMessageV1.MyMessage v1 = MyMessageV1.MyMessage.newBuilder().setName("v1").build(); + assertThat(nameFunction.apply(v1)).isEqualTo("v1"); + } + { + final MyMessageV2.MyMessage v2 = MyMessageV2.MyMessage.newBuilder().setName("v2").setAge(2).build(); + assertThat(nameFunction.apply(v2)).isEqualTo("v2"); + } + } + + @Test + public void myMessageV2toV1() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(MyMessageV2.MyMessage.getDescriptor()); + assertThat(functions.functions()).hasSize(2); + final ToObjectFunction nameFunction = + (ToObjectFunction) get(functions, "name"); + final ToIntFunction ageFunction = (ToIntFunction) get(functions, "age"); + { + final MyMessageV2.MyMessage v2 = MyMessageV2.MyMessage.newBuilder().setName("v2").setAge(2).build(); + assertThat(nameFunction.apply(v2)).isEqualTo("v2"); + assertThat(ageFunction.applyAsInt(v2)).isEqualTo(2); + } + { + final MyMessageV1.MyMessage v1 = MyMessageV1.MyMessage.newBuilder().setName("v1").build(); + assertThat(nameFunction.apply(v1)).isEqualTo("v1"); + assertThat(ageFunction.applyAsInt(v1)).isEqualTo(QueryConstants.NULL_INT); + } + } + + @Test + public void myMessageV2toV3() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(MyMessageV2.MyMessage.getDescriptor()); + assertThat(functions.functions()).hasSize(2); + final ToObjectFunction nameFunction = + (ToObjectFunction) get(functions, "name"); + final ToIntFunction ageFunction = (ToIntFunction) get(functions, "age"); + { + final MyMessageV2.MyMessage v2 = MyMessageV2.MyMessage.newBuilder().setName("v2").setAge(2).build(); + assertThat(nameFunction.apply(v2)).isEqualTo("v2"); + assertThat(ageFunction.applyAsInt(v2)).isEqualTo(2); + } + { + final MyMessageV2.MyMessage v2 = MyMessageV2.MyMessage.newBuilder().setAge(2).build(); + assertThat(nameFunction.apply(v2)).isEmpty(); + assertThat(ageFunction.applyAsInt(v2)).isEqualTo(2); + } + { + final MyMessageV3.MyMessage v3 = MyMessageV3.MyMessage.newBuilder().setName("v3").setAge(3).build(); + assertThat(nameFunction.apply(v3)).isEqualTo("v3"); + assertThat(ageFunction.applyAsInt(v3)).isEqualTo(3); + } + { + final MyMessageV3.MyMessage v3 = MyMessageV3.MyMessage.newBuilder() + .setFirstAndLast(FirstAndLast.newBuilder().setFirstName("First").setLastName("Last").build()) + .setAge(3) + .build(); + assertThat(nameFunction.apply(v3)).isNull(); + assertThat(ageFunction.applyAsInt(v3)).isEqualTo(3); + } + { + final MyMessageV3.MyMessage v3 = MyMessageV3.MyMessage.newBuilder().setAge(3).build(); + assertThat(nameFunction.apply(v3)).isNull(); + assertThat(ageFunction.applyAsInt(v3)).isEqualTo(3); + } + } + + @Test + public void myMessageV3toV2() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(MyMessageV3.MyMessage.getDescriptor()); + assertThat(functions.functions()).hasSize(4); + final ToObjectFunction nameFunction = + (ToObjectFunction) get(functions, "name"); + final ToObjectFunction firstNameFunction = + (ToObjectFunction) get(functions, "first_and_last", "first_name"); + final ToObjectFunction lastNameFunction = + (ToObjectFunction) get(functions, "first_and_last", "last_name"); + final ToIntFunction ageFunction = (ToIntFunction) get(functions, "age"); + { + final MyMessageV3.MyMessage v3 = MyMessageV3.MyMessage.newBuilder().setName("v3").setAge(3).build(); + assertThat(nameFunction.apply(v3)).isEqualTo("v3"); + assertThat(firstNameFunction.apply(v3)).isNull(); + assertThat(lastNameFunction.apply(v3)).isNull(); + assertThat(ageFunction.applyAsInt(v3)).isEqualTo(3); + } + { + final MyMessageV3.MyMessage v3 = MyMessageV3.MyMessage.newBuilder() + .setFirstAndLast(FirstAndLast.newBuilder().setFirstName("First").setLastName("Last").build()) + .setAge(3) + .build(); + assertThat(nameFunction.apply(v3)).isNull(); + assertThat(firstNameFunction.apply(v3)).isEqualTo("First"); + assertThat(lastNameFunction.apply(v3)).isEqualTo("Last"); + assertThat(ageFunction.applyAsInt(v3)).isEqualTo(3); + } + { + final MyMessageV3.MyMessage v3 = MyMessageV3.MyMessage.newBuilder().setAge(3).build(); + assertThat(nameFunction.apply(v3)).isNull(); + assertThat(firstNameFunction.apply(v3)).isNull(); + assertThat(lastNameFunction.apply(v3)).isNull(); + assertThat(ageFunction.applyAsInt(v3)).isEqualTo(3); + } + { + final MyMessageV2.MyMessage v2 = MyMessageV2.MyMessage.newBuilder().setName("v2").setAge(2).build(); + assertThat(nameFunction.apply(v2)).isEqualTo("v2"); + assertThat(firstNameFunction.apply(v2)).isNull(); + assertThat(lastNameFunction.apply(v2)).isNull(); + assertThat(ageFunction.applyAsInt(v2)).isEqualTo(2); + } + { + final MyMessageV2.MyMessage v2 = MyMessageV2.MyMessage.newBuilder().setAge(2).build(); + assertThat(nameFunction.apply(v2)).isEmpty(); + assertThat(firstNameFunction.apply(v2)).isNull(); + assertThat(lastNameFunction.apply(v2)).isNull(); + assertThat(ageFunction.applyAsInt(v2)).isEqualTo(2); + } + } + + @Test + public void myMessageV3toV4() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(MyMessageV3.MyMessage.getDescriptor()); + assertThat(functions.functions()).hasSize(4); + final ToObjectFunction nameFunction = + (ToObjectFunction) get(functions, "name"); + final ToIntFunction ageFunction = (ToIntFunction) get(functions, "age"); + { + final MyMessageV3.MyMessage v3 = MyMessageV3.MyMessage.newBuilder().setName("v3").setAge(3).build(); + assertThat(nameFunction.apply(v3)).isEqualTo("v3"); + assertThat(ageFunction.applyAsInt(v3)).isEqualTo(3); + } + { + final MyMessageV3.MyMessage v3 = MyMessageV3.MyMessage.newBuilder().setName("v3").build(); + assertThat(nameFunction.apply(v3)).isEqualTo("v3"); + assertThat(ageFunction.applyAsInt(v3)).isEqualTo(0); + } + { + final MyMessageV4.MyMessage v4 = MyMessageV4.MyMessage.newBuilder().setName("v4").setAge(4).build(); + assertThat(nameFunction.apply(v4)).isEqualTo("v4"); + assertThat(ageFunction.applyAsInt(v4)).isEqualTo(4); + } + { + final MyMessageV4.MyMessage v4 = MyMessageV4.MyMessage.newBuilder().setName("v4").setAgef(4.4f).build(); + assertThat(nameFunction.apply(v4)).isEqualTo("v4"); + assertThat(ageFunction.applyAsInt(v4)).isEqualTo(QueryConstants.NULL_INT); + } + { + final MyMessageV4.MyMessage v4 = MyMessageV4.MyMessage.newBuilder().setName("v4").build(); + assertThat(nameFunction.apply(v4)).isEqualTo("v4"); + assertThat(ageFunction.applyAsInt(v4)).isEqualTo(QueryConstants.NULL_INT); + } + } + + @Test + public void myMessageV4toV3() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(MyMessageV4.MyMessage.getDescriptor()); + assertThat(functions.functions()).hasSize(5); + final ToObjectFunction nameFunction = + (ToObjectFunction) get(functions, "name"); + final ToIntFunction ageFunction = (ToIntFunction) get(functions, "age"); + final ToFloatFunction agefFunction = (ToFloatFunction) get(functions, "agef"); + { + final MyMessageV4.MyMessage v4 = MyMessageV4.MyMessage.newBuilder().setName("v4").setAge(4).build(); + assertThat(nameFunction.apply(v4)).isEqualTo("v4"); + assertThat(ageFunction.applyAsInt(v4)).isEqualTo(4); + assertThat(agefFunction.applyAsFloat(v4)).isEqualTo(QueryConstants.NULL_FLOAT); + } + { + final MyMessageV4.MyMessage v4 = MyMessageV4.MyMessage.newBuilder().setName("v4").setAgef(4.4f).build(); + assertThat(nameFunction.apply(v4)).isEqualTo("v4"); + assertThat(ageFunction.applyAsInt(v4)).isEqualTo(QueryConstants.NULL_INT); + assertThat(agefFunction.applyAsFloat(v4)).isEqualTo(4.4f); + } + { + final MyMessageV4.MyMessage v4 = MyMessageV4.MyMessage.newBuilder().setName("v4").build(); + assertThat(nameFunction.apply(v4)).isEqualTo("v4"); + assertThat(ageFunction.applyAsInt(v4)).isEqualTo(QueryConstants.NULL_INT); + assertThat(agefFunction.applyAsFloat(v4)).isEqualTo(QueryConstants.NULL_FLOAT); + } + { + final MyMessageV3.MyMessage v3 = MyMessageV3.MyMessage.newBuilder().setName("v3").setAge(3).build(); + assertThat(nameFunction.apply(v3)).isEqualTo("v3"); + assertThat(ageFunction.applyAsInt(v3)).isEqualTo(3); + assertThat(agefFunction.applyAsFloat(v3)).isEqualTo(QueryConstants.NULL_FLOAT); + } + { + final MyMessageV3.MyMessage v3 = MyMessageV3.MyMessage.newBuilder().setName("v3").build(); + assertThat(nameFunction.apply(v3)).isEqualTo("v3"); + assertThat(ageFunction.applyAsInt(v3)).isEqualTo(0); + assertThat(agefFunction.applyAsFloat(v3)).isEqualTo(QueryConstants.NULL_FLOAT); + } + } + + @Test + public void myMessageV4toV5() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(MyMessageV4.MyMessage.getDescriptor()); + assertThat(functions.functions()).hasSize(5); + final ToObjectFunction lastNameFunction = + (ToObjectFunction) get(functions, "first_and_last", "last_name"); + { + final MyMessageV5.MyMessage v5 = MyMessageV5.MyMessage.getDefaultInstance(); + try { + lastNameFunction.apply(v5); + failBecauseExceptionWasNotThrown(UncheckedDeephavenException.class); + } catch (UncheckedDeephavenException e) { + assertThat(e).hasMessage( + "Incompatible schema change for [first_and_last, last_name], originalType=io.deephaven.qst.type.StringType, newType=io.deephaven.qst.type.LongType"); + } + } + } + + @Test + public void myMessageV5toV4() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(MyMessageV5.MyMessage.getDescriptor()); + assertThat(functions.functions()).hasSize(5); + final ToLongFunction lastNameFunction = + (ToLongFunction) get(functions, "first_and_last", "last_name"); + { + final MyMessageV4.MyMessage v4 = MyMessageV4.MyMessage.getDefaultInstance(); + try { + lastNameFunction.applyAsLong(v4); + failBecauseExceptionWasNotThrown(UncheckedDeephavenException.class); + } catch (UncheckedDeephavenException e) { + assertThat(e).hasMessage( + "Incompatible schema change for [first_and_last, last_name], originalType=io.deephaven.qst.type.LongType, newType=io.deephaven.qst.type.StringType"); + } + } + } + + @Test + public void renameV1toV2() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(RenameV1.Rename.getDescriptor()); + assertThat(functions.functions()).hasSize(1); + final ToObjectFunction nameFunction = + (ToObjectFunction) get(functions, "name"); + { + final RenameV1.Rename v1 = RenameV1.Rename.newBuilder().setName("v1").build(); + assertThat(nameFunction.apply(v1)).isEqualTo("v1"); + } + { + final RenameV2.Rename v2 = RenameV2.Rename.newBuilder().setNameOld("v2").setName("v2-new-name").build(); + assertThat(nameFunction.apply(v2)).isEqualTo("v2"); + } + } + + @Test + public void renameV2toV1() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(RenameV2.Rename.getDescriptor()); + assertThat(functions.functions()).hasSize(2); + final ToObjectFunction nameOldFunction = + (ToObjectFunction) get(functions, "name_old"); + final ToObjectFunction nameFunction = + (ToObjectFunction) get(functions, "name"); + { + final RenameV2.Rename v2 = RenameV2.Rename.newBuilder().setNameOld("v2").setName("v2-new-name").build(); + assertThat(nameOldFunction.apply(v2)).isEqualTo("v2"); + assertThat(nameFunction.apply(v2)).isEqualTo("v2-new-name"); + } + { + final RenameV1.Rename v1 = RenameV1.Rename.newBuilder().setName("v1").build(); + assertThat(nameOldFunction.apply(v1)).isEqualTo("v1"); + assertThat(nameFunction.apply(v1)).isNull(); + } + } + + @Test + public void specialTypesWellKnown() { + final ToBooleanFunction isTs = namePathEquals(List.of("ts")); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(options(isTs, isTs, null, null)) + .build(); + final ProtobufFunctions functions = + ProtobufImpl.schemaChangeAwareFunctions(SpecialTypesV1.SpecialTypes.getDescriptor(), options); + assertThat(functions.functions()).hasSize(1); + final ToObjectFunction tsFunction = (ToObjectFunction) get(functions, "ts"); + final Timestamp ts = Timestamp.newBuilder().setSeconds(42).build(); + { + final SpecialTypesV1.SpecialTypes v1 = SpecialTypesV1.SpecialTypes.newBuilder().setTs(ts).build(); + assertThat(tsFunction.apply(v1)).isEqualTo(Instant.ofEpochSecond(42)); + } + { + final SpecialTypesV2.SpecialTypes v2 = SpecialTypesV2.SpecialTypes.newBuilder().setTsRenamed(ts).build(); + assertThat(tsFunction.apply(v2)).isEqualTo(Instant.ofEpochSecond(42)); + } + } + + @Test + public void specialTypesAntiWellKnown() { + final ToBooleanFunction startsWithTs = namePathStartsWith(List.of("ts")); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(options(startsWithTs, ToBooleanFunction.not(startsWithTs), null, null)) + .build(); + final ProtobufFunctions functions = + ProtobufImpl.schemaChangeAwareFunctions(SpecialTypesV1.SpecialTypes.getDescriptor(), options); + assertThat(functions.functions()).hasSize(2); + final ToLongFunction seconds = (ToLongFunction) get(functions, "ts", "seconds"); + final ToIntFunction nanos = (ToIntFunction) get(functions, "ts", "nanos"); + final Timestamp ts = Timestamp.newBuilder().setSeconds(42).setNanos(43).build(); + { + final SpecialTypesV1.SpecialTypes v1 = SpecialTypesV1.SpecialTypes.newBuilder().setTs(ts).build(); + assertThat(seconds.applyAsLong(v1)).isEqualTo(42L); + assertThat(nanos.applyAsInt(v1)).isEqualTo(43); + } + { + final SpecialTypesV2.SpecialTypes v2 = SpecialTypesV2.SpecialTypes.newBuilder().setTsRenamed(ts).build(); + assertThat(seconds.applyAsLong(v2)).isEqualTo(42L); + assertThat(nanos.applyAsInt(v2)).isEqualTo(43); + } + } + + @Test + public void specialTypesBytes() { + final ToBooleanFunction isBs = namePathEquals(List.of("bs")); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(options(isBs, null, isBs, null)) + .build(); + final ProtobufFunctions functions = + ProtobufImpl.schemaChangeAwareFunctions(SpecialTypesV1.SpecialTypes.getDescriptor(), options); + assertThat(functions.functions()).hasSize(1); + final ToObjectFunction bsFunction = (ToObjectFunction) get(functions, "bs"); + final ByteString bs = ByteString.copyFromUtf8("foo"); + { + final SpecialTypesV1.SpecialTypes v1 = SpecialTypesV1.SpecialTypes.newBuilder().setBs(bs).build(); + assertThat(bsFunction.apply(v1)).isEqualTo("foo".getBytes(StandardCharsets.UTF_8)); + } + { + final SpecialTypesV2.SpecialTypes v2 = SpecialTypesV2.SpecialTypes.newBuilder().setBsRenamed(bs).build(); + assertThat(bsFunction.apply(v2)).isEqualTo("foo".getBytes(StandardCharsets.UTF_8)); + } + } + + @Test + public void specialTypesByteString() { + final ToBooleanFunction isBs = namePathEquals(List.of("bs")); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(options(isBs, null, ToBooleanFunction.not(isBs), null)) + .build(); + final ProtobufFunctions functions = + ProtobufImpl.schemaChangeAwareFunctions(SpecialTypesV1.SpecialTypes.getDescriptor(), options); + assertThat(functions.functions()).hasSize(1); + final ToObjectFunction bsFunction = + (ToObjectFunction) get(functions, "bs"); + final ByteString bs = ByteString.copyFromUtf8("foo"); + { + final SpecialTypesV1.SpecialTypes v1 = SpecialTypesV1.SpecialTypes.newBuilder().setBs(bs).build(); + assertThat(bsFunction.apply(v1)).isEqualTo(bs); + } + { + final SpecialTypesV2.SpecialTypes v2 = SpecialTypesV2.SpecialTypes.newBuilder().setBsRenamed(bs).build(); + assertThat(bsFunction.apply(v2)).isEqualTo(bs); + } + } + + @Test + public void specialTypesMap() { + final ToBooleanFunction isMp = namePathEquals(List.of("mp")); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(options(isMp, null, null, isMp)) + .build(); + final ProtobufFunctions functions = + ProtobufImpl.schemaChangeAwareFunctions(SpecialTypesV1.SpecialTypes.getDescriptor(), options); + assertThat(functions.functions()).hasSize(1); + final ToObjectFunction> mpFunction = + (ToObjectFunction>) get(functions, "mp"); + { + final SpecialTypesV1.SpecialTypes v1 = SpecialTypesV1.SpecialTypes.newBuilder().putMp(42, 43).build(); + assertThat(mpFunction.apply(v1)).isEqualTo(Map.of(42, 43)); + } + { + final SpecialTypesV2.SpecialTypes v2 = + SpecialTypesV2.SpecialTypes.newBuilder().putMpRenamed(42, 43).build(); + assertThat(mpFunction.apply(v2)).isEqualTo(Map.of(42, 43)); + } + } + + @Test + public void specialTypesAntiMap() { + final ToBooleanFunction startsWithMp = namePathStartsWith(List.of("mp")); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(options(startsWithMp, null, null, ToBooleanFunction.not(startsWithMp))) + .build(); + final ProtobufFunctions functions = + ProtobufImpl.schemaChangeAwareFunctions(SpecialTypesV1.SpecialTypes.getDescriptor(), options); + assertThat(functions.functions()).hasSize(2); + final ToObjectFunction keyFunction = + (ToObjectFunction) get(functions, "mp", "key"); + final ToObjectFunction valueFunction = + (ToObjectFunction) get(functions, "mp", "value"); + { + final SpecialTypesV1.SpecialTypes v1 = SpecialTypesV1.SpecialTypes.newBuilder().putMp(42, 43).build(); + assertThat(keyFunction.apply(v1)).containsExactly(42); + assertThat(valueFunction.apply(v1)).containsExactly(43); + } + { + final SpecialTypesV2.SpecialTypes v2 = + SpecialTypesV2.SpecialTypes.newBuilder().putMpRenamed(42, 43).build(); + assertThat(keyFunction.apply(v2)).containsExactly(42); + assertThat(valueFunction.apply(v2)).containsExactly(43); + } + } + // + + @Test + public void boolV1toV2() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(BoolV1.MyBool.getDescriptor()); + assertThat(functions.functions()).hasSize(1); + // Note: it's important that this is parsed as ObjectFunction instead of + // BooleanFunction because we need to be able to handle schema changes which might remove the field + final ToObjectFunction myBoolFunction = + (ToObjectFunction) get(functions, "my_bool"); + { + final BoolV1.MyBool v1 = BoolV1.MyBool.newBuilder().setMyBool(true).build(); + assertThat(myBoolFunction.apply(v1)).isTrue(); + } + { + final BoolV1.MyBool v1 = BoolV1.MyBool.newBuilder().setMyBool(false).build(); + assertThat(myBoolFunction.apply(v1)).isFalse(); + } + { + final BoolV2.MyBool v2 = BoolV2.MyBool.getDefaultInstance(); + assertThat(myBoolFunction.apply(v2)).isNull(); + } + } + + @Test + public void boolV2IsEmpty() { + final ProtobufFunctions functions = schemaChangeAwareFunctions(BoolV2.MyBool.getDescriptor()); + assertThat(functions.functions()).isEmpty(); + } + + private static ProtobufFunctions schemaChangeAwareFunctions(Descriptor descriptor) { + return ProtobufImpl.schemaChangeAwareFunctions(descriptor, ProtobufDescriptorParserOptions.defaults()); + } + + private static TypedFunction get(ProtobufFunctions functions, String... namePath) { + return find(functions, Arrays.asList(namePath)).map(ProtobufFunction::function).get(); + } + + private static Function options( + ToBooleanFunction include, + ToBooleanFunction wellKnown, + ToBooleanFunction bytes, + ToBooleanFunction map) { + return fp -> { + final Builder builder = FieldOptions.builder(); + if (include != null) { + builder.include(include.test(fp)); + } + if (wellKnown != null) { + builder.wellKnown(wellKnown.test(fp) + ? WellKnownBehavior.asWellKnown() + : WellKnownBehavior.asRecursive()); + } + if (bytes != null) { + builder.bytes(bytes.test(fp) + ? BytesBehavior.asByteArray() + : BytesBehavior.asByteString()); + } + if (map != null) { + builder.map(map.test(fp) + ? MapBehavior.asMap() + : MapBehavior.asRepeated()); + } + return builder.build(); + }; + } + + private static ToBooleanFunction namePathEquals(List namePath) { + return map(FieldPath::namePath, namePath::equals); + } + + private static ToBooleanFunction namePathStartsWith(List prefix) { + return fieldPath -> fieldPath.startsWith(prefix); + } + + private static Optional find(ProtobufFunctions f, List namePath) { + for (ProtobufFunction function : f.functions()) { + if (namePath.equals(function.path().namePath())) { + return Optional.of(function); + } + } + return Optional.empty(); + } +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/BoolV1.java b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/BoolV1.java new file mode 100644 index 00000000000..e1e6f0db862 --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/BoolV1.java @@ -0,0 +1,518 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: BoolV1.proto + +package io.deephaven.kafka.protobuf.gen; + +public final class BoolV1 { + private BoolV1() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + public interface MyBoolOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.MyBool) + com.google.protobuf.MessageOrBuilder { + + /** + * bool my_bool = 1; + * @return The myBool. + */ + boolean getMyBool(); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyBool} + */ + public static final class MyBool extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.MyBool) + MyBoolOrBuilder { + private static final long serialVersionUID = 0L; + // Use MyBool.newBuilder() to construct. + private MyBool(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private MyBool() { + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new MyBool(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return BoolV1.internal_static_io_deephaven_kafka_test_MyBool_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return BoolV1.internal_static_io_deephaven_kafka_test_MyBool_fieldAccessorTable + .ensureFieldAccessorsInitialized( + BoolV1.MyBool.class, BoolV1.MyBool.Builder.class); + } + + public static final int MY_BOOL_FIELD_NUMBER = 1; + private boolean myBool_; + /** + * bool my_bool = 1; + * @return The myBool. + */ + @java.lang.Override + public boolean getMyBool() { + return myBool_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (myBool_ != false) { + output.writeBool(1, myBool_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (myBool_ != false) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(1, myBool_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof BoolV1.MyBool)) { + return super.equals(obj); + } + BoolV1.MyBool other = (BoolV1.MyBool) obj; + + if (getMyBool() + != other.getMyBool()) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + MY_BOOL_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean( + getMyBool()); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static BoolV1.MyBool parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static BoolV1.MyBool parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static BoolV1.MyBool parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static BoolV1.MyBool parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static BoolV1.MyBool parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static BoolV1.MyBool parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static BoolV1.MyBool parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static BoolV1.MyBool parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static BoolV1.MyBool parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static BoolV1.MyBool parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static BoolV1.MyBool parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static BoolV1.MyBool parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(BoolV1.MyBool prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyBool} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.MyBool) + BoolV1.MyBoolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return BoolV1.internal_static_io_deephaven_kafka_test_MyBool_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return BoolV1.internal_static_io_deephaven_kafka_test_MyBool_fieldAccessorTable + .ensureFieldAccessorsInitialized( + BoolV1.MyBool.class, BoolV1.MyBool.Builder.class); + } + + // Construct using io.deephaven.kafka.test.BoolV1.MyBool.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + myBool_ = false; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return BoolV1.internal_static_io_deephaven_kafka_test_MyBool_descriptor; + } + + @java.lang.Override + public BoolV1.MyBool getDefaultInstanceForType() { + return BoolV1.MyBool.getDefaultInstance(); + } + + @java.lang.Override + public BoolV1.MyBool build() { + BoolV1.MyBool result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public BoolV1.MyBool buildPartial() { + BoolV1.MyBool result = new BoolV1.MyBool(this); + result.myBool_ = myBool_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof BoolV1.MyBool) { + return mergeFrom((BoolV1.MyBool)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(BoolV1.MyBool other) { + if (other == BoolV1.MyBool.getDefaultInstance()) return this; + if (other.getMyBool() != false) { + setMyBool(other.getMyBool()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + myBool_ = input.readBool(); + + break; + } // case 8 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + private boolean myBool_ ; + /** + * bool my_bool = 1; + * @return The myBool. + */ + @java.lang.Override + public boolean getMyBool() { + return myBool_; + } + /** + * bool my_bool = 1; + * @param value The myBool to set. + * @return This builder for chaining. + */ + public Builder setMyBool(boolean value) { + + myBool_ = value; + onChanged(); + return this; + } + /** + * bool my_bool = 1; + * @return This builder for chaining. + */ + public Builder clearMyBool() { + + myBool_ = false; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.MyBool) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.MyBool) + private static final BoolV1.MyBool DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new BoolV1.MyBool(); + } + + public static BoolV1.MyBool getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public MyBool parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public BoolV1.MyBool getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_MyBool_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_MyBool_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\014BoolV1.proto\022\027io.deephaven.kafka.test\"" + + "\031\n\006MyBool\022\017\n\007my_bool\030\001 \001(\010B\004H\001P\000b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_io_deephaven_kafka_test_MyBool_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_io_deephaven_kafka_test_MyBool_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_MyBool_descriptor, + new java.lang.String[] { "MyBool", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/BoolV2.java b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/BoolV2.java new file mode 100644 index 00000000000..a10d619111d --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/BoolV2.java @@ -0,0 +1,447 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: BoolV2.proto + +package io.deephaven.kafka.protobuf.gen; + +public final class BoolV2 { + private BoolV2() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + public interface MyBoolOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.MyBool) + com.google.protobuf.MessageOrBuilder { + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyBool} + */ + public static final class MyBool extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.MyBool) + MyBoolOrBuilder { + private static final long serialVersionUID = 0L; + // Use MyBool.newBuilder() to construct. + private MyBool(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private MyBool() { + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new MyBool(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return BoolV2.internal_static_io_deephaven_kafka_test_MyBool_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return BoolV2.internal_static_io_deephaven_kafka_test_MyBool_fieldAccessorTable + .ensureFieldAccessorsInitialized( + BoolV2.MyBool.class, BoolV2.MyBool.Builder.class); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof BoolV2.MyBool)) { + return super.equals(obj); + } + BoolV2.MyBool other = (BoolV2.MyBool) obj; + + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static BoolV2.MyBool parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static BoolV2.MyBool parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static BoolV2.MyBool parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static BoolV2.MyBool parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static BoolV2.MyBool parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static BoolV2.MyBool parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static BoolV2.MyBool parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static BoolV2.MyBool parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static BoolV2.MyBool parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static BoolV2.MyBool parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static BoolV2.MyBool parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static BoolV2.MyBool parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(BoolV2.MyBool prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyBool} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.MyBool) + BoolV2.MyBoolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return BoolV2.internal_static_io_deephaven_kafka_test_MyBool_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return BoolV2.internal_static_io_deephaven_kafka_test_MyBool_fieldAccessorTable + .ensureFieldAccessorsInitialized( + BoolV2.MyBool.class, BoolV2.MyBool.Builder.class); + } + + // Construct using io.deephaven.kafka.test.BoolV2.MyBool.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return BoolV2.internal_static_io_deephaven_kafka_test_MyBool_descriptor; + } + + @java.lang.Override + public BoolV2.MyBool getDefaultInstanceForType() { + return BoolV2.MyBool.getDefaultInstance(); + } + + @java.lang.Override + public BoolV2.MyBool build() { + BoolV2.MyBool result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public BoolV2.MyBool buildPartial() { + BoolV2.MyBool result = new BoolV2.MyBool(this); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof BoolV2.MyBool) { + return mergeFrom((BoolV2.MyBool)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(BoolV2.MyBool other) { + if (other == BoolV2.MyBool.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.MyBool) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.MyBool) + private static final BoolV2.MyBool DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new BoolV2.MyBool(); + } + + public static BoolV2.MyBool getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public MyBool parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public BoolV2.MyBool getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_MyBool_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_MyBool_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\014BoolV2.proto\022\027io.deephaven.kafka.test\"" + + "\010\n\006MyBoolB\004H\001P\000b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_io_deephaven_kafka_test_MyBool_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_io_deephaven_kafka_test_MyBool_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_MyBool_descriptor, + new java.lang.String[] { }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV1.java b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV1.java new file mode 100644 index 00000000000..2728f83e68f --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV1.java @@ -0,0 +1,597 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: MyMessageV1.proto + +package io.deephaven.kafka.protobuf.gen; + +public final class MyMessageV1 { + private MyMessageV1() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + public interface MyMessageOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.MyMessage) + com.google.protobuf.MessageOrBuilder { + + /** + * string name = 1; + * @return The name. + */ + java.lang.String getName(); + /** + * string name = 1; + * @return The bytes for name. + */ + com.google.protobuf.ByteString + getNameBytes(); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage} + */ + public static final class MyMessage extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.MyMessage) + MyMessageOrBuilder { + private static final long serialVersionUID = 0L; + // Use MyMessage.newBuilder() to construct. + private MyMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private MyMessage() { + name_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new MyMessage(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV1.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV1.internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV1.MyMessage.class, MyMessageV1.MyMessage.Builder.class); + } + + public static final int NAME_FIELD_NUMBER = 1; + private volatile java.lang.Object name_; + /** + * string name = 1; + * @return The name. + */ + @java.lang.Override + public java.lang.String getName() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + name_ = s; + return s; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(name_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(name_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof MyMessageV1.MyMessage)) { + return super.equals(obj); + } + MyMessageV1.MyMessage other = (MyMessageV1.MyMessage) obj; + + if (!getName() + .equals(other.getName())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + NAME_FIELD_NUMBER; + hash = (53 * hash) + getName().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static MyMessageV1.MyMessage parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV1.MyMessage parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV1.MyMessage parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV1.MyMessage parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV1.MyMessage parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV1.MyMessage parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV1.MyMessage parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV1.MyMessage parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV1.MyMessage parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static MyMessageV1.MyMessage parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV1.MyMessage parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV1.MyMessage parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(MyMessageV1.MyMessage prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.MyMessage) + MyMessageV1.MyMessageOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV1.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV1.internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV1.MyMessage.class, MyMessageV1.MyMessage.Builder.class); + } + + // Construct using io.deephaven.kafka.test.MyMessageV1.MyMessage.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + name_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return MyMessageV1.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + public MyMessageV1.MyMessage getDefaultInstanceForType() { + return MyMessageV1.MyMessage.getDefaultInstance(); + } + + @java.lang.Override + public MyMessageV1.MyMessage build() { + MyMessageV1.MyMessage result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public MyMessageV1.MyMessage buildPartial() { + MyMessageV1.MyMessage result = new MyMessageV1.MyMessage(this); + result.name_ = name_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof MyMessageV1.MyMessage) { + return mergeFrom((MyMessageV1.MyMessage)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(MyMessageV1.MyMessage other) { + if (other == MyMessageV1.MyMessage.getDefaultInstance()) return this; + if (!other.getName().isEmpty()) { + name_ = other.name_; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + name_ = input.readStringRequireUtf8(); + + break; + } // case 10 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + private java.lang.Object name_ = ""; + /** + * string name = 1; + * @return The name. + */ + public java.lang.String getName() { + java.lang.Object ref = name_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + name_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string name = 1; + * @param value The name to set. + * @return This builder for chaining. + */ + public Builder setName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + name_ = value; + onChanged(); + return this; + } + /** + * string name = 1; + * @return This builder for chaining. + */ + public Builder clearName() { + + name_ = getDefaultInstance().getName(); + onChanged(); + return this; + } + /** + * string name = 1; + * @param value The bytes for name to set. + * @return This builder for chaining. + */ + public Builder setNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + name_ = value; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.MyMessage) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.MyMessage) + private static final MyMessageV1.MyMessage DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new MyMessageV1.MyMessage(); + } + + public static MyMessageV1.MyMessage getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public MyMessage parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public MyMessageV1.MyMessage getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\021MyMessageV1.proto\022\027io.deephaven.kafka." + + "test\"\031\n\tMyMessage\022\014\n\004name\030\001 \001(\tB\004H\001P\000b\006p" + + "roto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_io_deephaven_kafka_test_MyMessage_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_MyMessage_descriptor, + new java.lang.String[] { "Name", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV2.java b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV2.java new file mode 100644 index 00000000000..2c082d24d5b --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV2.java @@ -0,0 +1,667 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: MyMessageV2.proto + +package io.deephaven.kafka.protobuf.gen; + +public final class MyMessageV2 { + private MyMessageV2() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + public interface MyMessageOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.MyMessage) + com.google.protobuf.MessageOrBuilder { + + /** + * string name = 1; + * @return The name. + */ + java.lang.String getName(); + /** + * string name = 1; + * @return The bytes for name. + */ + com.google.protobuf.ByteString + getNameBytes(); + + /** + * int32 age = 2; + * @return The age. + */ + int getAge(); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage} + */ + public static final class MyMessage extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.MyMessage) + MyMessageOrBuilder { + private static final long serialVersionUID = 0L; + // Use MyMessage.newBuilder() to construct. + private MyMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private MyMessage() { + name_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new MyMessage(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV2.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV2.internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV2.MyMessage.class, MyMessageV2.MyMessage.Builder.class); + } + + public static final int NAME_FIELD_NUMBER = 1; + private volatile java.lang.Object name_; + /** + * string name = 1; + * @return The name. + */ + @java.lang.Override + public java.lang.String getName() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + name_ = s; + return s; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int AGE_FIELD_NUMBER = 2; + private int age_; + /** + * int32 age = 2; + * @return The age. + */ + @java.lang.Override + public int getAge() { + return age_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(name_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_); + } + if (age_ != 0) { + output.writeInt32(2, age_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(name_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_); + } + if (age_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(2, age_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof MyMessageV2.MyMessage)) { + return super.equals(obj); + } + MyMessageV2.MyMessage other = (MyMessageV2.MyMessage) obj; + + if (!getName() + .equals(other.getName())) return false; + if (getAge() + != other.getAge()) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + NAME_FIELD_NUMBER; + hash = (53 * hash) + getName().hashCode(); + hash = (37 * hash) + AGE_FIELD_NUMBER; + hash = (53 * hash) + getAge(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static MyMessageV2.MyMessage parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV2.MyMessage parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV2.MyMessage parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV2.MyMessage parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV2.MyMessage parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV2.MyMessage parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV2.MyMessage parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV2.MyMessage parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV2.MyMessage parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static MyMessageV2.MyMessage parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV2.MyMessage parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV2.MyMessage parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(MyMessageV2.MyMessage prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.MyMessage) + MyMessageV2.MyMessageOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV2.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV2.internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV2.MyMessage.class, MyMessageV2.MyMessage.Builder.class); + } + + // Construct using io.deephaven.kafka.test.MyMessageV2.MyMessage.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + name_ = ""; + + age_ = 0; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return MyMessageV2.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + public MyMessageV2.MyMessage getDefaultInstanceForType() { + return MyMessageV2.MyMessage.getDefaultInstance(); + } + + @java.lang.Override + public MyMessageV2.MyMessage build() { + MyMessageV2.MyMessage result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public MyMessageV2.MyMessage buildPartial() { + MyMessageV2.MyMessage result = new MyMessageV2.MyMessage(this); + result.name_ = name_; + result.age_ = age_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof MyMessageV2.MyMessage) { + return mergeFrom((MyMessageV2.MyMessage)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(MyMessageV2.MyMessage other) { + if (other == MyMessageV2.MyMessage.getDefaultInstance()) return this; + if (!other.getName().isEmpty()) { + name_ = other.name_; + onChanged(); + } + if (other.getAge() != 0) { + setAge(other.getAge()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + name_ = input.readStringRequireUtf8(); + + break; + } // case 10 + case 16: { + age_ = input.readInt32(); + + break; + } // case 16 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + private java.lang.Object name_ = ""; + /** + * string name = 1; + * @return The name. + */ + public java.lang.String getName() { + java.lang.Object ref = name_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + name_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string name = 1; + * @param value The name to set. + * @return This builder for chaining. + */ + public Builder setName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + name_ = value; + onChanged(); + return this; + } + /** + * string name = 1; + * @return This builder for chaining. + */ + public Builder clearName() { + + name_ = getDefaultInstance().getName(); + onChanged(); + return this; + } + /** + * string name = 1; + * @param value The bytes for name to set. + * @return This builder for chaining. + */ + public Builder setNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + name_ = value; + onChanged(); + return this; + } + + private int age_ ; + /** + * int32 age = 2; + * @return The age. + */ + @java.lang.Override + public int getAge() { + return age_; + } + /** + * int32 age = 2; + * @param value The age to set. + * @return This builder for chaining. + */ + public Builder setAge(int value) { + + age_ = value; + onChanged(); + return this; + } + /** + * int32 age = 2; + * @return This builder for chaining. + */ + public Builder clearAge() { + + age_ = 0; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.MyMessage) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.MyMessage) + private static final MyMessageV2.MyMessage DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new MyMessageV2.MyMessage(); + } + + public static MyMessageV2.MyMessage getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public MyMessage parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public MyMessageV2.MyMessage getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\021MyMessageV2.proto\022\027io.deephaven.kafka." + + "test\"&\n\tMyMessage\022\014\n\004name\030\001 \001(\t\022\013\n\003age\030\002" + + " \001(\005B\004H\001P\000b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_io_deephaven_kafka_test_MyMessage_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_MyMessage_descriptor, + new java.lang.String[] { "Name", "Age", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV3.java b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV3.java new file mode 100644 index 00000000000..764684c9e8b --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV3.java @@ -0,0 +1,1724 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: MyMessageV3.proto + +package io.deephaven.kafka.protobuf.gen; + +public final class MyMessageV3 { + private MyMessageV3() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + public interface MyMessageOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.MyMessage) + com.google.protobuf.MessageOrBuilder { + + /** + * string name = 1; + * @return Whether the name field is set. + */ + boolean hasName(); + /** + * string name = 1; + * @return The name. + */ + java.lang.String getName(); + /** + * string name = 1; + * @return The bytes for name. + */ + com.google.protobuf.ByteString + getNameBytes(); + + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return Whether the firstAndLast field is set. + */ + boolean hasFirstAndLast(); + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return The firstAndLast. + */ + MyMessageV3.MyMessage.FirstAndLast getFirstAndLast(); + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + MyMessageV3.MyMessage.FirstAndLastOrBuilder getFirstAndLastOrBuilder(); + + /** + * int32 age = 2; + * @return The age. + */ + int getAge(); + + public MyMessageV3.MyMessage.NameTypeCase getNameTypeCase(); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage} + */ + public static final class MyMessage extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.MyMessage) + MyMessageOrBuilder { + private static final long serialVersionUID = 0L; + // Use MyMessage.newBuilder() to construct. + private MyMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private MyMessage() { + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new MyMessage(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV3.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV3.internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV3.MyMessage.class, MyMessageV3.MyMessage.Builder.class); + } + + public interface FirstAndLastOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.MyMessage.FirstAndLast) + com.google.protobuf.MessageOrBuilder { + + /** + * string first_name = 1; + * @return The firstName. + */ + java.lang.String getFirstName(); + /** + * string first_name = 1; + * @return The bytes for firstName. + */ + com.google.protobuf.ByteString + getFirstNameBytes(); + + /** + * string last_name = 2; + * @return The lastName. + */ + java.lang.String getLastName(); + /** + * string last_name = 2; + * @return The bytes for lastName. + */ + com.google.protobuf.ByteString + getLastNameBytes(); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage.FirstAndLast} + */ + public static final class FirstAndLast extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.MyMessage.FirstAndLast) + FirstAndLastOrBuilder { + private static final long serialVersionUID = 0L; + // Use FirstAndLast.newBuilder() to construct. + private FirstAndLast(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private FirstAndLast() { + firstName_ = ""; + lastName_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new FirstAndLast(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV3.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV3.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV3.MyMessage.FirstAndLast.class, MyMessageV3.MyMessage.FirstAndLast.Builder.class); + } + + public static final int FIRST_NAME_FIELD_NUMBER = 1; + private volatile java.lang.Object firstName_; + /** + * string first_name = 1; + * @return The firstName. + */ + @java.lang.Override + public java.lang.String getFirstName() { + java.lang.Object ref = firstName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + firstName_ = s; + return s; + } + } + /** + * string first_name = 1; + * @return The bytes for firstName. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFirstNameBytes() { + java.lang.Object ref = firstName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + firstName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int LAST_NAME_FIELD_NUMBER = 2; + private volatile java.lang.Object lastName_; + /** + * string last_name = 2; + * @return The lastName. + */ + @java.lang.Override + public java.lang.String getLastName() { + java.lang.Object ref = lastName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + lastName_ = s; + return s; + } + } + /** + * string last_name = 2; + * @return The bytes for lastName. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getLastNameBytes() { + java.lang.Object ref = lastName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + lastName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(firstName_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, firstName_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(lastName_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, lastName_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(firstName_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, firstName_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(lastName_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, lastName_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof MyMessageV3.MyMessage.FirstAndLast)) { + return super.equals(obj); + } + MyMessageV3.MyMessage.FirstAndLast other = (MyMessageV3.MyMessage.FirstAndLast) obj; + + if (!getFirstName() + .equals(other.getFirstName())) return false; + if (!getLastName() + .equals(other.getLastName())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + FIRST_NAME_FIELD_NUMBER; + hash = (53 * hash) + getFirstName().hashCode(); + hash = (37 * hash) + LAST_NAME_FIELD_NUMBER; + hash = (53 * hash) + getLastName().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static MyMessageV3.MyMessage.FirstAndLast parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV3.MyMessage.FirstAndLast parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV3.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV3.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV3.MyMessage.FirstAndLast parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV3.MyMessage.FirstAndLast parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV3.MyMessage.FirstAndLast parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV3.MyMessage.FirstAndLast parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV3.MyMessage.FirstAndLast parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static MyMessageV3.MyMessage.FirstAndLast parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV3.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV3.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(MyMessageV3.MyMessage.FirstAndLast prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage.FirstAndLast} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.MyMessage.FirstAndLast) + MyMessageV3.MyMessage.FirstAndLastOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV3.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV3.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV3.MyMessage.FirstAndLast.class, MyMessageV3.MyMessage.FirstAndLast.Builder.class); + } + + // Construct using io.deephaven.kafka.test.MyMessageV3.MyMessage.FirstAndLast.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + firstName_ = ""; + + lastName_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return MyMessageV3.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + } + + @java.lang.Override + public MyMessageV3.MyMessage.FirstAndLast getDefaultInstanceForType() { + return MyMessageV3.MyMessage.FirstAndLast.getDefaultInstance(); + } + + @java.lang.Override + public MyMessageV3.MyMessage.FirstAndLast build() { + MyMessageV3.MyMessage.FirstAndLast result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public MyMessageV3.MyMessage.FirstAndLast buildPartial() { + MyMessageV3.MyMessage.FirstAndLast result = new MyMessageV3.MyMessage.FirstAndLast(this); + result.firstName_ = firstName_; + result.lastName_ = lastName_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof MyMessageV3.MyMessage.FirstAndLast) { + return mergeFrom((MyMessageV3.MyMessage.FirstAndLast)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(MyMessageV3.MyMessage.FirstAndLast other) { + if (other == MyMessageV3.MyMessage.FirstAndLast.getDefaultInstance()) return this; + if (!other.getFirstName().isEmpty()) { + firstName_ = other.firstName_; + onChanged(); + } + if (!other.getLastName().isEmpty()) { + lastName_ = other.lastName_; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + firstName_ = input.readStringRequireUtf8(); + + break; + } // case 10 + case 18: { + lastName_ = input.readStringRequireUtf8(); + + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + private java.lang.Object firstName_ = ""; + /** + * string first_name = 1; + * @return The firstName. + */ + public java.lang.String getFirstName() { + java.lang.Object ref = firstName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + firstName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string first_name = 1; + * @return The bytes for firstName. + */ + public com.google.protobuf.ByteString + getFirstNameBytes() { + java.lang.Object ref = firstName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + firstName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string first_name = 1; + * @param value The firstName to set. + * @return This builder for chaining. + */ + public Builder setFirstName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + firstName_ = value; + onChanged(); + return this; + } + /** + * string first_name = 1; + * @return This builder for chaining. + */ + public Builder clearFirstName() { + + firstName_ = getDefaultInstance().getFirstName(); + onChanged(); + return this; + } + /** + * string first_name = 1; + * @param value The bytes for firstName to set. + * @return This builder for chaining. + */ + public Builder setFirstNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + firstName_ = value; + onChanged(); + return this; + } + + private java.lang.Object lastName_ = ""; + /** + * string last_name = 2; + * @return The lastName. + */ + public java.lang.String getLastName() { + java.lang.Object ref = lastName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + lastName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string last_name = 2; + * @return The bytes for lastName. + */ + public com.google.protobuf.ByteString + getLastNameBytes() { + java.lang.Object ref = lastName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + lastName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string last_name = 2; + * @param value The lastName to set. + * @return This builder for chaining. + */ + public Builder setLastName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + lastName_ = value; + onChanged(); + return this; + } + /** + * string last_name = 2; + * @return This builder for chaining. + */ + public Builder clearLastName() { + + lastName_ = getDefaultInstance().getLastName(); + onChanged(); + return this; + } + /** + * string last_name = 2; + * @param value The bytes for lastName to set. + * @return This builder for chaining. + */ + public Builder setLastNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + lastName_ = value; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.MyMessage.FirstAndLast) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.MyMessage.FirstAndLast) + private static final MyMessageV3.MyMessage.FirstAndLast DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new MyMessageV3.MyMessage.FirstAndLast(); + } + + public static MyMessageV3.MyMessage.FirstAndLast getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public FirstAndLast parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public MyMessageV3.MyMessage.FirstAndLast getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private int nameTypeCase_ = 0; + private java.lang.Object nameType_; + public enum NameTypeCase + implements com.google.protobuf.Internal.EnumLite, + com.google.protobuf.AbstractMessage.InternalOneOfEnum { + NAME(1), + FIRST_AND_LAST(3), + NAMETYPE_NOT_SET(0); + private final int value; + private NameTypeCase(int value) { + this.value = value; + } + /** + * @param value The number of the enum to look for. + * @return The enum associated with the given number. + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static NameTypeCase valueOf(int value) { + return forNumber(value); + } + + public static NameTypeCase forNumber(int value) { + switch (value) { + case 1: return NAME; + case 3: return FIRST_AND_LAST; + case 0: return NAMETYPE_NOT_SET; + default: return null; + } + } + public int getNumber() { + return this.value; + } + }; + + public NameTypeCase + getNameTypeCase() { + return NameTypeCase.forNumber( + nameTypeCase_); + } + + public static final int NAME_FIELD_NUMBER = 1; + /** + * string name = 1; + * @return Whether the name field is set. + */ + public boolean hasName() { + return nameTypeCase_ == 1; + } + /** + * string name = 1; + * @return The name. + */ + public java.lang.String getName() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (nameTypeCase_ == 1) { + nameType_ = s; + } + return s; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + if (nameTypeCase_ == 1) { + nameType_ = b; + } + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FIRST_AND_LAST_FIELD_NUMBER = 3; + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return Whether the firstAndLast field is set. + */ + @java.lang.Override + public boolean hasFirstAndLast() { + return nameTypeCase_ == 3; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return The firstAndLast. + */ + @java.lang.Override + public MyMessageV3.MyMessage.FirstAndLast getFirstAndLast() { + if (nameTypeCase_ == 3) { + return (MyMessageV3.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV3.MyMessage.FirstAndLast.getDefaultInstance(); + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + @java.lang.Override + public MyMessageV3.MyMessage.FirstAndLastOrBuilder getFirstAndLastOrBuilder() { + if (nameTypeCase_ == 3) { + return (MyMessageV3.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV3.MyMessage.FirstAndLast.getDefaultInstance(); + } + + public static final int AGE_FIELD_NUMBER = 2; + private int age_; + /** + * int32 age = 2; + * @return The age. + */ + @java.lang.Override + public int getAge() { + return age_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (nameTypeCase_ == 1) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, nameType_); + } + if (age_ != 0) { + output.writeInt32(2, age_); + } + if (nameTypeCase_ == 3) { + output.writeMessage(3, (MyMessageV3.MyMessage.FirstAndLast) nameType_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (nameTypeCase_ == 1) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, nameType_); + } + if (age_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(2, age_); + } + if (nameTypeCase_ == 3) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, (MyMessageV3.MyMessage.FirstAndLast) nameType_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof MyMessageV3.MyMessage)) { + return super.equals(obj); + } + MyMessageV3.MyMessage other = (MyMessageV3.MyMessage) obj; + + if (getAge() + != other.getAge()) return false; + if (!getNameTypeCase().equals(other.getNameTypeCase())) return false; + switch (nameTypeCase_) { + case 1: + if (!getName() + .equals(other.getName())) return false; + break; + case 3: + if (!getFirstAndLast() + .equals(other.getFirstAndLast())) return false; + break; + case 0: + default: + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + AGE_FIELD_NUMBER; + hash = (53 * hash) + getAge(); + switch (nameTypeCase_) { + case 1: + hash = (37 * hash) + NAME_FIELD_NUMBER; + hash = (53 * hash) + getName().hashCode(); + break; + case 3: + hash = (37 * hash) + FIRST_AND_LAST_FIELD_NUMBER; + hash = (53 * hash) + getFirstAndLast().hashCode(); + break; + case 0: + default: + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static MyMessageV3.MyMessage parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV3.MyMessage parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV3.MyMessage parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV3.MyMessage parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV3.MyMessage parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV3.MyMessage parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV3.MyMessage parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV3.MyMessage parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV3.MyMessage parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static MyMessageV3.MyMessage parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV3.MyMessage parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV3.MyMessage parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(MyMessageV3.MyMessage prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.MyMessage) + MyMessageV3.MyMessageOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV3.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV3.internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV3.MyMessage.class, MyMessageV3.MyMessage.Builder.class); + } + + // Construct using io.deephaven.kafka.test.MyMessageV3.MyMessage.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + if (firstAndLastBuilder_ != null) { + firstAndLastBuilder_.clear(); + } + age_ = 0; + + nameTypeCase_ = 0; + nameType_ = null; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return MyMessageV3.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + public MyMessageV3.MyMessage getDefaultInstanceForType() { + return MyMessageV3.MyMessage.getDefaultInstance(); + } + + @java.lang.Override + public MyMessageV3.MyMessage build() { + MyMessageV3.MyMessage result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public MyMessageV3.MyMessage buildPartial() { + MyMessageV3.MyMessage result = new MyMessageV3.MyMessage(this); + if (nameTypeCase_ == 1) { + result.nameType_ = nameType_; + } + if (nameTypeCase_ == 3) { + if (firstAndLastBuilder_ == null) { + result.nameType_ = nameType_; + } else { + result.nameType_ = firstAndLastBuilder_.build(); + } + } + result.age_ = age_; + result.nameTypeCase_ = nameTypeCase_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof MyMessageV3.MyMessage) { + return mergeFrom((MyMessageV3.MyMessage)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(MyMessageV3.MyMessage other) { + if (other == MyMessageV3.MyMessage.getDefaultInstance()) return this; + if (other.getAge() != 0) { + setAge(other.getAge()); + } + switch (other.getNameTypeCase()) { + case NAME: { + nameTypeCase_ = 1; + nameType_ = other.nameType_; + onChanged(); + break; + } + case FIRST_AND_LAST: { + mergeFirstAndLast(other.getFirstAndLast()); + break; + } + case NAMETYPE_NOT_SET: { + break; + } + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + java.lang.String s = input.readStringRequireUtf8(); + nameTypeCase_ = 1; + nameType_ = s; + break; + } // case 10 + case 16: { + age_ = input.readInt32(); + + break; + } // case 16 + case 26: { + input.readMessage( + getFirstAndLastFieldBuilder().getBuilder(), + extensionRegistry); + nameTypeCase_ = 3; + break; + } // case 26 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int nameTypeCase_ = 0; + private java.lang.Object nameType_; + public NameTypeCase + getNameTypeCase() { + return NameTypeCase.forNumber( + nameTypeCase_); + } + + public Builder clearNameType() { + nameTypeCase_ = 0; + nameType_ = null; + onChanged(); + return this; + } + + + /** + * string name = 1; + * @return Whether the name field is set. + */ + @java.lang.Override + public boolean hasName() { + return nameTypeCase_ == 1; + } + /** + * string name = 1; + * @return The name. + */ + @java.lang.Override + public java.lang.String getName() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (nameTypeCase_ == 1) { + nameType_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + if (nameTypeCase_ == 1) { + nameType_ = b; + } + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string name = 1; + * @param value The name to set. + * @return This builder for chaining. + */ + public Builder setName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + nameTypeCase_ = 1; + nameType_ = value; + onChanged(); + return this; + } + /** + * string name = 1; + * @return This builder for chaining. + */ + public Builder clearName() { + if (nameTypeCase_ == 1) { + nameTypeCase_ = 0; + nameType_ = null; + onChanged(); + } + return this; + } + /** + * string name = 1; + * @param value The bytes for name to set. + * @return This builder for chaining. + */ + public Builder setNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + nameTypeCase_ = 1; + nameType_ = value; + onChanged(); + return this; + } + + private com.google.protobuf.SingleFieldBuilderV3< + MyMessageV3.MyMessage.FirstAndLast, MyMessageV3.MyMessage.FirstAndLast.Builder, MyMessageV3.MyMessage.FirstAndLastOrBuilder> firstAndLastBuilder_; + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return Whether the firstAndLast field is set. + */ + @java.lang.Override + public boolean hasFirstAndLast() { + return nameTypeCase_ == 3; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return The firstAndLast. + */ + @java.lang.Override + public MyMessageV3.MyMessage.FirstAndLast getFirstAndLast() { + if (firstAndLastBuilder_ == null) { + if (nameTypeCase_ == 3) { + return (MyMessageV3.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV3.MyMessage.FirstAndLast.getDefaultInstance(); + } else { + if (nameTypeCase_ == 3) { + return firstAndLastBuilder_.getMessage(); + } + return MyMessageV3.MyMessage.FirstAndLast.getDefaultInstance(); + } + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder setFirstAndLast(MyMessageV3.MyMessage.FirstAndLast value) { + if (firstAndLastBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + nameType_ = value; + onChanged(); + } else { + firstAndLastBuilder_.setMessage(value); + } + nameTypeCase_ = 3; + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder setFirstAndLast( + MyMessageV3.MyMessage.FirstAndLast.Builder builderForValue) { + if (firstAndLastBuilder_ == null) { + nameType_ = builderForValue.build(); + onChanged(); + } else { + firstAndLastBuilder_.setMessage(builderForValue.build()); + } + nameTypeCase_ = 3; + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder mergeFirstAndLast(MyMessageV3.MyMessage.FirstAndLast value) { + if (firstAndLastBuilder_ == null) { + if (nameTypeCase_ == 3 && + nameType_ != MyMessageV3.MyMessage.FirstAndLast.getDefaultInstance()) { + nameType_ = MyMessageV3.MyMessage.FirstAndLast.newBuilder((MyMessageV3.MyMessage.FirstAndLast) nameType_) + .mergeFrom(value).buildPartial(); + } else { + nameType_ = value; + } + onChanged(); + } else { + if (nameTypeCase_ == 3) { + firstAndLastBuilder_.mergeFrom(value); + } + firstAndLastBuilder_.setMessage(value); + } + nameTypeCase_ = 3; + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder clearFirstAndLast() { + if (firstAndLastBuilder_ == null) { + if (nameTypeCase_ == 3) { + nameTypeCase_ = 0; + nameType_ = null; + onChanged(); + } + } else { + if (nameTypeCase_ == 3) { + nameTypeCase_ = 0; + nameType_ = null; + } + firstAndLastBuilder_.clear(); + } + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public MyMessageV3.MyMessage.FirstAndLast.Builder getFirstAndLastBuilder() { + return getFirstAndLastFieldBuilder().getBuilder(); + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + @java.lang.Override + public MyMessageV3.MyMessage.FirstAndLastOrBuilder getFirstAndLastOrBuilder() { + if ((nameTypeCase_ == 3) && (firstAndLastBuilder_ != null)) { + return firstAndLastBuilder_.getMessageOrBuilder(); + } else { + if (nameTypeCase_ == 3) { + return (MyMessageV3.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV3.MyMessage.FirstAndLast.getDefaultInstance(); + } + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + private com.google.protobuf.SingleFieldBuilderV3< + MyMessageV3.MyMessage.FirstAndLast, MyMessageV3.MyMessage.FirstAndLast.Builder, MyMessageV3.MyMessage.FirstAndLastOrBuilder> + getFirstAndLastFieldBuilder() { + if (firstAndLastBuilder_ == null) { + if (!(nameTypeCase_ == 3)) { + nameType_ = MyMessageV3.MyMessage.FirstAndLast.getDefaultInstance(); + } + firstAndLastBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + MyMessageV3.MyMessage.FirstAndLast, MyMessageV3.MyMessage.FirstAndLast.Builder, MyMessageV3.MyMessage.FirstAndLastOrBuilder>( + (MyMessageV3.MyMessage.FirstAndLast) nameType_, + getParentForChildren(), + isClean()); + nameType_ = null; + } + nameTypeCase_ = 3; + onChanged();; + return firstAndLastBuilder_; + } + + private int age_ ; + /** + * int32 age = 2; + * @return The age. + */ + @java.lang.Override + public int getAge() { + return age_; + } + /** + * int32 age = 2; + * @param value The age to set. + * @return This builder for chaining. + */ + public Builder setAge(int value) { + + age_ = value; + onChanged(); + return this; + } + /** + * int32 age = 2; + * @return This builder for chaining. + */ + public Builder clearAge() { + + age_ = 0; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.MyMessage) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.MyMessage) + private static final MyMessageV3.MyMessage DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new MyMessageV3.MyMessage(); + } + + public static MyMessageV3.MyMessage getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public MyMessage parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public MyMessageV3.MyMessage getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\021MyMessageV3.proto\022\027io.deephaven.kafka." + + "test\"\267\001\n\tMyMessage\022\016\n\004name\030\001 \001(\tH\000\022I\n\016fi" + + "rst_and_last\030\003 \001(\0132/.io.deephaven.kafka." + + "test.MyMessage.FirstAndLastH\000\022\013\n\003age\030\002 \001" + + "(\005\0325\n\014FirstAndLast\022\022\n\nfirst_name\030\001 \001(\t\022\021" + + "\n\tlast_name\030\002 \001(\tB\013\n\tname_typeB\004H\001P\000b\006pr" + + "oto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_io_deephaven_kafka_test_MyMessage_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_MyMessage_descriptor, + new java.lang.String[] { "Name", "FirstAndLast", "Age", "NameType", }); + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor = + internal_static_io_deephaven_kafka_test_MyMessage_descriptor.getNestedTypes().get(0); + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor, + new java.lang.String[] { "FirstName", "LastName", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV4.java b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV4.java new file mode 100644 index 00000000000..917859e2007 --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV4.java @@ -0,0 +1,1935 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: MyMessageV4.proto + +package io.deephaven.kafka.protobuf.gen; + +public final class MyMessageV4 { + private MyMessageV4() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + public interface MyMessageOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.MyMessage) + com.google.protobuf.MessageOrBuilder { + + /** + * string name = 1; + * @return Whether the name field is set. + */ + boolean hasName(); + /** + * string name = 1; + * @return The name. + */ + java.lang.String getName(); + /** + * string name = 1; + * @return The bytes for name. + */ + com.google.protobuf.ByteString + getNameBytes(); + + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return Whether the firstAndLast field is set. + */ + boolean hasFirstAndLast(); + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return The firstAndLast. + */ + MyMessageV4.MyMessage.FirstAndLast getFirstAndLast(); + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + MyMessageV4.MyMessage.FirstAndLastOrBuilder getFirstAndLastOrBuilder(); + + /** + * int32 age = 2; + * @return Whether the age field is set. + */ + boolean hasAge(); + /** + * int32 age = 2; + * @return The age. + */ + int getAge(); + + /** + * float agef = 4; + * @return Whether the agef field is set. + */ + boolean hasAgef(); + /** + * float agef = 4; + * @return The agef. + */ + float getAgef(); + + public MyMessageV4.MyMessage.NameTypeCase getNameTypeCase(); + + public MyMessageV4.MyMessage.AgeTypeCase getAgeTypeCase(); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage} + */ + public static final class MyMessage extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.MyMessage) + MyMessageOrBuilder { + private static final long serialVersionUID = 0L; + // Use MyMessage.newBuilder() to construct. + private MyMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private MyMessage() { + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new MyMessage(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV4.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV4.internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV4.MyMessage.class, MyMessageV4.MyMessage.Builder.class); + } + + public interface FirstAndLastOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.MyMessage.FirstAndLast) + com.google.protobuf.MessageOrBuilder { + + /** + * string first_name = 1; + * @return The firstName. + */ + java.lang.String getFirstName(); + /** + * string first_name = 1; + * @return The bytes for firstName. + */ + com.google.protobuf.ByteString + getFirstNameBytes(); + + /** + * string last_name = 2; + * @return The lastName. + */ + java.lang.String getLastName(); + /** + * string last_name = 2; + * @return The bytes for lastName. + */ + com.google.protobuf.ByteString + getLastNameBytes(); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage.FirstAndLast} + */ + public static final class FirstAndLast extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.MyMessage.FirstAndLast) + FirstAndLastOrBuilder { + private static final long serialVersionUID = 0L; + // Use FirstAndLast.newBuilder() to construct. + private FirstAndLast(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private FirstAndLast() { + firstName_ = ""; + lastName_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new FirstAndLast(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV4.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV4.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV4.MyMessage.FirstAndLast.class, MyMessageV4.MyMessage.FirstAndLast.Builder.class); + } + + public static final int FIRST_NAME_FIELD_NUMBER = 1; + private volatile java.lang.Object firstName_; + /** + * string first_name = 1; + * @return The firstName. + */ + @java.lang.Override + public java.lang.String getFirstName() { + java.lang.Object ref = firstName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + firstName_ = s; + return s; + } + } + /** + * string first_name = 1; + * @return The bytes for firstName. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFirstNameBytes() { + java.lang.Object ref = firstName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + firstName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int LAST_NAME_FIELD_NUMBER = 2; + private volatile java.lang.Object lastName_; + /** + * string last_name = 2; + * @return The lastName. + */ + @java.lang.Override + public java.lang.String getLastName() { + java.lang.Object ref = lastName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + lastName_ = s; + return s; + } + } + /** + * string last_name = 2; + * @return The bytes for lastName. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getLastNameBytes() { + java.lang.Object ref = lastName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + lastName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(firstName_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, firstName_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(lastName_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, lastName_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(firstName_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, firstName_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(lastName_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, lastName_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof MyMessageV4.MyMessage.FirstAndLast)) { + return super.equals(obj); + } + MyMessageV4.MyMessage.FirstAndLast other = (MyMessageV4.MyMessage.FirstAndLast) obj; + + if (!getFirstName() + .equals(other.getFirstName())) return false; + if (!getLastName() + .equals(other.getLastName())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + FIRST_NAME_FIELD_NUMBER; + hash = (53 * hash) + getFirstName().hashCode(); + hash = (37 * hash) + LAST_NAME_FIELD_NUMBER; + hash = (53 * hash) + getLastName().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static MyMessageV4.MyMessage.FirstAndLast parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV4.MyMessage.FirstAndLast parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV4.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV4.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV4.MyMessage.FirstAndLast parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV4.MyMessage.FirstAndLast parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV4.MyMessage.FirstAndLast parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV4.MyMessage.FirstAndLast parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV4.MyMessage.FirstAndLast parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static MyMessageV4.MyMessage.FirstAndLast parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV4.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV4.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(MyMessageV4.MyMessage.FirstAndLast prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage.FirstAndLast} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.MyMessage.FirstAndLast) + MyMessageV4.MyMessage.FirstAndLastOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV4.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV4.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV4.MyMessage.FirstAndLast.class, MyMessageV4.MyMessage.FirstAndLast.Builder.class); + } + + // Construct using io.deephaven.kafka.test.MyMessageV4.MyMessage.FirstAndLast.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + firstName_ = ""; + + lastName_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return MyMessageV4.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + } + + @java.lang.Override + public MyMessageV4.MyMessage.FirstAndLast getDefaultInstanceForType() { + return MyMessageV4.MyMessage.FirstAndLast.getDefaultInstance(); + } + + @java.lang.Override + public MyMessageV4.MyMessage.FirstAndLast build() { + MyMessageV4.MyMessage.FirstAndLast result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public MyMessageV4.MyMessage.FirstAndLast buildPartial() { + MyMessageV4.MyMessage.FirstAndLast result = new MyMessageV4.MyMessage.FirstAndLast(this); + result.firstName_ = firstName_; + result.lastName_ = lastName_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof MyMessageV4.MyMessage.FirstAndLast) { + return mergeFrom((MyMessageV4.MyMessage.FirstAndLast)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(MyMessageV4.MyMessage.FirstAndLast other) { + if (other == MyMessageV4.MyMessage.FirstAndLast.getDefaultInstance()) return this; + if (!other.getFirstName().isEmpty()) { + firstName_ = other.firstName_; + onChanged(); + } + if (!other.getLastName().isEmpty()) { + lastName_ = other.lastName_; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + firstName_ = input.readStringRequireUtf8(); + + break; + } // case 10 + case 18: { + lastName_ = input.readStringRequireUtf8(); + + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + private java.lang.Object firstName_ = ""; + /** + * string first_name = 1; + * @return The firstName. + */ + public java.lang.String getFirstName() { + java.lang.Object ref = firstName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + firstName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string first_name = 1; + * @return The bytes for firstName. + */ + public com.google.protobuf.ByteString + getFirstNameBytes() { + java.lang.Object ref = firstName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + firstName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string first_name = 1; + * @param value The firstName to set. + * @return This builder for chaining. + */ + public Builder setFirstName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + firstName_ = value; + onChanged(); + return this; + } + /** + * string first_name = 1; + * @return This builder for chaining. + */ + public Builder clearFirstName() { + + firstName_ = getDefaultInstance().getFirstName(); + onChanged(); + return this; + } + /** + * string first_name = 1; + * @param value The bytes for firstName to set. + * @return This builder for chaining. + */ + public Builder setFirstNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + firstName_ = value; + onChanged(); + return this; + } + + private java.lang.Object lastName_ = ""; + /** + * string last_name = 2; + * @return The lastName. + */ + public java.lang.String getLastName() { + java.lang.Object ref = lastName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + lastName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string last_name = 2; + * @return The bytes for lastName. + */ + public com.google.protobuf.ByteString + getLastNameBytes() { + java.lang.Object ref = lastName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + lastName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string last_name = 2; + * @param value The lastName to set. + * @return This builder for chaining. + */ + public Builder setLastName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + lastName_ = value; + onChanged(); + return this; + } + /** + * string last_name = 2; + * @return This builder for chaining. + */ + public Builder clearLastName() { + + lastName_ = getDefaultInstance().getLastName(); + onChanged(); + return this; + } + /** + * string last_name = 2; + * @param value The bytes for lastName to set. + * @return This builder for chaining. + */ + public Builder setLastNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + lastName_ = value; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.MyMessage.FirstAndLast) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.MyMessage.FirstAndLast) + private static final MyMessageV4.MyMessage.FirstAndLast DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new MyMessageV4.MyMessage.FirstAndLast(); + } + + public static MyMessageV4.MyMessage.FirstAndLast getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public FirstAndLast parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public MyMessageV4.MyMessage.FirstAndLast getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private int nameTypeCase_ = 0; + private java.lang.Object nameType_; + public enum NameTypeCase + implements com.google.protobuf.Internal.EnumLite, + com.google.protobuf.AbstractMessage.InternalOneOfEnum { + NAME(1), + FIRST_AND_LAST(3), + NAMETYPE_NOT_SET(0); + private final int value; + private NameTypeCase(int value) { + this.value = value; + } + /** + * @param value The number of the enum to look for. + * @return The enum associated with the given number. + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static NameTypeCase valueOf(int value) { + return forNumber(value); + } + + public static NameTypeCase forNumber(int value) { + switch (value) { + case 1: return NAME; + case 3: return FIRST_AND_LAST; + case 0: return NAMETYPE_NOT_SET; + default: return null; + } + } + public int getNumber() { + return this.value; + } + }; + + public NameTypeCase + getNameTypeCase() { + return NameTypeCase.forNumber( + nameTypeCase_); + } + + private int ageTypeCase_ = 0; + private java.lang.Object ageType_; + public enum AgeTypeCase + implements com.google.protobuf.Internal.EnumLite, + com.google.protobuf.AbstractMessage.InternalOneOfEnum { + AGE(2), + AGEF(4), + AGETYPE_NOT_SET(0); + private final int value; + private AgeTypeCase(int value) { + this.value = value; + } + /** + * @param value The number of the enum to look for. + * @return The enum associated with the given number. + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static AgeTypeCase valueOf(int value) { + return forNumber(value); + } + + public static AgeTypeCase forNumber(int value) { + switch (value) { + case 2: return AGE; + case 4: return AGEF; + case 0: return AGETYPE_NOT_SET; + default: return null; + } + } + public int getNumber() { + return this.value; + } + }; + + public AgeTypeCase + getAgeTypeCase() { + return AgeTypeCase.forNumber( + ageTypeCase_); + } + + public static final int NAME_FIELD_NUMBER = 1; + /** + * string name = 1; + * @return Whether the name field is set. + */ + public boolean hasName() { + return nameTypeCase_ == 1; + } + /** + * string name = 1; + * @return The name. + */ + public java.lang.String getName() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (nameTypeCase_ == 1) { + nameType_ = s; + } + return s; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + if (nameTypeCase_ == 1) { + nameType_ = b; + } + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FIRST_AND_LAST_FIELD_NUMBER = 3; + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return Whether the firstAndLast field is set. + */ + @java.lang.Override + public boolean hasFirstAndLast() { + return nameTypeCase_ == 3; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return The firstAndLast. + */ + @java.lang.Override + public MyMessageV4.MyMessage.FirstAndLast getFirstAndLast() { + if (nameTypeCase_ == 3) { + return (MyMessageV4.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV4.MyMessage.FirstAndLast.getDefaultInstance(); + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + @java.lang.Override + public MyMessageV4.MyMessage.FirstAndLastOrBuilder getFirstAndLastOrBuilder() { + if (nameTypeCase_ == 3) { + return (MyMessageV4.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV4.MyMessage.FirstAndLast.getDefaultInstance(); + } + + public static final int AGE_FIELD_NUMBER = 2; + /** + * int32 age = 2; + * @return Whether the age field is set. + */ + @java.lang.Override + public boolean hasAge() { + return ageTypeCase_ == 2; + } + /** + * int32 age = 2; + * @return The age. + */ + @java.lang.Override + public int getAge() { + if (ageTypeCase_ == 2) { + return (java.lang.Integer) ageType_; + } + return 0; + } + + public static final int AGEF_FIELD_NUMBER = 4; + /** + * float agef = 4; + * @return Whether the agef field is set. + */ + @java.lang.Override + public boolean hasAgef() { + return ageTypeCase_ == 4; + } + /** + * float agef = 4; + * @return The agef. + */ + @java.lang.Override + public float getAgef() { + if (ageTypeCase_ == 4) { + return (java.lang.Float) ageType_; + } + return 0F; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (nameTypeCase_ == 1) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, nameType_); + } + if (ageTypeCase_ == 2) { + output.writeInt32( + 2, (int)((java.lang.Integer) ageType_)); + } + if (nameTypeCase_ == 3) { + output.writeMessage(3, (MyMessageV4.MyMessage.FirstAndLast) nameType_); + } + if (ageTypeCase_ == 4) { + output.writeFloat( + 4, (float)((java.lang.Float) ageType_)); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (nameTypeCase_ == 1) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, nameType_); + } + if (ageTypeCase_ == 2) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size( + 2, (int)((java.lang.Integer) ageType_)); + } + if (nameTypeCase_ == 3) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, (MyMessageV4.MyMessage.FirstAndLast) nameType_); + } + if (ageTypeCase_ == 4) { + size += com.google.protobuf.CodedOutputStream + .computeFloatSize( + 4, (float)((java.lang.Float) ageType_)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof MyMessageV4.MyMessage)) { + return super.equals(obj); + } + MyMessageV4.MyMessage other = (MyMessageV4.MyMessage) obj; + + if (!getNameTypeCase().equals(other.getNameTypeCase())) return false; + switch (nameTypeCase_) { + case 1: + if (!getName() + .equals(other.getName())) return false; + break; + case 3: + if (!getFirstAndLast() + .equals(other.getFirstAndLast())) return false; + break; + case 0: + default: + } + if (!getAgeTypeCase().equals(other.getAgeTypeCase())) return false; + switch (ageTypeCase_) { + case 2: + if (getAge() + != other.getAge()) return false; + break; + case 4: + if (java.lang.Float.floatToIntBits(getAgef()) + != java.lang.Float.floatToIntBits( + other.getAgef())) return false; + break; + case 0: + default: + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + switch (nameTypeCase_) { + case 1: + hash = (37 * hash) + NAME_FIELD_NUMBER; + hash = (53 * hash) + getName().hashCode(); + break; + case 3: + hash = (37 * hash) + FIRST_AND_LAST_FIELD_NUMBER; + hash = (53 * hash) + getFirstAndLast().hashCode(); + break; + case 0: + default: + } + switch (ageTypeCase_) { + case 2: + hash = (37 * hash) + AGE_FIELD_NUMBER; + hash = (53 * hash) + getAge(); + break; + case 4: + hash = (37 * hash) + AGEF_FIELD_NUMBER; + hash = (53 * hash) + java.lang.Float.floatToIntBits( + getAgef()); + break; + case 0: + default: + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static MyMessageV4.MyMessage parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV4.MyMessage parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV4.MyMessage parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV4.MyMessage parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV4.MyMessage parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV4.MyMessage parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV4.MyMessage parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV4.MyMessage parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV4.MyMessage parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static MyMessageV4.MyMessage parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV4.MyMessage parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV4.MyMessage parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(MyMessageV4.MyMessage prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.MyMessage) + MyMessageV4.MyMessageOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV4.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV4.internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV4.MyMessage.class, MyMessageV4.MyMessage.Builder.class); + } + + // Construct using io.deephaven.kafka.test.MyMessageV4.MyMessage.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + if (firstAndLastBuilder_ != null) { + firstAndLastBuilder_.clear(); + } + nameTypeCase_ = 0; + nameType_ = null; + ageTypeCase_ = 0; + ageType_ = null; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return MyMessageV4.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + public MyMessageV4.MyMessage getDefaultInstanceForType() { + return MyMessageV4.MyMessage.getDefaultInstance(); + } + + @java.lang.Override + public MyMessageV4.MyMessage build() { + MyMessageV4.MyMessage result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public MyMessageV4.MyMessage buildPartial() { + MyMessageV4.MyMessage result = new MyMessageV4.MyMessage(this); + if (nameTypeCase_ == 1) { + result.nameType_ = nameType_; + } + if (nameTypeCase_ == 3) { + if (firstAndLastBuilder_ == null) { + result.nameType_ = nameType_; + } else { + result.nameType_ = firstAndLastBuilder_.build(); + } + } + if (ageTypeCase_ == 2) { + result.ageType_ = ageType_; + } + if (ageTypeCase_ == 4) { + result.ageType_ = ageType_; + } + result.nameTypeCase_ = nameTypeCase_; + result.ageTypeCase_ = ageTypeCase_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof MyMessageV4.MyMessage) { + return mergeFrom((MyMessageV4.MyMessage)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(MyMessageV4.MyMessage other) { + if (other == MyMessageV4.MyMessage.getDefaultInstance()) return this; + switch (other.getNameTypeCase()) { + case NAME: { + nameTypeCase_ = 1; + nameType_ = other.nameType_; + onChanged(); + break; + } + case FIRST_AND_LAST: { + mergeFirstAndLast(other.getFirstAndLast()); + break; + } + case NAMETYPE_NOT_SET: { + break; + } + } + switch (other.getAgeTypeCase()) { + case AGE: { + setAge(other.getAge()); + break; + } + case AGEF: { + setAgef(other.getAgef()); + break; + } + case AGETYPE_NOT_SET: { + break; + } + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + java.lang.String s = input.readStringRequireUtf8(); + nameTypeCase_ = 1; + nameType_ = s; + break; + } // case 10 + case 16: { + ageType_ = input.readInt32(); + ageTypeCase_ = 2; + break; + } // case 16 + case 26: { + input.readMessage( + getFirstAndLastFieldBuilder().getBuilder(), + extensionRegistry); + nameTypeCase_ = 3; + break; + } // case 26 + case 37: { + ageType_ = input.readFloat(); + ageTypeCase_ = 4; + break; + } // case 37 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int nameTypeCase_ = 0; + private java.lang.Object nameType_; + public NameTypeCase + getNameTypeCase() { + return NameTypeCase.forNumber( + nameTypeCase_); + } + + public Builder clearNameType() { + nameTypeCase_ = 0; + nameType_ = null; + onChanged(); + return this; + } + + private int ageTypeCase_ = 0; + private java.lang.Object ageType_; + public AgeTypeCase + getAgeTypeCase() { + return AgeTypeCase.forNumber( + ageTypeCase_); + } + + public Builder clearAgeType() { + ageTypeCase_ = 0; + ageType_ = null; + onChanged(); + return this; + } + + + /** + * string name = 1; + * @return Whether the name field is set. + */ + @java.lang.Override + public boolean hasName() { + return nameTypeCase_ == 1; + } + /** + * string name = 1; + * @return The name. + */ + @java.lang.Override + public java.lang.String getName() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (nameTypeCase_ == 1) { + nameType_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + if (nameTypeCase_ == 1) { + nameType_ = b; + } + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string name = 1; + * @param value The name to set. + * @return This builder for chaining. + */ + public Builder setName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + nameTypeCase_ = 1; + nameType_ = value; + onChanged(); + return this; + } + /** + * string name = 1; + * @return This builder for chaining. + */ + public Builder clearName() { + if (nameTypeCase_ == 1) { + nameTypeCase_ = 0; + nameType_ = null; + onChanged(); + } + return this; + } + /** + * string name = 1; + * @param value The bytes for name to set. + * @return This builder for chaining. + */ + public Builder setNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + nameTypeCase_ = 1; + nameType_ = value; + onChanged(); + return this; + } + + private com.google.protobuf.SingleFieldBuilderV3< + MyMessageV4.MyMessage.FirstAndLast, MyMessageV4.MyMessage.FirstAndLast.Builder, MyMessageV4.MyMessage.FirstAndLastOrBuilder> firstAndLastBuilder_; + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return Whether the firstAndLast field is set. + */ + @java.lang.Override + public boolean hasFirstAndLast() { + return nameTypeCase_ == 3; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return The firstAndLast. + */ + @java.lang.Override + public MyMessageV4.MyMessage.FirstAndLast getFirstAndLast() { + if (firstAndLastBuilder_ == null) { + if (nameTypeCase_ == 3) { + return (MyMessageV4.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV4.MyMessage.FirstAndLast.getDefaultInstance(); + } else { + if (nameTypeCase_ == 3) { + return firstAndLastBuilder_.getMessage(); + } + return MyMessageV4.MyMessage.FirstAndLast.getDefaultInstance(); + } + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder setFirstAndLast(MyMessageV4.MyMessage.FirstAndLast value) { + if (firstAndLastBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + nameType_ = value; + onChanged(); + } else { + firstAndLastBuilder_.setMessage(value); + } + nameTypeCase_ = 3; + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder setFirstAndLast( + MyMessageV4.MyMessage.FirstAndLast.Builder builderForValue) { + if (firstAndLastBuilder_ == null) { + nameType_ = builderForValue.build(); + onChanged(); + } else { + firstAndLastBuilder_.setMessage(builderForValue.build()); + } + nameTypeCase_ = 3; + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder mergeFirstAndLast(MyMessageV4.MyMessage.FirstAndLast value) { + if (firstAndLastBuilder_ == null) { + if (nameTypeCase_ == 3 && + nameType_ != MyMessageV4.MyMessage.FirstAndLast.getDefaultInstance()) { + nameType_ = MyMessageV4.MyMessage.FirstAndLast.newBuilder((MyMessageV4.MyMessage.FirstAndLast) nameType_) + .mergeFrom(value).buildPartial(); + } else { + nameType_ = value; + } + onChanged(); + } else { + if (nameTypeCase_ == 3) { + firstAndLastBuilder_.mergeFrom(value); + } + firstAndLastBuilder_.setMessage(value); + } + nameTypeCase_ = 3; + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder clearFirstAndLast() { + if (firstAndLastBuilder_ == null) { + if (nameTypeCase_ == 3) { + nameTypeCase_ = 0; + nameType_ = null; + onChanged(); + } + } else { + if (nameTypeCase_ == 3) { + nameTypeCase_ = 0; + nameType_ = null; + } + firstAndLastBuilder_.clear(); + } + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public MyMessageV4.MyMessage.FirstAndLast.Builder getFirstAndLastBuilder() { + return getFirstAndLastFieldBuilder().getBuilder(); + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + @java.lang.Override + public MyMessageV4.MyMessage.FirstAndLastOrBuilder getFirstAndLastOrBuilder() { + if ((nameTypeCase_ == 3) && (firstAndLastBuilder_ != null)) { + return firstAndLastBuilder_.getMessageOrBuilder(); + } else { + if (nameTypeCase_ == 3) { + return (MyMessageV4.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV4.MyMessage.FirstAndLast.getDefaultInstance(); + } + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + private com.google.protobuf.SingleFieldBuilderV3< + MyMessageV4.MyMessage.FirstAndLast, MyMessageV4.MyMessage.FirstAndLast.Builder, MyMessageV4.MyMessage.FirstAndLastOrBuilder> + getFirstAndLastFieldBuilder() { + if (firstAndLastBuilder_ == null) { + if (!(nameTypeCase_ == 3)) { + nameType_ = MyMessageV4.MyMessage.FirstAndLast.getDefaultInstance(); + } + firstAndLastBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + MyMessageV4.MyMessage.FirstAndLast, MyMessageV4.MyMessage.FirstAndLast.Builder, MyMessageV4.MyMessage.FirstAndLastOrBuilder>( + (MyMessageV4.MyMessage.FirstAndLast) nameType_, + getParentForChildren(), + isClean()); + nameType_ = null; + } + nameTypeCase_ = 3; + onChanged();; + return firstAndLastBuilder_; + } + + /** + * int32 age = 2; + * @return Whether the age field is set. + */ + public boolean hasAge() { + return ageTypeCase_ == 2; + } + /** + * int32 age = 2; + * @return The age. + */ + public int getAge() { + if (ageTypeCase_ == 2) { + return (java.lang.Integer) ageType_; + } + return 0; + } + /** + * int32 age = 2; + * @param value The age to set. + * @return This builder for chaining. + */ + public Builder setAge(int value) { + ageTypeCase_ = 2; + ageType_ = value; + onChanged(); + return this; + } + /** + * int32 age = 2; + * @return This builder for chaining. + */ + public Builder clearAge() { + if (ageTypeCase_ == 2) { + ageTypeCase_ = 0; + ageType_ = null; + onChanged(); + } + return this; + } + + /** + * float agef = 4; + * @return Whether the agef field is set. + */ + public boolean hasAgef() { + return ageTypeCase_ == 4; + } + /** + * float agef = 4; + * @return The agef. + */ + public float getAgef() { + if (ageTypeCase_ == 4) { + return (java.lang.Float) ageType_; + } + return 0F; + } + /** + * float agef = 4; + * @param value The agef to set. + * @return This builder for chaining. + */ + public Builder setAgef(float value) { + ageTypeCase_ = 4; + ageType_ = value; + onChanged(); + return this; + } + /** + * float agef = 4; + * @return This builder for chaining. + */ + public Builder clearAgef() { + if (ageTypeCase_ == 4) { + ageTypeCase_ = 0; + ageType_ = null; + onChanged(); + } + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.MyMessage) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.MyMessage) + private static final MyMessageV4.MyMessage DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new MyMessageV4.MyMessage(); + } + + public static MyMessageV4.MyMessage getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public MyMessage parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public MyMessageV4.MyMessage getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\021MyMessageV4.proto\022\027io.deephaven.kafka." + + "test\"\325\001\n\tMyMessage\022\016\n\004name\030\001 \001(\tH\000\022I\n\016fi" + + "rst_and_last\030\003 \001(\0132/.io.deephaven.kafka." + + "test.MyMessage.FirstAndLastH\000\022\r\n\003age\030\002 \001" + + "(\005H\001\022\016\n\004agef\030\004 \001(\002H\001\0325\n\014FirstAndLast\022\022\n\n" + + "first_name\030\001 \001(\t\022\021\n\tlast_name\030\002 \001(\tB\013\n\tn" + + "ame_typeB\n\n\010age_typeB\004H\001P\000b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_io_deephaven_kafka_test_MyMessage_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_MyMessage_descriptor, + new java.lang.String[] { "Name", "FirstAndLast", "Age", "Agef", "NameType", "AgeType", }); + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor = + internal_static_io_deephaven_kafka_test_MyMessage_descriptor.getNestedTypes().get(0); + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor, + new java.lang.String[] { "FirstName", "LastName", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV5.java b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV5.java new file mode 100644 index 00000000000..e1b88cd447d --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/MyMessageV5.java @@ -0,0 +1,1877 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: MyMessageV5.proto + +package io.deephaven.kafka.protobuf.gen; + +public final class MyMessageV5 { + private MyMessageV5() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + public interface MyMessageOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.MyMessage) + com.google.protobuf.MessageOrBuilder { + + /** + * string name = 1; + * @return Whether the name field is set. + */ + boolean hasName(); + /** + * string name = 1; + * @return The name. + */ + java.lang.String getName(); + /** + * string name = 1; + * @return The bytes for name. + */ + com.google.protobuf.ByteString + getNameBytes(); + + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return Whether the firstAndLast field is set. + */ + boolean hasFirstAndLast(); + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return The firstAndLast. + */ + MyMessageV5.MyMessage.FirstAndLast getFirstAndLast(); + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + MyMessageV5.MyMessage.FirstAndLastOrBuilder getFirstAndLastOrBuilder(); + + /** + * int32 age = 2; + * @return Whether the age field is set. + */ + boolean hasAge(); + /** + * int32 age = 2; + * @return The age. + */ + int getAge(); + + /** + * float agef = 4; + * @return Whether the agef field is set. + */ + boolean hasAgef(); + /** + * float agef = 4; + * @return The agef. + */ + float getAgef(); + + public MyMessageV5.MyMessage.NameTypeCase getNameTypeCase(); + + public MyMessageV5.MyMessage.AgeTypeCase getAgeTypeCase(); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage} + */ + public static final class MyMessage extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.MyMessage) + MyMessageOrBuilder { + private static final long serialVersionUID = 0L; + // Use MyMessage.newBuilder() to construct. + private MyMessage(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private MyMessage() { + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new MyMessage(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV5.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV5.internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV5.MyMessage.class, MyMessageV5.MyMessage.Builder.class); + } + + public interface FirstAndLastOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.MyMessage.FirstAndLast) + com.google.protobuf.MessageOrBuilder { + + /** + * string first_name = 1; + * @return The firstName. + */ + java.lang.String getFirstName(); + /** + * string first_name = 1; + * @return The bytes for firstName. + */ + com.google.protobuf.ByteString + getFirstNameBytes(); + + /** + *

+       * an invalid schema change
+       * 
+ * + * int64 last_name = 2; + * @return The lastName. + */ + long getLastName(); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage.FirstAndLast} + */ + public static final class FirstAndLast extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.MyMessage.FirstAndLast) + FirstAndLastOrBuilder { + private static final long serialVersionUID = 0L; + // Use FirstAndLast.newBuilder() to construct. + private FirstAndLast(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private FirstAndLast() { + firstName_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new FirstAndLast(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV5.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV5.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV5.MyMessage.FirstAndLast.class, MyMessageV5.MyMessage.FirstAndLast.Builder.class); + } + + public static final int FIRST_NAME_FIELD_NUMBER = 1; + private volatile java.lang.Object firstName_; + /** + * string first_name = 1; + * @return The firstName. + */ + @java.lang.Override + public java.lang.String getFirstName() { + java.lang.Object ref = firstName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + firstName_ = s; + return s; + } + } + /** + * string first_name = 1; + * @return The bytes for firstName. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getFirstNameBytes() { + java.lang.Object ref = firstName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + firstName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int LAST_NAME_FIELD_NUMBER = 2; + private long lastName_; + /** + *
+       * an invalid schema change
+       * 
+ * + * int64 last_name = 2; + * @return The lastName. + */ + @java.lang.Override + public long getLastName() { + return lastName_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(firstName_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, firstName_); + } + if (lastName_ != 0L) { + output.writeInt64(2, lastName_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(firstName_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, firstName_); + } + if (lastName_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(2, lastName_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof MyMessageV5.MyMessage.FirstAndLast)) { + return super.equals(obj); + } + MyMessageV5.MyMessage.FirstAndLast other = (MyMessageV5.MyMessage.FirstAndLast) obj; + + if (!getFirstName() + .equals(other.getFirstName())) return false; + if (getLastName() + != other.getLastName()) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + FIRST_NAME_FIELD_NUMBER; + hash = (53 * hash) + getFirstName().hashCode(); + hash = (37 * hash) + LAST_NAME_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getLastName()); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static MyMessageV5.MyMessage.FirstAndLast parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV5.MyMessage.FirstAndLast parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV5.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV5.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV5.MyMessage.FirstAndLast parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV5.MyMessage.FirstAndLast parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV5.MyMessage.FirstAndLast parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV5.MyMessage.FirstAndLast parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV5.MyMessage.FirstAndLast parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static MyMessageV5.MyMessage.FirstAndLast parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV5.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV5.MyMessage.FirstAndLast parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(MyMessageV5.MyMessage.FirstAndLast prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage.FirstAndLast} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.MyMessage.FirstAndLast) + MyMessageV5.MyMessage.FirstAndLastOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV5.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV5.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV5.MyMessage.FirstAndLast.class, MyMessageV5.MyMessage.FirstAndLast.Builder.class); + } + + // Construct using io.deephaven.kafka.test.MyMessageV5.MyMessage.FirstAndLast.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + firstName_ = ""; + + lastName_ = 0L; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return MyMessageV5.internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + } + + @java.lang.Override + public MyMessageV5.MyMessage.FirstAndLast getDefaultInstanceForType() { + return MyMessageV5.MyMessage.FirstAndLast.getDefaultInstance(); + } + + @java.lang.Override + public MyMessageV5.MyMessage.FirstAndLast build() { + MyMessageV5.MyMessage.FirstAndLast result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public MyMessageV5.MyMessage.FirstAndLast buildPartial() { + MyMessageV5.MyMessage.FirstAndLast result = new MyMessageV5.MyMessage.FirstAndLast(this); + result.firstName_ = firstName_; + result.lastName_ = lastName_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof MyMessageV5.MyMessage.FirstAndLast) { + return mergeFrom((MyMessageV5.MyMessage.FirstAndLast)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(MyMessageV5.MyMessage.FirstAndLast other) { + if (other == MyMessageV5.MyMessage.FirstAndLast.getDefaultInstance()) return this; + if (!other.getFirstName().isEmpty()) { + firstName_ = other.firstName_; + onChanged(); + } + if (other.getLastName() != 0L) { + setLastName(other.getLastName()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + firstName_ = input.readStringRequireUtf8(); + + break; + } // case 10 + case 16: { + lastName_ = input.readInt64(); + + break; + } // case 16 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + private java.lang.Object firstName_ = ""; + /** + * string first_name = 1; + * @return The firstName. + */ + public java.lang.String getFirstName() { + java.lang.Object ref = firstName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + firstName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string first_name = 1; + * @return The bytes for firstName. + */ + public com.google.protobuf.ByteString + getFirstNameBytes() { + java.lang.Object ref = firstName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + firstName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string first_name = 1; + * @param value The firstName to set. + * @return This builder for chaining. + */ + public Builder setFirstName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + firstName_ = value; + onChanged(); + return this; + } + /** + * string first_name = 1; + * @return This builder for chaining. + */ + public Builder clearFirstName() { + + firstName_ = getDefaultInstance().getFirstName(); + onChanged(); + return this; + } + /** + * string first_name = 1; + * @param value The bytes for firstName to set. + * @return This builder for chaining. + */ + public Builder setFirstNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + firstName_ = value; + onChanged(); + return this; + } + + private long lastName_ ; + /** + *
+         * an invalid schema change
+         * 
+ * + * int64 last_name = 2; + * @return The lastName. + */ + @java.lang.Override + public long getLastName() { + return lastName_; + } + /** + *
+         * an invalid schema change
+         * 
+ * + * int64 last_name = 2; + * @param value The lastName to set. + * @return This builder for chaining. + */ + public Builder setLastName(long value) { + + lastName_ = value; + onChanged(); + return this; + } + /** + *
+         * an invalid schema change
+         * 
+ * + * int64 last_name = 2; + * @return This builder for chaining. + */ + public Builder clearLastName() { + + lastName_ = 0L; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.MyMessage.FirstAndLast) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.MyMessage.FirstAndLast) + private static final MyMessageV5.MyMessage.FirstAndLast DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new MyMessageV5.MyMessage.FirstAndLast(); + } + + public static MyMessageV5.MyMessage.FirstAndLast getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public FirstAndLast parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public MyMessageV5.MyMessage.FirstAndLast getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private int nameTypeCase_ = 0; + private java.lang.Object nameType_; + public enum NameTypeCase + implements com.google.protobuf.Internal.EnumLite, + com.google.protobuf.AbstractMessage.InternalOneOfEnum { + NAME(1), + FIRST_AND_LAST(3), + NAMETYPE_NOT_SET(0); + private final int value; + private NameTypeCase(int value) { + this.value = value; + } + /** + * @param value The number of the enum to look for. + * @return The enum associated with the given number. + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static NameTypeCase valueOf(int value) { + return forNumber(value); + } + + public static NameTypeCase forNumber(int value) { + switch (value) { + case 1: return NAME; + case 3: return FIRST_AND_LAST; + case 0: return NAMETYPE_NOT_SET; + default: return null; + } + } + public int getNumber() { + return this.value; + } + }; + + public NameTypeCase + getNameTypeCase() { + return NameTypeCase.forNumber( + nameTypeCase_); + } + + private int ageTypeCase_ = 0; + private java.lang.Object ageType_; + public enum AgeTypeCase + implements com.google.protobuf.Internal.EnumLite, + com.google.protobuf.AbstractMessage.InternalOneOfEnum { + AGE(2), + AGEF(4), + AGETYPE_NOT_SET(0); + private final int value; + private AgeTypeCase(int value) { + this.value = value; + } + /** + * @param value The number of the enum to look for. + * @return The enum associated with the given number. + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static AgeTypeCase valueOf(int value) { + return forNumber(value); + } + + public static AgeTypeCase forNumber(int value) { + switch (value) { + case 2: return AGE; + case 4: return AGEF; + case 0: return AGETYPE_NOT_SET; + default: return null; + } + } + public int getNumber() { + return this.value; + } + }; + + public AgeTypeCase + getAgeTypeCase() { + return AgeTypeCase.forNumber( + ageTypeCase_); + } + + public static final int NAME_FIELD_NUMBER = 1; + /** + * string name = 1; + * @return Whether the name field is set. + */ + public boolean hasName() { + return nameTypeCase_ == 1; + } + /** + * string name = 1; + * @return The name. + */ + public java.lang.String getName() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (nameTypeCase_ == 1) { + nameType_ = s; + } + return s; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + if (nameTypeCase_ == 1) { + nameType_ = b; + } + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int FIRST_AND_LAST_FIELD_NUMBER = 3; + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return Whether the firstAndLast field is set. + */ + @java.lang.Override + public boolean hasFirstAndLast() { + return nameTypeCase_ == 3; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return The firstAndLast. + */ + @java.lang.Override + public MyMessageV5.MyMessage.FirstAndLast getFirstAndLast() { + if (nameTypeCase_ == 3) { + return (MyMessageV5.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV5.MyMessage.FirstAndLast.getDefaultInstance(); + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + @java.lang.Override + public MyMessageV5.MyMessage.FirstAndLastOrBuilder getFirstAndLastOrBuilder() { + if (nameTypeCase_ == 3) { + return (MyMessageV5.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV5.MyMessage.FirstAndLast.getDefaultInstance(); + } + + public static final int AGE_FIELD_NUMBER = 2; + /** + * int32 age = 2; + * @return Whether the age field is set. + */ + @java.lang.Override + public boolean hasAge() { + return ageTypeCase_ == 2; + } + /** + * int32 age = 2; + * @return The age. + */ + @java.lang.Override + public int getAge() { + if (ageTypeCase_ == 2) { + return (java.lang.Integer) ageType_; + } + return 0; + } + + public static final int AGEF_FIELD_NUMBER = 4; + /** + * float agef = 4; + * @return Whether the agef field is set. + */ + @java.lang.Override + public boolean hasAgef() { + return ageTypeCase_ == 4; + } + /** + * float agef = 4; + * @return The agef. + */ + @java.lang.Override + public float getAgef() { + if (ageTypeCase_ == 4) { + return (java.lang.Float) ageType_; + } + return 0F; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (nameTypeCase_ == 1) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, nameType_); + } + if (ageTypeCase_ == 2) { + output.writeInt32( + 2, (int)((java.lang.Integer) ageType_)); + } + if (nameTypeCase_ == 3) { + output.writeMessage(3, (MyMessageV5.MyMessage.FirstAndLast) nameType_); + } + if (ageTypeCase_ == 4) { + output.writeFloat( + 4, (float)((java.lang.Float) ageType_)); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (nameTypeCase_ == 1) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, nameType_); + } + if (ageTypeCase_ == 2) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size( + 2, (int)((java.lang.Integer) ageType_)); + } + if (nameTypeCase_ == 3) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, (MyMessageV5.MyMessage.FirstAndLast) nameType_); + } + if (ageTypeCase_ == 4) { + size += com.google.protobuf.CodedOutputStream + .computeFloatSize( + 4, (float)((java.lang.Float) ageType_)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof MyMessageV5.MyMessage)) { + return super.equals(obj); + } + MyMessageV5.MyMessage other = (MyMessageV5.MyMessage) obj; + + if (!getNameTypeCase().equals(other.getNameTypeCase())) return false; + switch (nameTypeCase_) { + case 1: + if (!getName() + .equals(other.getName())) return false; + break; + case 3: + if (!getFirstAndLast() + .equals(other.getFirstAndLast())) return false; + break; + case 0: + default: + } + if (!getAgeTypeCase().equals(other.getAgeTypeCase())) return false; + switch (ageTypeCase_) { + case 2: + if (getAge() + != other.getAge()) return false; + break; + case 4: + if (java.lang.Float.floatToIntBits(getAgef()) + != java.lang.Float.floatToIntBits( + other.getAgef())) return false; + break; + case 0: + default: + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + switch (nameTypeCase_) { + case 1: + hash = (37 * hash) + NAME_FIELD_NUMBER; + hash = (53 * hash) + getName().hashCode(); + break; + case 3: + hash = (37 * hash) + FIRST_AND_LAST_FIELD_NUMBER; + hash = (53 * hash) + getFirstAndLast().hashCode(); + break; + case 0: + default: + } + switch (ageTypeCase_) { + case 2: + hash = (37 * hash) + AGE_FIELD_NUMBER; + hash = (53 * hash) + getAge(); + break; + case 4: + hash = (37 * hash) + AGEF_FIELD_NUMBER; + hash = (53 * hash) + java.lang.Float.floatToIntBits( + getAgef()); + break; + case 0: + default: + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static MyMessageV5.MyMessage parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV5.MyMessage parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV5.MyMessage parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV5.MyMessage parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV5.MyMessage parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static MyMessageV5.MyMessage parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static MyMessageV5.MyMessage parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV5.MyMessage parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV5.MyMessage parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static MyMessageV5.MyMessage parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static MyMessageV5.MyMessage parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static MyMessageV5.MyMessage parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(MyMessageV5.MyMessage prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.MyMessage} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.MyMessage) + MyMessageV5.MyMessageOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return MyMessageV5.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return MyMessageV5.internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + MyMessageV5.MyMessage.class, MyMessageV5.MyMessage.Builder.class); + } + + // Construct using io.deephaven.kafka.test.MyMessageV5.MyMessage.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + if (firstAndLastBuilder_ != null) { + firstAndLastBuilder_.clear(); + } + nameTypeCase_ = 0; + nameType_ = null; + ageTypeCase_ = 0; + ageType_ = null; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return MyMessageV5.internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + } + + @java.lang.Override + public MyMessageV5.MyMessage getDefaultInstanceForType() { + return MyMessageV5.MyMessage.getDefaultInstance(); + } + + @java.lang.Override + public MyMessageV5.MyMessage build() { + MyMessageV5.MyMessage result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public MyMessageV5.MyMessage buildPartial() { + MyMessageV5.MyMessage result = new MyMessageV5.MyMessage(this); + if (nameTypeCase_ == 1) { + result.nameType_ = nameType_; + } + if (nameTypeCase_ == 3) { + if (firstAndLastBuilder_ == null) { + result.nameType_ = nameType_; + } else { + result.nameType_ = firstAndLastBuilder_.build(); + } + } + if (ageTypeCase_ == 2) { + result.ageType_ = ageType_; + } + if (ageTypeCase_ == 4) { + result.ageType_ = ageType_; + } + result.nameTypeCase_ = nameTypeCase_; + result.ageTypeCase_ = ageTypeCase_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof MyMessageV5.MyMessage) { + return mergeFrom((MyMessageV5.MyMessage)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(MyMessageV5.MyMessage other) { + if (other == MyMessageV5.MyMessage.getDefaultInstance()) return this; + switch (other.getNameTypeCase()) { + case NAME: { + nameTypeCase_ = 1; + nameType_ = other.nameType_; + onChanged(); + break; + } + case FIRST_AND_LAST: { + mergeFirstAndLast(other.getFirstAndLast()); + break; + } + case NAMETYPE_NOT_SET: { + break; + } + } + switch (other.getAgeTypeCase()) { + case AGE: { + setAge(other.getAge()); + break; + } + case AGEF: { + setAgef(other.getAgef()); + break; + } + case AGETYPE_NOT_SET: { + break; + } + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + java.lang.String s = input.readStringRequireUtf8(); + nameTypeCase_ = 1; + nameType_ = s; + break; + } // case 10 + case 16: { + ageType_ = input.readInt32(); + ageTypeCase_ = 2; + break; + } // case 16 + case 26: { + input.readMessage( + getFirstAndLastFieldBuilder().getBuilder(), + extensionRegistry); + nameTypeCase_ = 3; + break; + } // case 26 + case 37: { + ageType_ = input.readFloat(); + ageTypeCase_ = 4; + break; + } // case 37 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int nameTypeCase_ = 0; + private java.lang.Object nameType_; + public NameTypeCase + getNameTypeCase() { + return NameTypeCase.forNumber( + nameTypeCase_); + } + + public Builder clearNameType() { + nameTypeCase_ = 0; + nameType_ = null; + onChanged(); + return this; + } + + private int ageTypeCase_ = 0; + private java.lang.Object ageType_; + public AgeTypeCase + getAgeTypeCase() { + return AgeTypeCase.forNumber( + ageTypeCase_); + } + + public Builder clearAgeType() { + ageTypeCase_ = 0; + ageType_ = null; + onChanged(); + return this; + } + + + /** + * string name = 1; + * @return Whether the name field is set. + */ + @java.lang.Override + public boolean hasName() { + return nameTypeCase_ == 1; + } + /** + * string name = 1; + * @return The name. + */ + @java.lang.Override + public java.lang.String getName() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (nameTypeCase_ == 1) { + nameType_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = ""; + if (nameTypeCase_ == 1) { + ref = nameType_; + } + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + if (nameTypeCase_ == 1) { + nameType_ = b; + } + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string name = 1; + * @param value The name to set. + * @return This builder for chaining. + */ + public Builder setName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + nameTypeCase_ = 1; + nameType_ = value; + onChanged(); + return this; + } + /** + * string name = 1; + * @return This builder for chaining. + */ + public Builder clearName() { + if (nameTypeCase_ == 1) { + nameTypeCase_ = 0; + nameType_ = null; + onChanged(); + } + return this; + } + /** + * string name = 1; + * @param value The bytes for name to set. + * @return This builder for chaining. + */ + public Builder setNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + nameTypeCase_ = 1; + nameType_ = value; + onChanged(); + return this; + } + + private com.google.protobuf.SingleFieldBuilderV3< + MyMessageV5.MyMessage.FirstAndLast, MyMessageV5.MyMessage.FirstAndLast.Builder, MyMessageV5.MyMessage.FirstAndLastOrBuilder> firstAndLastBuilder_; + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return Whether the firstAndLast field is set. + */ + @java.lang.Override + public boolean hasFirstAndLast() { + return nameTypeCase_ == 3; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + * @return The firstAndLast. + */ + @java.lang.Override + public MyMessageV5.MyMessage.FirstAndLast getFirstAndLast() { + if (firstAndLastBuilder_ == null) { + if (nameTypeCase_ == 3) { + return (MyMessageV5.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV5.MyMessage.FirstAndLast.getDefaultInstance(); + } else { + if (nameTypeCase_ == 3) { + return firstAndLastBuilder_.getMessage(); + } + return MyMessageV5.MyMessage.FirstAndLast.getDefaultInstance(); + } + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder setFirstAndLast(MyMessageV5.MyMessage.FirstAndLast value) { + if (firstAndLastBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + nameType_ = value; + onChanged(); + } else { + firstAndLastBuilder_.setMessage(value); + } + nameTypeCase_ = 3; + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder setFirstAndLast( + MyMessageV5.MyMessage.FirstAndLast.Builder builderForValue) { + if (firstAndLastBuilder_ == null) { + nameType_ = builderForValue.build(); + onChanged(); + } else { + firstAndLastBuilder_.setMessage(builderForValue.build()); + } + nameTypeCase_ = 3; + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder mergeFirstAndLast(MyMessageV5.MyMessage.FirstAndLast value) { + if (firstAndLastBuilder_ == null) { + if (nameTypeCase_ == 3 && + nameType_ != MyMessageV5.MyMessage.FirstAndLast.getDefaultInstance()) { + nameType_ = MyMessageV5.MyMessage.FirstAndLast.newBuilder((MyMessageV5.MyMessage.FirstAndLast) nameType_) + .mergeFrom(value).buildPartial(); + } else { + nameType_ = value; + } + onChanged(); + } else { + if (nameTypeCase_ == 3) { + firstAndLastBuilder_.mergeFrom(value); + } + firstAndLastBuilder_.setMessage(value); + } + nameTypeCase_ = 3; + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public Builder clearFirstAndLast() { + if (firstAndLastBuilder_ == null) { + if (nameTypeCase_ == 3) { + nameTypeCase_ = 0; + nameType_ = null; + onChanged(); + } + } else { + if (nameTypeCase_ == 3) { + nameTypeCase_ = 0; + nameType_ = null; + } + firstAndLastBuilder_.clear(); + } + return this; + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + public MyMessageV5.MyMessage.FirstAndLast.Builder getFirstAndLastBuilder() { + return getFirstAndLastFieldBuilder().getBuilder(); + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + @java.lang.Override + public MyMessageV5.MyMessage.FirstAndLastOrBuilder getFirstAndLastOrBuilder() { + if ((nameTypeCase_ == 3) && (firstAndLastBuilder_ != null)) { + return firstAndLastBuilder_.getMessageOrBuilder(); + } else { + if (nameTypeCase_ == 3) { + return (MyMessageV5.MyMessage.FirstAndLast) nameType_; + } + return MyMessageV5.MyMessage.FirstAndLast.getDefaultInstance(); + } + } + /** + * .io.deephaven.kafka.test.MyMessage.FirstAndLast first_and_last = 3; + */ + private com.google.protobuf.SingleFieldBuilderV3< + MyMessageV5.MyMessage.FirstAndLast, MyMessageV5.MyMessage.FirstAndLast.Builder, MyMessageV5.MyMessage.FirstAndLastOrBuilder> + getFirstAndLastFieldBuilder() { + if (firstAndLastBuilder_ == null) { + if (!(nameTypeCase_ == 3)) { + nameType_ = MyMessageV5.MyMessage.FirstAndLast.getDefaultInstance(); + } + firstAndLastBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + MyMessageV5.MyMessage.FirstAndLast, MyMessageV5.MyMessage.FirstAndLast.Builder, MyMessageV5.MyMessage.FirstAndLastOrBuilder>( + (MyMessageV5.MyMessage.FirstAndLast) nameType_, + getParentForChildren(), + isClean()); + nameType_ = null; + } + nameTypeCase_ = 3; + onChanged();; + return firstAndLastBuilder_; + } + + /** + * int32 age = 2; + * @return Whether the age field is set. + */ + public boolean hasAge() { + return ageTypeCase_ == 2; + } + /** + * int32 age = 2; + * @return The age. + */ + public int getAge() { + if (ageTypeCase_ == 2) { + return (java.lang.Integer) ageType_; + } + return 0; + } + /** + * int32 age = 2; + * @param value The age to set. + * @return This builder for chaining. + */ + public Builder setAge(int value) { + ageTypeCase_ = 2; + ageType_ = value; + onChanged(); + return this; + } + /** + * int32 age = 2; + * @return This builder for chaining. + */ + public Builder clearAge() { + if (ageTypeCase_ == 2) { + ageTypeCase_ = 0; + ageType_ = null; + onChanged(); + } + return this; + } + + /** + * float agef = 4; + * @return Whether the agef field is set. + */ + public boolean hasAgef() { + return ageTypeCase_ == 4; + } + /** + * float agef = 4; + * @return The agef. + */ + public float getAgef() { + if (ageTypeCase_ == 4) { + return (java.lang.Float) ageType_; + } + return 0F; + } + /** + * float agef = 4; + * @param value The agef to set. + * @return This builder for chaining. + */ + public Builder setAgef(float value) { + ageTypeCase_ = 4; + ageType_ = value; + onChanged(); + return this; + } + /** + * float agef = 4; + * @return This builder for chaining. + */ + public Builder clearAgef() { + if (ageTypeCase_ == 4) { + ageTypeCase_ = 0; + ageType_ = null; + onChanged(); + } + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.MyMessage) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.MyMessage) + private static final MyMessageV5.MyMessage DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new MyMessageV5.MyMessage(); + } + + public static MyMessageV5.MyMessage getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public MyMessage parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public MyMessageV5.MyMessage getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_MyMessage_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\021MyMessageV5.proto\022\027io.deephaven.kafka." + + "test\"\325\001\n\tMyMessage\022\016\n\004name\030\001 \001(\tH\000\022I\n\016fi" + + "rst_and_last\030\003 \001(\0132/.io.deephaven.kafka." + + "test.MyMessage.FirstAndLastH\000\022\r\n\003age\030\002 \001" + + "(\005H\001\022\016\n\004agef\030\004 \001(\002H\001\0325\n\014FirstAndLast\022\022\n\n" + + "first_name\030\001 \001(\t\022\021\n\tlast_name\030\002 \001(\003B\013\n\tn" + + "ame_typeB\n\n\010age_typeB\004H\001P\000b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_io_deephaven_kafka_test_MyMessage_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_io_deephaven_kafka_test_MyMessage_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_MyMessage_descriptor, + new java.lang.String[] { "Name", "FirstAndLast", "Age", "Agef", "NameType", "AgeType", }); + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor = + internal_static_io_deephaven_kafka_test_MyMessage_descriptor.getNestedTypes().get(0); + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_MyMessage_FirstAndLast_descriptor, + new java.lang.String[] { "FirstName", "LastName", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/RenameV1.java b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/RenameV1.java new file mode 100644 index 00000000000..51f493d37af --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/RenameV1.java @@ -0,0 +1,596 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: RenameV1.proto + +package io.deephaven.kafka.protobuf.gen; + +public final class RenameV1 { + private RenameV1() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + public interface RenameOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.Rename) + com.google.protobuf.MessageOrBuilder { + + /** + * string name = 1; + * @return The name. + */ + java.lang.String getName(); + /** + * string name = 1; + * @return The bytes for name. + */ + com.google.protobuf.ByteString + getNameBytes(); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.Rename} + */ + public static final class Rename extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.Rename) + RenameOrBuilder { + private static final long serialVersionUID = 0L; + // Use Rename.newBuilder() to construct. + private Rename(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private Rename() { + name_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new Rename(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return RenameV1.internal_static_io_deephaven_kafka_test_Rename_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return RenameV1.internal_static_io_deephaven_kafka_test_Rename_fieldAccessorTable + .ensureFieldAccessorsInitialized( + RenameV1.Rename.class, RenameV1.Rename.Builder.class); + } + + public static final int NAME_FIELD_NUMBER = 1; + private volatile java.lang.Object name_; + /** + * string name = 1; + * @return The name. + */ + @java.lang.Override + public java.lang.String getName() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + name_ = s; + return s; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(name_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, name_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(name_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, name_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof RenameV1.Rename)) { + return super.equals(obj); + } + RenameV1.Rename other = (RenameV1.Rename) obj; + + if (!getName() + .equals(other.getName())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + NAME_FIELD_NUMBER; + hash = (53 * hash) + getName().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static RenameV1.Rename parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static RenameV1.Rename parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static RenameV1.Rename parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static RenameV1.Rename parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static RenameV1.Rename parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static RenameV1.Rename parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static RenameV1.Rename parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static RenameV1.Rename parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static RenameV1.Rename parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static RenameV1.Rename parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static RenameV1.Rename parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static RenameV1.Rename parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(RenameV1.Rename prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.Rename} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.Rename) + RenameV1.RenameOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return RenameV1.internal_static_io_deephaven_kafka_test_Rename_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return RenameV1.internal_static_io_deephaven_kafka_test_Rename_fieldAccessorTable + .ensureFieldAccessorsInitialized( + RenameV1.Rename.class, RenameV1.Rename.Builder.class); + } + + // Construct using io.deephaven.kafka.test.RenameV1.Rename.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + name_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return RenameV1.internal_static_io_deephaven_kafka_test_Rename_descriptor; + } + + @java.lang.Override + public RenameV1.Rename getDefaultInstanceForType() { + return RenameV1.Rename.getDefaultInstance(); + } + + @java.lang.Override + public RenameV1.Rename build() { + RenameV1.Rename result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public RenameV1.Rename buildPartial() { + RenameV1.Rename result = new RenameV1.Rename(this); + result.name_ = name_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof RenameV1.Rename) { + return mergeFrom((RenameV1.Rename)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(RenameV1.Rename other) { + if (other == RenameV1.Rename.getDefaultInstance()) return this; + if (!other.getName().isEmpty()) { + name_ = other.name_; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + name_ = input.readStringRequireUtf8(); + + break; + } // case 10 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + private java.lang.Object name_ = ""; + /** + * string name = 1; + * @return The name. + */ + public java.lang.String getName() { + java.lang.Object ref = name_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + name_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string name = 1; + * @return The bytes for name. + */ + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string name = 1; + * @param value The name to set. + * @return This builder for chaining. + */ + public Builder setName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + name_ = value; + onChanged(); + return this; + } + /** + * string name = 1; + * @return This builder for chaining. + */ + public Builder clearName() { + + name_ = getDefaultInstance().getName(); + onChanged(); + return this; + } + /** + * string name = 1; + * @param value The bytes for name to set. + * @return This builder for chaining. + */ + public Builder setNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + name_ = value; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.Rename) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.Rename) + private static final RenameV1.Rename DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new RenameV1.Rename(); + } + + public static RenameV1.Rename getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Rename parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public RenameV1.Rename getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_Rename_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_Rename_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\016RenameV1.proto\022\027io.deephaven.kafka.tes" + + "t\"\026\n\006Rename\022\014\n\004name\030\001 \001(\tB\004H\001P\000b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_io_deephaven_kafka_test_Rename_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_io_deephaven_kafka_test_Rename_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_Rename_descriptor, + new java.lang.String[] { "Name", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/RenameV2.java b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/RenameV2.java new file mode 100644 index 00000000000..34531a02e7a --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/RenameV2.java @@ -0,0 +1,746 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: RenameV2.proto + +package io.deephaven.kafka.protobuf.gen; + +public final class RenameV2 { + private RenameV2() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + public interface RenameOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.Rename) + com.google.protobuf.MessageOrBuilder { + + /** + * string name_old = 1; + * @return The nameOld. + */ + java.lang.String getNameOld(); + /** + * string name_old = 1; + * @return The bytes for nameOld. + */ + com.google.protobuf.ByteString + getNameOldBytes(); + + /** + * string name = 2; + * @return The name. + */ + java.lang.String getName(); + /** + * string name = 2; + * @return The bytes for name. + */ + com.google.protobuf.ByteString + getNameBytes(); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.Rename} + */ + public static final class Rename extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.Rename) + RenameOrBuilder { + private static final long serialVersionUID = 0L; + // Use Rename.newBuilder() to construct. + private Rename(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private Rename() { + nameOld_ = ""; + name_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new Rename(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return RenameV2.internal_static_io_deephaven_kafka_test_Rename_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return RenameV2.internal_static_io_deephaven_kafka_test_Rename_fieldAccessorTable + .ensureFieldAccessorsInitialized( + RenameV2.Rename.class, RenameV2.Rename.Builder.class); + } + + public static final int NAME_OLD_FIELD_NUMBER = 1; + private volatile java.lang.Object nameOld_; + /** + * string name_old = 1; + * @return The nameOld. + */ + @java.lang.Override + public java.lang.String getNameOld() { + java.lang.Object ref = nameOld_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + nameOld_ = s; + return s; + } + } + /** + * string name_old = 1; + * @return The bytes for nameOld. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getNameOldBytes() { + java.lang.Object ref = nameOld_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + nameOld_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int NAME_FIELD_NUMBER = 2; + private volatile java.lang.Object name_; + /** + * string name = 2; + * @return The name. + */ + @java.lang.Override + public java.lang.String getName() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + name_ = s; + return s; + } + } + /** + * string name = 2; + * @return The bytes for name. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(nameOld_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, nameOld_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(name_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, name_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(nameOld_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, nameOld_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(name_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, name_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof RenameV2.Rename)) { + return super.equals(obj); + } + RenameV2.Rename other = (RenameV2.Rename) obj; + + if (!getNameOld() + .equals(other.getNameOld())) return false; + if (!getName() + .equals(other.getName())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + NAME_OLD_FIELD_NUMBER; + hash = (53 * hash) + getNameOld().hashCode(); + hash = (37 * hash) + NAME_FIELD_NUMBER; + hash = (53 * hash) + getName().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static RenameV2.Rename parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static RenameV2.Rename parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static RenameV2.Rename parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static RenameV2.Rename parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static RenameV2.Rename parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static RenameV2.Rename parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static RenameV2.Rename parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static RenameV2.Rename parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static RenameV2.Rename parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static RenameV2.Rename parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static RenameV2.Rename parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static RenameV2.Rename parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(RenameV2.Rename prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.Rename} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.Rename) + RenameV2.RenameOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return RenameV2.internal_static_io_deephaven_kafka_test_Rename_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return RenameV2.internal_static_io_deephaven_kafka_test_Rename_fieldAccessorTable + .ensureFieldAccessorsInitialized( + RenameV2.Rename.class, RenameV2.Rename.Builder.class); + } + + // Construct using io.deephaven.kafka.test.RenameV2.Rename.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + nameOld_ = ""; + + name_ = ""; + + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return RenameV2.internal_static_io_deephaven_kafka_test_Rename_descriptor; + } + + @java.lang.Override + public RenameV2.Rename getDefaultInstanceForType() { + return RenameV2.Rename.getDefaultInstance(); + } + + @java.lang.Override + public RenameV2.Rename build() { + RenameV2.Rename result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public RenameV2.Rename buildPartial() { + RenameV2.Rename result = new RenameV2.Rename(this); + result.nameOld_ = nameOld_; + result.name_ = name_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof RenameV2.Rename) { + return mergeFrom((RenameV2.Rename)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(RenameV2.Rename other) { + if (other == RenameV2.Rename.getDefaultInstance()) return this; + if (!other.getNameOld().isEmpty()) { + nameOld_ = other.nameOld_; + onChanged(); + } + if (!other.getName().isEmpty()) { + name_ = other.name_; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + nameOld_ = input.readStringRequireUtf8(); + + break; + } // case 10 + case 18: { + name_ = input.readStringRequireUtf8(); + + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + + private java.lang.Object nameOld_ = ""; + /** + * string name_old = 1; + * @return The nameOld. + */ + public java.lang.String getNameOld() { + java.lang.Object ref = nameOld_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + nameOld_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string name_old = 1; + * @return The bytes for nameOld. + */ + public com.google.protobuf.ByteString + getNameOldBytes() { + java.lang.Object ref = nameOld_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + nameOld_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string name_old = 1; + * @param value The nameOld to set. + * @return This builder for chaining. + */ + public Builder setNameOld( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + nameOld_ = value; + onChanged(); + return this; + } + /** + * string name_old = 1; + * @return This builder for chaining. + */ + public Builder clearNameOld() { + + nameOld_ = getDefaultInstance().getNameOld(); + onChanged(); + return this; + } + /** + * string name_old = 1; + * @param value The bytes for nameOld to set. + * @return This builder for chaining. + */ + public Builder setNameOldBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + nameOld_ = value; + onChanged(); + return this; + } + + private java.lang.Object name_ = ""; + /** + * string name = 2; + * @return The name. + */ + public java.lang.String getName() { + java.lang.Object ref = name_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + name_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string name = 2; + * @return The bytes for name. + */ + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string name = 2; + * @param value The name to set. + * @return This builder for chaining. + */ + public Builder setName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + + name_ = value; + onChanged(); + return this; + } + /** + * string name = 2; + * @return This builder for chaining. + */ + public Builder clearName() { + + name_ = getDefaultInstance().getName(); + onChanged(); + return this; + } + /** + * string name = 2; + * @param value The bytes for name to set. + * @return This builder for chaining. + */ + public Builder setNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + checkByteStringIsUtf8(value); + + name_ = value; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.Rename) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.Rename) + private static final RenameV2.Rename DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new RenameV2.Rename(); + } + + public static RenameV2.Rename getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public Rename parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public RenameV2.Rename getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_Rename_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_Rename_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\016RenameV2.proto\022\027io.deephaven.kafka.tes" + + "t\"(\n\006Rename\022\020\n\010name_old\030\001 \001(\t\022\014\n\004name\030\002 " + + "\001(\tB\004H\001P\000b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_io_deephaven_kafka_test_Rename_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_io_deephaven_kafka_test_Rename_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_Rename_descriptor, + new java.lang.String[] { "NameOld", "Name", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/SpecialTypesV1.java b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/SpecialTypesV1.java new file mode 100644 index 00000000000..69d9c830eb0 --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/SpecialTypesV1.java @@ -0,0 +1,1194 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: SpecialTypesV1.proto + +package io.deephaven.kafka.protobuf.gen; + +public final class SpecialTypesV1 { + private SpecialTypesV1() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + public interface SpecialTypesOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.SpecialTypes) + com.google.protobuf.MessageOrBuilder { + + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts = 1; + * @return Whether the ts field is set. + */ + boolean hasTs(); + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts = 1; + * @return The ts. + */ + com.google.protobuf.Timestamp getTs(); + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts = 1; + */ + com.google.protobuf.TimestampOrBuilder getTsOrBuilder(); + + /** + *
+     * byte[] vs ByteString
+     * 
+ * + * bytes bs = 2; + * @return The bs. + */ + com.google.protobuf.ByteString getBs(); + + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp = 3; + */ + int getMpCount(); + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp = 3; + */ + boolean containsMp( + int key); + /** + * Use {@link #getMpMap()} instead. + */ + @java.lang.Deprecated + java.util.Map + getMp(); + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp = 3; + */ + java.util.Map + getMpMap(); + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp = 3; + */ + + int getMpOrDefault( + int key, + int defaultValue); + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp = 3; + */ + + int getMpOrThrow( + int key); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.SpecialTypes} + */ + public static final class SpecialTypes extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.SpecialTypes) + SpecialTypesOrBuilder { + private static final long serialVersionUID = 0L; + // Use SpecialTypes.newBuilder() to construct. + private SpecialTypes(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private SpecialTypes() { + bs_ = com.google.protobuf.ByteString.EMPTY; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new SpecialTypes(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return SpecialTypesV1.internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + @java.lang.Override + protected com.google.protobuf.MapField internalGetMapField( + int number) { + switch (number) { + case 3: + return internalGetMp(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return SpecialTypesV1.internal_static_io_deephaven_kafka_test_SpecialTypes_fieldAccessorTable + .ensureFieldAccessorsInitialized( + SpecialTypesV1.SpecialTypes.class, SpecialTypesV1.SpecialTypes.Builder.class); + } + + public static final int TS_FIELD_NUMBER = 1; + private com.google.protobuf.Timestamp ts_; + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts = 1; + * @return Whether the ts field is set. + */ + @java.lang.Override + public boolean hasTs() { + return ts_ != null; + } + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts = 1; + * @return The ts. + */ + @java.lang.Override + public com.google.protobuf.Timestamp getTs() { + return ts_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : ts_; + } + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts = 1; + */ + @java.lang.Override + public com.google.protobuf.TimestampOrBuilder getTsOrBuilder() { + return getTs(); + } + + public static final int BS_FIELD_NUMBER = 2; + private com.google.protobuf.ByteString bs_; + /** + *
+     * byte[] vs ByteString
+     * 
+ * + * bytes bs = 2; + * @return The bs. + */ + @java.lang.Override + public com.google.protobuf.ByteString getBs() { + return bs_; + } + + public static final int MP_FIELD_NUMBER = 3; + private static final class MpDefaultEntryHolder { + static final com.google.protobuf.MapEntry< + java.lang.Integer, java.lang.Integer> defaultEntry = + com.google.protobuf.MapEntry + .newDefaultInstance( + SpecialTypesV1.internal_static_io_deephaven_kafka_test_SpecialTypes_MpEntry_descriptor, + com.google.protobuf.WireFormat.FieldType.INT32, + 0, + com.google.protobuf.WireFormat.FieldType.INT32, + 0); + } + private com.google.protobuf.MapField< + java.lang.Integer, java.lang.Integer> mp_; + private com.google.protobuf.MapField + internalGetMp() { + if (mp_ == null) { + return com.google.protobuf.MapField.emptyMapField( + MpDefaultEntryHolder.defaultEntry); + } + return mp_; + } + + public int getMpCount() { + return internalGetMp().getMap().size(); + } + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp = 3; + */ + + @java.lang.Override + public boolean containsMp( + int key) { + + return internalGetMp().getMap().containsKey(key); + } + /** + * Use {@link #getMpMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getMp() { + return getMpMap(); + } + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp = 3; + */ + @java.lang.Override + + public java.util.Map getMpMap() { + return internalGetMp().getMap(); + } + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp = 3; + */ + @java.lang.Override + + public int getMpOrDefault( + int key, + int defaultValue) { + + java.util.Map map = + internalGetMp().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp = 3; + */ + @java.lang.Override + + public int getMpOrThrow( + int key) { + + java.util.Map map = + internalGetMp().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (ts_ != null) { + output.writeMessage(1, getTs()); + } + if (!bs_.isEmpty()) { + output.writeBytes(2, bs_); + } + com.google.protobuf.GeneratedMessageV3 + .serializeIntegerMapTo( + output, + internalGetMp(), + MpDefaultEntryHolder.defaultEntry, + 3); + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (ts_ != null) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getTs()); + } + if (!bs_.isEmpty()) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, bs_); + } + for (java.util.Map.Entry entry + : internalGetMp().getMap().entrySet()) { + com.google.protobuf.MapEntry + mp__ = MpDefaultEntryHolder.defaultEntry.newBuilderForType() + .setKey(entry.getKey()) + .setValue(entry.getValue()) + .build(); + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, mp__); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof SpecialTypesV1.SpecialTypes)) { + return super.equals(obj); + } + SpecialTypesV1.SpecialTypes other = (SpecialTypesV1.SpecialTypes) obj; + + if (hasTs() != other.hasTs()) return false; + if (hasTs()) { + if (!getTs() + .equals(other.getTs())) return false; + } + if (!getBs() + .equals(other.getBs())) return false; + if (!internalGetMp().equals( + other.internalGetMp())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasTs()) { + hash = (37 * hash) + TS_FIELD_NUMBER; + hash = (53 * hash) + getTs().hashCode(); + } + hash = (37 * hash) + BS_FIELD_NUMBER; + hash = (53 * hash) + getBs().hashCode(); + if (!internalGetMp().getMap().isEmpty()) { + hash = (37 * hash) + MP_FIELD_NUMBER; + hash = (53 * hash) + internalGetMp().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static SpecialTypesV1.SpecialTypes parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static SpecialTypesV1.SpecialTypes parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static SpecialTypesV1.SpecialTypes parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static SpecialTypesV1.SpecialTypes parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static SpecialTypesV1.SpecialTypes parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static SpecialTypesV1.SpecialTypes parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static SpecialTypesV1.SpecialTypes parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static SpecialTypesV1.SpecialTypes parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static SpecialTypesV1.SpecialTypes parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static SpecialTypesV1.SpecialTypes parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static SpecialTypesV1.SpecialTypes parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static SpecialTypesV1.SpecialTypes parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(SpecialTypesV1.SpecialTypes prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.SpecialTypes} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.SpecialTypes) + SpecialTypesV1.SpecialTypesOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return SpecialTypesV1.internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapField internalGetMapField( + int number) { + switch (number) { + case 3: + return internalGetMp(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapField internalGetMutableMapField( + int number) { + switch (number) { + case 3: + return internalGetMutableMp(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return SpecialTypesV1.internal_static_io_deephaven_kafka_test_SpecialTypes_fieldAccessorTable + .ensureFieldAccessorsInitialized( + SpecialTypesV1.SpecialTypes.class, SpecialTypesV1.SpecialTypes.Builder.class); + } + + // Construct using io.deephaven.kafka.test.SpecialTypesV1.SpecialTypes.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + if (tsBuilder_ == null) { + ts_ = null; + } else { + ts_ = null; + tsBuilder_ = null; + } + bs_ = com.google.protobuf.ByteString.EMPTY; + + internalGetMutableMp().clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return SpecialTypesV1.internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor; + } + + @java.lang.Override + public SpecialTypesV1.SpecialTypes getDefaultInstanceForType() { + return SpecialTypesV1.SpecialTypes.getDefaultInstance(); + } + + @java.lang.Override + public SpecialTypesV1.SpecialTypes build() { + SpecialTypesV1.SpecialTypes result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public SpecialTypesV1.SpecialTypes buildPartial() { + SpecialTypesV1.SpecialTypes result = new SpecialTypesV1.SpecialTypes(this); + int from_bitField0_ = bitField0_; + if (tsBuilder_ == null) { + result.ts_ = ts_; + } else { + result.ts_ = tsBuilder_.build(); + } + result.bs_ = bs_; + result.mp_ = internalGetMp(); + result.mp_.makeImmutable(); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof SpecialTypesV1.SpecialTypes) { + return mergeFrom((SpecialTypesV1.SpecialTypes)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(SpecialTypesV1.SpecialTypes other) { + if (other == SpecialTypesV1.SpecialTypes.getDefaultInstance()) return this; + if (other.hasTs()) { + mergeTs(other.getTs()); + } + if (other.getBs() != com.google.protobuf.ByteString.EMPTY) { + setBs(other.getBs()); + } + internalGetMutableMp().mergeFrom( + other.internalGetMp()); + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getTsFieldBuilder().getBuilder(), + extensionRegistry); + + break; + } // case 10 + case 18: { + bs_ = input.readBytes(); + + break; + } // case 18 + case 26: { + com.google.protobuf.MapEntry + mp__ = input.readMessage( + MpDefaultEntryHolder.defaultEntry.getParserForType(), extensionRegistry); + internalGetMutableMp().getMutableMap().put( + mp__.getKey(), mp__.getValue()); + break; + } // case 26 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private com.google.protobuf.Timestamp ts_; + private com.google.protobuf.SingleFieldBuilderV3< + com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> tsBuilder_; + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts = 1; + * @return Whether the ts field is set. + */ + public boolean hasTs() { + return tsBuilder_ != null || ts_ != null; + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts = 1; + * @return The ts. + */ + public com.google.protobuf.Timestamp getTs() { + if (tsBuilder_ == null) { + return ts_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : ts_; + } else { + return tsBuilder_.getMessage(); + } + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts = 1; + */ + public Builder setTs(com.google.protobuf.Timestamp value) { + if (tsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ts_ = value; + onChanged(); + } else { + tsBuilder_.setMessage(value); + } + + return this; + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts = 1; + */ + public Builder setTs( + com.google.protobuf.Timestamp.Builder builderForValue) { + if (tsBuilder_ == null) { + ts_ = builderForValue.build(); + onChanged(); + } else { + tsBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts = 1; + */ + public Builder mergeTs(com.google.protobuf.Timestamp value) { + if (tsBuilder_ == null) { + if (ts_ != null) { + ts_ = + com.google.protobuf.Timestamp.newBuilder(ts_).mergeFrom(value).buildPartial(); + } else { + ts_ = value; + } + onChanged(); + } else { + tsBuilder_.mergeFrom(value); + } + + return this; + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts = 1; + */ + public Builder clearTs() { + if (tsBuilder_ == null) { + ts_ = null; + onChanged(); + } else { + ts_ = null; + tsBuilder_ = null; + } + + return this; + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts = 1; + */ + public com.google.protobuf.Timestamp.Builder getTsBuilder() { + + onChanged(); + return getTsFieldBuilder().getBuilder(); + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts = 1; + */ + public com.google.protobuf.TimestampOrBuilder getTsOrBuilder() { + if (tsBuilder_ != null) { + return tsBuilder_.getMessageOrBuilder(); + } else { + return ts_ == null ? + com.google.protobuf.Timestamp.getDefaultInstance() : ts_; + } + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> + getTsFieldBuilder() { + if (tsBuilder_ == null) { + tsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>( + getTs(), + getParentForChildren(), + isClean()); + ts_ = null; + } + return tsBuilder_; + } + + private com.google.protobuf.ByteString bs_ = com.google.protobuf.ByteString.EMPTY; + /** + *
+       * byte[] vs ByteString
+       * 
+ * + * bytes bs = 2; + * @return The bs. + */ + @java.lang.Override + public com.google.protobuf.ByteString getBs() { + return bs_; + } + /** + *
+       * byte[] vs ByteString
+       * 
+ * + * bytes bs = 2; + * @param value The bs to set. + * @return This builder for chaining. + */ + public Builder setBs(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + + bs_ = value; + onChanged(); + return this; + } + /** + *
+       * byte[] vs ByteString
+       * 
+ * + * bytes bs = 2; + * @return This builder for chaining. + */ + public Builder clearBs() { + + bs_ = getDefaultInstance().getBs(); + onChanged(); + return this; + } + + private com.google.protobuf.MapField< + java.lang.Integer, java.lang.Integer> mp_; + private com.google.protobuf.MapField + internalGetMp() { + if (mp_ == null) { + return com.google.protobuf.MapField.emptyMapField( + MpDefaultEntryHolder.defaultEntry); + } + return mp_; + } + private com.google.protobuf.MapField + internalGetMutableMp() { + onChanged();; + if (mp_ == null) { + mp_ = com.google.protobuf.MapField.newMapField( + MpDefaultEntryHolder.defaultEntry); + } + if (!mp_.isMutable()) { + mp_ = mp_.copy(); + } + return mp_; + } + + public int getMpCount() { + return internalGetMp().getMap().size(); + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp = 3; + */ + + @java.lang.Override + public boolean containsMp( + int key) { + + return internalGetMp().getMap().containsKey(key); + } + /** + * Use {@link #getMpMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getMp() { + return getMpMap(); + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp = 3; + */ + @java.lang.Override + + public java.util.Map getMpMap() { + return internalGetMp().getMap(); + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp = 3; + */ + @java.lang.Override + + public int getMpOrDefault( + int key, + int defaultValue) { + + java.util.Map map = + internalGetMp().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp = 3; + */ + @java.lang.Override + + public int getMpOrThrow( + int key) { + + java.util.Map map = + internalGetMp().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + + public Builder clearMp() { + internalGetMutableMp().getMutableMap() + .clear(); + return this; + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp = 3; + */ + + public Builder removeMp( + int key) { + + internalGetMutableMp().getMutableMap() + .remove(key); + return this; + } + /** + * Use alternate mutation accessors instead. + */ + @java.lang.Deprecated + public java.util.Map + getMutableMp() { + return internalGetMutableMp().getMutableMap(); + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp = 3; + */ + public Builder putMp( + int key, + int value) { + + + internalGetMutableMp().getMutableMap() + .put(key, value); + return this; + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp = 3; + */ + + public Builder putAllMp( + java.util.Map values) { + internalGetMutableMp().getMutableMap() + .putAll(values); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.SpecialTypes) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.SpecialTypes) + private static final SpecialTypesV1.SpecialTypes DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new SpecialTypesV1.SpecialTypes(); + } + + public static SpecialTypesV1.SpecialTypes getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public SpecialTypes parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public SpecialTypesV1.SpecialTypes getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_SpecialTypes_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_SpecialTypes_MpEntry_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_SpecialTypes_MpEntry_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\024SpecialTypesV1.proto\022\027io.deephaven.kaf" + + "ka.test\032\037google/protobuf/timestamp.proto" + + "\"\250\001\n\014SpecialTypes\022&\n\002ts\030\001 \001(\0132\032.google.p" + + "rotobuf.Timestamp\022\n\n\002bs\030\002 \001(\014\0229\n\002mp\030\003 \003(" + + "\0132-.io.deephaven.kafka.test.SpecialTypes" + + ".MpEntry\032)\n\007MpEntry\022\013\n\003key\030\001 \001(\005\022\r\n\005valu" + + "e\030\002 \001(\005:\0028\001B\004H\001P\000b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + com.google.protobuf.TimestampProto.getDescriptor(), + }); + internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_io_deephaven_kafka_test_SpecialTypes_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor, + new java.lang.String[] { "Ts", "Bs", "Mp", }); + internal_static_io_deephaven_kafka_test_SpecialTypes_MpEntry_descriptor = + internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor.getNestedTypes().get(0); + internal_static_io_deephaven_kafka_test_SpecialTypes_MpEntry_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_SpecialTypes_MpEntry_descriptor, + new java.lang.String[] { "Key", "Value", }); + com.google.protobuf.TimestampProto.getDescriptor(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/SpecialTypesV2.java b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/SpecialTypesV2.java new file mode 100644 index 00000000000..0264d27b022 --- /dev/null +++ b/extensions/kafka/src/test/java/io/deephaven/kafka/protobuf/gen/SpecialTypesV2.java @@ -0,0 +1,1195 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: SpecialTypesV2.proto + +package io.deephaven.kafka.protobuf.gen; + +public final class SpecialTypesV2 { + private SpecialTypesV2() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + public interface SpecialTypesOrBuilder extends + // @@protoc_insertion_point(interface_extends:io.deephaven.kafka.test.SpecialTypes) + com.google.protobuf.MessageOrBuilder { + + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + * @return Whether the tsRenamed field is set. + */ + boolean hasTsRenamed(); + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + * @return The tsRenamed. + */ + com.google.protobuf.Timestamp getTsRenamed(); + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + */ + com.google.protobuf.TimestampOrBuilder getTsRenamedOrBuilder(); + + /** + *
+     * byte[] vs ByteString
+     * 
+ * + * bytes bs_renamed = 2; + * @return The bsRenamed. + */ + com.google.protobuf.ByteString getBsRenamed(); + + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + int getMpRenamedCount(); + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + boolean containsMpRenamed( + int key); + /** + * Use {@link #getMpRenamedMap()} instead. + */ + @java.lang.Deprecated + java.util.Map + getMpRenamed(); + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + java.util.Map + getMpRenamedMap(); + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + + int getMpRenamedOrDefault( + int key, + int defaultValue); + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + + int getMpRenamedOrThrow( + int key); + } + /** + * Protobuf type {@code io.deephaven.kafka.test.SpecialTypes} + */ + public static final class SpecialTypes extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:io.deephaven.kafka.test.SpecialTypes) + SpecialTypesOrBuilder { + private static final long serialVersionUID = 0L; + // Use SpecialTypes.newBuilder() to construct. + private SpecialTypes(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private SpecialTypes() { + bsRenamed_ = com.google.protobuf.ByteString.EMPTY; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new SpecialTypes(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return SpecialTypesV2.internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + @java.lang.Override + protected com.google.protobuf.MapField internalGetMapField( + int number) { + switch (number) { + case 3: + return internalGetMpRenamed(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return SpecialTypesV2.internal_static_io_deephaven_kafka_test_SpecialTypes_fieldAccessorTable + .ensureFieldAccessorsInitialized( + SpecialTypesV2.SpecialTypes.class, SpecialTypesV2.SpecialTypes.Builder.class); + } + + public static final int TS_RENAMED_FIELD_NUMBER = 1; + private com.google.protobuf.Timestamp tsRenamed_; + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + * @return Whether the tsRenamed field is set. + */ + @java.lang.Override + public boolean hasTsRenamed() { + return tsRenamed_ != null; + } + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + * @return The tsRenamed. + */ + @java.lang.Override + public com.google.protobuf.Timestamp getTsRenamed() { + return tsRenamed_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : tsRenamed_; + } + /** + *
+     * well-known
+     * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + */ + @java.lang.Override + public com.google.protobuf.TimestampOrBuilder getTsRenamedOrBuilder() { + return getTsRenamed(); + } + + public static final int BS_RENAMED_FIELD_NUMBER = 2; + private com.google.protobuf.ByteString bsRenamed_; + /** + *
+     * byte[] vs ByteString
+     * 
+ * + * bytes bs_renamed = 2; + * @return The bsRenamed. + */ + @java.lang.Override + public com.google.protobuf.ByteString getBsRenamed() { + return bsRenamed_; + } + + public static final int MP_RENAMED_FIELD_NUMBER = 3; + private static final class MpRenamedDefaultEntryHolder { + static final com.google.protobuf.MapEntry< + java.lang.Integer, java.lang.Integer> defaultEntry = + com.google.protobuf.MapEntry + .newDefaultInstance( + SpecialTypesV2.internal_static_io_deephaven_kafka_test_SpecialTypes_MpRenamedEntry_descriptor, + com.google.protobuf.WireFormat.FieldType.INT32, + 0, + com.google.protobuf.WireFormat.FieldType.INT32, + 0); + } + private com.google.protobuf.MapField< + java.lang.Integer, java.lang.Integer> mpRenamed_; + private com.google.protobuf.MapField + internalGetMpRenamed() { + if (mpRenamed_ == null) { + return com.google.protobuf.MapField.emptyMapField( + MpRenamedDefaultEntryHolder.defaultEntry); + } + return mpRenamed_; + } + + public int getMpRenamedCount() { + return internalGetMpRenamed().getMap().size(); + } + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + + @java.lang.Override + public boolean containsMpRenamed( + int key) { + + return internalGetMpRenamed().getMap().containsKey(key); + } + /** + * Use {@link #getMpRenamedMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getMpRenamed() { + return getMpRenamedMap(); + } + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + @java.lang.Override + + public java.util.Map getMpRenamedMap() { + return internalGetMpRenamed().getMap(); + } + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + @java.lang.Override + + public int getMpRenamedOrDefault( + int key, + int defaultValue) { + + java.util.Map map = + internalGetMpRenamed().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + *
+     * Map vs repeated MapFieldEntry
+     * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + @java.lang.Override + + public int getMpRenamedOrThrow( + int key) { + + java.util.Map map = + internalGetMpRenamed().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (tsRenamed_ != null) { + output.writeMessage(1, getTsRenamed()); + } + if (!bsRenamed_.isEmpty()) { + output.writeBytes(2, bsRenamed_); + } + com.google.protobuf.GeneratedMessageV3 + .serializeIntegerMapTo( + output, + internalGetMpRenamed(), + MpRenamedDefaultEntryHolder.defaultEntry, + 3); + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (tsRenamed_ != null) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getTsRenamed()); + } + if (!bsRenamed_.isEmpty()) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, bsRenamed_); + } + for (java.util.Map.Entry entry + : internalGetMpRenamed().getMap().entrySet()) { + com.google.protobuf.MapEntry + mpRenamed__ = MpRenamedDefaultEntryHolder.defaultEntry.newBuilderForType() + .setKey(entry.getKey()) + .setValue(entry.getValue()) + .build(); + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, mpRenamed__); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof SpecialTypesV2.SpecialTypes)) { + return super.equals(obj); + } + SpecialTypesV2.SpecialTypes other = (SpecialTypesV2.SpecialTypes) obj; + + if (hasTsRenamed() != other.hasTsRenamed()) return false; + if (hasTsRenamed()) { + if (!getTsRenamed() + .equals(other.getTsRenamed())) return false; + } + if (!getBsRenamed() + .equals(other.getBsRenamed())) return false; + if (!internalGetMpRenamed().equals( + other.internalGetMpRenamed())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasTsRenamed()) { + hash = (37 * hash) + TS_RENAMED_FIELD_NUMBER; + hash = (53 * hash) + getTsRenamed().hashCode(); + } + hash = (37 * hash) + BS_RENAMED_FIELD_NUMBER; + hash = (53 * hash) + getBsRenamed().hashCode(); + if (!internalGetMpRenamed().getMap().isEmpty()) { + hash = (37 * hash) + MP_RENAMED_FIELD_NUMBER; + hash = (53 * hash) + internalGetMpRenamed().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static SpecialTypesV2.SpecialTypes parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static SpecialTypesV2.SpecialTypes parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static SpecialTypesV2.SpecialTypes parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static SpecialTypesV2.SpecialTypes parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static SpecialTypesV2.SpecialTypes parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static SpecialTypesV2.SpecialTypes parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static SpecialTypesV2.SpecialTypes parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static SpecialTypesV2.SpecialTypes parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static SpecialTypesV2.SpecialTypes parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static SpecialTypesV2.SpecialTypes parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static SpecialTypesV2.SpecialTypes parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static SpecialTypesV2.SpecialTypes parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(SpecialTypesV2.SpecialTypes prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code io.deephaven.kafka.test.SpecialTypes} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:io.deephaven.kafka.test.SpecialTypes) + SpecialTypesV2.SpecialTypesOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return SpecialTypesV2.internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapField internalGetMapField( + int number) { + switch (number) { + case 3: + return internalGetMpRenamed(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapField internalGetMutableMapField( + int number) { + switch (number) { + case 3: + return internalGetMutableMpRenamed(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return SpecialTypesV2.internal_static_io_deephaven_kafka_test_SpecialTypes_fieldAccessorTable + .ensureFieldAccessorsInitialized( + SpecialTypesV2.SpecialTypes.class, SpecialTypesV2.SpecialTypes.Builder.class); + } + + // Construct using io.deephaven.kafka.test.SpecialTypesV2.SpecialTypes.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + if (tsRenamedBuilder_ == null) { + tsRenamed_ = null; + } else { + tsRenamed_ = null; + tsRenamedBuilder_ = null; + } + bsRenamed_ = com.google.protobuf.ByteString.EMPTY; + + internalGetMutableMpRenamed().clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return SpecialTypesV2.internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor; + } + + @java.lang.Override + public SpecialTypesV2.SpecialTypes getDefaultInstanceForType() { + return SpecialTypesV2.SpecialTypes.getDefaultInstance(); + } + + @java.lang.Override + public SpecialTypesV2.SpecialTypes build() { + SpecialTypesV2.SpecialTypes result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public SpecialTypesV2.SpecialTypes buildPartial() { + SpecialTypesV2.SpecialTypes result = new SpecialTypesV2.SpecialTypes(this); + int from_bitField0_ = bitField0_; + if (tsRenamedBuilder_ == null) { + result.tsRenamed_ = tsRenamed_; + } else { + result.tsRenamed_ = tsRenamedBuilder_.build(); + } + result.bsRenamed_ = bsRenamed_; + result.mpRenamed_ = internalGetMpRenamed(); + result.mpRenamed_.makeImmutable(); + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof SpecialTypesV2.SpecialTypes) { + return mergeFrom((SpecialTypesV2.SpecialTypes)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(SpecialTypesV2.SpecialTypes other) { + if (other == SpecialTypesV2.SpecialTypes.getDefaultInstance()) return this; + if (other.hasTsRenamed()) { + mergeTsRenamed(other.getTsRenamed()); + } + if (other.getBsRenamed() != com.google.protobuf.ByteString.EMPTY) { + setBsRenamed(other.getBsRenamed()); + } + internalGetMutableMpRenamed().mergeFrom( + other.internalGetMpRenamed()); + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getTsRenamedFieldBuilder().getBuilder(), + extensionRegistry); + + break; + } // case 10 + case 18: { + bsRenamed_ = input.readBytes(); + + break; + } // case 18 + case 26: { + com.google.protobuf.MapEntry + mpRenamed__ = input.readMessage( + MpRenamedDefaultEntryHolder.defaultEntry.getParserForType(), extensionRegistry); + internalGetMutableMpRenamed().getMutableMap().put( + mpRenamed__.getKey(), mpRenamed__.getValue()); + break; + } // case 26 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private com.google.protobuf.Timestamp tsRenamed_; + private com.google.protobuf.SingleFieldBuilderV3< + com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> tsRenamedBuilder_; + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + * @return Whether the tsRenamed field is set. + */ + public boolean hasTsRenamed() { + return tsRenamedBuilder_ != null || tsRenamed_ != null; + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + * @return The tsRenamed. + */ + public com.google.protobuf.Timestamp getTsRenamed() { + if (tsRenamedBuilder_ == null) { + return tsRenamed_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : tsRenamed_; + } else { + return tsRenamedBuilder_.getMessage(); + } + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + */ + public Builder setTsRenamed(com.google.protobuf.Timestamp value) { + if (tsRenamedBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + tsRenamed_ = value; + onChanged(); + } else { + tsRenamedBuilder_.setMessage(value); + } + + return this; + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + */ + public Builder setTsRenamed( + com.google.protobuf.Timestamp.Builder builderForValue) { + if (tsRenamedBuilder_ == null) { + tsRenamed_ = builderForValue.build(); + onChanged(); + } else { + tsRenamedBuilder_.setMessage(builderForValue.build()); + } + + return this; + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + */ + public Builder mergeTsRenamed(com.google.protobuf.Timestamp value) { + if (tsRenamedBuilder_ == null) { + if (tsRenamed_ != null) { + tsRenamed_ = + com.google.protobuf.Timestamp.newBuilder(tsRenamed_).mergeFrom(value).buildPartial(); + } else { + tsRenamed_ = value; + } + onChanged(); + } else { + tsRenamedBuilder_.mergeFrom(value); + } + + return this; + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + */ + public Builder clearTsRenamed() { + if (tsRenamedBuilder_ == null) { + tsRenamed_ = null; + onChanged(); + } else { + tsRenamed_ = null; + tsRenamedBuilder_ = null; + } + + return this; + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + */ + public com.google.protobuf.Timestamp.Builder getTsRenamedBuilder() { + + onChanged(); + return getTsRenamedFieldBuilder().getBuilder(); + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + */ + public com.google.protobuf.TimestampOrBuilder getTsRenamedOrBuilder() { + if (tsRenamedBuilder_ != null) { + return tsRenamedBuilder_.getMessageOrBuilder(); + } else { + return tsRenamed_ == null ? + com.google.protobuf.Timestamp.getDefaultInstance() : tsRenamed_; + } + } + /** + *
+       * well-known
+       * 
+ * + * .google.protobuf.Timestamp ts_renamed = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> + getTsRenamedFieldBuilder() { + if (tsRenamedBuilder_ == null) { + tsRenamedBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>( + getTsRenamed(), + getParentForChildren(), + isClean()); + tsRenamed_ = null; + } + return tsRenamedBuilder_; + } + + private com.google.protobuf.ByteString bsRenamed_ = com.google.protobuf.ByteString.EMPTY; + /** + *
+       * byte[] vs ByteString
+       * 
+ * + * bytes bs_renamed = 2; + * @return The bsRenamed. + */ + @java.lang.Override + public com.google.protobuf.ByteString getBsRenamed() { + return bsRenamed_; + } + /** + *
+       * byte[] vs ByteString
+       * 
+ * + * bytes bs_renamed = 2; + * @param value The bsRenamed to set. + * @return This builder for chaining. + */ + public Builder setBsRenamed(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + + bsRenamed_ = value; + onChanged(); + return this; + } + /** + *
+       * byte[] vs ByteString
+       * 
+ * + * bytes bs_renamed = 2; + * @return This builder for chaining. + */ + public Builder clearBsRenamed() { + + bsRenamed_ = getDefaultInstance().getBsRenamed(); + onChanged(); + return this; + } + + private com.google.protobuf.MapField< + java.lang.Integer, java.lang.Integer> mpRenamed_; + private com.google.protobuf.MapField + internalGetMpRenamed() { + if (mpRenamed_ == null) { + return com.google.protobuf.MapField.emptyMapField( + MpRenamedDefaultEntryHolder.defaultEntry); + } + return mpRenamed_; + } + private com.google.protobuf.MapField + internalGetMutableMpRenamed() { + onChanged();; + if (mpRenamed_ == null) { + mpRenamed_ = com.google.protobuf.MapField.newMapField( + MpRenamedDefaultEntryHolder.defaultEntry); + } + if (!mpRenamed_.isMutable()) { + mpRenamed_ = mpRenamed_.copy(); + } + return mpRenamed_; + } + + public int getMpRenamedCount() { + return internalGetMpRenamed().getMap().size(); + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + + @java.lang.Override + public boolean containsMpRenamed( + int key) { + + return internalGetMpRenamed().getMap().containsKey(key); + } + /** + * Use {@link #getMpRenamedMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getMpRenamed() { + return getMpRenamedMap(); + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + @java.lang.Override + + public java.util.Map getMpRenamedMap() { + return internalGetMpRenamed().getMap(); + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + @java.lang.Override + + public int getMpRenamedOrDefault( + int key, + int defaultValue) { + + java.util.Map map = + internalGetMpRenamed().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + @java.lang.Override + + public int getMpRenamedOrThrow( + int key) { + + java.util.Map map = + internalGetMpRenamed().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + + public Builder clearMpRenamed() { + internalGetMutableMpRenamed().getMutableMap() + .clear(); + return this; + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + + public Builder removeMpRenamed( + int key) { + + internalGetMutableMpRenamed().getMutableMap() + .remove(key); + return this; + } + /** + * Use alternate mutation accessors instead. + */ + @java.lang.Deprecated + public java.util.Map + getMutableMpRenamed() { + return internalGetMutableMpRenamed().getMutableMap(); + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + public Builder putMpRenamed( + int key, + int value) { + + + internalGetMutableMpRenamed().getMutableMap() + .put(key, value); + return this; + } + /** + *
+       * Map vs repeated MapFieldEntry
+       * 
+ * + * map<int32, int32> mp_renamed = 3; + */ + + public Builder putAllMpRenamed( + java.util.Map values) { + internalGetMutableMpRenamed().getMutableMap() + .putAll(values); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:io.deephaven.kafka.test.SpecialTypes) + } + + // @@protoc_insertion_point(class_scope:io.deephaven.kafka.test.SpecialTypes) + private static final SpecialTypesV2.SpecialTypes DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new SpecialTypesV2.SpecialTypes(); + } + + public static SpecialTypesV2.SpecialTypes getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public SpecialTypes parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public SpecialTypesV2.SpecialTypes getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_SpecialTypes_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_io_deephaven_kafka_test_SpecialTypes_MpRenamedEntry_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_io_deephaven_kafka_test_SpecialTypes_MpRenamedEntry_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\024SpecialTypesV2.proto\022\027io.deephaven.kaf" + + "ka.test\032\037google/protobuf/timestamp.proto" + + "\"\316\001\n\014SpecialTypes\022.\n\nts_renamed\030\001 \001(\0132\032." + + "google.protobuf.Timestamp\022\022\n\nbs_renamed\030" + + "\002 \001(\014\022H\n\nmp_renamed\030\003 \003(\01324.io.deephaven" + + ".kafka.test.SpecialTypes.MpRenamedEntry\032" + + "0\n\016MpRenamedEntry\022\013\n\003key\030\001 \001(\005\022\r\n\005value\030" + + "\002 \001(\005:\0028\001B\004H\001P\000b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + com.google.protobuf.TimestampProto.getDescriptor(), + }); + internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_io_deephaven_kafka_test_SpecialTypes_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor, + new java.lang.String[] { "TsRenamed", "BsRenamed", "MpRenamed", }); + internal_static_io_deephaven_kafka_test_SpecialTypes_MpRenamedEntry_descriptor = + internal_static_io_deephaven_kafka_test_SpecialTypes_descriptor.getNestedTypes().get(0); + internal_static_io_deephaven_kafka_test_SpecialTypes_MpRenamedEntry_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_io_deephaven_kafka_test_SpecialTypes_MpRenamedEntry_descriptor, + new java.lang.String[] { "Key", "Value", }); + com.google.protobuf.TimestampProto.getDescriptor(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions/kafka/src/test/proto/BoolV1.proto.txt b/extensions/kafka/src/test/proto/BoolV1.proto.txt new file mode 100644 index 00000000000..5612eb2f6ca --- /dev/null +++ b/extensions/kafka/src/test/proto/BoolV1.proto.txt @@ -0,0 +1,13 @@ +/* + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +package io.deephaven.kafka.protobuf.gen; + +option java_multiple_files = false; +option optimize_for = SPEED; + +message MyBool { + bool my_bool = 1; +} \ No newline at end of file diff --git a/extensions/kafka/src/test/proto/BoolV2.proto.txt b/extensions/kafka/src/test/proto/BoolV2.proto.txt new file mode 100644 index 00000000000..9f45ba86339 --- /dev/null +++ b/extensions/kafka/src/test/proto/BoolV2.proto.txt @@ -0,0 +1,13 @@ +/* + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +package io.deephaven.kafka.protobuf.gen; + +option java_multiple_files = false; +option optimize_for = SPEED; + +message MyBool { + +} \ No newline at end of file diff --git a/extensions/kafka/src/test/proto/MyMessageV1.proto.txt b/extensions/kafka/src/test/proto/MyMessageV1.proto.txt new file mode 100644 index 00000000000..290e7ec36b3 --- /dev/null +++ b/extensions/kafka/src/test/proto/MyMessageV1.proto.txt @@ -0,0 +1,13 @@ +/* + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +package io.deephaven.kafka.protobuf.gen; + +option java_multiple_files = false; +option optimize_for = SPEED; + +message MyMessage { + string name = 1; +} \ No newline at end of file diff --git a/extensions/kafka/src/test/proto/MyMessageV2.proto.txt b/extensions/kafka/src/test/proto/MyMessageV2.proto.txt new file mode 100644 index 00000000000..7eeadb19f71 --- /dev/null +++ b/extensions/kafka/src/test/proto/MyMessageV2.proto.txt @@ -0,0 +1,14 @@ +/* + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +package io.deephaven.kafka.protobuf.gen; + +option java_multiple_files = false; +option optimize_for = SPEED; + +message MyMessage { + string name = 1; + int32 age = 2; +} \ No newline at end of file diff --git a/extensions/kafka/src/test/proto/MyMessageV3.proto.txt b/extensions/kafka/src/test/proto/MyMessageV3.proto.txt new file mode 100644 index 00000000000..0ab9947e6a1 --- /dev/null +++ b/extensions/kafka/src/test/proto/MyMessageV3.proto.txt @@ -0,0 +1,21 @@ +/* + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +package io.deephaven.kafka.protobuf.gen; + +option java_multiple_files = false; +option optimize_for = SPEED; + +message MyMessage { + message FirstAndLast { + string first_name = 1; + string last_name = 2; + } + oneof name_type { + string name = 1; + FirstAndLast first_and_last = 3; + } + int32 age = 2; +} \ No newline at end of file diff --git a/extensions/kafka/src/test/proto/MyMessageV4.proto.txt b/extensions/kafka/src/test/proto/MyMessageV4.proto.txt new file mode 100644 index 00000000000..277c1b4bb5f --- /dev/null +++ b/extensions/kafka/src/test/proto/MyMessageV4.proto.txt @@ -0,0 +1,24 @@ +/* + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +package io.deephaven.kafka.protobuf.gen; + +option java_multiple_files = false; +option optimize_for = SPEED; + +message MyMessage { + message FirstAndLast { + string first_name = 1; + string last_name = 2; + } + oneof name_type { + string name = 1; + FirstAndLast first_and_last = 3; + } + oneof age_type { + int32 age = 2; + float agef = 4; + } +} \ No newline at end of file diff --git a/extensions/kafka/src/test/proto/MyMessageV5.proto.txt b/extensions/kafka/src/test/proto/MyMessageV5.proto.txt new file mode 100644 index 00000000000..91662f00cda --- /dev/null +++ b/extensions/kafka/src/test/proto/MyMessageV5.proto.txt @@ -0,0 +1,25 @@ +/* + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +package io.deephaven.kafka.protobuf.gen; + +option java_multiple_files = false; +option optimize_for = SPEED; + +message MyMessage { + message FirstAndLast { + string first_name = 1; + // an invalid schema change + int64 last_name = 2; + } + oneof name_type { + string name = 1; + FirstAndLast first_and_last = 3; + } + oneof age_type { + int32 age = 2; + float agef = 4; + } +} \ No newline at end of file diff --git a/extensions/kafka/src/test/proto/RenameV1.proto.txt b/extensions/kafka/src/test/proto/RenameV1.proto.txt new file mode 100644 index 00000000000..2eb994d23d9 --- /dev/null +++ b/extensions/kafka/src/test/proto/RenameV1.proto.txt @@ -0,0 +1,13 @@ +/* + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +package io.deephaven.kafka.protobuf.gen; + +option java_multiple_files = false; +option optimize_for = SPEED; + +message Rename { + string name = 1; +} \ No newline at end of file diff --git a/extensions/kafka/src/test/proto/RenameV2.proto.txt b/extensions/kafka/src/test/proto/RenameV2.proto.txt new file mode 100644 index 00000000000..7aa26bcc946 --- /dev/null +++ b/extensions/kafka/src/test/proto/RenameV2.proto.txt @@ -0,0 +1,14 @@ +/* + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +package io.deephaven.kafka.protobuf.gen; + +option java_multiple_files = false; +option optimize_for = SPEED; + +message Rename { + string name_old = 1; + string name = 2; +} \ No newline at end of file diff --git a/extensions/kafka/src/test/proto/SpecialTypesV1.proto.txt b/extensions/kafka/src/test/proto/SpecialTypesV1.proto.txt new file mode 100644 index 00000000000..90f22808da9 --- /dev/null +++ b/extensions/kafka/src/test/proto/SpecialTypesV1.proto.txt @@ -0,0 +1,22 @@ +/* + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +import "google/protobuf/timestamp.proto"; + +package io.deephaven.kafka.protobuf.gen; + +option java_multiple_files = false; +option optimize_for = SPEED; + +message SpecialTypes { + // well-known + google.protobuf.Timestamp ts = 1; + + // byte[] vs ByteString + bytes bs = 2; + + // Map vs repeated MapFieldEntry + map mp = 3; +} \ No newline at end of file diff --git a/extensions/kafka/src/test/proto/SpecialTypesV2.proto.txt b/extensions/kafka/src/test/proto/SpecialTypesV2.proto.txt new file mode 100644 index 00000000000..f10ddab6132 --- /dev/null +++ b/extensions/kafka/src/test/proto/SpecialTypesV2.proto.txt @@ -0,0 +1,22 @@ +/* + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +import "google/protobuf/timestamp.proto"; + +package io.deephaven.kafka.protobuf.gen; + +option java_multiple_files = false; +option optimize_for = SPEED; + +message SpecialTypes { + // well-known + google.protobuf.Timestamp ts_renamed = 1; + + // byte[] vs ByteString + bytes bs_renamed = 2; + + // Map vs repeated MapFieldEntry + map mp_renamed = 3; +} \ No newline at end of file diff --git a/extensions/protobuf/build.gradle b/extensions/protobuf/build.gradle new file mode 100644 index 00000000000..7c4dcd6eedd --- /dev/null +++ b/extensions/protobuf/build.gradle @@ -0,0 +1,44 @@ +plugins { + id 'java-library' + id 'io.deephaven.project.register' + id 'com.google.protobuf' version '0.9.4' +} + +dependencies { + api project(':qst') + api 'com.google.protobuf:protobuf-java:3.24.1' + implementation project(':engine-query-constants') + + Classpaths.inheritImmutables(project) + + Classpaths.inheritJUnitPlatform(project) + Classpaths.inheritAssertJ(project) + + testImplementation 'org.junit.jupiter:junit-jupiter' + testRuntimeOnly 'org.junit.jupiter:junit-jupiter-engine' + + testRuntimeOnly project(':log-to-slf4j') + testRuntimeOnly project(':configs') + testRuntimeOnly project(':test-configs') + Classpaths.inheritSlf4j(project, 'slf4j-simple', 'testRuntimeOnly') + + Classpaths.inheritAutoService(project, true) +} + +test { + useJUnitPlatform() +} + +spotless { + java { + targetExclude( + 'build/generated/source/proto/test/java/**', + ) + } +} + +protobuf { + protoc { + artifact = 'com.google.protobuf:protoc:3.24.1' + } +} diff --git a/extensions/protobuf/gradle.properties b/extensions/protobuf/gradle.properties new file mode 100644 index 00000000000..c186bbfdde1 --- /dev/null +++ b/extensions/protobuf/gradle.properties @@ -0,0 +1 @@ +io.deephaven.project.ProjectType=JAVA_PUBLIC diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/BooleanFunctions.java b/extensions/protobuf/src/main/java/io/deephaven/functions/BooleanFunctions.java new file mode 100644 index 00000000000..1a9e581cb01 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/BooleanFunctions.java @@ -0,0 +1,155 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import java.util.Collection; +import java.util.List; +import java.util.Objects; +import java.util.function.Function; +import java.util.function.Predicate; + +class BooleanFunctions { + + static ToBooleanFunction cast() { + // noinspection unchecked + return (ToBooleanFunction) PrimitiveBoolean.INSTANCE; + } + + static ToBooleanFunction of(Predicate predicate) { + return predicate instanceof ToBooleanFunction ? (ToBooleanFunction) predicate : predicate::test; + } + + public static ToBooleanFunction ofTrue() { + // noinspection unchecked + return (ToBooleanFunction) OfTrue.INSTANCE; + } + + public static ToBooleanFunction ofFalse() { + // noinspection unchecked + return (ToBooleanFunction) OfFalse.INSTANCE; + } + + static ToBooleanFunction map(Function f, Predicate g) { + return new BooleanMap<>(f, g); + } + + static ToBooleanFunction not(ToBooleanFunction f) { + return f instanceof BooleanNot ? of(((BooleanNot) f).function()) : new BooleanNot<>(f); + } + + static ToBooleanFunction or(Collection> functions) { + if (functions.isEmpty()) { + return ofFalse(); + } + if (functions.size() == 1) { + return of(functions.iterator().next()); + } + return new BooleanOr<>(functions); + } + + static ToBooleanFunction and(Collection> functions) { + if (functions.isEmpty()) { + return ofTrue(); + } + if (functions.size() == 1) { + return of(functions.iterator().next()); + } + return new BooleanAnd<>(functions); + } + + private enum OfTrue implements ToBooleanFunction { + INSTANCE; + + @Override + public boolean test(Object value) { + return true; + } + } + + private enum OfFalse implements ToBooleanFunction { + INSTANCE; + + @Override + public boolean test(Object value) { + return false; + } + } + + private enum PrimitiveBoolean implements ToBooleanFunction { + INSTANCE; + + @Override + public boolean test(Object value) { + return (boolean) value; + } + } + + private static class BooleanMap implements ToBooleanFunction { + private final Function f; + private final Predicate g; + + public BooleanMap(Function f, Predicate g) { + this.f = Objects.requireNonNull(f); + this.g = Objects.requireNonNull(g); + } + + @Override + public boolean test(T value) { + return g.test(f.apply(value)); + } + } + + private static class BooleanNot implements ToBooleanFunction { + private final Predicate function; + + public BooleanNot(ToBooleanFunction function) { + this.function = Objects.requireNonNull(function); + } + + public Predicate function() { + return function; + } + + @Override + public boolean test(T value) { + return !function.test(value); + } + } + + private static class BooleanAnd implements ToBooleanFunction { + private final Collection> functions; + + public BooleanAnd(Collection> functions) { + this.functions = List.copyOf(functions); + } + + @Override + public boolean test(T value) { + for (Predicate function : functions) { + if (!function.test(value)) { + return false; + } + } + return true; + } + } + + private static class BooleanOr implements ToBooleanFunction { + private final Collection> functions; + + public BooleanOr(Collection> functions) { + this.functions = List.copyOf(functions); + } + + @Override + public boolean test(T value) { + for (Predicate function : functions) { + if (function.test(value)) { + return true; + } + } + return false; + } + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ByteFunctions.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ByteFunctions.java new file mode 100644 index 00000000000..b1b3cdf3e6b --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ByteFunctions.java @@ -0,0 +1,42 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import java.util.Objects; +import java.util.function.Function; + +class ByteFunctions { + static ToByteFunction cast() { + // noinspection unchecked + return (ToByteFunction) PrimitiveByte.INSTANCE; + } + + static ToByteFunction map(Function f, ToByteFunction g) { + return new ByteMap<>(f, g); + } + + private enum PrimitiveByte implements ToByteFunction { + INSTANCE; + + @Override + public byte applyAsByte(Object value) { + return (byte) value; + } + } + + private static class ByteMap implements ToByteFunction { + private final Function f; + private final ToByteFunction g; + + public ByteMap(Function f, ToByteFunction g) { + this.f = Objects.requireNonNull(f); + this.g = Objects.requireNonNull(g); + } + + @Override + public byte applyAsByte(T value) { + return g.applyAsByte(f.apply(value)); + } + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/CharFunctions.java b/extensions/protobuf/src/main/java/io/deephaven/functions/CharFunctions.java new file mode 100644 index 00000000000..4921bbb7f6d --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/CharFunctions.java @@ -0,0 +1,42 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import java.util.Objects; +import java.util.function.Function; + +class CharFunctions { + static ToCharFunction cast() { + // noinspection unchecked + return (ToCharFunction) PrimitiveChar.INSTANCE; + } + + static ToCharFunction map(Function f, ToCharFunction g) { + return new CharMap<>(f, g); + } + + private enum PrimitiveChar implements ToCharFunction { + INSTANCE; + + @Override + public char applyAsChar(Object value) { + return (char) value; + } + } + + private static class CharMap implements ToCharFunction { + private final Function f; + private final ToCharFunction g; + + public CharMap(Function f, ToCharFunction g) { + this.f = Objects.requireNonNull(f); + this.g = Objects.requireNonNull(g); + } + + @Override + public char applyAsChar(T value) { + return g.applyAsChar(f.apply(value)); + } + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/DoubleFunctions.java b/extensions/protobuf/src/main/java/io/deephaven/functions/DoubleFunctions.java new file mode 100644 index 00000000000..bbe3ff205ed --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/DoubleFunctions.java @@ -0,0 +1,47 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import java.util.Objects; +import java.util.function.Function; + +class DoubleFunctions { + + static ToDoubleFunction cast() { + // noinspection unchecked + return (ToDoubleFunction) PrimitiveDouble.INSTANCE; + } + + static ToDoubleFunction of(java.util.function.ToDoubleFunction f) { + return f instanceof ToDoubleFunction ? (ToDoubleFunction) f : f::applyAsDouble; + } + + static ToDoubleFunction map(Function f, java.util.function.ToDoubleFunction g) { + return new DoubleFunctionMap<>(f, g); + } + + private enum PrimitiveDouble implements ToDoubleFunction { + INSTANCE; + + @Override + public double applyAsDouble(Object value) { + return (double) value; + } + } + + private static class DoubleFunctionMap implements ToDoubleFunction { + private final Function f; + private final java.util.function.ToDoubleFunction g; + + public DoubleFunctionMap(Function f, java.util.function.ToDoubleFunction g) { + this.f = Objects.requireNonNull(f); + this.g = Objects.requireNonNull(g); + } + + @Override + public double applyAsDouble(T value) { + return g.applyAsDouble(f.apply(value)); + } + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/FloatFunctions.java b/extensions/protobuf/src/main/java/io/deephaven/functions/FloatFunctions.java new file mode 100644 index 00000000000..f3f6f9303a6 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/FloatFunctions.java @@ -0,0 +1,42 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import java.util.Objects; +import java.util.function.Function; + +class FloatFunctions { + static ToFloatFunction cast() { + // noinspection unchecked + return (ToFloatFunction) PrimitiveFloat.INSTANCE; + } + + static ToFloatFunction map(Function f, ToFloatFunction g) { + return new FloatMap<>(f, g); + } + + private enum PrimitiveFloat implements ToFloatFunction { + INSTANCE; + + @Override + public float applyAsFloat(Object value) { + return (float) value; + } + } + + private static class FloatMap implements ToFloatFunction { + private final Function f; + private final ToFloatFunction g; + + public FloatMap(Function f, ToFloatFunction g) { + this.f = Objects.requireNonNull(f); + this.g = Objects.requireNonNull(g); + } + + @Override + public float applyAsFloat(T value) { + return g.applyAsFloat(f.apply(value)); + } + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/IntFunctions.java b/extensions/protobuf/src/main/java/io/deephaven/functions/IntFunctions.java new file mode 100644 index 00000000000..e97673a937e --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/IntFunctions.java @@ -0,0 +1,46 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import java.util.Objects; +import java.util.function.Function; + +class IntFunctions { + static ToIntFunction cast() { + // noinspection unchecked + return (ToIntFunction) PrimitiveInt.INSTANCE; + } + + static ToIntFunction of(java.util.function.ToIntFunction f) { + return f instanceof ToIntFunction ? (ToIntFunction) f : f::applyAsInt; + } + + static ToIntFunction map(Function f, java.util.function.ToIntFunction g) { + return new IntMap<>(f, g); + } + + private enum PrimitiveInt implements ToIntFunction { + INSTANCE; + + @Override + public int applyAsInt(Object value) { + return (int) value; + } + } + + private static class IntMap implements ToIntFunction { + private final Function f; + private final java.util.function.ToIntFunction g; + + public IntMap(Function f, java.util.function.ToIntFunction g) { + this.f = Objects.requireNonNull(f); + this.g = Objects.requireNonNull(g); + } + + @Override + public int applyAsInt(T value) { + return g.applyAsInt(f.apply(value)); + } + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/LongFunctions.java b/extensions/protobuf/src/main/java/io/deephaven/functions/LongFunctions.java new file mode 100644 index 00000000000..20b135f8b88 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/LongFunctions.java @@ -0,0 +1,47 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import java.util.Objects; +import java.util.function.Function; + +class LongFunctions { + + static ToLongFunction cast() { + // noinspection unchecked + return (ToLongFunction) PrimitiveLong.INSTANCE; + } + + static ToLongFunction of(java.util.function.ToLongFunction f) { + return f instanceof ToLongFunction ? (ToLongFunction) f : f::applyAsLong; + } + + static ToLongFunction map(Function f, java.util.function.ToLongFunction g) { + return new LongMap<>(f, g); + } + + private enum PrimitiveLong implements ToLongFunction { + INSTANCE; + + @Override + public long applyAsLong(Object value) { + return (long) value; + } + } + + private static class LongMap implements ToLongFunction { + private final Function f; + private final java.util.function.ToLongFunction g; + + public LongMap(Function f, java.util.function.ToLongFunction g) { + this.f = Objects.requireNonNull(f); + this.g = Objects.requireNonNull(g); + } + + @Override + public long applyAsLong(T value) { + return g.applyAsLong(f.apply(value)); + } + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ObjectFunctions.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ObjectFunctions.java new file mode 100644 index 00000000000..bab369e5552 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ObjectFunctions.java @@ -0,0 +1,256 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.CustomType; +import io.deephaven.qst.type.GenericType; +import io.deephaven.qst.type.Type; + +import java.util.Objects; +import java.util.function.Function; +import java.util.function.Predicate; + +class ObjectFunctions { + + static ToObjectFunction identity() { + // noinspection unchecked + return (ToObjectFunction) Identity.INSTANCE; + } + + static ToObjectFunction cast(GenericType type) { + return new Casted<>(type); + } + + static ToObjectFunction of(Function f, GenericType returnType) { + return new FunctionImpl<>(f, returnType); + } + + static ToObjectFunction map(Function f, ToObjectFunction g) { + return new ObjectMap<>(f, g, g.returnType()); + } + + static ToObjectFunction map(Function f, Function g, GenericType returnType) { + return new ObjectMap<>(f, g, returnType); + } + + static ToPrimitiveFunction mapPrimitive(ToObjectFunction f, ToPrimitiveFunction g) { + return MapPrimitiveVisitor.of(f, g); + } + + static TypedFunction map(ToObjectFunction f, TypedFunction g) { + return MapVisitor.of(f, g); + } + + private enum Identity implements ToObjectFunction { + INSTANCE; + + private static final CustomType RETURN_TYPE = Type.ofCustom(Object.class); + + @Override + public GenericType returnType() { + return RETURN_TYPE; + } + + @Override + public Object apply(Object value) { + return value; + } + + @Override + public ToBooleanFunction mapToBoolean(Predicate g) { + return BooleanFunctions.of(g); + } + + @Override + public ToCharFunction mapToChar(ToCharFunction g) { + return g; + } + + @Override + public ToByteFunction mapToByte(ToByteFunction g) { + return g; + } + + @Override + public ToShortFunction mapToShort(ToShortFunction g) { + return g; + } + + @Override + public ToIntFunction mapToInt(java.util.function.ToIntFunction g) { + return IntFunctions.of(g); + } + + @Override + public ToLongFunction mapToLong(java.util.function.ToLongFunction g) { + return LongFunctions.of(g); + } + + @Override + public ToFloatFunction mapToFloat(ToFloatFunction g) { + return g; + } + + @Override + public ToDoubleFunction mapToDouble(java.util.function.ToDoubleFunction g) { + return DoubleFunctions.of(g); + } + + @Override + public ToObjectFunction mapToObj(ToObjectFunction g) { + return g; + } + + @Override + public ToObjectFunction mapToObj(Function g, GenericType returnType) { + return ToObjectFunction.of(g, returnType); + } + + @Override + public ToPrimitiveFunction mapToPrimitive(ToPrimitiveFunction g) { + return g; + } + + @Override + public TypedFunction map(TypedFunction g) { + return g; + } + } + + private static class Casted implements ToObjectFunction { + private final GenericType returnType; + + public Casted(GenericType returnType) { + this.returnType = Objects.requireNonNull(returnType); + } + + @Override + public GenericType returnType() { + return returnType; + } + + @Override + public R apply(T value) { + return returnType.clazz().cast(value); + } + } + + private static final class FunctionImpl implements ToObjectFunction { + private final Function f; + private final GenericType returnType; + + FunctionImpl(Function f, GenericType returnType) { + this.f = Objects.requireNonNull(f); + this.returnType = Objects.requireNonNull(returnType); + } + + @Override + public GenericType returnType() { + return returnType; + } + + @Override + public R apply(T value) { + return f.apply(value); + } + } + + private static class ObjectMap implements ToObjectFunction { + private final Function f; + private final Function g; + private final GenericType returnType; + + public ObjectMap(Function f, Function g, GenericType returnType) { + this.f = Objects.requireNonNull(f); + this.g = Objects.requireNonNull(g); + this.returnType = Objects.requireNonNull(returnType); + } + + @Override + public GenericType returnType() { + return returnType; + } + + @Override + public Z apply(T value) { + return g.apply(f.apply(value)); + } + } + + private static class MapPrimitiveVisitor implements ToPrimitiveFunction.Visitor> { + + public static ToPrimitiveFunction of(ToObjectFunction f, ToPrimitiveFunction g) { + return g.walk(new MapPrimitiveVisitor<>(f)); + } + + private final ToObjectFunction f; + + private MapPrimitiveVisitor(ToObjectFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public ToBooleanFunction visit(ToBooleanFunction g) { + return f.mapToBoolean(g); + } + + @Override + public ToCharFunction visit(ToCharFunction g) { + return f.mapToChar(g); + } + + @Override + public ToByteFunction visit(ToByteFunction g) { + return f.mapToByte(g); + } + + @Override + public ToShortFunction visit(ToShortFunction g) { + return f.mapToShort(g); + } + + @Override + public ToIntFunction visit(ToIntFunction g) { + return f.mapToInt(g); + } + + @Override + public ToLongFunction visit(ToLongFunction g) { + return f.mapToLong(g); + } + + @Override + public ToFloatFunction visit(ToFloatFunction g) { + return f.mapToFloat(g); + } + + @Override + public ToDoubleFunction visit(ToDoubleFunction g) { + return f.mapToDouble(g); + } + } + + private static class MapVisitor implements TypedFunction.Visitor> { + + public static TypedFunction of(ToObjectFunction f, TypedFunction g) { + return g.walk(new MapVisitor<>(f)); + } + + private final ToObjectFunction f; + + private MapVisitor(ToObjectFunction f) { + this.f = Objects.requireNonNull(f); + } + + @Override + public ToPrimitiveFunction visit(ToPrimitiveFunction g) { + return f.mapToPrimitive(g); + } + + @Override + public ToObjectFunction visit(ToObjectFunction g) { + return f.mapToObj(g); + } + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ShortFunctions.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ShortFunctions.java new file mode 100644 index 00000000000..32f37824cc1 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ShortFunctions.java @@ -0,0 +1,43 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import java.util.Objects; +import java.util.function.Function; + +class ShortFunctions { + + static ToShortFunction cast() { + // noinspection unchecked + return (ToShortFunction) PrimitiveShort.INSTANCE; + } + + static ToShortFunction map(Function f, ToShortFunction g) { + return new ShortMap<>(f, g); + } + + private enum PrimitiveShort implements ToShortFunction { + INSTANCE; + + @Override + public short applyAsShort(Object value) { + return (short) value; + } + } + + private static class ShortMap implements ToShortFunction { + private final Function f; + private final ToShortFunction g; + + public ShortMap(Function f, ToShortFunction g) { + this.f = Objects.requireNonNull(f); + this.g = Objects.requireNonNull(g); + } + + @Override + public short applyAsShort(T value) { + return g.applyAsShort(f.apply(value)); + } + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ToBooleanFunction.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ToBooleanFunction.java new file mode 100644 index 00000000000..bf0f1353b6b --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ToBooleanFunction.java @@ -0,0 +1,114 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.BooleanType; +import io.deephaven.qst.type.Type; + +import java.util.Collection; +import java.util.function.Function; +import java.util.function.Predicate; + +/** + * A {@code boolean} function. + * + * @param the input type + */ +@FunctionalInterface +public interface ToBooleanFunction extends ToPrimitiveFunction, Predicate { + + /** + * Assumes the object value is directly castable to a boolean. Equivalent to {@code x -> (boolean)x}. + * + * @return the boolean function + * @param the value type + */ + static ToBooleanFunction cast() { + return BooleanFunctions.cast(); + } + + /** + * A function that always returns {@code true}. + * + * @return the true function + * @param the input type + */ + static ToBooleanFunction ofTrue() { + return BooleanFunctions.ofTrue(); + } + + /** + * A function that always returns {@code false}. + * + * @return the false function + * @param the input type + */ + static ToBooleanFunction ofFalse() { + return BooleanFunctions.ofFalse(); + } + + /** + * Creates the function composition {@code g ∘ f}. + * + *

+ * Equivalent to {@code x -> g.test(f.apply(x))}. + * + * @param f the inner function + * @param g the outer function + * @return the boolean function + * @param the input type + * @param the intermediate type + */ + static ToBooleanFunction map(Function f, Predicate g) { + return BooleanFunctions.map(f, g); + } + + /** + * Creates a function that returns {@code true} if any function in {@code functions} returns {@code true}. If + * {@code functions} is empty, returns {@link #ofFalse()}. + * + * @param functions the functions + * @return the or-function + * @param the input type + */ + static ToBooleanFunction or(Collection> functions) { + return BooleanFunctions.or(functions); + } + + /** + * Creates a function that returns {@code true} if all functions in {@code functions} returns {@code true}. If + * {@code functions} is empty, returns {@link #ofTrue()}. + * + * @param functions the functions + * @return the and-function + * @param the input type + */ + static ToBooleanFunction and(Collection> functions) { + return BooleanFunctions.and(functions); + } + + /** + * Creates a function that is the opposite of {@code f}. Equivalent to {@code x -> !x}. + * + * @param f the function + * @return the not-function + * @param the input type + */ + static ToBooleanFunction not(ToBooleanFunction f) { + return BooleanFunctions.not(f); + } + + @Override + boolean test(T value); + + @Override + default BooleanType returnType() { + return Type.booleanType(); + } + + @Override + default R walk(Visitor visitor) { + return visitor.visit(this); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ToByteFunction.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ToByteFunction.java new file mode 100644 index 00000000000..93a7a5439a4 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ToByteFunction.java @@ -0,0 +1,62 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.ByteType; +import io.deephaven.qst.type.Type; + +import java.util.function.Function; + +/** + * A {@code byte} function. + * + * @param the input type + */ +@FunctionalInterface +public interface ToByteFunction extends ToPrimitiveFunction { + + /** + * Assumes the object value is directly castable to a byte. Equivalent to {@code x -> (byte)x}. + * + * @return the byte function + * @param the value type + */ + static ToByteFunction cast() { + return ByteFunctions.cast(); + } + + /** + * Creates the function composition {@code g ∘ f}. + * + *

+ * Equivalent to {@code x -> g.applyAsByte(f.apply(x))}. + * + * @param f the inner function + * @param g the outer function + * @return the byte function + * @param the input type + * @param the intermediate type + */ + static ToByteFunction map(Function f, ToByteFunction g) { + return ByteFunctions.map(f, g); + } + + /** + * Applies this function to the given argument. + * + * @param value the function argument + * @return the function result + */ + byte applyAsByte(T value); + + @Override + default ByteType returnType() { + return Type.byteType(); + } + + @Override + default R walk(Visitor visitor) { + return visitor.visit(this); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ToCharFunction.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ToCharFunction.java new file mode 100644 index 00000000000..33cf7931176 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ToCharFunction.java @@ -0,0 +1,61 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.CharType; +import io.deephaven.qst.type.Type; + +import java.util.function.Function; + +/** + * A {@code char} function. + * + * @param the input type + */ +@FunctionalInterface +public interface ToCharFunction extends ToPrimitiveFunction { + /** + * Assumes the object value is directly castable to a char. Equivalent to {@code x -> (char)x}. + * + * @return the char function + * @param the value type + */ + static ToCharFunction cast() { + return CharFunctions.cast(); + } + + /** + * Creates the function composition {@code g ∘ f}. + * + *

+ * Equivalent to {@code x -> g.applyAsChar(f.apply(x))}. + * + * @param f the inner function + * @param g the outer function + * @return the char function + * @param the input type + * @param the intermediate type + */ + static ToCharFunction map(Function f, ToCharFunction g) { + return CharFunctions.map(f, g); + } + + /** + * Applies this function to the given argument. + * + * @param value the function argument + * @return the function result + */ + char applyAsChar(T value); + + @Override + default CharType returnType() { + return Type.charType(); + } + + @Override + default R walk(Visitor visitor) { + return visitor.visit(this); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ToDoubleFunction.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ToDoubleFunction.java new file mode 100644 index 00000000000..4d7a6c0d725 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ToDoubleFunction.java @@ -0,0 +1,57 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.DoubleType; +import io.deephaven.qst.type.Type; + +import java.util.function.Function; + +/** + * A {@code double} function. + * + * @param the input type + */ +@FunctionalInterface +public interface ToDoubleFunction extends ToPrimitiveFunction, java.util.function.ToDoubleFunction { + + /** + * Assumes the object value is directly castable to a double. Equivalent to {@code x -> (double)x}. + * + * @return the double function + * @param the value type + */ + static ToDoubleFunction cast() { + return DoubleFunctions.cast(); + } + + /** + * Creates the function composition {@code g ∘ f}. + * + *

+ * Equivalent to {@code x -> g.applyAsDouble(f.apply(x))}. + * + * @param f the inner function + * @param g the outer function + * @return the double function + * @param the input type + * @param the intermediate type + */ + static ToDoubleFunction map(Function f, java.util.function.ToDoubleFunction g) { + return DoubleFunctions.map(f, g); + } + + @Override + double applyAsDouble(T value); + + @Override + default DoubleType returnType() { + return Type.doubleType(); + } + + @Override + default R walk(Visitor visitor) { + return visitor.visit(this); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ToFloatFunction.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ToFloatFunction.java new file mode 100644 index 00000000000..fc0d34e8ac3 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ToFloatFunction.java @@ -0,0 +1,61 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.FloatType; +import io.deephaven.qst.type.Type; + +import java.util.function.Function; + +/** + * A {@code float} function. + * + * @param the input type + */ +@FunctionalInterface +public interface ToFloatFunction extends ToPrimitiveFunction { + /** + * Assumes the object value is directly castable to a float. Equivalent to {@code x -> (float)x}. + * + * @return the float function + * @param the value type + */ + static ToFloatFunction cast() { + return FloatFunctions.cast(); + } + + /** + * Creates the function composition {@code g ∘ f}. + * + *

+ * Equivalent to {@code x -> g.applyAsFloat(f.apply(x))}. + * + * @param f the inner function + * @param g the outer function + * @return the float function + * @param the input type + * @param the intermediate type + */ + static ToFloatFunction map(Function f, ToFloatFunction g) { + return FloatFunctions.map(f, g); + } + + /** + * Applies this function to the given argument. + * + * @param value the function argument + * @return the function result + */ + float applyAsFloat(T value); + + @Override + default FloatType returnType() { + return Type.floatType(); + } + + @Override + default R walk(Visitor visitor) { + return visitor.visit(this); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ToIntFunction.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ToIntFunction.java new file mode 100644 index 00000000000..26da4dfad5a --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ToIntFunction.java @@ -0,0 +1,57 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.IntType; +import io.deephaven.qst.type.Type; + +import java.util.function.Function; + +/** + * An {@code int} function. + * + * @param the input type + */ +@FunctionalInterface +public interface ToIntFunction extends ToPrimitiveFunction, java.util.function.ToIntFunction { + + /** + * Assumes the object value is directly castable to an int. Equivalent to {@code x -> (int)x}. + * + * @return the int function + * @param the value type + */ + static ToIntFunction cast() { + return IntFunctions.cast(); + } + + /** + * Creates the function composition {@code g ∘ f}. + * + *

+ * Equivalent to {@code x -> g.applyAsInt(f.apply(x))}. + * + * @param f the inner function + * @param g the outer function + * @return the int function + * @param the input type + * @param the intermediate type + */ + static ToIntFunction map(Function f, java.util.function.ToIntFunction g) { + return IntFunctions.map(f, g); + } + + @Override + int applyAsInt(T value); + + @Override + default IntType returnType() { + return Type.intType(); + } + + @Override + default R walk(Visitor visitor) { + return visitor.visit(this); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ToLongFunction.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ToLongFunction.java new file mode 100644 index 00000000000..f21c629a4e5 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ToLongFunction.java @@ -0,0 +1,56 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.LongType; +import io.deephaven.qst.type.Type; + +import java.util.function.Function; + +/** + * A {@code long} function. + * + * @param the input type + */ +@FunctionalInterface +public interface ToLongFunction extends ToPrimitiveFunction, java.util.function.ToLongFunction { + /** + * Assumes the object value is directly castable to a long. Equivalent to {@code x -> (long)x}. + * + * @return the long function + * @param the value type + */ + static ToLongFunction cast() { + return LongFunctions.cast(); + } + + /** + * Creates the function composition {@code g ∘ f}. + * + *

+ * Equivalent to {@code x -> g.applyAsLong(f.apply(x))}. + * + * @param f the inner function + * @param g the outer function + * @return the long function + * @param the input type + * @param the intermediate type + */ + static ToLongFunction map(Function f, java.util.function.ToLongFunction g) { + return LongFunctions.map(f, g); + } + + @Override + long applyAsLong(T value); + + @Override + default LongType returnType() { + return Type.longType(); + } + + @Override + default R walk(Visitor visitor) { + return visitor.visit(this); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ToObjectFunction.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ToObjectFunction.java new file mode 100644 index 00000000000..9e476f903bc --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ToObjectFunction.java @@ -0,0 +1,282 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.GenericType; + +import java.util.function.Function; +import java.util.function.Predicate; + +/** + * An object function. + * + *

+ * Note: this is not a {@link FunctionalInterface}, as {@link #returnType()} must be provided. + * + * @param the input type + * @param the return type + */ +public interface ToObjectFunction extends TypedFunction, Function { + + /** + * Creates an object function from {@code f} and {@code returnType}. + * + * @param f the function + * @param returnType the return type + * @return the object function + * @param the input type + * @param the return type + */ + static ToObjectFunction of(Function f, GenericType returnType) { + return ObjectFunctions.of(f, returnType); + } + + /** + * Creates an object function that casts each input to {@link Object}. Equivalent to {@code x -> (Object)x}. + * + * @return the object function + * @param the input type + */ + static ToObjectFunction identity() { + return ObjectFunctions.identity(); + } + + /** + * Creates an object function that casts each input to {@code returnType}. Equivalent to {@code x -> (R)x}. + * + * @param returnType the return type + * @return the object function + * @param the input type + * @param the return type + */ + static ToObjectFunction identity(GenericType returnType) { + return ObjectFunctions.identity().cast(returnType); + } + + /** + * Creates the function composition {@code g ∘ f}. + * + *

+ * Equivalent to {@code x -> g.apply(f.apply(x))}. + * + * @param f the inner function + * @param g the outer function + * @return the object function + * @param the input type + * @param the intermediate type + */ + static ToObjectFunction map(Function f, ToObjectFunction g) { + return ObjectFunctions.map(f, g); + } + + /** + * Creates the function composition {@code g ∘ f}. + * + *

+ * Equivalent to {@code x -> g.apply(f.apply(x))}. + * + * @param f the inner function + * @param g the outer function + * @param returnType the return type + * @return the object function + * @param the input type + * @param the intermediate type + */ + static ToObjectFunction map(Function f, Function g, GenericType returnType) { + return ObjectFunctions.map(f, g, returnType); + } + + GenericType returnType(); + + R apply(T value); + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Equivalent to {@code x -> g.test(this.apply(x))}. + * + * @param g the outer function + * @return the boolean function + */ + default ToBooleanFunction mapToBoolean(Predicate g) { + return ToBooleanFunction.map(this, g); + } + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Equivalent to {@code x -> g.applyAsChar(this.apply(x))}. + * + * @param g the outer function + * @return the char function + */ + default ToCharFunction mapToChar(ToCharFunction g) { + return ToCharFunction.map(this, g); + } + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Equivalent to {@code x -> g.applyAsByte(this.apply(x))}. + * + * @param g the outer function + * @return the byte function + */ + default ToByteFunction mapToByte(ToByteFunction g) { + return ToByteFunction.map(this, g); + } + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Equivalent to {@code x -> g.applyAsShort(this.apply(x))}. + * + * @param g the outer function + * @return the short function + */ + default ToShortFunction mapToShort(ToShortFunction g) { + return ToShortFunction.map(this, g); + } + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Equivalent to {@code x -> g.applyAsInt(this.apply(x))}. + * + * @param g the outer function + * @return the int function + */ + default ToIntFunction mapToInt(java.util.function.ToIntFunction g) { + return ToIntFunction.map(this, g); + } + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Equivalent to {@code x -> g.applyAsLong(this.apply(x))}. + * + * @param g the outer function + * @return the long function + */ + default ToLongFunction mapToLong(java.util.function.ToLongFunction g) { + return ToLongFunction.map(this, g); + } + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Equivalent to {@code x -> g.applyAsFloat(this.apply(x))}. + * + * @param g the outer function + * @return the float function + */ + default ToFloatFunction mapToFloat(ToFloatFunction g) { + return ToFloatFunction.map(this, g); + } + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Equivalent to {@code x -> g.applyAsDouble(this.apply(x))}. + * + * @param g the outer function + * @return the double function + */ + default ToDoubleFunction mapToDouble(java.util.function.ToDoubleFunction g) { + return ToDoubleFunction.map(this, g); + } + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Equivalent to {@code x -> g.apply(this.apply(x))}. + * + * @param g the outer function + * @return the object function + */ + default ToObjectFunction mapToObj(ToObjectFunction g) { + return map(this, g); + } + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Equivalent to {@code x -> g.apply(this.apply(x))}. + * + * @param g the outer function + * @return the object function + */ + default ToObjectFunction mapToObj(Function g, GenericType returnType) { + return map(this, g, returnType); + } + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Prefer to call one of the more strongly-typed map methods if you have a more specific function type. + * + * @param g the outer function + * @return the function + * @see #mapToBoolean(Predicate) + * @see #mapToChar(ToCharFunction) + * @see #mapToByte(ToByteFunction) + * @see #mapToShort(ToShortFunction) + * @see #mapToInt(java.util.function.ToIntFunction) + * @see #mapToLong(java.util.function.ToLongFunction) + * @see #mapToFloat(ToFloatFunction) + * @see #mapToDouble(java.util.function.ToDoubleFunction) + */ + default ToPrimitiveFunction mapToPrimitive(ToPrimitiveFunction g) { + return ObjectFunctions.mapPrimitive(this, g); + } + + /** + * Creates the function composition {@code g ∘ this}. + * + *

+ * Prefer to call one of the more strongly-typed map methods if you have a more specific function type. + * + * @param g the outer function + * @return the function + * @see #mapToPrimitive(ToPrimitiveFunction) + * @see #mapToObj(ToObjectFunction) + */ + default TypedFunction map(TypedFunction g) { + return ObjectFunctions.map(this, g); + } + + /** + * Creates a function by casting to {@code returnType}. + * + *

+ * In the case where {@code returnType().equals(returnType)}, the result is {@code (ObjectFunction) this}. + * Otherwise, the result is equivalent to {@code x -> (R2) this.apply(x)}. + * + * @param returnType the return type + * @return the object function + * @param the return type + */ + default ToObjectFunction cast(GenericType returnType) { + // noinspection unchecked + return returnType().equals(returnType) + ? (ToObjectFunction) this + : mapToObj(ObjectFunctions.cast(returnType)); + } + + @Override + default Z walk(Visitor visitor) { + return visitor.visit(this); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ToPrimitiveFunction.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ToPrimitiveFunction.java new file mode 100644 index 00000000000..6d0c6b6b8c9 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ToPrimitiveFunction.java @@ -0,0 +1,50 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.PrimitiveType; + +/** + * A function interface that allows for differentiation of primitive return types. + * + * @param the input type + * @see ToBooleanFunction + * @see ToCharFunction + * @see ToByteFunction + * @see ToShortFunction + * @see ToIntFunction + * @see ToLongFunction + * @see ToFloatFunction + * @see ToDoubleFunction + */ +public interface ToPrimitiveFunction extends TypedFunction { + + @Override + PrimitiveType returnType(); + + @Override + default R walk(TypedFunction.Visitor visitor) { + return visitor.visit(this); + } + + R walk(Visitor visitor); + + interface Visitor { + R visit(ToBooleanFunction f); + + R visit(ToCharFunction f); + + R visit(ToByteFunction f); + + R visit(ToShortFunction f); + + R visit(ToIntFunction f); + + R visit(ToLongFunction f); + + R visit(ToFloatFunction f); + + R visit(ToDoubleFunction f); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/ToShortFunction.java b/extensions/protobuf/src/main/java/io/deephaven/functions/ToShortFunction.java new file mode 100644 index 00000000000..312ed3cef45 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/ToShortFunction.java @@ -0,0 +1,62 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.ShortType; +import io.deephaven.qst.type.Type; + +import java.util.function.Function; + +/** + * A {@code short} function. + * + * @param the input type + */ +@FunctionalInterface +public interface ToShortFunction extends ToPrimitiveFunction { + + /** + * Assumes the object value is directly castable to a short. Equivalent to {@code x -> (short)x}. + * + * @return the short function + * @param the value type + */ + static ToShortFunction cast() { + return ShortFunctions.cast(); + } + + /** + * Creates the function composition {@code g ∘ f}. + * + *

+ * Equivalent to {@code x -> g.applyAsShort(f.apply(x))}. + * + * @param f the inner function + * @param g the outer function + * @return the short function + * @param the input type + * @param the intermediate type + */ + static ToShortFunction map(Function f, ToShortFunction g) { + return ShortFunctions.map(f, g); + } + + /** + * Applies this function to the given argument. + * + * @param value the function argument + * @return the function result + */ + short applyAsShort(T value); + + @Override + default ShortType returnType() { + return Type.shortType(); + } + + @Override + default R walk(Visitor visitor) { + return visitor.visit(this); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/functions/TypedFunction.java b/extensions/protobuf/src/main/java/io/deephaven/functions/TypedFunction.java new file mode 100644 index 00000000000..71894b1266e --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/functions/TypedFunction.java @@ -0,0 +1,33 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.functions; + +import io.deephaven.qst.type.Type; + +import java.util.function.Function; + +/** + * A common function interface that allows for differentiation based on the return type. + * + * @param the input type + * @see ToPrimitiveFunction + * @see ToObjectFunction + */ +public interface TypedFunction { + + /** + * This function's return type. + * + * @return the type + */ + Type returnType(); + + R walk(Visitor visitor); + + interface Visitor { + R visit(ToPrimitiveFunction f); + + R visit(ToObjectFunction f); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/Box.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/Box.java new file mode 100644 index 00000000000..618374f1680 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/Box.java @@ -0,0 +1,83 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToCharFunction; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToPrimitiveFunction; +import io.deephaven.functions.ToShortFunction; +import io.deephaven.functions.TypedFunction; +import io.deephaven.functions.TypedFunction.Visitor; + +class Box implements Visitor, ToPrimitiveFunction.Visitor { + public static Object apply(TypedFunction f, T value) { + return f.walk(new Box<>(value)); + } + + public static Object apply(ToPrimitiveFunction f, T value) { + return f.walk((ToPrimitiveFunction.Visitor) new Box<>(value)); + } + + private final T value; + + private Box(T value) { + this.value = value; + } + + @Override + public Object visit(ToObjectFunction f) { + return f.apply(value); + } + + @Override + public Object visit(ToPrimitiveFunction f) { + return f.walk((ToPrimitiveFunction.Visitor) this); + } + + @Override + public Boolean visit(ToBooleanFunction f) { + return f.test(value); + } + + @Override + public Character visit(ToCharFunction f) { + return f.applyAsChar(value); + } + + @Override + public Byte visit(ToByteFunction f) { + return f.applyAsByte(value); + } + + @Override + public Short visit(ToShortFunction f) { + return f.applyAsShort(value); + } + + @Override + public Integer visit(ToIntFunction f) { + return f.applyAsInt(value); + } + + @Override + public Long visit(ToLongFunction f) { + return f.applyAsLong(value); + } + + @Override + public Float visit(ToFloatFunction f) { + return f.applyAsFloat(value); + } + + @Override + public Double visit(ToDoubleFunction f) { + return f.applyAsDouble(value); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/Builtin.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/Builtin.java new file mode 100644 index 00000000000..4837d967c93 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/Builtin.java @@ -0,0 +1,607 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import com.google.protobuf.Any; +import com.google.protobuf.BoolValue; +import com.google.protobuf.ByteString; +import com.google.protobuf.BytesValue; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Descriptors.FieldDescriptor; +import com.google.protobuf.Descriptors.FieldDescriptor.JavaType; +import com.google.protobuf.DoubleValue; +import com.google.protobuf.FieldMask; +import com.google.protobuf.FloatValue; +import com.google.protobuf.Int32Value; +import com.google.protobuf.Int64Value; +import com.google.protobuf.Message; +import com.google.protobuf.StringValue; +import com.google.protobuf.Timestamp; +import com.google.protobuf.UInt32Value; +import com.google.protobuf.UInt64Value; +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.protobuf.FieldOptions.BytesBehavior; +import io.deephaven.qst.type.CustomType; +import io.deephaven.qst.type.GenericType; +import io.deephaven.qst.type.Type; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.time.Duration; +import java.time.Instant; +import java.util.List; +import java.util.Objects; + +class Builtin { + + // Due to https://github.com/confluentinc/schema-registry/issues/2708, the parsers need to be built in such a way + // that they work with DynamicMessages. + + static List parsers() { + // Update javadoc in io.deephaven.protobuf.SingleValuedMessageParser.builtin when editing + return List.of( + TimestampParser.of(), + DurationParser.of(), + BoolValueParser.of(), + Int32ValueParser.of(), + UInt32ValueParser.of(), + Int64ValueParser.of(), + UInt64ValueParser.of(), + FloatValueParser.of(), + DoubleValueParser.of(), + StringValueParser.of(), + BytesValueParser.of(), + customParser(Any.class), + customParser(FieldMask.class)); + } + + static MessageParserSingle customParser(Class clazz) { + try { + final Method method = clazz.getDeclaredMethod("getDescriptor"); + final Descriptor descriptor = (Descriptor) method.invoke(null); + return new GenericSVMP<>(CustomType.of(clazz), descriptor); + } catch (InvocationTargetException | NoSuchMethodException | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + private enum TimestampParser implements MessageParserSingle { + INSTANCE; + + private static final ToObjectFunction CANONICAL_FUNCTION = ToObjectFunction + .identity(Type.ofCustom(Timestamp.class)) + .mapToObj(TimestampParser::parseCanonical, Type.instantType()); + + public static MessageParserSingle of() { + return INSTANCE; + } + + private static Instant parseCanonical(Timestamp timestamp) { + return Instant.ofEpochSecond(timestamp.getSeconds(), timestamp.getNanos()); + } + + @Override + public Descriptor canonicalDescriptor() { + return Timestamp.getDescriptor(); + } + + @Override + public ToObjectFunction messageParser(Descriptor descriptor, + ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + if (canonicalDescriptor() == descriptor) { + return CANONICAL_FUNCTION; + } + checkCompatible(canonicalDescriptor(), descriptor); + return TimestampFunction.of(descriptor); + } + } + + private enum DurationParser implements MessageParserSingle { + INSTANCE; + + private static final ToObjectFunction CANONICAL_FUNCTION = + ToObjectFunction.identity( + Type.ofCustom(com.google.protobuf.Duration.class)) + .mapToObj(DurationParser::parseCanonical, Type.ofCustom(Duration.class)); + + public static MessageParserSingle of() { + return INSTANCE; + } + + private static Duration parseCanonical(com.google.protobuf.Duration duration) { + return Duration.ofSeconds(duration.getSeconds(), duration.getNanos()); + } + + @Override + public Descriptor canonicalDescriptor() { + return com.google.protobuf.Duration.getDescriptor(); + } + + @Override + public ToObjectFunction messageParser(Descriptor descriptor, + ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + if (canonicalDescriptor() == descriptor) { + return CANONICAL_FUNCTION; + } + checkCompatible(canonicalDescriptor(), descriptor); + return DurationFunction.of(descriptor); + } + } + + private enum BoolValueParser implements MessageParserSingle { + INSTANCE; + + private static final ToBooleanFunction CANONICAL_FUNCTION = ToObjectFunction + .identity(Type.ofCustom(BoolValue.class)) + .mapToBoolean(BoolValue::getValue); + + public static MessageParserSingle of() { + return INSTANCE; + } + + @Override + public Descriptor canonicalDescriptor() { + return BoolValue.getDescriptor(); + } + + @Override + public ToBooleanFunction messageParser(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + if (canonicalDescriptor() == descriptor) { + return CANONICAL_FUNCTION; + } + checkCompatible(canonicalDescriptor(), descriptor); + return new BoolFieldFunction(descriptor.findFieldByNumber(BoolValue.VALUE_FIELD_NUMBER)); + } + } + + private enum Int32ValueParser implements MessageParserSingle { + INSTANCE; + + private static final ToIntFunction CANONICAL_FUNCTION = ToObjectFunction + .identity(Type.ofCustom(Int32Value.class)) + .mapToInt(Int32Value::getValue); + + public static MessageParserSingle of() { + return INSTANCE; + } + + @Override + public Descriptor canonicalDescriptor() { + return Int32Value.getDescriptor(); + } + + @Override + public ToIntFunction messageParser(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + if (canonicalDescriptor() == descriptor) { + return CANONICAL_FUNCTION; + } + checkCompatible(canonicalDescriptor(), descriptor); + return new IntFieldFunction(descriptor.findFieldByNumber(Int32Value.VALUE_FIELD_NUMBER)); + } + } + + private enum UInt32ValueParser implements MessageParserSingle { + INSTANCE; + + private static final ToIntFunction CANONICAL_FUNCTION = ToObjectFunction + .identity(Type.ofCustom(UInt32Value.class)) + .mapToInt(UInt32Value::getValue); + + public static MessageParserSingle of() { + return INSTANCE; + } + + @Override + public Descriptor canonicalDescriptor() { + return UInt32Value.getDescriptor(); + } + + @Override + public ToIntFunction messageParser(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + if (canonicalDescriptor() == descriptor) { + return CANONICAL_FUNCTION; + } + checkCompatible(canonicalDescriptor(), descriptor); + return new IntFieldFunction(descriptor.findFieldByNumber(UInt32Value.VALUE_FIELD_NUMBER)); + } + } + + private enum Int64ValueParser implements MessageParserSingle { + INSTANCE; + + private static final ToLongFunction CANONICAL_FUNCTION = ToObjectFunction + .identity(Type.ofCustom(Int64Value.class)) + .mapToLong(Int64Value::getValue); + + public static MessageParserSingle of() { + return INSTANCE; + } + + @Override + public Descriptor canonicalDescriptor() { + return Int64Value.getDescriptor(); + } + + @Override + public ToLongFunction messageParser(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + if (canonicalDescriptor() == descriptor) { + return CANONICAL_FUNCTION; + } + checkCompatible(canonicalDescriptor(), descriptor); + return new LongFieldFunction(descriptor.findFieldByNumber(Int64Value.VALUE_FIELD_NUMBER)); + } + } + + private enum UInt64ValueParser implements MessageParserSingle { + INSTANCE; + + private static final ToLongFunction CANONICAL_FUNCTION = ToObjectFunction + .identity(Type.ofCustom(UInt64Value.class)) + .mapToLong(UInt64Value::getValue); + + public static MessageParserSingle of() { + return INSTANCE; + } + + @Override + public Descriptor canonicalDescriptor() { + return UInt64Value.getDescriptor(); + } + + @Override + public ToLongFunction messageParser(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + if (canonicalDescriptor() == descriptor) { + return CANONICAL_FUNCTION; + } + checkCompatible(canonicalDescriptor(), descriptor); + return new LongFieldFunction(descriptor.findFieldByNumber(UInt64Value.VALUE_FIELD_NUMBER)); + } + } + + private enum FloatValueParser implements MessageParserSingle { + INSTANCE; + + private static final ToFloatFunction CANONICAL_FUNCTION = ToObjectFunction + .identity(Type.ofCustom(FloatValue.class)) + .mapToFloat(FloatValue::getValue); + + public static MessageParserSingle of() { + return INSTANCE; + } + + @Override + public Descriptor canonicalDescriptor() { + return FloatValue.getDescriptor(); + } + + @Override + public ToFloatFunction messageParser(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + if (canonicalDescriptor() == descriptor) { + return CANONICAL_FUNCTION; + } + checkCompatible(canonicalDescriptor(), descriptor); + return new FloatFieldFunction(descriptor.findFieldByNumber(FloatValue.VALUE_FIELD_NUMBER)); + } + } + + private enum DoubleValueParser implements MessageParserSingle { + INSTANCE; + + private static final ToDoubleFunction CANONICAL_FUNCTION = ToObjectFunction + .identity(Type.ofCustom(DoubleValue.class)) + .mapToDouble(DoubleValue::getValue); + + public static MessageParserSingle of() { + return INSTANCE; + } + + @Override + public Descriptor canonicalDescriptor() { + return DoubleValue.getDescriptor(); + } + + @Override + public ToDoubleFunction messageParser(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + if (canonicalDescriptor() == descriptor) { + return CANONICAL_FUNCTION; + } + checkCompatible(canonicalDescriptor(), descriptor); + return new DoubleFieldFunction(descriptor.findFieldByNumber(DoubleValue.VALUE_FIELD_NUMBER)); + } + } + + private enum StringValueParser implements MessageParserSingle { + INSTANCE; + + private static final ToObjectFunction CANONICAL_FUNCTION = ToObjectFunction + .identity(Type.ofCustom(StringValue.class)) + .mapToObj(StringValue::getValue, Type.stringType()); + + public static MessageParserSingle of() { + return INSTANCE; + } + + @Override + public Descriptor canonicalDescriptor() { + return StringValue.getDescriptor(); + } + + @Override + public ToObjectFunction messageParser(Descriptor descriptor, + ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + if (canonicalDescriptor() == descriptor) { + return CANONICAL_FUNCTION; + } + checkCompatible(canonicalDescriptor(), descriptor); + return new StringFieldFunction(descriptor.findFieldByNumber(StringValue.VALUE_FIELD_NUMBER)); + } + } + + private enum BytesValueParser implements MessageParserSingle { + INSTANCE; + + private static final ToObjectFunction CANONICAL_FUNCTION_BYTESTRING = ToObjectFunction + .identity(Type.ofCustom(BytesValue.class)) + .mapToObj(BytesValue::getValue, Type.ofCustom(ByteString.class)); + private static final ToObjectFunction BYTESTRING_TO_BYTES = + ToObjectFunction.of(ByteString::toByteArray, Type.byteType().arrayType()); + private static final ToObjectFunction CANONICAL_FUNCTION_BYTEARRAY = + CANONICAL_FUNCTION_BYTESTRING.mapToObj(BYTESTRING_TO_BYTES); + + public static MessageParserSingle of() { + return INSTANCE; + } + + @Override + public Descriptor canonicalDescriptor() { + return BytesValue.getDescriptor(); + } + + @Override + public ToObjectFunction messageParser(Descriptor descriptor, + ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + final boolean asByteArray = options.fieldOptions().apply(fieldPath).bytes() == BytesBehavior.asByteArray(); + if (canonicalDescriptor() == descriptor) { + return asByteArray + ? CANONICAL_FUNCTION_BYTEARRAY + : CANONICAL_FUNCTION_BYTESTRING; + } + checkCompatible(canonicalDescriptor(), descriptor); + final ByteStringFieldFunction bsf = + new ByteStringFieldFunction(descriptor.findFieldByNumber(BytesValue.VALUE_FIELD_NUMBER)); + return asByteArray + ? bsf.mapToObj(BYTESTRING_TO_BYTES) + : bsf; + } + } + + private static class GenericSVMP implements MessageParserSingle { + private final GenericType type; + private final Descriptor descriptor; + + public GenericSVMP(GenericType type, Descriptor descriptor) { + this.type = Objects.requireNonNull(type); + this.descriptor = Objects.requireNonNull(descriptor); + } + + @Override + public Descriptor canonicalDescriptor() { + return descriptor; + } + + @Override + public ToObjectFunction messageParser(Descriptor descriptor, + ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + checkCompatible(canonicalDescriptor(), descriptor); + return ToObjectFunction.identity(type); + } + } + + private static void checkCompatible(Descriptor expected, Descriptor actual) { + final String expectedName = expected.getFullName(); + final String actualName = actual.getFullName(); + if (!expectedName.equals(actualName)) { + throw new IllegalArgumentException( + String.format("Incompatible descriptors, expected=%s, actual=%s", expectedName, actualName)); + } + } + + private static void checkCompatible(JavaType expected, FieldDescriptor actual) { + if (!expected.equals(actual.getJavaType())) { + throw new IllegalArgumentException(String.format("Incompatible field type, expected=%s, actual=%s (%s)", + expected, actual.getJavaType(), actual.getFullName())); + } + } + + private static final class TimestampFunction implements ToObjectFunction { + + public static ToObjectFunction of(Descriptor descriptor) { + final FieldDescriptor secondsField = descriptor.findFieldByNumber(Timestamp.SECONDS_FIELD_NUMBER); + final FieldDescriptor nanosField = descriptor.findFieldByNumber(Timestamp.NANOS_FIELD_NUMBER); + return new TimestampFunction(secondsField, nanosField); + } + + private final FieldDescriptor seconds; + private final FieldDescriptor nanos; + + private TimestampFunction(FieldDescriptor seconds, FieldDescriptor nanos) { + checkCompatible(JavaType.LONG, seconds); + checkCompatible(JavaType.INT, nanos); + this.seconds = Objects.requireNonNull(seconds); + this.nanos = Objects.requireNonNull(nanos); + } + + @Override + public GenericType returnType() { + return Type.instantType(); + } + + @Override + public Instant apply(Message value) { + return Instant.ofEpochSecond((long) value.getField(seconds), (int) value.getField(nanos)); + } + } + + private static final class DurationFunction implements ToObjectFunction { + + private static final GenericType RETURN_TYPE = Type.ofCustom(Duration.class); + + public static ToObjectFunction of(Descriptor descriptor) { + final FieldDescriptor secondsField = + descriptor.findFieldByNumber(com.google.protobuf.Duration.SECONDS_FIELD_NUMBER); + final FieldDescriptor nanosField = + descriptor.findFieldByNumber(com.google.protobuf.Duration.NANOS_FIELD_NUMBER); + return new DurationFunction(secondsField, nanosField); + } + + private final FieldDescriptor seconds; + private final FieldDescriptor nanos; + + private DurationFunction(FieldDescriptor seconds, FieldDescriptor nanos) { + checkCompatible(JavaType.LONG, seconds); + checkCompatible(JavaType.INT, nanos); + this.seconds = Objects.requireNonNull(seconds); + this.nanos = Objects.requireNonNull(nanos); + } + + @Override + public GenericType returnType() { + return RETURN_TYPE; + } + + @Override + public Duration apply(Message value) { + return Duration.ofSeconds((long) value.getField(seconds), (int) value.getField(nanos)); + } + } + + private static final class BoolFieldFunction implements ToBooleanFunction { + private final FieldDescriptor valueField; + + public BoolFieldFunction(FieldDescriptor valueField) { + checkCompatible(JavaType.BOOLEAN, valueField); + this.valueField = Objects.requireNonNull(valueField); + } + + @Override + public boolean test(Message value) { + return (boolean) value.getField(valueField); + } + } + + private static final class IntFieldFunction implements ToIntFunction { + + private final FieldDescriptor valueField; + + public IntFieldFunction(FieldDescriptor valueField) { + checkCompatible(JavaType.INT, valueField); + this.valueField = Objects.requireNonNull(valueField); + } + + @Override + public int applyAsInt(Message value) { + return (int) value.getField(valueField); + } + } + + private static final class LongFieldFunction implements ToLongFunction { + + private final FieldDescriptor valueField; + + public LongFieldFunction(FieldDescriptor valueField) { + checkCompatible(JavaType.LONG, valueField); + this.valueField = Objects.requireNonNull(valueField); + } + + @Override + public long applyAsLong(Message value) { + return (long) value.getField(valueField); + } + } + + private static final class FloatFieldFunction implements ToFloatFunction { + private final FieldDescriptor valueField; + + public FloatFieldFunction(FieldDescriptor valueField) { + checkCompatible(JavaType.FLOAT, valueField); + this.valueField = Objects.requireNonNull(valueField); + } + + @Override + public float applyAsFloat(Message value) { + return (float) value.getField(valueField); + } + } + + private static final class DoubleFieldFunction implements ToDoubleFunction { + private final FieldDescriptor valueField; + + public DoubleFieldFunction(FieldDescriptor valueField) { + checkCompatible(JavaType.DOUBLE, valueField); + this.valueField = Objects.requireNonNull(valueField); + } + + @Override + public double applyAsDouble(Message value) { + return (double) value.getField(valueField); + } + } + + private static final class StringFieldFunction implements ToObjectFunction { + private final FieldDescriptor valueField; + + public StringFieldFunction(FieldDescriptor valueField) { + checkCompatible(JavaType.STRING, valueField); + this.valueField = Objects.requireNonNull(valueField); + } + + @Override + public GenericType returnType() { + return Type.stringType(); + } + + @Override + public String apply(Message value) { + return (String) value.getField(valueField); + } + } + + private static final class ByteStringFieldFunction implements ToObjectFunction { + private static final CustomType RETURN_TYPE = Type.ofCustom(ByteString.class); + + private final FieldDescriptor valueField; + + public ByteStringFieldFunction(FieldDescriptor valueField) { + checkCompatible(JavaType.BYTE_STRING, valueField); + this.valueField = Objects.requireNonNull(valueField); + } + + @Override + public GenericType returnType() { + return RETURN_TYPE; + } + + @Override + public ByteString apply(Message value) { + return (ByteString) value.getField(valueField); + } + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/BypassOnNull.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/BypassOnNull.java new file mode 100644 index 00000000000..1286c35dd3b --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/BypassOnNull.java @@ -0,0 +1,171 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToCharFunction; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToPrimitiveFunction; +import io.deephaven.functions.ToShortFunction; +import io.deephaven.functions.TypedFunction; +import io.deephaven.qst.type.BoxedBooleanType; +import io.deephaven.util.QueryConstants; + +/** + * Wraps existing functions to guard against null inputs; essentially, implementing + * {@code x -> x == null ? QueryConstants.NULL_X : f.theApplyFunction(x)} for primitive functions (except for boolean + * which goes to Boolean), and {@code x -> x == null ? null : f.apply(x)} for object functions. + */ +enum BypassOnNull implements + TypedFunction.Visitor>, + ToPrimitiveFunction.Visitor> { + INSTANCE; + + private static TypedFunction.Visitor> visitor() { + // noinspection unchecked,rawtypes + return (TypedFunction.Visitor>) (TypedFunction.Visitor) INSTANCE; + } + + private static ToPrimitiveFunction.Visitor> primitiveVisitor() { + // noinspection unchecked,rawtypes + return (ToPrimitiveFunction.Visitor>) (ToPrimitiveFunction.Visitor) INSTANCE; + } + + public static TypedFunction of(TypedFunction x) { + return x.walk(visitor()); + } + + public static TypedFunction of(ToPrimitiveFunction x) { + return x.walk(primitiveVisitor()); + } + + public static ToObjectFunction of(ToObjectFunction f) { + return ToObjectFunction.of(x -> apply(f, x), f.returnType()); + } + + public static ToObjectFunction of(ToBooleanFunction f) { + // Note: it's important that we don't adapt to byte at this level; we need to preserve the nominal boolean type + // as long as we can (until it needs to get adapted into a chunk, that's where the transform to byte can happen) + return ToObjectFunction.of(x -> applyAsBoolean(f, x), BoxedBooleanType.of()); + } + + public static ToCharFunction of(ToCharFunction f) { + return x -> applyAsChar(f, x); + } + + public static ToByteFunction of(ToByteFunction f) { + return x -> applyAsByte(f, x); + } + + public static ToShortFunction of(ToShortFunction f) { + return x -> applyAsShort(f, x); + } + + public static ToIntFunction of(ToIntFunction f) { + return x -> applyAsInt(f, x); + } + + public static ToLongFunction of(ToLongFunction f) { + return x -> applyAsLong(f, x); + } + + public static ToFloatFunction of(ToFloatFunction f) { + return x -> applyAsFloat(f, x); + } + + public static ToDoubleFunction of(ToDoubleFunction f) { + return x -> applyAsDouble(f, x); + } + + @Override + public ToObjectFunction visit(ToObjectFunction f) { + return of(f); + } + + @Override + public TypedFunction visit(ToPrimitiveFunction f) { + return f.walk((ToPrimitiveFunction.Visitor>) this); + } + + @Override + public ToObjectFunction visit(ToBooleanFunction f) { + return of(f); + } + + @Override + public ToCharFunction visit(ToCharFunction f) { + return of(f); + } + + @Override + public ToByteFunction visit(ToByteFunction f) { + return of(f); + } + + @Override + public ToShortFunction visit(ToShortFunction f) { + return of(f); + } + + @Override + public ToIntFunction visit(ToIntFunction f) { + return of(f); + } + + @Override + public ToLongFunction visit(ToLongFunction f) { + return of(f); + } + + @Override + public ToFloatFunction visit(ToFloatFunction f) { + return of(f); + } + + @Override + public ToDoubleFunction visit(ToDoubleFunction f) { + return of(f); + } + + private static R apply(ToObjectFunction f, T x) { + return x == null ? null : f.apply(x); + } + + private static Boolean applyAsBoolean(ToBooleanFunction f, T x) { + return x == null ? null : f.test(x); + } + + private static char applyAsChar(ToCharFunction f, T x) { + return x == null ? QueryConstants.NULL_CHAR : f.applyAsChar(x); + } + + private static byte applyAsByte(ToByteFunction f, T x) { + return x == null ? QueryConstants.NULL_BYTE : f.applyAsByte(x); + } + + private static short applyAsShort(ToShortFunction f, T x) { + return x == null ? QueryConstants.NULL_SHORT : f.applyAsShort(x); + } + + private static int applyAsInt(ToIntFunction f, T x) { + return x == null ? QueryConstants.NULL_INT : f.applyAsInt(x); + } + + private static long applyAsLong(ToLongFunction f, T x) { + return x == null ? QueryConstants.NULL_LONG : f.applyAsLong(x); + } + + private static float applyAsFloat(ToFloatFunction f, T x) { + return x == null ? QueryConstants.NULL_FLOAT : f.applyAsFloat(x); + } + + private static double applyAsDouble(ToDoubleFunction f, T x) { + return x == null ? QueryConstants.NULL_DOUBLE : f.applyAsDouble(x); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldNumberPath.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldNumberPath.java new file mode 100644 index 00000000000..8f2267f58ea --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldNumberPath.java @@ -0,0 +1,20 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import io.deephaven.annotations.SimpleStyle; +import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Parameter; + +@Immutable +@SimpleStyle +public abstract class FieldNumberPath { + + public static FieldNumberPath of(int... path) { + return ImmutableFieldNumberPath.of(path); + } + + @Parameter + public abstract int[] path(); +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldOptions.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldOptions.java new file mode 100644 index 00000000000..69a9c9befde --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldOptions.java @@ -0,0 +1,177 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value.Default; +import org.immutables.value.Value.Immutable; + +import java.util.function.Function; +import java.util.function.Predicate; + +@Immutable +@BuildableStyle +public abstract class FieldOptions { + + private static final FieldOptions DEFAULTS = builder().build(); + private static final FieldOptions EXCLUDE = builder().include(false).build(); + + public static Builder builder() { + return ImmutableFieldOptions.builder(); + } + + /** + * The defaults options. Equivalent to {@code builder().build()}. + * + * @return the default options + */ + public static FieldOptions defaults() { + return DEFAULTS; + } + + /** + * Creates the options to exclude a field. Equivalent to {@code builder().include(false).build()}. + * + * @return the options to exclude a field + */ + public static FieldOptions exclude() { + return EXCLUDE; + } + + /** + * Creates a field options function that maps the {@code include} predicate to {@link FieldOptions#include()}. + * Equivalent to {@code fp -> builder().include(include.test(fp)).build()}. + * + * @param include the include function + * @return the field path functions + */ + @SuppressWarnings("unused") + public static Function includeIf(Predicate include) { + return fp -> builder().include(include.test(fp)).build(); + } + + /** + * The behavior when handling a protobuf {@link com.google.protobuf.Descriptors.FieldDescriptor.Type#MESSAGE + * message} type. + */ + public interface WellKnownBehavior { + /** + * Parse the field as a well-known type if known, otherwise parse recursively. + */ + static WellKnownBehavior asWellKnown() { + return WellKnownImpl.AS_WELL_KNOWN; + } + + /** + * Parse the field recursively. + */ + static WellKnownBehavior asRecursive() { + return WellKnownImpl.AS_RECURSIVE; + } + } + + /** + * The behavior when handling a protobuf {@link com.google.protobuf.Descriptors.FieldDescriptor.Type#BYTES bytes} + * field. + */ + public interface BytesBehavior { + + /** + * Parsers the field as a {@code byte[]}. + */ + static BytesBehavior asByteArray() { + return BytesImpl.AS_BYTE_ARRAY; + } + + /** + * Parses the field as a {@link com.google.protobuf.ByteString ByteString}. + */ + static BytesBehavior asByteString() { + return BytesImpl.AS_BYTES_STRING; + } + } + + /** + * The behavior when handling a protobuf {@code map} field. + */ + public interface MapBehavior { + /** + * Parses the field as a {@link java.util.Map Map}. + */ + static MapBehavior asMap() { + return MapsImpl.AS_MAP; + } + + /** + * Parses the field as a {@code repeated MapFieldEntry}. + */ + static MapBehavior asRepeated() { + return MapsImpl.AS_REPEATED; + } + } + + /** + * If the field should be included for parsing. By default, is {@code true}. + * + * @return if the field should be included + */ + @Default + public boolean include() { + return true; + } + + /** + * The well-known message behavior. By default, is {@link WellKnownBehavior#asWellKnown()}. + * + * @return the well-known message behavior + */ + @Default + public WellKnownBehavior wellKnown() { + return WellKnownBehavior.asWellKnown(); + } + + /** + * The {@code bytes} type behavior. By default, is {@link BytesBehavior#asByteArray()}. + * + * @return the bytes field behavior + */ + @Default + public BytesBehavior bytes() { + return BytesBehavior.asByteArray(); + } + + /** + * The {@code map} type behavior. By default, is {@link MapBehavior#asMap()}. + * + * @return the map field behavior. + */ + @Default + public MapBehavior map() { + return MapBehavior.asMap(); + } + + public interface Builder { + Builder include(boolean include); + + Builder wellKnown(WellKnownBehavior wellKnown); + + Builder bytes(BytesBehavior bytes); + + Builder map(MapBehavior map); + + FieldOptions build(); + } + + enum WellKnownImpl implements WellKnownBehavior { + AS_WELL_KNOWN, AS_RECURSIVE; + } + + enum BytesImpl implements BytesBehavior { + AS_BYTE_ARRAY, AS_BYTES_STRING; + } + + enum MapsImpl implements MapBehavior { + AS_MAP, AS_REPEATED; + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldPath.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldPath.java new file mode 100644 index 00000000000..c7f1be54baa --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/FieldPath.java @@ -0,0 +1,148 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import com.google.protobuf.Descriptors.FieldDescriptor; +import io.deephaven.annotations.SimpleStyle; +import io.deephaven.functions.ToBooleanFunction; +import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Lazy; +import org.immutables.value.Value.Parameter; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** + * The {@link #path()} to a {@link com.google.protobuf.Descriptors.Descriptor Descriptor's} field. + * + *

+ * {@link FieldDescriptor} objects are not equal across {@link com.google.protobuf.DynamicMessage dynamic messages}. + * Helpers {@link #numberPath()} and {@link #namePath()} may be used in lieu of direct equivalence depending on context. + */ +@Immutable +@SimpleStyle +public abstract class FieldPath { + + private static final FieldPath EMPTY = of(List.of()); + + /** + * Creates an empty field path. Equivalent to {@code of(List.of())}. + * + * @return the empty field path + */ + public static FieldPath empty() { + return EMPTY; + } + + /** + * Creates a field path with {@code descriptors}. + * + * @param descriptors the descriptors + * @return the field path + */ + public static FieldPath of(FieldDescriptor... descriptors) { + return of(Arrays.asList(descriptors)); + } + + /** + * Creates a field path with {@code descriptors}. + * + * @param descriptors the descriptors + * @return the field path + */ + public static FieldPath of(List descriptors) { + return ImmutableFieldPath.of(descriptors); + } + + /** + * Creates a boolean function according to {@code simplePath}, where {@code simplePath} represents a + * {@link FieldPath#namePath() name path} that is '/' separated. The final name path part may be a '*' to + * additionally match everything that starts with {@code simplePath}. + * + *

+ * For example, {@code simplePath="/foo/bar"} will provide a function that matches the field path name paths + * {@code []}, {@code ["foo"]}, and {@code ["foo", "bar"]}. {@code simplePath="/foo/bar/*"} will provide a function + * that matches the previous example, as well as any field path name paths that start with {@code ["foo", "bar"]}: + * {@code ["foo", "bar", "baz"]}, {@code ["foo", "bar", "baz", "zap"}, {@code ["foo", "bar", "zip"]}, etc. + * + * @param simplePath the simple path + * @return the field path function + */ + public static ToBooleanFunction matches(String simplePath) { + simplePath = !simplePath.isEmpty() && simplePath.charAt(0) == '/' ? simplePath.substring(1) : simplePath; + final List np = Arrays.asList(simplePath.split("/")); + final boolean star = !np.isEmpty() && "*".equals(np.get(np.size() - 1)); + final List namePath = star ? np.subList(0, np.size() - 1) : np; + // This matches everything leading up to the name path. + // For example, simplePath=/foo/bar, namePath=[foo, bar], this will match field paths: + // [], [foo], and [foo, bar]. + final ToBooleanFunction leadingUpToMatch = fieldPath -> fieldPath.otherStartsWithThis(namePath); + if (!star) { + return leadingUpToMatch; + } + // This matches everything at or after name path in the case of a star. + // For example, simplePath=/foo/bar/*, namePath=[foo, bar], this will match field paths: + // [foo, bar], [foo, bar, baz], [foo, bar, baz, zip], etc. + final ToBooleanFunction starMatch = fieldPath -> fieldPath.startsWith(namePath); + return ToBooleanFunction.or(List.of(leadingUpToMatch, starMatch)); + } + + /** + * Creates a boolean function that is {@code true} when any component of {@code simplePaths} would + * {@link #matches(String)}. + * + *

+ * Equivalent to + * {@code ToBooleanFunction.or(simplePaths.stream().map(FieldPath::matches).collect(Collectors.toList()))}. + * + * @param simplePaths the simple paths + * @return the field path function + */ + public static ToBooleanFunction anyMatches(List simplePaths) { + return ToBooleanFunction.or(simplePaths.stream().map(FieldPath::matches).collect(Collectors.toList())); + } + + /** + * The ordered field descriptors which make up the field path. + * + * @return the path + */ + @Parameter + public abstract List path(); + + /** + * The number path for this field path. Equivalent to + * {@code FieldNumberPath.of(path().stream().mapToInt(FieldDescriptor::getNumber).toArray())}. + * + * @return the number path + */ + @Lazy + public FieldNumberPath numberPath() { + return FieldNumberPath.of(path().stream().mapToInt(FieldDescriptor::getNumber).toArray()); + } + + /** + * The name path for this field path. Equivalent to + * {@code path().stream().map(FieldDescriptor::getName).collect(Collectors.toList())}. + * + * @return the name path + **/ + @Lazy + public List namePath() { + return path().stream().map(FieldDescriptor::getName).collect(Collectors.toList()); + } + + public final boolean startsWith(List prefix) { + return startsWith(namePath(), prefix); + } + + public final boolean otherStartsWithThis(List other) { + return startsWith(other, namePath()); + } + + private static boolean startsWith(List x, List prefix) { + return x.subList(0, Math.min(prefix.size(), x.size())).equals(prefix); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/MessageParser.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/MessageParser.java new file mode 100644 index 00000000000..521ea482d39 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/MessageParser.java @@ -0,0 +1,128 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Message; +import io.deephaven.functions.TypedFunction; + +import java.util.ArrayList; +import java.util.List; +import java.util.ServiceLoader; + +/** + * The interface for creating {@link com.google.protobuf.Message message} {@link ProtobufFunctions protobuf functions}. + */ +public interface MessageParser { + + /** + * The built-in message parsers. + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
{@link Message message} type{@link TypedFunction function} type
{@link com.google.protobuf.Timestamp}{@link java.time.Instant}
{@link com.google.protobuf.Duration}{@link java.time.Duration}
{@link com.google.protobuf.BoolValue}{@code boolean}
{@link com.google.protobuf.Int32Value}{@code int}
{@link com.google.protobuf.UInt32Value}{@code int}
{@link com.google.protobuf.Int64Value}{@code long}
{@link com.google.protobuf.UInt64Value}{@code long}
{@link com.google.protobuf.FloatValue}{@code float}
{@link com.google.protobuf.DoubleValue}{@code double}
{@link com.google.protobuf.StringValue}{@link String}
{@link com.google.protobuf.BytesValue}{@code byte[]} (or {@link com.google.protobuf.ByteString}, see {@link FieldOptions#bytes()})
{@link com.google.protobuf.Any}{@link com.google.protobuf.Any}
{@link com.google.protobuf.FieldMask}{@link com.google.protobuf.FieldMask}
+ * + * @return the built-in parsers + */ + static List builtin() { + return Builtin.parsers(); + } + + /** + * Equivalent to {@code ServiceLoader.load(MessageParser.class)}. + * + * @return the service-loader message parsers + * @see ServiceLoader#load(Class) + */ + static Iterable serviceLoaders() { + return ServiceLoader.load(MessageParser.class); + } + + /** + * The default single-valued message parsers. Is the concatenation of {@link #builtin()} and + * {@link #serviceLoaders()}. + * + * @return the default message parsers + */ + static List defaults() { + final List out = new ArrayList<>(builtin()); + for (MessageParser parser : serviceLoaders()) { + out.add(parser); + } + return out; + } + + /** + * The canonical descriptor for the message. + * + * @return the descriptor + */ + Descriptor canonicalDescriptor(); + + /** + * The protobuf functions. + * + * @param descriptor the actual descriptor + * @param options the parser options + * @param fieldPath the field path + * @return the protobuf functions + */ + ProtobufFunctions messageParsers(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath); +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/MessageParserSingle.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/MessageParserSingle.java new file mode 100644 index 00000000000..9771721ec19 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/MessageParserSingle.java @@ -0,0 +1,31 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Message; +import io.deephaven.functions.TypedFunction; + +/** + * A specialized / simplified version of {@link MessageParser} that produces a single unnamed function. + */ +public interface MessageParserSingle extends MessageParser { + + /** + * The message parsing function. + * + * @param descriptor the actual descriptor + * @param options the parser options + * @param fieldPath the field path + * @return the message parsing function + */ + TypedFunction messageParser(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath); + + @Override + default ProtobufFunctions messageParsers(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + return ProtobufFunctions.unnamed(messageParser(descriptor, options, fieldPath)); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParser.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParser.java new file mode 100644 index 00000000000..9884b1d542d --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParser.java @@ -0,0 +1,175 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Descriptors.EnumValueDescriptor; +import com.google.protobuf.Descriptors.FieldDescriptor; +import com.google.protobuf.Descriptors.FieldDescriptor.Type; +import com.google.protobuf.Message; + +public final class ProtobufDescriptorParser { + + /** + * Creates {@link Message message} {@link ProtobufFunctions functions} than can parse messages according to the + * given {@code descriptor} and {@code options}. The resulting message functions require that the passed-in + * {@link Message#getDescriptorForType() messages' descriptor} be the same instance as {@code descriptor}. + * + *

+ * Parsing proceeds through each {@link Descriptor#getFields() descriptor field} that matches + * {@link FieldOptions#include()}}. By default, this is {@code true} for all fields. + * + *

+ * For simple types, the fields are parsed as: + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
{@link Type protobuf type}function type (no presence)function type (presence)function type (repeated)
{@link Type#INT32 int32}{@code int}{@link Integer}{@code int[]}
{@link Type#UINT32 uint32} (1){@code int}{@link Integer}{@code int[]}
{@link Type#SINT32 sint32}{@code int}{@link Integer}{@code int[]}
{@link Type#FIXED32 fixed32} (1){@code int}{@link Integer}{@code int[]}
{@link Type#SFIXED32 sfixed32}{@code int}{@link Integer}{@code int[]}
{@link Type#INT64 int64}{@code long}{@link Long}{@code long[]}
{@link Type#UINT64 uint64} (1){@code long}{@link Long}{@code long[]}
{@link Type#SINT64 sint64}{@code long}{@link Long}{@code long[]}
{@link Type#FIXED64 fixed64} (1){@code long}{@link Long}{@code long[]}
{@link Type#SFIXED64 sfixed64}{@code long}{@link Long}{@code long[]}
{@link Type#FLOAT float}{@code float}{@link Float}{@code float[]}
{@link Type#DOUBLE double}{@code double}{@link Double}{@code double[]}
{@link Type#BOOL bool}{@code boolean}{@link Boolean}{@code boolean[]}
{@link Type#STRING string}{@link String}{@link String}{@code String[]}
{@link Type#BYTES bytes}{@code byte[]}{@code byte[]}{@code byte[][]}
{@link Type#BYTES bytes} (2){@link com.google.protobuf.ByteString ByteString}{@link com.google.protobuf.ByteString ByteString}{@code ByteString[]}
{@link Type#ENUM enum}{@link EnumValueDescriptor EnumValueDescriptor}{@link EnumValueDescriptor EnumValueDescriptor}{@code EnumValueDescriptor[]}
+ * + * ^1 Unsigned 32-bit and 64-bit integers are represented using their signed counterpart, with the top bit being + * stored in the sign bit. This matches the Java protobuf behavior, + * scalar. Users may use + * {@link Integer#toUnsignedLong(int)} or {@link Long#toUnsignedString(long)} / + * {@link java.math.BigInteger#BigInteger(String)} to adapt as appropriate. + * + *

+ * ^2 The default behavior for {@link Type#BYTES bytes} is {@code byte[]}. To parse as + * {@link com.google.protobuf.ByteString ByteString} instead, configure {@link FieldOptions#bytes()} for the field. + * + *

+ * For {@link Type#MESSAGE message} (and {@link Type#GROUP group} if proto2) fields, the + * {@link ProtobufDescriptorParserOptions#parsers()} are used for well-known message types (for example, + * {@link com.google.protobuf.Timestamp} to {@link java.time.Instant}), otherwise parsing continues recursively with + * the the {@link FieldDescriptor#getMessageType() field's message descriptor}. To skip parsing as a well-known + * type, configure {@link FieldOptions#wellKnown()} for the field. If the field is repeated, the function return + * type will be the array-type with component type equal to what the non-repeated field function return type would + * be (for example, {@code repeated com.google.protobuf.Timestamp} will result in {@code java.time.Instant[]}). + * + *

+ * Protobuf maps are a special case, which result in a function type that returns a {@code Map}, + * where the keys are the equivalent {@code KeyType} and the values are the equivalent {@code ValueType}. To parse + * as a {@code repeated MapFieldEntry} instead of a {@code map}, configure {@link FieldOptions#map()} for the field. + * + *

+ * The {@link FieldPath} context is kept during traversal and is an important part of the returned message + * functions. Callers will typically use the returned field path to assign appropriate names to the functions. + * + * @param descriptor the descriptor + * @param options the options + * @return the parsed protobuf functions + * @see protobuf programming guide + * @see protobuf field presence + * @see protobuf maps + */ + public static ProtobufFunctions parse(Descriptor descriptor, ProtobufDescriptorParserOptions options) { + return new ProtobufDescriptorParserImpl(options).translate(descriptor); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParserImpl.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParserImpl.java new file mode 100644 index 00000000000..2f404fbd442 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParserImpl.java @@ -0,0 +1,551 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Descriptors.EnumValueDescriptor; +import com.google.protobuf.Descriptors.FieldDescriptor; +import com.google.protobuf.Descriptors.FieldDescriptor.JavaType; +import com.google.protobuf.Message; +import io.deephaven.protobuf.FieldOptions.BytesBehavior; +import io.deephaven.protobuf.FieldOptions.MapBehavior; +import io.deephaven.protobuf.FieldOptions.WellKnownBehavior; +import io.deephaven.protobuf.ProtobufFunctions.Builder; +import io.deephaven.qst.type.BoxedBooleanType; +import io.deephaven.qst.type.BoxedDoubleType; +import io.deephaven.qst.type.BoxedFloatType; +import io.deephaven.qst.type.BoxedIntType; +import io.deephaven.qst.type.BoxedLongType; +import io.deephaven.qst.type.GenericType; +import io.deephaven.qst.type.Type; +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToCharFunction; +import io.deephaven.functions.ToDoubleFunction; +import io.deephaven.functions.ToFloatFunction; +import io.deephaven.functions.ToIntFunction; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.ToPrimitiveFunction; +import io.deephaven.functions.ToShortFunction; +import io.deephaven.functions.TypedFunction; +import io.deephaven.functions.TypedFunction.Visitor; +import io.deephaven.util.QueryConstants; + +import java.lang.reflect.Array; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +class ProtobufDescriptorParserImpl { + + private static final ToObjectFunction STRING_OBJ = ToObjectFunction.identity(Type.stringType()); + private static final ToObjectFunction BOXED_INT_OBJ = ToObjectFunction.identity(BoxedIntType.of()); + private static final ToObjectFunction BOXED_LONG_OBJ = ToObjectFunction.identity(BoxedLongType.of()); + private static final ToObjectFunction BOXED_FLOAT_OBJ = + ToObjectFunction.identity(BoxedFloatType.of()); + private static final ToObjectFunction BOXED_DOUBLE_OBJ = + ToObjectFunction.identity(BoxedDoubleType.of()); + private static final ToObjectFunction BOXED_BOOLEAN_OBJ = + ToObjectFunction.identity(BoxedBooleanType.of()); + private static final ToObjectFunction MESSAGE_OBJ = + ToObjectFunction.identity(Type.ofCustom(Message.class)); + private static final ToObjectFunction BYTE_STRING_OBJ = + ToObjectFunction.identity(Type.ofCustom(ByteString.class)); + private static final ToObjectFunction ENUM_VALUE_DESCRIPTOR_OBJ = + ToObjectFunction.identity(Type.ofCustom(EnumValueDescriptor.class)); + private static final ToObjectFunction BYTE_STRING_FUNCTION = + BypassOnNull.of(ToObjectFunction.of(ByteString::toByteArray, Type.byteType().arrayType())); + + private final ProtobufDescriptorParserOptions options; + private final Map byFullName; + + public ProtobufDescriptorParserImpl(ProtobufDescriptorParserOptions options) { + this.options = Objects.requireNonNull(options); + this.byFullName = options.parsers().stream() + .collect(Collectors.toMap(x -> x.canonicalDescriptor().getFullName(), Function.identity())); + } + + public ProtobufFunctions translate(Descriptor descriptor) { + return new DescriptorContext(FieldPath.empty(), descriptor).functions(); + } + + private class DescriptorContext { + private final FieldPath fieldPath; + private final Descriptor descriptor; + + public DescriptorContext(FieldPath fieldPath, Descriptor descriptor) { + this.fieldPath = Objects.requireNonNull(fieldPath); + this.descriptor = Objects.requireNonNull(descriptor); + } + + private ProtobufFunctions functions() { + final ProtobufFunctions wellKnown = wellKnown().orElse(null); + if (wellKnown != null) { + return wellKnown; + } + final Builder builder = ProtobufFunctions.builder(); + for (FieldContext fc : fcs()) { + builder.addAllFunctions(fc.functions().functions()); + } + return builder.build(); + } + + private Optional wellKnown() { + final MessageParser mp = byFullName.get(descriptor.getFullName()); + if (mp == null) { + return Optional.empty(); + } + if (options.fieldOptions().apply(fieldPath).wellKnown() == WellKnownBehavior.asWellKnown()) { + return Optional.of(mp.messageParsers(descriptor, options, fieldPath)); + } + return Optional.empty(); + } + + private List fcs() { + return descriptor.getFields().stream().map(this::fc).collect(Collectors.toList()); + } + + private FieldContext fc(FieldDescriptor fd) { + return new FieldContext(this, fd); + } + } + + private class FieldContext { + private final DescriptorContext parent; + private final FieldDescriptor fd; + private final FieldPath fieldPath; + + public FieldContext(DescriptorContext parent, FieldDescriptor fd) { + this.parent = Objects.requireNonNull(parent); + this.fd = Objects.requireNonNull(fd); + this.fieldPath = append(parent.fieldPath, fd); + } + + private ProtobufFunctions functions() { + return functions(false); + } + + private ProtobufFunctions functions(boolean forceInclude) { + final FieldOptions fo = options.fieldOptions().apply(fieldPath); + if (!forceInclude && !fo.include()) { + return ProtobufFunctions.empty(); + } + if (fd.isMapField() && fo.map() == MapBehavior.asMap()) { + return new MapFieldObject().functions(); + } + if (fd.isRepeated()) { + return new RepeatedFieldObject().functions(); + } + return new FieldObject().functions(); + } + + private ProtobufFunctions namedField(TypedFunction tf) { + return ProtobufFunctions.of(ProtobufFunction.of(FieldPath.of(fd), tf)); + } + + private DescriptorContext toMessageContext() { + if (fd.getJavaType() != JavaType.MESSAGE) { + throw new IllegalStateException(); + } + return new DescriptorContext(fieldPath, fd.getMessageType()); + } + + private class FieldObject implements ToObjectFunction { + + @Override + public GenericType returnType() { + return Type.ofCustom(Object.class); + } + + @Override + public Object apply(Message message) { + // Note: in protobuf an actualized Message is never null - getField will always return non-null (and + // thus, sub Messages will never be natively null). + // + // In the case of our translation layer though, the presence of a null Message means that this field, or + // some parent of this field, was "not present". + if (message == null) { + return null; + } + // hasField only semantically meaningful when hasPresence == true + if (fd.hasPresence() && !message.hasField(fd)) { + return null; + } + return message.getField(fd); + } + + private ProtobufFunctions functions() { + // Note: we might be tempted at this layer to treat null boxed primitives as DH null primitives, but + // 1) this parsing layer doesn't / shouldn't need to know about DH nulls + // 2) protobuf already has the null object, so it doesn't harm us to propagate it to the calling layer, + // and for the calling layer to unbox if desired. + switch (fd.getJavaType()) { + case INT: + return fd.hasPresence() + ? namedField(mapToObj(BOXED_INT_OBJ)) + : namedField(mapToInt(ToIntFunction.cast())); + case LONG: + return fd.hasPresence() + ? namedField(mapToObj(BOXED_LONG_OBJ)) + : namedField(mapToLong(ToLongFunction.cast())); + case FLOAT: + return fd.hasPresence() + ? namedField(mapToObj(BOXED_FLOAT_OBJ)) + : namedField(mapToFloat(ToFloatFunction.cast())); + case DOUBLE: + return fd.hasPresence() + ? namedField(mapToObj(BOXED_DOUBLE_OBJ)) + : namedField(mapToDouble(ToDoubleFunction.cast())); + case BOOLEAN: + return fd.hasPresence() + ? namedField(mapToObj(BOXED_BOOLEAN_OBJ)) + : namedField(mapToBoolean(ToBooleanFunction.cast())); + case STRING: + return namedField(mapToObj(STRING_OBJ)); + case BYTE_STRING: + return options.fieldOptions().apply(fieldPath).bytes() == BytesBehavior.asByteArray() + ? namedField(mapToObj(BYTE_STRING_OBJ).mapToObj(BYTE_STRING_FUNCTION)) + : namedField(mapToObj(BYTE_STRING_OBJ)); + case ENUM: + return namedField(mapToObj(ENUM_VALUE_DESCRIPTOR_OBJ)); + case MESSAGE: { + final ToObjectFunction fieldAsMessage = mapToObj(MESSAGE_OBJ); + final DescriptorContext messageContext = toMessageContext(); + final ProtobufFunctions subF = messageContext.functions(); + final Builder builder = ProtobufFunctions.builder(); + final boolean parentFieldIsRepeated = !parent.fieldPath.path().isEmpty() + && parent.fieldPath.path().get(parent.fieldPath.path().size() - 1).isRepeated(); + for (ProtobufFunction e : subF.functions()) { + // The majority of the time, we need to BypassOnNull b/c the Message may be null. In the + // case where the message is part of a repeated field though, the Message is never null + // (it's always implicitly present based on the repeated count). + final TypedFunction value = parentFieldIsRepeated + ? e.function() + : BypassOnNull.of(e.function()); + builder.addFunctions( + ProtobufFunction.of(prepend(e.path(), fd), fieldAsMessage.map(value))); + } + return builder.build(); + } + default: + throw new IllegalStateException(); + } + } + } + + private class MapFieldObject { + + private ProtobufFunctions functions() { + // https://protobuf.dev/programming-guides/proto3/#maps + // For maps fields: + // map map_field = 1; + // The parsed descriptor looks like: + // @formatter:off + // message MapFieldEntry { + // option map_entry = true; + // optional KeyType key = 1; + // optional ValueType value = 2; + // } + // repeated MapFieldEntry map_field = 1; + // @formatter:on + + if (fd.getMessageType().getFields().size() != 2) { + throw new IllegalStateException("Expected map to have exactly 2 field descriptors"); + } + final FieldDescriptor keyFd = fd.getMessageType().findFieldByNumber(1); + if (keyFd == null) { + throw new IllegalStateException("Expected map to have field descriptor number 1 (key)"); + } + final FieldDescriptor valueFd = fd.getMessageType().findFieldByNumber(2); + if (valueFd == null) { + throw new IllegalStateException("Expected map to have field descriptor number 2 (value)"); + } + final DescriptorContext dc = new DescriptorContext(append(parent.fieldPath, fd), fd.getMessageType()); + + // Note: maps are a "special" case, where even though we don't include the key / value FDs as a return + // io.deephaven.protobuf.ProtobufFunction#path, it's important that we force their inclusion if we've + // gotten this far. + // + // For example, if we have the schema: + // @formatter:off + // message MyMessage { + // map my_map = 1; + // } + // @formatter:on + // + // The user should be able to use: + // @formatter:off + // ProtobufDescriptorParserOptions.builder() + // .fieldOptions(FieldOptions.includeIf(fp -> fp.namePath().equals(List.of("my_map")))) + // .build(); + // @formatter:on + // + // This involves parsing ["my_map", "key"] and ["my_map", "value"]. + // + // See io.deephaven.protobuf.ProtobufDescriptorParserTest#intIntMapRestrictiveInclude + final ProtobufFunctions keyFunctions = new FieldContext(dc, keyFd).functions(true); + if (keyFunctions.functions().size() != 1) { + throw new IllegalStateException("Protobuf map keys must be a single type"); + } + + final ProtobufFunctions valueFunctions = new FieldContext(dc, valueFd).functions(true); + if (valueFunctions.functions().size() != 1) { + // We've parsed the value type as an entity that has multiple values (as opposed to a single value + // we can put into a map). We may wish to have more configuration options for these situations in + // the future (ie, throw an exception or something else). For now, we're going to treat this case as + // "simple" repeated type. + return delegate(); + } + + final TypedFunction keyFunction = keyFunctions.functions().get(0).function(); + final TypedFunction valueFunction = valueFunctions.functions().get(0).function(); + return namedField(ToObjectFunction.of(message -> { + final Map map = new HashMap<>(); + final int count = message.getRepeatedFieldCount(fd); + for (int i = 0; i < count; ++i) { + final Message obj = (Message) message.getRepeatedField(fd, i); + final Object key = Box.apply(keyFunction, obj); + final Object value = Box.apply(valueFunction, obj); + map.put(key, value); + } + return map; + }, Type.ofCustom(Map.class))); + } + + private ProtobufFunctions delegate() { + return new RepeatedFieldObject().functions(); + } + } + + private class RepeatedFieldObject { + + private ProtobufFunctions functions() { + switch (fd.getJavaType()) { + case INT: + return namedField(mapInts(ToIntFunction.cast())); + case LONG: + return namedField(mapLongs(ToLongFunction.cast())); + case FLOAT: + return namedField(mapFloats(ToFloatFunction.cast())); + case DOUBLE: + return namedField(mapDoubles(ToDoubleFunction.cast())); + case BOOLEAN: + return namedField(mapBooleans(ToBooleanFunction.cast())); + case STRING: + return namedField(mapGenerics(STRING_OBJ)); + case BYTE_STRING: + return options.fieldOptions().apply(fieldPath).bytes() == BytesBehavior.asByteArray() + ? namedField(mapGenerics(BYTE_STRING_OBJ.mapToObj(BYTE_STRING_FUNCTION))) + : namedField(mapGenerics(BYTE_STRING_OBJ)); + case ENUM: + return namedField(mapGenerics(ENUM_VALUE_DESCRIPTOR_OBJ)); + case MESSAGE: { + final DescriptorContext messageContext = toMessageContext(); + final ProtobufFunctions functions = messageContext.functions(); + final Builder builder = ProtobufFunctions.builder(); + for (ProtobufFunction f : functions.functions()) { + final ToObjectFunction repeatedTf = f.function().walk(new ToRepeatedType()); + builder.addFunctions(ProtobufFunction.of(prepend(f.path(), fd), repeatedTf)); + } + return builder.build(); + } + default: + throw new IllegalStateException(); + } + } + + private ToObjectFunction mapChars(ToCharFunction f) { + return ToObjectFunction.of(m -> toChars(m, fd, f), Type.charType().arrayType()); + } + + private ToObjectFunction mapBytes(ToByteFunction f) { + return ToObjectFunction.of(m -> toBytes(m, fd, f), Type.byteType().arrayType()); + } + + private ToObjectFunction mapShorts(ToShortFunction f) { + return ToObjectFunction.of(m -> toShorts(m, fd, f), Type.shortType().arrayType()); + } + + private ToObjectFunction mapInts(ToIntFunction f) { + return ToObjectFunction.of(m -> toInts(m, fd, f), Type.intType().arrayType()); + } + + private ToObjectFunction mapLongs(ToLongFunction f) { + return ToObjectFunction.of(m -> toLongs(m, fd, f), Type.longType().arrayType()); + } + + private ToObjectFunction mapFloats(ToFloatFunction f) { + return ToObjectFunction.of(m -> toFloats(m, fd, f), Type.floatType().arrayType()); + } + + private ToObjectFunction mapDoubles(ToDoubleFunction f) { + return ToObjectFunction.of(m -> toDoubles(m, fd, f), Type.doubleType().arrayType()); + } + + private ToObjectFunction mapBooleans(ToBooleanFunction f) { + return ToObjectFunction.of(m -> toBooleans(m, fd, f), Type.booleanType().arrayType()); + } + + private ToObjectFunction mapGenerics(ToObjectFunction f) { + return ToObjectFunction.of(message -> toArray(message, fd, f), f.returnType().arrayType()); + } + + private class ToRepeatedType implements + Visitor>, + ToPrimitiveFunction.Visitor> { + + @Override + public ToObjectFunction visit(ToObjectFunction f) { + return mapGenerics(MESSAGE_OBJ.mapToObj(f)); + } + + @Override + public ToObjectFunction visit(ToPrimitiveFunction f) { + return f.walk((ToPrimitiveFunction.Visitor>) this); + } + + @Override + public ToObjectFunction visit(ToBooleanFunction f) { + return mapBooleans(MESSAGE_OBJ.mapToBoolean(f)); + } + + @Override + public ToObjectFunction visit(ToCharFunction f) { + return mapChars(MESSAGE_OBJ.mapToChar(f)); + } + + @Override + public ToObjectFunction visit(ToByteFunction f) { + return mapBytes(MESSAGE_OBJ.mapToByte(f)); + } + + @Override + public ToObjectFunction visit(ToShortFunction f) { + return mapShorts(MESSAGE_OBJ.mapToShort(f)); + } + + @Override + public ToObjectFunction visit(ToIntFunction f) { + return mapInts(MESSAGE_OBJ.mapToInt(f)); + } + + @Override + public ToObjectFunction visit(ToLongFunction f) { + return mapLongs(MESSAGE_OBJ.mapToLong(f)); + } + + @Override + public ToObjectFunction visit(ToFloatFunction f) { + return mapFloats(MESSAGE_OBJ.mapToFloat(f)); + } + + @Override + public ToObjectFunction visit(ToDoubleFunction f) { + return mapDoubles(MESSAGE_OBJ.mapToDouble(f)); + } + } + + private ProtobufFunctions namedField(TypedFunction tf) { + return ProtobufFunctions.of(ProtobufFunction.of(FieldPath.of(fd), tf)); + } + } + } + + private static char[] toChars(Message message, FieldDescriptor fd, ToCharFunction f) { + final int count = message.getRepeatedFieldCount(fd); + final char[] array = new char[count]; + for (int i = 0; i < count; ++i) { + array[i] = f.applyAsChar(message.getRepeatedField(fd, i)); + } + return array; + } + + private static byte[] toBytes(Message message, FieldDescriptor fd, ToByteFunction f) { + final int count = message.getRepeatedFieldCount(fd); + final byte[] array = new byte[count]; + for (int i = 0; i < count; ++i) { + array[i] = f.applyAsByte(message.getRepeatedField(fd, i)); + } + return array; + } + + private static short[] toShorts(Message message, FieldDescriptor fd, ToShortFunction f) { + final int count = message.getRepeatedFieldCount(fd); + final short[] array = new short[count]; + for (int i = 0; i < count; ++i) { + array[i] = f.applyAsShort(message.getRepeatedField(fd, i)); + } + return array; + } + + private static int[] toInts(Message message, FieldDescriptor fd, ToIntFunction f) { + final int count = message.getRepeatedFieldCount(fd); + final int[] array = new int[count]; + for (int i = 0; i < count; ++i) { + array[i] = f.applyAsInt(message.getRepeatedField(fd, i)); + } + return array; + } + + private static long[] toLongs(Message message, FieldDescriptor fd, ToLongFunction f) { + final int count = message.getRepeatedFieldCount(fd); + final long[] array = new long[count]; + for (int i = 0; i < count; ++i) { + array[i] = f.applyAsLong(message.getRepeatedField(fd, i)); + } + return array; + } + + private static float[] toFloats(Message message, FieldDescriptor fd, ToFloatFunction f) { + final int count = message.getRepeatedFieldCount(fd); + final float[] array = new float[count]; + for (int i = 0; i < count; ++i) { + array[i] = f.applyAsFloat(message.getRepeatedField(fd, i)); + } + return array; + } + + private static double[] toDoubles(Message message, FieldDescriptor fd, ToDoubleFunction f) { + final int count = message.getRepeatedFieldCount(fd); + final double[] array = new double[count]; + for (int i = 0; i < count; ++i) { + array[i] = f.applyAsDouble(message.getRepeatedField(fd, i)); + } + return array; + } + + private static boolean[] toBooleans(Message message, FieldDescriptor fd, ToBooleanFunction f) { + final int count = message.getRepeatedFieldCount(fd); + final boolean[] array = new boolean[count]; + for (int i = 0; i < count; ++i) { + array[i] = f.test(message.getRepeatedField(fd, i)); + } + return array; + } + + private static T[] toArray(Message message, FieldDescriptor fd, ToObjectFunction f) { + final int count = message.getRepeatedFieldCount(fd); + // noinspection unchecked + final T[] array = (T[]) Array.newInstance(f.returnType().clazz(), count); + for (int i = 0; i < count; ++i) { + array[i] = f.apply(message.getRepeatedField(fd, i)); + } + return array; + } + + private static FieldPath prepend(FieldPath f, FieldDescriptor fd) { + return FieldPath.of(Stream.concat(Stream.of(fd), f.path().stream()).collect(Collectors.toList())); + } + + private static FieldPath append(FieldPath f, FieldDescriptor fd) { + return FieldPath.of(Stream.concat(f.path().stream(), Stream.of(fd)).collect(Collectors.toList())); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParserOptions.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParserOptions.java new file mode 100644 index 00000000000..fe652e886e1 --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufDescriptorParserOptions.java @@ -0,0 +1,66 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import com.google.protobuf.Descriptors.Descriptor; +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value.Default; +import org.immutables.value.Value.Immutable; + +import java.util.List; +import java.util.function.Function; + +/** + * The {@link ProtobufDescriptorParser} options. + * + * @see ProtobufDescriptorParser#parse(Descriptor, ProtobufDescriptorParserOptions) + */ +@Immutable +@BuildableStyle +public abstract class ProtobufDescriptorParserOptions { + private static final Function DEFAULT_FIELD_OPTIONS = fp -> FieldOptions.defaults(); + private static final ProtobufDescriptorParserOptions DEFAULTS = builder().build(); + + public static Builder builder() { + return ImmutableProtobufDescriptorParserOptions.builder(); + } + + /** + * Creates a default options instance. + * + * @return the options + */ + public static ProtobufDescriptorParserOptions defaults() { + return DEFAULTS; + } + + /** + * The field options, allows the caller to specify different options for different field paths. Equivalent to + * {@code fieldPath -> FieldOptions.defaults()}. + * + * @return the field options + */ + @Default + public Function fieldOptions() { + return DEFAULT_FIELD_OPTIONS; + } + + /** + * Controls which message parsers to use. By default, is {@link MessageParser#defaults()}. + * + * @return the single-valued message parsers + */ + @Default + public List parsers() { + return MessageParser.defaults(); + } + + public interface Builder { + Builder fieldOptions(Function fieldOptions); + + Builder parsers(List parsers); + + ProtobufDescriptorParserOptions build(); + } +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufFunction.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufFunction.java new file mode 100644 index 00000000000..0fc8e95542f --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufFunction.java @@ -0,0 +1,55 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import com.google.protobuf.Message; +import io.deephaven.annotations.SimpleStyle; +import io.deephaven.functions.TypedFunction; +import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Parameter; + +/** + * Encapsulates the logic to extract a result from a {@link Message}. + */ +@Immutable +@SimpleStyle +public abstract class ProtobufFunction { + + /** + * Creates the unnamed protobuf function. Equivalent to {@code of(FieldPath.empty(), f)}. + * + * @param f the function + * @return the unnamed protobuf function + */ + public static ProtobufFunction unnamed(TypedFunction f) { + return of(FieldPath.empty(), f); + } + + /** + * Creates the protobuf function. + * + * @param path the field path + * @param f the function + * @return the protobuf function + */ + public static ProtobufFunction of(FieldPath path, TypedFunction f) { + return ImmutableProtobufFunction.of(path, f); + } + + /** + * The path that {@link #function()} uses to produce its result. + * + * @return the path + */ + @Parameter + public abstract FieldPath path(); + + /** + * The function to extract a result from a {@link Message}. + * + * @return the function + */ + @Parameter + public abstract TypedFunction function(); +} diff --git a/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufFunctions.java b/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufFunctions.java new file mode 100644 index 00000000000..7529314853c --- /dev/null +++ b/extensions/protobuf/src/main/java/io/deephaven/protobuf/ProtobufFunctions.java @@ -0,0 +1,73 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.protobuf; + +import com.google.protobuf.Message; +import io.deephaven.annotations.BuildableStyle; +import io.deephaven.functions.TypedFunction; +import org.immutables.value.Value.Immutable; + +import java.util.List; + +/** + * A list of {@link #functions()}. + */ +@Immutable +@BuildableStyle +public abstract class ProtobufFunctions { + + private static final ProtobufFunctions EMPTY = builder().build(); + + public static Builder builder() { + return ImmutableProtobufFunctions.builder(); + } + + /** + * Creates an empty protobuf functions. Equivalent to {@code builder().build()}. + * + * @return the empty protobuf functions + */ + public static ProtobufFunctions empty() { + return EMPTY; + } + + /** + * Creates a protobuf functions with a single, unnamed {@code function}. Equivalent to + * {@code builder().addFunctions(ProtobufFunction.unnammed(function)).build()}. + * + * @param function the function + * @return the protobuf functions + */ + public static ProtobufFunctions unnamed(TypedFunction function) { + return builder().addFunctions(ProtobufFunction.unnamed(function)).build(); + } + + /** + * Creates a protobuf fuctions with {@code functions}. Equivalent to + * {@code builder().addFunctions(functions).build()}. + * + * @param functions the functions + * @return the protobuf functions + */ + public static ProtobufFunctions of(ProtobufFunction... functions) { + return builder().addFunctions(functions).build(); + } + + /** + * The protobuf functions. + * + * @return the functions + */ + public abstract List functions(); + + public interface Builder { + Builder addFunctions(ProtobufFunction element); + + Builder addFunctions(ProtobufFunction... elements); + + Builder addAllFunctions(Iterable elements); + + ProtobufFunctions build(); + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/functions/ToBooleanFunctionTest.java b/extensions/protobuf/src/test/java/io/deephaven/functions/ToBooleanFunctionTest.java new file mode 100644 index 00000000000..f9828e8ce52 --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/functions/ToBooleanFunctionTest.java @@ -0,0 +1,64 @@ +package io.deephaven.functions; + +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static io.deephaven.functions.ToBooleanFunction.and; +import static io.deephaven.functions.ToBooleanFunction.map; +import static io.deephaven.functions.ToBooleanFunction.not; +import static io.deephaven.functions.ToBooleanFunction.ofFalse; +import static io.deephaven.functions.ToBooleanFunction.ofTrue; +import static io.deephaven.functions.ToBooleanFunction.or; +import static org.assertj.core.api.Assertions.assertThat; + +public class ToBooleanFunctionTest { + + @Test + void ofTrue_() { + assertThat(ofTrue().test(new Object())).isTrue(); + } + + @Test + void ofFalse_() { + assertThat(ofFalse().test(new Object())).isFalse(); + } + + @Test + void or_() { + assertThat(or(List.of()).test(new Object())).isFalse(); + assertThat(or(List.of(ofFalse())).test(new Object())).isFalse(); + assertThat(or(List.of(ofTrue())).test(new Object())).isTrue(); + assertThat(or(List.of(ofFalse(), ofFalse())).test(new Object())).isFalse(); + assertThat(or(List.of(ofFalse(), ofTrue())).test(new Object())).isTrue(); + assertThat(or(List.of(ofTrue(), ofFalse())).test(new Object())).isTrue(); + assertThat(or(List.of(ofTrue(), ofTrue())).test(new Object())).isTrue(); + } + + @Test + void and_() { + assertThat(and(List.of()).test(new Object())).isTrue(); + assertThat(and(List.of(ofFalse())).test(new Object())).isFalse(); + assertThat(and(List.of(ofTrue())).test(new Object())).isTrue(); + assertThat(and(List.of(ofFalse(), ofFalse())).test(new Object())).isFalse(); + assertThat(and(List.of(ofFalse(), ofTrue())).test(new Object())).isFalse(); + assertThat(and(List.of(ofTrue(), ofFalse())).test(new Object())).isFalse(); + assertThat(and(List.of(ofTrue(), ofTrue())).test(new Object())).isTrue(); + } + + @Test + void not_() { + assertThat(not(ofTrue()).test(new Object())).isFalse(); + assertThat(not(ofFalse()).test(new Object())).isTrue(); + } + + @Test + void map_() { + final ToBooleanFunction trimIsFoo = map(String::trim, "foo"::equals); + assertThat(trimIsFoo.test("")).isFalse(); + assertThat(trimIsFoo.test(" ")).isFalse(); + assertThat(trimIsFoo.test("foo")).isTrue(); + assertThat(trimIsFoo.test(" foo ")).isTrue(); + assertThat(trimIsFoo.test(" foo bar")).isFalse(); + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/functions/ToByteFunctionTest.java b/extensions/protobuf/src/test/java/io/deephaven/functions/ToByteFunctionTest.java new file mode 100644 index 00000000000..612e24fb0c6 --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/functions/ToByteFunctionTest.java @@ -0,0 +1,20 @@ +package io.deephaven.functions; + +import org.junit.jupiter.api.Test; + +import static io.deephaven.functions.ToByteFunction.map; +import static org.assertj.core.api.Assertions.assertThat; + +public class ToByteFunctionTest { + + @Test + void map_() { + final ToByteFunction firstByte = map(String::getBytes, ToByteFunctionTest::firstByte); + assertThat(firstByte.applyAsByte("foo")).isEqualTo((byte) 'f'); + assertThat(firstByte.applyAsByte("oof")).isEqualTo((byte) 'o'); + } + + private static byte firstByte(byte[] x) { + return x[0]; + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/functions/ToCharFunctionTest.java b/extensions/protobuf/src/test/java/io/deephaven/functions/ToCharFunctionTest.java new file mode 100644 index 00000000000..551ed87ce46 --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/functions/ToCharFunctionTest.java @@ -0,0 +1,20 @@ +package io.deephaven.functions; + +import org.junit.jupiter.api.Test; + +import static io.deephaven.functions.ToCharFunction.map; +import static org.assertj.core.api.Assertions.assertThat; + +public class ToCharFunctionTest { + + @Test + void map_() { + final ToCharFunction firstByte = map(String::toCharArray, ToCharFunctionTest::firstChar); + assertThat(firstByte.applyAsChar("foo")).isEqualTo('f'); + assertThat(firstByte.applyAsChar("oof")).isEqualTo('o'); + } + + private static char firstChar(char[] x) { + return x[0]; + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/functions/ToDoubleFunctionTest.java b/extensions/protobuf/src/test/java/io/deephaven/functions/ToDoubleFunctionTest.java new file mode 100644 index 00000000000..40fdc738c09 --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/functions/ToDoubleFunctionTest.java @@ -0,0 +1,16 @@ +package io.deephaven.functions; + +import org.junit.jupiter.api.Test; + +import static io.deephaven.functions.ToDoubleFunction.map; +import static org.assertj.core.api.Assertions.assertThat; + +public class ToDoubleFunctionTest { + + @Test + void map_() { + final ToDoubleFunction bytesLength = map(String::getBytes, x -> (double) x.length); + assertThat(bytesLength.applyAsDouble("foo")).isEqualTo(3.0); + assertThat(bytesLength.applyAsDouble("food")).isEqualTo(4.0); + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/functions/ToFloatFunctionTest.java b/extensions/protobuf/src/test/java/io/deephaven/functions/ToFloatFunctionTest.java new file mode 100644 index 00000000000..c408030d40b --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/functions/ToFloatFunctionTest.java @@ -0,0 +1,16 @@ +package io.deephaven.functions; + +import org.junit.jupiter.api.Test; + +import static io.deephaven.functions.ToFloatFunction.map; +import static org.assertj.core.api.Assertions.assertThat; + +public class ToFloatFunctionTest { + + @Test + void map_() { + final ToFloatFunction bytesLength = map(String::getBytes, x -> (float) x.length); + assertThat(bytesLength.applyAsFloat("foo")).isEqualTo(3.0f); + assertThat(bytesLength.applyAsFloat("food")).isEqualTo(4.0f); + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/functions/ToIntFunctionTest.java b/extensions/protobuf/src/test/java/io/deephaven/functions/ToIntFunctionTest.java new file mode 100644 index 00000000000..b2d2c80f314 --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/functions/ToIntFunctionTest.java @@ -0,0 +1,16 @@ +package io.deephaven.functions; + +import org.junit.jupiter.api.Test; + +import static io.deephaven.functions.ToIntFunction.map; +import static org.assertj.core.api.Assertions.assertThat; + +public class ToIntFunctionTest { + + @Test + void map_() { + final ToIntFunction bytesLength = map(String::getBytes, x -> x.length); + assertThat(bytesLength.applyAsInt("foo")).isEqualTo(3); + assertThat(bytesLength.applyAsInt("food")).isEqualTo(4); + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/functions/ToLongFunctionTest.java b/extensions/protobuf/src/test/java/io/deephaven/functions/ToLongFunctionTest.java new file mode 100644 index 00000000000..10847dbd72c --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/functions/ToLongFunctionTest.java @@ -0,0 +1,16 @@ +package io.deephaven.functions; + +import org.junit.jupiter.api.Test; + +import static io.deephaven.functions.ToLongFunction.map; +import static org.assertj.core.api.Assertions.assertThat; + +public class ToLongFunctionTest { + + @Test + void map_() { + final ToLongFunction bytesLength = map(String::getBytes, x -> (long) x.length); + assertThat(bytesLength.applyAsLong("foo")).isEqualTo(3L); + assertThat(bytesLength.applyAsLong("food")).isEqualTo(4L); + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/functions/ToObjectFunctionTest.java b/extensions/protobuf/src/test/java/io/deephaven/functions/ToObjectFunctionTest.java new file mode 100644 index 00000000000..4aac140c0ee --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/functions/ToObjectFunctionTest.java @@ -0,0 +1,23 @@ +package io.deephaven.functions; + +import io.deephaven.qst.type.Type; +import org.junit.jupiter.api.Test; + +import static io.deephaven.functions.ToObjectFunction.map; +import static org.assertj.core.api.Assertions.assertThat; + +public class ToObjectFunctionTest { + + @Test + void map_() { + final ToObjectFunction bytesLength = + map(String::getBytes, ToObjectFunctionTest::firstAndLast, Type.byteType().arrayType()); + assertThat(bytesLength.apply("f")).containsExactly((byte) 'f', (byte) 'f'); + assertThat(bytesLength.apply("foo")).containsExactly((byte) 'f', (byte) 'o'); + assertThat(bytesLength.apply("food")).containsExactly((byte) 'f', (byte) 'd'); + } + + private static byte[] firstAndLast(byte[] x) { + return new byte[] {x[0], x[x.length - 1]}; + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/functions/ToShortFunctionTest.java b/extensions/protobuf/src/test/java/io/deephaven/functions/ToShortFunctionTest.java new file mode 100644 index 00000000000..9c70b949d11 --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/functions/ToShortFunctionTest.java @@ -0,0 +1,16 @@ +package io.deephaven.functions; + +import org.junit.jupiter.api.Test; + +import static io.deephaven.functions.ToShortFunction.map; +import static org.assertj.core.api.Assertions.assertThat; + +public class ToShortFunctionTest { + + @Test + void map_() { + final ToShortFunction bytesLength = map(String::getBytes, x -> (short) x.length); + assertThat(bytesLength.applyAsShort("foo")).isEqualTo((short) 3); + assertThat(bytesLength.applyAsShort("food")).isEqualTo((short) 4); + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/protobuf/ByteWrapperCustomType.java b/extensions/protobuf/src/test/java/io/deephaven/protobuf/ByteWrapperCustomType.java new file mode 100644 index 00000000000..3ca71a558d4 --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/protobuf/ByteWrapperCustomType.java @@ -0,0 +1,34 @@ +package io.deephaven.protobuf; + +import com.google.auto.service.AutoService; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Message; +import io.deephaven.functions.ToByteFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.TypedFunction; +import io.deephaven.protobuf.test.ByteWrapper; +import io.deephaven.qst.type.Type; + +@AutoService(MessageParser.class) +public class ByteWrapperCustomType implements MessageParserSingle { + private static final ToByteFunction CANONICAL_FUNCTION = ToObjectFunction + .identity(Type.ofCustom(ByteWrapper.class)) + .mapToByte(ByteWrapperCustomType::getByte); + + public ByteWrapperCustomType() {} + + @Override + public Descriptor canonicalDescriptor() { + return ByteWrapper.getDescriptor(); + } + + @Override + public TypedFunction messageParser(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + return CANONICAL_FUNCTION; + } + + private static byte getByte(ByteWrapper wrapper) { + return (byte) wrapper.getValue(); + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/protobuf/FieldPathTest.java b/extensions/protobuf/src/test/java/io/deephaven/protobuf/FieldPathTest.java new file mode 100644 index 00000000000..858d008c384 --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/protobuf/FieldPathTest.java @@ -0,0 +1,93 @@ +package io.deephaven.protobuf; + +import com.google.protobuf.Descriptors.FieldDescriptor; +import io.deephaven.functions.ToBooleanFunction; +import io.deephaven.protobuf.test.FieldPathTesting; +import io.deephaven.protobuf.test.FieldPathTesting.Bar; +import io.deephaven.protobuf.test.FieldPathTesting.Baz; +import io.deephaven.protobuf.test.FieldPathTesting.Foo; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +public class FieldPathTest { + + private static final FieldDescriptor FOO = FieldPathTesting.getDescriptor().findFieldByName("foo"); + private static final FieldDescriptor BAR = Foo.getDescriptor().findFieldByName("bar"); + private static final FieldDescriptor BAZ = Bar.getDescriptor().findFieldByName("baz"); + private static final FieldDescriptor ZIP = Baz.getDescriptor().findFieldByName("zip"); + private static final FieldDescriptor ZAP = Foo.getDescriptor().findFieldByName("zap"); + private static final FieldDescriptor ZOOM = Foo.getDescriptor().findFieldByName("zoom"); + + @Test + void namePath() { + assertThat(FieldPath.of(FOO, ZOOM).namePath()).containsExactly("foo", "zoom"); + } + + @Test + void numberPath() { + assertThat(FieldPath.of(FOO, ZOOM).numberPath()).isEqualTo(FieldNumberPath.of(1, 3)); + } + + @Test + void namePathStartsWith() { + assertThat(FieldPath.of(FOO, ZOOM).startsWith(List.of())).isTrue(); + assertThat(FieldPath.of(FOO, ZOOM).startsWith(List.of("foo"))).isTrue(); + assertThat(FieldPath.of(FOO, ZOOM).startsWith(List.of("foo", "zoom"))).isTrue(); + assertThat(FieldPath.of(FOO, ZOOM).startsWith(List.of("foo", "zoom", "oops"))).isFalse(); + assertThat(FieldPath.of(FOO, ZOOM).startsWith(List.of("foo", "bar"))).isFalse(); + assertThat(FieldPath.of(FOO, ZOOM).startsWith(List.of("bar"))).isFalse(); + } + + @Test + void namePathOtherStartsWithThis() { + assertThat(FieldPath.of(FOO, ZOOM).otherStartsWithThis(List.of())).isFalse(); + assertThat(FieldPath.of(FOO, ZOOM).otherStartsWithThis(List.of("foo"))).isFalse(); + assertThat(FieldPath.of(FOO, ZOOM).otherStartsWithThis(List.of("foo", "zoom"))).isTrue(); + assertThat(FieldPath.of(FOO, ZOOM).otherStartsWithThis(List.of("foo", "zoom", "oops"))).isTrue(); + assertThat(FieldPath.of(FOO, ZOOM).otherStartsWithThis(List.of("foo", "bar"))).isFalse(); + assertThat(FieldPath.of(FOO, ZOOM).otherStartsWithThis(List.of("bar"))).isFalse(); + } + + @Test + void fooBarMatches() { + for (final String simplePath : List.of("/foo/bar", "foo/bar")) { + final ToBooleanFunction matcher = FieldPath.matches(simplePath); + assertThat(matcher.test(FieldPath.empty())).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO))).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO, BAR))).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO, BAR, BAZ))).isFalse(); + assertThat(matcher.test(FieldPath.of(FOO, BAR, BAZ, ZIP))).isFalse(); + assertThat(matcher.test(FieldPath.of(FOO, ZAP))).isFalse(); + assertThat(matcher.test(FieldPath.of(FOO, ZOOM))).isFalse(); + } + } + + @Test + void fooBarStarMatches() { + for (final String simplePath : List.of("/foo/bar/*", "foo/bar/*")) { + final ToBooleanFunction matcher = FieldPath.matches(simplePath); + assertThat(matcher.test(FieldPath.empty())).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO))).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO, BAR))).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO, BAR, BAZ))).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO, BAR, BAZ, ZIP))).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO, ZAP))).isFalse(); + assertThat(matcher.test(FieldPath.of(FOO, ZOOM))).isFalse(); + } + } + + @Test + void zipZoomAnyMatches() { + final ToBooleanFunction matcher = FieldPath.anyMatches(List.of("/foo/bar/baz/zip", "/foo/zoom")); + assertThat(matcher.test(FieldPath.empty())).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO))).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO, BAR))).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO, BAR, BAZ))).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO, BAR, BAZ, ZIP))).isTrue(); + assertThat(matcher.test(FieldPath.of(FOO, ZAP))).isFalse(); + assertThat(matcher.test(FieldPath.of(FOO, ZOOM))).isTrue(); + } +} diff --git a/extensions/protobuf/src/test/java/io/deephaven/protobuf/ProtobufDescriptorParserTest.java b/extensions/protobuf/src/test/java/io/deephaven/protobuf/ProtobufDescriptorParserTest.java new file mode 100644 index 00000000000..47db6a87599 --- /dev/null +++ b/extensions/protobuf/src/test/java/io/deephaven/protobuf/ProtobufDescriptorParserTest.java @@ -0,0 +1,1515 @@ +package io.deephaven.protobuf; + +import com.google.protobuf.Any; +import com.google.protobuf.BoolValue; +import com.google.protobuf.ByteString; +import com.google.protobuf.BytesValue; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Descriptors.EnumValueDescriptor; +import com.google.protobuf.DoubleValue; +import com.google.protobuf.FieldMask; +import com.google.protobuf.FloatValue; +import com.google.protobuf.Int32Value; +import com.google.protobuf.Int64Value; +import com.google.protobuf.Message; +import com.google.protobuf.StringValue; +import com.google.protobuf.Timestamp; +import com.google.protobuf.UInt32Value; +import com.google.protobuf.UInt64Value; +import io.deephaven.functions.ToLongFunction; +import io.deephaven.functions.ToObjectFunction; +import io.deephaven.functions.TypedFunction; +import io.deephaven.protobuf.FieldOptions.BytesBehavior; +import io.deephaven.protobuf.FieldOptions.MapBehavior; +import io.deephaven.protobuf.FieldOptions.WellKnownBehavior; +import io.deephaven.protobuf.test.ADuration; +import io.deephaven.protobuf.test.ALongNestedTimestampMap; +import io.deephaven.protobuf.test.ALongTimestampMap; +import io.deephaven.protobuf.test.AMultiNested; +import io.deephaven.protobuf.test.AMultiNested.SubMessage1; +import io.deephaven.protobuf.test.AMultiNested.SubMessage1.SubMessage2; +import io.deephaven.protobuf.test.ANested; +import io.deephaven.protobuf.test.ANested.SubMessage; +import io.deephaven.protobuf.test.AStringStringMap; +import io.deephaven.protobuf.test.ATimestamp; +import io.deephaven.protobuf.test.AnEnum; +import io.deephaven.protobuf.test.AnEnum.TheEnum; +import io.deephaven.protobuf.test.AnIntFooBarMap; +import io.deephaven.protobuf.test.AnIntFooBarMap.FooBar; +import io.deephaven.protobuf.test.AnIntIntMap; +import io.deephaven.protobuf.test.AnyWrapper; +import io.deephaven.protobuf.test.ByteWrapper; +import io.deephaven.protobuf.test.ByteWrapperRepeated; +import io.deephaven.protobuf.test.FieldMaskWrapper; +import io.deephaven.protobuf.test.MultiRepeated; +import io.deephaven.protobuf.test.NestedByteWrapper; +import io.deephaven.protobuf.test.NestedRepeatedTimestamps; +import io.deephaven.protobuf.test.NestedRepeatedTimestamps.Timestamps; +import io.deephaven.protobuf.test.OptionalBasics; +import io.deephaven.protobuf.test.RepeatedBasics; +import io.deephaven.protobuf.test.RepeatedDuration; +import io.deephaven.protobuf.test.RepeatedMessage; +import io.deephaven.protobuf.test.RepeatedMessage.Person; +import io.deephaven.protobuf.test.RepeatedTimestamp; +import io.deephaven.protobuf.test.RepeatedWrappers; +import io.deephaven.protobuf.test.TheWrappers; +import io.deephaven.protobuf.test.TwoTs; +import io.deephaven.protobuf.test.UnionType; +import io.deephaven.qst.type.BoxedBooleanType; +import io.deephaven.qst.type.BoxedDoubleType; +import io.deephaven.qst.type.BoxedFloatType; +import io.deephaven.qst.type.BoxedIntType; +import io.deephaven.qst.type.BoxedLongType; +import io.deephaven.qst.type.CustomType; +import io.deephaven.qst.type.Type; +import io.deephaven.util.QueryConstants; +import org.junit.jupiter.api.Test; + +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.function.Function; +import java.util.function.Predicate; + +import static org.assertj.core.api.Assertions.assertThat; + +public class ProtobufDescriptorParserTest { + + @Test + public void string() { + checkKey(StringValue.getDescriptor(), List.of(), Type.stringType(), Map.of( + StringValue.of("foo"), "foo", + StringValue.of("bar"), "bar")); + } + + @Test + public void int32() { + checkKey(Int32Value.getDescriptor(), List.of(), Type.intType(), Map.of( + Int32Value.of(42), 42, + Int32Value.of(43), 43)); + } + + @Test + public void uint32() { + checkKey(UInt32Value.getDescriptor(), List.of(), Type.intType(), Map.of( + UInt32Value.of(42), 42, + UInt32Value.of(43), 43)); + } + + @Test + public void int64() { + checkKey(Int64Value.getDescriptor(), List.of(), Type.longType(), Map.of( + Int64Value.of(42), 42L, + Int64Value.of(43), 43L)); + } + + @Test + public void uint64() { + checkKey(UInt64Value.getDescriptor(), List.of(), Type.longType(), Map.of( + UInt64Value.of(42), 42L, + UInt64Value.of(43), 43L)); + } + + @Test + public void float_() { + checkKey(FloatValue.getDescriptor(), List.of(), Type.floatType(), Map.of( + FloatValue.of(42), 42.0f, + FloatValue.of(43), 43.0f)); + } + + @Test + public void double_() { + checkKey(DoubleValue.getDescriptor(), List.of(), Type.doubleType(), Map.of( + DoubleValue.of(42), 42.0d, + DoubleValue.of(43), 43.0d)); + } + + @Test + public void bool() { + checkKey(BoolValue.getDescriptor(), List.of(), Type.booleanType(), Map.of( + BoolValue.of(true), true, + BoolValue.of(false), false)); + } + + @Test + public void bytes() { + final ByteString foo = ByteString.copyFromUtf8("foo"); + final ByteString bar = ByteString.copyFromUtf8("bar"); + checkKey(BytesValue.getDescriptor(), List.of(), Type.byteType().arrayType(), Map.of( + BytesValue.of(foo), "foo".getBytes(StandardCharsets.UTF_8), + BytesValue.of(bar), "bar".getBytes(StandardCharsets.UTF_8))); + } + + @Test + void timestamp() { + final Map, TypedFunction> nf = nf(Timestamp.getDescriptor()); + assertThat(nf.keySet()).containsExactly(List.of()); + checkKey( + Timestamp.getDescriptor(), + List.of(), + Type.instantType(), + Map.of( + Timestamp.getDefaultInstance(), + Instant.ofEpochSecond(0), + Timestamp.newBuilder().setSeconds(1).setNanos(2).build(), + Instant.ofEpochSecond(1, 2))); + } + + @Test + void unionTypes() { + final Map, TypedFunction> nf = + nf(UnionType.getDescriptor()); + assertThat(nf.keySet()).containsExactly( + List.of("bool"), + List.of("int32"), + List.of("uint32"), + List.of("int64"), + List.of("uint64"), + List.of("float"), + List.of("double"), + List.of("string"), + List.of("bytes")); + } + + @Test + void oneOfBool() { + final Map, TypedFunction> nf = + nf(UnionType.getDescriptor()); + final UnionType message = UnionType.newBuilder().setBool(true).build(); + assertThat(Box.apply(nf.get(List.of("bool")), message)).isEqualTo(true); + assertThat(Box.apply(nf.get(List.of("int32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("float")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("double")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("string")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("bytes")), message)).isNull(); + } + + @Test + void oneOfInt32() { + final Map, TypedFunction> nf = + nf(UnionType.getDescriptor()); + final UnionType message = UnionType.newBuilder().setInt32(42).build(); + assertThat(Box.apply(nf.get(List.of("bool")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int32")), message)).isEqualTo(42); + assertThat(Box.apply(nf.get(List.of("uint32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("float")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("double")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("string")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("bytes")), message)).isNull(); + } + + @Test + void oneOfUInt32() { + final Map, TypedFunction> nf = + nf(UnionType.getDescriptor()); + final UnionType message = UnionType.newBuilder().setUint32(42).build(); + assertThat(Box.apply(nf.get(List.of("bool")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint32")), message)).isEqualTo(42); + assertThat(Box.apply(nf.get(List.of("int64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("float")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("double")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("string")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("bytes")), message)).isNull(); + } + + @Test + void oneOfInt64() { + final Map, TypedFunction> nf = + nf(UnionType.getDescriptor()); + final UnionType message = UnionType.newBuilder().setInt64(42).build(); + assertThat(Box.apply(nf.get(List.of("bool")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int64")), message)).isEqualTo(42L); + assertThat(Box.apply(nf.get(List.of("uint64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("float")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("double")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("string")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("bytes")), message)).isNull(); + } + + @Test + void oneOfUInt64() { + final Map, TypedFunction> nf = + nf(UnionType.getDescriptor()); + final UnionType message = UnionType.newBuilder().setUint64(42).build(); + assertThat(Box.apply(nf.get(List.of("bool")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint64")), message)).isEqualTo(42L); + assertThat(Box.apply(nf.get(List.of("float")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("double")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("string")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("bytes")), message)).isNull(); + } + + @Test + void oneOfFloat() { + final Map, TypedFunction> nf = + nf(UnionType.getDescriptor()); + final UnionType message = UnionType.newBuilder().setFloat(42.0f).build(); + assertThat(Box.apply(nf.get(List.of("bool")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("float")), message)).isEqualTo(42.0f); + assertThat(Box.apply(nf.get(List.of("double")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("string")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("bytes")), message)).isNull(); + } + + @Test + void oneOfDouble() { + final Map, TypedFunction> nf = + nf(UnionType.getDescriptor()); + final UnionType message = UnionType.newBuilder().setDouble(42.0d).build(); + assertThat(Box.apply(nf.get(List.of("bool")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("float")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("double")), message)).isEqualTo(42.0d); + assertThat(Box.apply(nf.get(List.of("string")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("bytes")), message)).isNull(); + } + + @Test + void oneOfString() { + final Map, TypedFunction> nf = + nf(UnionType.getDescriptor()); + final UnionType message = UnionType.newBuilder().setString("hello").build(); + assertThat(Box.apply(nf.get(List.of("bool")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("float")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("double")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("string")), message)).isEqualTo("hello"); + assertThat(Box.apply(nf.get(List.of("bytes")), message)).isNull(); + } + + @Test + void oneOfBytes() { + final Map, TypedFunction> nf = + nf(UnionType.getDescriptor()); + final UnionType message = UnionType.newBuilder().setBytes(ByteString.copyFromUtf8("world")).build(); + assertThat(Box.apply(nf.get(List.of("bool")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint32")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("int64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("uint64")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("float")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("double")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("string")), message)).isNull(); + assertThat(Box.apply(nf.get(List.of("bytes")), message)) + .isEqualTo("world".getBytes(StandardCharsets.UTF_8)); + } + + @Test + void stringStringMap() { + checkKey(AStringStringMap.getDescriptor(), List.of("properties"), Type.ofCustom(Map.class), + Map.of(AStringStringMap.newBuilder() + .putProperties("foo", "bar") + .putProperties("hello", "world").build(), + Map.of("foo", "bar", "hello", "world"))); + } + + @Test + void intIntMap() { + checkKey(AnIntIntMap.getDescriptor(), List.of("properties"), Type.ofCustom(Map.class), + Map.of(AnIntIntMap.newBuilder() + .putProperties(1, 2) + .putProperties(3, 4).build(), + Map.of(1, 2, 3, 4))); + } + + @Test + void intIntMapRestrictiveInclude() { + final List path = List.of("properties"); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(includeIf(fp -> fp.namePath().equals(path))) + .build(); + checkKey(AnIntIntMap.getDescriptor(), options, path, Type.ofCustom(Map.class), + Map.of(AnIntIntMap.newBuilder() + .putProperties(1, 2) + .putProperties(3, 4).build(), + Map.of(1, 2, 3, 4))); + } + + @Test + void longTimestampMap() { + checkKey(ALongTimestampMap.getDescriptor(), List.of("properties"), Type.ofCustom(Map.class), + Map.of(ALongTimestampMap.newBuilder() + .putProperties(1, Timestamp.newBuilder().setSeconds(42).build()) + .putProperties(2, Timestamp.newBuilder().setSeconds(43).build()).build(), + Map.of(1L, Instant.ofEpochSecond(42), + 2L, Instant.ofEpochSecond(43)))); + } + + @Test + void longNestedTimestampMap() { + checkKey(ALongNestedTimestampMap.getDescriptor(), List.of("properties"), Type.ofCustom(Map.class), + Map.of(ALongNestedTimestampMap.newBuilder() + .putProperties(1, + ATimestamp.newBuilder().setTs(Timestamp.newBuilder().setSeconds(42).build()).build()) + .putProperties(2, + ATimestamp.newBuilder().setTs(Timestamp.newBuilder().setSeconds(43).build()).build()) + .build(), + Map.of(1L, Instant.ofEpochSecond(42), + 2L, Instant.ofEpochSecond(43)))); + } + + @Test + void intFooBarMap() { + // this test is exercising the case where the value is a complex type, and we default back to treating it as a + // repeating type instead of a map + final AnIntFooBarMap empty = AnIntFooBarMap.getDefaultInstance(); + final AnIntFooBarMap map1 = AnIntFooBarMap.newBuilder() + .putProperties(1, FooBar.newBuilder().build()) + .build(); + final AnIntFooBarMap map2 = AnIntFooBarMap.newBuilder() + .putProperties(1, FooBar.newBuilder().build()) + .putProperties(2, FooBar.newBuilder().setFoo("fighter").setBar(42).build()) + .build(); + + checkKey( + AnIntFooBarMap.getDescriptor(), + List.of("properties", "key"), + Type.intType().arrayType(), + Map.of( + empty, new int[0], + map1, new int[] {1}, + map2, new int[] {1, 2})); + + checkKey( + AnIntFooBarMap.getDescriptor(), + List.of("properties", "value", "foo"), + Type.stringType().arrayType(), + Map.of( + empty, new String[0], + map1, new String[] {""}, + map2, new String[] {"", "fighter"})); + + checkKey( + AnIntFooBarMap.getDescriptor(), + List.of("properties", "value", "bar"), + Type.longType().arrayType(), + Map.of( + empty, new long[0], + map1, new long[] {0L}, + map2, new long[] {0L, 42L})); + } + + @Test + void longTimestampMapAsRepeated() { + final FieldOptions antiMap = FieldOptions.builder().map(MapBehavior.asRepeated()).build(); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(x -> antiMap) + .build(); + final ALongTimestampMap e1 = ALongTimestampMap.newBuilder() + .putProperties(1, Timestamp.newBuilder().setSeconds(42).build()) + .putProperties(2, Timestamp.newBuilder().setSeconds(43).build()) + .build(); + checkKey(ALongTimestampMap.getDescriptor(), + options, + List.of("properties", "key"), + Type.longType().arrayType(), + Map.of(e1, new long[] {1, 2})); + checkKey(ALongTimestampMap.getDescriptor(), + options, + List.of("properties", "value"), + Type.instantType().arrayType(), + Map.of(e1, new Instant[] {Instant.ofEpochSecond(42), Instant.ofEpochSecond(43)})); + } + + @Test + void aTimestamp() { + checkKey(ATimestamp.getDescriptor(), List.of("ts"), Type.instantType(), Map.of( + ATimestamp.newBuilder().setTs(Timestamp.newBuilder().setSeconds(42).setNanos(43).build()) + .build(), + Instant.ofEpochSecond(42, 43))); + } + + @Test + void aDuration() { + checkKey(ADuration.getDescriptor(), List.of("dur"), Type.ofCustom(Duration.class), Map.of( + ADuration.newBuilder() + .setDur(com.google.protobuf.Duration.newBuilder().setSeconds(4200).setNanos(4300) + .build()) + .build(), + Duration.ofSeconds(4200, 4300))); + } + + @Test + void enum_() { + checkKey(AnEnum.getDescriptor(), List.of("fbb"), Type.ofCustom(EnumValueDescriptor.class), Map.of( + AnEnum.newBuilder().setFbb(TheEnum.FOO).build(), TheEnum.FOO.getValueDescriptor(), + AnEnum.newBuilder().setFbb(TheEnum.BAR).build(), TheEnum.BAR.getValueDescriptor(), + AnEnum.newBuilder().setFbb(TheEnum.BAZ).build(), TheEnum.BAZ.getValueDescriptor(), + AnEnum.newBuilder().setFbbValue(999).build(), + TheEnum.getDescriptor().findValueByNumberCreatingIfUnknown(999))); + } + + @Test + void optionalBasics() { + final Map, TypedFunction> nf = nf(OptionalBasics.getDescriptor()); + assertThat(nf.keySet()).containsExactly( + List.of("bool"), + List.of("int32"), + List.of("uint32"), + List.of("int64"), + List.of("uint64"), + List.of("float"), + List.of("double"), + List.of("string"), + List.of("bytes")); + + checkKey( + OptionalBasics.getDescriptor(), + List.of("bool"), + BoxedBooleanType.of(), + new HashMap<>() { + { + put(OptionalBasics.getDefaultInstance(), null); + put(OptionalBasics.newBuilder().setBool(false).build(), false); + put(OptionalBasics.newBuilder().setBool(true).build(), true); + } + }); + + checkKey( + OptionalBasics.getDescriptor(), + List.of("int32"), + BoxedIntType.of(), + new HashMap<>() { + { + put(OptionalBasics.getDefaultInstance(), null); + put(OptionalBasics.newBuilder().setInt32(0).build(), 0); + put(OptionalBasics.newBuilder().setInt32(42).build(), 42); + } + }); + + checkKey( + OptionalBasics.getDescriptor(), + List.of("uint32"), + BoxedIntType.of(), + new HashMap<>() { + { + put(OptionalBasics.getDefaultInstance(), null); + put(OptionalBasics.newBuilder().setUint32(0).build(), 0); + put(OptionalBasics.newBuilder().setUint32(42).build(), 42); + } + }); + + checkKey( + OptionalBasics.getDescriptor(), + List.of("int64"), + BoxedLongType.of(), + new HashMap<>() { + { + put(OptionalBasics.getDefaultInstance(), null); + put(OptionalBasics.newBuilder().setInt64(0).build(), 0L); + put(OptionalBasics.newBuilder().setInt64(42).build(), 42L); + } + }); + + checkKey( + OptionalBasics.getDescriptor(), + List.of("uint64"), + BoxedLongType.of(), + new HashMap<>() { + { + put(OptionalBasics.getDefaultInstance(), null); + put(OptionalBasics.newBuilder().setUint64(0).build(), 0L); + put(OptionalBasics.newBuilder().setUint64(42).build(), 42L); + } + }); + + checkKey( + OptionalBasics.getDescriptor(), + List.of("float"), + BoxedFloatType.of(), + new HashMap<>() { + { + put(OptionalBasics.getDefaultInstance(), null); + put(OptionalBasics.newBuilder().setFloat(0).build(), 0f); + put(OptionalBasics.newBuilder().setFloat(42).build(), 42f); + } + }); + + checkKey( + OptionalBasics.getDescriptor(), + List.of("double"), + BoxedDoubleType.of(), + new HashMap<>() { + { + put(OptionalBasics.getDefaultInstance(), null); + put(OptionalBasics.newBuilder().setDouble(0).build(), 0d); + put(OptionalBasics.newBuilder().setDouble(42).build(), 42d); + } + }); + + checkKey( + OptionalBasics.getDescriptor(), + List.of("string"), + Type.stringType(), + new HashMap<>() { + { + put(OptionalBasics.getDefaultInstance(), null); + put(OptionalBasics.newBuilder().setString("").build(), ""); + put(OptionalBasics.newBuilder().setString("hello").build(), "hello"); + } + }); + + checkKey( + OptionalBasics.getDescriptor(), + List.of("bytes"), + Type.byteType().arrayType(), + new HashMap<>() { + { + put(OptionalBasics.getDefaultInstance(), null); + put(OptionalBasics.newBuilder().setBytes(ByteString.EMPTY).build(), new byte[0]); + put(OptionalBasics.newBuilder().setBytes(ByteString.copyFromUtf8("hello")).build(), + "hello".getBytes(StandardCharsets.UTF_8)); + } + }); + } + + @Test + void optionalBasicsByteString() { + final FieldOptions asByteString = FieldOptions.builder().bytes(BytesBehavior.asByteString()).build(); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(x -> asByteString) + .build(); + checkKey( + OptionalBasics.getDescriptor(), + options, + List.of("bytes"), + CustomType.of(ByteString.class), + new HashMap<>() { + { + put(OptionalBasics.getDefaultInstance(), null); + put(OptionalBasics.newBuilder().setBytes(ByteString.EMPTY).build(), ByteString.EMPTY); + final ByteString hello = ByteString.copyFromUtf8("hello"); + put(OptionalBasics.newBuilder().setBytes(hello).build(), hello); + } + }); + } + + @Test + void wrappers() { + final Map, TypedFunction> nf = nf(TheWrappers.getDescriptor()); + assertThat(nf.keySet()).containsExactly( + List.of("bool"), + List.of("int32"), + List.of("uint32"), + List.of("int64"), + List.of("uint64"), + List.of("float"), + List.of("double"), + List.of("string"), + List.of("bytes")); + + TheWrappers allNull = TheWrappers.getDefaultInstance(); + checkKey(TheWrappers.getDescriptor(), List.of("bool"), BoxedBooleanType.of(), new HashMap<>() { + { + put(allNull, null); + put(TheWrappers.newBuilder().setBool(BoolValue.newBuilder().setValue(true).build()).build(), true); + } + }); + + checkKey(TheWrappers.getDescriptor(), List.of("int32"), Type.intType(), + new HashMap<>() { + { + put(allNull, QueryConstants.NULL_INT); + put(TheWrappers.newBuilder().setInt32(Int32Value.newBuilder().setValue(42).build()).build(), + 42); + } + }); + + checkKey(TheWrappers.getDescriptor(), List.of("uint32"), Type.intType(), + new HashMap<>() { + { + put(allNull, QueryConstants.NULL_INT); + put(TheWrappers.newBuilder().setUint32(UInt32Value.newBuilder().setValue(42).build()).build(), + 42); + } + }); + + checkKey(TheWrappers.getDescriptor(), List.of("int64"), Type.longType(), + new HashMap<>() { + { + put(allNull, QueryConstants.NULL_LONG); + put(TheWrappers.newBuilder().setInt64(Int64Value.newBuilder().setValue(42).build()).build(), + 42L); + } + }); + + checkKey(TheWrappers.getDescriptor(), List.of("uint64"), Type.longType(), + new HashMap<>() { + { + put(allNull, QueryConstants.NULL_LONG); + put(TheWrappers.newBuilder().setUint64(UInt64Value.newBuilder().setValue(42).build()).build(), + 42L); + } + }); + + checkKey(TheWrappers.getDescriptor(), List.of("float"), Type.floatType(), + new HashMap<>() { + { + put(allNull, QueryConstants.NULL_FLOAT); + put(TheWrappers.newBuilder().setFloat(FloatValue.newBuilder().setValue(42).build()).build(), + 42.0f); + } + }); + + checkKey(TheWrappers.getDescriptor(), List.of("double"), Type.doubleType(), + new HashMap<>() { + { + put(allNull, QueryConstants.NULL_DOUBLE); + put(TheWrappers.newBuilder().setDouble(DoubleValue.newBuilder().setValue(42).build()).build(), + 42.0d); + } + }); + + checkKey(TheWrappers.getDescriptor(), List.of("string"), Type.stringType(), new HashMap<>() { + { + put(allNull, null); + put(TheWrappers.newBuilder().setString(StringValue.newBuilder().setValue("foo").build()) + .build(), "foo"); + } + }); + + { + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(fp -> FieldOptions.builder().bytes(BytesBehavior.asByteString()).build()) + .build(); + checkKey(TheWrappers.getDescriptor(), options, List.of("bytes"), Type.ofCustom(ByteString.class), + new HashMap<>() { + { + put(allNull, null); + final ByteString foo = ByteString.copyFromUtf8("foo"); + put(TheWrappers.newBuilder().setBytes(BytesValue.newBuilder().setValue(foo).build()) + .build(), foo); + } + }); + } + } + + + + @Test + void repeated() { + final Map, TypedFunction> nf = + nf(RepeatedBasics.getDescriptor()); + assertThat(nf.keySet()).containsExactly( + List.of("bool"), + List.of("int32"), + List.of("uint32"), + List.of("int64"), + List.of("uint64"), + List.of("float"), + List.of("double"), + List.of("string"), + List.of("bytes")); + + final RepeatedBasics allEmpty = RepeatedBasics.getDefaultInstance(); + + checkKey(RepeatedBasics.getDescriptor(), List.of("bool"), Type.booleanType().arrayType(), new HashMap<>() { + { + put(allEmpty, new boolean[] {}); + put(RepeatedBasics.newBuilder().addBool(true).addBool(false).build(), new boolean[] {true, false}); + } + }); + + checkKey(RepeatedBasics.getDescriptor(), List.of("int32"), Type.intType().arrayType(), Map.of( + allEmpty, new int[] {}, + RepeatedBasics.newBuilder().addInt32(42).addInt32(43).build(), new int[] {42, 43})); + + checkKey(RepeatedBasics.getDescriptor(), List.of("uint32"), Type.intType().arrayType(), Map.of( + allEmpty, new int[] {}, + RepeatedBasics.newBuilder().addUint32(42).addUint32(43).build(), new int[] {42, 43})); + + checkKey(RepeatedBasics.getDescriptor(), List.of("int64"), Type.longType().arrayType(), Map.of( + allEmpty, new long[] {}, + RepeatedBasics.newBuilder().addInt64(42).addInt64(43).build(), new long[] {42, 43})); + + checkKey(RepeatedBasics.getDescriptor(), List.of("uint64"), Type.longType().arrayType(), Map.of( + allEmpty, new long[] {}, + RepeatedBasics.newBuilder().addUint64(42).addUint64(43).build(), new long[] {42, 43})); + + checkKey(RepeatedBasics.getDescriptor(), List.of("float"), Type.floatType().arrayType(), Map.of( + allEmpty, new float[] {}, + RepeatedBasics.newBuilder().addFloat(42).addFloat(43).build(), new float[] {42, 43})); + + checkKey(RepeatedBasics.getDescriptor(), List.of("double"), Type.doubleType().arrayType(), Map.of( + allEmpty, new double[] {}, + RepeatedBasics.newBuilder().addDouble(42).addDouble(43).build(), + new double[] {42, 43})); + + checkKey(RepeatedBasics.getDescriptor(), List.of("string"), Type.stringType().arrayType(), Map.of( + allEmpty, new String[] {}, + RepeatedBasics.newBuilder().addString("foo").addString("bar").build(), + new String[] {"foo", "bar"})); + + checkKey(RepeatedBasics.getDescriptor(), List.of("bytes"), Type.byteType().arrayType().arrayType(), Map.of( + allEmpty, new byte[][] {}, + RepeatedBasics.newBuilder().addBytes(ByteString.copyFromUtf8("hello")) + .addBytes(ByteString.copyFromUtf8("foo")).build(), + new byte[][] {"hello".getBytes(StandardCharsets.UTF_8), "foo".getBytes(StandardCharsets.UTF_8)})); + } + + @Test + void repeatedByteString() { + final FieldOptions asByteString = FieldOptions.builder().bytes(BytesBehavior.asByteString()).build(); + final RepeatedBasics allEmpty = RepeatedBasics.getDefaultInstance(); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(x -> asByteString) + .build(); + final ByteString hello = ByteString.copyFromUtf8("hello"); + final ByteString foo = ByteString.copyFromUtf8("foo"); + checkKey(RepeatedBasics.getDescriptor(), + options, + List.of("bytes"), + CustomType.of(ByteString.class).arrayType(), + Map.of( + allEmpty, new ByteString[] {}, + RepeatedBasics.newBuilder().addBytes(hello).addBytes(foo).build(), + new ByteString[] {hello, foo})); + } + + @Test + void repeatedWrappers() { + final Map, TypedFunction> nf = + nf(RepeatedWrappers.getDescriptor()); + assertThat(nf.keySet()).containsExactly( + List.of("bool"), + List.of("int32"), + List.of("uint32"), + List.of("int64"), + List.of("uint64"), + List.of("float"), + List.of("double"), + List.of("string"), + List.of("bytes")); + + final RepeatedWrappers allEmpty = RepeatedWrappers.getDefaultInstance(); + + checkKey(RepeatedWrappers.getDescriptor(), List.of("bool"), Type.booleanType().arrayType(), Map.of( + allEmpty, new boolean[] {}, + RepeatedWrappers.newBuilder() + .addBool(BoolValue.newBuilder().build()) + .addBool(BoolValue.newBuilder().setValue(false).build()) + .addBool(BoolValue.newBuilder().setValue(true).build()) + .build(), + new boolean[] {false, false, true})); + + checkKey(RepeatedWrappers.getDescriptor(), List.of("int32"), Type.intType().arrayType(), Map.of( + allEmpty, new int[] {}, + RepeatedWrappers.newBuilder().addInt32(Int32Value.of(42)).addInt32(Int32Value.of(43)).build(), + new int[] {42, 43})); + + checkKey(RepeatedWrappers.getDescriptor(), List.of("uint32"), Type.intType().arrayType(), Map.of( + allEmpty, new int[] {}, + RepeatedWrappers.newBuilder().addUint32(UInt32Value.of(42)).addUint32(UInt32Value.of(43)) + .build(), + new int[] {42, 43})); + + checkKey(RepeatedWrappers.getDescriptor(), List.of("int64"), Type.longType().arrayType(), Map.of( + allEmpty, new long[] {}, + RepeatedWrappers.newBuilder().addInt64(Int64Value.of(42)).addInt64(Int64Value.of(43)).build(), + new long[] {42, 43})); + + checkKey(RepeatedWrappers.getDescriptor(), List.of("uint64"), Type.longType().arrayType(), Map.of( + allEmpty, new long[] {}, + RepeatedWrappers.newBuilder().addUint64(UInt64Value.of(42)).addUint64(UInt64Value.of(43)) + .build(), + new long[] {42, 43})); + + checkKey(RepeatedWrappers.getDescriptor(), List.of("float"), Type.floatType().arrayType(), Map.of( + allEmpty, new float[] {}, + RepeatedWrappers.newBuilder().addFloat(FloatValue.of(42)).addFloat(FloatValue.of(43)).build(), + new float[] {42, 43})); + + checkKey(RepeatedWrappers.getDescriptor(), List.of("double"), Type.doubleType().arrayType(), Map.of( + allEmpty, new double[] {}, + RepeatedWrappers.newBuilder().addDouble(DoubleValue.of(42)).addDouble(DoubleValue.of(43)) + .build(), + new double[] {42, 43})); + + checkKey(RepeatedWrappers.getDescriptor(), List.of("string"), Type.stringType().arrayType(), Map.of( + allEmpty, new String[] {}, + RepeatedWrappers.newBuilder().addString(StringValue.of("foo")).addString(StringValue.of("bar")).build(), + new String[] {"foo", "bar"})); + + checkKey(RepeatedWrappers.getDescriptor(), List.of("bytes"), Type.byteType().arrayType().arrayType(), Map.of( + allEmpty, new byte[][] {}, + RepeatedWrappers.newBuilder().addBytes(BytesValue.of(ByteString.copyFromUtf8("hello"))) + .addBytes(BytesValue.of(ByteString.copyFromUtf8("foo"))).build(), + new byte[][] {"hello".getBytes(StandardCharsets.UTF_8), "foo".getBytes(StandardCharsets.UTF_8)})); + } + + @Test + void multiRepeated() { + final Map, TypedFunction> nf = nf(MultiRepeated.getDescriptor()); + assertThat(nf.keySet()).containsExactly( + List.of("my_basics", "bool"), + List.of("my_basics", "int32"), + List.of("my_basics", "uint32"), + List.of("my_basics", "int64"), + List.of("my_basics", "uint64"), + List.of("my_basics", "float"), + List.of("my_basics", "double"), + List.of("my_basics", "string"), + List.of("my_basics", "bytes"), + List.of("my_wrappers", "bool"), + List.of("my_wrappers", "int32"), + List.of("my_wrappers", "uint32"), + List.of("my_wrappers", "int64"), + List.of("my_wrappers", "uint64"), + List.of("my_wrappers", "float"), + List.of("my_wrappers", "double"), + List.of("my_wrappers", "string"), + List.of("my_wrappers", "bytes"), + List.of("my_objects", "xyz", "x"), + List.of("my_objects", "xyz", "y"), + List.of("my_objects", "xyz", "z")); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_basics", "bool"), + Type.booleanType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new boolean[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_basics", "int32"), + Type.intType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new int[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_basics", "uint32"), + Type.intType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new int[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_basics", "int64"), + Type.longType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new long[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_basics", "uint64"), + Type.longType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new long[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_basics", "float"), + Type.floatType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new float[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_basics", "double"), + Type.doubleType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new double[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_wrappers", "bool"), + Type.booleanType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new boolean[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_wrappers", "int32"), + Type.intType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new int[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_wrappers", "uint32"), + Type.intType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new int[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_wrappers", "int64"), + Type.longType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new long[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_wrappers", "uint64"), + Type.longType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new long[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_wrappers", "float"), + Type.floatType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new float[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_wrappers", "double"), + Type.doubleType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new double[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_objects", "xyz", "x"), + Type.intType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new int[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_objects", "xyz", "y"), + Type.stringType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new String[][] {})); + + checkKey( + MultiRepeated.getDescriptor(), + List.of("my_objects", "xyz", "z"), + Type.instantType().arrayType().arrayType(), + Map.of(MultiRepeated.getDefaultInstance(), new Instant[][] {})); + } + + @Test + void repeatedTimestamp() { + checkKey( + RepeatedTimestamp.getDescriptor(), + List.of("ts"), + Type.instantType().arrayType(), + Map.of( + RepeatedTimestamp.getDefaultInstance(), new Instant[] {}, + RepeatedTimestamp.newBuilder() + .addTs(Timestamp.newBuilder().setSeconds(1).setNanos(2).build()) + .addTs(Timestamp.newBuilder().setSeconds(3).setNanos(4).build()) + .build(), + new Instant[] { + Instant.ofEpochSecond(1, 2), + Instant.ofEpochSecond(3, 4)})); + } + + @Test + void repeatedDuration() { + checkKey(RepeatedDuration.getDescriptor(), List.of("dur"), Type.ofCustom(Duration.class).arrayType(), Map.of( + RepeatedDuration.getDefaultInstance(), new Duration[] {}, + RepeatedDuration.newBuilder() + .addDur(com.google.protobuf.Duration.newBuilder().setSeconds(1).setNanos(2).build()) + .addDur(com.google.protobuf.Duration.newBuilder().setSeconds(3).setNanos(4).build()) + .build(), + new Duration[] { + Duration.ofSeconds(1, 2), + Duration.ofSeconds(3, 4)})); + } + + @Test + void nested() { + final Map, TypedFunction> nf = + nf(ANested.getDescriptor()); + assertThat(nf.keySet()).containsExactly(List.of("baz", "foo"), List.of("baz", "bar")); + + checkKey( + ANested.getDescriptor(), + List.of("baz", "foo"), + Type.intType(), + new HashMap<>() { + { + put(ANested.getDefaultInstance(), QueryConstants.NULL_INT); + put(ANested.newBuilder().setBaz(SubMessage.newBuilder().setFoo(42).build()).build(), 42); + } + }); + + checkKey( + ANested.getDescriptor(), + List.of("baz", "bar"), + Type.longType(), + new HashMap<>() { + { + put(ANested.getDefaultInstance(), QueryConstants.NULL_LONG); + put(ANested.newBuilder().setBaz(SubMessage.newBuilder().setBar(42L).build()).build(), 42L); + } + }); + } + + @Test + void multiNested() { + final Map, TypedFunction> nf = + nf(AMultiNested.getDescriptor()); + assertThat(nf.keySet()).containsExactly( + List.of("hello", "foo"), + List.of("hello", "bar"), + List.of("hello", "baz", "world"), + List.of("hello", "baz", "world2"), + List.of("hello", "baz", "world3")); + + final AMultiNested defaultInstance = AMultiNested.getDefaultInstance(); + final AMultiNested noBaz = AMultiNested.newBuilder() + .setHello(SubMessage1.newBuilder().setFoo(42).setBar(43).build()) + .build(); + final AMultiNested bazDefault = AMultiNested.newBuilder() + .setHello(SubMessage1.newBuilder().setBaz(SubMessage2.getDefaultInstance()).build()) + .build(); + final AMultiNested bazWorld = AMultiNested.newBuilder() + .setHello(SubMessage1.newBuilder().setBaz(SubMessage2.newBuilder() + .setWorld("OK") + .setWorld2(StringValue.newBuilder().setValue("OK2")) + .setWorld3(DoubleValue.newBuilder().setValue(42.0d)).build()) + .build()) + .build(); + + checkKey( + AMultiNested.getDescriptor(), + List.of("hello", "foo"), + Type.intType(), + new HashMap<>() { + { + put(defaultInstance, QueryConstants.NULL_INT); + put(noBaz, 42); + put(bazDefault, 0); + put(bazWorld, 0); + } + }); + + checkKey( + AMultiNested.getDescriptor(), + List.of("hello", "bar"), + Type.longType(), + new HashMap<>() { + { + put(defaultInstance, QueryConstants.NULL_LONG); + put(noBaz, 43L); + put(bazDefault, 0L); + put(bazWorld, 0L); + } + }); + + checkKey( + AMultiNested.getDescriptor(), + List.of("hello", "baz", "world"), + Type.stringType(), + new HashMap<>() { + { + put(defaultInstance, null); + put(noBaz, null); + put(bazDefault, ""); + put(bazWorld, "OK"); + } + }); + + checkKey( + AMultiNested.getDescriptor(), + List.of("hello", "baz", "world2"), + Type.stringType(), + new HashMap<>() { + { + put(defaultInstance, null); + put(noBaz, null); + put(bazDefault, null); + put(bazWorld, "OK2"); + } + }); + + checkKey( + AMultiNested.getDescriptor(), + List.of("hello", "baz", "world3"), + Type.doubleType(), + new HashMap<>() { + { + put(defaultInstance, QueryConstants.NULL_DOUBLE); + put(noBaz, QueryConstants.NULL_DOUBLE); + put(bazDefault, QueryConstants.NULL_DOUBLE); + put(bazWorld, 42.0d); + } + }); + } + + @Test + void repeatedPerson() { + final MessageParser personParser = new MessageParserSingle() { + @Override + public Descriptor canonicalDescriptor() { + return Person.getDescriptor(); + } + + @Override + public ToObjectFunction messageParser(Descriptor descriptor, + ProtobufDescriptorParserOptions options, FieldPath fieldPath) { + return ToObjectFunction.identity(Type.ofCustom(Person.class)); + } + }; + final ProtobufDescriptorParserOptions options = + ProtobufDescriptorParserOptions.builder().parsers(List.of(personParser)).build(); + final Map, TypedFunction> nf = nf(RepeatedMessage.getDescriptor(), options); + assertThat(nf.keySet()).containsExactly(List.of("persons")); + + final Person p1 = Person.newBuilder().setFirstName("First").setLastName("Last").build(); + final Person p2 = Person.newBuilder().setFirstName("Foo").setLastName("Bar").build(); + checkKey( + RepeatedMessage.getDescriptor(), + options, + List.of("persons"), + Type.ofCustom(Person.class).arrayType(), + Map.of( + RepeatedMessage.getDefaultInstance(), new Person[] {}, + RepeatedMessage.newBuilder().addPersons(p1).addPersons(p2).build(), new Person[] {p1, p2})); + } + + @Test + void customMultiParser() { + final MessageParser parser = new MessageParser() { + @Override + public Descriptor canonicalDescriptor() { + return Timestamp.getDescriptor(); + } + + @Override + public ProtobufFunctions messageParsers(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + final ToObjectFunction ts = + ToObjectFunction.identity(Type.ofCustom(Timestamp.class)); + // This is a dumb thing to do; but it shows that we can derive two functions from the same field path + final ToLongFunction seconds = ts.mapToLong(Timestamp::getSeconds); + final ToLongFunction secondsX2 = ts.mapToLong(x -> x.getSeconds() * 2); + final FieldPath secondsPath = + FieldPath.of(descriptor.findFieldByNumber(Timestamp.SECONDS_FIELD_NUMBER)); + return ProtobufFunctions.builder() + .addFunctions( + ProtobufFunction.of(secondsPath, seconds), + ProtobufFunction.of(secondsPath, secondsX2)) + .build(); + } + }; + + final ProtobufDescriptorParserOptions options = + ProtobufDescriptorParserOptions.builder().parsers(List.of(parser)).build(); + + final ProtobufFunctions pf = ProtobufDescriptorParser.parse(Timestamp.getDescriptor(), options); + final List functions = pf.functions(); + assertThat(functions).hasSize(2); + + assertThat(functions.get(0).path().namePath()).containsExactly("seconds"); + assertThat(functions.get(1).path().namePath()).containsExactly("seconds"); + + final ToLongFunction f0 = (ToLongFunction) functions.get(0).function(); + final ToLongFunction f1 = (ToLongFunction) functions.get(1).function(); + + final Timestamp aTs = Timestamp.newBuilder().setSeconds(42).build(); + assertThat(f0.applyAsLong(aTs)).isEqualTo(42); + assertThat(f1.applyAsLong(aTs)).isEqualTo(84); + + assertThat(f0.applyAsLong(Timestamp.getDefaultInstance())).isEqualTo(0); + assertThat(f1.applyAsLong(Timestamp.getDefaultInstance())).isEqualTo(0); + } + + @Test + void customMultiParserNested() { + final MessageParser parser = new MessageParser() { + @Override + public Descriptor canonicalDescriptor() { + return Timestamp.getDescriptor(); + } + + @Override + public ProtobufFunctions messageParsers(Descriptor descriptor, ProtobufDescriptorParserOptions options, + FieldPath fieldPath) { + final ToObjectFunction ts = + ToObjectFunction.identity(Type.ofCustom(Timestamp.class)); + // This is a dumb thing to do; but it shows that we can derive two functions from the same field path + final ToLongFunction seconds = ts.mapToLong(Timestamp::getSeconds); + final ToLongFunction secondsX2 = ts.mapToLong(x -> x.getSeconds() * 2); + final FieldPath secondsPath = + FieldPath.of(descriptor.findFieldByNumber(Timestamp.SECONDS_FIELD_NUMBER)); + return ProtobufFunctions.builder() + .addFunctions( + ProtobufFunction.of(secondsPath, seconds), + ProtobufFunction.of(secondsPath, secondsX2)) + .build(); + } + }; + + final ProtobufDescriptorParserOptions options = + ProtobufDescriptorParserOptions.builder().parsers(List.of(parser)).build(); + + final ProtobufFunctions pf = ProtobufDescriptorParser.parse(ATimestamp.getDescriptor(), options); + final List functions = pf.functions(); + assertThat(functions).hasSize(2); + + assertThat(functions.get(0).path().namePath()).containsExactly("ts", "seconds"); + assertThat(functions.get(1).path().namePath()).containsExactly("ts", "seconds"); + + final ToLongFunction f0 = (ToLongFunction) functions.get(0).function(); + final ToLongFunction f1 = (ToLongFunction) functions.get(1).function(); + + final ATimestamp aTs = ATimestamp.newBuilder().setTs(Timestamp.newBuilder().setSeconds(42).build()).build(); + + assertThat(f0.applyAsLong(aTs)).isEqualTo(42); + assertThat(f1.applyAsLong(aTs)).isEqualTo(84); + + assertThat(f0.applyAsLong(ATimestamp.getDefaultInstance())).isEqualTo(QueryConstants.NULL_LONG); + assertThat(f1.applyAsLong(ATimestamp.getDefaultInstance())).isEqualTo(QueryConstants.NULL_LONG); + } + + // This is a potential improvement in parsing we might want in the future + @Test + void repeatedMessageDestructured() { + final Map, TypedFunction> nf = nf( + RepeatedMessage.getDescriptor()); + assertThat(nf.keySet()).containsExactly(List.of("persons", "first_name"), List.of("persons", "last_name")); + + final Person p1 = Person.newBuilder().setFirstName("First").setLastName("Last").build(); + final Person p2 = Person.newBuilder().setFirstName("Foo").setLastName("Bar").build(); + + checkKey( + RepeatedMessage.getDescriptor(), + List.of("persons", "first_name"), + Type.stringType().arrayType(), + Map.of( + RepeatedMessage.getDefaultInstance(), new String[] {}, + RepeatedMessage.newBuilder().addPersons(p1).addPersons(p2).build(), + new String[] {"First", "Foo"})); + + checkKey( + RepeatedMessage.getDescriptor(), + List.of("persons", "last_name"), + Type.stringType().arrayType(), + Map.of( + RepeatedMessage.getDefaultInstance(), new String[] {}, + RepeatedMessage.newBuilder().addPersons(p1).addPersons(p2).build(), + new String[] {"Last", "Bar"})); + } + + @Test + void nestedRepeatedTimestamps() { + final Map, TypedFunction> nf = nf(NestedRepeatedTimestamps.getDescriptor()); + assertThat(nf.keySet()).containsExactly(List.of("stamps", "ts")); + checkKey( + NestedRepeatedTimestamps.getDescriptor(), + List.of("stamps", "ts"), + Type.instantType().arrayType().arrayType(), + Map.of( + NestedRepeatedTimestamps.getDefaultInstance(), new Instant[][] {}, + NestedRepeatedTimestamps.newBuilder().addStamps(Timestamps.getDefaultInstance()).build(), + new Instant[][] {new Instant[] {}}, + NestedRepeatedTimestamps.newBuilder() + .addStamps(Timestamps.newBuilder().addTs(Timestamp.getDefaultInstance()).build()) + .build(), + new Instant[][] {new Instant[] {Instant.ofEpochMilli(0)}})); + + } + + @Test + void byteWrapper() { + final Map, TypedFunction> nf = nf(ByteWrapper.getDescriptor()); + assertThat(nf.keySet()).containsExactly(List.of()); + checkKey( + ByteWrapper.getDescriptor(), + List.of(), + Type.byteType(), + Map.of( + ByteWrapper.getDefaultInstance(), (byte) 0, + ByteWrapper.newBuilder().setValue(42).build(), (byte) 42)); + } + + @Test + void byteWrapperNested() { + final Map, TypedFunction> nf = nf(NestedByteWrapper.getDescriptor()); + assertThat(nf.keySet()).containsExactly(List.of("my_byte")); + checkKey( + NestedByteWrapper.getDescriptor(), + List.of("my_byte"), + Type.byteType(), + new HashMap<>() { + { + put(NestedByteWrapper.getDefaultInstance(), QueryConstants.NULL_BYTE); + put(NestedByteWrapper.newBuilder().setMyByte(ByteWrapper.getDefaultInstance()).build(), + (byte) 0); + put(NestedByteWrapper.newBuilder().setMyByte(ByteWrapper.newBuilder().setValue(42)).build(), + (byte) 42); + } + }); + } + + @Test + void repeatedByteWrapper() { + final Map, TypedFunction> nf = nf(ByteWrapperRepeated.getDescriptor()); + assertThat(nf.keySet()).containsExactly(List.of("my_bytes")); + checkKey( + ByteWrapperRepeated.getDescriptor(), + List.of("my_bytes"), + Type.byteType().arrayType(), + Map.of( + ByteWrapperRepeated.getDefaultInstance(), new byte[0], + ByteWrapperRepeated.newBuilder().addMyBytes(ByteWrapper.newBuilder().setValue(42).build()) + .build(), + new byte[] {(byte) 42})); + } + + @Test + void any() { + final Map, TypedFunction> nf = nf(AnyWrapper.getDescriptor()); + assertThat(nf.keySet()).containsExactly(List.of("my_any")); + checkKey( + AnyWrapper.getDescriptor(), + List.of("my_any"), + Type.ofCustom(Any.class), + new HashMap<>() { + { + put(AnyWrapper.getDefaultInstance(), null); + put(AnyWrapper.newBuilder().setMyAny(Any.getDefaultInstance()).build(), + Any.getDefaultInstance()); + final Any someAny = Any.newBuilder().setTypeUrl("some-url").build(); + put(AnyWrapper.newBuilder().setMyAny(someAny).build(), someAny); + } + }); + } + + @Test + void fieldMask() { + final Map, TypedFunction> nf = nf(FieldMaskWrapper.getDescriptor()); + assertThat(nf.keySet()).containsExactly(List.of("my_field_mask")); + checkKey( + FieldMaskWrapper.getDescriptor(), + List.of("my_field_mask"), + Type.ofCustom(FieldMask.class), + new HashMap<>() { + { + put(FieldMaskWrapper.getDefaultInstance(), null); + put(FieldMaskWrapper.newBuilder() + .setMyFieldMask(FieldMask.getDefaultInstance()).build(), + FieldMask.getDefaultInstance()); + final FieldMask someFieldMask = FieldMask.newBuilder().addPaths("foo").build(); + put(FieldMaskWrapper.newBuilder().setMyFieldMask(someFieldMask).build(), someFieldMask); + } + }); + } + + @Test + void includeNamePaths() { + final List path = List.of("float"); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(includeIf(fp -> fp.namePath().equals(path))) + .build(); + final Map, TypedFunction> nf = nf(UnionType.getDescriptor(), options); + assertThat(nf.keySet()).containsExactly(path); + } + + @Test + void includeNumberPaths() { + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(includeIf(fp -> fp.numberPath().equals(FieldNumberPath.of(6)))) + .build(); + final Map, TypedFunction> nf = nf(UnionType.getDescriptor(), options); + assertThat(nf.keySet()).containsExactly(List.of("float")); + } + + @Test + void excludeNamePaths() { + final List path = List.of("float"); + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(includeIf(fp -> !fp.namePath().equals(path))) + .build(); + final Map, TypedFunction> nf = nf(UnionType.getDescriptor(), options); + assertThat(nf).hasSize(8); + assertThat(nf.keySet()).doesNotContain(path); + } + + @Test + void excludeNumberPaths() { + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(includeIf(fp -> !fp.numberPath().equals(FieldNumberPath.of(6)))) + .build(); + final Map, TypedFunction> nf = nf(UnionType.getDescriptor(), options); + assertThat(nf).hasSize(8); + assertThat(nf.keySet()).doesNotContain(List.of("float")); + } + + @Test + void timestampNoParsers() { + final ProtobufDescriptorParserOptions options = + ProtobufDescriptorParserOptions.builder().parsers(List.of()).build(); + final Map, TypedFunction> nf = nf(Timestamp.getDescriptor(), options); + assertThat(nf.keySet()).containsExactly( + List.of("seconds"), + List.of("nanos")); + } + + @Test + void twoTimestampsOneAsWellKnown() { + // only treat t1 as well-known + final ProtobufDescriptorParserOptions options = ProtobufDescriptorParserOptions.builder() + .fieldOptions(fieldPath -> FieldOptions.builder() + .wellKnown(fieldPath.namePath().equals(List.of("ts1")) ? WellKnownBehavior.asWellKnown() + : WellKnownBehavior.asRecursive()) + .build()) + .build(); + final Map, TypedFunction> nf = nf(TwoTs.getDescriptor(), options); + assertThat(nf.keySet()).containsExactly(List.of("ts1"), List.of("ts2", "seconds"), List.of("ts2", "nanos")); + } + + private static Map, TypedFunction> nf(Descriptor descriptor) { + return nf(descriptor, ProtobufDescriptorParserOptions.defaults()); + } + + private static Map, TypedFunction> nf(Descriptor descriptor, + ProtobufDescriptorParserOptions options) { + final ProtobufFunctions results = ProtobufDescriptorParser.parse(descriptor, options); + final Map, TypedFunction> out = new LinkedHashMap<>(results.functions().size()); + for (ProtobufFunction function : results.functions()) { + out.put(function.path().namePath(), function.function()); + } + return out; + } + + private static void checkKey( + Descriptor descriptor, + List expectedPath, + Type expectedType, + Map expectedExamples) { + checkKey(descriptor, ProtobufDescriptorParserOptions.defaults(), expectedPath, expectedType, expectedExamples); + } + + private static void checkKey( + Descriptor descriptor, + ProtobufDescriptorParserOptions options, + List expectedPath, + Type expectedType, + Map expectedExamples) { + final Map, TypedFunction> map = nf(descriptor, options); + assertThat(map).containsKey(expectedPath); + assertThat(map) + .extractingByKey(expectedPath) + .extracting(TypedFunction::returnType) + .isEqualTo(expectedType); + for (Entry e : expectedExamples.entrySet()) { + assertThat(map) + .extractingByKey(expectedPath) + .extracting(t -> Box.apply(t, e.getKey())) + .isEqualTo(e.getValue()); + } + } + + private static Function includeIf(Predicate include) { + return fp -> FieldOptions.builder().include(include.test(fp)).build(); + } +} diff --git a/extensions/protobuf/src/test/proto/mytest.proto b/extensions/protobuf/src/test/proto/mytest.proto new file mode 100644 index 00000000000..da6a6e169db --- /dev/null +++ b/extensions/protobuf/src/test/proto/mytest.proto @@ -0,0 +1,222 @@ +/* + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +syntax = "proto3"; + +import "google/protobuf/any.proto"; +import "google/protobuf/field_mask.proto"; +import "google/protobuf/timestamp.proto"; +import "google/protobuf/duration.proto"; +import "google/protobuf/wrappers.proto"; + +package io.deephaven.protobuf.test; + +option java_multiple_files = true; +option optimize_for = SPEED; + +message UnionType { + oneof type { + bool bool = 1; + int32 int32 = 2; + uint32 uint32 = 3; + int64 int64 = 4; + uint64 uint64 = 5; + float float = 6; + double double = 7; + string string = 8; + bytes bytes = 9; + } +} + +message AStringStringMap { + map properties = 1; +} + +message AnIntIntMap { + map properties = 1; +} + +message ALongTimestampMap { + map properties = 1; +} + +message ALongNestedTimestampMap { + map properties = 1; +} + +message AnIntFooBarMap { + message FooBar { + string foo = 1; + int64 bar = 2; + } + map properties = 1; +} + +message ATimestamp { + google.protobuf.Timestamp ts = 1; +} + +message TwoTs { + google.protobuf.Timestamp ts1 = 1; + google.protobuf.Timestamp ts2 = 2; +} + +message ADuration { + google.protobuf.Duration dur = 1; +} + +message AnEnum { + enum TheEnum { + FOO = 0; + BAR = 1; + BAZ = 2; + } + TheEnum fbb = 1; +} + +message OptionalBasics { + optional bool bool = 1; + optional int32 int32 = 2; + optional uint32 uint32 = 3; + optional int64 int64 = 4; + optional uint64 uint64 = 5; + optional float float = 6; + optional double double = 7; + optional string string = 8; + optional bytes bytes = 9; +} + +message TheWrappers { + google.protobuf.BoolValue bool = 1; + google.protobuf.Int32Value int32 = 2; + google.protobuf.UInt32Value uint32 = 3; + google.protobuf.Int64Value int64 = 4; + google.protobuf.UInt64Value uint64 = 5; + google.protobuf.FloatValue float = 6; + google.protobuf.DoubleValue double = 7; + google.protobuf.StringValue string = 8; + google.protobuf.BytesValue bytes = 9; +} + +message RepeatedBasics { + repeated bool bool = 1; + repeated int32 int32 = 2; + repeated uint32 uint32 = 3; + repeated int64 int64 = 4; + repeated uint64 uint64 = 5; + repeated float float = 6; + repeated double double = 7; + repeated string string = 8; + repeated bytes bytes = 9; +} + +message RepeatedWrappers { + repeated google.protobuf.BoolValue bool = 1; + repeated google.protobuf.Int32Value int32 = 2; + repeated google.protobuf.UInt32Value uint32 = 3; + repeated google.protobuf.Int64Value int64 = 4; + repeated google.protobuf.UInt64Value uint64 = 5; + repeated google.protobuf.FloatValue float = 6; + repeated google.protobuf.DoubleValue double = 7; + repeated google.protobuf.StringValue string = 8; + repeated google.protobuf.BytesValue bytes = 9; +} + +message RepeatedObject { + message XYZ { + int32 x = 1; + string y = 2; + google.protobuf.Timestamp z = 3; + } + repeated XYZ xyz = 1; +} + +message MultiRepeated { + repeated RepeatedBasics my_basics = 1; + repeated RepeatedWrappers my_wrappers = 2; + repeated RepeatedObject my_objects = 3; +} + +message RepeatedTimestamp { + repeated google.protobuf.Timestamp ts = 1; +} + +message RepeatedDuration { + repeated google.protobuf.Duration dur = 1; +} + +message ANested { + message SubMessage { + int32 foo = 1; + int64 bar = 2; + } + + SubMessage baz = 1; +} + +message AMultiNested { + message SubMessage1 { + message SubMessage2 { + string world = 1; + google.protobuf.StringValue world2 = 2; + google.protobuf.DoubleValue world3 = 3; + } + int32 foo = 1; + int64 bar = 2; + SubMessage2 baz = 3; + } + SubMessage1 hello = 1; +} + +message RepeatedMessage { + message Person { + string first_name = 1; + string last_name = 2; + } + repeated Person persons = 1; +} + +message NestedRepeatedTimestamps { + message Timestamps { + repeated google.protobuf.Timestamp ts = 1; + } + repeated Timestamps stamps = 1; +} + +// tread this type as a byte +message ByteWrapper { + int32 value = 1; +} + +message NestedByteWrapper { + // this should be treated as byte + ByteWrapper my_byte = 1; +} + +message ByteWrapperRepeated { + // this should be treated as byte[] + repeated ByteWrapper my_bytes = 1; +} + +message AnyWrapper { + google.protobuf.Any my_any = 1; +} + +message FieldMaskWrapper { + google.protobuf.FieldMask my_field_mask = 1; +} + +message FieldPathTesting { + message Baz { + int32 zip = 1; + } + message Bar { + Baz baz = 1; + } + message Foo { + Bar bar = 1; + int64 zap = 2; + string zoom = 3; + } + Foo foo = 1; +} diff --git a/py/server/deephaven/stream/kafka/consumer.py b/py/server/deephaven/stream/kafka/consumer.py index 0842bc71382..0c84beecc27 100644 --- a/py/server/deephaven/stream/kafka/consumer.py +++ b/py/server/deephaven/stream/kafka/consumer.py @@ -3,21 +3,24 @@ # """ The kafka.consumer module supports consuming a Kakfa topic as a Deephaven live table. """ -from typing import Dict, Tuple, List, Callable, Union -from warnings import warn - import jpy +from typing import Dict, Tuple, List, Callable, Union, Optional +from warnings import warn from deephaven import dtypes -from deephaven.jcompat import j_hashmap, j_properties from deephaven._wrapper import JObjectWrapper from deephaven.column import Column from deephaven.dherror import DHError from deephaven.dtypes import DType +from deephaven.jcompat import j_hashmap, j_properties, j_array_list from deephaven.table import Table, PartitionedTable _JKafkaTools = jpy.get_type("io.deephaven.kafka.KafkaTools") _JKafkaTools_Consume = jpy.get_type("io.deephaven.kafka.KafkaTools$Consume") +_JProtobufConsumeOptions = jpy.get_type("io.deephaven.kafka.protobuf.ProtobufConsumeOptions") +_JProtobufDescriptorParserOptions = jpy.get_type("io.deephaven.protobuf.ProtobufDescriptorParserOptions") +_JFieldOptions = jpy.get_type("io.deephaven.protobuf.FieldOptions") +_JFieldPath = jpy.get_type("io.deephaven.protobuf.FieldPath") _JPythonTools = jpy.get_type("io.deephaven.integrations.python.PythonTools") ALL_PARTITIONS = _JKafkaTools.ALL_PARTITIONS @@ -281,6 +284,57 @@ def _consume( raise DHError(e, "failed to consume a Kafka stream.") from e +def protobuf_spec( + schema: str, + schema_version: Optional[int] = None, + schema_message_name: Optional[str] = None, + include: Optional[List[str]] = None, +) -> KeyValueSpec: + """Creates a spec for how to use Kafka data when consuming a Kafka stream to a Deephaven table. This will fetch the + protobuf descriptor for the schema subject from the schema registry using version schema_version and create protobuf + message parsing functions according to parsing options. These functions will be adapted to handle schema changes. + + Args: + schema (str): the schema subject name + schema_version (Optional[int]): the schema version, or None for latest, default is None. For purposes of + reproducibility across restarts where schema changes may occur, it is advisable for callers to set this. + This will ensure the resulting table definition will not change across restarts. This gives the caller an + explicit opportunity to update any downstream consumers when updating schema_version if necessary. + schema_message_name (Optional[str]): the fully-qualified protobuf message name, for example + "com.example.MyMessage". This message's descriptor will be used as the basis for the resulting table's + definition. If None, the first message descriptor in the protobuf schema will be used. The default is None. + It is advisable for callers to explicitly set this. + include (Optional[List[str]]): the '/' separated paths to include. The final path may be a '*' to additionally + match everything that starts with path. For example, include=["/foo/bar"] will include the field path + name paths [], ["foo"], and ["foo", "bar"]. include=["/foo/bar/*"] will additionally include any field path + name paths that start with ["foo", "bar"]: ["foo", "bar", "baz"], ["foo", "bar", "baz", "zap"], etc. When + multiple includes are specified, the fields will be included when any of the components matches. Default is + None, which includes all paths. + + Returns: + a KeyValueSpec + """ + parser_options_builder = _JProtobufDescriptorParserOptions.builder() + if include is not None: + parser_options_builder.fieldOptions( + _JFieldOptions.includeIf( + _JFieldPath.anyMatches(j_array_list(include)) + ) + ) + pb_consume_builder = ( + _JProtobufConsumeOptions.builder() + .schemaSubject(schema) + .parserOptions(parser_options_builder.build()) + ) + if schema_version: + pb_consume_builder.schemaVersion(schema_version) + if schema_message_name: + pb_consume_builder.schemaMessageName(schema_message_name) + return KeyValueSpec( + j_spec=_JKafkaTools_Consume.protobufSpec(pb_consume_builder.build()) + ) + + def avro_spec( schema: str, schema_version: str = "latest", diff --git a/py/server/tests/test_kafka_consumer.py b/py/server/tests/test_kafka_consumer.py index 0ea15dfe4e0..2122030140a 100644 --- a/py/server/tests/test_kafka_consumer.py +++ b/py/server/tests/test_kafka_consumer.py @@ -89,6 +89,113 @@ def test_json_spec(self): self.assertEqual("Tstamp", cols[7].name) self.assertEqual(dtypes.Instant, cols[7].data_type) + def test_protobuf_spec(self): + """ + Check an Protobuf Kafka subscription creates the right table. + """ + schema = """syntax = "proto3"; +import "google/protobuf/timestamp.proto"; + +package io.deephaven.example; + +message Sub { + string first = 1; + string last = 2; +} + +message SomeMessage { + google.protobuf.Timestamp ts = 1; + string name = 2; + int32 foo = 3; + double bar = 4; + Sub sub = 5; +} +""" + schema_normalized = schema.replace("\n", " ").replace('"', '\\"') + schema_str = '{ "schemaType": "PROTOBUF", "schema" : "%s" }' % schema_normalized + sys_str = f""" +curl -X POST \ + -H 'Content-type: application/vnd.schemaregistry.v1+json' \ + --data-binary '{schema_str}' \ + http://redpanda:8081/subjects/io%2Fdeephaven%2Fexample%2FMySchema.proto/versions +""" + + r = os.system(sys_str) + self.assertEqual(0, r) + + def consume(value_spec): + return ck.consume( + { + "bootstrap.servers": "redpanda:29092", + "schema.registry.url": "http://redpanda:8081", + }, + "my_pb_topic", + key_spec=KeyValueSpec.IGNORE, + value_spec=value_spec, + table_type=TableType.append(), + ) + + with self.subTest(msg="regular"): + t = consume( + ck.protobuf_spec( + "io/deephaven/example/MySchema.proto", + schema_version=1, + schema_message_name="io.deephaven.example.SomeMessage", + ) + ) + cols = t.columns + self.assertEqual(9, len(cols)) + self._assert_common_cols(cols) + self.assertEqual("ts", cols[3].name) + self.assertEqual(dtypes.Instant, cols[3].data_type) + self.assertEqual("name", cols[4].name) + self.assertEqual(dtypes.string, cols[4].data_type) + self.assertEqual("foo", cols[5].name) + self.assertEqual(dtypes.int32, cols[5].data_type) + self.assertEqual("bar", cols[6].name) + self.assertEqual(dtypes.double, cols[6].data_type) + self.assertEqual("sub_first", cols[7].name) + self.assertEqual(dtypes.string, cols[7].data_type) + self.assertEqual("sub_last", cols[8].name) + self.assertEqual(dtypes.string, cols[8].data_type) + + with self.subTest(msg="include /foo /bar"): + t = consume( + ck.protobuf_spec( + "io/deephaven/example/MySchema.proto", + schema_version=1, + schema_message_name="io.deephaven.example.SomeMessage", + include=["/foo", "/bar"], + ) + ) + cols = t.columns + self.assertEqual(5, len(cols)) + self._assert_common_cols(cols) + self.assertEqual("foo", cols[3].name) + self.assertEqual(dtypes.int32, cols[3].data_type) + self.assertEqual("bar", cols[4].name) + self.assertEqual(dtypes.double, cols[4].data_type) + + + with self.subTest(msg="include /ts /sub/*"): + t = consume( + ck.protobuf_spec( + "io/deephaven/example/MySchema.proto", + schema_version=1, + schema_message_name="io.deephaven.example.SomeMessage", + include=["/ts", "/sub/*"], + ) + ) + cols = t.columns + self.assertEqual(6, len(cols)) + self._assert_common_cols(cols) + self.assertEqual("ts", cols[3].name) + self.assertEqual(dtypes.Instant, cols[3].data_type) + self.assertEqual("sub_first", cols[4].name) + self.assertEqual(dtypes.string, cols[4].data_type) + self.assertEqual("sub_last", cols[5].name) + self.assertEqual(dtypes.string, cols[5].data_type) + def test_avro_spec(self): """ Check an Avro Kafka subscription creates the right table. diff --git a/redpanda-standalone/docker-compose.yml b/redpanda-standalone/docker-compose.yml index edac6326b13..12249145c39 100644 --- a/redpanda-standalone/docker-compose.yml +++ b/redpanda-standalone/docker-compose.yml @@ -15,7 +15,7 @@ services: - PLAINTEXT://redpanda:29092,OUTSIDE://localhost:9092 - --pandaproxy-addr 0.0.0.0:8082 - --advertise-pandaproxy-addr redpanda:8082 - image: docker.redpanda.com/vectorized/redpanda:v22.2.5 + image: docker.redpanda.com/redpandadata/redpanda:v23.2.8 ports: - 8081:8081 - 8082:8082 diff --git a/settings.gradle b/settings.gradle index 2e1d5989a32..c1ad45d0946 100644 --- a/settings.gradle +++ b/settings.gradle @@ -255,6 +255,9 @@ project(':extensions-performance').projectDir = file('extensions/performance') include(':extensions-source-support') project(':extensions-source-support').projectDir = file('extensions/source-support') +include(':extensions-protobuf') +project(':extensions-protobuf').projectDir = file('extensions/protobuf') + include(':plugin') include(':plugin-dagger')