Loading...
Loading...
Loading...
Loading...
Loading...
Loading...
Loading...
Loading...
Loading...
Loading...
Loading...
Loading...
Cluster is a set of nodes comprising of servers, brokers, controllers and minions.
Pinot uses Apache Helix for cluster management. Helix is a cluster management framework that manages replicated, partitioned resources in a distributed system. Helix uses Zookeeper to store cluster state and metadata.
Helix divides nodes into logical components based on their responsibilities:
The nodes that host distributed, partitioned resources
Pinot Servers are modeled as Participants. For more details about server nodes, see Server.
The nodes that observe the current state of each Participant and use that information to access the resources. Spectators are notified of state changes in the cluster (state of a participant, or that of a partition in a participant).
Pinot Brokers are modeled as Spectators. For more details about broker nodes, see Broker.
The node that observes and controls the Participant nodes. It is responsible for coordinating all transitions in the cluster and ensuring that state constraints are satisfied while maintaining cluster stability.
Pinot Controllers are modeled as Controllers. For more details about controller nodes, see Controller.
Another way to visualize the cluster is a logical view, where:
Typically, there is only one cluster per environment/data center. There is no need to create multiple Pinot clusters since Pinot supports the concept of tenants. At LinkedIn, the largest Pinot cluster consists of 1000+ nodes.
To set up a cluster, see one of the following guides:
The Pinot Controller is responsible for the following:
Maintaining global metadata (e.g. configs and schemas) of the system with the help of Zookeeper which is used as the persistent metadata store.
Hosting the Helix Controller and managing other Pinot components (brokers, servers, minions)
Maintaining the mapping of which servers are responsible for which segments. This mapping is used by the servers to download the portion of the segments that they are responsible for. This mapping is also used by the broker to decide which servers to route the queries to.
Serving admin endpoints for viewing, creating, updating, and deleting configs, which are used to manage and operate the cluster.
Serving endpoints for segment uploads, which are used in offline data pushes. They are responsible for initializing real-time consumption and coordination of persisting real-time segments into the segment store periodically.
Undertaking other management activities such as managing retention of segments, validations.
For redundancy, there can be multiple instances of Pinot controllers. Pinot expects that all controllers are configured with the same back-end storage system so that they have a common view of the segments (e.g. NFS). Pinot can use other storage systems such as HDFS or ADLS.
The Controller runs several periodic tasks in the background, to perform activities such as management and validation. Each periodic task has its own configs to define the run frequency and default frequency. Each task runs at its own schedule or can also be triggered manually if needed. The task runs on the lead controller for each table.
Here's a list of all the periodic tasks
This task rebuilds the BrokerResource if the instance set has changed.
controller.broker.resource.validation.frequencyPeriod
1h
controller.broker.resource.validation.initialDelayInSeconds
between 2m-5m
TBD
This task manages the segment ValidationMetrics (missingSegmentCount, offlineSegmentDelayHours, lastPushTimeDelayHours, TotalDocumentCount, NonConsumingPartitionCount, SegmentCount), to ensure that all offline segments are contiguous (no missing segments) and that the offline push delay isn't too high.
controller.offline.segment.interval.checker.frequencyPeriod
24h
controller.statuschecker.waitForPushTimePeriod
10m
controller.offlineSegmentIntervalChecker.initialDelayInSeconds
between 2m-5m
TBD
This task validates the ideal state and segment zk metadata of realtime tables,
fixing any partitions which have stopped consuming
starting consumption from new partitions
uploading segments to deep store if segment download url is missing
This task ensures that the consumption of the realtime tables gets fixed and keeps going when met with erroneous conditions.
This task does not fix consumption stalled due to
CONSUMING segment being deleted
Kafka OOR exceptions
controller.realtime.segment.validation.frequencyPeriod
1h
controller.realtime.segment.validation.initialDelayInSeconds
between 2m-5m
This task manages retention of segments for all tables. During the run, it looks at the retentionTimeUnit
and retentionTimeValue
inside the segmentsConfig
of every table, and deletes segments which are older than the retention. The deleted segments are moved to a DeletedSegments folder colocated with the dataDir on segment store, and permanently deleted from that folder in a configurable number of days.
controller.retention.frequencyPeriod
6h
controller.retentionManager.initialDelayInSeconds
between 2m-5m
controller.deleted.segments.retentionInDays
7d
This task is applicable only if you have tierConfig or tagOverrideConfig. It runs rebalance in the background to
relocate COMPLETED segments to tag overrides
relocate ONLINE segments to tiers if tier configs are set
At most one replica is allowed to be unavailable during rebalance.
controller.segment.relocator.frequencyPeriod
1h
controller.segmentRelocator.initialDelayInSeconds
between 2m-5m
This task manages segment status metrics such as realtimeTableCount, offlineTableCount, disableTableCount, numberOfReplicas, percentOfReplicas, percentOfSegments, idealStateZnodeSize, idealStateZnodeByteSize, segmentCount, segmentsInErrorState, tableCompressedSize.
controller.statuschecker.frequencyPeriod
5m
controller.statusChecker.initialDelayInSeconds
between 2m-5m
TBD
Use the GET /periodictask/names
API to fetch the names of all the Periodic Tasks running on your Pinot cluster.
To manually run a named Periodic Task use the GET /periodictask/run
API
The Log Request Id
(api-09630c07
) can be used to search through pinot-controller log file to see log entries related to execution of the Periodic task that was manually run.
If tableName
(and its type OFFLINE
or REALTIME
) is not provided, the task will run against all tables.
Make sure you've setup Zookeeper. If you're using docker, make sure to pull the pinot docker image. To start a controller
A tenant is a logical component defined as a group of server/broker nodes with the same Helix tag.
In order to support multi-tenancy, Pinot has first-class support for tenants. Every table is associated with a server tenant and a broker tenant. This controls the nodes that will be used by this table as servers and brokers. This allows all tables belonging to a particular use case to be grouped under a single tenant name.
The concept of tenants is very important when the multiple use cases are using Pinot and there is a need to provide quotas or some sort of isolation across tenants. For example, consider we have two tables Table A
and Table B
in the same Pinot cluster.
We can configure Table A
with server tenant Tenant A
and Table B
with server tenant Tenant B
. We can tag some of the server nodes for Tenant A
and some for Tenant B
. This will ensure that segments of Table A
only reside on servers tagged with Tenant A
, and segment of Table B
only reside on servers tagged with Tenant B
. The same isolation can be achieved at the broker level, by configuring broker tenants to the tables.
No need to create separate clusters for every table or use case!
This tenant is defined in the tenants section of the table config.
This section contains 2 main fields broker
and server
, which decide the tenants used for the broker and server components of this table.
In the above example:
The table will be served by brokers that have been tagged as brokerTenantName_BROKER
in Helix.
If this were an offline table, the offline segments for the table will be hosted in Pinot servers tagged in Helix as serverTenantName_OFFLINE
If this were a real-time table, the real-time segments (both consuming as well as completed ones) will be hosted in pinot servers tagged in Helix as serverTenantName_REALTIME
.
Here's a sample broker tenant config. This will create a broker tenant sampleBrokerTenant
by tagging 3 untagged broker nodes as sampleBrokerTenant_BROKER
.
To create this tenant use the following command. The creation will fail if number of untagged broker nodes is less than numberOfInstances
.
Follow instructions in Getting Pinot to get Pinot locally, and then
Check out the table config in the Rest API to make sure it was successfully uploaded.
Here's a sample server tenant config. This will create a server tenant sampleServerTenant
by tagging 1 untagged server node as sampleServerTenant_OFFLINE
and 1 untagged server node as sampleServerTenant_REALTIME
.
To create this tenant use the following command. The creation will fail if number of untagged server nodes is less than offlineInstances
+ realtimeInstances
.
Follow instructions in Getting Pinot to get Pinot locally, and then
Check out the table config in the Rest API to make sure it was successfully uploaded.
Brokers handle Pinot queries. They accept queries from clients and forward them to the right servers. They collect results back from the servers and consolidate them into a single response, to send back to the client.
Pinot Brokers are modeled as Helix Spectators. They need to know the location of each segment of a table (and each replica of the segments) and route requests to the appropriate server that hosts the segments of the table being queried.
The broker ensures that all the rows of the table are queried exactly once so as to return correct, consistent results for a query. The brokers may optimize to prune some of the segments as long as accuracy is not sacrificed.
Helix provides the framework by which spectators can learn the location in which each partition of a resource (i.e. participant) resides. The brokers use this mechanism to learn the servers that host specific segments of a table.
In the case of hybrid tables, the brokers ensure that the overlap between real-time and offline segment data is queried exactly once, by performing offline and real-time federation.
Let's take this example, we have real-time data for 5 days - March 23 to March 27, and offline data has been pushed until Mar 25, which is 2 days behind real-time. The brokers maintain this time boundary.
Suppose, we get a query to this table : select sum(metric) from table
. The broker will split the query into 2 queries based on this time boundary - one for offline and one for realtime. This query becomes - select sum(metric) from table_REALTIME where date >= Mar 25
and select sum(metric) from table_OFFLINE where date < Mar 25
The broker merges results from both these queries before returning the result to the client.
Learn about the different components and logical abstractions
Learn about the deep store that stores a compressed copy of segment files in Pinot.
The deep store (or deep storage) is the permanent store for files.
It is used for backup and restore operations. New nodes in a cluster will pull down a copy of segment files from the deep store. If the local segment files on a server gets damaged in some way (or accidentally deleted), a new copy will be pulled down from the deep store on server restart.
The deep store stores a compressed version of the segment files and it typically won't include any indexes. These compressed files can be stored on a local file system or on a variety of other file systems. For more details on supported file systems, see .
Note: Deep Store by itself is not sufficient for restore operations. Pinot stores metadata such as table config, schema, segment metadata in Zookeeper. For restore operations, both Deep Store as well as Zookeeper metadata are required.
There are several different ways that segments are persisted in the deep store.
For offline tables, the batch ingestion job writes the segment directly into the deep store, as shown in the diagram below:
The ingestion job then sends a notification about the new segment to the controller, which in turn notifies the appropriate server to pull down that segment.
For real-time tables, by default, a segment is first built-in memory by the server. It is then uploaded to the lead controller (as part of the Segment Completion Protocol sequence), which writes the segment into the deep store, as shown in the diagram below:
When using this configuration the server will directly write a completed segment to the deep store, as shown in the diagram below:
For hands-on examples of how to configure the deep store, see the following tutorials:
Explore the data on our Pinot cluster
Once you have set up the Cluster, you can start exploring the data and the APIs using the Pinot Data Explorer.
Navigate to in your browser to open the controller UI.
The first screen that you'll see when you open the Pinot Data Explorer is the Cluster Manager. The Cluster Manager provides a UI to operate and manage your cluster.
If you want to view the contents of a server, click on its instance name. You'll then see the following:
To view the baseballStats table, click on its name, which will show the following screen:
From this screen, we can edit or delete the table, edit or adjust its schema, as well as several other operations.
For example, if we want to add yearID to the list of inverted indexes, click on Edit Table, add the extra column, and click Save:
You can also execute a sample query select * from baseballStats limit 10
by typing it in the text box and clicking the Run Query button.
Cmd + Enter
can also be used to run the query when focused on the console.
You can also try out the following queries:
Servers host the data segments and serve queries off the data they host. There are two types of servers:
Offline Offline servers are responsible for downloading segments from the segment store, to host and serve queries off. When a new segment is uploaded to the controller, the controller decides the servers (as many as replication) that will host the new segment and notifies them to download the segment from the segment store. On receiving this notification, the servers download the segment file and load the segment onto the server, to server queries off them.
Real-time Real-time servers directly ingest from a real-time stream (such as Kafka, EventHubs). Periodically, they make segments of the in-memory ingested data, based on certain thresholds. This segment is then persisted onto the segment store.
Pinot Servers are modeled as Helix Participants, hosting Pinot tables (referred to as resources in Helix terminology). Segments of a table are modeled as Helix partitions (of a resource). Thus, a Pinot server hosts one or more helix partitions of one or more helix resources (i.e. one or more segments of one or more tables).
USAGE
A table is a logical abstraction that represents a collection of related data. It is composed of columns and rows (known as documents in Pinot). The columns, data types, and other metadata related to the table are defined using a .
Pinot breaks a table into multiple and stores these segments in a deep-store such as HDFS as well as Pinot servers.
In the Pinot cluster, a table is modeled as a and each segment of a table is modeled as a .
Pinot supports the following types of table:
The user querying the database does not need to know the type of the table. They only need to specify the table name in the query.
e.g. regardless of whether we have an offline table myTable_OFFLINE
, a real-time table myTable_REALTIME
, or a hybrid table containing both of these, the query will be:
is used to define the table properties, such as name, type, indexing, routing, retention etc. It is written in JSON format and is stored in Zookeeper, along with the table schema.
You can use the following properties to make your tables faster or leaner:
Segment
Indexing
Tenants
For real-time tables, segments are built in a specific interval inside Pinot. You can tune the following for the real-time segments:
The Pinot real-time consumer ingests the data, creates the segment, and then flushes the in-memory segment to disk. Pinot allows you to configure when to flush the segment in the following ways:
Number of consumed rows - After consuming X no. of rows from the stream, Pinot will persist the segment to disk
Number of desired rows per segment - Pinot learns and then estimates the number of rows that need to be consumed so that the persisted segment is approximately the size. The learning phase starts by setting the number of rows to 100,000 (this value can be changed) and adjusts it to reach the desired segment size. The segment size may go significantly over the desired size during the learning phase. Pinot corrects the estimation as it goes along, so it is not guaranteed that the resulting completed segments are of the exact size as configured. You should set this value to optimize the performance of queries.
Max time duration to wait - Pinot consumers wait for the configured time duration after which segments are persisted to the disk.
Replicas A segment can have multiple replicas to provide higher availability. You can configure the number of replicas for a table segment using
However, in certain scenarios, the segment build can get very memory intensive. It might be desirable to enforce the non-committer servers to just download the segment from the controller, instead of building it again. You can do this by setting completionMode: "DOWNLOAD"
in the table configuration
Download Scheme
A Pinot server may fail to download segments from the deep store such as HDFS after its completion. However, you can configure servers to download these segments from peer servers instead of the deep store. Currently, only HTTP and HTTPS download schemes are supported. More methods such as gRPC/Thrift can be added in the future.
You can create multiple indices on a table to increase the performance of the queries. The following types of indices are supported:
Dictionary-encoded forward index with bit compression
Raw value forward index
Sorted forward index with run-length encoding
Bitmap inverted index
Sorted inverted index
You can aggregate the real-time stream data as it is consumed to reduce segment sizes. We sum the metric column values of all rows that have the same values for all dimension and time columns and create a single row in the segment. This feature is only available on REALTIME
tables.
The only supported aggregation is SUM
. The columns on which pre-aggregation is to be done need to satisfy the following requirements:
All metrics should be listed in noDictionaryColumns
.
There should not be any multi-value dimensions.
All dimension columns are treated to have a dictionary, even if they appear as noDictionaryColumns
in the config.
The following table config snippet shows an example of enabling pre-aggregation during real-time ingestion.
You can also override if a table should move to a server with different tenant based on segment status.
A tagOverrideConfig
can be added under the tenants
section for realtime tables, to override tags for consuming and completed segments. For example:
A hybrid table is a table composed of 2 tables, one offline and one real-time that share the same name. In such a table, offline segments may be pushed periodically. The retention on the offline table can be set to a high value since segments are coming in on a periodic basis, whereas the retention on the real-time part can be small.
Once an offline segment is pushed to cover a recent time period, the brokers automatically switch to using the offline table for segments for that time period and use the real-time table only for data not available in the offline table.
A typical scenario is pushing a deduped cleaned up data into an offline table every day while consuming real-time data as and when it arrives. The data can be kept in offline tables for even a few years while the real-time data would be cleaned every few days.
Prerequisites
Sample Console Output
Start Kafka
Create a Kafka Topic
Create a Streaming table
Sample output
Start Kafka-Zookeeper
Start Kafka
Create stream table
Note that creating a hybrid table has to be done in 2 separate steps of creating an offline and real-time table individually.
A Minion is a standby component that leverages the to offload computationally intensive tasks from other components.
It can be attached to an existing Pinot cluster and then execute tasks as provided by the controller. Custom tasks can be plugged via annotations into the cluster. Some typical minion tasks are:
Segment creation
Segment purge
Segment merge
Make sure you've . If you're using docker, make sure to . To start a minion
PinotTaskGenerator interface defines the APIs for the controller to generate tasks for minions to execute.
Factory for PinotTaskExecutor
which defines the APIs for Minion to execute the tasks.
Factory for MinionEventObserver
which defines the APIs for task event callbacks on minion.
The PushTask can fetch files from an input folder e.g. from a S3 bucket and converts them into segments. The PushTask converts one file into one segment and keeps file name in segment metadata to avoid duplicate ingestion. Below is an example task config to put in TableConfig to enable this task. The task is scheduled every 10min to keep ingesting remaining files, with 10 parallel task at max and 1 file per task.
NOTE: You may want to simply omit "tableMaxNumTasks" due to this caveat: the task generates one segment per file, and derives segment name based on the time column of the file. If two files happen to have same time range and are ingested by tasks from different schedules, there might be segment name conflict. To overcome this issue for now, you can omit “tableMaxNumTasks” and by default it’s Integer.MAX_VALUE, meaning to schedule as many tasks as possible to ingest all input files in a single batch. Within one batch, a sequence number suffix is used to ensure no segment name conflict. Because the sequence number suffix is scoped within one batch, tasks from different batches might encounter segment name conflict issue said above.
To be added
Tasks are enabled on a per-table basis. To enable a certain task type (e.g. myTask
) on a table, update the table config to include the task type:
Under each enable task type, custom properties can be configured for the task type.
There are also two task configs to be set as part of cluster configs like below. One controls task's overall timeout (1hr by default) and one for how many tasks to run on a single minion worker (1 by default).
There are 2 ways to enable task scheduling:
Tasks can be scheduled periodically for all task types on all enabled tables. Enable auto task scheduling by configuring the schedule frequency in the controller config with the key controller.task.frequencyPeriod
. This takes period strings as values, e.g. 2h, 30m, 1d.
Tasks can also be scheduled based on cron expressions. The cron expression is set in the schedule
config for each task type separately. This config in the controller config, controller.task.scheduler.enabled
should be set to true
to enable cron scheduling.
Tasks can be manually scheduled using the following controller rest APIs:
To plug in a custom task, implement PinotTaskGenerator
, PinotTaskExecutorFactory
and MinionEventObserverFactory
(optional) for the task type (all of them should return the same string for getTaskType()
), and annotate them with the following annotations:
After annotating the classes, put them under the package of name org.apache.pinot.*.plugin.minion.tasks.*
, then they will be auto-registered by the controller and minion.
There is a controller job that runs every 5 minutes by default and emits metrics about Minion tasks scheduled in Pinot. The following metrics are emitted for each task type:
NumMinionTasksInProgress: Number of running tasks
NumMinionSubtasksRunning: Number of running sub-tasks
NumMinionSubtasksWaiting: Number of waiting sub-tasks (unassigned to a minion as yet)
NumMinionSubtasksError: Number of error sub-tasks (completed with an error/exception)
PercentMinionSubtasksInQueue: Percent of sub-tasks in waiting or running states
PercentMinionSubtasksInError: Percent of sub-tasks in error
For each task, the Minion will emit these metrics:
TASK_QUEUEING: Task queueing time (task_dequeue_time - task_inqueue_time), assuming the time drift between helix controller and pinot minion is minor, otherwise the value may be negative
TASK_EXECUTION: Task execution time, which is the time spent on executing the task
NUMBER_OF_TASKS: number of tasks in progress on that minion. Whenever a Minion starts a task, increase the Gauge by 1, whenever a Minion completes (either succeeded or failed) a task, decrease it by 1
Each table in Pinot is associated with a Schema. A schema defines what fields are present in the table along with the data types.
The schema is stored in the Zookeeper, along with the table configuration.
A schema also defines what category a column belongs to. Columns in a Pinot table can be categorized into three categories:
Pinot does not enforce strict rules on which of these categories columns belong to, rather the categories can be thought of as hints to Pinot to do internal optimizations.
For example, metrics may be stored without a dictionary and can have a different default null value.
The categories are also relevant when doing segment merge and rollups. Pinot uses the dimension and time fields to identify records against which to apply merge/rollups.
Metrics aggregation is another example where Pinot uses dimensions and time are used as the key, and automatically aggregates values for the metric columns.
Data types determine the operations that can be performed on a column. Pinot supports the following data types:
BOOLEAN
, TIMESTAMP
, JSON
are added after release 0.7.1
. In release 0.7.1
and older releases, BOOLEAN
is equivalent to STRING.
BIG_DECIMAL
is added after release 0.10.0
.
Since Pinot doesn't have a dedicated DATETIME
datatype support, you need to input time in either STRING, LONG, or INT format. However, Pinot needs to convert the date into an understandable format such as epoch timestamp to do operations.
To achieve this conversion, you will need to provide the format of the date along with the data type in the schema. The format is described using the following syntax: timeSize:timeUnit:timeFormat:pattern
.
time size - the size of the time unit. This size is multiplied to the value present in the time column to get an actual timestamp. e.g. if timesize is 5 and value in time column is 4996308 minutes. The value that will be converted to epoch timestamp will be 4996308 * 5 * 60 * 1000 = 1498892400000 milliseconds.
If your date is not in EPOCH
format, this value is not used and can be set to 1 or any other integer.\
timeFormat - can be either EPOCH
or SIMPLE_DATE_FORMAT
. If it is SIMPLE_DATE_FORMAT
, the pattern string is also specified. \
Here are some sample date-time formats you can use in the schema:
1:MILLISECONDS:EPOCH
- used when timestamp is in the epoch milliseconds and stored in LONG
format
1:HOURS:EPOCH
- used when timestamp is in the epoch hours and stored in LONG
or INT
format
1:DAYS:SIMPLE_DATE_FORMAT:yyyy-MM-dd
- when the date is in STRING
format and has the pattern year-month-date. e.g. 2020-08-21
1:HOURS:SIMPLE_DATE_FORMAT:EEE MMM dd HH:mm:ss ZZZ yyyy
- when date is in STRING
format. e.g. Mon Aug 24 12:36:50 America/Los_Angeles 2019
From Pinot release 0.11.0, We have simplified date time formats for the users. The formats now follow the pattern - timeFormat|pattern/timeUnit|
[timeZone/timeSize]
. The fields present in []
are completely optional. timeFormat can be one of EPOCH
, SIMPLE_DATE_FORMAT
or TIMESTAMP
.
TIMESTAMP
- This represents timestamp in milliseconds. It is equivalent to specifying EPOCH:MILLISECONDS:1
EPOCH
- This represents time in timeUnit
since 00:00:00 UTC on 1 January 1970.
You can also specify the timeSize parameter.This size is multiplied to the value present in the time column to get an actual timestamp. e.g. if timesize is 5 and value in time column is 4996308 minutes. The value that will be converted to epoch timestamp will be 4996308 * 5 * 60 * 1000 = 1498892400000 milliseconds.
Examples -
EPOCH|SECONDS
EPOCH|SECONDS|10
SIMPLE_DATE_FORMAT
SIMPLE_DATE_FORMAT|yyyy-MM-dd HH:mm:ss
SIMPLE_DATE_FORMAT|yyyy-MM-dd|IST
There are several built-in virtual columns inside the schema the can be used for debugging purposes:
These virtual columns can be used in queries in a similar way to regular columns.
Let's create a schema and put it in a JSON file. For this example, we have created a schema for flight data.
Then, we can upload the sample schema provided above using either a Bash command or REST API call.
Pinot has the concept of a , which is a logical abstraction to refer to a collection of related data. Pinot has a distributed architecture and scales horizontally. Pinot expects the size of a table to grow infinitely over time. In order to achieve this, the entire data needs to be distributed across multiple nodes.
Pinot achieves this by breaking the data into smaller chunks known as segments (similar to shards/partitions in relational databases). Segments can be seen as time-based partitions.
Thus, a segment is a horizontal shard representing a chunk of table data with some number of rows. The segment stores data for all columns of the table. Each segment packs the data in a columnar fashion, along with the dictionaries and indices for the columns. The segment is laid out in a columnar format so that it can be directly mapped into memory for serving queries.
Columns can be single or multi-valued and the following types are supported: STRING, BOOLEAN, INT, LONG, FLOAT, DOUBLE, TIMESTAMP or BYTES. Only single-valued BIG_DECIMAL data type is supported.
Columns may be declared to be metric or dimension (or specifically as a time dimension) in the schema. Columns can have default null values. For example, the default null value of a integer column can be 0. The default value for bytes columns must be hex-encoded before it's added to the schema.
Pinot uses dictionary encoding to store values as a dictionary ID. Columns may be configured to be “no-dictionary” column in which case raw values are stored. Dictionary IDs are encoded using minimum number of bits for efficient storage (e.g. a column with a cardinality of 3 will use only 2 bits for each dictionary ID).
A forward index is built for each column and compressed for efficient memory use. In addition, you can optionally configure inverted indices for any set of columns. Inverted indices take up more storage, but improve query performance. Specialized indexes like Star-Tree index are also supported. For more details, see .
Once the table is configured, we can load some data. Loading data involves generating pinot segments from raw data and pushing them to the pinot cluster. Data can be loaded in batch mode or streaming mode. For more details, see the page.
Below are instructions to generate and push segments to Pinot via standalone scripts. For a production setup, you should use frameworks such as Hadoop or Spark. For more details on setting up data ingestion jobs, see
To create and push the segment in one go, use
Sample Console Output
Alternately, you can separately create and then push, by changing the jobType to SegmentCreation
or SegmenTarPush
.
The Ingestion job spec supports templating with Groovy Syntax.
This is convenient if you want to generate one ingestion job template file and schedule it on a daily basis with extra parameters updated daily.
e.g. you could set inputDirURI
with parameters to indicate the date, so that the ingestion job only processes the data for a particular date. Below is an example that templates the date for input and output directories.
You can pass in arguments containing values for ${year}, ${month}, ${day}
when kicking off the ingestion job: -values $param=value1 $param2=value2
...
This ingestion job only generates segments for date 2014-01-03
Prerequisites
Below is an example of how to publish sample data to your stream. As soon as data is available to the realtime stream, it starts getting consumed by the realtime servers
Run below command to stream JSON data into Kafka topic: flights-realtime
Run below command to stream JSON data into Kafka topic: flights-realtime
Make sure you've . If you're using docker, make sure to . To start a broker
Having all segments go through the controller can become a system bottleneck under heavy load, in which case you can use the peer download policy, as described in .
Let us run some queries on the data in the Pinot cluster. Head over to to see the querying interface.
We can see our baseballStats
table listed on the left (you will see meetupRSVP
or airlineStats
if you used the streaming or the hybrid ). Click on the table name to display all the names along with the data types of the columns of the table.
Pinot supports a subset of standard SQL. For more information, see .
The contains all the APIs that you will need to operate and manage your cluster. It provides a set of APIs for Pinot cluster management including health check, instances management, schema and table management, data segments management.
Let's check out the tables in this cluster by going to , click Try it out, and then click Execute. We can see thebaseballStats
table listed here. We can also see the exact cURL call made to the controller API.
You can look at the configuration of this table by going to , click Try it out, type baseballStats
in the table name, and then click Execute.
Let's check out the schemas in the cluster by going to , click Try it out, and then click Execute. We can see a schema called baseballStats
in this list.
Take a look at the schema by going to , click Try it out, type baseballStats
in the schema name, and then click Execute.
Finally, let's check out the data segments in the cluster by going to , click Try it out, type in baseballStats
in the table name, and then click Execute. There's 1 segment for this table, called baseballStats_OFFLINE_0
.
To learn how to upload your own data and schema, see or .
Make sure you've . If you're using docker, make sure to . To start a server
A table is comprised of small chunks of data. These chunks are known as Segments. To learn more about how Pinot creates and manages segments see
For offline tables, Segments are built outside of pinot and uploaded using a distributed executor such as Spark or Hadoop. For more details, see .
Completion Mode By default, if the in-memory segment in the is equivalent to the committed segment, then the non-winner server builds and replaces the segment. If the available segment is not equivalent to the committed segment, the server simply downloads the committed segment from the controller.
For more details on why this is needed, see
For more details about peer segment download during real-time ingestion, please refer to this design doc on
For more details on each indexing mechanism and corresponding configurations, see .
You can also set up on columns to make queries faster. Further, you can also keep segments in off-heap instead of on-heap memory for faster queries.
Each table is associated with a tenant. A segment resides on the server, which has the same tenant as itself. For more details on how tenants work, see .
In the above example, the consuming segments will still be assigned to serverTenantName_REALTIME
hosts, but once they are completed, the segments will be moved to serverTeantnName_OFFLINE
. It is possible to specify the full name of any tag in this section (so, for example, you could decide that completed segments for this table should be in pinot servers tagged as allTables_COMPLETED
). To learn more about this config, see the section.
To understand how time boundary works in the case of a hybrid table, see .
Create a table config for your data, or see for all possible batch/streaming tables.
Check out the table config in the to make sure it was successfully uploaded.
Check out the table config in the to make sure it was successfully uploaded.
See for details.
See for details.
As shown below, the RealtimeToOfflineSegmentsTask will be scheduled at the first second of every minute (following the syntax ).
See where the TestTask
is plugged-in.
Pinot also supports columns that contain lists or arrays of items, but there isn't an explicit data type to represent these lists or arrays. Instead, you can indicate that a dimension column accepts multiple values. For more information, see in the Schema configuration reference.
time unit - one of enum values. e.g. HOURS
, MINUTES
etc. If your date is not in EPOCH
format, this value is not used and can be set to MILLISECONDS
or any other unit.\
pattern - This is optional and is only specified when the date is in SIMPLE_DATE_FORMAT
. The pattern should be specified using the java representation. e.g. 2020-08-21 can be represented as yyyy-MM-dd
.\
SIMPLE_DATE_FORMAT
- This represents time in the string format. The pattern should be specified using the java representation. If no pattern is specified, we use to parse the date times. Optionals are supported with ISO format so users can specify date time string in yyyy
or yyyy-MM
or yyyy-MM-dd
and so on
You can also specify optional timeZone
parameter which is the ID for a TimeZone, either an abbreviation such as PST
, a full name such as America/Los_Angeles
, or a custom ID such as GMT-8:00
.
Examples -
First, Make sure your and running.
For more details on constructing a schema file, see the .
Check out the schema in the to make sure it was successfully uploaded
To generate a segment, we need to first create a job spec YAML file. This file contains all the information regarding data format, input data location, and pinot cluster coordinates. Note that this assumes that the controller is RUNNING to fetch the table config and schema. If not, you will have to configure the spec to point at their location. For full configurations, see .
POST /tasks/schedule
Schedule tasks for all task types on all enabled tables
POST /tasks/schedule?taskType=myTask
Schedule tasks for the given task type on all enabled tables
POST /tasks/schedule?tableName=myTable_OFFLINE
Schedule tasks for all task types on the given table
POST /tasks/schedule?taskType=myTask&tableName=myTable_OFFLINE
Schedule tasks for the given task type on the given table
PinotTaskGenerator
@TaskGenerator
PinotTaskExecutorFactory
@TaskExecutorFactory
MinionEventObserverFactory
@EventObserverFactory
Dimension
Dimension columns are typically used in slice and dice operations for answering business queries. Some operations for which dimension columns are used:
GROUP BY
- group by one or more dimension columns along with aggregations on one or more metric columns
Filter clauses such as WHERE
Metric
These columns represent the quantitative data of the table. Such columns are used for aggregation. In data warehouse terminology, these can also be referred to as fact or measure columns.
Some operation for which metric columns are used:
Aggregation - SUM
, MIN
, MAX
, COUNT
, AVG
etc
Filter clause such as WHERE
DateTime
This column represents time columns in the data. There can be multiple time columns in a table, but only one of them can be treated as primary. The primary time column is the one that is present in the segment config.
The primary time column is used by Pinot to maintain the time boundary between offline and real-time data in a hybrid table and for retention management. A primary time column is mandatory if the table's push type is APPEND
and optional if the push type is REFRESH
.
Common operations that can be done on time column:
GROUP BY
Filter clauses such as WHERE
INT
0
LONG
0
FLOAT
0.0
DOUBLE
0.0
BIG_DECIMAL
Not supported
0.0
BOOLEAN
0 (false)
N/A
TIMESTAMP
0 (1970-01-01 00:00:00 UTC)
N/A
STRING
"null"
N/A
JSON
"null"
N/A
BYTES
byte array of length 0
byte array of length 0
$hostName
Dimension
STRING
Name of the server hosting the data
$segmentName
Dimension
STRING
Name of the segment containing the record
$docId
Dimension
INT
Document id of the record within the segment
Offline
Offline tables ingest pre-built pinot-segments from external data stores. This is generally used for batch ingestion.
Realtime
Realtime tables ingest data from streams (such as Kafka) and build segments from the consumed data.
Hybrid
A hybrid Pinot table has both realtime as well as offline tables under the hood. By default, all tables in Pinot are Hybrid in nature.