mirror of https://github.com/apache/druid.git
parent
257bc5c62f
commit
59c8430d29
|
@ -56,7 +56,7 @@ Here is a JSON example of the 'metrics' data schema used in the example.
|
|||
|
||||
### Proto file
|
||||
|
||||
The corresponding proto file for our 'metrics' dataset looks like this. You can use Protobuf parser with a proto file or [Confluent Schema Registry](https://docs.confluent.io/platform/current/schema-registry/index.html).
|
||||
The corresponding proto file for our 'metrics' dataset looks like this. You can use Protobuf `inputFormat` with a proto file or [Confluent Schema Registry](https://docs.confluent.io/platform/current/schema-registry/index.html).
|
||||
```
|
||||
syntax = "proto3";
|
||||
message Metrics {
|
||||
|
@ -112,22 +112,14 @@ Important supervisor properties
|
|||
- `protoBytesDecoder.descriptor` for the descriptor file URL
|
||||
- `protoBytesDecoder.protoMessageType` from the proto definition
|
||||
- `protoBytesDecoder.type` set to `file`, indicate use descriptor file to decode Protobuf file
|
||||
- `parser` should have `type` set to `protobuf`, but note that the `format` of the `parseSpec` must be `json`
|
||||
- `inputFormat` should have `type` set to `protobuf`
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "kafka",
|
||||
"spec": {
|
||||
"dataSchema": {
|
||||
"dataSource": "metrics-protobuf",
|
||||
"parser": {
|
||||
"type": "protobuf",
|
||||
"protoBytesDecoder": {
|
||||
"type": "file",
|
||||
"descriptor": "file:///tmp/metrics.desc",
|
||||
"protoMessageType": "Metrics"
|
||||
},
|
||||
"parseSpec": {
|
||||
"format": "json",
|
||||
"timestampSpec": {
|
||||
"column": "timestamp",
|
||||
"format": "auto"
|
||||
|
@ -145,8 +137,6 @@ Important supervisor properties
|
|||
"timestamp",
|
||||
"value"
|
||||
]
|
||||
}
|
||||
}
|
||||
},
|
||||
"metricsSpec": [
|
||||
{
|
||||
|
@ -184,9 +174,23 @@ Important supervisor properties
|
|||
"consumerProperties": {
|
||||
"bootstrap.servers": "localhost:9092"
|
||||
},
|
||||
"inputFormat": {
|
||||
"type": "protobuf",
|
||||
"protoBytesDecoder": {
|
||||
"type": "file",
|
||||
"descriptor": "file:///tmp/metrics.desc",
|
||||
"protoMessageType": "Metrics"
|
||||
},
|
||||
"flattenSpec": {
|
||||
"useFieldDiscovery": true
|
||||
},
|
||||
"binaryAsString": false
|
||||
},
|
||||
"taskCount": 1,
|
||||
"replicas": 1,
|
||||
"taskDuration": "PT1H"
|
||||
"taskDuration": "PT1H",
|
||||
"type": "kafka"
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
@ -253,7 +257,7 @@ If necessary, from your Kafka installation directory run the following command t
|
|||
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
|
||||
|
||||
```
|
||||
./bin/generate-example-metrics | ./quickstart/protobuf/pb_publisher.py
|
||||
./bin/generate-example-metrics | python /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
|
||||
|
|
Loading…
Reference in New Issue