This page has a collection of frequently asked questions about ingestion with answers from the community.
This is a list of questions frequently asked in our troubleshooting channel on Slack. To contribute additional questions and answers, make a pull request.
While Apache Pinot can work with segments of various sizes, for optimal use of Pinot, you want to get your segments sized in the 100MB to 500MB (un-tarred/uncompressed) range. Having too many (thousands or more) tiny segments for a single table creates overhead in terms of the metadata storage in Zookeeper as well as in the Pinot servers' heap. At the same time, having too few really large (GBs) segments reduces parallelism of query execution, as on the server side, the thread parallelism of query execution is at segment level.
Yes. Each table can be independently configured to consume from any given Kafka topic, regardless of whether there are other tables that are also consuming from the same Kafka topic.
Pinot automatically detects new partitions in Kafka topics. It checks for new partitions whenever RealtimeSegmentValidationManager
periodic job runs and starts consumers for new partitions.
You can configure the interval for this job using thecontroller.realtime.segment.validation.frequencyPeriod
property in the controller configuration.
Pinot supports multi-column partitioning for offline tables. Map multiple columns under tableIndexConfig.segmentPartitionConfig.columnPartitionMap
. Pinot assigns the input data to each partition according to the partition configuration individually for each column.
The following example partitions the segment based on two columns, memberID
and caseNumber
. Note that each partition column is handled separately, so in this case the segment is partitioned on memberID
(partition ID 1) and also partiitoned on caseNumber
(partition ID 2).
For multi-column partitioning to work, you must also set routing.segementPrunerTypes
as follows:
Set up partitioner in the Kafka producer: https://docs.confluent.io/current/clients/producer.html
The partitioning logic in the stream should match the partitioning config in Pinot. Kafka uses murmur2
, and the equivalent in Pinot is the Murmur
function.
Set the partitioning configuration as below using same column used in Kafka:
and also set:
To learn how partition works, see routing tuning.
For JSON, you can use a hex encoded string to ingest BYTES.
See the json_format(field) function which can store a top level json field as a STRING in Pinot.
Then you can use these json functions during query time, to extract fields from the json string.
NOTE This works well if some of your fields are nested json, but most of your fields are top level json keys. If all of your fields are within a nested JSON key, you will have to store the entire payload as 1 column, which is not ideal.
To use explicit code points, you must double-quote (not single-quote) the string, and escape the code point via "\uHHHH", where HHHH is the four digit hex code for the character. See https://yaml.org/spec/spec.html#escaping/in%20double-quoted%20scalars/ for more details.
By default, Pinot limits the length of a String column to 512 bytes. If you want to overwrite this value, you can set the maxLength attribute in the schema as follows:
Events are available to queries as soon as they are ingested. This is because events are instantly indexed in memory upon ingestion.
The ingestion of events into the real-time table is not transactional, so replicas of the open segment are not immediately consistent. Pinot trades consistency for availability upon network partitioning (CAP theorem) to provide ultra-low ingestion latencies at high throughput.
However, when the open segment is closed and its in-memory indexes are flushed to persistent storage, all its replicas are guaranteed to be consistent, with the commit protocol.
This typically happens if:
The consumer is lagging a lot.
The consumer was down (server down, cluster down), and the stream moved on, resulting in offset not found when consumer comes back up.
In case of Kafka, to recover, set property "auto.offset.reset":"earliest"
in the streamConfigs
section and reset the CONSUMING
segment. See Real-time table configs for more details about the configuration.
You can also also use the "Resume Consumption" endpoint with "resumeFrom" parameter set to "smallest" (or "largest" if you want). See Pause Stream Ingestion for more details.
Inverted indexes are set in the tableConfig
's tableIndexConfig
-> invertedIndexColumns
list. For more info on table configuration, see Table Config Reference. For an example showing how to configure an inverted index, see Inverted Index.
Applying inverted indexes to a table configuration will generate an inverted index for all new segments. To apply the inverted indexes to all existing segments, see How to apply an inverted index to existing segments?
Add the columns you wish to index to the tableIndexConfig
-> invertedIndexColumns
list. To update the table configuration use the Pinot Swagger API: http://localhost:9000/help#!/Table/updateTableConfig.
Invoke the reload API: http://localhost:9000/help#!/Segment/reloadAllSegments.
Once you've done that, you can check whether the index has been applied by querying the segment metadata API at http://localhost:9000/help#/Segment/getServerMetadata. Don't forget to include the names of the column on which you have applied the index.
The output from this API should look something like the following:
Not all indexes can be retrospectively applied to existing segments.
If you want to add or change the sorted index column or adjust the dictionary encoding of the default forward index you will need to manually re-load any existing segments.
Star-tree indexes are configured in the table config under the tableIndexConfig
-> starTreeIndexConfigs
(list) and enableDefaultStarTree
(boolean). See here for more about how to configure star-tree indexes: https://docs.pinot.apache.org/basics/indexing/star-tree-index#index-generation
The new segments will have star-tree indexes generated after applying the star-tree index configurations to the table configuration. Currently, Pinot does not support adding star-tree indexes to the existing segments.
Pinot does not require ordering of event time stamps. Out of order events are still consumed and indexed into the "currently consuming" segment. In a pathological case, if you have a 2 day old event come in "now", it will still be stored in the segment that is open for consumption "now". There is no strict time-based partitioning for segments, but star-indexes and hybrid tables will handle this as appropriate.
See the Components > Broker for more details about how hybrid tables handle this. Specifically, the time-boundary is computed as max(OfflineTIme) - 1 unit of granularity
. Pinot does store the min-max time for each segment and uses it for pruning segments, so segments with multiple time intervals may not be perfectly pruned.
When generating star-indexes, the time column will be part of the star-tree so the tree can still be efficiently queried for segments with multiple time intervals.
max(OfflineTime)
to determine the time-boundary, and instead using an offset?This lets you have an old event up come in without building complex offline pipelines that perfectly partition your events by event timestamps. With this offset, even if your offline data pipeline produces segments with a maximum timestamp, Pinot will not use the offline dataset for that last chunk of segments. The expectation is if you process offline the next time-range of data, your data pipeline will include any late events.
It might seem odd that segments are not strictly time-partitioned, unlike similar systems such as Apache Druid. This allows real-time ingestion to consume out-of-order events. Even though segments are not strictly time-partitioned, Pinot will still index, prune, and query segments intelligently by time intervals for the performance of hybrid tables and time-filtered data.
When generating offline segments, the segments generated such that segments only contain one time interval and are well partitioned by the time column.
This page lists pages with frequently asked questions with answers from the community.
This is a list of questions frequently asked in our troubleshooting channel on Slack. To contribute additional questions and answers, make a pull request.
This page has a collection of frequently asked questions about queries with answers from the community.
This is a list of questions frequently asked in our troubleshooting channel on Slack. To contribute additional questions and answers, make a pull request.
This implies that the Pinot Broker assigned to the table specified in the query was not found. A common root cause for this is a typo in the table name in the query. Another uncommon reason could be if there wasn't actually a broker with required broker tenant tag for the table.
See this page explaining the Pinot response format: https://docs.pinot.apache.org/users/api/querying-pinot-using-standard-sql/response-format.
"timestamp" is a reserved keyword in SQL. Escape timestamp with double quotes.
Other commonly encountered reserved keywords are date, time, table.
For filtering on STRING columns, use single quotes:
The fields in the ORDER BY
clause must be one of the group by clauses or aggregations, BEFORE applying the alias. Therefore, this will not work:
But, this will work:
No. Pagination only works for SELECTION queries.
You can add this at the end of your query: option(timeoutMs=X)
. Tthe following example uses a timeout of 20 seconds for the query:
You can also use SET "timeoutMs" = 20000; SELECT COUNT(*) from myTable
.
For changing the timeout on the entire cluster, set this property pinot.broker.timeoutMs
in either broker configs or cluster configs (using the POST /cluster/configs API from Swagger).
Add these two configs for Pinot server and broker to start tracking of running queries. The query tracks are added and cleaned as query starts and ends, so should not consume much resource.
Then use the Rest APIs on Pinot controller to list running queries and cancel them via the query ID and broker ID (as query ID is only local to broker), like in the following:
In order to speed up aggregations, you can enable metrics aggregation on the required column by adding a metric field in the corresponding schema and setting aggregateMetrics
to true in the table configuration. You can also use a star-tree index config for columns like these (see here for more about star-tree).
There are two ways to verify this:
Log in to a server that hosts segments of this table. Inside the data directory, locate the segment directory for this table. In this directory, there is a file named index_map
which lists all the indexes and other data structures created for each segment. Verify that the requested index is present here.
During query: Use the column in the filter predicate and check the value of numEntriesScannedInFilter
. If this value is 0, then indexing is working as expected (works for Inverted index).
Yes, Pinot uses a default value of LIMIT 10
in queries. The reason behind this default value is to avoid unintentionally submitting expensive queries that end up fetching or processing a lot of data from Pinot. Users can always overwrite this by explicitly specifying a LIMIT
value.
Pinot does not cache query results. Each query is computed in its entirety. Note though, running the same or similar query multiple times will naturally pull in segment pages into memory making subsequent calls faster. Also, for real-time systems, the data is changing in real-time, so results cannot be cached. For offline-only systems, caching layer can be built on top of Pinot, with invalidation mechanism built-in to invalidate the cache when data is pushed into Pinot.
Pinot memory maps segments. It warms up during the first query, when segments are pulled into the memory by the OS. Subsequent queries will have the segment already loaded in memory, and hence will be faster. The OS is responsible for bringing the segments into memory, and also removing them in favor of other segments when other segments not already in memory are accessed.
The query execution engine will prefer to use the star-tree index for all queries where it can be used. The criteria to determine whether the star-tree index can be used is as follows:
All aggregation function + column pairs in the query must exist in the star-tree index.
All dimensions that appear in filter predicates and group-by should be star-tree dimensions.
For queries where above is true, a star-tree index is used. For other queries, the execution engine will default to using the next best index available.
This page has a collection of frequently asked questions of a general nature with answers from the community.
This is a list of questions frequently asked in our troubleshooting channel on Slack. To contribute additional questions and answers, .
When data is pushed to Apache Pinot, Pinot makes a backup copy of the data and stores it on the configured deep-storage (S3/GCP/ADLS/NFS/etc). This copy is stored as tar.gz Pinot segments. Note, that Pinot servers keep a (untarred) copy of the segments on their local disk as well. This is done for performance reasons.
Pinot uses Apache Helix for cluster management, which in turn is built on top of Zookeeper. Helix uses Zookeeper to store the cluster state, including Ideal State, External View, Participants, and so on. Pinot also uses Zookeeper to store information such as Table configurations, schemas, Segment Metadata, and so on.
Please check the JDK version you are using. You may be getting this error if you are using an older version than the current Pinot binary release was built on. If so, you have two options: switch to the same JDK release as Pinot was built with or download the for the Pinot release and it locally.
This page has a collection of frequently asked questions about operations with answers from the community.
This is a list of questions frequently asked in our troubleshooting channel on Slack. To contribute additional questions and answers, make a pull request.
Typically, Apache Pinot components try to use as much off-heap (MMAP/DirectMemory) wherever possible. For example, Pinot servers load segments in memory-mapped files in MMAP mode (recommended), or direct memory in HEAP mode. Heap memory is used mostly for query execution and storing some metadata. We have seen production deployments with high throughput and low-latency work well with just 16 GB of heap for Pinot servers and brokers. The Pinot controller may also cache some metadata (table configurations etc) in heap, so if there are just a few tables in the Pinot cluster, a few GB of heap should suffice.
Pinot relies on deep-storage for storing a backup copy of segments (offline as well as real-time). It relies on Zookeeper to store metadata (table configurations, schema, cluster state, and so on). It does not explicitly provide tools to take backups or restore these data, but relies on the deep-storage (ADLS/S3/GCP/etc), and ZK to persist these data/metadata.
Changing a column name or data type is considered backward incompatible change. While Pinot does support schema evolution for backward compatible changes, it does not support backward incompatible changes like changing name/data-type of a column.
You can change the number of replicas by updating the table configuration's segmentsConfig section. Make sure you have at least as many servers as the replication.
For offline tables, update replication:
For real-time tables, update replicasPerPartition:
After changing the replication, run a table rebalance.
Note that if you are using replica groups, it's expected these configurations equal numReplicaGroups
. If they do not match, Pinot will use numReplicaGroups.
By default there is no retention set for a table in Apache Pinot. You may however, set retention by setting the following properties in the segmentsConfig section inside table configs:
retentionTimeUnit
retentionTimeValue
Updating the retention value in the table config should be good enough, there is no need to rebalance the table or reload its segments.
See Rebalance.
Likely explanation: num partitions * num replicas < num servers.
In real-time tables, segments of the same partition always remain on the same node. This sticky assignment is needed for replica groups and is critical if using upserts. For instance, if you have 3 partitions, 1 replica, and 4 nodes, only ¾ nodes will be used, and all of p0 segments will be on 1 node, p1 on 1 node, and p2 on 1 node. One server will be unused, and will remain unused through rebalances.
There’s nothing we can do about CONSUMING segments, they will continue to use only 3 nodes if you have 3 partitions. But we can rebalance such that completed segments use all nodes. If you want to force the completed segments of the table to use the new server use this config:
The number of segments generated depends on the number of input files. If you provide only 1 input file, you will get 1 segment. If you break up the input file into multiple files, you will get as many segments as the input files.
This typically happens when the server is unable to load the segment. Possible causes: out-of-memory, no disk space, unable to download segment from deep-store, and similar other errors. Check server logs for more information.
Use the segment reset controller REST API to reset the segment:
Refer to Pause Stream Ingestion.
Reset: Gets a segment in ERROR
state back to ONLINE
or CONSUMING
state. Behind the scenes, the Pinot controller takes the segment to the OFFLINE
state, waits for External View
to stabilize, and then moves it back to ONLINE
or CONSUMING
state, thus effectively resetting segments or consumers in error states.
Refresh: Replaces the segment with a new one, with the same name but often different data. Under the hood, the Pinot controller sets new segment metadata in Zookeeper, and notifies brokers and servers to check their local states about this segment and update accordingly. Servers also download the new segment to replace the old one, when both have different checksums. There is no separate rest API for refreshing, and it is done as part of the SegmentUpload API
.
Reload: Loads the segment again, often to generate a new index as updated in the table configuration. Underlying, the Pinot server gets the new table configuration from Zookeeper, and uses it to guide the segment reloading. In fact, the last step of REFRESH
as explained above is to load the segment into memory to serve queries. There is a dedicated rest API for reloading. By default, it doesn't download segments, but the option is provided to force the server to download the segment to replace the local one cleanly.
In addition, RESET
brings the segment OFFLINE
temporarily; while REFRESH
and RELOAD
swap the segment on server atomically without bringing down the segment or affecting ongoing queries.
Set this property in your controller.conf file:
Now your brokers and servers should join the cluster as broker_untagged
and server_untagged
. You can then directly use the POST /tenants
API to create the desired tenants, as in the following:
There are two task configurations, but they are set as part of cluster configurations, like in the following example. One controls the task's overall timeout (1hr by default) and one sets how many tasks to run on a single minion worker (1 by default). The <taskType> is the task to tune, such as MergeRollupTask
or RealtimeToOfflineSegmentsTask
etc.
See Running a Periodic Task Manually.
Yes, replica groups work for real-time. There's 2 parts to enabling replica groups:
Replica groups segment assignment.
Replica group query routing.
Replica group segment assignment
Replica group segment assignment is achieved in real-time, if number of servers is a multiple of number of replicas. The partitions get uniformly sprayed across the servers, creating replica groups. For example, consider we have 6 partitions, 2 replicas, and 4 servers.
p1
S0
S1
p2
S2
S3
p3
S0
S1
p4
S2
S3
p5
S0
S1
p6
S2
S3
As you can see, the set (S0, S2) contains r1 of every partition, and (s1, S3) contains r2 of every partition. The query will only be routed to one of the sets, and not span every server. If you are are adding/removing servers from an existing table setup, you have to run rebalance for segment assignment changes to take effect.
Replica group query routing
Once replica group segment assignment is in effect, the query routing can take advantage of it. For replica group based query routing, set the following in the table config's routing section, and then restart brokers
When using tiered storage, user may want to have different encoding and indexing types for a column in different tiers to balance query latency and cost saving more flexibly. For example, segments in the hot tier can use dict-encoding, bloom filter and all kinds of relevant index types for very fast query execution. But for segments in the cold tier, where cost saving matters more than low query latency, one may want to use raw values and bloom filters only.
The following two examples show how to overwrite encoding type and index configs for tiers. Similar changes are also demonstrated in the MultiDirQuickStart example.
Overwriting single-column index configs using fieldConfigList
. All top level fields in FieldConfig class can be overwritten, and fields not overwritten are kept intact.
Overwriting star-tree index configurations using tableIndexConfig
. The StarTreeIndexConfigs
is overwritten as a whole. In fact, all top level fields defined in IndexingConfig class can be overwritten, so single-column index configs defined in tableIndexConfig
can also be overwritten but it's less clear than using fieldConfigList
.
Wait for the pause status to change to success.
Update the credential in the table config.
Resume the consumption.
This page has a collection of frequently asked questions about Pinot on Kubernetes with answers from the community.
This is a list of questions frequently asked in our troubleshooting channel on Slack. To contribute additional questions and answers, make a pull request.
The following is an example using Amazon Elastic Kubernetes Service (Amazon EKS).
In the Kubernetes (k8s) cluster, check the storage class: in Amazon EKS, it should be gp2
.
Then update StorageClass to ensure:
Once StorageClass is updated, it should look like this:
Once the storage class is updated, then we can update the PersistentVolumeClaim (PVC) for the server disk size.
Now we want to double the disk size for pinot-server-3
.
The following is an example of current disks:
The following is the output of data-pinot-server-3
:
Now, let's change the PVC size to 2T
by editing the server PVC.
Once updated, the specification's PVC size is updated to 2T
, but the status's PVC size is still 1T
.
Restart the pinot-server-3
pod:
Recheck the PVC size: