From 44d629319d4fa274276f11951b0141c420e073c4 Mon Sep 17 00:00:00 2001
From: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
Date: Mon, 7 Jun 2021 15:19:39 +0530
Subject: [PATCH] handle timestamps of complex types when parsing protobuf
messages (#11293)
* handle timestamps correctly when parsing protobuf
* Add timestamp handling to ProtobufReader
* disable checkstyle for generated sourcecode
* Fix test
* try this
* refactor tests
---
codestyle/checkstyle-suppressions.xml | 2 +-
.../protobuf/ProtobufInputRowParser.java | 28 +-
.../protobuf/ProtobufInputRowSchema.java | 76 +
.../data/input/protobuf/ProtobufReader.java | 10 +-
.../input/protobuf/ProtoTestEventWrapper.java | 2193 ++++++++---------
.../protobuf/ProtobufInputFormatTest.java | 83 +-
.../protobuf/ProtobufInputRowParserTest.java | 183 +-
.../input/protobuf/ProtobufReaderTest.java | 125 +
...RegistryBasedProtobufBytesDecoderTest.java | 40 +-
.../src/test/resources/ProtoTest.proto | 7 +-
.../src/test/resources/prototest.desc | Bin 639 -> 998 bytes
11 files changed, 1402 insertions(+), 1345 deletions(-)
create mode 100644 extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java
create mode 100644 extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufReaderTest.java
diff --git a/codestyle/checkstyle-suppressions.xml b/codestyle/checkstyle-suppressions.xml
index 060b1e1f6f2..01868d73f34 100644
--- a/codestyle/checkstyle-suppressions.xml
+++ b/codestyle/checkstyle-suppressions.xml
@@ -48,7 +48,7 @@
-
+
diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParser.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParser.java
index 7624f239bf7..ccf3b826607 100644
--- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParser.java
+++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParser.java
@@ -32,10 +32,11 @@ import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.data.input.impl.JSONParseSpec;
import org.apache.druid.data.input.impl.ParseSpec;
-import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.java.util.common.parsers.Parser;
import org.apache.druid.utils.CollectionUtils;
+import org.joda.time.DateTime;
import java.nio.ByteBuffer;
import java.util.List;
@@ -43,9 +44,11 @@ import java.util.Map;
public class ProtobufInputRowParser implements ByteBufferInputRowParser
{
- private static final Logger LOG = new Logger(ByteBufferInputRowParser.class);
-
private final ParseSpec parseSpec;
+ // timestamp spec to be used for parsing timestamp
+ private final TimestampSpec timestampSpec;
+ // whether the spec has any fields to flat
+ private final boolean isFlatSpec;
private final ProtobufBytesDecoder protobufBytesDecoder;
private Parser parser;
private final List dimensions;
@@ -62,12 +65,18 @@ public class ProtobufInputRowParser implements ByteBufferInputRowParser
{
this.parseSpec = parseSpec;
this.dimensions = parseSpec.getDimensionsSpec().getDimensionNames();
-
+ this.isFlatSpec = parseSpec instanceof JSONParseSpec && ((JSONParseSpec) parseSpec).getFlattenSpec().getFields().isEmpty();
if (descriptorFilePath != null || protoMessageType != null) {
this.protobufBytesDecoder = new FileBasedProtobufBytesDecoder(descriptorFilePath, protoMessageType);
} else {
this.protobufBytesDecoder = protobufBytesDecoder;
}
+ if (isFlatSpec) {
+ this.timestampSpec = new ProtobufInputRowSchema.ProtobufTimestampSpec(parseSpec.getTimestampSpec());
+ } else {
+ this.timestampSpec = parseSpec.getTimestampSpec();
+ }
+
}
@Override
@@ -89,11 +98,13 @@ public class ProtobufInputRowParser implements ByteBufferInputRowParser
parser = parseSpec.makeParser();
}
Map record;
+ DateTime timestamp;
- if (parseSpec instanceof JSONParseSpec && ((JSONParseSpec) parseSpec).getFlattenSpec().getFields().isEmpty()) {
+ if (isFlatSpec) {
try {
DynamicMessage message = protobufBytesDecoder.parse(input);
record = CollectionUtils.mapKeys(message.getAllFields(), k -> k.getJsonName());
+ timestamp = this.timestampSpec.extractTimestamp(record);
}
catch (Exception ex) {
throw new ParseException(ex, "Protobuf message could not be parsed");
@@ -103,6 +114,7 @@ public class ProtobufInputRowParser implements ByteBufferInputRowParser
DynamicMessage message = protobufBytesDecoder.parse(input);
String json = JsonFormat.printer().print(message);
record = parser.parseToMap(json);
+ timestamp = this.timestampSpec.extractTimestamp(record);
}
catch (InvalidProtocolBufferException e) {
throw new ParseException(e, "Protobuf message could not be parsed");
@@ -117,10 +129,6 @@ public class ProtobufInputRowParser implements ByteBufferInputRowParser
Sets.difference(record.keySet(), parseSpec.getDimensionsSpec().getDimensionExclusions())
);
}
- return ImmutableList.of(new MapBasedInputRow(
- parseSpec.getTimestampSpec().extractTimestamp(record),
- dimensions,
- record
- ));
+ return ImmutableList.of(new MapBasedInputRow(timestamp, dimensions, record));
}
}
diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java
new file mode 100644
index 00000000000..c37cfb2423a
--- /dev/null
+++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufInputRowSchema.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.data.input.protobuf;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.Message;
+import com.google.protobuf.Timestamp;
+import com.google.protobuf.util.JsonFormat;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.impl.TimestampSpec;
+import org.apache.druid.java.util.common.parsers.ParseException;
+import org.joda.time.DateTime;
+
+import javax.annotation.Nullable;
+
+import java.util.Map;
+
+/**
+ * Extension of {@link InputRowSchema} with a custom {@link TimestampSpec} to support timestamp extraction for
+ * complex timestamp types
+ */
+public class ProtobufInputRowSchema extends InputRowSchema
+{
+ public ProtobufInputRowSchema(InputRowSchema inputRowSchema)
+ {
+ super(new ProtobufTimestampSpec(inputRowSchema.getTimestampSpec()), inputRowSchema.getDimensionsSpec(), inputRowSchema.getColumnsFilter());
+ }
+
+ static class ProtobufTimestampSpec extends TimestampSpec
+ {
+ public ProtobufTimestampSpec(TimestampSpec timestampSpec)
+ {
+ super(timestampSpec.getTimestampColumn(), timestampSpec.getTimestampFormat(), timestampSpec.getMissingValue());
+ }
+
+ /**
+ * Extracts the timestamp from the record. If the timestamp column is of complex type such as {@link Timestamp}, then the timestamp
+ * is first serialized to string via {@link JsonFormat}. Directly calling {@code toString()} on {@code Timestamp}
+ * returns an unparseable string.
+ */
+ @Override
+ @Nullable
+ public DateTime extractTimestamp(@Nullable Map input)
+ {
+ Object rawTimestamp = getRawTimestamp(input);
+ if (rawTimestamp instanceof Message) {
+ try {
+ String timestampStr = JsonFormat.printer().print((Message) rawTimestamp);
+ return parseDateTime(timestampStr);
+ }
+ catch (InvalidProtocolBufferException e) {
+ throw new ParseException(e, "Protobuf message could not be parsed");
+ }
+ } else {
+ return parseDateTime(rawTimestamp);
+ }
+ }
+ }
+}
diff --git a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java
index 5f7aed65f26..d3e553096c4 100644
--- a/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java
+++ b/extensions-core/protobuf-extensions/src/main/java/org/apache/druid/data/input/protobuf/ProtobufReader.java
@@ -62,11 +62,17 @@ public class ProtobufReader extends IntermediateRowParsingReader
JSONPathSpec flattenSpec
)
{
- this.inputRowSchema = inputRowSchema;
+ if (flattenSpec == null) {
+ this.inputRowSchema = new ProtobufInputRowSchema(inputRowSchema);
+ this.recordFlattener = null;
+ } else {
+ this.inputRowSchema = inputRowSchema;
+ this.recordFlattener = ObjectFlatteners.create(flattenSpec, new JSONFlattenerMaker(true));
+ }
+
this.source = source;
this.protobufBytesDecoder = protobufBytesDecoder;
this.flattenSpec = flattenSpec;
- this.recordFlattener = ObjectFlatteners.create(flattenSpec, new JSONFlattenerMaker(true));
}
@Override
diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtoTestEventWrapper.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtoTestEventWrapper.java
index 1aac2a2aeb1..0b370c2d972 100644
--- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtoTestEventWrapper.java
+++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtoTestEventWrapper.java
@@ -18,148 +18,147 @@
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
-// source: ProtoTest.proto
+// source: Prototest.proto
package org.apache.druid.data.input.protobuf;
@SuppressWarnings("ALL")
-public final class ProtoTestEventWrapper
-{
- private ProtoTestEventWrapper()
- {
+public final class ProtoTestEventWrapper {
+ private ProtoTestEventWrapper() {}
+ public static void registerAllExtensions(
+ com.google.protobuf.ExtensionRegistryLite registry) {
}
public static void registerAllExtensions(
- com.google.protobuf.ExtensionRegistryLite registry
- )
- {
- }
-
- public static void registerAllExtensions(
- com.google.protobuf.ExtensionRegistry registry
- )
- {
+ com.google.protobuf.ExtensionRegistry registry) {
registerAllExtensions(
(com.google.protobuf.ExtensionRegistryLite) registry);
}
-
public interface ProtoTestEventOrBuilder extends
// @@protoc_insertion_point(interface_extends:prototest.ProtoTestEvent)
- com.google.protobuf.MessageOrBuilder
- {
+ com.google.protobuf.MessageOrBuilder {
/**
* required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ * @return Whether the eventType field is set.
*/
boolean hasEventType();
-
/**
* required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ * @return The eventType.
*/
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType();
/**
* required uint64 id = 2;
+ * @return Whether the id field is set.
*/
boolean hasId();
-
/**
* required uint64 id = 2;
+ * @return The id.
*/
long getId();
/**
* required string timestamp = 3;
+ * @return Whether the timestamp field is set.
*/
boolean hasTimestamp();
-
/**
* required string timestamp = 3;
+ * @return The timestamp.
*/
java.lang.String getTimestamp();
-
/**
* required string timestamp = 3;
+ * @return The bytes for timestamp.
*/
com.google.protobuf.ByteString
- getTimestampBytes();
+ getTimestampBytes();
/**
* optional uint32 someOtherId = 4;
+ * @return Whether the someOtherId field is set.
*/
boolean hasSomeOtherId();
-
/**
* optional uint32 someOtherId = 4;
+ * @return The someOtherId.
*/
int getSomeOtherId();
/**
* optional bool isValid = 5;
+ * @return Whether the isValid field is set.
*/
boolean hasIsValid();
-
/**
* optional bool isValid = 5;
+ * @return The isValid.
*/
boolean getIsValid();
/**
* optional string description = 6;
+ * @return Whether the description field is set.
*/
boolean hasDescription();
-
/**
* optional string description = 6;
+ * @return The description.
*/
java.lang.String getDescription();
-
/**
* optional string description = 6;
+ * @return The bytes for description.
*/
com.google.protobuf.ByteString
- getDescriptionBytes();
+ getDescriptionBytes();
/**
* optional float someFloatColumn = 7;
+ * @return Whether the someFloatColumn field is set.
*/
boolean hasSomeFloatColumn();
-
/**
* optional float someFloatColumn = 7;
+ * @return The someFloatColumn.
*/
float getSomeFloatColumn();
/**
* optional uint32 someIntColumn = 8;
+ * @return Whether the someIntColumn field is set.
*/
boolean hasSomeIntColumn();
-
/**
* optional uint32 someIntColumn = 8;
+ * @return The someIntColumn.
*/
int getSomeIntColumn();
/**
* optional uint64 someLongColumn = 9;
+ * @return Whether the someLongColumn field is set.
*/
boolean hasSomeLongColumn();
-
/**
* optional uint64 someLongColumn = 9;
+ * @return The someLongColumn.
*/
long getSomeLongColumn();
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
+ * @return Whether the foo field is set.
*/
boolean hasFoo();
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
+ * @return The foo.
*/
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getFoo();
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
*/
@@ -168,75 +167,81 @@ public final class ProtoTestEventWrapper
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- java.util.List
- getBarList();
-
+ java.util.List
+ getBarList();
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getBar(int index);
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
int getBarCount();
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- java.util.List extends org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
- getBarOrBuilderList();
-
+ java.util.List extends org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
+ getBarOrBuilderList();
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getBarOrBuilder(
- int index
- );
- }
+ int index);
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ * @return Whether the otherTimestamp field is set.
+ */
+ boolean hasOtherTimestamp();
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ * @return The otherTimestamp.
+ */
+ com.google.protobuf.Timestamp getOtherTimestamp();
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ */
+ com.google.protobuf.TimestampOrBuilder getOtherTimestampOrBuilder();
+ }
/**
* Protobuf type {@code prototest.ProtoTestEvent}
*/
public static final class ProtoTestEvent extends
com.google.protobuf.GeneratedMessageV3 implements
// @@protoc_insertion_point(message_implements:prototest.ProtoTestEvent)
- ProtoTestEventOrBuilder
- {
+ ProtoTestEventOrBuilder {
+ private static final long serialVersionUID = 0L;
// Use ProtoTestEvent.newBuilder() to construct.
- private ProtoTestEvent(com.google.protobuf.GeneratedMessageV3.Builder> builder)
- {
+ private ProtoTestEvent(com.google.protobuf.GeneratedMessageV3.Builder> builder) {
super(builder);
}
-
- private ProtoTestEvent()
- {
+ private ProtoTestEvent() {
eventType_ = 0;
- id_ = 0L;
timestamp_ = "";
- someOtherId_ = 0;
- isValid_ = false;
description_ = "";
- someFloatColumn_ = 0F;
- someIntColumn_ = 0;
- someLongColumn_ = 0L;
bar_ = java.util.Collections.emptyList();
}
@java.lang.Override
- public com.google.protobuf.UnknownFieldSet
- getUnknownFields()
- {
- return this.unknownFields;
+ @SuppressWarnings({"unused"})
+ protected java.lang.Object newInstance(
+ UnusedPrivateParameter unused) {
+ return new ProtoTestEvent();
}
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
private ProtoTestEvent(
com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
this();
+ if (extensionRegistry == null) {
+ throw new java.lang.NullPointerException();
+ }
int mutable_bitField0_ = 0;
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
com.google.protobuf.UnknownFieldSet.newBuilder();
@@ -248,18 +253,10 @@ public final class ProtoTestEventWrapper
case 0:
done = true;
break;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag
- )) {
- done = true;
- }
- break;
- }
case 8: {
int rawValue = input.readEnum();
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value = org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory
- .valueOf(rawValue);
+ @SuppressWarnings("deprecation")
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value = org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.valueOf(rawValue);
if (value == null) {
unknownFields.mergeVarintField(1, rawValue);
} else {
@@ -312,13 +309,10 @@ public final class ProtoTestEventWrapper
}
case 82: {
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder subBuilder = null;
- if (((bitField0_ & 0x00000200) == 0x00000200)) {
+ if (((bitField0_ & 0x00000200) != 0)) {
subBuilder = foo_.toBuilder();
}
- foo_ = input.readMessage(
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.PARSER,
- extensionRegistry
- );
+ foo_ = input.readMessage(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.PARSER, extensionRegistry);
if (subBuilder != null) {
subBuilder.mergeFrom(foo_);
foo_ = subBuilder.buildPartial();
@@ -327,58 +321,67 @@ public final class ProtoTestEventWrapper
break;
}
case 90: {
- if (!((mutable_bitField0_ & 0x00000400) == 0x00000400)) {
+ if (!((mutable_bitField0_ & 0x00000400) != 0)) {
bar_ = new java.util.ArrayList();
mutable_bitField0_ |= 0x00000400;
}
bar_.add(
- input.readMessage(
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.PARSER,
- extensionRegistry
- ));
+ input.readMessage(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.PARSER, extensionRegistry));
+ break;
+ }
+ case 98: {
+ com.google.protobuf.Timestamp.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000400) != 0)) {
+ subBuilder = otherTimestamp_.toBuilder();
+ }
+ otherTimestamp_ = input.readMessage(com.google.protobuf.Timestamp.parser(), extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(otherTimestamp_);
+ otherTimestamp_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000400;
+ break;
+ }
+ default: {
+ if (!parseUnknownField(
+ input, unknownFields, extensionRegistry, tag)) {
+ done = true;
+ }
break;
}
}
}
- }
- catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
throw e.setUnfinishedMessage(this);
- }
- catch (java.io.IOException e) {
+ } catch (java.io.IOException e) {
throw new com.google.protobuf.InvalidProtocolBufferException(
e).setUnfinishedMessage(this);
- }
- finally {
- if (((mutable_bitField0_ & 0x00000400) == 0x00000400)) {
+ } finally {
+ if (((mutable_bitField0_ & 0x00000400) != 0)) {
bar_ = java.util.Collections.unmodifiableList(bar_);
}
this.unknownFields = unknownFields.build();
makeExtensionsImmutable();
}
}
-
- public static com.google.protobuf.Descriptors.Descriptor
- getDescriptor()
- {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor;
}
+ @java.lang.Override
protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
- internalGetFieldAccessorTable()
- {
+ internalGetFieldAccessorTable() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.class,
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Builder.class
- );
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.class, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Builder.class);
}
/**
* Protobuf enum {@code prototest.ProtoTestEvent.EventCategory}
*/
public enum EventCategory
- implements com.google.protobuf.ProtocolMessageEnum
- {
+ implements com.google.protobuf.ProtocolMessageEnum {
/**
* CATEGORY_ZERO = 0;
*/
@@ -390,7 +393,8 @@ public final class ProtoTestEventWrapper
/**
* CATEGORY_TWO = 2;
*/
- CATEGORY_TWO(2),;
+ CATEGORY_TWO(2),
+ ;
/**
* CATEGORY_ZERO = 0;
@@ -406,85 +410,72 @@ public final class ProtoTestEventWrapper
public static final int CATEGORY_TWO_VALUE = 2;
- public int getNumber()
- {
+ public final int getNumber() {
return value;
}
/**
+ * @param value The numeric wire value of the corresponding enum entry.
+ * @return The enum associated with the given numeric wire value.
* @deprecated Use {@link #forNumber(int)} instead.
*/
@java.lang.Deprecated
- public static EventCategory valueOf(int value)
- {
+ public static EventCategory valueOf(int value) {
return forNumber(value);
}
- public static EventCategory forNumber(int value)
- {
+ /**
+ * @param value The numeric wire value of the corresponding enum entry.
+ * @return The enum associated with the given numeric wire value.
+ */
+ public static EventCategory forNumber(int value) {
switch (value) {
- case 0:
- return CATEGORY_ZERO;
- case 1:
- return CATEGORY_ONE;
- case 2:
- return CATEGORY_TWO;
- default:
- return null;
+ case 0: return CATEGORY_ZERO;
+ case 1: return CATEGORY_ONE;
+ case 2: return CATEGORY_TWO;
+ default: return null;
}
}
public static com.google.protobuf.Internal.EnumLiteMap
- internalGetValueMap()
- {
+ internalGetValueMap() {
return internalValueMap;
}
-
private static final com.google.protobuf.Internal.EnumLiteMap<
EventCategory> internalValueMap =
- new com.google.protobuf.Internal.EnumLiteMap()
- {
- public EventCategory findValueByNumber(int number)
- {
- return EventCategory.forNumber(number);
- }
- };
+ new com.google.protobuf.Internal.EnumLiteMap() {
+ public EventCategory findValueByNumber(int number) {
+ return EventCategory.forNumber(number);
+ }
+ };
- public com.google.protobuf.Descriptors.EnumValueDescriptor
- getValueDescriptor()
- {
+ public final com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor() {
return getDescriptor().getValues().get(ordinal());
}
-
- public com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptorForType()
- {
+ public final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType() {
return getDescriptor();
}
-
- public static com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptor()
- {
+ public static final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.getDescriptor().getEnumTypes().get(0);
}
private static final EventCategory[] VALUES = values();
public static EventCategory valueOf(
- com.google.protobuf.Descriptors.EnumValueDescriptor desc
- )
- {
+ com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
if (desc.getType() != getDescriptor()) {
throw new java.lang.IllegalArgumentException(
- "EnumValueDescriptor is not for this type.");
+ "EnumValueDescriptor is not for this type.");
}
return VALUES[desc.getIndex()];
}
private final int value;
- EventCategory(int value)
- {
+ EventCategory(int value) {
this.value = value;
}
@@ -493,59 +484,61 @@ public final class ProtoTestEventWrapper
public interface FooOrBuilder extends
// @@protoc_insertion_point(interface_extends:prototest.ProtoTestEvent.Foo)
- com.google.protobuf.MessageOrBuilder
- {
+ com.google.protobuf.MessageOrBuilder {
/**
* required string bar = 1;
+ * @return Whether the bar field is set.
*/
boolean hasBar();
-
/**
* required string bar = 1;
+ * @return The bar.
*/
java.lang.String getBar();
-
/**
* required string bar = 1;
+ * @return The bytes for bar.
*/
com.google.protobuf.ByteString
- getBarBytes();
+ getBarBytes();
}
-
/**
* Protobuf type {@code prototest.ProtoTestEvent.Foo}
*/
public static final class Foo extends
com.google.protobuf.GeneratedMessageV3 implements
// @@protoc_insertion_point(message_implements:prototest.ProtoTestEvent.Foo)
- FooOrBuilder
- {
+ FooOrBuilder {
+ private static final long serialVersionUID = 0L;
// Use Foo.newBuilder() to construct.
- private Foo(com.google.protobuf.GeneratedMessageV3.Builder> builder)
- {
+ private Foo(com.google.protobuf.GeneratedMessageV3.Builder> builder) {
super(builder);
}
-
- private Foo()
- {
+ private Foo() {
bar_ = "";
}
@java.lang.Override
- public com.google.protobuf.UnknownFieldSet
- getUnknownFields()
- {
- return this.unknownFields;
+ @SuppressWarnings({"unused"})
+ protected java.lang.Object newInstance(
+ UnusedPrivateParameter unused) {
+ return new Foo();
}
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
private Foo(
com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
this();
+ if (extensionRegistry == null) {
+ throw new java.lang.NullPointerException();
+ }
int mutable_bitField0_ = 0;
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
com.google.protobuf.UnknownFieldSet.newBuilder();
@@ -557,74 +550,66 @@ public final class ProtoTestEventWrapper
case 0:
done = true;
break;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag
- )) {
- done = true;
- }
- break;
- }
case 10: {
com.google.protobuf.ByteString bs = input.readBytes();
bitField0_ |= 0x00000001;
bar_ = bs;
break;
}
+ default: {
+ if (!parseUnknownField(
+ input, unknownFields, extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
}
}
- }
- catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
throw e.setUnfinishedMessage(this);
- }
- catch (java.io.IOException e) {
+ } catch (java.io.IOException e) {
throw new com.google.protobuf.InvalidProtocolBufferException(
e).setUnfinishedMessage(this);
- }
- finally {
+ } finally {
this.unknownFields = unknownFields.build();
makeExtensionsImmutable();
}
}
-
- public static com.google.protobuf.Descriptors.Descriptor
- getDescriptor()
- {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_descriptor;
}
+ @java.lang.Override
protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
- internalGetFieldAccessorTable()
- {
+ internalGetFieldAccessorTable() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.class,
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder.class
- );
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.class, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder.class);
}
private int bitField0_;
public static final int BAR_FIELD_NUMBER = 1;
private volatile java.lang.Object bar_;
-
/**
* required string bar = 1;
+ * @return Whether the bar field is set.
*/
- public boolean hasBar()
- {
- return ((bitField0_ & 0x00000001) == 0x00000001);
+ @java.lang.Override
+ public boolean hasBar() {
+ return ((bitField0_ & 0x00000001) != 0);
}
-
/**
* required string bar = 1;
+ * @return The bar.
*/
- public java.lang.String getBar()
- {
+ @java.lang.Override
+ public java.lang.String getBar() {
java.lang.Object ref = bar_;
if (ref instanceof java.lang.String) {
return (java.lang.String) ref;
} else {
- com.google.protobuf.ByteString bs =
+ com.google.protobuf.ByteString bs =
(com.google.protobuf.ByteString) ref;
java.lang.String s = bs.toStringUtf8();
if (bs.isValidUtf8()) {
@@ -633,16 +618,16 @@ public final class ProtoTestEventWrapper
return s;
}
}
-
/**
* required string bar = 1;
+ * @return The bytes for bar.
*/
+ @java.lang.Override
public com.google.protobuf.ByteString
- getBarBytes()
- {
+ getBarBytes() {
java.lang.Object ref = bar_;
if (ref instanceof java.lang.String) {
- com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8(
(java.lang.String) ref);
bar_ = b;
@@ -653,16 +638,11 @@ public final class ProtoTestEventWrapper
}
private byte memoizedIsInitialized = -1;
-
- public boolean isInitialized()
- {
+ @java.lang.Override
+ public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized;
- if (isInitialized == 1) {
- return true;
- }
- if (isInitialized == 0) {
- return false;
- }
+ if (isInitialized == 1) return true;
+ if (isInitialized == 0) return false;
if (!hasBar()) {
memoizedIsInitialized = 0;
@@ -672,24 +652,22 @@ public final class ProtoTestEventWrapper
return true;
}
+ @java.lang.Override
public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException
- {
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ throws java.io.IOException {
+ if (((bitField0_ & 0x00000001) != 0)) {
com.google.protobuf.GeneratedMessageV3.writeString(output, 1, bar_);
}
unknownFields.writeTo(output);
}
- public int getSerializedSize()
- {
+ @java.lang.Override
+ public int getSerializedSize() {
int size = memoizedSize;
- if (size != -1) {
- return size;
- }
+ if (size != -1) return size;
size = 0;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ if (((bitField0_ & 0x00000001) != 0)) {
size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, bar_);
}
size += unknownFields.getSerializedSize();
@@ -697,31 +675,27 @@ public final class ProtoTestEventWrapper
return size;
}
- private static final long serialVersionUID = 0L;
-
@java.lang.Override
- public boolean equals(final java.lang.Object obj)
- {
+ public boolean equals(final java.lang.Object obj) {
if (obj == this) {
- return true;
+ return true;
}
if (!(obj instanceof org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo)) {
return super.equals(obj);
}
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo other = (org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) obj;
- boolean result = hasBar() == other.hasBar();
+ if (hasBar() != other.hasBar()) return false;
if (hasBar()) {
- result = result && getBar()
- .equals(other.getBar());
+ if (!getBar()
+ .equals(other.getBar())) return false;
}
- result = result && unknownFields.equals(other.unknownFields);
- return result;
+ if (!unknownFields.equals(other.unknownFields)) return false;
+ return true;
}
@java.lang.Override
- public int hashCode()
- {
+ public int hashCode() {
if (memoizedHashCode != 0) {
return memoizedHashCode;
}
@@ -737,186 +711,151 @@ public final class ProtoTestEventWrapper
}
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
- com.google.protobuf.ByteString data
- )
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ java.nio.ByteBuffer data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
+ java.nio.ByteBuffer data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(java.io.InputStream input)
- throws java.io.IOException
- {
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws java.io.IOException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input, extensionRegistry);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException
- {
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseDelimitedWithIOException(PARSER, input);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseDelimitedFrom(
java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws java.io.IOException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseDelimitedWithIOException(PARSER, input, extensionRegistry);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
- com.google.protobuf.CodedInputStream input
- )
- throws java.io.IOException
- {
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws java.io.IOException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input, extensionRegistry);
}
- public Builder newBuilderForType()
- {
- return newBuilder();
- }
-
- public static Builder newBuilder()
- {
+ @java.lang.Override
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder() {
return DEFAULT_INSTANCE.toBuilder();
}
-
- public static Builder newBuilder(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo prototype)
- {
+ public static Builder newBuilder(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo prototype) {
return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
}
-
- public Builder toBuilder()
- {
+ @java.lang.Override
+ public Builder toBuilder() {
return this == DEFAULT_INSTANCE
- ? new Builder() : new Builder().mergeFrom(this);
+ ? new Builder() : new Builder().mergeFrom(this);
}
@java.lang.Override
protected Builder newBuilderForType(
- com.google.protobuf.GeneratedMessageV3.BuilderParent parent
- )
- {
+ com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
Builder builder = new Builder(parent);
return builder;
}
-
/**
* Protobuf type {@code prototest.ProtoTestEvent.Foo}
*/
public static final class Builder extends
com.google.protobuf.GeneratedMessageV3.Builder implements
// @@protoc_insertion_point(builder_implements:prototest.ProtoTestEvent.Foo)
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder
- {
- public static com.google.protobuf.Descriptors.Descriptor
- getDescriptor()
- {
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_descriptor;
}
+ @java.lang.Override
protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
- internalGetFieldAccessorTable()
- {
+ internalGetFieldAccessorTable() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.class,
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder.class
- );
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.class, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder.class);
}
// Construct using org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.newBuilder()
- private Builder()
- {
+ private Builder() {
maybeForceBuilderInitialization();
}
private Builder(
- com.google.protobuf.GeneratedMessageV3.BuilderParent parent
- )
- {
+ com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
super(parent);
maybeForceBuilderInitialization();
}
-
- private void maybeForceBuilderInitialization()
- {
+ private void maybeForceBuilderInitialization() {
if (com.google.protobuf.GeneratedMessageV3
- .alwaysUseFieldBuilders) {
+ .alwaysUseFieldBuilders) {
}
}
-
- public Builder clear()
- {
+ @java.lang.Override
+ public Builder clear() {
super.clear();
bar_ = "";
bitField0_ = (bitField0_ & ~0x00000001);
return this;
}
+ @java.lang.Override
public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType()
- {
+ getDescriptorForType() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_descriptor;
}
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getDefaultInstanceForType()
- {
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getDefaultInstanceForType() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance();
}
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo build()
- {
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo build() {
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(result);
@@ -924,13 +863,12 @@ public final class ProtoTestEventWrapper
return result;
}
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo buildPartial()
- {
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo result = new org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo(
- this);
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo buildPartial() {
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo result = new org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo(this);
int from_bitField0_ = bitField0_;
int to_bitField0_ = 0;
- if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ if (((from_bitField0_ & 0x00000001) != 0)) {
to_bitField0_ |= 0x00000001;
}
result.bar_ = bar_;
@@ -939,64 +877,50 @@ public final class ProtoTestEventWrapper
return result;
}
- public Builder clone()
- {
- return (Builder) super.clone();
+ @java.lang.Override
+ public Builder clone() {
+ return super.clone();
}
-
+ @java.lang.Override
public Builder setField(
com.google.protobuf.Descriptors.FieldDescriptor field,
- Object value
- )
- {
- return (Builder) super.setField(field, value);
+ java.lang.Object value) {
+ return super.setField(field, value);
}
-
+ @java.lang.Override
public Builder clearField(
- com.google.protobuf.Descriptors.FieldDescriptor field
- )
- {
- return (Builder) super.clearField(field);
+ com.google.protobuf.Descriptors.FieldDescriptor field) {
+ return super.clearField(field);
}
-
+ @java.lang.Override
public Builder clearOneof(
- com.google.protobuf.Descriptors.OneofDescriptor oneof
- )
- {
- return (Builder) super.clearOneof(oneof);
+ com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+ return super.clearOneof(oneof);
}
-
+ @java.lang.Override
public Builder setRepeatedField(
com.google.protobuf.Descriptors.FieldDescriptor field,
- int index, Object value
- )
- {
- return (Builder) super.setRepeatedField(field, index, value);
+ int index, java.lang.Object value) {
+ return super.setRepeatedField(field, index, value);
}
-
+ @java.lang.Override
public Builder addRepeatedField(
com.google.protobuf.Descriptors.FieldDescriptor field,
- Object value
- )
- {
- return (Builder) super.addRepeatedField(field, value);
+ java.lang.Object value) {
+ return super.addRepeatedField(field, value);
}
-
- public Builder mergeFrom(com.google.protobuf.Message other)
- {
+ @java.lang.Override
+ public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) {
- return mergeFrom((org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) other);
+ return mergeFrom((org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo)other);
} else {
super.mergeFrom(other);
return this;
}
}
- public Builder mergeFrom(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo other)
- {
- if (other == org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()) {
- return this;
- }
+ public Builder mergeFrom(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo other) {
+ if (other == org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()) return this;
if (other.hasBar()) {
bitField0_ |= 0x00000001;
bar_ = other.bar_;
@@ -1007,53 +931,47 @@ public final class ProtoTestEventWrapper
return this;
}
- public boolean isInitialized()
- {
+ @java.lang.Override
+ public final boolean isInitialized() {
if (!hasBar()) {
return false;
}
return true;
}
+ @java.lang.Override
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws java.io.IOException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parsedMessage = null;
try {
parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
- }
- catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
parsedMessage = (org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) e.getUnfinishedMessage();
throw e.unwrapIOException();
- }
- finally {
+ } finally {
if (parsedMessage != null) {
mergeFrom(parsedMessage);
}
}
return this;
}
-
private int bitField0_;
private java.lang.Object bar_ = "";
-
/**
* required string bar = 1;
+ * @return Whether the bar field is set.
*/
- public boolean hasBar()
- {
- return ((bitField0_ & 0x00000001) == 0x00000001);
+ public boolean hasBar() {
+ return ((bitField0_ & 0x00000001) != 0);
}
-
/**
* required string bar = 1;
+ * @return The bar.
*/
- public java.lang.String getBar()
- {
+ public java.lang.String getBar() {
java.lang.Object ref = bar_;
if (!(ref instanceof java.lang.String)) {
com.google.protobuf.ByteString bs =
@@ -1067,16 +985,15 @@ public final class ProtoTestEventWrapper
return (java.lang.String) ref;
}
}
-
/**
* required string bar = 1;
+ * @return The bytes for bar.
*/
public com.google.protobuf.ByteString
- getBarBytes()
- {
+ getBarBytes() {
java.lang.Object ref = bar_;
if (ref instanceof String) {
- com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8(
(java.lang.String) ref);
bar_ = b;
@@ -1085,61 +1002,55 @@ public final class ProtoTestEventWrapper
return (com.google.protobuf.ByteString) ref;
}
}
-
/**
* required string bar = 1;
+ * @param value The bar to set.
+ * @return This builder for chaining.
*/
public Builder setBar(
- java.lang.String value
- )
- {
+ java.lang.String value) {
if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000001;
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
bar_ = value;
onChanged();
return this;
}
-
/**
* required string bar = 1;
+ * @return This builder for chaining.
*/
- public Builder clearBar()
- {
+ public Builder clearBar() {
bitField0_ = (bitField0_ & ~0x00000001);
bar_ = getDefaultInstance().getBar();
onChanged();
return this;
}
-
/**
* required string bar = 1;
+ * @param value The bytes for bar to set.
+ * @return This builder for chaining.
*/
public Builder setBarBytes(
- com.google.protobuf.ByteString value
- )
- {
+ com.google.protobuf.ByteString value) {
if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000001;
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
bar_ = value;
onChanged();
return this;
}
-
- public Builder setUnknownFields(
- final com.google.protobuf.UnknownFieldSet unknownFields
- )
- {
+ @java.lang.Override
+ public final Builder setUnknownFields(
+ final com.google.protobuf.UnknownFieldSet unknownFields) {
return super.setUnknownFields(unknownFields);
}
- public Builder mergeUnknownFields(
- final com.google.protobuf.UnknownFieldSet unknownFields
- )
- {
+ @java.lang.Override
+ public final Builder mergeUnknownFields(
+ final com.google.protobuf.UnknownFieldSet unknownFields) {
return super.mergeUnknownFields(unknownFields);
}
@@ -1149,43 +1060,36 @@ public final class ProtoTestEventWrapper
// @@protoc_insertion_point(class_scope:prototest.ProtoTestEvent.Foo)
private static final org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo DEFAULT_INSTANCE;
-
static {
DEFAULT_INSTANCE = new org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo();
}
- public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getDefaultInstance()
- {
+ public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getDefaultInstance() {
return DEFAULT_INSTANCE;
}
- @java.lang.Deprecated
- public static final com.google.protobuf.Parser
- PARSER = new com.google.protobuf.AbstractParser()
- {
+ @java.lang.Deprecated public static final com.google.protobuf.Parser
+ PARSER = new com.google.protobuf.AbstractParser() {
+ @java.lang.Override
public Foo parsePartialFrom(
com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
return new Foo(input, extensionRegistry);
}
};
- public static com.google.protobuf.Parser parser()
- {
+ public static com.google.protobuf.Parser parser() {
return PARSER;
}
@java.lang.Override
- public com.google.protobuf.Parser getParserForType()
- {
+ public com.google.protobuf.Parser getParserForType() {
return PARSER;
}
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getDefaultInstanceForType()
- {
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getDefaultInstanceForType() {
return DEFAULT_INSTANCE;
}
@@ -1194,67 +1098,63 @@ public final class ProtoTestEventWrapper
private int bitField0_;
public static final int EVENTTYPE_FIELD_NUMBER = 1;
private int eventType_;
-
/**
* required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ * @return Whether the eventType field is set.
*/
- public boolean hasEventType()
- {
- return ((bitField0_ & 0x00000001) == 0x00000001);
+ @java.lang.Override public boolean hasEventType() {
+ return ((bitField0_ & 0x00000001) != 0);
}
-
/**
* required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ * @return The eventType.
*/
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType()
- {
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory result = org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory
- .valueOf(eventType_);
- return result == null
- ? org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO
- : result;
+ @java.lang.Override public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() {
+ @SuppressWarnings("deprecation")
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory result = org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.valueOf(eventType_);
+ return result == null ? org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO : result;
}
public static final int ID_FIELD_NUMBER = 2;
private long id_;
-
/**
* required uint64 id = 2;
+ * @return Whether the id field is set.
*/
- public boolean hasId()
- {
- return ((bitField0_ & 0x00000002) == 0x00000002);
+ @java.lang.Override
+ public boolean hasId() {
+ return ((bitField0_ & 0x00000002) != 0);
}
-
/**
* required uint64 id = 2;
+ * @return The id.
*/
- public long getId()
- {
+ @java.lang.Override
+ public long getId() {
return id_;
}
public static final int TIMESTAMP_FIELD_NUMBER = 3;
private volatile java.lang.Object timestamp_;
-
/**
* required string timestamp = 3;
+ * @return Whether the timestamp field is set.
*/
- public boolean hasTimestamp()
- {
- return ((bitField0_ & 0x00000004) == 0x00000004);
+ @java.lang.Override
+ public boolean hasTimestamp() {
+ return ((bitField0_ & 0x00000004) != 0);
}
-
/**
* required string timestamp = 3;
+ * @return The timestamp.
*/
- public java.lang.String getTimestamp()
- {
+ @java.lang.Override
+ public java.lang.String getTimestamp() {
java.lang.Object ref = timestamp_;
if (ref instanceof java.lang.String) {
return (java.lang.String) ref;
} else {
- com.google.protobuf.ByteString bs =
+ com.google.protobuf.ByteString bs =
(com.google.protobuf.ByteString) ref;
java.lang.String s = bs.toStringUtf8();
if (bs.isValidUtf8()) {
@@ -1263,16 +1163,16 @@ public final class ProtoTestEventWrapper
return s;
}
}
-
/**
* required string timestamp = 3;
+ * @return The bytes for timestamp.
*/
+ @java.lang.Override
public com.google.protobuf.ByteString
- getTimestampBytes()
- {
+ getTimestampBytes() {
java.lang.Object ref = timestamp_;
if (ref instanceof java.lang.String) {
- com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8(
(java.lang.String) ref);
timestamp_ = b;
@@ -1284,63 +1184,63 @@ public final class ProtoTestEventWrapper
public static final int SOMEOTHERID_FIELD_NUMBER = 4;
private int someOtherId_;
-
/**
* optional uint32 someOtherId = 4;
+ * @return Whether the someOtherId field is set.
*/
- public boolean hasSomeOtherId()
- {
- return ((bitField0_ & 0x00000008) == 0x00000008);
+ @java.lang.Override
+ public boolean hasSomeOtherId() {
+ return ((bitField0_ & 0x00000008) != 0);
}
-
/**
* optional uint32 someOtherId = 4;
+ * @return The someOtherId.
*/
- public int getSomeOtherId()
- {
+ @java.lang.Override
+ public int getSomeOtherId() {
return someOtherId_;
}
public static final int ISVALID_FIELD_NUMBER = 5;
private boolean isValid_;
-
/**
* optional bool isValid = 5;
+ * @return Whether the isValid field is set.
*/
- public boolean hasIsValid()
- {
- return ((bitField0_ & 0x00000010) == 0x00000010);
+ @java.lang.Override
+ public boolean hasIsValid() {
+ return ((bitField0_ & 0x00000010) != 0);
}
-
/**
* optional bool isValid = 5;
+ * @return The isValid.
*/
- public boolean getIsValid()
- {
+ @java.lang.Override
+ public boolean getIsValid() {
return isValid_;
}
public static final int DESCRIPTION_FIELD_NUMBER = 6;
private volatile java.lang.Object description_;
-
/**
* optional string description = 6;
+ * @return Whether the description field is set.
*/
- public boolean hasDescription()
- {
- return ((bitField0_ & 0x00000020) == 0x00000020);
+ @java.lang.Override
+ public boolean hasDescription() {
+ return ((bitField0_ & 0x00000020) != 0);
}
-
/**
* optional string description = 6;
+ * @return The description.
*/
- public java.lang.String getDescription()
- {
+ @java.lang.Override
+ public java.lang.String getDescription() {
java.lang.Object ref = description_;
if (ref instanceof java.lang.String) {
return (java.lang.String) ref;
} else {
- com.google.protobuf.ByteString bs =
+ com.google.protobuf.ByteString bs =
(com.google.protobuf.ByteString) ref;
java.lang.String s = bs.toStringUtf8();
if (bs.isValidUtf8()) {
@@ -1349,16 +1249,16 @@ public final class ProtoTestEventWrapper
return s;
}
}
-
/**
* optional string description = 6;
+ * @return The bytes for description.
*/
+ @java.lang.Override
public com.google.protobuf.ByteString
- getDescriptionBytes()
- {
+ getDescriptionBytes() {
java.lang.Object ref = description_;
if (ref instanceof java.lang.String) {
- com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8(
(java.lang.String) ref);
description_ = b;
@@ -1370,149 +1270,159 @@ public final class ProtoTestEventWrapper
public static final int SOMEFLOATCOLUMN_FIELD_NUMBER = 7;
private float someFloatColumn_;
-
/**
* optional float someFloatColumn = 7;
+ * @return Whether the someFloatColumn field is set.
*/
- public boolean hasSomeFloatColumn()
- {
- return ((bitField0_ & 0x00000040) == 0x00000040);
+ @java.lang.Override
+ public boolean hasSomeFloatColumn() {
+ return ((bitField0_ & 0x00000040) != 0);
}
-
/**
* optional float someFloatColumn = 7;
+ * @return The someFloatColumn.
*/
- public float getSomeFloatColumn()
- {
+ @java.lang.Override
+ public float getSomeFloatColumn() {
return someFloatColumn_;
}
public static final int SOMEINTCOLUMN_FIELD_NUMBER = 8;
private int someIntColumn_;
-
/**
* optional uint32 someIntColumn = 8;
+ * @return Whether the someIntColumn field is set.
*/
- public boolean hasSomeIntColumn()
- {
- return ((bitField0_ & 0x00000080) == 0x00000080);
+ @java.lang.Override
+ public boolean hasSomeIntColumn() {
+ return ((bitField0_ & 0x00000080) != 0);
}
-
/**
* optional uint32 someIntColumn = 8;
+ * @return The someIntColumn.
*/
- public int getSomeIntColumn()
- {
+ @java.lang.Override
+ public int getSomeIntColumn() {
return someIntColumn_;
}
public static final int SOMELONGCOLUMN_FIELD_NUMBER = 9;
private long someLongColumn_;
-
/**
* optional uint64 someLongColumn = 9;
+ * @return Whether the someLongColumn field is set.
*/
- public boolean hasSomeLongColumn()
- {
- return ((bitField0_ & 0x00000100) == 0x00000100);
+ @java.lang.Override
+ public boolean hasSomeLongColumn() {
+ return ((bitField0_ & 0x00000100) != 0);
}
-
/**
* optional uint64 someLongColumn = 9;
+ * @return The someLongColumn.
*/
- public long getSomeLongColumn()
- {
+ @java.lang.Override
+ public long getSomeLongColumn() {
return someLongColumn_;
}
public static final int FOO_FIELD_NUMBER = 10;
private org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo foo_;
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
+ * @return Whether the foo field is set.
*/
- public boolean hasFoo()
- {
- return ((bitField0_ & 0x00000200) == 0x00000200);
+ @java.lang.Override
+ public boolean hasFoo() {
+ return ((bitField0_ & 0x00000200) != 0);
}
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
+ * @return The foo.
*/
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getFoo()
- {
- return foo_ == null
- ? org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()
- : foo_;
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getFoo() {
+ return foo_ == null ? org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance() : foo_;
}
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
*/
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getFooOrBuilder()
- {
- return foo_ == null
- ? org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()
- : foo_;
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getFooOrBuilder() {
+ return foo_ == null ? org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance() : foo_;
}
public static final int BAR_FIELD_NUMBER = 11;
private java.util.List bar_;
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public java.util.List getBarList()
- {
+ @java.lang.Override
+ public java.util.List getBarList() {
return bar_;
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public java.util.List extends org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
- getBarOrBuilderList()
- {
+ @java.lang.Override
+ public java.util.List extends org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
+ getBarOrBuilderList() {
return bar_;
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public int getBarCount()
- {
+ @java.lang.Override
+ public int getBarCount() {
return bar_.size();
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getBar(int index)
- {
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getBar(int index) {
+ return bar_.get(index);
+ }
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getBarOrBuilder(
+ int index) {
return bar_.get(index);
}
+ public static final int OTHERTIMESTAMP_FIELD_NUMBER = 12;
+ private com.google.protobuf.Timestamp otherTimestamp_;
/**
- * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ * @return Whether the otherTimestamp field is set.
*/
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getBarOrBuilder(
- int index
- )
- {
- return bar_.get(index);
+ @java.lang.Override
+ public boolean hasOtherTimestamp() {
+ return ((bitField0_ & 0x00000400) != 0);
+ }
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ * @return The otherTimestamp.
+ */
+ @java.lang.Override
+ public com.google.protobuf.Timestamp getOtherTimestamp() {
+ return otherTimestamp_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : otherTimestamp_;
+ }
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ */
+ @java.lang.Override
+ public com.google.protobuf.TimestampOrBuilder getOtherTimestampOrBuilder() {
+ return otherTimestamp_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : otherTimestamp_;
}
private byte memoizedIsInitialized = -1;
-
- public boolean isInitialized()
- {
+ @java.lang.Override
+ public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized;
- if (isInitialized == 1) {
- return true;
- }
- if (isInitialized == 0) {
- return false;
- }
+ if (isInitialized == 1) return true;
+ if (isInitialized == 0) return false;
if (!hasEventType()) {
memoizedIsInitialized = 0;
@@ -1542,174 +1452,178 @@ public final class ProtoTestEventWrapper
return true;
}
+ @java.lang.Override
public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException
- {
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ throws java.io.IOException {
+ if (((bitField0_ & 0x00000001) != 0)) {
output.writeEnum(1, eventType_);
}
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ if (((bitField0_ & 0x00000002) != 0)) {
output.writeUInt64(2, id_);
}
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ if (((bitField0_ & 0x00000004) != 0)) {
com.google.protobuf.GeneratedMessageV3.writeString(output, 3, timestamp_);
}
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ if (((bitField0_ & 0x00000008) != 0)) {
output.writeUInt32(4, someOtherId_);
}
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ if (((bitField0_ & 0x00000010) != 0)) {
output.writeBool(5, isValid_);
}
- if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ if (((bitField0_ & 0x00000020) != 0)) {
com.google.protobuf.GeneratedMessageV3.writeString(output, 6, description_);
}
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ if (((bitField0_ & 0x00000040) != 0)) {
output.writeFloat(7, someFloatColumn_);
}
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ if (((bitField0_ & 0x00000080) != 0)) {
output.writeUInt32(8, someIntColumn_);
}
- if (((bitField0_ & 0x00000100) == 0x00000100)) {
+ if (((bitField0_ & 0x00000100) != 0)) {
output.writeUInt64(9, someLongColumn_);
}
- if (((bitField0_ & 0x00000200) == 0x00000200)) {
+ if (((bitField0_ & 0x00000200) != 0)) {
output.writeMessage(10, getFoo());
}
for (int i = 0; i < bar_.size(); i++) {
output.writeMessage(11, bar_.get(i));
}
+ if (((bitField0_ & 0x00000400) != 0)) {
+ output.writeMessage(12, getOtherTimestamp());
+ }
unknownFields.writeTo(output);
}
- public int getSerializedSize()
- {
+ @java.lang.Override
+ public int getSerializedSize() {
int size = memoizedSize;
- if (size != -1) {
- return size;
- }
+ if (size != -1) return size;
size = 0;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ if (((bitField0_ & 0x00000001) != 0)) {
size += com.google.protobuf.CodedOutputStream
- .computeEnumSize(1, eventType_);
+ .computeEnumSize(1, eventType_);
}
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ if (((bitField0_ & 0x00000002) != 0)) {
size += com.google.protobuf.CodedOutputStream
- .computeUInt64Size(2, id_);
+ .computeUInt64Size(2, id_);
}
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ if (((bitField0_ & 0x00000004) != 0)) {
size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, timestamp_);
}
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ if (((bitField0_ & 0x00000008) != 0)) {
size += com.google.protobuf.CodedOutputStream
- .computeUInt32Size(4, someOtherId_);
+ .computeUInt32Size(4, someOtherId_);
}
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ if (((bitField0_ & 0x00000010) != 0)) {
size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(5, isValid_);
+ .computeBoolSize(5, isValid_);
}
- if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ if (((bitField0_ & 0x00000020) != 0)) {
size += com.google.protobuf.GeneratedMessageV3.computeStringSize(6, description_);
}
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ if (((bitField0_ & 0x00000040) != 0)) {
size += com.google.protobuf.CodedOutputStream
- .computeFloatSize(7, someFloatColumn_);
+ .computeFloatSize(7, someFloatColumn_);
}
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ if (((bitField0_ & 0x00000080) != 0)) {
size += com.google.protobuf.CodedOutputStream
- .computeUInt32Size(8, someIntColumn_);
+ .computeUInt32Size(8, someIntColumn_);
}
- if (((bitField0_ & 0x00000100) == 0x00000100)) {
+ if (((bitField0_ & 0x00000100) != 0)) {
size += com.google.protobuf.CodedOutputStream
- .computeUInt64Size(9, someLongColumn_);
+ .computeUInt64Size(9, someLongColumn_);
}
- if (((bitField0_ & 0x00000200) == 0x00000200)) {
+ if (((bitField0_ & 0x00000200) != 0)) {
size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(10, getFoo());
+ .computeMessageSize(10, getFoo());
}
for (int i = 0; i < bar_.size(); i++) {
size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(11, bar_.get(i));
+ .computeMessageSize(11, bar_.get(i));
+ }
+ if (((bitField0_ & 0x00000400) != 0)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(12, getOtherTimestamp());
}
size += unknownFields.getSerializedSize();
memoizedSize = size;
return size;
}
- private static final long serialVersionUID = 0L;
-
@java.lang.Override
- public boolean equals(final java.lang.Object obj)
- {
+ public boolean equals(final java.lang.Object obj) {
if (obj == this) {
- return true;
+ return true;
}
if (!(obj instanceof org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent)) {
return super.equals(obj);
}
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent other = (org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) obj;
- boolean result = true;
- result = result && (hasEventType() == other.hasEventType());
+ if (hasEventType() != other.hasEventType()) return false;
if (hasEventType()) {
- result = result && eventType_ == other.eventType_;
+ if (eventType_ != other.eventType_) return false;
}
- result = result && (hasId() == other.hasId());
+ if (hasId() != other.hasId()) return false;
if (hasId()) {
- result = result && (getId()
- == other.getId());
+ if (getId()
+ != other.getId()) return false;
}
- result = result && (hasTimestamp() == other.hasTimestamp());
+ if (hasTimestamp() != other.hasTimestamp()) return false;
if (hasTimestamp()) {
- result = result && getTimestamp()
- .equals(other.getTimestamp());
+ if (!getTimestamp()
+ .equals(other.getTimestamp())) return false;
}
- result = result && (hasSomeOtherId() == other.hasSomeOtherId());
+ if (hasSomeOtherId() != other.hasSomeOtherId()) return false;
if (hasSomeOtherId()) {
- result = result && (getSomeOtherId()
- == other.getSomeOtherId());
+ if (getSomeOtherId()
+ != other.getSomeOtherId()) return false;
}
- result = result && (hasIsValid() == other.hasIsValid());
+ if (hasIsValid() != other.hasIsValid()) return false;
if (hasIsValid()) {
- result = result && (getIsValid()
- == other.getIsValid());
+ if (getIsValid()
+ != other.getIsValid()) return false;
}
- result = result && (hasDescription() == other.hasDescription());
+ if (hasDescription() != other.hasDescription()) return false;
if (hasDescription()) {
- result = result && getDescription()
- .equals(other.getDescription());
+ if (!getDescription()
+ .equals(other.getDescription())) return false;
}
- result = result && (hasSomeFloatColumn() == other.hasSomeFloatColumn());
+ if (hasSomeFloatColumn() != other.hasSomeFloatColumn()) return false;
if (hasSomeFloatColumn()) {
- result = result && (
- java.lang.Float.floatToIntBits(getSomeFloatColumn())
- == java.lang.Float.floatToIntBits(
- other.getSomeFloatColumn()));
+ if (java.lang.Float.floatToIntBits(getSomeFloatColumn())
+ != java.lang.Float.floatToIntBits(
+ other.getSomeFloatColumn())) return false;
}
- result = result && (hasSomeIntColumn() == other.hasSomeIntColumn());
+ if (hasSomeIntColumn() != other.hasSomeIntColumn()) return false;
if (hasSomeIntColumn()) {
- result = result && (getSomeIntColumn()
- == other.getSomeIntColumn());
+ if (getSomeIntColumn()
+ != other.getSomeIntColumn()) return false;
}
- result = result && (hasSomeLongColumn() == other.hasSomeLongColumn());
+ if (hasSomeLongColumn() != other.hasSomeLongColumn()) return false;
if (hasSomeLongColumn()) {
- result = result && (getSomeLongColumn()
- == other.getSomeLongColumn());
+ if (getSomeLongColumn()
+ != other.getSomeLongColumn()) return false;
}
- result = result && (hasFoo() == other.hasFoo());
+ if (hasFoo() != other.hasFoo()) return false;
if (hasFoo()) {
- result = result && getFoo()
- .equals(other.getFoo());
+ if (!getFoo()
+ .equals(other.getFoo())) return false;
}
- result = result && getBarList()
- .equals(other.getBarList());
- result = result && unknownFields.equals(other.unknownFields);
- return result;
+ if (!getBarList()
+ .equals(other.getBarList())) return false;
+ if (hasOtherTimestamp() != other.hasOtherTimestamp()) return false;
+ if (hasOtherTimestamp()) {
+ if (!getOtherTimestamp()
+ .equals(other.getOtherTimestamp())) return false;
+ }
+ if (!unknownFields.equals(other.unknownFields)) return false;
+ return true;
}
@java.lang.Override
- public int hashCode()
- {
+ public int hashCode() {
if (memoizedHashCode != 0) {
return memoizedHashCode;
}
@@ -1763,175 +1677,145 @@ public final class ProtoTestEventWrapper
hash = (37 * hash) + BAR_FIELD_NUMBER;
hash = (53 * hash) + getBarList().hashCode();
}
+ if (hasOtherTimestamp()) {
+ hash = (37 * hash) + OTHERTIMESTAMP_FIELD_NUMBER;
+ hash = (53 * hash) + getOtherTimestamp().hashCode();
+ }
hash = (29 * hash) + unknownFields.hashCode();
memoizedHashCode = hash;
return hash;
}
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
- com.google.protobuf.ByteString data
- )
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ java.nio.ByteBuffer data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
+ java.nio.ByteBuffer data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data, extensionRegistry);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(java.io.InputStream input)
- throws java.io.IOException
- {
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws java.io.IOException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input, extensionRegistry);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException
- {
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseDelimitedWithIOException(PARSER, input);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(
java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws java.io.IOException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseDelimitedWithIOException(PARSER, input, extensionRegistry);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
- com.google.protobuf.CodedInputStream input
- )
- throws java.io.IOException
- {
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input);
}
-
public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws java.io.IOException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
return com.google.protobuf.GeneratedMessageV3
.parseWithIOException(PARSER, input, extensionRegistry);
}
- public Builder newBuilderForType()
- {
- return newBuilder();
- }
-
- public static Builder newBuilder()
- {
+ @java.lang.Override
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder() {
return DEFAULT_INSTANCE.toBuilder();
}
-
- public static Builder newBuilder(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent prototype)
- {
+ public static Builder newBuilder(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent prototype) {
return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
}
-
- public Builder toBuilder()
- {
+ @java.lang.Override
+ public Builder toBuilder() {
return this == DEFAULT_INSTANCE
- ? new Builder() : new Builder().mergeFrom(this);
+ ? new Builder() : new Builder().mergeFrom(this);
}
@java.lang.Override
protected Builder newBuilderForType(
- com.google.protobuf.GeneratedMessageV3.BuilderParent parent
- )
- {
+ com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
Builder builder = new Builder(parent);
return builder;
}
-
/**
* Protobuf type {@code prototest.ProtoTestEvent}
*/
public static final class Builder extends
com.google.protobuf.GeneratedMessageV3.Builder implements
// @@protoc_insertion_point(builder_implements:prototest.ProtoTestEvent)
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEventOrBuilder
- {
- public static com.google.protobuf.Descriptors.Descriptor
- getDescriptor()
- {
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEventOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor;
}
+ @java.lang.Override
protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
- internalGetFieldAccessorTable()
- {
+ internalGetFieldAccessorTable() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.class,
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Builder.class
- );
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.class, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Builder.class);
}
// Construct using org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
- private Builder()
- {
+ private Builder() {
maybeForceBuilderInitialization();
}
private Builder(
- com.google.protobuf.GeneratedMessageV3.BuilderParent parent
- )
- {
+ com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
super(parent);
maybeForceBuilderInitialization();
}
-
- private void maybeForceBuilderInitialization()
- {
+ private void maybeForceBuilderInitialization() {
if (com.google.protobuf.GeneratedMessageV3
- .alwaysUseFieldBuilders) {
+ .alwaysUseFieldBuilders) {
getFooFieldBuilder();
getBarFieldBuilder();
+ getOtherTimestampFieldBuilder();
}
}
-
- public Builder clear()
- {
+ @java.lang.Override
+ public Builder clear() {
super.clear();
eventType_ = 0;
bitField0_ = (bitField0_ & ~0x00000001);
@@ -1963,22 +1847,28 @@ public final class ProtoTestEventWrapper
} else {
barBuilder_.clear();
}
+ if (otherTimestampBuilder_ == null) {
+ otherTimestamp_ = null;
+ } else {
+ otherTimestampBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000800);
return this;
}
+ @java.lang.Override
public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType()
- {
+ getDescriptorForType() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor;
}
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType()
- {
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() {
return org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance();
}
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent build()
- {
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent build() {
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(result);
@@ -1986,58 +1876,57 @@ public final class ProtoTestEventWrapper
return result;
}
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent buildPartial()
- {
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent result = new org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent(
- this);
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent buildPartial() {
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent result = new org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent(this);
int from_bitField0_ = bitField0_;
int to_bitField0_ = 0;
- if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ if (((from_bitField0_ & 0x00000001) != 0)) {
to_bitField0_ |= 0x00000001;
}
result.eventType_ = eventType_;
- if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ if (((from_bitField0_ & 0x00000002) != 0)) {
+ result.id_ = id_;
to_bitField0_ |= 0x00000002;
}
- result.id_ = id_;
- if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ if (((from_bitField0_ & 0x00000004) != 0)) {
to_bitField0_ |= 0x00000004;
}
result.timestamp_ = timestamp_;
- if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ if (((from_bitField0_ & 0x00000008) != 0)) {
+ result.someOtherId_ = someOtherId_;
to_bitField0_ |= 0x00000008;
}
- result.someOtherId_ = someOtherId_;
- if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+ if (((from_bitField0_ & 0x00000010) != 0)) {
+ result.isValid_ = isValid_;
to_bitField0_ |= 0x00000010;
}
- result.isValid_ = isValid_;
- if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+ if (((from_bitField0_ & 0x00000020) != 0)) {
to_bitField0_ |= 0x00000020;
}
result.description_ = description_;
- if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+ if (((from_bitField0_ & 0x00000040) != 0)) {
+ result.someFloatColumn_ = someFloatColumn_;
to_bitField0_ |= 0x00000040;
}
- result.someFloatColumn_ = someFloatColumn_;
- if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+ if (((from_bitField0_ & 0x00000080) != 0)) {
+ result.someIntColumn_ = someIntColumn_;
to_bitField0_ |= 0x00000080;
}
- result.someIntColumn_ = someIntColumn_;
- if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+ if (((from_bitField0_ & 0x00000100) != 0)) {
+ result.someLongColumn_ = someLongColumn_;
to_bitField0_ |= 0x00000100;
}
- result.someLongColumn_ = someLongColumn_;
- if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
+ if (((from_bitField0_ & 0x00000200) != 0)) {
+ if (fooBuilder_ == null) {
+ result.foo_ = foo_;
+ } else {
+ result.foo_ = fooBuilder_.build();
+ }
to_bitField0_ |= 0x00000200;
}
- if (fooBuilder_ == null) {
- result.foo_ = foo_;
- } else {
- result.foo_ = fooBuilder_.build();
- }
if (barBuilder_ == null) {
- if (((bitField0_ & 0x00000400) == 0x00000400)) {
+ if (((bitField0_ & 0x00000400) != 0)) {
bar_ = java.util.Collections.unmodifiableList(bar_);
bitField0_ = (bitField0_ & ~0x00000400);
}
@@ -2045,69 +1934,63 @@ public final class ProtoTestEventWrapper
} else {
result.bar_ = barBuilder_.build();
}
+ if (((from_bitField0_ & 0x00000800) != 0)) {
+ if (otherTimestampBuilder_ == null) {
+ result.otherTimestamp_ = otherTimestamp_;
+ } else {
+ result.otherTimestamp_ = otherTimestampBuilder_.build();
+ }
+ to_bitField0_ |= 0x00000400;
+ }
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
}
- public Builder clone()
- {
- return (Builder) super.clone();
+ @java.lang.Override
+ public Builder clone() {
+ return super.clone();
}
-
+ @java.lang.Override
public Builder setField(
com.google.protobuf.Descriptors.FieldDescriptor field,
- Object value
- )
- {
- return (Builder) super.setField(field, value);
+ java.lang.Object value) {
+ return super.setField(field, value);
}
-
+ @java.lang.Override
public Builder clearField(
- com.google.protobuf.Descriptors.FieldDescriptor field
- )
- {
- return (Builder) super.clearField(field);
+ com.google.protobuf.Descriptors.FieldDescriptor field) {
+ return super.clearField(field);
}
-
+ @java.lang.Override
public Builder clearOneof(
- com.google.protobuf.Descriptors.OneofDescriptor oneof
- )
- {
- return (Builder) super.clearOneof(oneof);
+ com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+ return super.clearOneof(oneof);
}
-
+ @java.lang.Override
public Builder setRepeatedField(
com.google.protobuf.Descriptors.FieldDescriptor field,
- int index, Object value
- )
- {
- return (Builder) super.setRepeatedField(field, index, value);
+ int index, java.lang.Object value) {
+ return super.setRepeatedField(field, index, value);
}
-
+ @java.lang.Override
public Builder addRepeatedField(
com.google.protobuf.Descriptors.FieldDescriptor field,
- Object value
- )
- {
- return (Builder) super.addRepeatedField(field, value);
+ java.lang.Object value) {
+ return super.addRepeatedField(field, value);
}
-
- public Builder mergeFrom(com.google.protobuf.Message other)
- {
+ @java.lang.Override
+ public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) {
- return mergeFrom((org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) other);
+ return mergeFrom((org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent)other);
} else {
super.mergeFrom(other);
return this;
}
}
- public Builder mergeFrom(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent other)
- {
- if (other == org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance()) {
- return this;
- }
+ public Builder mergeFrom(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent other) {
+ if (other == org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance()) return this;
if (other.hasEventType()) {
setEventType(other.getEventType());
}
@@ -2160,21 +2043,24 @@ public final class ProtoTestEventWrapper
barBuilder_ = null;
bar_ = other.bar_;
bitField0_ = (bitField0_ & ~0x00000400);
- barBuilder_ =
- com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
- getBarFieldBuilder() : null;
+ barBuilder_ =
+ com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+ getBarFieldBuilder() : null;
} else {
barBuilder_.addAllMessages(other.bar_);
}
}
}
+ if (other.hasOtherTimestamp()) {
+ mergeOtherTimestamp(other.getOtherTimestamp());
+ }
this.mergeUnknownFields(other.unknownFields);
onChanged();
return this;
}
- public boolean isInitialized()
- {
+ @java.lang.Override
+ public final boolean isInitialized() {
if (!hasEventType()) {
return false;
}
@@ -2197,57 +2083,50 @@ public final class ProtoTestEventWrapper
return true;
}
+ @java.lang.Override
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws java.io.IOException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parsedMessage = null;
try {
parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
- }
- catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
parsedMessage = (org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) e.getUnfinishedMessage();
throw e.unwrapIOException();
- }
- finally {
+ } finally {
if (parsedMessage != null) {
mergeFrom(parsedMessage);
}
}
return this;
}
-
private int bitField0_;
private int eventType_ = 0;
-
/**
* required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ * @return Whether the eventType field is set.
*/
- public boolean hasEventType()
- {
- return ((bitField0_ & 0x00000001) == 0x00000001);
+ @java.lang.Override public boolean hasEventType() {
+ return ((bitField0_ & 0x00000001) != 0);
}
-
/**
* required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ * @return The eventType.
*/
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType()
- {
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory result = org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory
- .valueOf(eventType_);
- return result == null
- ? org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO
- : result;
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() {
+ @SuppressWarnings("deprecation")
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory result = org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.valueOf(eventType_);
+ return result == null ? org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO : result;
}
-
/**
* required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ * @param value The eventType to set.
+ * @return This builder for chaining.
*/
- public Builder setEventType(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value)
- {
+ public Builder setEventType(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value) {
if (value == null) {
throw new NullPointerException();
}
@@ -2256,52 +2135,50 @@ public final class ProtoTestEventWrapper
onChanged();
return this;
}
-
/**
* required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ * @return This builder for chaining.
*/
- public Builder clearEventType()
- {
+ public Builder clearEventType() {
bitField0_ = (bitField0_ & ~0x00000001);
eventType_ = 0;
onChanged();
return this;
}
- private long id_;
-
+ private long id_ ;
/**
* required uint64 id = 2;
+ * @return Whether the id field is set.
*/
- public boolean hasId()
- {
- return ((bitField0_ & 0x00000002) == 0x00000002);
+ @java.lang.Override
+ public boolean hasId() {
+ return ((bitField0_ & 0x00000002) != 0);
}
-
/**
* required uint64 id = 2;
+ * @return The id.
*/
- public long getId()
- {
+ @java.lang.Override
+ public long getId() {
return id_;
}
-
/**
* required uint64 id = 2;
+ * @param value The id to set.
+ * @return This builder for chaining.
*/
- public Builder setId(long value)
- {
+ public Builder setId(long value) {
bitField0_ |= 0x00000002;
id_ = value;
onChanged();
return this;
}
-
/**
* required uint64 id = 2;
+ * @return This builder for chaining.
*/
- public Builder clearId()
- {
+ public Builder clearId() {
bitField0_ = (bitField0_ & ~0x00000002);
id_ = 0L;
onChanged();
@@ -2309,20 +2186,18 @@ public final class ProtoTestEventWrapper
}
private java.lang.Object timestamp_ = "";
-
/**
* required string timestamp = 3;
+ * @return Whether the timestamp field is set.
*/
- public boolean hasTimestamp()
- {
- return ((bitField0_ & 0x00000004) == 0x00000004);
+ public boolean hasTimestamp() {
+ return ((bitField0_ & 0x00000004) != 0);
}
-
/**
* required string timestamp = 3;
+ * @return The timestamp.
*/
- public java.lang.String getTimestamp()
- {
+ public java.lang.String getTimestamp() {
java.lang.Object ref = timestamp_;
if (!(ref instanceof java.lang.String)) {
com.google.protobuf.ByteString bs =
@@ -2336,16 +2211,15 @@ public final class ProtoTestEventWrapper
return (java.lang.String) ref;
}
}
-
/**
* required string timestamp = 3;
+ * @return The bytes for timestamp.
*/
public com.google.protobuf.ByteString
- getTimestampBytes()
- {
+ getTimestampBytes() {
java.lang.Object ref = timestamp_;
if (ref instanceof String) {
- com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8(
(java.lang.String) ref);
timestamp_ = b;
@@ -2354,124 +2228,119 @@ public final class ProtoTestEventWrapper
return (com.google.protobuf.ByteString) ref;
}
}
-
/**
* required string timestamp = 3;
+ * @param value The timestamp to set.
+ * @return This builder for chaining.
*/
public Builder setTimestamp(
- java.lang.String value
- )
- {
+ java.lang.String value) {
if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000004;
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000004;
timestamp_ = value;
onChanged();
return this;
}
-
/**
* required string timestamp = 3;
+ * @return This builder for chaining.
*/
- public Builder clearTimestamp()
- {
+ public Builder clearTimestamp() {
bitField0_ = (bitField0_ & ~0x00000004);
timestamp_ = getDefaultInstance().getTimestamp();
onChanged();
return this;
}
-
/**
* required string timestamp = 3;
+ * @param value The bytes for timestamp to set.
+ * @return This builder for chaining.
*/
public Builder setTimestampBytes(
- com.google.protobuf.ByteString value
- )
- {
+ com.google.protobuf.ByteString value) {
if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000004;
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000004;
timestamp_ = value;
onChanged();
return this;
}
- private int someOtherId_;
-
+ private int someOtherId_ ;
/**
* optional uint32 someOtherId = 4;
+ * @return Whether the someOtherId field is set.
*/
- public boolean hasSomeOtherId()
- {
- return ((bitField0_ & 0x00000008) == 0x00000008);
+ @java.lang.Override
+ public boolean hasSomeOtherId() {
+ return ((bitField0_ & 0x00000008) != 0);
}
-
/**
* optional uint32 someOtherId = 4;
+ * @return The someOtherId.
*/
- public int getSomeOtherId()
- {
+ @java.lang.Override
+ public int getSomeOtherId() {
return someOtherId_;
}
-
/**
* optional uint32 someOtherId = 4;
+ * @param value The someOtherId to set.
+ * @return This builder for chaining.
*/
- public Builder setSomeOtherId(int value)
- {
+ public Builder setSomeOtherId(int value) {
bitField0_ |= 0x00000008;
someOtherId_ = value;
onChanged();
return this;
}
-
/**
* optional uint32 someOtherId = 4;
+ * @return This builder for chaining.
*/
- public Builder clearSomeOtherId()
- {
+ public Builder clearSomeOtherId() {
bitField0_ = (bitField0_ & ~0x00000008);
someOtherId_ = 0;
onChanged();
return this;
}
- private boolean isValid_;
-
+ private boolean isValid_ ;
/**
* optional bool isValid = 5;
+ * @return Whether the isValid field is set.
*/
- public boolean hasIsValid()
- {
- return ((bitField0_ & 0x00000010) == 0x00000010);
+ @java.lang.Override
+ public boolean hasIsValid() {
+ return ((bitField0_ & 0x00000010) != 0);
}
-
/**
* optional bool isValid = 5;
+ * @return The isValid.
*/
- public boolean getIsValid()
- {
+ @java.lang.Override
+ public boolean getIsValid() {
return isValid_;
}
-
/**
* optional bool isValid = 5;
+ * @param value The isValid to set.
+ * @return This builder for chaining.
*/
- public Builder setIsValid(boolean value)
- {
+ public Builder setIsValid(boolean value) {
bitField0_ |= 0x00000010;
isValid_ = value;
onChanged();
return this;
}
-
/**
* optional bool isValid = 5;
+ * @return This builder for chaining.
*/
- public Builder clearIsValid()
- {
+ public Builder clearIsValid() {
bitField0_ = (bitField0_ & ~0x00000010);
isValid_ = false;
onChanged();
@@ -2479,20 +2348,18 @@ public final class ProtoTestEventWrapper
}
private java.lang.Object description_ = "";
-
/**
* optional string description = 6;
+ * @return Whether the description field is set.
*/
- public boolean hasDescription()
- {
- return ((bitField0_ & 0x00000020) == 0x00000020);
+ public boolean hasDescription() {
+ return ((bitField0_ & 0x00000020) != 0);
}
-
/**
* optional string description = 6;
+ * @return The description.
*/
- public java.lang.String getDescription()
- {
+ public java.lang.String getDescription() {
java.lang.Object ref = description_;
if (!(ref instanceof java.lang.String)) {
com.google.protobuf.ByteString bs =
@@ -2506,16 +2373,15 @@ public final class ProtoTestEventWrapper
return (java.lang.String) ref;
}
}
-
/**
* optional string description = 6;
+ * @return The bytes for description.
*/
public com.google.protobuf.ByteString
- getDescriptionBytes()
- {
+ getDescriptionBytes() {
java.lang.Object ref = description_;
if (ref instanceof String) {
- com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8(
(java.lang.String) ref);
description_ = b;
@@ -2524,201 +2390,189 @@ public final class ProtoTestEventWrapper
return (com.google.protobuf.ByteString) ref;
}
}
-
/**
* optional string description = 6;
+ * @param value The description to set.
+ * @return This builder for chaining.
*/
public Builder setDescription(
- java.lang.String value
- )
- {
+ java.lang.String value) {
if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000020;
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000020;
description_ = value;
onChanged();
return this;
}
-
/**
* optional string description = 6;
+ * @return This builder for chaining.
*/
- public Builder clearDescription()
- {
+ public Builder clearDescription() {
bitField0_ = (bitField0_ & ~0x00000020);
description_ = getDefaultInstance().getDescription();
onChanged();
return this;
}
-
/**
* optional string description = 6;
+ * @param value The bytes for description to set.
+ * @return This builder for chaining.
*/
public Builder setDescriptionBytes(
- com.google.protobuf.ByteString value
- )
- {
+ com.google.protobuf.ByteString value) {
if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000020;
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000020;
description_ = value;
onChanged();
return this;
}
- private float someFloatColumn_;
-
+ private float someFloatColumn_ ;
/**
* optional float someFloatColumn = 7;
+ * @return Whether the someFloatColumn field is set.
*/
- public boolean hasSomeFloatColumn()
- {
- return ((bitField0_ & 0x00000040) == 0x00000040);
+ @java.lang.Override
+ public boolean hasSomeFloatColumn() {
+ return ((bitField0_ & 0x00000040) != 0);
}
-
/**
* optional float someFloatColumn = 7;
+ * @return The someFloatColumn.
*/
- public float getSomeFloatColumn()
- {
+ @java.lang.Override
+ public float getSomeFloatColumn() {
return someFloatColumn_;
}
-
/**
* optional float someFloatColumn = 7;
+ * @param value The someFloatColumn to set.
+ * @return This builder for chaining.
*/
- public Builder setSomeFloatColumn(float value)
- {
+ public Builder setSomeFloatColumn(float value) {
bitField0_ |= 0x00000040;
someFloatColumn_ = value;
onChanged();
return this;
}
-
/**
* optional float someFloatColumn = 7;
+ * @return This builder for chaining.
*/
- public Builder clearSomeFloatColumn()
- {
+ public Builder clearSomeFloatColumn() {
bitField0_ = (bitField0_ & ~0x00000040);
someFloatColumn_ = 0F;
onChanged();
return this;
}
- private int someIntColumn_;
-
+ private int someIntColumn_ ;
/**
* optional uint32 someIntColumn = 8;
+ * @return Whether the someIntColumn field is set.
*/
- public boolean hasSomeIntColumn()
- {
- return ((bitField0_ & 0x00000080) == 0x00000080);
+ @java.lang.Override
+ public boolean hasSomeIntColumn() {
+ return ((bitField0_ & 0x00000080) != 0);
}
-
/**
* optional uint32 someIntColumn = 8;
+ * @return The someIntColumn.
*/
- public int getSomeIntColumn()
- {
+ @java.lang.Override
+ public int getSomeIntColumn() {
return someIntColumn_;
}
-
/**
* optional uint32 someIntColumn = 8;
+ * @param value The someIntColumn to set.
+ * @return This builder for chaining.
*/
- public Builder setSomeIntColumn(int value)
- {
+ public Builder setSomeIntColumn(int value) {
bitField0_ |= 0x00000080;
someIntColumn_ = value;
onChanged();
return this;
}
-
/**
* optional uint32 someIntColumn = 8;
+ * @return This builder for chaining.
*/
- public Builder clearSomeIntColumn()
- {
+ public Builder clearSomeIntColumn() {
bitField0_ = (bitField0_ & ~0x00000080);
someIntColumn_ = 0;
onChanged();
return this;
}
- private long someLongColumn_;
-
+ private long someLongColumn_ ;
/**
* optional uint64 someLongColumn = 9;
+ * @return Whether the someLongColumn field is set.
*/
- public boolean hasSomeLongColumn()
- {
- return ((bitField0_ & 0x00000100) == 0x00000100);
+ @java.lang.Override
+ public boolean hasSomeLongColumn() {
+ return ((bitField0_ & 0x00000100) != 0);
}
-
/**
* optional uint64 someLongColumn = 9;
+ * @return The someLongColumn.
*/
- public long getSomeLongColumn()
- {
+ @java.lang.Override
+ public long getSomeLongColumn() {
return someLongColumn_;
}
-
/**
* optional uint64 someLongColumn = 9;
+ * @param value The someLongColumn to set.
+ * @return This builder for chaining.
*/
- public Builder setSomeLongColumn(long value)
- {
+ public Builder setSomeLongColumn(long value) {
bitField0_ |= 0x00000100;
someLongColumn_ = value;
onChanged();
return this;
}
-
/**
* optional uint64 someLongColumn = 9;
+ * @return This builder for chaining.
*/
- public Builder clearSomeLongColumn()
- {
+ public Builder clearSomeLongColumn() {
bitField0_ = (bitField0_ & ~0x00000100);
someLongColumn_ = 0L;
onChanged();
return this;
}
- private org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo foo_ = null;
+ private org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo foo_;
private com.google.protobuf.SingleFieldBuilderV3<
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder> fooBuilder_;
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
+ * @return Whether the foo field is set.
*/
- public boolean hasFoo()
- {
- return ((bitField0_ & 0x00000200) == 0x00000200);
+ public boolean hasFoo() {
+ return ((bitField0_ & 0x00000200) != 0);
}
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
+ * @return The foo.
*/
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getFoo()
- {
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getFoo() {
if (fooBuilder_ == null) {
- return foo_ == null
- ? org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()
- : foo_;
+ return foo_ == null ? org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance() : foo_;
} else {
return fooBuilder_.getMessage();
}
}
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
*/
- public Builder setFoo(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value)
- {
+ public Builder setFoo(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value) {
if (fooBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
@@ -2731,14 +2585,11 @@ public final class ProtoTestEventWrapper
bitField0_ |= 0x00000200;
return this;
}
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
*/
public Builder setFoo(
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue
- )
- {
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue) {
if (fooBuilder_ == null) {
foo_ = builderForValue.build();
onChanged();
@@ -2748,20 +2599,16 @@ public final class ProtoTestEventWrapper
bitField0_ |= 0x00000200;
return this;
}
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
*/
- public Builder mergeFoo(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value)
- {
+ public Builder mergeFoo(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value) {
if (fooBuilder_ == null) {
- if (((bitField0_ & 0x00000200) == 0x00000200) &&
+ if (((bitField0_ & 0x00000200) != 0) &&
foo_ != null &&
foo_ != org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()) {
foo_ =
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.newBuilder(foo_)
- .mergeFrom(value)
- .buildPartial();
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.newBuilder(foo_).mergeFrom(value).buildPartial();
} else {
foo_ = value;
}
@@ -2772,12 +2619,10 @@ public final class ProtoTestEventWrapper
bitField0_ |= 0x00000200;
return this;
}
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
*/
- public Builder clearFoo()
- {
+ public Builder clearFoo() {
if (fooBuilder_ == null) {
foo_ = null;
onChanged();
@@ -2787,58 +2632,49 @@ public final class ProtoTestEventWrapper
bitField0_ = (bitField0_ & ~0x00000200);
return this;
}
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
*/
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder getFooBuilder()
- {
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder getFooBuilder() {
bitField0_ |= 0x00000200;
onChanged();
return getFooFieldBuilder().getBuilder();
}
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
*/
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getFooOrBuilder()
- {
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getFooOrBuilder() {
if (fooBuilder_ != null) {
return fooBuilder_.getMessageOrBuilder();
} else {
return foo_ == null ?
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance() : foo_;
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance() : foo_;
}
}
-
/**
* optional .prototest.ProtoTestEvent.Foo foo = 10;
*/
private com.google.protobuf.SingleFieldBuilderV3<
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
- getFooFieldBuilder()
- {
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
+ getFooFieldBuilder() {
if (fooBuilder_ == null) {
fooBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>(
- getFoo(),
- getParentForChildren(),
- isClean()
- );
+ getFoo(),
+ getParentForChildren(),
+ isClean());
foo_ = null;
}
return fooBuilder_;
}
private java.util.List bar_ =
- java.util.Collections.emptyList();
-
- private void ensureBarIsMutable()
- {
- if (!((bitField0_ & 0x00000400) == 0x00000400)) {
+ java.util.Collections.emptyList();
+ private void ensureBarIsMutable() {
+ if (!((bitField0_ & 0x00000400) != 0)) {
bar_ = new java.util.ArrayList(bar_);
bitField0_ |= 0x00000400;
- }
+ }
}
private com.google.protobuf.RepeatedFieldBuilderV3<
@@ -2847,46 +2683,38 @@ public final class ProtoTestEventWrapper
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public java.util.List getBarList()
- {
+ public java.util.List getBarList() {
if (barBuilder_ == null) {
return java.util.Collections.unmodifiableList(bar_);
} else {
return barBuilder_.getMessageList();
}
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public int getBarCount()
- {
+ public int getBarCount() {
if (barBuilder_ == null) {
return bar_.size();
} else {
return barBuilder_.getCount();
}
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getBar(int index)
- {
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getBar(int index) {
if (barBuilder_ == null) {
return bar_.get(index);
} else {
return barBuilder_.getMessage(index);
}
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
public Builder setBar(
- int index, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value
- )
- {
+ int index, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value) {
if (barBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
@@ -2899,14 +2727,11 @@ public final class ProtoTestEventWrapper
}
return this;
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
public Builder setBar(
- int index, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue
- )
- {
+ int index, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue) {
if (barBuilder_ == null) {
ensureBarIsMutable();
bar_.set(index, builderForValue.build());
@@ -2916,12 +2741,10 @@ public final class ProtoTestEventWrapper
}
return this;
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public Builder addBar(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value)
- {
+ public Builder addBar(org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value) {
if (barBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
@@ -2934,14 +2757,11 @@ public final class ProtoTestEventWrapper
}
return this;
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
public Builder addBar(
- int index, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value
- )
- {
+ int index, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value) {
if (barBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
@@ -2954,14 +2774,11 @@ public final class ProtoTestEventWrapper
}
return this;
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
public Builder addBar(
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue
- )
- {
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue) {
if (barBuilder_ == null) {
ensureBarIsMutable();
bar_.add(builderForValue.build());
@@ -2971,14 +2788,11 @@ public final class ProtoTestEventWrapper
}
return this;
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
public Builder addBar(
- int index, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue
- )
- {
+ int index, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue) {
if (barBuilder_ == null) {
ensureBarIsMutable();
bar_.add(index, builderForValue.build());
@@ -2988,14 +2802,11 @@ public final class ProtoTestEventWrapper
}
return this;
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
public Builder addAllBar(
- java.lang.Iterable extends org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo> values
- )
- {
+ java.lang.Iterable extends org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo> values) {
if (barBuilder_ == null) {
ensureBarIsMutable();
com.google.protobuf.AbstractMessageLite.Builder.addAll(
@@ -3006,12 +2817,10 @@ public final class ProtoTestEventWrapper
}
return this;
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public Builder clearBar()
- {
+ public Builder clearBar() {
if (barBuilder_ == null) {
bar_ = java.util.Collections.emptyList();
bitField0_ = (bitField0_ & ~0x00000400);
@@ -3021,12 +2830,10 @@ public final class ProtoTestEventWrapper
}
return this;
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public Builder removeBar(int index)
- {
+ public Builder removeBar(int index) {
if (barBuilder_ == null) {
ensureBarIsMutable();
bar_.remove(index);
@@ -3036,101 +2843,199 @@ public final class ProtoTestEventWrapper
}
return this;
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder getBarBuilder(
- int index
- )
- {
+ int index) {
return getBarFieldBuilder().getBuilder(index);
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getBarOrBuilder(
- int index
- )
- {
+ int index) {
if (barBuilder_ == null) {
- return bar_.get(index);
- } else {
+ return bar_.get(index); } else {
return barBuilder_.getMessageOrBuilder(index);
}
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public java.util.List extends org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
- getBarOrBuilderList()
- {
+ public java.util.List extends org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
+ getBarOrBuilderList() {
if (barBuilder_ != null) {
return barBuilder_.getMessageOrBuilderList();
} else {
return java.util.Collections.unmodifiableList(bar_);
}
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder addBarBuilder()
- {
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder addBarBuilder() {
return getBarFieldBuilder().addBuilder(
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance());
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder addBarBuilder(
- int index
- )
- {
+ int index) {
return getBarFieldBuilder().addBuilder(
index, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance());
}
-
/**
* repeated .prototest.ProtoTestEvent.Foo bar = 11;
*/
- public java.util.List
- getBarBuilderList()
- {
+ public java.util.List
+ getBarBuilderList() {
return getBarFieldBuilder().getBuilderList();
}
-
private com.google.protobuf.RepeatedFieldBuilderV3<
- org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
- getBarFieldBuilder()
- {
+ org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
+ getBarFieldBuilder() {
if (barBuilder_ == null) {
barBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>(
- bar_,
- ((bitField0_ & 0x00000400) == 0x00000400),
- getParentForChildren(),
- isClean()
- );
+ bar_,
+ ((bitField0_ & 0x00000400) != 0),
+ getParentForChildren(),
+ isClean());
bar_ = null;
}
return barBuilder_;
}
- public Builder setUnknownFields(
- final com.google.protobuf.UnknownFieldSet unknownFields
- )
- {
+ private com.google.protobuf.Timestamp otherTimestamp_;
+ private com.google.protobuf.SingleFieldBuilderV3<
+ com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder> otherTimestampBuilder_;
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ * @return Whether the otherTimestamp field is set.
+ */
+ public boolean hasOtherTimestamp() {
+ return ((bitField0_ & 0x00000800) != 0);
+ }
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ * @return The otherTimestamp.
+ */
+ public com.google.protobuf.Timestamp getOtherTimestamp() {
+ if (otherTimestampBuilder_ == null) {
+ return otherTimestamp_ == null ? com.google.protobuf.Timestamp.getDefaultInstance() : otherTimestamp_;
+ } else {
+ return otherTimestampBuilder_.getMessage();
+ }
+ }
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ */
+ public Builder setOtherTimestamp(com.google.protobuf.Timestamp value) {
+ if (otherTimestampBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ otherTimestamp_ = value;
+ onChanged();
+ } else {
+ otherTimestampBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000800;
+ return this;
+ }
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ */
+ public Builder setOtherTimestamp(
+ com.google.protobuf.Timestamp.Builder builderForValue) {
+ if (otherTimestampBuilder_ == null) {
+ otherTimestamp_ = builderForValue.build();
+ onChanged();
+ } else {
+ otherTimestampBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000800;
+ return this;
+ }
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ */
+ public Builder mergeOtherTimestamp(com.google.protobuf.Timestamp value) {
+ if (otherTimestampBuilder_ == null) {
+ if (((bitField0_ & 0x00000800) != 0) &&
+ otherTimestamp_ != null &&
+ otherTimestamp_ != com.google.protobuf.Timestamp.getDefaultInstance()) {
+ otherTimestamp_ =
+ com.google.protobuf.Timestamp.newBuilder(otherTimestamp_).mergeFrom(value).buildPartial();
+ } else {
+ otherTimestamp_ = value;
+ }
+ onChanged();
+ } else {
+ otherTimestampBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000800;
+ return this;
+ }
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ */
+ public Builder clearOtherTimestamp() {
+ if (otherTimestampBuilder_ == null) {
+ otherTimestamp_ = null;
+ onChanged();
+ } else {
+ otherTimestampBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000800);
+ return this;
+ }
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ */
+ public com.google.protobuf.Timestamp.Builder getOtherTimestampBuilder() {
+ bitField0_ |= 0x00000800;
+ onChanged();
+ return getOtherTimestampFieldBuilder().getBuilder();
+ }
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ */
+ public com.google.protobuf.TimestampOrBuilder getOtherTimestampOrBuilder() {
+ if (otherTimestampBuilder_ != null) {
+ return otherTimestampBuilder_.getMessageOrBuilder();
+ } else {
+ return otherTimestamp_ == null ?
+ com.google.protobuf.Timestamp.getDefaultInstance() : otherTimestamp_;
+ }
+ }
+ /**
+ * optional .google.protobuf.Timestamp otherTimestamp = 12;
+ */
+ private com.google.protobuf.SingleFieldBuilderV3<
+ com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>
+ getOtherTimestampFieldBuilder() {
+ if (otherTimestampBuilder_ == null) {
+ otherTimestampBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+ com.google.protobuf.Timestamp, com.google.protobuf.Timestamp.Builder, com.google.protobuf.TimestampOrBuilder>(
+ getOtherTimestamp(),
+ getParentForChildren(),
+ isClean());
+ otherTimestamp_ = null;
+ }
+ return otherTimestampBuilder_;
+ }
+ @java.lang.Override
+ public final Builder setUnknownFields(
+ final com.google.protobuf.UnknownFieldSet unknownFields) {
return super.setUnknownFields(unknownFields);
}
- public Builder mergeUnknownFields(
- final com.google.protobuf.UnknownFieldSet unknownFields
- )
- {
+ @java.lang.Override
+ public final Builder mergeUnknownFields(
+ final com.google.protobuf.UnknownFieldSet unknownFields) {
return super.mergeUnknownFields(unknownFields);
}
@@ -3140,126 +3045,94 @@ public final class ProtoTestEventWrapper
// @@protoc_insertion_point(class_scope:prototest.ProtoTestEvent)
private static final org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent DEFAULT_INSTANCE;
-
static {
DEFAULT_INSTANCE = new org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent();
}
- public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstance()
- {
+ public static org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstance() {
return DEFAULT_INSTANCE;
}
- @java.lang.Deprecated
- public static final com.google.protobuf.Parser
- PARSER = new com.google.protobuf.AbstractParser()
- {
+ @java.lang.Deprecated public static final com.google.protobuf.Parser
+ PARSER = new com.google.protobuf.AbstractParser() {
+ @java.lang.Override
public ProtoTestEvent parsePartialFrom(
com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry
- )
- throws com.google.protobuf.InvalidProtocolBufferException
- {
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
return new ProtoTestEvent(input, extensionRegistry);
}
};
- public static com.google.protobuf.Parser parser()
- {
+ public static com.google.protobuf.Parser parser() {
return PARSER;
}
@java.lang.Override
- public com.google.protobuf.Parser getParserForType()
- {
+ public com.google.protobuf.Parser getParserForType() {
return PARSER;
}
- public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType()
- {
+ @java.lang.Override
+ public org.apache.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() {
return DEFAULT_INSTANCE;
}
}
private static final com.google.protobuf.Descriptors.Descriptor
- internal_static_prototest_ProtoTestEvent_descriptor;
- private static final
- com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internal_static_prototest_ProtoTestEvent_descriptor;
+ private static final
+ com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
internal_static_prototest_ProtoTestEvent_fieldAccessorTable;
private static final com.google.protobuf.Descriptors.Descriptor
- internal_static_prototest_ProtoTestEvent_Foo_descriptor;
- private static final
- com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internal_static_prototest_ProtoTestEvent_Foo_descriptor;
+ private static final
+ com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
internal_static_prototest_ProtoTestEvent_Foo_fieldAccessorTable;
public static com.google.protobuf.Descriptors.FileDescriptor
- getDescriptor()
- {
+ getDescriptor() {
return descriptor;
}
-
- private static com.google.protobuf.Descriptors.FileDescriptor
+ private static com.google.protobuf.Descriptors.FileDescriptor
descriptor;
-
static {
java.lang.String[] descriptorData = {
- "\n\017ProtoTest.proto\022\tprototest\"\242\003\n\016ProtoTe" +
- "stEvent\022:\n\teventType\030\001 \002(\0162\'.prototest.P" +
- "rotoTestEvent.EventCategory\022\n\n\002id\030\002 \002(\004\022" +
- "\021\n\ttimestamp\030\003 \002(\t\022\023\n\013someOtherId\030\004 \001(\r\022" +
- "\017\n\007isValid\030\005 \001(\010\022\023\n\013description\030\006 \001(\t\022\027\n" +
- "\017someFloatColumn\030\007 \001(\002\022\025\n\rsomeIntColumn\030" +
- "\010 \001(\r\022\026\n\016someLongColumn\030\t \001(\004\022*\n\003foo\030\n \001" +
- "(\0132\035.prototest.ProtoTestEvent.Foo\022*\n\003bar" +
- "\030\013 \003(\0132\035.prototest.ProtoTestEvent.Foo\032\022\n" +
- "\003Foo\022\013\n\003bar\030\001 \002(\t\"F\n\rEventCategory\022\021\n\rCA",
- "TEGORY_ZERO\020\000\022\020\n\014CATEGORY_ONE\020\001\022\020\n\014CATEG" +
- "ORY_TWO\020\002B5\n\034org.apache.druid.data.input.protobu" +
- "fB\025ProtoTestEventWrapper"
+ "\n\017Prototest.proto\022\tprototest\032\037google/pro" +
+ "tobuf/timestamp.proto\"\326\003\n\016ProtoTestEvent" +
+ "\022:\n\teventType\030\001 \002(\0162\'.prototest.ProtoTes" +
+ "tEvent.EventCategory\022\n\n\002id\030\002 \002(\004\022\021\n\ttime" +
+ "stamp\030\003 \002(\t\022\023\n\013someOtherId\030\004 \001(\r\022\017\n\007isVa" +
+ "lid\030\005 \001(\010\022\023\n\013description\030\006 \001(\t\022\027\n\017someFl" +
+ "oatColumn\030\007 \001(\002\022\025\n\rsomeIntColumn\030\010 \001(\r\022\026" +
+ "\n\016someLongColumn\030\t \001(\004\022*\n\003foo\030\n \001(\0132\035.pr" +
+ "ototest.ProtoTestEvent.Foo\022*\n\003bar\030\013 \003(\0132" +
+ "\035.prototest.ProtoTestEvent.Foo\0222\n\016otherT" +
+ "imestamp\030\014 \001(\0132\032.google.protobuf.Timesta" +
+ "mp\032\022\n\003Foo\022\013\n\003bar\030\001 \002(\t\"F\n\rEventCategory\022" +
+ "\021\n\rCATEGORY_ZERO\020\000\022\020\n\014CATEGORY_ONE\020\001\022\020\n\014" +
+ "CATEGORY_TWO\020\002B=\n$org.apache.druid.data." +
+ "input.protobufB\025ProtoTestEventWrapper"
};
- com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
- new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner()
- {
- public com.google.protobuf.ExtensionRegistry assignDescriptors(
- com.google.protobuf.Descriptors.FileDescriptor root
- )
- {
- descriptor = root;
- return null;
- }
- };
- com.google.protobuf.Descriptors.FileDescriptor
- .internalBuildGeneratedFileFrom(descriptorData,
- new com.google.protobuf.Descriptors.FileDescriptor[]{
- }, assigner
- );
+ descriptor = com.google.protobuf.Descriptors.FileDescriptor
+ .internalBuildGeneratedFileFrom(descriptorData,
+ new com.google.protobuf.Descriptors.FileDescriptor[] {
+ com.google.protobuf.TimestampProto.getDescriptor(),
+ });
internal_static_prototest_ProtoTestEvent_descriptor =
- getDescriptor().getMessageTypes().get(0);
+ getDescriptor().getMessageTypes().get(0);
internal_static_prototest_ProtoTestEvent_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+ com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
internal_static_prototest_ProtoTestEvent_descriptor,
- new java.lang.String[]{
- "EventType",
- "Id",
- "Timestamp",
- "SomeOtherId",
- "IsValid",
- "Description",
- "SomeFloatColumn",
- "SomeIntColumn",
- "SomeLongColumn",
- "Foo",
- "Bar",
- }
- );
+ new java.lang.String[] { "EventType", "Id", "Timestamp", "SomeOtherId", "IsValid", "Description", "SomeFloatColumn", "SomeIntColumn", "SomeLongColumn", "Foo", "Bar", "OtherTimestamp", });
internal_static_prototest_ProtoTestEvent_Foo_descriptor =
- internal_static_prototest_ProtoTestEvent_descriptor.getNestedTypes().get(0);
+ internal_static_prototest_ProtoTestEvent_descriptor.getNestedTypes().get(0);
internal_static_prototest_ProtoTestEvent_Foo_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+ com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
internal_static_prototest_ProtoTestEvent_Foo_descriptor,
- new java.lang.String[]{"Bar"}
- );
+ new java.lang.String[] { "Bar", });
+ com.google.protobuf.TimestampProto.getDescriptor();
}
// @@protoc_insertion_point(outer_class_scope)
diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java
index c73f513dd90..e9e15ffe260 100644
--- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java
+++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java
@@ -41,10 +41,7 @@ import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
-import java.io.ByteArrayOutputStream;
import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
public class ProtobufInputFormatTest
{
@@ -119,49 +116,13 @@ public class ProtobufInputFormatTest
//create binary of proto test event
DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC());
- ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
- .setDescription("description")
- .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE)
- .setId(4711L)
- .setIsValid(true)
- .setSomeOtherId(4712)
- .setTimestamp(dateTime.toString())
- .setSomeFloatColumn(47.11F)
- .setSomeIntColumn(815)
- .setSomeLongColumn(816L)
- .setFoo(ProtoTestEventWrapper.ProtoTestEvent.Foo
- .newBuilder()
- .setBar("baz"))
- .addBar(ProtoTestEventWrapper.ProtoTestEvent.Foo
- .newBuilder()
- .setBar("bar0"))
- .addBar(ProtoTestEventWrapper.ProtoTestEvent.Foo
- .newBuilder()
- .setBar("bar1"))
- .build();
+ ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildNestedData(dateTime);
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- event.writeTo(out);
-
- final ByteEntity entity = new ByteEntity(ByteBuffer.wrap(out.toByteArray()));
+ final ByteEntity entity = new ByteEntity(ProtobufInputRowParserTest.toByteBuffer(event));
InputRow row = protobufInputFormat.createReader(new InputRowSchema(timestampSpec, dimensionsSpec, null), entity, null).read().next();
- Assert.assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch());
-
- assertDimensionEquals(row, "id", "4711");
- assertDimensionEquals(row, "isValid", "true");
- assertDimensionEquals(row, "someOtherId", "4712");
- assertDimensionEquals(row, "description", "description");
-
- assertDimensionEquals(row, "eventType", ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE.name());
- assertDimensionEquals(row, "foobar", "baz");
- assertDimensionEquals(row, "bar0", "bar0");
-
-
- Assert.assertEquals(47.11F, row.getMetric("someFloatColumn").floatValue(), 0.0);
- Assert.assertEquals(815.0F, row.getMetric("someIntColumn").floatValue(), 0.0);
- Assert.assertEquals(816.0F, row.getMetric("someLongColumn").floatValue(), 0.0);
+ ProtobufInputRowParserTest.verifyNestedData(row, dateTime);
}
@Test
@@ -172,44 +133,12 @@ public class ProtobufInputFormatTest
//create binary of proto test event
DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC());
- ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
- .setDescription("description")
- .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE)
- .setId(4711L)
- .setIsValid(true)
- .setSomeOtherId(4712)
- .setTimestamp(dateTime.toString())
- .setSomeFloatColumn(47.11F)
- .setSomeIntColumn(815)
- .setSomeLongColumn(816L)
- .build();
+ ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildFlatData(dateTime);
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- event.writeTo(out);
-
- final ByteEntity entity = new ByteEntity(ByteBuffer.wrap(out.toByteArray()));
+ final ByteEntity entity = new ByteEntity(ProtobufInputRowParserTest.toByteBuffer(event));
InputRow row = protobufInputFormat.createReader(new InputRowSchema(timestampSpec, dimensionsSpec, null), entity, null).read().next();
- System.out.println(row);
-
- Assert.assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch());
-
- assertDimensionEquals(row, "id", "4711");
- assertDimensionEquals(row, "isValid", "true");
- assertDimensionEquals(row, "someOtherId", "4712");
- assertDimensionEquals(row, "description", "description");
-
-
- Assert.assertEquals(47.11F, row.getMetric("someFloatColumn").floatValue(), 0.0);
- Assert.assertEquals(815.0F, row.getMetric("someIntColumn").floatValue(), 0.0);
- Assert.assertEquals(816.0F, row.getMetric("someLongColumn").floatValue(), 0.0);
- }
-
- private void assertDimensionEquals(InputRow row, String dimension, Object expected)
- {
- List values = row.getDimension(dimension);
- Assert.assertEquals(1, values.size());
- Assert.assertEquals(expected, values.get(0));
+ ProtobufInputRowParserTest.verifyFlatData(row, dateTime);
}
}
diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java
index 03d4f874c7d..7d0b1cd3977 100644
--- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java
+++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java
@@ -21,6 +21,7 @@ package org.apache.druid.data.input.protobuf;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
+import com.google.protobuf.Timestamp;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JSONParseSpec;
@@ -42,6 +43,7 @@ import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.ByteArrayOutputStream;
+import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.List;
@@ -52,6 +54,7 @@ public class ProtobufInputRowParserTest
private ParseSpec parseSpec;
private ParseSpec flatParseSpec;
+ private ParseSpec flatParseSpecWithComplexTimestamp;
private FileBasedProtobufBytesDecoder decoder;
@Before
@@ -90,6 +93,20 @@ public class ProtobufInputRowParserTest
null,
null
);
+
+ flatParseSpecWithComplexTimestamp = new JSONParseSpec(
+ new TimestampSpec("otherTimestamp", "iso", null),
+ new DimensionsSpec(Lists.newArrayList(
+ new StringDimensionSchema("event"),
+ new StringDimensionSchema("id"),
+ new StringDimensionSchema("someOtherId"),
+ new StringDimensionSchema("isValid")
+ ), null, null),
+
+ null,
+ null,
+ null
+ );
decoder = new FileBasedProtobufBytesDecoder("prototest.desc", "ProtoTestEvent");
}
@@ -101,47 +118,11 @@ public class ProtobufInputRowParserTest
//create binary of proto test event
DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC());
- ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
- .setDescription("description")
- .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE)
- .setId(4711L)
- .setIsValid(true)
- .setSomeOtherId(4712)
- .setTimestamp(dateTime.toString())
- .setSomeFloatColumn(47.11F)
- .setSomeIntColumn(815)
- .setSomeLongColumn(816L)
- .setFoo(ProtoTestEventWrapper.ProtoTestEvent.Foo
- .newBuilder()
- .setBar("baz"))
- .addBar(ProtoTestEventWrapper.ProtoTestEvent.Foo
- .newBuilder()
- .setBar("bar0"))
- .addBar(ProtoTestEventWrapper.ProtoTestEvent.Foo
- .newBuilder()
- .setBar("bar1"))
- .build();
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- event.writeTo(out);
+ ProtoTestEventWrapper.ProtoTestEvent event = buildNestedData(dateTime);
- InputRow row = parser.parseBatch(ByteBuffer.wrap(out.toByteArray())).get(0);
-
- Assert.assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch());
-
- assertDimensionEquals(row, "id", "4711");
- assertDimensionEquals(row, "isValid", "true");
- assertDimensionEquals(row, "someOtherId", "4712");
- assertDimensionEquals(row, "description", "description");
-
- assertDimensionEquals(row, "eventType", ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE.name());
- assertDimensionEquals(row, "foobar", "baz");
- assertDimensionEquals(row, "bar0", "bar0");
-
-
- Assert.assertEquals(47.11F, row.getMetric("someFloatColumn").floatValue(), 0.0);
- Assert.assertEquals(815.0F, row.getMetric("someIntColumn").floatValue(), 0.0);
- Assert.assertEquals(816.0F, row.getMetric("someLongColumn").floatValue(), 0.0);
+ InputRow row = parser.parseBatch(toByteBuffer(event)).get(0);
+ verifyNestedData(row, dateTime);
}
@Test
@@ -152,35 +133,24 @@ public class ProtobufInputRowParserTest
//create binary of proto test event
DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC());
- ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
- .setDescription("description")
- .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE)
- .setId(4711L)
- .setIsValid(true)
- .setSomeOtherId(4712)
- .setTimestamp(dateTime.toString())
- .setSomeFloatColumn(47.11F)
- .setSomeIntColumn(815)
- .setSomeLongColumn(816L)
- .build();
+ ProtoTestEventWrapper.ProtoTestEvent event = buildFlatData(dateTime);
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- event.writeTo(out);
+ InputRow row = parser.parseBatch(toByteBuffer(event)).get(0);
+ verifyFlatData(row, dateTime);
+ }
- InputRow row = parser.parseBatch(ByteBuffer.wrap(out.toByteArray())).get(0);
- System.out.println(row);
+ @Test
+ public void testParseFlatDataWithComplexTimestamp() throws Exception
+ {
+ ProtobufInputRowParser parser = new ProtobufInputRowParser(flatParseSpecWithComplexTimestamp, decoder, null, null);
- Assert.assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch());
+ //create binary of proto test event
+ DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC());
+ ProtoTestEventWrapper.ProtoTestEvent event = buildFlatDataWithComplexTimestamp(dateTime);
- assertDimensionEquals(row, "id", "4711");
- assertDimensionEquals(row, "isValid", "true");
- assertDimensionEquals(row, "someOtherId", "4712");
- assertDimensionEquals(row, "description", "description");
+ InputRow row = parser.parseBatch(toByteBuffer(event)).get(0);
-
- Assert.assertEquals(47.11F, row.getMetric("someFloatColumn").floatValue(), 0.0);
- Assert.assertEquals(815.0F, row.getMetric("someIntColumn").floatValue(), 0.0);
- Assert.assertEquals(816.0F, row.getMetric("someLongColumn").floatValue(), 0.0);
+ verifyFlatDataWithComplexTimestamp(row, dateTime);
}
@Test
@@ -218,7 +188,53 @@ public class ProtobufInputRowParserTest
//create binary of proto test event
DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC());
- ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
+ ProtoTestEventWrapper.ProtoTestEvent event = buildNestedData(dateTime);
+
+ InputRow row = parser.parseBatch(toByteBuffer(event)).get(0);
+
+ verifyNestedData(row, dateTime);
+ }
+
+ private static void assertDimensionEquals(InputRow row, String dimension, Object expected)
+ {
+ List values = row.getDimension(dimension);
+ Assert.assertEquals(1, values.size());
+ Assert.assertEquals(expected, values.get(0));
+ }
+
+ static ProtoTestEventWrapper.ProtoTestEvent buildFlatData(DateTime dateTime)
+ {
+ return ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
+ .setDescription("description")
+ .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE)
+ .setId(4711L)
+ .setIsValid(true)
+ .setSomeOtherId(4712)
+ .setTimestamp(dateTime.toString())
+ .setSomeFloatColumn(47.11F)
+ .setSomeIntColumn(815)
+ .setSomeLongColumn(816L)
+ .build();
+ }
+
+ static void verifyFlatData(InputRow row, DateTime dateTime)
+ {
+ Assert.assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch());
+
+ assertDimensionEquals(row, "id", "4711");
+ assertDimensionEquals(row, "isValid", "true");
+ assertDimensionEquals(row, "someOtherId", "4712");
+ assertDimensionEquals(row, "description", "description");
+
+
+ Assert.assertEquals(47.11F, row.getMetric("someFloatColumn").floatValue(), 0.0);
+ Assert.assertEquals(815.0F, row.getMetric("someIntColumn").floatValue(), 0.0);
+ Assert.assertEquals(816.0F, row.getMetric("someLongColumn").floatValue(), 0.0);
+ }
+
+ static ProtoTestEventWrapper.ProtoTestEvent buildNestedData(DateTime dateTime)
+ {
+ return ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
.setDescription("description")
.setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE)
.setId(4711L)
@@ -238,12 +254,10 @@ public class ProtobufInputRowParserTest
.newBuilder()
.setBar("bar1"))
.build();
+ }
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- event.writeTo(out);
-
- InputRow row = parser.parseBatch(ByteBuffer.wrap(out.toByteArray())).get(0);
-
+ static void verifyNestedData(InputRow row, DateTime dateTime)
+ {
Assert.assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch());
assertDimensionEquals(row, "id", "4711");
@@ -261,10 +275,33 @@ public class ProtobufInputRowParserTest
Assert.assertEquals(816.0F, row.getMetric("someLongColumn").floatValue(), 0.0);
}
- private void assertDimensionEquals(InputRow row, String dimension, Object expected)
+ static ProtoTestEventWrapper.ProtoTestEvent buildFlatDataWithComplexTimestamp(DateTime dateTime)
{
- List values = row.getDimension(dimension);
- Assert.assertEquals(1, values.size());
- Assert.assertEquals(expected, values.get(0));
+ Timestamp timestamp = Timestamp.newBuilder().setSeconds(dateTime.getMillis() / 1000).setNanos((int) ((dateTime.getMillis() % 1000) * 1000 * 1000)).build();
+ return ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
+ .setDescription("description")
+ .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE)
+ .setId(4711L)
+ .setIsValid(true)
+ .setSomeOtherId(4712)
+ .setOtherTimestamp(timestamp)
+ .setTimestamp("unused")
+ .setSomeFloatColumn(47.11F)
+ .setSomeIntColumn(815)
+ .setSomeLongColumn(816L)
+ .build();
+ }
+
+ static void verifyFlatDataWithComplexTimestamp(InputRow row, DateTime dateTime)
+ {
+ verifyFlatData(row, dateTime);
+ }
+
+ static ByteBuffer toByteBuffer(ProtoTestEventWrapper.ProtoTestEvent event) throws IOException
+ {
+ try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
+ event.writeTo(out);
+ return ByteBuffer.wrap(out.toByteArray());
+ }
}
}
diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufReaderTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufReaderTest.java
new file mode 100644
index 00000000000..2df960df527
--- /dev/null
+++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufReaderTest.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.data.input.protobuf;
+
+import com.google.common.collect.Lists;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.data.input.impl.TimestampSpec;
+import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec;
+import org.apache.druid.java.util.common.parsers.JSONPathFieldType;
+import org.apache.druid.java.util.common.parsers.JSONPathSpec;
+import org.joda.time.DateTime;
+import org.joda.time.chrono.ISOChronology;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.nio.ByteBuffer;
+
+public class ProtobufReaderTest
+{
+ @Rule
+ public ExpectedException expectedException = ExpectedException.none();
+
+ private InputRowSchema inputRowSchema;
+ private InputRowSchema inputRowSchemaWithComplexTimestamp;
+ private JSONPathSpec flattenSpec;
+ private FileBasedProtobufBytesDecoder decoder;
+
+ @Before
+ public void setUp()
+ {
+ TimestampSpec timestampSpec = new TimestampSpec("timestamp", "iso", null);
+ DimensionsSpec dimensionsSpec = new DimensionsSpec(Lists.newArrayList(
+ new StringDimensionSchema("event"),
+ new StringDimensionSchema("id"),
+ new StringDimensionSchema("someOtherId"),
+ new StringDimensionSchema("isValid")
+ ), null, null);
+ flattenSpec = new JSONPathSpec(
+ true,
+ Lists.newArrayList(
+ new JSONPathFieldSpec(JSONPathFieldType.ROOT, "eventType", "eventType"),
+ new JSONPathFieldSpec(JSONPathFieldType.PATH, "foobar", "$.foo.bar"),
+ new JSONPathFieldSpec(JSONPathFieldType.PATH, "bar0", "$.bar[0].bar")
+ )
+ );
+
+ inputRowSchema = new InputRowSchema(timestampSpec, dimensionsSpec, null);
+ inputRowSchemaWithComplexTimestamp = new InputRowSchema(
+ new TimestampSpec("otherTimestamp", "iso", null),
+ dimensionsSpec,
+ null
+ );
+ decoder = new FileBasedProtobufBytesDecoder("prototest.desc", "ProtoTestEvent");
+ }
+
+ @Test
+ public void testParseNestedData() throws Exception
+ {
+ ProtobufReader reader = new ProtobufReader(inputRowSchema, null, decoder, flattenSpec);
+
+ //create binary of proto test event
+ DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC());
+ ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildNestedData(dateTime);
+
+ ByteBuffer buffer = ProtobufInputRowParserTest.toByteBuffer(event);
+
+ InputRow row = reader.parseInputRows(decoder.parse(buffer)).get(0);
+
+ ProtobufInputRowParserTest.verifyNestedData(row, dateTime);
+ }
+
+ @Test
+ public void testParseFlatData() throws Exception
+ {
+ ProtobufReader reader = new ProtobufReader(inputRowSchema, null, decoder, null);
+
+ //create binary of proto test event
+ DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC());
+ ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildFlatData(dateTime);
+
+ ByteBuffer buffer = ProtobufInputRowParserTest.toByteBuffer(event);
+
+ InputRow row = reader.parseInputRows(decoder.parse(buffer)).get(0);
+
+ ProtobufInputRowParserTest.verifyFlatData(row, dateTime);
+ }
+
+ @Test
+ public void testParseFlatDataWithComplexTimestamp() throws Exception
+ {
+ ProtobufReader reader = new ProtobufReader(inputRowSchemaWithComplexTimestamp, null, decoder, null);
+
+ //create binary of proto test event
+ DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC());
+ ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildFlatDataWithComplexTimestamp(dateTime);
+
+ ByteBuffer buffer = ProtobufInputRowParserTest.toByteBuffer(event);
+
+ InputRow row = reader.parseInputRows(decoder.parse(buffer)).get(0);
+
+ ProtobufInputRowParserTest.verifyFlatDataWithComplexTimestamp(row, dateTime);
+ }
+}
diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/SchemaRegistryBasedProtobufBytesDecoderTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/SchemaRegistryBasedProtobufBytesDecoderTest.java
index 8f4c9219b5a..0d77b11ec4a 100644
--- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/SchemaRegistryBasedProtobufBytesDecoderTest.java
+++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/SchemaRegistryBasedProtobufBytesDecoderTest.java
@@ -20,6 +20,7 @@
package org.apache.druid.data.input.protobuf;
import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableMap;
import com.google.protobuf.DynamicMessage;
import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient;
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
@@ -38,6 +39,7 @@ import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
+import java.util.Collections;
public class SchemaRegistryBasedProtobufBytesDecoderTest
{
@@ -52,11 +54,8 @@ public class SchemaRegistryBasedProtobufBytesDecoderTest
@Test
public void testParse() throws Exception
{
- // Given
- InputStream fin;
- fin = this.getClass().getClassLoader().getResourceAsStream("ProtoTest.proto");
- String protobufString = IOUtils.toString(fin, StandardCharsets.UTF_8);
- Mockito.when(registry.getSchemaById(ArgumentMatchers.eq(1234))).thenReturn(new ProtobufSchema(protobufString));
+
+ Mockito.when(registry.getSchemaById(ArgumentMatchers.eq(1234))).thenReturn(parseProtobufSchema());
ProtoTestEventWrapper.ProtoTestEvent event = getTestEvent();
byte[] bytes = event.toByteArray();
ByteBuffer bb = ByteBuffer.allocate(bytes.length + 6).put((byte) 0).putInt(1234).put((byte) 0).put(bytes);
@@ -70,11 +69,7 @@ public class SchemaRegistryBasedProtobufBytesDecoderTest
@Test(expected = ParseException.class)
public void testParseCorrupted() throws Exception
{
- // Given
- InputStream fin;
- fin = this.getClass().getClassLoader().getResourceAsStream("ProtoTest.proto");
- String protobufString = IOUtils.toString(fin, StandardCharsets.UTF_8);
- Mockito.when(registry.getSchemaById(ArgumentMatchers.eq(1234))).thenReturn(new ProtobufSchema(protobufString));
+ Mockito.when(registry.getSchemaById(ArgumentMatchers.eq(1234))).thenReturn(parseProtobufSchema());
byte[] bytes = getTestEvent().toByteArray();
ByteBuffer bb = ByteBuffer.allocate(bytes.length + 6).put((byte) 0).putInt(1234).put((bytes), 5, 10);
bb.rewind();
@@ -116,17 +111,7 @@ public class SchemaRegistryBasedProtobufBytesDecoderTest
private ProtoTestEventWrapper.ProtoTestEvent getTestEvent()
{
DateTime dateTime = new DateTime(2012, 7, 12, 9, 30, ISOChronology.getInstanceUTC());
- ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
- .setDescription("description")
- .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE)
- .setId(4711L)
- .setIsValid(true)
- .setSomeOtherId(4712)
- .setTimestamp(dateTime.toString())
- .setSomeFloatColumn(47.11F)
- .setSomeIntColumn(815)
- .setSomeLongColumn(816L)
- .build();
+ ProtoTestEventWrapper.ProtoTestEvent event = ProtobufInputRowParserTest.buildFlatData(dateTime);
return event;
}
@@ -172,4 +157,17 @@ public class SchemaRegistryBasedProtobufBytesDecoderTest
// Then
Assert.assertNotEquals(decoder.hashCode(), 0);
}
+
+ private ProtobufSchema parseProtobufSchema() throws IOException
+ {
+ // Given
+ InputStream fin;
+ fin = this.getClass().getClassLoader().getResourceAsStream("ProtoTest.proto");
+ String protobufString = IOUtils.toString(fin, StandardCharsets.UTF_8);
+
+ fin = this.getClass().getClassLoader().getResourceAsStream("google/protobuf/timestamp.proto");
+ String timestampProtobufString = IOUtils.toString(fin, StandardCharsets.UTF_8);
+ return new ProtobufSchema(protobufString, Collections.emptyList(),
+ ImmutableMap.of("google/protobuf/timestamp.proto", timestampProtobufString), null, null);
+ }
}
diff --git a/extensions-core/protobuf-extensions/src/test/resources/ProtoTest.proto b/extensions-core/protobuf-extensions/src/test/resources/ProtoTest.proto
index 8371357d86a..96b00d437dd 100644
--- a/extensions-core/protobuf-extensions/src/test/resources/ProtoTest.proto
+++ b/extensions-core/protobuf-extensions/src/test/resources/ProtoTest.proto
@@ -13,11 +13,15 @@
// See the License for the specific language governing permissions and
// limitations under the License.
+/*
+If you are changing this class, make sure to add --include_imports flag while generating the descriptor file
+*/
+
syntax = "proto2";
package prototest;
option java_package = "org.apache.druid.data.input.protobuf";
option java_outer_classname = "ProtoTestEventWrapper";
-
+import "google/protobuf/timestamp.proto";
message ProtoTestEvent {
enum EventCategory {
@@ -42,4 +46,5 @@ message ProtoTestEvent {
optional uint64 someLongColumn = 9;
optional Foo foo = 10;
repeated Foo bar = 11;
+ optional google.protobuf.Timestamp otherTimestamp = 12;
}
diff --git a/extensions-core/protobuf-extensions/src/test/resources/prototest.desc b/extensions-core/protobuf-extensions/src/test/resources/prototest.desc
index d1226c81f36387fdb037385b7baa3ad6c3ad40af..7509c6b82a32cdf58f2363382c351bd29179ca71 100644
GIT binary patch
delta 408
zcmey*@{GNn>pvrxe0qL+oZh;~!8
zoK(1}PJD1P13)ebV2m|{Q
v!bj=RM#X)ME<#RReEB6AsYP&aNbmst&21zF_9~K3_24Rk&~!|`&$s{pO8Jnr
delta 47
wcmaFH{+~sW>kkVTe?U=wNq$Ibafx05h$X}crb>VUO7k|mFz#cV?8dwR0G#v@O#lD@