diff --git a/distribution/pom.xml b/distribution/pom.xml
index 630b4c836ea5..0636378d0343 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -97,10 +97,12 @@
-c
io.druid.extensions:druid-lookups-cached-single
-c
+ io.druid.extensions:druid-protobuf-extensions
+ -c
io.druid.extensions:mysql-metadata-storage
-c
io.druid.extensions:postgresql-metadata-storage
- -c
+ -c
io.druid.extensions:druid-kerberos
-c
io.druid.extensions:druid-s3-extensions
diff --git a/distribution/src/assembly/assembly.xml b/distribution/src/assembly/assembly.xml
index 9e7853a8aefd..02a71d3ede65 100644
--- a/distribution/src/assembly/assembly.xml
+++ b/distribution/src/assembly/assembly.xml
@@ -114,6 +114,13 @@
conf-quickstart/tranquility
+
+ ../examples/quickstart/protobuf
+
+ *
+
+ quickstart/protobuf
+
../examples/conf
diff --git a/docs/content/development/extensions-core/protobuf.md b/docs/content/development/extensions-core/protobuf.md
new file mode 100644
index 000000000000..c45e47c6ecd5
--- /dev/null
+++ b/docs/content/development/extensions-core/protobuf.md
@@ -0,0 +1,203 @@
+---
+layout: doc_page
+---
+
+# Protobuf
+
+This extension enables Druid to ingest and understand the Protobuf data format. Make sure to [include](../../operations/including-extensions.html) `druid-protobuf-extensions` as an extension.
+
+## Protobuf Parser
+
+
+| Field | Type | Description | Required |
+|-------|------|-------------|----------|
+| type | String | This should say `protobuf`. | no |
+| descriptor | String | Protobuf descriptor file name in the classpath or URL. | yes |
+| protoMessageType | String | Protobuf message type in the descriptor. Both short name and fully qualified name are accepted. The parser uses the first message type found in the descriptor if not specified. | no |
+| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. The format must be json. See [JSON ParseSpec](../../ingestion/index.html) for more configuration options. Please note timeAndDims parseSpec is no longer supported. | yes |
+
+## Example: Load Protobuf messages from Kafka
+
+This example demonstrates how to load Protobuf messages from Kafka. Please read the [Load from Kafka tutorial](../../tutorial/tutorial-kafka.html) first. This example will use the same "metrics" dataset.
+
+Files used in this example are found at `./examples/quickstart/protobuf` in your Druid directory.
+
+- We will use [Kafka Indexing Service](./kafka-ingestion.html) instead of Tranquility.
+- Kafka broker host is `localhost:9092`.
+- Kafka topic is `metrics_pb` instead of `metrics`.
+- datasource name is `metrics-kafka-pb` instead of `metrics-kafka` to avoid the confusion.
+
+Here is the metrics JSON example.
+
+```json
+{
+ "unit": "milliseconds",
+ "http_method": "GET",
+ "value": 44,
+ "timestamp": "2017-04-06T02:36:22Z",
+ "http_code": "200",
+ "page": "/",
+ "metricType": "request/latency",
+ "server": "www1.example.com"
+}
+```
+
+### Proto file
+
+The proto file should look like this. Save it as metrics.proto.
+
+```
+syntax = "proto3";
+message Metrics {
+ string unit = 1;
+ string http_method = 2;
+ int32 value = 3;
+ string timestamp = 4;
+ string http_code = 5;
+ string page = 6;
+ string metricType = 7;
+ string server = 8;
+}
+```
+
+### Descriptor file
+
+Using the `protoc` Protobuf compiler to generate the descriptor file. Save the metrics.desc file either in the classpath or reachable by URL. In this example the descriptor file was saved at /tmp/metrics.desc.
+
+```
+protoc -o /tmp/metrics.desc metrics.proto
+```
+
+### Supervisor spec JSON
+
+Below is the complete Supervisor spec JSON to be submitted to the Overlord.
+Please make sure these keys are properly configured for successful ingestion.
+
+- `descriptor` for the descriptor file URL.
+- `protoMessageType` from the proto definition.
+- parseSpec `format` must be `json`.
+- `topic` to subscribe. The topic is "metrics_pb" instead of "metrics".
+- `bootstrap.server` is the kafka broker host.
+
+```json
+{
+ "type": "kafka",
+ "dataSchema": {
+ "dataSource": "metrics-kafka2",
+ "parser": {
+ "type": "protobuf",
+ "descriptor": "file:///tmp/metrics.desc",
+ "protoMessageType": "Metrics",
+ "parseSpec": {
+ "format": "json",
+ "timestampSpec": {
+ "column": "timestamp",
+ "format": "auto"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "unit",
+ "http_method",
+ "http_code",
+ "page",
+ "metricType",
+ "server"
+ ],
+ "dimensionExclusions": [
+ "timestamp",
+ "value"
+ ]
+ }
+ }
+ },
+ "metricsSpec": [
+ {
+ "name": "count",
+ "type": "count"
+ },
+ {
+ "name": "value_sum",
+ "fieldName": "value",
+ "type": "doubleSum"
+ },
+ {
+ "name": "value_min",
+ "fieldName": "value",
+ "type": "doubleMin"
+ },
+ {
+ "name": "value_max",
+ "fieldName": "value",
+ "type": "doubleMax"
+ }
+ ],
+ "granularitySpec": {
+ "type": "uniform",
+ "segmentGranularity": "HOUR",
+ "queryGranularity": "NONE"
+ }
+ },
+ "tuningConfig": {
+ "type": "kafka",
+ "maxRowsPerSegment": 5000000
+ },
+ "ioConfig": {
+ "topic": "metrics_pb",
+ "consumerProperties": {
+ "bootstrap.servers": "localhost:9092"
+ },
+ "taskCount": 1,
+ "replicas": 1,
+ "taskDuration": "PT1H"
+ }
+}
+```
+
+## Kafka Producer
+
+Here is the sample script that publishes the metrics to Kafka in Protobuf format.
+
+1. Run `protoc` again with the Python binding option. This command generates `metrics_pb2.py` file.
+ ```
+ protoc -o metrics.desc metrics.proto --python_out=.
+ ```
+
+2. Create Kafka producer script.
+
+This script requires `protobuf` and `kafka-python` modules.
+
+```python
+#!/usr/bin/env python
+
+import sys
+import json
+
+from kafka import KafkaProducer
+from metrics_pb2 import Metrics
+
+producer = KafkaProducer(bootstrap_servers='localhost:9092')
+topic = 'metrics_pb'
+metrics = Metrics()
+
+for row in iter(sys.stdin):
+ d = json.loads(row)
+ for k, v in d.items():
+ setattr(metrics, k, v)
+ pb = metrics.SerializeToString()
+ producer.send(topic, pb)
+```
+
+3. run producer
+
+```
+./bin/generate-example-metrics | ./pb_publisher.py
+```
+
+4. test
+
+```
+kafka-console-consumer --zookeeper localhost --topic metrics_pb
+```
+
+It should print messages like this
+> millisecondsGETR"2017-04-06T03:23:56Z*2002/list:request/latencyBwww1.example.com
diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md
index 779ea6b65d99..d6bfca71dba0 100644
--- a/docs/content/development/extensions.md
+++ b/docs/content/development/extensions.md
@@ -32,6 +32,7 @@ Core extensions are maintained by Druid committers.
|druid-kerberos|Kerberos authentication for druid nodes.|[link](../development/extensions-core/druid-kerberos.html)|
|druid-lookups-cached-global|A module for [lookups](../querying/lookups.html) providing a jvm-global eager caching for lookups. It provides JDBC and URI implementations for fetching lookup data.|[link](../development/extensions-core/lookups-cached-global.html)|
|druid-lookups-cached-single| Per lookup caching module to support the use cases where a lookup need to be isolated from the global pool of lookups |[link](../development/extensions-core/druid-lookups.html)|
+|druid-protobuf-extensions| Support for data in Protobuf data format.|[link](../development/extensions-core/protobuf.html)|
|druid-s3-extensions|Interfacing with data in AWS S3, and using S3 as deep storage.|[link](../development/extensions-core/s3.html)|
|druid-stats|Statistics related module including variance and standard deviation.|[link](../development/extensions-core/stats.html)|
|mysql-metadata-storage|MySQL metadata store.|[link](../development/extensions-core/mysql.html)|
diff --git a/docs/content/ingestion/index.md b/docs/content/ingestion/index.md
index 6caf9b133672..58874055166b 100644
--- a/docs/content/ingestion/index.md
+++ b/docs/content/ingestion/index.md
@@ -109,13 +109,6 @@ If `type` is not included, the parser defaults to `string`. For additional data
| type | String | This should say `string` in general, or `hadoopyString` when used in a Hadoop indexing job. | no |
| parseSpec | JSON Object | Specifies the format, timestamp, and dimensions of the data. | yes |
-### Protobuf Parser
-
-| Field | Type | Description | Required |
-|-------|------|-------------|----------|
-| type | String | This should say `protobuf`. | no |
-| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes |
-
### ParseSpec
ParseSpecs serve two purposes:
diff --git a/examples/quickstart/protobuf/kafka-metrics-pb.json b/examples/quickstart/protobuf/kafka-metrics-pb.json
new file mode 100644
index 000000000000..0b896a8b9370
--- /dev/null
+++ b/examples/quickstart/protobuf/kafka-metrics-pb.json
@@ -0,0 +1,71 @@
+{
+ "type": "kafka",
+ "dataSchema": {
+ "dataSource": "metrics-kafka2",
+ "parser": {
+ "type": "protobuf",
+ "descriptor": "file:///tmp/metrics.desc",
+ "protoMessageType": "Metrics",
+ "parseSpec": {
+ "format": "json",
+ "timestampSpec": {
+ "column": "timestamp",
+ "format": "auto"
+ },
+ "dimensionsSpec": {
+ "dimensions": [
+ "unit",
+ "http_method",
+ "http_code",
+ "page",
+ "metricType",
+ "server"
+ ],
+ "dimensionExclusions": [
+ "timestamp",
+ "value"
+ ]
+ }
+ }
+ },
+ "metricsSpec": [
+ {
+ "name": "count",
+ "type": "count"
+ },
+ {
+ "name": "value_sum",
+ "fieldName": "value",
+ "type": "doubleSum"
+ },
+ {
+ "name": "value_min",
+ "fieldName": "value",
+ "type": "doubleMin"
+ },
+ {
+ "name": "value_max",
+ "fieldName": "value",
+ "type": "doubleMax"
+ }
+ ],
+ "granularitySpec": {
+ "type": "uniform",
+ "segmentGranularity": "HOUR",
+ "queryGranularity": "NONE"
+ }
+ },
+ "tuningConfig": {
+ "type": "kafka",
+ "maxRowsPerSegment": 5000000
+ },
+ "ioConfig": {
+ "topic": "metrics_pb",
+ "consumerProperties": {
+ "bootstrap.servers": "localhost:9092"
+ },
+ "taskCount": 1,
+ "replicas": 1,
+ "taskDuration": "PT1H"
+ }
+}
diff --git a/examples/quickstart/protobuf/metrics.desc b/examples/quickstart/protobuf/metrics.desc
new file mode 100644
index 000000000000..4def5e6fe8d1
--- /dev/null
+++ b/examples/quickstart/protobuf/metrics.desc
@@ -0,0 +1,15 @@
+
+õ
+
metrics.proto"Û
+Metrics
+unit ( Runit
+http_method ( R
+httpMethod
+value (Rvalue
+ timestamp ( R timestamp
+ http_code ( RhttpCode
+page ( Rpage
+
+metricType ( R
+metricType
+server ( Rserverbproto3
\ No newline at end of file
diff --git a/examples/quickstart/protobuf/metrics.proto b/examples/quickstart/protobuf/metrics.proto
new file mode 100644
index 000000000000..76f9d1ce2611
--- /dev/null
+++ b/examples/quickstart/protobuf/metrics.proto
@@ -0,0 +1,11 @@
+syntax = "proto3";
+message Metrics {
+ string unit = 1;
+ string http_method = 2;
+ int32 value = 3;
+ string timestamp = 4;
+ string http_code = 5;
+ string page = 6;
+ string metricType = 7;
+ string server = 8;
+}
diff --git a/examples/quickstart/protobuf/metrics_pb2.py b/examples/quickstart/protobuf/metrics_pb2.py
new file mode 100644
index 000000000000..af4c5d24ff2e
--- /dev/null
+++ b/examples/quickstart/protobuf/metrics_pb2.py
@@ -0,0 +1,118 @@
+# Generated by the protocol buffer compiler. DO NOT EDIT!
+# source: metrics.proto
+
+import sys
+_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1'))
+from google.protobuf import descriptor as _descriptor
+from google.protobuf import message as _message
+from google.protobuf import reflection as _reflection
+from google.protobuf import symbol_database as _symbol_database
+from google.protobuf import descriptor_pb2
+# @@protoc_insertion_point(imports)
+
+_sym_db = _symbol_database.Default()
+
+
+
+
+DESCRIPTOR = _descriptor.FileDescriptor(
+ name='metrics.proto',
+ package='',
+ syntax='proto3',
+ serialized_pb=_b('\n\rmetrics.proto\"\x93\x01\n\x07Metrics\x12\x0c\n\x04unit\x18\x01 \x01(\t\x12\x13\n\x0bhttp_method\x18\x02 \x01(\t\x12\r\n\x05value\x18\x03 \x01(\x05\x12\x11\n\ttimestamp\x18\x04 \x01(\t\x12\x11\n\thttp_code\x18\x05 \x01(\t\x12\x0c\n\x04page\x18\x06 \x01(\t\x12\x12\n\nmetricType\x18\x07 \x01(\t\x12\x0e\n\x06server\x18\x08 \x01(\tb\x06proto3')
+)
+_sym_db.RegisterFileDescriptor(DESCRIPTOR)
+
+
+
+
+_METRICS = _descriptor.Descriptor(
+ name='Metrics',
+ full_name='Metrics',
+ filename=None,
+ file=DESCRIPTOR,
+ containing_type=None,
+ fields=[
+ _descriptor.FieldDescriptor(
+ name='unit', full_name='Metrics.unit', index=0,
+ number=1, type=9, cpp_type=9, label=1,
+ has_default_value=False, default_value=_b("").decode('utf-8'),
+ message_type=None, enum_type=None, containing_type=None,
+ is_extension=False, extension_scope=None,
+ options=None),
+ _descriptor.FieldDescriptor(
+ name='http_method', full_name='Metrics.http_method', index=1,
+ number=2, type=9, cpp_type=9, label=1,
+ has_default_value=False, default_value=_b("").decode('utf-8'),
+ message_type=None, enum_type=None, containing_type=None,
+ is_extension=False, extension_scope=None,
+ options=None),
+ _descriptor.FieldDescriptor(
+ name='value', full_name='Metrics.value', index=2,
+ number=3, type=5, cpp_type=1, label=1,
+ has_default_value=False, default_value=0,
+ message_type=None, enum_type=None, containing_type=None,
+ is_extension=False, extension_scope=None,
+ options=None),
+ _descriptor.FieldDescriptor(
+ name='timestamp', full_name='Metrics.timestamp', index=3,
+ number=4, type=9, cpp_type=9, label=1,
+ has_default_value=False, default_value=_b("").decode('utf-8'),
+ message_type=None, enum_type=None, containing_type=None,
+ is_extension=False, extension_scope=None,
+ options=None),
+ _descriptor.FieldDescriptor(
+ name='http_code', full_name='Metrics.http_code', index=4,
+ number=5, type=9, cpp_type=9, label=1,
+ has_default_value=False, default_value=_b("").decode('utf-8'),
+ message_type=None, enum_type=None, containing_type=None,
+ is_extension=False, extension_scope=None,
+ options=None),
+ _descriptor.FieldDescriptor(
+ name='page', full_name='Metrics.page', index=5,
+ number=6, type=9, cpp_type=9, label=1,
+ has_default_value=False, default_value=_b("").decode('utf-8'),
+ message_type=None, enum_type=None, containing_type=None,
+ is_extension=False, extension_scope=None,
+ options=None),
+ _descriptor.FieldDescriptor(
+ name='metricType', full_name='Metrics.metricType', index=6,
+ number=7, type=9, cpp_type=9, label=1,
+ has_default_value=False, default_value=_b("").decode('utf-8'),
+ message_type=None, enum_type=None, containing_type=None,
+ is_extension=False, extension_scope=None,
+ options=None),
+ _descriptor.FieldDescriptor(
+ name='server', full_name='Metrics.server', index=7,
+ number=8, type=9, cpp_type=9, label=1,
+ has_default_value=False, default_value=_b("").decode('utf-8'),
+ message_type=None, enum_type=None, containing_type=None,
+ is_extension=False, extension_scope=None,
+ options=None),
+ ],
+ extensions=[
+ ],
+ nested_types=[],
+ enum_types=[
+ ],
+ options=None,
+ is_extendable=False,
+ syntax='proto3',
+ extension_ranges=[],
+ oneofs=[
+ ],
+ serialized_start=18,
+ serialized_end=165,
+)
+
+DESCRIPTOR.message_types_by_name['Metrics'] = _METRICS
+
+Metrics = _reflection.GeneratedProtocolMessageType('Metrics', (_message.Message,), dict(
+ DESCRIPTOR = _METRICS,
+ __module__ = 'metrics_pb2'
+ # @@protoc_insertion_point(class_scope:Metrics)
+ ))
+_sym_db.RegisterMessage(Metrics)
+
+
+# @@protoc_insertion_point(module_scope)
diff --git a/examples/quickstart/protobuf/pb_publisher.py b/examples/quickstart/protobuf/pb_publisher.py
new file mode 100755
index 000000000000..73aeef2797b7
--- /dev/null
+++ b/examples/quickstart/protobuf/pb_publisher.py
@@ -0,0 +1,19 @@
+#!/usr/bin/env python
+
+import sys
+import json
+
+from kafka import KafkaProducer
+from metrics_pb2 import Metrics
+
+
+producer = KafkaProducer(bootstrap_servers='localhost:9092')
+topic = 'metrics_pb'
+
+for row in iter(sys.stdin):
+ d = json.loads(row)
+ metrics = Metrics()
+ for k, v in d.items():
+ setattr(metrics, k, v)
+ pb = metrics.SerializeToString()
+ producer.send(topic, pb)
diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml
new file mode 100644
index 000000000000..01173413f238
--- /dev/null
+++ b/extensions-core/protobuf-extensions/pom.xml
@@ -0,0 +1,123 @@
+
+
+
+
+ 4.0.0
+
+ io.druid.extensions
+ druid-protobuf-extensions
+ druid-protobuf-extensions
+ druid-protobuf-extensions
+
+
+ druid
+ io.druid
+ 0.10.1-SNAPSHOT
+ ../../pom.xml
+
+
+
+ 3.2.0
+
+
+
+
+ io.druid
+ druid-common
+ ${project.parent.version}
+ provided
+
+
+ com.google.protobuf
+ protobuf-java
+ ${protobuf.version}
+
+
+ com.google.protobuf
+ protobuf-java-util
+ ${protobuf.version}
+
+
+ com.github.os72
+ protobuf-dynamic
+ 0.9.3
+
+
+
+ junit
+ junit
+ test
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+ 3.0.0
+
+
+
+ com.google.protobuf
+ shaded.com.google.protobuf
+
+
+
+
+
+ package
+
+ shade
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-resources-plugin
+ 3.0.2
+
+
+ desc
+
+
+
+
+
+
+
+ strict
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+
+
+
+ -Xep:MissingOverride:WARN
+
+
+
+
+
+
+
+
diff --git a/extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufExtensionsModule.java b/extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufExtensionsModule.java
new file mode 100644
index 000000000000..d5b7055a1405
--- /dev/null
+++ b/extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufExtensionsModule.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.data.input.protobuf;
+
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.inject.Binder;
+import io.druid.initialization.DruidModule;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class ProtobufExtensionsModule implements DruidModule
+{
+
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ return Arrays.asList(
+ new SimpleModule("ProtobufInputRowParserModule")
+ .registerSubtypes(
+ new NamedType(ProtobufInputRowParser.class, "protobuf")
+ )
+ );
+ }
+
+ @Override
+ public void configure(Binder binder)
+ { }
+}
diff --git a/extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufInputRowParser.java b/extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufInputRowParser.java
new file mode 100644
index 000000000000..0423e28d8ade
--- /dev/null
+++ b/extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufInputRowParser.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.data.input.protobuf;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.github.os72.protobuf.dynamic.DynamicSchema;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.DynamicMessage;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.util.JsonFormat;
+import io.druid.data.input.ByteBufferInputRowParser;
+import io.druid.data.input.InputRow;
+import io.druid.data.input.MapBasedInputRow;
+import io.druid.data.input.impl.ParseSpec;
+import io.druid.java.util.common.parsers.ParseException;
+import io.druid.java.util.common.parsers.Parser;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.Set;
+
+public class ProtobufInputRowParser implements ByteBufferInputRowParser
+{
+ private final ParseSpec parseSpec;
+ private Parser parser;
+ private final String descriptorFilePath;
+ private final String protoMessageType;
+ private Descriptor descriptor;
+
+
+ @JsonCreator
+ public ProtobufInputRowParser(
+ @JsonProperty("parseSpec") ParseSpec parseSpec,
+ @JsonProperty("descriptor") String descriptorFilePath,
+ @JsonProperty("protoMessageType") String protoMessageType
+ )
+ {
+ this.parseSpec = parseSpec;
+ this.descriptorFilePath = descriptorFilePath;
+ this.protoMessageType = protoMessageType;
+ this.parser = parseSpec.makeParser();
+ this.descriptor = getDescriptor(descriptorFilePath);
+ }
+
+ @Override
+ public ParseSpec getParseSpec()
+ {
+ return parseSpec;
+ }
+
+ @Override
+ public ProtobufInputRowParser withParseSpec(ParseSpec parseSpec)
+ {
+ return new ProtobufInputRowParser(parseSpec, descriptorFilePath, protoMessageType);
+ }
+
+ @Override
+ public InputRow parse(ByteBuffer input)
+ {
+ String json;
+ try {
+ DynamicMessage message = DynamicMessage.parseFrom(descriptor, ByteString.copyFrom(input));
+ json = JsonFormat.printer().print(message);
+ }
+ catch (InvalidProtocolBufferException e) {
+ throw new ParseException(e, "Protobuf message could not be parsed");
+ }
+
+ Map record = parser.parse(json);
+ return new MapBasedInputRow(
+ parseSpec.getTimestampSpec().extractTimestamp(record),
+ parseSpec.getDimensionsSpec().getDimensionNames(),
+ record
+ );
+ }
+
+ private Descriptor getDescriptor(String descriptorFilePath)
+ {
+ InputStream fin;
+
+ fin = this.getClass().getClassLoader().getResourceAsStream(descriptorFilePath);
+ if (fin == null) {
+ URL url = null;
+ try {
+ url = new URL(descriptorFilePath);
+ }
+ catch (MalformedURLException e) {
+ throw new ParseException(e, "Descriptor not found in class path or malformed URL:" + descriptorFilePath);
+ }
+ try {
+ fin = url.openConnection().getInputStream();
+ }
+ catch (IOException e) {
+ throw new ParseException(e, "Cannot read descriptor file: " + url.toString());
+ }
+ }
+
+ DynamicSchema dynamicSchema = null;
+ try {
+ dynamicSchema = DynamicSchema.parseFrom(fin);
+ }
+ catch (Descriptors.DescriptorValidationException e) {
+ throw new ParseException(e, "Invalid descriptor file: " + descriptorFilePath);
+ }
+ catch (IOException e) {
+ throw new ParseException(e, "Cannot read descriptor file: " + descriptorFilePath);
+ }
+
+ Set messageTypes = dynamicSchema.getMessageTypes();
+ if (messageTypes.size() == 0) {
+ throw new ParseException("No message types found in the descriptor: " + descriptorFilePath);
+ }
+
+ String messageType = protoMessageType == null ? (String) messageTypes.toArray()[0] : protoMessageType;
+ Descriptor desc = dynamicSchema.getMessageDescriptor(messageType);
+ if (desc == null) {
+ throw new ParseException(
+ String.format(
+ "Protobuf message type %s not found in the specified descriptor. Available messages types are %s",
+ protoMessageType,
+ messageTypes
+ )
+ );
+ }
+ return desc;
+ }
+}
diff --git a/extensions-core/protobuf-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/protobuf-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule
new file mode 100755
index 000000000000..0a86169f8122
--- /dev/null
+++ b/extensions-core/protobuf-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule
@@ -0,0 +1 @@
+io.druid.data.input.protobuf.ProtobufExtensionsModule
diff --git a/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtoTestEventWrapper.java b/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtoTestEventWrapper.java
new file mode 100644
index 000000000000..bb16245c5b1f
--- /dev/null
+++ b/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtoTestEventWrapper.java
@@ -0,0 +1,3258 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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.
+ */
+
+// Generated by the protocol buffer compiler. DO NOT EDIT!
+// source: ProtoTest.proto
+
+package io.druid.data.input.protobuf;
+
+public final class ProtoTestEventWrapper
+{
+ private ProtoTestEventWrapper() {}
+
+ public static void registerAllExtensions(
+ com.google.protobuf.ExtensionRegistryLite registry
+ )
+ {
+ }
+
+ public static void registerAllExtensions(
+ com.google.protobuf.ExtensionRegistry registry
+ )
+ {
+ registerAllExtensions(
+ (com.google.protobuf.ExtensionRegistryLite) registry);
+ }
+
+ public interface ProtoTestEventOrBuilder extends
+ // @@protoc_insertion_point(interface_extends:prototest.ProtoTestEvent)
+ com.google.protobuf.MessageOrBuilder
+ {
+
+ /**
+ * required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ */
+ boolean hasEventType();
+
+ /**
+ * required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ */
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType();
+
+ /**
+ * required uint64 id = 2;
+ */
+ boolean hasId();
+
+ /**
+ * required uint64 id = 2;
+ */
+ long getId();
+
+ /**
+ * required string timestamp = 3;
+ */
+ boolean hasTimestamp();
+
+ /**
+ * required string timestamp = 3;
+ */
+ java.lang.String getTimestamp();
+
+ /**
+ * required string timestamp = 3;
+ */
+ com.google.protobuf.ByteString
+ getTimestampBytes();
+
+ /**
+ * optional uint32 someOtherId = 4;
+ */
+ boolean hasSomeOtherId();
+
+ /**
+ * optional uint32 someOtherId = 4;
+ */
+ int getSomeOtherId();
+
+ /**
+ * optional bool isValid = 5;
+ */
+ boolean hasIsValid();
+
+ /**
+ * optional bool isValid = 5;
+ */
+ boolean getIsValid();
+
+ /**
+ * optional string description = 6;
+ */
+ boolean hasDescription();
+
+ /**
+ * optional string description = 6;
+ */
+ java.lang.String getDescription();
+
+ /**
+ * optional string description = 6;
+ */
+ com.google.protobuf.ByteString
+ getDescriptionBytes();
+
+ /**
+ * optional float someFloatColumn = 7;
+ */
+ boolean hasSomeFloatColumn();
+
+ /**
+ * optional float someFloatColumn = 7;
+ */
+ float getSomeFloatColumn();
+
+ /**
+ * optional uint32 someIntColumn = 8;
+ */
+ boolean hasSomeIntColumn();
+
+ /**
+ * optional uint32 someIntColumn = 8;
+ */
+ int getSomeIntColumn();
+
+ /**
+ * optional uint64 someLongColumn = 9;
+ */
+ boolean hasSomeLongColumn();
+
+ /**
+ * optional uint64 someLongColumn = 9;
+ */
+ long getSomeLongColumn();
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ boolean hasFoo();
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getFoo();
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getFooOrBuilder();
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ java.util.List
+ getBarList();
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ io.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 io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
+ getBarOrBuilderList();
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getBarOrBuilder(
+ int index
+ );
+ }
+
+ /**
+ * Protobuf type {@code prototest.ProtoTestEvent}
+ */
+ public static final class ProtoTestEvent extends
+ com.google.protobuf.GeneratedMessageV3 implements
+ // @@protoc_insertion_point(message_implements:prototest.ProtoTestEvent)
+ ProtoTestEventOrBuilder
+ {
+ // Use ProtoTestEvent.newBuilder() to construct.
+ private ProtoTestEvent(com.google.protobuf.GeneratedMessageV3.Builder> builder)
+ {
+ super(builder);
+ }
+
+ 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 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
+ {
+ this();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag
+ )) {
+ done = true;
+ }
+ break;
+ }
+ case 8: {
+ int rawValue = input.readEnum();
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value = io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory
+ .valueOf(rawValue);
+ if (value == null) {
+ unknownFields.mergeVarintField(1, rawValue);
+ } else {
+ bitField0_ |= 0x00000001;
+ eventType_ = rawValue;
+ }
+ break;
+ }
+ case 16: {
+ bitField0_ |= 0x00000002;
+ id_ = input.readUInt64();
+ break;
+ }
+ case 26: {
+ com.google.protobuf.ByteString bs = input.readBytes();
+ bitField0_ |= 0x00000004;
+ timestamp_ = bs;
+ break;
+ }
+ case 32: {
+ bitField0_ |= 0x00000008;
+ someOtherId_ = input.readUInt32();
+ break;
+ }
+ case 40: {
+ bitField0_ |= 0x00000010;
+ isValid_ = input.readBool();
+ break;
+ }
+ case 50: {
+ com.google.protobuf.ByteString bs = input.readBytes();
+ bitField0_ |= 0x00000020;
+ description_ = bs;
+ break;
+ }
+ case 61: {
+ bitField0_ |= 0x00000040;
+ someFloatColumn_ = input.readFloat();
+ break;
+ }
+ case 64: {
+ bitField0_ |= 0x00000080;
+ someIntColumn_ = input.readUInt32();
+ break;
+ }
+ case 72: {
+ bitField0_ |= 0x00000100;
+ someLongColumn_ = input.readUInt64();
+ break;
+ }
+ case 82: {
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000200) == 0x00000200)) {
+ subBuilder = foo_.toBuilder();
+ }
+ foo_ = input.readMessage(
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.PARSER,
+ extensionRegistry
+ );
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(foo_);
+ foo_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000200;
+ break;
+ }
+ case 90: {
+ if (!((mutable_bitField0_ & 0x00000400) == 0x00000400)) {
+ bar_ = new java.util.ArrayList();
+ mutable_bitField0_ |= 0x00000400;
+ }
+ bar_.add(
+ input.readMessage(
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.PARSER,
+ extensionRegistry
+ ));
+ break;
+ }
+ }
+ }
+ }
+ catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ }
+ catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e).setUnfinishedMessage(this);
+ }
+ finally {
+ if (((mutable_bitField0_ & 0x00000400) == 0x00000400)) {
+ bar_ = java.util.Collections.unmodifiableList(bar_);
+ }
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internalGetFieldAccessorTable()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.class,
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Builder.class
+ );
+ }
+
+ /**
+ * Protobuf enum {@code prototest.ProtoTestEvent.EventCategory}
+ */
+ public enum EventCategory
+ implements com.google.protobuf.ProtocolMessageEnum
+ {
+ /**
+ * CATEGORY_ZERO = 0;
+ */
+ CATEGORY_ZERO(0),
+ /**
+ * CATEGORY_ONE = 1;
+ */
+ CATEGORY_ONE(1),
+ /**
+ * CATEGORY_TWO = 2;
+ */
+ CATEGORY_TWO(2),;
+
+ /**
+ * CATEGORY_ZERO = 0;
+ */
+ public static final int CATEGORY_ZERO_VALUE = 0;
+ /**
+ * CATEGORY_ONE = 1;
+ */
+ public static final int CATEGORY_ONE_VALUE = 1;
+ /**
+ * CATEGORY_TWO = 2;
+ */
+ public static final int CATEGORY_TWO_VALUE = 2;
+
+
+ public final int getNumber()
+ {
+ return value;
+ }
+
+ /**
+ * @deprecated Use {@link #forNumber(int)} instead.
+ */
+ @java.lang.Deprecated
+ public static EventCategory valueOf(int value)
+ {
+ return forNumber(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;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap
+ 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);
+ }
+ };
+
+ public final com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor()
+ {
+ return getDescriptor().getValues().get(ordinal());
+ }
+
+ public final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType()
+ {
+ return getDescriptor();
+ }
+
+ public static final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor()
+ {
+ return io.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
+ )
+ {
+ if (desc.getType() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "EnumValueDescriptor is not for this type.");
+ }
+ return VALUES[desc.getIndex()];
+ }
+
+ private final int value;
+
+ private EventCategory(int value)
+ {
+ this.value = value;
+ }
+
+ // @@protoc_insertion_point(enum_scope:prototest.ProtoTestEvent.EventCategory)
+ }
+
+ public interface FooOrBuilder extends
+ // @@protoc_insertion_point(interface_extends:prototest.ProtoTestEvent.Foo)
+ com.google.protobuf.MessageOrBuilder
+ {
+
+ /**
+ * required string bar = 1;
+ */
+ boolean hasBar();
+
+ /**
+ * required string bar = 1;
+ */
+ java.lang.String getBar();
+
+ /**
+ * required string bar = 1;
+ */
+ com.google.protobuf.ByteString
+ 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
+ {
+ // Use Foo.newBuilder() to construct.
+ private Foo(com.google.protobuf.GeneratedMessageV3.Builder> builder)
+ {
+ super(builder);
+ }
+
+ private Foo()
+ {
+ bar_ = "";
+ }
+
+ @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
+ {
+ this();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ 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;
+ }
+ }
+ }
+ }
+ catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ }
+ catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e).setUnfinishedMessage(this);
+ }
+ finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internalGetFieldAccessorTable()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.class,
+ io.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;
+ */
+ public boolean hasBar()
+ {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+
+ /**
+ * required string bar = 1;
+ */
+ 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) ref;
+ java.lang.String s = bs.toStringUtf8();
+ if (bs.isValidUtf8()) {
+ bar_ = s;
+ }
+ return s;
+ }
+ }
+
+ /**
+ * required string bar = 1;
+ */
+ public com.google.protobuf.ByteString
+ getBarBytes()
+ {
+ java.lang.Object ref = bar_;
+ if (ref instanceof java.lang.String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ bar_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ private byte memoizedIsInitialized = -1;
+
+ public final boolean isInitialized()
+ {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized == 1) {
+ return true;
+ }
+ if (isInitialized == 0) {
+ return false;
+ }
+
+ if (!hasBar()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException
+ {
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ com.google.protobuf.GeneratedMessageV3.writeString(output, 1, bar_);
+ }
+ unknownFields.writeTo(output);
+ }
+
+ public int getSerializedSize()
+ {
+ int size = memoizedSize;
+ if (size != -1) {
+ return size;
+ }
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, bar_);
+ }
+ size += unknownFields.getSerializedSize();
+ memoizedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+
+ @java.lang.Override
+ public boolean equals(final java.lang.Object obj)
+ {
+ if (obj == this) {
+ return true;
+ }
+ if (!(obj instanceof io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo)) {
+ return super.equals(obj);
+ }
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo other = (io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) obj;
+
+ boolean result = true;
+ result = result && (hasBar() == other.hasBar());
+ if (hasBar()) {
+ result = result && getBar()
+ .equals(other.getBar());
+ }
+ result = result && unknownFields.equals(other.unknownFields);
+ return result;
+ }
+
+ @java.lang.Override
+ public int hashCode()
+ {
+ if (memoizedHashCode != 0) {
+ return memoizedHashCode;
+ }
+ int hash = 41;
+ hash = (19 * hash) + getDescriptor().hashCode();
+ if (hasBar()) {
+ hash = (37 * hash) + BAR_FIELD_NUMBER;
+ hash = (53 * hash) + getBar().hashCode();
+ }
+ hash = (29 * hash) + unknownFields.hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
+ com.google.protobuf.ByteString data
+ )
+ throws com.google.protobuf.InvalidProtocolBufferException
+ {
+ return PARSER.parseFrom(data);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws com.google.protobuf.InvalidProtocolBufferException
+ {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException
+ {
+ return PARSER.parseFrom(data);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws com.google.protobuf.InvalidProtocolBufferException
+ {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(java.io.InputStream input)
+ throws java.io.IOException
+ {
+ return com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws java.io.IOException
+ {
+ return com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input, extensionRegistry);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException
+ {
+ return com.google.protobuf.GeneratedMessageV3
+ .parseDelimitedWithIOException(PARSER, input);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws java.io.IOException
+ {
+ return com.google.protobuf.GeneratedMessageV3
+ .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
+ com.google.protobuf.CodedInputStream input
+ )
+ throws java.io.IOException
+ {
+ return com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ 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()
+ {
+ return DEFAULT_INSTANCE.toBuilder();
+ }
+
+ public static Builder newBuilder(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo prototype)
+ {
+ return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+ }
+
+ public Builder toBuilder()
+ {
+ return this == DEFAULT_INSTANCE
+ ? new Builder() : new Builder().mergeFrom(this);
+ }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessageV3.BuilderParent parent
+ )
+ {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+
+ /**
+ * Protobuf type {@code prototest.ProtoTestEvent.Foo}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessageV3.Builder implements
+ // @@protoc_insertion_point(builder_implements:prototest.ProtoTestEvent.Foo)
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder
+ {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internalGetFieldAccessorTable()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.class,
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder.class
+ );
+ }
+
+ // Construct using io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.newBuilder()
+ private Builder()
+ {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessageV3.BuilderParent parent
+ )
+ {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+
+ private void maybeForceBuilderInitialization()
+ {
+ if (com.google.protobuf.GeneratedMessageV3
+ .alwaysUseFieldBuilders) {
+ }
+ }
+
+ public Builder clear()
+ {
+ super.clear();
+ bar_ = "";
+ bitField0_ = (bitField0_ & ~0x00000001);
+ return this;
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_descriptor;
+ }
+
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getDefaultInstanceForType()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance();
+ }
+
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo build()
+ {
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo buildPartial()
+ {
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo result = new io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo(
+ this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.bar_ = bar_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder clone()
+ {
+ return (Builder) super.clone();
+ }
+
+ public Builder setField(
+ com.google.protobuf.Descriptors.FieldDescriptor field,
+ Object value
+ )
+ {
+ return (Builder) super.setField(field, value);
+ }
+
+ public Builder clearField(
+ com.google.protobuf.Descriptors.FieldDescriptor field
+ )
+ {
+ return (Builder) super.clearField(field);
+ }
+
+ public Builder clearOneof(
+ com.google.protobuf.Descriptors.OneofDescriptor oneof
+ )
+ {
+ return (Builder) super.clearOneof(oneof);
+ }
+
+ public Builder setRepeatedField(
+ com.google.protobuf.Descriptors.FieldDescriptor field,
+ int index, Object value
+ )
+ {
+ return (Builder) super.setRepeatedField(field, index, value);
+ }
+
+ public Builder addRepeatedField(
+ com.google.protobuf.Descriptors.FieldDescriptor field,
+ Object value
+ )
+ {
+ return (Builder) super.addRepeatedField(field, value);
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other)
+ {
+ if (other instanceof io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) {
+ return mergeFrom((io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo other)
+ {
+ if (other == io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()) {
+ return this;
+ }
+ if (other.hasBar()) {
+ bitField0_ |= 0x00000001;
+ bar_ = other.bar_;
+ onChanged();
+ }
+ this.mergeUnknownFields(other.unknownFields);
+ onChanged();
+ return this;
+ }
+
+ public final boolean isInitialized()
+ {
+ if (!hasBar()) {
+ return false;
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws java.io.IOException
+ {
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ }
+ catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) e.getUnfinishedMessage();
+ throw e.unwrapIOException();
+ }
+ finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+
+ private int bitField0_;
+
+ private java.lang.Object bar_ = "";
+
+ /**
+ * required string bar = 1;
+ */
+ public boolean hasBar()
+ {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+
+ /**
+ * required string bar = 1;
+ */
+ public java.lang.String getBar()
+ {
+ java.lang.Object ref = bar_;
+ if (!(ref instanceof java.lang.String)) {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ java.lang.String s = bs.toStringUtf8();
+ if (bs.isValidUtf8()) {
+ bar_ = s;
+ }
+ return s;
+ } else {
+ return (java.lang.String) ref;
+ }
+ }
+
+ /**
+ * required string bar = 1;
+ */
+ public com.google.protobuf.ByteString
+ getBarBytes()
+ {
+ java.lang.Object ref = bar_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ bar_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ /**
+ * required string bar = 1;
+ */
+ public Builder setBar(
+ java.lang.String value
+ )
+ {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ bar_ = value;
+ onChanged();
+ return this;
+ }
+
+ /**
+ * required string bar = 1;
+ */
+ public Builder clearBar()
+ {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ bar_ = getDefaultInstance().getBar();
+ onChanged();
+ return this;
+ }
+
+ /**
+ * required string bar = 1;
+ */
+ public Builder setBarBytes(
+ com.google.protobuf.ByteString value
+ )
+ {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ bar_ = value;
+ onChanged();
+ return this;
+ }
+
+ public final Builder setUnknownFields(
+ final com.google.protobuf.UnknownFieldSet unknownFields
+ )
+ {
+ return super.setUnknownFields(unknownFields);
+ }
+
+ public final Builder mergeUnknownFields(
+ final com.google.protobuf.UnknownFieldSet unknownFields
+ )
+ {
+ return super.mergeUnknownFields(unknownFields);
+ }
+
+
+ // @@protoc_insertion_point(builder_scope:prototest.ProtoTestEvent.Foo)
+ }
+
+ // @@protoc_insertion_point(class_scope:prototest.ProtoTestEvent.Foo)
+ private static final io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo DEFAULT_INSTANCE;
+
+ static {
+ DEFAULT_INSTANCE = new io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo();
+ }
+
+ public static io.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()
+ {
+ public Foo parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws com.google.protobuf.InvalidProtocolBufferException
+ {
+ return new Foo(input, extensionRegistry);
+ }
+ };
+
+ public static com.google.protobuf.Parser parser()
+ {
+ return PARSER;
+ }
+
+ @java.lang.Override
+ public com.google.protobuf.Parser getParserForType()
+ {
+ return PARSER;
+ }
+
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getDefaultInstanceForType()
+ {
+ return DEFAULT_INSTANCE;
+ }
+
+ }
+
+ private int bitField0_;
+ public static final int EVENTTYPE_FIELD_NUMBER = 1;
+ private int eventType_;
+
+ /**
+ * required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ */
+ public boolean hasEventType()
+ {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+
+ /**
+ * required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType()
+ {
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory result = io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory
+ .valueOf(eventType_);
+ return result == null
+ ? io.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;
+ */
+ public boolean hasId()
+ {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+
+ /**
+ * required uint64 id = 2;
+ */
+ public long getId()
+ {
+ return id_;
+ }
+
+ public static final int TIMESTAMP_FIELD_NUMBER = 3;
+ private volatile java.lang.Object timestamp_;
+
+ /**
+ * required string timestamp = 3;
+ */
+ public boolean hasTimestamp()
+ {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+
+ /**
+ * required string timestamp = 3;
+ */
+ 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) ref;
+ java.lang.String s = bs.toStringUtf8();
+ if (bs.isValidUtf8()) {
+ timestamp_ = s;
+ }
+ return s;
+ }
+ }
+
+ /**
+ * required string timestamp = 3;
+ */
+ public com.google.protobuf.ByteString
+ getTimestampBytes()
+ {
+ java.lang.Object ref = timestamp_;
+ if (ref instanceof java.lang.String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ timestamp_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ public static final int SOMEOTHERID_FIELD_NUMBER = 4;
+ private int someOtherId_;
+
+ /**
+ * optional uint32 someOtherId = 4;
+ */
+ public boolean hasSomeOtherId()
+ {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+
+ /**
+ * optional uint32 someOtherId = 4;
+ */
+ public int getSomeOtherId()
+ {
+ return someOtherId_;
+ }
+
+ public static final int ISVALID_FIELD_NUMBER = 5;
+ private boolean isValid_;
+
+ /**
+ * optional bool isValid = 5;
+ */
+ public boolean hasIsValid()
+ {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+
+ /**
+ * optional bool isValid = 5;
+ */
+ public boolean getIsValid()
+ {
+ return isValid_;
+ }
+
+ public static final int DESCRIPTION_FIELD_NUMBER = 6;
+ private volatile java.lang.Object description_;
+
+ /**
+ * optional string description = 6;
+ */
+ public boolean hasDescription()
+ {
+ return ((bitField0_ & 0x00000020) == 0x00000020);
+ }
+
+ /**
+ * optional string description = 6;
+ */
+ 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) ref;
+ java.lang.String s = bs.toStringUtf8();
+ if (bs.isValidUtf8()) {
+ description_ = s;
+ }
+ return s;
+ }
+ }
+
+ /**
+ * optional string description = 6;
+ */
+ public com.google.protobuf.ByteString
+ getDescriptionBytes()
+ {
+ java.lang.Object ref = description_;
+ if (ref instanceof java.lang.String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ description_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ public static final int SOMEFLOATCOLUMN_FIELD_NUMBER = 7;
+ private float someFloatColumn_;
+
+ /**
+ * optional float someFloatColumn = 7;
+ */
+ public boolean hasSomeFloatColumn()
+ {
+ return ((bitField0_ & 0x00000040) == 0x00000040);
+ }
+
+ /**
+ * optional float someFloatColumn = 7;
+ */
+ public float getSomeFloatColumn()
+ {
+ return someFloatColumn_;
+ }
+
+ public static final int SOMEINTCOLUMN_FIELD_NUMBER = 8;
+ private int someIntColumn_;
+
+ /**
+ * optional uint32 someIntColumn = 8;
+ */
+ public boolean hasSomeIntColumn()
+ {
+ return ((bitField0_ & 0x00000080) == 0x00000080);
+ }
+
+ /**
+ * optional uint32 someIntColumn = 8;
+ */
+ public int getSomeIntColumn()
+ {
+ return someIntColumn_;
+ }
+
+ public static final int SOMELONGCOLUMN_FIELD_NUMBER = 9;
+ private long someLongColumn_;
+
+ /**
+ * optional uint64 someLongColumn = 9;
+ */
+ public boolean hasSomeLongColumn()
+ {
+ return ((bitField0_ & 0x00000100) == 0x00000100);
+ }
+
+ /**
+ * optional uint64 someLongColumn = 9;
+ */
+ public long getSomeLongColumn()
+ {
+ return someLongColumn_;
+ }
+
+ public static final int FOO_FIELD_NUMBER = 10;
+ private io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo foo_;
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ public boolean hasFoo()
+ {
+ return ((bitField0_ & 0x00000200) == 0x00000200);
+ }
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getFoo()
+ {
+ return foo_ == null
+ ? io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()
+ : foo_;
+ }
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getFooOrBuilder()
+ {
+ return foo_ == null
+ ? io.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()
+ {
+ return bar_;
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public java.util.List extends io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
+ getBarOrBuilderList()
+ {
+ return bar_;
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public int getBarCount()
+ {
+ return bar_.size();
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getBar(int index)
+ {
+ return bar_.get(index);
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getBarOrBuilder(
+ int index
+ )
+ {
+ return bar_.get(index);
+ }
+
+ private byte memoizedIsInitialized = -1;
+
+ public final boolean isInitialized()
+ {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized == 1) {
+ return true;
+ }
+ if (isInitialized == 0) {
+ return false;
+ }
+
+ if (!hasEventType()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasId()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasTimestamp()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (hasFoo()) {
+ if (!getFoo().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
+ for (int i = 0; i < getBarCount(); i++) {
+ if (!getBar(i).isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException
+ {
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeEnum(1, eventType_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeUInt64(2, id_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ com.google.protobuf.GeneratedMessageV3.writeString(output, 3, timestamp_);
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ output.writeUInt32(4, someOtherId_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ output.writeBool(5, isValid_);
+ }
+ if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ com.google.protobuf.GeneratedMessageV3.writeString(output, 6, description_);
+ }
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ output.writeFloat(7, someFloatColumn_);
+ }
+ if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ output.writeUInt32(8, someIntColumn_);
+ }
+ if (((bitField0_ & 0x00000100) == 0x00000100)) {
+ output.writeUInt64(9, someLongColumn_);
+ }
+ if (((bitField0_ & 0x00000200) == 0x00000200)) {
+ output.writeMessage(10, getFoo());
+ }
+ for (int i = 0; i < bar_.size(); i++) {
+ output.writeMessage(11, bar_.get(i));
+ }
+ unknownFields.writeTo(output);
+ }
+
+ public int getSerializedSize()
+ {
+ int size = memoizedSize;
+ if (size != -1) {
+ return size;
+ }
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeEnumSize(1, eventType_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(2, id_);
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, timestamp_);
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt32Size(4, someOtherId_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(5, isValid_);
+ }
+ if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ size += com.google.protobuf.GeneratedMessageV3.computeStringSize(6, description_);
+ }
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeFloatSize(7, someFloatColumn_);
+ }
+ if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt32Size(8, someIntColumn_);
+ }
+ if (((bitField0_ & 0x00000100) == 0x00000100)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(9, someLongColumn_);
+ }
+ if (((bitField0_ & 0x00000200) == 0x00000200)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(10, getFoo());
+ }
+ for (int i = 0; i < bar_.size(); i++) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(11, bar_.get(i));
+ }
+ size += unknownFields.getSerializedSize();
+ memoizedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+
+ @java.lang.Override
+ public boolean equals(final java.lang.Object obj)
+ {
+ if (obj == this) {
+ return true;
+ }
+ if (!(obj instanceof io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent)) {
+ return super.equals(obj);
+ }
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent other = (io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) obj;
+
+ boolean result = true;
+ result = result && (hasEventType() == other.hasEventType());
+ if (hasEventType()) {
+ result = result && eventType_ == other.eventType_;
+ }
+ result = result && (hasId() == other.hasId());
+ if (hasId()) {
+ result = result && (getId()
+ == other.getId());
+ }
+ result = result && (hasTimestamp() == other.hasTimestamp());
+ if (hasTimestamp()) {
+ result = result && getTimestamp()
+ .equals(other.getTimestamp());
+ }
+ result = result && (hasSomeOtherId() == other.hasSomeOtherId());
+ if (hasSomeOtherId()) {
+ result = result && (getSomeOtherId()
+ == other.getSomeOtherId());
+ }
+ result = result && (hasIsValid() == other.hasIsValid());
+ if (hasIsValid()) {
+ result = result && (getIsValid()
+ == other.getIsValid());
+ }
+ result = result && (hasDescription() == other.hasDescription());
+ if (hasDescription()) {
+ result = result && getDescription()
+ .equals(other.getDescription());
+ }
+ result = result && (hasSomeFloatColumn() == other.hasSomeFloatColumn());
+ if (hasSomeFloatColumn()) {
+ result = result && (
+ java.lang.Float.floatToIntBits(getSomeFloatColumn())
+ == java.lang.Float.floatToIntBits(
+ other.getSomeFloatColumn()));
+ }
+ result = result && (hasSomeIntColumn() == other.hasSomeIntColumn());
+ if (hasSomeIntColumn()) {
+ result = result && (getSomeIntColumn()
+ == other.getSomeIntColumn());
+ }
+ result = result && (hasSomeLongColumn() == other.hasSomeLongColumn());
+ if (hasSomeLongColumn()) {
+ result = result && (getSomeLongColumn()
+ == other.getSomeLongColumn());
+ }
+ result = result && (hasFoo() == other.hasFoo());
+ if (hasFoo()) {
+ result = result && getFoo()
+ .equals(other.getFoo());
+ }
+ result = result && getBarList()
+ .equals(other.getBarList());
+ result = result && unknownFields.equals(other.unknownFields);
+ return result;
+ }
+
+ @java.lang.Override
+ public int hashCode()
+ {
+ if (memoizedHashCode != 0) {
+ return memoizedHashCode;
+ }
+ int hash = 41;
+ hash = (19 * hash) + getDescriptor().hashCode();
+ if (hasEventType()) {
+ hash = (37 * hash) + EVENTTYPE_FIELD_NUMBER;
+ hash = (53 * hash) + eventType_;
+ }
+ if (hasId()) {
+ hash = (37 * hash) + ID_FIELD_NUMBER;
+ hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+ getId());
+ }
+ if (hasTimestamp()) {
+ hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
+ hash = (53 * hash) + getTimestamp().hashCode();
+ }
+ if (hasSomeOtherId()) {
+ hash = (37 * hash) + SOMEOTHERID_FIELD_NUMBER;
+ hash = (53 * hash) + getSomeOtherId();
+ }
+ if (hasIsValid()) {
+ hash = (37 * hash) + ISVALID_FIELD_NUMBER;
+ hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+ getIsValid());
+ }
+ if (hasDescription()) {
+ hash = (37 * hash) + DESCRIPTION_FIELD_NUMBER;
+ hash = (53 * hash) + getDescription().hashCode();
+ }
+ if (hasSomeFloatColumn()) {
+ hash = (37 * hash) + SOMEFLOATCOLUMN_FIELD_NUMBER;
+ hash = (53 * hash) + java.lang.Float.floatToIntBits(
+ getSomeFloatColumn());
+ }
+ if (hasSomeIntColumn()) {
+ hash = (37 * hash) + SOMEINTCOLUMN_FIELD_NUMBER;
+ hash = (53 * hash) + getSomeIntColumn();
+ }
+ if (hasSomeLongColumn()) {
+ hash = (37 * hash) + SOMELONGCOLUMN_FIELD_NUMBER;
+ hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
+ getSomeLongColumn());
+ }
+ if (hasFoo()) {
+ hash = (37 * hash) + FOO_FIELD_NUMBER;
+ hash = (53 * hash) + getFoo().hashCode();
+ }
+ if (getBarCount() > 0) {
+ hash = (37 * hash) + BAR_FIELD_NUMBER;
+ hash = (53 * hash) + getBarList().hashCode();
+ }
+ hash = (29 * hash) + unknownFields.hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
+ com.google.protobuf.ByteString data
+ )
+ throws com.google.protobuf.InvalidProtocolBufferException
+ {
+ return PARSER.parseFrom(data);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws com.google.protobuf.InvalidProtocolBufferException
+ {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException
+ {
+ return PARSER.parseFrom(data);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws com.google.protobuf.InvalidProtocolBufferException
+ {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(java.io.InputStream input)
+ throws java.io.IOException
+ {
+ return com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws java.io.IOException
+ {
+ return com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input, extensionRegistry);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException
+ {
+ return com.google.protobuf.GeneratedMessageV3
+ .parseDelimitedWithIOException(PARSER, input);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws java.io.IOException
+ {
+ return com.google.protobuf.GeneratedMessageV3
+ .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
+ com.google.protobuf.CodedInputStream input
+ )
+ throws java.io.IOException
+ {
+ return com.google.protobuf.GeneratedMessageV3
+ .parseWithIOException(PARSER, input);
+ }
+
+ public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ 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()
+ {
+ return DEFAULT_INSTANCE.toBuilder();
+ }
+
+ public static Builder newBuilder(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent prototype)
+ {
+ return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+ }
+
+ public Builder toBuilder()
+ {
+ return this == DEFAULT_INSTANCE
+ ? new Builder() : new Builder().mergeFrom(this);
+ }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessageV3.BuilderParent parent
+ )
+ {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+
+ /**
+ * Protobuf type {@code prototest.ProtoTestEvent}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessageV3.Builder implements
+ // @@protoc_insertion_point(builder_implements:prototest.ProtoTestEvent)
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEventOrBuilder
+ {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+ internalGetFieldAccessorTable()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.class,
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Builder.class
+ );
+ }
+
+ // Construct using io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
+ private Builder()
+ {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessageV3.BuilderParent parent
+ )
+ {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+
+ private void maybeForceBuilderInitialization()
+ {
+ if (com.google.protobuf.GeneratedMessageV3
+ .alwaysUseFieldBuilders) {
+ getFooFieldBuilder();
+ getBarFieldBuilder();
+ }
+ }
+
+ public Builder clear()
+ {
+ super.clear();
+ eventType_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ id_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ timestamp_ = "";
+ bitField0_ = (bitField0_ & ~0x00000004);
+ someOtherId_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000008);
+ isValid_ = false;
+ bitField0_ = (bitField0_ & ~0x00000010);
+ description_ = "";
+ bitField0_ = (bitField0_ & ~0x00000020);
+ someFloatColumn_ = 0F;
+ bitField0_ = (bitField0_ & ~0x00000040);
+ someIntColumn_ = 0;
+ bitField0_ = (bitField0_ & ~0x00000080);
+ someLongColumn_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000100);
+ if (fooBuilder_ == null) {
+ foo_ = null;
+ } else {
+ fooBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000200);
+ if (barBuilder_ == null) {
+ bar_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000400);
+ } else {
+ barBuilder_.clear();
+ }
+ return this;
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor;
+ }
+
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType()
+ {
+ return io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance();
+ }
+
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent build()
+ {
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent buildPartial()
+ {
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent result = new io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent(
+ this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.eventType_ = eventType_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.id_ = id_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.timestamp_ = timestamp_;
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000008;
+ }
+ result.someOtherId_ = someOtherId_;
+ if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+ to_bitField0_ |= 0x00000010;
+ }
+ result.isValid_ = isValid_;
+ if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+ to_bitField0_ |= 0x00000020;
+ }
+ result.description_ = description_;
+ if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+ to_bitField0_ |= 0x00000040;
+ }
+ result.someFloatColumn_ = someFloatColumn_;
+ if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+ to_bitField0_ |= 0x00000080;
+ }
+ result.someIntColumn_ = someIntColumn_;
+ if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+ to_bitField0_ |= 0x00000100;
+ }
+ result.someLongColumn_ = someLongColumn_;
+ if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
+ to_bitField0_ |= 0x00000200;
+ }
+ if (fooBuilder_ == null) {
+ result.foo_ = foo_;
+ } else {
+ result.foo_ = fooBuilder_.build();
+ }
+ if (barBuilder_ == null) {
+ if (((bitField0_ & 0x00000400) == 0x00000400)) {
+ bar_ = java.util.Collections.unmodifiableList(bar_);
+ bitField0_ = (bitField0_ & ~0x00000400);
+ }
+ result.bar_ = bar_;
+ } else {
+ result.bar_ = barBuilder_.build();
+ }
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder clone()
+ {
+ return (Builder) super.clone();
+ }
+
+ public Builder setField(
+ com.google.protobuf.Descriptors.FieldDescriptor field,
+ Object value
+ )
+ {
+ return (Builder) super.setField(field, value);
+ }
+
+ public Builder clearField(
+ com.google.protobuf.Descriptors.FieldDescriptor field
+ )
+ {
+ return (Builder) super.clearField(field);
+ }
+
+ public Builder clearOneof(
+ com.google.protobuf.Descriptors.OneofDescriptor oneof
+ )
+ {
+ return (Builder) super.clearOneof(oneof);
+ }
+
+ public Builder setRepeatedField(
+ com.google.protobuf.Descriptors.FieldDescriptor field,
+ int index, Object value
+ )
+ {
+ return (Builder) super.setRepeatedField(field, index, value);
+ }
+
+ public Builder addRepeatedField(
+ com.google.protobuf.Descriptors.FieldDescriptor field,
+ Object value
+ )
+ {
+ return (Builder) super.addRepeatedField(field, value);
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other)
+ {
+ if (other instanceof io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) {
+ return mergeFrom((io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent other)
+ {
+ if (other == io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance()) {
+ return this;
+ }
+ if (other.hasEventType()) {
+ setEventType(other.getEventType());
+ }
+ if (other.hasId()) {
+ setId(other.getId());
+ }
+ if (other.hasTimestamp()) {
+ bitField0_ |= 0x00000004;
+ timestamp_ = other.timestamp_;
+ onChanged();
+ }
+ if (other.hasSomeOtherId()) {
+ setSomeOtherId(other.getSomeOtherId());
+ }
+ if (other.hasIsValid()) {
+ setIsValid(other.getIsValid());
+ }
+ if (other.hasDescription()) {
+ bitField0_ |= 0x00000020;
+ description_ = other.description_;
+ onChanged();
+ }
+ if (other.hasSomeFloatColumn()) {
+ setSomeFloatColumn(other.getSomeFloatColumn());
+ }
+ if (other.hasSomeIntColumn()) {
+ setSomeIntColumn(other.getSomeIntColumn());
+ }
+ if (other.hasSomeLongColumn()) {
+ setSomeLongColumn(other.getSomeLongColumn());
+ }
+ if (other.hasFoo()) {
+ mergeFoo(other.getFoo());
+ }
+ if (barBuilder_ == null) {
+ if (!other.bar_.isEmpty()) {
+ if (bar_.isEmpty()) {
+ bar_ = other.bar_;
+ bitField0_ = (bitField0_ & ~0x00000400);
+ } else {
+ ensureBarIsMutable();
+ bar_.addAll(other.bar_);
+ }
+ onChanged();
+ }
+ } else {
+ if (!other.bar_.isEmpty()) {
+ if (barBuilder_.isEmpty()) {
+ barBuilder_.dispose();
+ barBuilder_ = null;
+ bar_ = other.bar_;
+ bitField0_ = (bitField0_ & ~0x00000400);
+ barBuilder_ =
+ com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+ getBarFieldBuilder() : null;
+ } else {
+ barBuilder_.addAllMessages(other.bar_);
+ }
+ }
+ }
+ this.mergeUnknownFields(other.unknownFields);
+ onChanged();
+ return this;
+ }
+
+ public final boolean isInitialized()
+ {
+ if (!hasEventType()) {
+ return false;
+ }
+ if (!hasId()) {
+ return false;
+ }
+ if (!hasTimestamp()) {
+ return false;
+ }
+ if (hasFoo()) {
+ if (!getFoo().isInitialized()) {
+ return false;
+ }
+ }
+ for (int i = 0; i < getBarCount(); i++) {
+ if (!getBar(i).isInitialized()) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws java.io.IOException
+ {
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ }
+ catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) e.getUnfinishedMessage();
+ throw e.unwrapIOException();
+ }
+ finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+
+ private int bitField0_;
+
+ private int eventType_ = 0;
+
+ /**
+ * required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ */
+ public boolean hasEventType()
+ {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+
+ /**
+ * required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType()
+ {
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory result = io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory
+ .valueOf(eventType_);
+ return result == null
+ ? io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO
+ : result;
+ }
+
+ /**
+ * required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ */
+ public Builder setEventType(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value)
+ {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ eventType_ = value.getNumber();
+ onChanged();
+ return this;
+ }
+
+ /**
+ * required .prototest.ProtoTestEvent.EventCategory eventType = 1;
+ */
+ public Builder clearEventType()
+ {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ eventType_ = 0;
+ onChanged();
+ return this;
+ }
+
+ private long id_;
+
+ /**
+ * required uint64 id = 2;
+ */
+ public boolean hasId()
+ {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+
+ /**
+ * required uint64 id = 2;
+ */
+ public long getId()
+ {
+ return id_;
+ }
+
+ /**
+ * required uint64 id = 2;
+ */
+ public Builder setId(long value)
+ {
+ bitField0_ |= 0x00000002;
+ id_ = value;
+ onChanged();
+ return this;
+ }
+
+ /**
+ * required uint64 id = 2;
+ */
+ public Builder clearId()
+ {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ id_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ private java.lang.Object timestamp_ = "";
+
+ /**
+ * required string timestamp = 3;
+ */
+ public boolean hasTimestamp()
+ {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+
+ /**
+ * required string timestamp = 3;
+ */
+ public java.lang.String getTimestamp()
+ {
+ java.lang.Object ref = timestamp_;
+ if (!(ref instanceof java.lang.String)) {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ java.lang.String s = bs.toStringUtf8();
+ if (bs.isValidUtf8()) {
+ timestamp_ = s;
+ }
+ return s;
+ } else {
+ return (java.lang.String) ref;
+ }
+ }
+
+ /**
+ * required string timestamp = 3;
+ */
+ public com.google.protobuf.ByteString
+ getTimestampBytes()
+ {
+ java.lang.Object ref = timestamp_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ timestamp_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ /**
+ * required string timestamp = 3;
+ */
+ public Builder setTimestamp(
+ java.lang.String value
+ )
+ {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000004;
+ timestamp_ = value;
+ onChanged();
+ return this;
+ }
+
+ /**
+ * required string timestamp = 3;
+ */
+ public Builder clearTimestamp()
+ {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ timestamp_ = getDefaultInstance().getTimestamp();
+ onChanged();
+ return this;
+ }
+
+ /**
+ * required string timestamp = 3;
+ */
+ public Builder setTimestampBytes(
+ com.google.protobuf.ByteString value
+ )
+ {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000004;
+ timestamp_ = value;
+ onChanged();
+ return this;
+ }
+
+ private int someOtherId_;
+
+ /**
+ * optional uint32 someOtherId = 4;
+ */
+ public boolean hasSomeOtherId()
+ {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+
+ /**
+ * optional uint32 someOtherId = 4;
+ */
+ public int getSomeOtherId()
+ {
+ return someOtherId_;
+ }
+
+ /**
+ * optional uint32 someOtherId = 4;
+ */
+ public Builder setSomeOtherId(int value)
+ {
+ bitField0_ |= 0x00000008;
+ someOtherId_ = value;
+ onChanged();
+ return this;
+ }
+
+ /**
+ * optional uint32 someOtherId = 4;
+ */
+ public Builder clearSomeOtherId()
+ {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ someOtherId_ = 0;
+ onChanged();
+ return this;
+ }
+
+ private boolean isValid_;
+
+ /**
+ * optional bool isValid = 5;
+ */
+ public boolean hasIsValid()
+ {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+
+ /**
+ * optional bool isValid = 5;
+ */
+ public boolean getIsValid()
+ {
+ return isValid_;
+ }
+
+ /**
+ * optional bool isValid = 5;
+ */
+ public Builder setIsValid(boolean value)
+ {
+ bitField0_ |= 0x00000010;
+ isValid_ = value;
+ onChanged();
+ return this;
+ }
+
+ /**
+ * optional bool isValid = 5;
+ */
+ public Builder clearIsValid()
+ {
+ bitField0_ = (bitField0_ & ~0x00000010);
+ isValid_ = false;
+ onChanged();
+ return this;
+ }
+
+ private java.lang.Object description_ = "";
+
+ /**
+ * optional string description = 6;
+ */
+ public boolean hasDescription()
+ {
+ return ((bitField0_ & 0x00000020) == 0x00000020);
+ }
+
+ /**
+ * optional string description = 6;
+ */
+ public java.lang.String getDescription()
+ {
+ java.lang.Object ref = description_;
+ if (!(ref instanceof java.lang.String)) {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ java.lang.String s = bs.toStringUtf8();
+ if (bs.isValidUtf8()) {
+ description_ = s;
+ }
+ return s;
+ } else {
+ return (java.lang.String) ref;
+ }
+ }
+
+ /**
+ * optional string description = 6;
+ */
+ public com.google.protobuf.ByteString
+ getDescriptionBytes()
+ {
+ java.lang.Object ref = description_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ description_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ /**
+ * optional string description = 6;
+ */
+ public Builder setDescription(
+ java.lang.String value
+ )
+ {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000020;
+ description_ = value;
+ onChanged();
+ return this;
+ }
+
+ /**
+ * optional string description = 6;
+ */
+ public Builder clearDescription()
+ {
+ bitField0_ = (bitField0_ & ~0x00000020);
+ description_ = getDefaultInstance().getDescription();
+ onChanged();
+ return this;
+ }
+
+ /**
+ * optional string description = 6;
+ */
+ public Builder setDescriptionBytes(
+ com.google.protobuf.ByteString value
+ )
+ {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000020;
+ description_ = value;
+ onChanged();
+ return this;
+ }
+
+ private float someFloatColumn_;
+
+ /**
+ * optional float someFloatColumn = 7;
+ */
+ public boolean hasSomeFloatColumn()
+ {
+ return ((bitField0_ & 0x00000040) == 0x00000040);
+ }
+
+ /**
+ * optional float someFloatColumn = 7;
+ */
+ public float getSomeFloatColumn()
+ {
+ return someFloatColumn_;
+ }
+
+ /**
+ * optional float someFloatColumn = 7;
+ */
+ public Builder setSomeFloatColumn(float value)
+ {
+ bitField0_ |= 0x00000040;
+ someFloatColumn_ = value;
+ onChanged();
+ return this;
+ }
+
+ /**
+ * optional float someFloatColumn = 7;
+ */
+ public Builder clearSomeFloatColumn()
+ {
+ bitField0_ = (bitField0_ & ~0x00000040);
+ someFloatColumn_ = 0F;
+ onChanged();
+ return this;
+ }
+
+ private int someIntColumn_;
+
+ /**
+ * optional uint32 someIntColumn = 8;
+ */
+ public boolean hasSomeIntColumn()
+ {
+ return ((bitField0_ & 0x00000080) == 0x00000080);
+ }
+
+ /**
+ * optional uint32 someIntColumn = 8;
+ */
+ public int getSomeIntColumn()
+ {
+ return someIntColumn_;
+ }
+
+ /**
+ * optional uint32 someIntColumn = 8;
+ */
+ public Builder setSomeIntColumn(int value)
+ {
+ bitField0_ |= 0x00000080;
+ someIntColumn_ = value;
+ onChanged();
+ return this;
+ }
+
+ /**
+ * optional uint32 someIntColumn = 8;
+ */
+ public Builder clearSomeIntColumn()
+ {
+ bitField0_ = (bitField0_ & ~0x00000080);
+ someIntColumn_ = 0;
+ onChanged();
+ return this;
+ }
+
+ private long someLongColumn_;
+
+ /**
+ * optional uint64 someLongColumn = 9;
+ */
+ public boolean hasSomeLongColumn()
+ {
+ return ((bitField0_ & 0x00000100) == 0x00000100);
+ }
+
+ /**
+ * optional uint64 someLongColumn = 9;
+ */
+ public long getSomeLongColumn()
+ {
+ return someLongColumn_;
+ }
+
+ /**
+ * optional uint64 someLongColumn = 9;
+ */
+ public Builder setSomeLongColumn(long value)
+ {
+ bitField0_ |= 0x00000100;
+ someLongColumn_ = value;
+ onChanged();
+ return this;
+ }
+
+ /**
+ * optional uint64 someLongColumn = 9;
+ */
+ public Builder clearSomeLongColumn()
+ {
+ bitField0_ = (bitField0_ & ~0x00000100);
+ someLongColumn_ = 0L;
+ onChanged();
+ return this;
+ }
+
+ private io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo foo_ = null;
+ private com.google.protobuf.SingleFieldBuilderV3<
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder> fooBuilder_;
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ public boolean hasFoo()
+ {
+ return ((bitField0_ & 0x00000200) == 0x00000200);
+ }
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getFoo()
+ {
+ if (fooBuilder_ == null) {
+ return foo_ == null
+ ? io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()
+ : foo_;
+ } else {
+ return fooBuilder_.getMessage();
+ }
+ }
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ public Builder setFoo(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value)
+ {
+ if (fooBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ foo_ = value;
+ onChanged();
+ } else {
+ fooBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000200;
+ return this;
+ }
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ public Builder setFoo(
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue
+ )
+ {
+ if (fooBuilder_ == null) {
+ foo_ = builderForValue.build();
+ onChanged();
+ } else {
+ fooBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000200;
+ return this;
+ }
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ public Builder mergeFoo(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value)
+ {
+ if (fooBuilder_ == null) {
+ if (((bitField0_ & 0x00000200) == 0x00000200) &&
+ foo_ != null &&
+ foo_ != io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()) {
+ foo_ =
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.newBuilder(foo_)
+ .mergeFrom(value)
+ .buildPartial();
+ } else {
+ foo_ = value;
+ }
+ onChanged();
+ } else {
+ fooBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000200;
+ return this;
+ }
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ public Builder clearFoo()
+ {
+ if (fooBuilder_ == null) {
+ foo_ = null;
+ onChanged();
+ } else {
+ fooBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000200);
+ return this;
+ }
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder getFooBuilder()
+ {
+ bitField0_ |= 0x00000200;
+ onChanged();
+ return getFooFieldBuilder().getBuilder();
+ }
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getFooOrBuilder()
+ {
+ if (fooBuilder_ != null) {
+ return fooBuilder_.getMessageOrBuilder();
+ } else {
+ return foo_ == null ?
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance() : foo_;
+ }
+ }
+
+ /**
+ * optional .prototest.ProtoTestEvent.Foo foo = 10;
+ */
+ private com.google.protobuf.SingleFieldBuilderV3<
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
+ getFooFieldBuilder()
+ {
+ if (fooBuilder_ == null) {
+ fooBuilder_ = new com.google.protobuf.SingleFieldBuilderV3<
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>(
+ getFoo(),
+ getParentForChildren(),
+ isClean()
+ );
+ foo_ = null;
+ }
+ return fooBuilder_;
+ }
+
+ private java.util.List bar_ =
+ java.util.Collections.emptyList();
+
+ private void ensureBarIsMutable()
+ {
+ if (!((bitField0_ & 0x00000400) == 0x00000400)) {
+ bar_ = new java.util.ArrayList(bar_);
+ bitField0_ |= 0x00000400;
+ }
+ }
+
+ private com.google.protobuf.RepeatedFieldBuilderV3<
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder> barBuilder_;
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ 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()
+ {
+ if (barBuilder_ == null) {
+ return bar_.size();
+ } else {
+ return barBuilder_.getCount();
+ }
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public io.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, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value
+ )
+ {
+ if (barBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureBarIsMutable();
+ bar_.set(index, value);
+ onChanged();
+ } else {
+ barBuilder_.setMessage(index, value);
+ }
+ return this;
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public Builder setBar(
+ int index, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue
+ )
+ {
+ if (barBuilder_ == null) {
+ ensureBarIsMutable();
+ bar_.set(index, builderForValue.build());
+ onChanged();
+ } else {
+ barBuilder_.setMessage(index, builderForValue.build());
+ }
+ return this;
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public Builder addBar(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value)
+ {
+ if (barBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureBarIsMutable();
+ bar_.add(value);
+ onChanged();
+ } else {
+ barBuilder_.addMessage(value);
+ }
+ return this;
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public Builder addBar(
+ int index, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value
+ )
+ {
+ if (barBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureBarIsMutable();
+ bar_.add(index, value);
+ onChanged();
+ } else {
+ barBuilder_.addMessage(index, value);
+ }
+ return this;
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public Builder addBar(
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue
+ )
+ {
+ if (barBuilder_ == null) {
+ ensureBarIsMutable();
+ bar_.add(builderForValue.build());
+ onChanged();
+ } else {
+ barBuilder_.addMessage(builderForValue.build());
+ }
+ return this;
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public Builder addBar(
+ int index, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue
+ )
+ {
+ if (barBuilder_ == null) {
+ ensureBarIsMutable();
+ bar_.add(index, builderForValue.build());
+ onChanged();
+ } else {
+ barBuilder_.addMessage(index, builderForValue.build());
+ }
+ return this;
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public Builder addAllBar(
+ java.lang.Iterable extends io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo> values
+ )
+ {
+ if (barBuilder_ == null) {
+ ensureBarIsMutable();
+ com.google.protobuf.AbstractMessageLite.Builder.addAll(
+ values, bar_);
+ onChanged();
+ } else {
+ barBuilder_.addAllMessages(values);
+ }
+ return this;
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public Builder clearBar()
+ {
+ if (barBuilder_ == null) {
+ bar_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000400);
+ onChanged();
+ } else {
+ barBuilder_.clear();
+ }
+ return this;
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public Builder removeBar(int index)
+ {
+ if (barBuilder_ == null) {
+ ensureBarIsMutable();
+ bar_.remove(index);
+ onChanged();
+ } else {
+ barBuilder_.remove(index);
+ }
+ return this;
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder getBarBuilder(
+ int index
+ )
+ {
+ return getBarFieldBuilder().getBuilder(index);
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getBarOrBuilder(
+ int index
+ )
+ {
+ if (barBuilder_ == null) {
+ return bar_.get(index);
+ } else {
+ return barBuilder_.getMessageOrBuilder(index);
+ }
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public java.util.List extends io.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 io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder addBarBuilder()
+ {
+ return getBarFieldBuilder().addBuilder(
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance());
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder addBarBuilder(
+ int index
+ )
+ {
+ return getBarFieldBuilder().addBuilder(
+ index, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance());
+ }
+
+ /**
+ * repeated .prototest.ProtoTestEvent.Foo bar = 11;
+ */
+ public java.util.List
+ getBarBuilderList()
+ {
+ return getBarFieldBuilder().getBuilderList();
+ }
+
+ private com.google.protobuf.RepeatedFieldBuilderV3<
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>
+ getBarFieldBuilder()
+ {
+ if (barBuilder_ == null) {
+ barBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+ io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>(
+ bar_,
+ ((bitField0_ & 0x00000400) == 0x00000400),
+ getParentForChildren(),
+ isClean()
+ );
+ bar_ = null;
+ }
+ return barBuilder_;
+ }
+
+ public final Builder setUnknownFields(
+ final com.google.protobuf.UnknownFieldSet unknownFields
+ )
+ {
+ return super.setUnknownFields(unknownFields);
+ }
+
+ public final Builder mergeUnknownFields(
+ final com.google.protobuf.UnknownFieldSet unknownFields
+ )
+ {
+ return super.mergeUnknownFields(unknownFields);
+ }
+
+
+ // @@protoc_insertion_point(builder_scope:prototest.ProtoTestEvent)
+ }
+
+ // @@protoc_insertion_point(class_scope:prototest.ProtoTestEvent)
+ private static final io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent DEFAULT_INSTANCE;
+
+ static {
+ DEFAULT_INSTANCE = new io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent();
+ }
+
+ public static io.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()
+ {
+ public ProtoTestEvent parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry
+ )
+ throws com.google.protobuf.InvalidProtocolBufferException
+ {
+ return new ProtoTestEvent(input, extensionRegistry);
+ }
+ };
+
+ public static com.google.protobuf.Parser parser()
+ {
+ return PARSER;
+ }
+
+ @java.lang.Override
+ public com.google.protobuf.Parser getParserForType()
+ {
+ return PARSER;
+ }
+
+ public io.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_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_fieldAccessorTable;
+
+ public static com.google.protobuf.Descriptors.FileDescriptor
+ getDescriptor()
+ {
+ return descriptor;
+ }
+
+ private static com.google.protobuf.Descriptors.FileDescriptor
+ descriptor;
+
+ static {
+ java.lang.String[] descriptorData = {
+ "\n\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\034io.druid.data.input.protobu" +
+ "fB\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
+ );
+ internal_static_prototest_ProtoTestEvent_descriptor =
+ getDescriptor().getMessageTypes().get(0);
+ internal_static_prototest_ProtoTestEvent_fieldAccessorTable = new
+ 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",
+ }
+ );
+ internal_static_prototest_ProtoTestEvent_Foo_descriptor =
+ internal_static_prototest_ProtoTestEvent_descriptor.getNestedTypes().get(0);
+ internal_static_prototest_ProtoTestEvent_Foo_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+ internal_static_prototest_ProtoTestEvent_Foo_descriptor,
+ new java.lang.String[]{"Bar",}
+ );
+ }
+
+ // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtobufInputRowParserTest.java b/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtobufInputRowParserTest.java
new file mode 100644
index 000000000000..19a95087ca1d
--- /dev/null
+++ b/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtobufInputRowParserTest.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.data.input.protobuf;
+
+import com.google.common.collect.Lists;
+import io.druid.data.input.InputRow;
+import io.druid.data.input.impl.DimensionSchema;
+import io.druid.data.input.impl.DimensionsSpec;
+import io.druid.data.input.impl.JSONParseSpec;
+import io.druid.data.input.impl.JSONPathFieldSpec;
+import io.druid.data.input.impl.JSONPathFieldType;
+import io.druid.data.input.impl.JSONPathSpec;
+import io.druid.data.input.impl.ParseSpec;
+import io.druid.data.input.impl.StringDimensionSchema;
+import io.druid.data.input.impl.TimestampSpec;
+import io.druid.java.util.common.parsers.ParseException;
+import org.joda.time.DateTime;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class ProtobufInputRowParserTest
+{
+ private ParseSpec parseSpec;
+
+ @Before
+ public void setUp() throws Exception
+ {
+ parseSpec = new JSONParseSpec(
+ new TimestampSpec("timestamp", "iso", null),
+ new DimensionsSpec(Lists.newArrayList(
+ new StringDimensionSchema("event"),
+ new StringDimensionSchema("id"),
+ new StringDimensionSchema("someOtherId"),
+ new StringDimensionSchema("isValid")
+ ), null, null),
+ 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")
+ )
+ ), null
+ );
+
+ }
+
+ @Test
+ public void testShortMessageType() throws Exception
+ {
+ //configure parser with desc file, and specify which file name to use
+ ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", "ProtoTestEvent");
+
+ }
+
+
+ @Test
+ public void testLongMessageType() throws Exception
+ {
+ //configure parser with desc file, and specify which file name to use
+ ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", "prototest.ProtoTestEvent");
+
+ }
+
+
+ @Test(expected = ParseException.class)
+ public void testBadProto() throws Exception
+ {
+ //configure parser with desc file
+ ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", "BadName");
+
+ }
+
+ @Test(expected = ParseException.class)
+ public void testMalformedDescriptorUrl() throws Exception
+ {
+ //configure parser with non existent desc file
+ ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "file:/nonexist.desc", "BadName");
+ }
+
+ @Test
+ public void testSingleDescriptorNoMessageType() throws Exception
+ {
+ // For the backward compatibility, protoMessageType allows null when the desc file has only one message type.
+ ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", null);
+ }
+
+ @Test
+ public void testParse() throws Exception
+ {
+
+ //configure parser with desc file
+ ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", "ProtoTestEvent");
+
+ //create binary of proto test event
+ DateTime dateTime = new DateTime(2012, 07, 12, 9, 30);
+ 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);
+
+ InputRow row = parser.parse(ByteBuffer.wrap(out.toByteArray()));
+ System.out.println(row);
+
+ 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");
+
+
+ assertEquals(47.11F, row.getFloatMetric("someFloatColumn"), 0.0);
+ assertEquals(815.0F, row.getFloatMetric("someIntColumn"), 0.0);
+ assertEquals(816.0F, row.getFloatMetric("someLongColumn"), 0.0);
+ }
+
+ private void assertDimensionEquals(InputRow row, String dimension, Object expected)
+ {
+ List values = row.getDimension(dimension);
+ assertEquals(1, values.size());
+ assertEquals(expected, values.get(0));
+ }
+}
diff --git a/extensions-core/protobuf-extensions/src/test/resources/ProtoTest.proto b/extensions-core/protobuf-extensions/src/test/resources/ProtoTest.proto
new file mode 100644
index 000000000000..1a68ab09fe72
--- /dev/null
+++ b/extensions-core/protobuf-extensions/src/test/resources/ProtoTest.proto
@@ -0,0 +1,30 @@
+syntax = "proto2";
+package prototest;
+option java_package = "io.druid.data.input.protobuf";
+option java_outer_classname = "ProtoTestEventWrapper";
+
+
+message ProtoTestEvent {
+ enum EventCategory {
+ CATEGORY_ZERO = 0;
+ CATEGORY_ONE = 1;
+ CATEGORY_TWO = 2;
+ }
+
+ message Foo {
+ required string bar = 1;
+ }
+
+ required EventCategory eventType = 1;
+ required uint64 id = 2;
+ required string timestamp = 3;
+ optional uint32 someOtherId = 4;
+ optional bool isValid = 5;
+ optional string description = 6;
+
+ optional float someFloatColumn = 7;
+ optional uint32 someIntColumn = 8;
+ optional uint64 someLongColumn = 9;
+ optional Foo foo = 10;
+ repeated Foo bar = 11;
+}
diff --git a/extensions-core/protobuf-extensions/src/test/resources/prototest.desc b/extensions-core/protobuf-extensions/src/test/resources/prototest.desc
new file mode 100644
index 000000000000..af7d67d102ca
Binary files /dev/null and b/extensions-core/protobuf-extensions/src/test/resources/prototest.desc differ
diff --git a/pom.xml b/pom.xml
index 68e864fff10c..68a6fb245ad7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -106,6 +106,7 @@
extensions-core/kafka-indexing-service
extensions-core/mysql-metadata-storage
extensions-core/postgresql-metadata-storage
+ extensions-core/protobuf-extensions
extensions-core/lookups-cached-global
extensions-core/lookups-cached-single
extensions-core/s3-extensions
diff --git a/processing/pom.xml b/processing/pom.xml
index 335a7a17a3e8..9d95e413dc05 100644
--- a/processing/pom.xml
+++ b/processing/pom.xml
@@ -65,10 +65,6 @@
org.skife.config
config-magic
-
- com.google.protobuf
- protobuf-java
-
commons-io
commons-io
diff --git a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java
deleted file mode 100644
index 557ca0941259..000000000000
--- a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to Metamarkets Group Inc. (Metamarkets) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. Metamarkets 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 io.druid.data.input;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Throwables;
-import com.google.common.collect.Maps;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.DescriptorProtos;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.DynamicMessage;
-import com.google.protobuf.InvalidProtocolBufferException;
-import io.druid.data.input.impl.MapInputRowParser;
-import io.druid.data.input.impl.ParseSpec;
-import io.druid.java.util.common.logger.Logger;
-
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import java.util.Map;
-
-@JsonTypeName("protobuf")
-public class ProtoBufInputRowParser implements ByteBufferInputRowParser
-{
- private static final Logger log = new Logger(ProtoBufInputRowParser.class);
-
- private final ParseSpec parseSpec;
- private final MapInputRowParser mapParser;
- private final String descriptorFileInClasspath;
-
- @JsonCreator
- public ProtoBufInputRowParser(
- @JsonProperty("parseSpec") ParseSpec parseSpec,
- @JsonProperty("descriptor") String descriptorFileInClasspath
- )
- {
- this.parseSpec = parseSpec;
- this.descriptorFileInClasspath = descriptorFileInClasspath;
- this.mapParser = new MapInputRowParser(this.parseSpec);
- }
-
- @Override
- public ParseSpec getParseSpec()
- {
- return parseSpec;
- }
-
- @Override
- public ProtoBufInputRowParser withParseSpec(ParseSpec parseSpec)
- {
- return new ProtoBufInputRowParser(parseSpec, descriptorFileInClasspath);
- }
-
- @Override
- public InputRow parse(ByteBuffer input)
- {
- // We should really create a ProtoBufBasedInputRow that does not need an intermediate map but accesses
- // the DynamicMessage directly...
- Map theMap = buildStringKeyMap(input);
-
- return mapParser.parse(theMap);
- }
-
- private Map buildStringKeyMap(ByteBuffer input)
- {
- final Descriptors.Descriptor descriptor = getDescriptor(descriptorFileInClasspath);
- final Map theMap = Maps.newHashMap();
-
- try {
- DynamicMessage message = DynamicMessage.parseFrom(descriptor, ByteString.copyFrom(input));
- Map allFields = message.getAllFields();
-
- for (Map.Entry entry : allFields.entrySet()) {
- String name = entry.getKey().getName();
- if (theMap.containsKey(name)) {
- continue;
- // Perhaps throw an exception here?
- // throw new RuntimeException("dupicate key " + name + " in " + message);
- }
- Object value = entry.getValue();
- if (value instanceof Descriptors.EnumValueDescriptor) {
- Descriptors.EnumValueDescriptor desc = (Descriptors.EnumValueDescriptor) value;
- value = desc.getName();
- }
-
- theMap.put(name, value);
- }
-
- }
- catch (InvalidProtocolBufferException e) {
- log.warn(e, "Problem with protobuf something");
- }
- return theMap;
- }
-
- private Descriptors.Descriptor getDescriptor(String descriptorFileInClassPath)
- {
- try {
- InputStream fin = this.getClass().getClassLoader().getResourceAsStream(descriptorFileInClassPath);
- DescriptorProtos.FileDescriptorSet set = DescriptorProtos.FileDescriptorSet.parseFrom(fin);
- Descriptors.FileDescriptor file = Descriptors.FileDescriptor.buildFrom(
- set.getFile(0), new Descriptors.FileDescriptor[]
- {}
- );
- return file.getMessageTypes().get(0);
- }
- catch (Exception e) {
- throw Throwables.propagate(e);
- }
- }
-}
diff --git a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java
deleted file mode 100644
index 23da291170fd..000000000000
--- a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to Metamarkets Group Inc. (Metamarkets) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. Metamarkets 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 io.druid.data.input;
-
-import io.druid.data.input.impl.DimensionsSpec;
-import io.druid.data.input.impl.TimeAndDimsParseSpec;
-import io.druid.data.input.impl.TimestampSpec;
-import org.joda.time.DateTime;
-import org.junit.Test;
-
-import java.io.ByteArrayOutputStream;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
-public class ProtoBufInputRowParserTest
-{
-
- public static final String[] DIMENSIONS = new String[]{"eventType", "id", "someOtherId", "isValid"};
-
- /*
- * eventType = 1;
- *
- * required uint64 id = 2;
- * required string timestamp = 3;
- * optional uint32 someOtherId = 4;
- * optional bool isValid = 5;
- * optional string description = 6;
- *
- * optional float someFloatColumn = 7;
- * optional uint32 someIntColumn = 8;
- * optional uint64 someLongColumn = 9;
- */
-
- @Test
- public void testParse() throws Exception
- {
-
- //configure parser with desc file
- ProtoBufInputRowParser parser = new ProtoBufInputRowParser(
- new TimeAndDimsParseSpec(
- new TimestampSpec("timestamp", "iso", null),
- new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList(DIMENSIONS)), Arrays.asList(), null)
- ),
- "prototest.desc"
- );
-
-
- //create binary of proto test event
- DateTime dateTime = new DateTime(2012, 07, 12, 9, 30);
- 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();
-
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- event.writeTo(out);
-
- InputRow row = parser.parse(ByteBuffer.wrap(out.toByteArray()));
- System.out.println(row);
-
- assertEquals(Arrays.asList(DIMENSIONS), row.getDimensions());
- 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());
-
-
- assertEquals(47.11F, row.getFloatMetric("someFloatColumn"), 0.0);
- assertEquals(815.0F, row.getFloatMetric("someIntColumn"), 0.0);
- assertEquals(816.0F, row.getFloatMetric("someLongColumn"), 0.0);
-
- }
-
- private void assertDimensionEquals(InputRow row, String dimension, Object expected)
- {
- List values = row.getDimension(dimension);
- assertEquals(1, values.size());
- assertEquals(expected, values.get(0));
- }
-}
diff --git a/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java b/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java
deleted file mode 100644
index ddf2a1d3079a..000000000000
--- a/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java
+++ /dev/null
@@ -1,1128 +0,0 @@
-/*
- * Licensed to Metamarkets Group Inc. (Metamarkets) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. Metamarkets 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.
- */
-
-// Generated by the protocol buffer compiler. DO NOT EDIT!
-// source: ProtoTest.proto
-
-package io.druid.data.input;
-
-import com.google.protobuf.AbstractMessage;
-import com.google.protobuf.GeneratedMessage;
-import com.google.protobuf.UnknownFieldSet;
-
-public final class ProtoTestEventWrapper {
- private ProtoTestEventWrapper() {}
- public static void registerAllExtensions(
- com.google.protobuf.ExtensionRegistry registry) {
- }
- public interface ProtoTestEventOrBuilder
- extends com.google.protobuf.MessageOrBuilder {
-
- // required .prototest.ProtoTestEvent.EventCategory eventType = 1;
- boolean hasEventType();
- ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType();
-
- // required uint64 id = 2;
- boolean hasId();
- long getId();
-
- // required string timestamp = 3;
- boolean hasTimestamp();
- String getTimestamp();
-
- // optional uint32 someOtherId = 4;
- boolean hasSomeOtherId();
- int getSomeOtherId();
-
- // optional bool isValid = 5;
- boolean hasIsValid();
- boolean getIsValid();
-
- // optional string description = 6;
- boolean hasDescription();
- String getDescription();
-
- // optional float someFloatColumn = 7;
- boolean hasSomeFloatColumn();
- float getSomeFloatColumn();
-
- // optional uint32 someIntColumn = 8;
- boolean hasSomeIntColumn();
- int getSomeIntColumn();
-
- // optional uint64 someLongColumn = 9;
- boolean hasSomeLongColumn();
- long getSomeLongColumn();
- }
- public static final class ProtoTestEvent extends
- GeneratedMessage
- implements ProtoTestEventOrBuilder {
- // Use ProtoTestEvent.newBuilder() to construct.
- private ProtoTestEvent(Builder builder) {
- super(builder);
- }
- private ProtoTestEvent(boolean noInit) {}
-
- private static final ProtoTestEvent defaultInstance;
- public static ProtoTestEvent getDefaultInstance() {
- return defaultInstance;
- }
-
- @Override
- public ProtoTestEvent getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- @Override
- public UnknownFieldSet getUnknownFields()
- {
- return UnknownFieldSet.getDefaultInstance();
- }
-
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor;
- }
-
- @Override
- protected GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable;
- }
-
- public enum EventCategory
- implements com.google.protobuf.ProtocolMessageEnum {
- CATEGORY_ZERO(0, 0),
- CATEGORY_ONE(1, 1),
- CATEGORY_TWO(2, 2),
- ;
-
- public static final int CATEGORY_ZERO_VALUE = 0;
- public static final int CATEGORY_ONE_VALUE = 1;
- public static final int CATEGORY_TWO_VALUE = 2;
-
-
- @Override
- public final int getNumber() { return value; }
-
- public static EventCategory valueOf(int value) {
- switch (value) {
- 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() {
- return internalValueMap;
- }
- private static com.google.protobuf.Internal.EnumLiteMap
- internalValueMap =
- new com.google.protobuf.Internal.EnumLiteMap() {
- @Override
- public EventCategory findValueByNumber(int number) {
- return EventCategory.valueOf(number);
- }
- };
-
- @Override
- public final com.google.protobuf.Descriptors.EnumValueDescriptor
- getValueDescriptor() {
- return getDescriptor().getValues().get(index);
- }
- @Override
- public final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptorForType() {
- return getDescriptor();
- }
- public static final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptor() {
- return ProtoTestEventWrapper.ProtoTestEvent.getDescriptor().getEnumTypes().get(0);
- }
-
- private static final EventCategory[] VALUES = {
- CATEGORY_ZERO, CATEGORY_ONE, CATEGORY_TWO,
- };
-
- public static EventCategory valueOf(
- com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
- if (desc.getType() != getDescriptor()) {
- throw new java.lang.IllegalArgumentException(
- "EnumValueDescriptor is not for this type.");
- }
- return VALUES[desc.getIndex()];
- }
-
- private final int index;
- private final int value;
-
- private EventCategory(int index, int value) {
- this.index = index;
- this.value = value;
- }
-
- // @@protoc_insertion_point(enum_scope:prototest.ProtoTestEvent.EventCategory)
- }
-
- private int bitField0_;
- // required .prototest.ProtoTestEvent.EventCategory eventType = 1;
- public static final int EVENTTYPE_FIELD_NUMBER = 1;
- private ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_;
- @Override
- public boolean hasEventType() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- @Override
- public ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() {
- return eventType_;
- }
-
- // required uint64 id = 2;
- public static final int ID_FIELD_NUMBER = 2;
- private long id_;
- @Override
- public boolean hasId() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- @Override
- public long getId() {
- return id_;
- }
-
- // required string timestamp = 3;
- public static final int TIMESTAMP_FIELD_NUMBER = 3;
- private java.lang.Object timestamp_;
- @Override
- public boolean hasTimestamp() {
- return ((bitField0_ & 0x00000004) == 0x00000004);
- }
- @Override
- public String getTimestamp() {
- java.lang.Object ref = timestamp_;
- if (ref instanceof String) {
- return (String) ref;
- } else {
- com.google.protobuf.ByteString bs =
- (com.google.protobuf.ByteString) ref;
- String s = bs.toStringUtf8();
- if (com.google.protobuf.Internal.isValidUtf8(bs)) {
- timestamp_ = s;
- }
- return s;
- }
- }
- private com.google.protobuf.ByteString getTimestampBytes() {
- java.lang.Object ref = timestamp_;
- if (ref instanceof String) {
- com.google.protobuf.ByteString b =
- com.google.protobuf.ByteString.copyFromUtf8((String) ref);
- timestamp_ = b;
- return b;
- } else {
- return (com.google.protobuf.ByteString) ref;
- }
- }
-
- // optional uint32 someOtherId = 4;
- public static final int SOMEOTHERID_FIELD_NUMBER = 4;
- private int someOtherId_;
- @Override
- public boolean hasSomeOtherId() {
- return ((bitField0_ & 0x00000008) == 0x00000008);
- }
- @Override
- public int getSomeOtherId() {
- return someOtherId_;
- }
-
- // optional bool isValid = 5;
- public static final int ISVALID_FIELD_NUMBER = 5;
- private boolean isValid_;
- @Override
- public boolean hasIsValid() {
- return ((bitField0_ & 0x00000010) == 0x00000010);
- }
- @Override
- public boolean getIsValid() {
- return isValid_;
- }
-
- // optional string description = 6;
- public static final int DESCRIPTION_FIELD_NUMBER = 6;
- private java.lang.Object description_;
- @Override
- public boolean hasDescription() {
- return ((bitField0_ & 0x00000020) == 0x00000020);
- }
- @Override
- public String getDescription() {
- java.lang.Object ref = description_;
- if (ref instanceof String) {
- return (String) ref;
- } else {
- com.google.protobuf.ByteString bs =
- (com.google.protobuf.ByteString) ref;
- String s = bs.toStringUtf8();
- if (com.google.protobuf.Internal.isValidUtf8(bs)) {
- description_ = s;
- }
- return s;
- }
- }
- private com.google.protobuf.ByteString getDescriptionBytes() {
- java.lang.Object ref = description_;
- if (ref instanceof String) {
- com.google.protobuf.ByteString b =
- com.google.protobuf.ByteString.copyFromUtf8((String) ref);
- description_ = b;
- return b;
- } else {
- return (com.google.protobuf.ByteString) ref;
- }
- }
-
- // optional float someFloatColumn = 7;
- public static final int SOMEFLOATCOLUMN_FIELD_NUMBER = 7;
- private float someFloatColumn_;
- @Override
- public boolean hasSomeFloatColumn() {
- return ((bitField0_ & 0x00000040) == 0x00000040);
- }
- @Override
- public float getSomeFloatColumn() {
- return someFloatColumn_;
- }
-
- // optional uint32 someIntColumn = 8;
- public static final int SOMEINTCOLUMN_FIELD_NUMBER = 8;
- private int someIntColumn_;
- @Override
- public boolean hasSomeIntColumn() {
- return ((bitField0_ & 0x00000080) == 0x00000080);
- }
- @Override
- public int getSomeIntColumn() {
- return someIntColumn_;
- }
-
- // optional uint64 someLongColumn = 9;
- public static final int SOMELONGCOLUMN_FIELD_NUMBER = 9;
- private long someLongColumn_;
- @Override
- public boolean hasSomeLongColumn() {
- return ((bitField0_ & 0x00000100) == 0x00000100);
- }
- @Override
- public long getSomeLongColumn() {
- return someLongColumn_;
- }
-
- private void initFields() {
- eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO;
- id_ = 0L;
- timestamp_ = "";
- someOtherId_ = 0;
- isValid_ = false;
- description_ = "";
- someFloatColumn_ = 0F;
- someIntColumn_ = 0;
- someLongColumn_ = 0L;
- }
- private byte memoizedIsInitialized = -1;
- @Override
- public final boolean isInitialized() {
- byte isInitialized = memoizedIsInitialized;
- if (isInitialized != -1) {
- return isInitialized == 1;
- }
-
- if (!hasEventType()) {
- memoizedIsInitialized = 0;
- return false;
- }
- if (!hasId()) {
- memoizedIsInitialized = 0;
- return false;
- }
- if (!hasTimestamp()) {
- memoizedIsInitialized = 0;
- return false;
- }
- memoizedIsInitialized = 1;
- return true;
- }
-
- @Override
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- output.writeEnum(1, eventType_.getNumber());
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- output.writeUInt64(2, id_);
- }
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
- output.writeBytes(3, getTimestampBytes());
- }
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
- output.writeUInt32(4, someOtherId_);
- }
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- output.writeBool(5, isValid_);
- }
- if (((bitField0_ & 0x00000020) == 0x00000020)) {
- output.writeBytes(6, getDescriptionBytes());
- }
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
- output.writeFloat(7, someFloatColumn_);
- }
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
- output.writeUInt32(8, someIntColumn_);
- }
- if (((bitField0_ & 0x00000100) == 0x00000100)) {
- output.writeUInt64(9, someLongColumn_);
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- @Override
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) {
- return size;
- }
-
- size = 0;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- size += com.google.protobuf.CodedOutputStream
- .computeEnumSize(1, eventType_.getNumber());
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- size += com.google.protobuf.CodedOutputStream
- .computeUInt64Size(2, id_);
- }
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
- size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(3, getTimestampBytes());
- }
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
- size += com.google.protobuf.CodedOutputStream
- .computeUInt32Size(4, someOtherId_);
- }
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(5, isValid_);
- }
- if (((bitField0_ & 0x00000020) == 0x00000020)) {
- size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(6, getDescriptionBytes());
- }
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
- size += com.google.protobuf.CodedOutputStream
- .computeFloatSize(7, someFloatColumn_);
- }
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
- size += com.google.protobuf.CodedOutputStream
- .computeUInt32Size(8, someIntColumn_);
- }
- if (((bitField0_ & 0x00000100) == 0x00000100)) {
- size += com.google.protobuf.CodedOutputStream
- .computeUInt64Size(9, someLongColumn_);
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- private static final long serialVersionUID = 0L;
- @Override
- protected java.lang.Object writeReplace()
- throws java.io.ObjectStreamException {
- return super.writeReplace();
- }
-
- public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
- public static ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- @Override
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(ProtoTestEventWrapper.ProtoTestEvent prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- @Override
- public Builder toBuilder() { return newBuilder(this); }
-
- @SuppressWarnings("ParameterPackage")
- @Override
- protected Builder newBuilderForType(
- GeneratedMessage.BuilderParent parent) {
- Builder builder = new Builder(parent);
- return builder;
- }
- public static final class Builder extends
- GeneratedMessage.Builder
- implements ProtoTestEventWrapper.ProtoTestEventOrBuilder {
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor;
- }
-
- @Override
- protected GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable;
- }
-
- // Construct using io.druid.data.input.ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
- private Builder() {
- maybeForceBuilderInitialization();
- }
-
- private Builder(BuilderParent parent) {
- super(parent);
- maybeForceBuilderInitialization();
- }
- private void maybeForceBuilderInitialization() {
- if (GeneratedMessage.alwaysUseFieldBuilders) {
- }
- }
- private static Builder create() {
- return new Builder();
- }
-
- @Override
- public Builder clear() {
- super.clear();
- eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO;
- bitField0_ = (bitField0_ & ~0x00000001);
- id_ = 0L;
- bitField0_ = (bitField0_ & ~0x00000002);
- timestamp_ = "";
- bitField0_ = (bitField0_ & ~0x00000004);
- someOtherId_ = 0;
- bitField0_ = (bitField0_ & ~0x00000008);
- isValid_ = false;
- bitField0_ = (bitField0_ & ~0x00000010);
- description_ = "";
- bitField0_ = (bitField0_ & ~0x00000020);
- someFloatColumn_ = 0F;
- bitField0_ = (bitField0_ & ~0x00000040);
- someIntColumn_ = 0;
- bitField0_ = (bitField0_ & ~0x00000080);
- someLongColumn_ = 0L;
- bitField0_ = (bitField0_ & ~0x00000100);
- return this;
- }
-
- @Override
- public Builder clone() {
- return create().mergeFrom(buildPartial());
- }
-
- @Override
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return ProtoTestEventWrapper.ProtoTestEvent.getDescriptor();
- }
-
- @Override
- public ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() {
- return ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance();
- }
-
- @Override
- public ProtoTestEventWrapper.ProtoTestEvent build() {
- ProtoTestEventWrapper.ProtoTestEvent result = buildPartial();
- if (!result.isInitialized()) {
- throw AbstractMessage.Builder.newUninitializedMessageException(result);
- }
- return result;
- }
-
- private ProtoTestEventWrapper.ProtoTestEvent buildParsed()
- throws com.google.protobuf.InvalidProtocolBufferException {
- ProtoTestEventWrapper.ProtoTestEvent result = buildPartial();
- if (!result.isInitialized()) {
- throw AbstractMessage.Builder.newUninitializedMessageException(
- result
- ).asInvalidProtocolBufferException();
- }
- return result;
- }
-
- @Override
- public ProtoTestEventWrapper.ProtoTestEvent buildPartial() {
- ProtoTestEventWrapper.ProtoTestEvent result = new ProtoTestEventWrapper.ProtoTestEvent(this);
- int from_bitField0_ = bitField0_;
- int to_bitField0_ = 0;
- if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
- to_bitField0_ |= 0x00000001;
- }
- result.eventType_ = eventType_;
- if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
- to_bitField0_ |= 0x00000002;
- }
- result.id_ = id_;
- if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
- to_bitField0_ |= 0x00000004;
- }
- result.timestamp_ = timestamp_;
- if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
- to_bitField0_ |= 0x00000008;
- }
- result.someOtherId_ = someOtherId_;
- if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
- to_bitField0_ |= 0x00000010;
- }
- result.isValid_ = isValid_;
- if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
- to_bitField0_ |= 0x00000020;
- }
- result.description_ = description_;
- if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
- to_bitField0_ |= 0x00000040;
- }
- result.someFloatColumn_ = someFloatColumn_;
- if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
- to_bitField0_ |= 0x00000080;
- }
- result.someIntColumn_ = someIntColumn_;
- if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
- to_bitField0_ |= 0x00000100;
- }
- result.someLongColumn_ = someLongColumn_;
- result.bitField0_ = to_bitField0_;
- onBuilt();
- return result;
- }
-
- @Override
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof ProtoTestEventWrapper.ProtoTestEvent) {
- return mergeFrom((ProtoTestEventWrapper.ProtoTestEvent)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(ProtoTestEventWrapper.ProtoTestEvent other) {
- if (other == ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance()) {
- return this;
- }
- if (other.hasEventType()) {
- setEventType(other.getEventType());
- }
- if (other.hasId()) {
- setId(other.getId());
- }
- if (other.hasTimestamp()) {
- setTimestamp(other.getTimestamp());
- }
- if (other.hasSomeOtherId()) {
- setSomeOtherId(other.getSomeOtherId());
- }
- if (other.hasIsValid()) {
- setIsValid(other.getIsValid());
- }
- if (other.hasDescription()) {
- setDescription(other.getDescription());
- }
- if (other.hasSomeFloatColumn()) {
- setSomeFloatColumn(other.getSomeFloatColumn());
- }
- if (other.hasSomeIntColumn()) {
- setSomeIntColumn(other.getSomeIntColumn());
- }
- if (other.hasSomeLongColumn()) {
- setSomeLongColumn(other.getSomeLongColumn());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- @Override
- public final boolean isInitialized() {
- if (!hasEventType()) {
-
- return false;
- }
- if (!hasId()) {
-
- return false;
- }
- if (!hasTimestamp()) {
-
- return false;
- }
- return true;
- }
-
- @Override
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder(
- this.getUnknownFields());
- while (true) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- this.setUnknownFields(unknownFields.build());
- onChanged();
- return this;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- this.setUnknownFields(unknownFields.build());
- onChanged();
- return this;
- }
- break;
- }
- case 8: {
- int rawValue = input.readEnum();
- ProtoTestEventWrapper.ProtoTestEvent.EventCategory value = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.valueOf(rawValue);
- if (value == null) {
- unknownFields.mergeVarintField(1, rawValue);
- } else {
- bitField0_ |= 0x00000001;
- eventType_ = value;
- }
- break;
- }
- case 16: {
- bitField0_ |= 0x00000002;
- id_ = input.readUInt64();
- break;
- }
- case 26: {
- bitField0_ |= 0x00000004;
- timestamp_ = input.readBytes();
- break;
- }
- case 32: {
- bitField0_ |= 0x00000008;
- someOtherId_ = input.readUInt32();
- break;
- }
- case 40: {
- bitField0_ |= 0x00000010;
- isValid_ = input.readBool();
- break;
- }
- case 50: {
- bitField0_ |= 0x00000020;
- description_ = input.readBytes();
- break;
- }
- case 61: {
- bitField0_ |= 0x00000040;
- someFloatColumn_ = input.readFloat();
- break;
- }
- case 64: {
- bitField0_ |= 0x00000080;
- someIntColumn_ = input.readUInt32();
- break;
- }
- case 72: {
- bitField0_ |= 0x00000100;
- someLongColumn_ = input.readUInt64();
- break;
- }
- }
- }
- }
-
- private int bitField0_;
-
- // required .prototest.ProtoTestEvent.EventCategory eventType = 1;
- private ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO;
- @Override
- public boolean hasEventType() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- @Override
- public ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() {
- return eventType_;
- }
- public Builder setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory value) {
- if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000001;
- eventType_ = value;
- onChanged();
- return this;
- }
- public Builder clearEventType() {
- bitField0_ = (bitField0_ & ~0x00000001);
- eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO;
- onChanged();
- return this;
- }
-
- // required uint64 id = 2;
- private long id_ ;
- @Override
- public boolean hasId() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- @Override
- public long getId() {
- return id_;
- }
- public Builder setId(long value) {
- bitField0_ |= 0x00000002;
- id_ = value;
- onChanged();
- return this;
- }
- public Builder clearId() {
- bitField0_ = (bitField0_ & ~0x00000002);
- id_ = 0L;
- onChanged();
- return this;
- }
-
- // required string timestamp = 3;
- private java.lang.Object timestamp_ = "";
- @Override
- public boolean hasTimestamp() {
- return ((bitField0_ & 0x00000004) == 0x00000004);
- }
- @Override
- public String getTimestamp() {
- java.lang.Object ref = timestamp_;
- if (!(ref instanceof String)) {
- String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
- timestamp_ = s;
- return s;
- } else {
- return (String) ref;
- }
- }
- public Builder setTimestamp(String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000004;
- timestamp_ = value;
- onChanged();
- return this;
- }
- public Builder clearTimestamp() {
- bitField0_ = (bitField0_ & ~0x00000004);
- timestamp_ = getDefaultInstance().getTimestamp();
- onChanged();
- return this;
- }
- void setTimestamp(com.google.protobuf.ByteString value) {
- bitField0_ |= 0x00000004;
- timestamp_ = value;
- onChanged();
- }
-
- // optional uint32 someOtherId = 4;
- private int someOtherId_ ;
- @Override
- public boolean hasSomeOtherId() {
- return ((bitField0_ & 0x00000008) == 0x00000008);
- }
- @Override
- public int getSomeOtherId() {
- return someOtherId_;
- }
- public Builder setSomeOtherId(int value) {
- bitField0_ |= 0x00000008;
- someOtherId_ = value;
- onChanged();
- return this;
- }
- public Builder clearSomeOtherId() {
- bitField0_ = (bitField0_ & ~0x00000008);
- someOtherId_ = 0;
- onChanged();
- return this;
- }
-
- // optional bool isValid = 5;
- private boolean isValid_ ;
- @Override
- public boolean hasIsValid() {
- return ((bitField0_ & 0x00000010) == 0x00000010);
- }
- @Override
- public boolean getIsValid() {
- return isValid_;
- }
- public Builder setIsValid(boolean value) {
- bitField0_ |= 0x00000010;
- isValid_ = value;
- onChanged();
- return this;
- }
- public Builder clearIsValid() {
- bitField0_ = (bitField0_ & ~0x00000010);
- isValid_ = false;
- onChanged();
- return this;
- }
-
- // optional string description = 6;
- private java.lang.Object description_ = "";
- @Override
- public boolean hasDescription() {
- return ((bitField0_ & 0x00000020) == 0x00000020);
- }
- @Override
- public String getDescription() {
- java.lang.Object ref = description_;
- if (!(ref instanceof String)) {
- String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
- description_ = s;
- return s;
- } else {
- return (String) ref;
- }
- }
- public Builder setDescription(String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000020;
- description_ = value;
- onChanged();
- return this;
- }
- public Builder clearDescription() {
- bitField0_ = (bitField0_ & ~0x00000020);
- description_ = getDefaultInstance().getDescription();
- onChanged();
- return this;
- }
- void setDescription(com.google.protobuf.ByteString value) {
- bitField0_ |= 0x00000020;
- description_ = value;
- onChanged();
- }
-
- // optional float someFloatColumn = 7;
- private float someFloatColumn_ ;
- @Override
- public boolean hasSomeFloatColumn() {
- return ((bitField0_ & 0x00000040) == 0x00000040);
- }
- @Override
- public float getSomeFloatColumn() {
- return someFloatColumn_;
- }
- public Builder setSomeFloatColumn(float value) {
- bitField0_ |= 0x00000040;
- someFloatColumn_ = value;
- onChanged();
- return this;
- }
- public Builder clearSomeFloatColumn() {
- bitField0_ = (bitField0_ & ~0x00000040);
- someFloatColumn_ = 0F;
- onChanged();
- return this;
- }
-
- // optional uint32 someIntColumn = 8;
- private int someIntColumn_ ;
- @Override
- public boolean hasSomeIntColumn() {
- return ((bitField0_ & 0x00000080) == 0x00000080);
- }
- @Override
- public int getSomeIntColumn() {
- return someIntColumn_;
- }
- public Builder setSomeIntColumn(int value) {
- bitField0_ |= 0x00000080;
- someIntColumn_ = value;
- onChanged();
- return this;
- }
- public Builder clearSomeIntColumn() {
- bitField0_ = (bitField0_ & ~0x00000080);
- someIntColumn_ = 0;
- onChanged();
- return this;
- }
-
- // optional uint64 someLongColumn = 9;
- private long someLongColumn_ ;
- @Override
- public boolean hasSomeLongColumn() {
- return ((bitField0_ & 0x00000100) == 0x00000100);
- }
- @Override
- public long getSomeLongColumn() {
- return someLongColumn_;
- }
- public Builder setSomeLongColumn(long value) {
- bitField0_ |= 0x00000100;
- someLongColumn_ = value;
- onChanged();
- return this;
- }
- public Builder clearSomeLongColumn() {
- bitField0_ = (bitField0_ & ~0x00000100);
- someLongColumn_ = 0L;
- onChanged();
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:prototest.ProtoTestEvent)
- }
-
- static {
- defaultInstance = new ProtoTestEvent(true);
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:prototest.ProtoTestEvent)
- }
-
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_prototest_ProtoTestEvent_descriptor;
- private static
- GeneratedMessage.FieldAccessorTable
- internal_static_prototest_ProtoTestEvent_fieldAccessorTable;
-
- public static com.google.protobuf.Descriptors.FileDescriptor
- getDescriptor() {
- return descriptor;
- }
- private static com.google.protobuf.Descriptors.FileDescriptor
- descriptor;
- static {
- java.lang.String[] descriptorData = {
- "\n\017ProtoTest.proto\022\tprototest\"\266\002\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\"F\n\rEventCa" +
- "tegory\022\021\n\rCATEGORY_ZERO\020\000\022\020\n\014CATEGORY_ON" +
- "E\020\001\022\020\n\014CATEGORY_TWO\020\002B6\n\035com.metamx.drui" +
- "d.indexer.dataB\025ProtoTestEventWrapper"
- };
- com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
- new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
- @Override
- public com.google.protobuf.ExtensionRegistry assignDescriptors(
- com.google.protobuf.Descriptors.FileDescriptor root) {
- descriptor = root;
- internal_static_prototest_ProtoTestEvent_descriptor =
- getDescriptor().getMessageTypes().get(0);
- internal_static_prototest_ProtoTestEvent_fieldAccessorTable = new
- GeneratedMessage.FieldAccessorTable(
- internal_static_prototest_ProtoTestEvent_descriptor,
- new java.lang.String[] { "EventType", "Id", "Timestamp", "SomeOtherId", "IsValid", "Description", "SomeFloatColumn", "SomeIntColumn", "SomeLongColumn", },
- ProtoTestEventWrapper.ProtoTestEvent.class,
- ProtoTestEventWrapper.ProtoTestEvent.Builder.class);
- return null;
- }
- };
- com.google.protobuf.Descriptors.FileDescriptor
- .internalBuildGeneratedFileFrom(descriptorData,
- new com.google.protobuf.Descriptors.FileDescriptor[] {
- }, assigner);
- }
-
-
- // @@protoc_insertion_point(outer_class_scope)
-}
diff --git a/processing/src/test/resources/ProtoTest.proto b/processing/src/test/resources/ProtoTest.proto
deleted file mode 100644
index 956db5259ac9..000000000000
--- a/processing/src/test/resources/ProtoTest.proto
+++ /dev/null
@@ -1,31 +0,0 @@
-
-package prototest;
-option java_package = "io.druid.indexer.data";
-option java_outer_classname = "ProtoTestEventWrapper";
-
-
-
-message ProtoTestEvent {
-
-
-enum EventCategory {
- CATEGORY_ZERO = 0;
- CATEGORY_ONE = 1;
- CATEGORY_TWO = 2;
-}
-
- required EventCategory eventType = 1;
-
- required uint64 id = 2;
- required string timestamp = 3;
- optional uint32 someOtherId = 4;
- optional bool isValid = 5;
- optional string description = 6;
-
- optional float someFloatColumn = 7;
- optional uint32 someIntColumn = 8;
- optional uint64 someLongColumn = 9;
-
-
-
-}
diff --git a/processing/src/test/resources/prototest.desc b/processing/src/test/resources/prototest.desc
deleted file mode 100644
index 649ce5bcb8fe..000000000000
Binary files a/processing/src/test/resources/prototest.desc and /dev/null differ
diff --git a/server/src/main/java/io/druid/guice/ParsersModule.java b/server/src/main/java/io/druid/guice/ParsersModule.java
index 934083bcefbd..54c59ed0c351 100644
--- a/server/src/main/java/io/druid/guice/ParsersModule.java
+++ b/server/src/main/java/io/druid/guice/ParsersModule.java
@@ -23,7 +23,6 @@
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.inject.Binder;
-import io.druid.data.input.ProtoBufInputRowParser;
import io.druid.initialization.DruidModule;
import io.druid.segment.realtime.firehose.IrcInputRowParser;
@@ -45,7 +44,6 @@ public List extends Module> getJacksonModules()
return Arrays.asList(
new SimpleModule("ParsersModule")
.registerSubtypes(
- new NamedType(ProtoBufInputRowParser.class, "protobuf"),
new NamedType(IrcInputRowParser.class, "irc")
)
);