Pluggable Streams
Prior to commit ba9f2d, Pinot was only able to support consuming from Kafka stream.
Pinot now enables its users to write plug-ins to consume from pub-sub streams other than Kafka. (Please refer to Issue #2583)
Some of the streams for which plug-ins can be added are:
You may encounter some limitations either in Pinot or in the stream system while developing plug-ins. Please feel free to get in touch with us when you start writing a stream plug-in, and we can help you out. We are open to receiving PRs in order to improve these abstractions if they do not work for a certain stream implementation.
Refer to Consuming and Indexing rows in Realtime for details on how Pinot consumes streaming data.
Requirements to support Stream Level (High Level) consumers
The stream should provide the following guarantees:
Exactly once delivery (unless restarting from a checkpoint) for each consumer of the stream.
(Optionally) support mechanism to split events (in some arbitrary fashion) so that each event in the stream is delivered exactly to one host out of set of hosts.
Provide ways to save a checkpoint for the data consumed so far. If the stream is partitioned, then this checkpoint is a vector of checkpoints for events consumed from individual partitions.
The checkpoints should be recorded only when Pinot makes a call to do so.
The consumer should be able to start consumption from one of:
latest avaialble data
earliest available data
last saved checkpoint
Requirements to support Partition Level (Low Level) consumers
While consuming rows at a partition level, the stream should support the following properties:
Stream should provide a mechanism to get the current number of partitions.
Each event in a partition should have a unique offset that is not more than 64 bits long.
Refer to a partition as a number not exceeding 32 bits long.
Stream should provide the following mechanisms to get an offset for a given partition of the stream:
get the offset of the oldest event available (assuming events are aged out periodically) in the partition.
get the offset of the most recent event published in the partition
(optionally) get the offset of an event that was published at a specified time
Stream should provide a mechanism to consume a set of events from a partition starting from a specified offset.
Pinot assumes that the offsets of incoming events are monotonically increasing; i.e., if Pinot consumes an event at offset
o1
, then the offseto2
of the following event should be such thato2 > o1
.
In addition, we have an operational requirement that the number of partitions should not be reduced over time.
Stream plug-in implementation
In order to add a new type of stream (say,Foo) implement the following classes:
FooConsumerFactory extends StreamConsumerFactory
FooPartitionLevelConsumer implements PartitionLevelConsumer
FooStreamLevelConsumer implements StreamLevelConsumer
FooMetadataProvider implements StreamMetadataProvider
FooMessageDecoder implements StreamMessageDecoder
Depending on stream level or partition level, your implementation needs to include StreamLevelConsumer or PartitionLevelConsumer.
The properties for the stream implementation are to be set in the table configuration, inside streamConfigs section.
Use the streamType
property to define the stream type. For example, for the implementation of stream foo
, set the property "streamType" : "foo"
.
The rest of the configuration properties for your stream should be set with the prefix "stream.foo"
. Be sure to use the same suffix for: (see examples below):
topic
consumer type
stream consumer factory
offset
decoder class name
decoder properties
connnection timeout
fetch timeout
All values should be strings. For example:
You can have additional properties that are specific to your stream. For example:
In addition to these properties, you can define thresholds for the consuming segments:
rows threshold
time threshold
The properties for the thresholds are as follows:
An example of this implementation can be found in the KafkaConsumerFactory, which is an implementation for the kafka stream.
Kafka 2.x Plugin
Pinot provides stream plugin support for Kafka 2.x version. Although the version used in this implementation is kafka 2.0.0, it’s possible to compile it with higher kafka lib version, e.g. 2.1.1.
How to build and release Pinot package with Kafka 2.x connector
How to use Kafka 2.x connector
Use Kafka Stream(High) Level Consumer
Below is a sample streamConfigs
used to create a realtime table with Kafka Stream(High) level consumer.
Kafka 2.x HLC consumer uses org.apache.pinot.core.realtime.impl.kafka2.KafkaConsumerFactory
in config stream.kafka.consumer.factory.class.name
.
Use Kafka Partition(Low) Level Consumer
Below is a sample table config used to create a realtime table with Kafka Partition(Low) level consumer:
Please note:
Config
replicasPerPartition
undersegmentsConfig
is required to specify table replication.Config
stream.kafka.consumer.type
should be specified asLowLevel
to use partition level consumer. (The use ofsimple
instead ofLowLevel
is deprecated)Configs
stream.kafka.zk.broker.url
andstream.kafka.broker.list
are required undertableIndexConfig.streamConfigs
to provide kafka related information.
Upgrade from Kafka 0.9 connector to Kafka 2.x connector
Update table config for both high level and low level consumer: Update config:
stream.kafka.consumer.factory.class.name
fromorg.apache.pinot.core.realtime.impl.kafka.KafkaConsumerFactory
toorg.apache.pinot.core.realtime.impl.kafka2.KafkaConsumerFactory
.If using Stream(High) level consumer: Please also add config
stream.kafka.hlc.bootstrap.server
intotableIndexConfig.streamConfigs
. This config should be the URI of Kafka broker lists, e.g.localhost:9092
.
How to use this plugin with higher Kafka version?
This connector is also suitable for Kafka lib version higher than 2.0.0
. In pinot-connector-kafka-2.0/pom.xml
change the kafka.lib.version
from 2.0.0
to 2.1.1
will make this Connector working with Kafka 2.1.1
.
Last updated