Protobuf

This Apache Druid extension enables Druid to ingest and understand the Protobuf data format. Make sure to include druid-protobuf-extensions as an extension.

The druid-protobuf-extensions provides the Protobuf Parser for stream ingestion. See corresponding docs for details.

Example: Load Protobuf messages from Kafka

This example demonstrates how to load Protobuf messages from Kafka. Please read the Load from Kafka tutorial first, and see Kafka Indexing Service documentation for more details.

The files used in this example are found at ./examples/quickstart/protobuf in your Druid directory.

For this example:

  • Kafka broker host is localhost:9092
  • Kafka topic is metrics_pb
  • Datasource name is metrics-protobuf

Here is a JSON example of the ‘metrics’ data schema used in the example.

  1. {
  2. "unit": "milliseconds",
  3. "http_method": "GET",
  4. "value": 44,
  5. "timestamp": "2017-04-06T02:36:22Z",
  6. "http_code": "200",
  7. "page": "/",
  8. "metricType": "request/latency",
  9. "server": "www1.example.com"
  10. }

Proto file

The corresponding proto file for our ‘metrics’ dataset looks like this.

  1. syntax = "proto3";
  2. message Metrics {
  3. string unit = 1;
  4. string http_method = 2;
  5. int32 value = 3;
  6. string timestamp = 4;
  7. string http_code = 5;
  8. string page = 6;
  9. string metricType = 7;
  10. string server = 8;
  11. }

Descriptor file

Next, we use the protoc Protobuf compiler to generate the descriptor file and save it as metrics.desc. The descriptor file must be either in the classpath or reachable by URL. In this example the descriptor file was saved at /tmp/metrics.desc, however this file is also available in the example files. From your Druid install directory:

  1. protoc -o /tmp/metrics.desc ./quickstart/protobuf/metrics.proto

Create Kafka Supervisor

Below is the complete Supervisor spec JSON to be submitted to the Overlord. Make sure these keys are properly configured for successful ingestion.

Important supervisor properties

  • descriptor for the descriptor file URL
  • protoMessageType from the proto definition
  • parser should have type set to protobuf, but note that the format of the parseSpec must be json
  1. {
  2. "type": "kafka",
  3. "dataSchema": {
  4. "dataSource": "metrics-protobuf",
  5. "parser": {
  6. "type": "protobuf",
  7. "descriptor": "file:///tmp/metrics.desc",
  8. "protoMessageType": "Metrics",
  9. "parseSpec": {
  10. "format": "json",
  11. "timestampSpec": {
  12. "column": "timestamp",
  13. "format": "auto"
  14. },
  15. "dimensionsSpec": {
  16. "dimensions": [
  17. "unit",
  18. "http_method",
  19. "http_code",
  20. "page",
  21. "metricType",
  22. "server"
  23. ],
  24. "dimensionExclusions": [
  25. "timestamp",
  26. "value"
  27. ]
  28. }
  29. }
  30. },
  31. "metricsSpec": [
  32. {
  33. "name": "count",
  34. "type": "count"
  35. },
  36. {
  37. "name": "value_sum",
  38. "fieldName": "value",
  39. "type": "doubleSum"
  40. },
  41. {
  42. "name": "value_min",
  43. "fieldName": "value",
  44. "type": "doubleMin"
  45. },
  46. {
  47. "name": "value_max",
  48. "fieldName": "value",
  49. "type": "doubleMax"
  50. }
  51. ],
  52. "granularitySpec": {
  53. "type": "uniform",
  54. "segmentGranularity": "HOUR",
  55. "queryGranularity": "NONE"
  56. }
  57. },
  58. "tuningConfig": {
  59. "type": "kafka",
  60. "maxRowsPerSegment": 5000000
  61. },
  62. "ioConfig": {
  63. "topic": "metrics_pb",
  64. "consumerProperties": {
  65. "bootstrap.servers": "localhost:9092"
  66. },
  67. "taskCount": 1,
  68. "replicas": 1,
  69. "taskDuration": "PT1H"
  70. }
  71. }

Adding Protobuf messages to Kafka

If necessary, from your Kafka installation directory run the following command to create the Kafka topic

  1. ./bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic metrics_pb

This example script requires protobuf and kafka-python modules. With the topic in place, messages can be inserted running the following command from your Druid installation directory

  1. ./bin/generate-example-metrics | ./quickstart/protobuf/pb_publisher.py

You can confirm that data has been inserted to your Kafka topic using the following command from your Kafka installation directory

  1. ./bin/kafka-console-consumer --zookeeper localhost --topic metrics_pb

which should print messages like this

  1. millisecondsGETR"2017-04-06T03:23:56Z*2002/list:request/latencyBwww1.example.com

If your supervisor created in the previous step is running, the indexing tasks should begin producing the messages and the data will soon be available for querying in Druid.

Generating the example files

The files provided in the example quickstart can be generated in the following manner starting with only metrics.proto.

metrics.desc

The descriptor file is generated using protoc Protobuf compiler. Given a .proto file, a .desc file can be generated like so.

  1. protoc -o metrics.desc metrics.proto

metrics_pb2.py

metrics_pb2.py is also generated with protoc

  1. protoc -o metrics.desc metrics.proto --python_out=.

pb_publisher.py

After metrics_pb2.py is generated, another script can be constructed to parse JSON data, convert it to Protobuf, and produce to a Kafka topic

  1. #!/usr/bin/env python
  2. import sys
  3. import json
  4. from kafka import KafkaProducer
  5. from metrics_pb2 import Metrics
  6. producer = KafkaProducer(bootstrap_servers='localhost:9092')
  7. topic = 'metrics_pb'
  8. for row in iter(sys.stdin):
  9. d = json.loads(row)
  10. metrics = Metrics()
  11. for k, v in d.items():
  12. setattr(metrics, k, v)
  13. pb = metrics.SerializeToString()
  14. producer.send(topic, pb)
  15. producer.flush()