Releases: apache/pinot
Apache Pinot Release 1.2.0
What Changed
This release comes with several Improvements and Bug Fixes for the Multistage Engine, Upserts and Compaction. There are a ton of other small features and general bug fixes.
Multistage Engine Improvements
Features
New Window Functions: LEAD, LAG, FIRST_VALUE, LAST_VALUE #12878 #13340
- LEAD allows you to access values after the current row in a frame.
- LAG allows you to access values before the current row in a frame.
- FIRST_VALUE and LAST_VALUE return the respective extremal values in the frame.
Support for Logical Database in V2 Engine #12591 #12695
- V2 Engine now supports a "database" construct, enabling table namespace isolation within the same Pinot cluster.
- Improves user experience when multiple users are using the same Pinot Cluster.
- Access control policies can be set at the database level.
- Database can be selected in a query using a SET statement, such as
SET database=my_db;
.
Improved Multi-Value (MV) and Array Function Support
- Added array sum aggregation functions for point-wise array operations #13324.
- Added support for
valueIn
MV transform function #13443. - Fixed bug in numeric casts for MV columns in filters #13425.
- Fixed NPE in ArrayAgg when a column contains no data #13358.
- Fixed array literal handling #13345.
Support for WITHIN GROUP Clause and ListAgg #13146
WITHIN GROUP
Clause can be used to process rows in a given order within a group.- One of the most common use-cases for this is the
ListAgg
function, which when combined withWITHIN GROUP
can be used to concatenate strings in a given order.
Scalar/Transform Function and Set Operation Improvements
- Added Geospatial Scalar Function support for use in intermediate stage in the v2 query engine #13457.
- Fix 'WEEK' transform function #13483.
- Support
EXTRACT
as a scalar function #13463. - Added support for ALL modifier for INTERSECT and EXCEPT Set Operations #13151 #13166.
Improved Literal Handling Support
- Fixed bug in handling literal arguments in aggregation functions like Percentile #13282.
- Allow INT and FLOAT literals #13078.
- Fixed literal handling for all types #13344 #13345.
- Fixed null literal handling for null intolerant functions #13255.
Metrics Improvements
- Added new metrics for tracking queries executed globally and at the table level #12982.
- New metrics to track join counts and window function counts #13032.
- Multiple meters and timers to track Multistage Engine Internals #13035.
Notable Improvements and Bug Fixes
- Improved Window operators resiliency, with new checks to make sure the window doesn't grow too large #13180 #13428 #13441.
- Optimized Group Key generation #12394.
- Fixed
SortedMailboxReceiveOperator
to honor convention of pulling at most 1 EOS block #12406. - Improvement in how execution stats are handled #12517 #12704 #13136.
- Use Protobuf instead of Reflection for Plan Serialization #13221.
Upsert Compaction and Minion Improvements
Features and Improvements
Minion Resource Isolation #12459 #12786
- Minions now support resource isolation based on an instance tag.
- Instance tag is configured at table level, and can be set for each task on a table.
- This enables you to implement arbitrary resource isolation strategies, i.e. you can use a set of Minion Nodes for running any set of tasks across any set of tables.
Greedy Upsert Compaction Scheduling #12461
- Upsert compaction now schedules segments for compaction based on the number of invalid docs.
- This helps the compaction task to handle arbitrary temporal distribution of invalid docs.
Notable Improvements
- Minions can now download segments from servers when deepstore copy is missing. This feature is enabled via a cluster level config
allowDownloadFromServer
#12960 #13247. - Added support for TLS Port in Minions #12943.
- New metrics added for Minions to track segment/record processing information #12710.
Bug Fixes
- Minions can now handle invalid instance tags in Task Configs gracefully. Prior to this change, Minions would be stuck in
IN_PROGRESS
state until task timeout #13092. - Fix bug to return validDocIDsMetadata from all servers #12431.
- Upsert compaction doesn't retain maxLength information and trims string fields #13157.
Upsert Improvements
Features and Improvements
Consistent Table View for Upsert Tables #12976
- Adds different modes of consistency guarantees for Upsert tables.
- Adds a new UpsertConfig called
consistencyMode
which can be set toNONE, SYNC, SNAPSHOT
. SYNC
is optimized for data freshness but can lead to elevated query latencies and is best for low-qps use-cases. In this mode, the ingestion threads will take a WLock when updating validDocID bitmaps.SNAPSHOT
mode can handle high-qps/high-ingestion use-cases by getting the list of valid docs from a snapshot of validDocID. The snapshot can be refreshed every few seconds and the tolerance can be set via a query optionupsertViewFreshnessMs
.
Pluggable Partial Upsert Merger #11983
- Partial Upsert merges the old record and the new incoming record to generate the final ingested record.
- Pinot now allows users to customize how this merge of an old row and the new row is computed.
- This allows a column value in the new row to be an arbitrary function of the old and the new row.
Support for Uploading Externally Partitioned Segments for Upsert Backfill 13107
- Segments uploaded for Upsert Backfill can now explicitly specify the Kafka partition they belong to.
- This enables backfilling an Upsert table where the externally generated segments are partitioned using an arbitrary hash function on an arbitrary primary key.
Misc Improvements and Bug Fixes
- Fixed a Bug in Handling Equal Comparison Column Values in Upsert, which could lead to data inconsistency (#12395)
- Upsert snapshot will now snapshot only those segments which have updates. #13285.
Notable Features
JSON Support Improvements
- JSON Index can now be used for evaluating Regex and Range Predicates. #12568
jsonExtractIndex
now supports contextual array filters. #12683 #12531.- JSON column type now supports filter predicates like
=
,!=
,IN
andNOT IN
. This is convenient for scenarios where the JSON values are very small. #13283. JSON_MATCH
now supports exclusive predicates correctly. For instance, you can use predicates such asJSON_MATCH(person, '"$.addresses[*].country" != ''us'''
to find all people who have at least one address that is not in the US. #13139.jsonExtractIndex
supports extracting Multi-Value JSON Fields, and also supports providing any default value when the key doesn't exist. #12748.- Added
isJson
UDF which increases your options to handle invalid JSONs. This can be used in queries and for filtering invalid json column values in ingestion. #12603. - Fix
ArrayIndexOutOfBoundsException
injsonExtractIndex
. #13479.
Lucene and Text Search Improvements
Apache Pinot Release 1.1.0
What Changed
This release comes with several features, SQL /UI/Perf enhancements Bugfixes across areas ranging from Multistage Query Engine to Ingestion, Storage format, SQL support, etc.
Multi-stage Query Engine
Features
Support RelDistribution-based trait Planning (#11976, #12079)
- Adds support for RelDistribution optimization for more accurate leaf-stage direct exchange/shuffle. Also extends partition optimization beyond leaf stage to entire query plan.
- Applies optimization based on distribution trait in the mailbox/worker assignment stage
- Fixes previous direct exchange which was decided based on the table partition hint. Now direct exchange is decided via distribution trait: it will applied if-and-only-if the trait propagated matches the exchange requirement.
- As a side effect,
is_colocated_by_join_keys
query option is reintroduced to ensure dynamic broadcast which can also benefit from direct exchange optimization - Allows propagation of partition distribution trait info across the tree to be used during Physical Planning phase. It can be used in the following scenarios (will follow up in separate PRs)
- Note on backward incompatbility
is_colocated_by_join_keys
hint is now required for making colocated joins- it should only affect semi-join b/c it is the only one utilizing broadcast exchange but were pulled to act as direct exchange.
- inner/left/right/full join should automatically apply colocation thus the backward incompatibility should not affect these.
Leaf stage planning with multi-semi join support (#11937)
- Solves the limitation of pinotQuery that supports limited amount of PlanNodes.
- Splits the ServerRequest planning into 2 stages
- First plan as much as possible into PinotQuery
- for any remainder nodes that cannot be planned into PinotQuery, will be run together with the LeafStageTransferrableBlockOperator as the input locally.
Support for ArrayAgg aggregation function (#11822)
- Usage: ArrayAgg(column, 'dataType' [, 'isDistinct'])
- Float type column is treated as Double in the multistage engine, so
FLOAT
type is not supported. - Supports data
BOOLEAN
,INT
,LONG
,FLOAT
(only in V1),DOUBLE
,STRING
,TIMESTAMP
.
E.g.ArrayAgg(intCol, 'INT')
returnsARRAY<INT>
Enhancements
-
Canonicalize
SqlKind.OTHERS
andSqlKind.OTHER_FUNCTIONS
and supportconcat
as||
operator (#12025) -
Capability for constant filter in
QueryContext
, with support for server to handle it (#11956) -
Tests for filter pushdown (#11994)
-
Enhancements to query plan tests (#11966)
-
Refactor PlanFragmenter to make the logic clear (#11912)
-
Observability enhancements to emit metrics for grpc request and multi-stage leaf stage (#11838)
pinot.server.query.log.maxRatePerSecond
: query log max rate (QPS, default 10K)pinot.server.query.log.droppedReportMaxRatePerSecond
: dropped query log report max rate (QPS, default 1)
-
Security enhancement to add RBAC authorization checks for multi-stage query engine (#11830)
-
Enhancement to leaf-stage execution stats NPE handling (#11805)
-
Enhancement to add a framework to back-propagate metadata across opChains (#11746)
-
Use of BinaryArray to wire proto for multi-stage engine bytes literal handling (#11738)
-
Enable dynamic broadcast for SEMI joins. Adds a fallback option to enable hash table join using
joinOptions(join_strategy = 'hash_table')
(#11696) -
Improvements to dispatch exception handling (#11688)
-
Allow malformed dateTime string to return default value configurable in the function signature (#11258)
-
fromDateTime(colContainsMalformedStr, '<dateTimeFormat>', '<timezone>', <default_value>)
-
-
Improvement in multi-stage aggregation to directly store column index as identifier (#11617)
-
Perf optimization to avoid unnecessary rows conversion in aggregation (#11607)
-
Enhance
SegmentPartitionMetadataManager
to handle new segment (#11585) -
Optimize mailbox info in query plan to reduce memory footprint (#12382)
- This PR changes the proto object structure, which will cause backward incompatibility when broker and server are running different version.
-
Optimizations to query plan serialization (#12370)
-
Optimization for parallel execution of Ser/de stage plan (#12363)
-
Optimizations in query dispatch (#12358)
-
Perf optimization for group-by and join for single key scenario (#11630)
Bugfixes, Refactoring, Cleanups, Tests
- Bugfix for evaluation of chained literal functions (#12248)
- Fixes to sort copy rule (#12251 and #12237)
- Fixes duplicate results for literal queries (#12240)
- Bugfix to use UTF-8 encoding for default Charset (#12213)
- Bugfix to escape table name when routing queries (#12212)
- Refactoring of planner code and removing unnecessary rules (#12070, #12052)
- Fix to remove unnecessar project after agg during relBuilder (#12058)
- Fixes issues multi-semi-join (#12038)
- Fixes leaf limit refactor issue (#12001)
- Add back filter merge after rule (#11989)
- Fix operator EOS pull (#11970)
- Fix type cast issue with dateTimeConvert scalar function (#11839, #11971)
- Fix to set explicit warning flags set on each stage stats (#11936)
- Fix mailbox visitor mismatch receive/send (#11908)
- Fix eliminate multiple exchanges in nested semi-join queries (#11882)
- Bugfix for multiple consecutive Exchange returning empty response (#11885)
- Fixing unit-test-2 build (#11889)
- Fix issue with realtime partition mismatch metric (#11871)
- Fix the NPE for rebalance retry (#11883)
- Bugfix to make Agg literal attach happen after BASIC_RULES (#11863)
- Fix NPE by init execution stats map (#11801)
- Test cases for special column escape (#11737)
- Fix StPoint scalar function usage in multi-stage engine intermediate stage (#11731)
- Clean up for transform function type (#11726)
- Add capability to ignore test (#11703)
- Fix custom property naming (#11675)
- Log warning when multi-stage engine planning throws exception (#11595)
- Fix usage of metadata overrides (#11587)
- Test change to enable metadata manager by default for colocated join quickstart (#11579)
- Tests for IN/NOT-IN operation (#12349)
- Fix stage id in stage plan (#12366)
- Bugfix for IN and NOT IN filters within case statements (#12305)
Notable Features
Server-level throttling for realtime consumption (#12292)
- Use server config
pinot.server.consumption.rate.limit
to enable this feature - Server rate limiter is disabled by default (default value 0)
Reduce segment generation disk footprint for Minion Tasks (#12220)
- Supported in
MergeRollupTask
andRealtimeToOfflineSegmentsTask
minion tasks - Use taskConfig
segmentMapperFileSizeThresholdInBytes
to specify the threshold size
"task": {
"taskTypeConfigsMap": {
"<task_name>": {
"segmentMapperFileSizeThresholdInBytes": "1000000000"
}
}
}
Support for swapping of TLS keyst...
Apache Pinot Release 1.0.0
What's changed
Major features and updates
Multistage Engine
Multistage engine new features:
Support for Window Functions
Initial (phase 1) Query runtime for window functions with ORDER BY within the OVER() clause (#10449)
Support for the ranking ROW_NUMBER() window function (#10527, #10587)
Set Operations Support
Support SetOperations (UNION, INTERSECT, MINUS) compilation in query planner (#10535)
Timestamp and Date Operations
Support TIMESTAMP type and date ops functions (#11350)
Aggregate Functions
Support more aggregation functions that are currently implementable (#11208)
Support multi-value aggregation functions (#11216)
Support Sketch based functions (#)
Make Intermediate Stage Worker Assignment Tenant Aware (#10617)
Evaluate literal expressions during query parsing, enabling more efficient query execution. (#11438)
Added support for partition parallelism in partitioned table scans, allowing for more efficient data retrieval (#11266).
[multistage]Adding more tuple sketch scalar functions and integration tests (#11517)
Multistage engine enhancements
Turn on v2 engine by default (#10543)
Introduced the ability to stream leaf stage blocks for more efficient data processing (#11472).
Early terminate SortOperator if there is a limit (#11334)
Implement ordering for SortExchange (#10408)
Table level Access Validation, QPS Quota, Phase Metrics for multistage queries (#10534)
Support partition based leaf stage processing (#11234)
Populate queryOption down to leaf (#10626)
Pushdown explain plan queries from the controller to the broker (#10505)
Enhanced the multi-stage group-by executor to support limiting the number of groups, improving query performance and resource utilization (#11424).
Improved resilience and reliability of the multi-stage join operator, now with added support for hash join right table protection (#11401).
Multistage engine bug fixes
Fix Predicate Pushdown by Using Rule Collection (#10409)
Try fixing mailbox cancel race condition (#10432)
Catch Throwable to Propagate Proper Error Message (#10438)
Fix tenant detection issues (#10546)
Handle Integer.MIN_VALUE in hashCode based FieldSelectionKeySelector (#10596)
Improve error message in case of non-existent table queried from the controller (#10599)
Derive SUM return type to be PostgreSQL compatible (#11151)
Index SPI
Add the ability to include new index types at runtime in Apache Pinot. This opens the ability of adding third party indexes, including proprietary indexes. More details here
Null value support for pinot queries
NULL support for ORDER BY, DISTINCT, GROUP BY, value transform functions and filtering.
Upsert enhancements
Delete support in upsert enabled tables (#10703)
Support added to extend upserts and allow deleting records from a realtime table. The design details can be found here.
Preload segments with upsert snapshots to speedup table loading (#11020)
Adds a feature to preload segments from a table that uses the upsert snapshot feature. The segments with validDocIds snapshots can be preloaded in a more efficient manner to speed up the table loading (thus server restarts).
TTL configs for upsert primary keys (#10915)
Adds support for specifying expiry TTL for upsert primary key metadata cleanup.
Segment compaction for upsert real-time tables (#10463)
Adds a new minion task to compact segments belonging to a real-time table with upserts.
Pinot Spark Connector for Spark3 (#10394)
- Added spark3 support for Pinot Spark Connector (#10394)
- Also added support to pass pinot query options to spark connector (#10443)
PinotDataBufferFactory and new PinotDataBuffer implementations (#10528)
Adds new implementations of PinotDataBuffer that uses Unsafe java APIs and foreign memory APIs. Also added support for PinotDataBufferFactory to allow plugging in custom PinotDataBuffer implementations.
Query functions enhancements
- Add PercentileKLL aggregation function (#10643)
- Support for ARG_MIN and ARG_MAX Functions (#10636)
- refactor argmin/max to exprmin/max and make it calcite compliant (#11296)
- Integer Tuple Sketch support (#10427)
- Adding vector scalar functions (#11222)
- [feature] multi-value datetime transform variants (#10841)
- FUNNEL_COUNT Aggregation Function (#10867)
- [multistage] Add support for RANK and DENSE_RANK ranking window functions (#10700)
- add theta sketch scalar (#11153)
- Register dateTimeConverter,timeConvert,dateTrunc, regexpReplace to v2 functions (#11097)
- Add extract(quarter/dow/doy) support (#11388)
- Funnel Count - Multiple Strategies (no partitioning requisites) (#11092)
- Add Boolean assertion transform functions. (#11547)
JSON and CLP encoded message ingestion and querying
- Add clpDecode transform function for decoding CLP-encoded fields. (#10885)
- Add CLPDecodeRewriter to make it easier to call clpDecode with a column-group name rather than the individual columns. (#11006)
- Add SchemaConformingTransformer to transform records with varying keys to fit a table's schema without dropping fields. (#11210)
Tier level index config override (#10553)
Allows overriding index configs at tier level, allowing for more flexible index configurations for different tiers.
Ingestion connectors and features
- Kinesis stream header extraction (#9713)
- Extract record keys, headers and metadata from Pulsar sources (#10995)
- Realtime pre-aggregation for Distinct Count HLL & Big Decimal (#10926)
- Added support to skip unparseable records in the csv record reader (#11487)
- Null support for protobuf ingestion. (#11553)
UI enhancements
- Adds persistence of authentication details in the browser session. This means that even if you refresh the app, you will still be logged in until the authentication session expires (#10389)
- AuthProvider logic updated to decode the access token and extract user name and email. This information will now be available in the app for features to consume. (#10925)
Pinot docker image improvements and enhancements
- Make Pinot base build and runtime images support Amazon Corretto and MS OpenJDK (#10422)
- Support multi-arch pinot docker image (#10429)
- Update dockerfile with recent jdk distro changes (#10963)
Operational improvements
Rebalance
- Rebalance status API (#10359)
- Tenant level rebalance API Tenant rebalance and status tracking APIs (#11128)
Config to use customized broker query thread pool (#10614)
Added new configuration options below which allow use of a bounded thread pool and allocate capacities for it.
pinot.broker.enable.bounded.http.async.executor
pinot.broker.http.async.executor.max.pool.size
pinot.broker.http.async.executor.core.pool.size
pinot.broker.http.async.executor.queue.size
This feature allows better management of broker resources.
Drop results support (#10419)
Adds a parameter to queryOptions to drop the resultTable from the response. This mode can be used to troubleshoot a query (which may have sensitive data in the result) using metadata only.
Make column order deterministic in segment (#10468)
In segment metadata and index map, store columns in alphabetical order so that the result is deterministic. Segments generated before/after this PR will have different CRC, so during the upgrade, we might get segments with different CRC from old and new consuming servers. For the segment consumed during the upgrade, some downloads might be needed.
Allow configuring helix timeouts for EV dropped in Instance manager (#10510)
Adds options to configure helix timeouts
external.view.dropped.max.wait.ms`` - The duration of time in milliseconds to wait for the external view to be dropped. Default - 20 minutes.
external.view.check.interval.ms`` - The period in milliseconds in which to ping ZK for latest EV state.
Enable case insensitivity by default (#10771)
This PR makes Pinot case insensitive be default, and removes the deprecated property enable.case.insensitive.pql
Newly added APIs and client methods
- Add Server API to get tenant pools (#11273)
- Add new broker query point for querying multi-stage engine (#11341)
- Add a new controller endpoint for segment deletion with a time window (#10758)
- New API to get tenant tags (#10937)
- Instance retag validation check api (#11077)
- Use PUT request to enable/disable table/instance (#11109)
- Update the pinot tenants tables api to support returning broker tagged tables (#11184)
- Add requestId for BrokerResponse in pinot-broker and java-client (#10943)
- Provide results in CompletableFuture for java clients and expose metrics (#10326)
Cleanup and backward incompatible changes
High level consumers are no longer supported
Type information preservation of query literals
- [feature] [backward-incompat] [null support # 2] Preserve null literal information in literal context and literal transform (#10380)
String versions of numerical values are no longer accepted. For example, "123" won't be treated as a numerical anymore.
Controller job status ZNode path update
- Moving Zk updates for reload, force_commit to their own Znodes which … (#10451)
The status of previously completed reload jobs will not be available after this change is deployed.
Metric names for mutable indexes to change
- Implement mutable index using index SPI (#10687)
Due to a change in the IndexType enum used for some logs an...
Apache Pinot Release 0.12.1
What's Changed
Major updates
- Fixed backward compatibility issues related to CASE-WHEN statement. #10291
Apache Pinot Release 0.12.0
What's Changed
Major updates
- Force commit consuming segments by @sajjad-moradi in #9197
- add a freshness based consumption status checker by @jadami10 in #9244
- Add metrics to track controller segment download and upload requests in progress by @gviedma in #9258
- Adding endpoint to download local log files for each component by @xiangfu0 in #9259
- [Feature] Add an option to search input files recursively in ingestion job. The default is set to true to be backward compatible. by @61yao in #9265
- add query cancel APIs on controller backed by those on brokers by @klsince in #9276
- Add Spark Job Launcher tool by @KKcorps in #9288
- Enable Consistent Data Push for Standalone Segment Push Job Runners by @yuanbenson in #9295
- Allow server to directly return the final aggregation result by @Jackie-Jiang in #9304
- TierBasedSegmentDirectoryLoader to keep segments in multi-datadir by @klsince in #9306
- Adaptive Server Selection by @vvivekiyer in #9311
- [Feature] Support IsDistinctFrom and IsNotDistinctFrom by @61yao in #9312
- Allow ingestion of errored records with incorrect datatype by @KKcorps in #9320
- Allow setting custom time boundary for hybrid table queries by @saurabhd336 in #9356
- skip late cron job with max allowed delay by @klsince in #9372
- Do not allow implicit cast for BOOLEAN and TIMESTAMP by @Jackie-Jiang in #9385
- Add missing properties in CSV plugin by @KKcorps in #9399
- set MDC so that one can route minion task logs to separate files cleanly by @klsince in #9400
- Add a new API to fix segment date time in metadata by @KKcorps in #9413
- Update get bytes to return raw bytes of string and support getBytesMV by @61yao in #9441
- Exposing consumer's record lag in /consumingSegmentsInfo by @navina in #9515
- Do not create dictionary for high-cardinality columns by @KKcorps in #9527
- get task runtime configs tracked in Helix by @klsince in #9540
- Add more options to json index by @Jackie-Jiang in #9543
- add SegmentTierAssigner and refine restful APIs to get segment tier info by @klsince in #9598
- Add segment level debug API by @saurabhd336 in #9609
- Add record availability lag for Kafka connector by @navina in #9621
- notify servers that need to move segments to new tiers via SegmentReloadMessage by @klsince in #9624
- Allow to configure multi-datadirs as instance configs and a Quickstart example about them by @klsince in #9705
- Customize stopword for Lucene Index by @jasperjiaguo in #9708
- Add memory optimized dimension table by @KKcorps in #9802
- ADLS file system upgrade by @xiangfu0 in #9855
- Added Delete Schema/Table pinot admin commands by @bagipriyank in #9857
- Adding new ADLSPinotFS auth type: DEFAULT by @xiangfu0 in #9860
- Add rate limit to Kinesis requests by @KKcorps in #9863
- Adding configs for zk client timeout by @xiangfu0 in #9975
Other features/changes
- Show most recent scheduling errors by @satishwaghela in #9161
- Do not use aggregation result for distinct query in IntermediateResultsBlock by @Jackie-Jiang in #9262
- Emit metrics for ratio of actual consumption rate to rate limit in realtime tables by @sajjad-moradi in #9201
- add metrics entry offlineTableCount by @walterddr in #9270
- refine query cancel resp msg by @klsince in #9242
- add @ManualAuthorization annotation for non-standard endpoints by @apucher in #9252
- Optimize ser/de to avoid using output stream by @Jackie-Jiang in #9278
- Add Support for Covariance Function by @SabrinaZhaozyf in #9236
- Throw an exception when MV columns are present in the order-by expression list in selection order-by only queries by @somandal in #9078
- Improve server query cancellation and timeout checking during execution by @jasperjiaguo in #9286
- Add capabilities to ingest from another stream without disabling the realtime table by @sajjad-moradi in #9289
- Add minMaxInvalid flag to avoid unnecessary needPreprocess by @npawar in #9238
- Add array cardinality function by @walterddr in #9300
- TierBasedSegmentDirectoryLoader to keep segments in multi-datadir by @klsince in #9306
- Add support for custom null values in CSV record reader by @KKcorps in #9318
- Infer parquet reader type based on file metadata by @saurabhd336 in #9294
- Include fmpp plugin module inside the src assembly file by @xiangfu0 in #9321
- Add Support for Cast Function on MV Columns by @SabrinaZhaozyf in #9296
- Allow ingestion of errored records with incorrect datatype by @KKcorps in #9320
- [Feature] Not Operator Transformation by @61yao in #9330
- Handle null string in CSV decoder by @KKcorps in #9340
- [Feature] Not scalar function by @61yao in #9338
- Add support for EXTRACT syntax and converts it to appropriate Pinot expression by @tanmesh in #9184
- Add support for Auth in controller requests in java query client by @KKcorps in #9230
- delete all related minion task metadata when deleting a table by @zhtaoxiang in #9339
- BloomFilterRule should only recommend for supported column type by @yuanbenson in #9364
- Support all the types in ParquetNativeRecordReader by @xiangfu0 in #9352
- Improve segment name check in metadata push by @zhtaoxiang in #9359
- Allow expression transformer cotinue on error by @xiangfu0 in #9376
- skip late cron job with max allowed delay by @klsince in #9372
- Enhance
and
filter predicate evaluation efficiency by @jasperjiaguo in #9336 - Deprecate instanceId Config For Broker/Minion Specific Configs by @ankitsultana in #9308
- Optimize combine operator to fully utilize threads by @Jackie-Jiang in #9387
- Terminate the query after plan generation if timeout by @jasperjiaguo in #9386
- [Feature] Support IsDistinctFrom and IsNotDistinctFrom by @61yao in #9312
- [Feature] Support Coalesce for Column Names by @61yao in #9327
- Disable logging for interrupted exceptions in kinesis by @KKcorps in #9405
- Benchmark thread cpu time by @jasperjiaguo in #9408
- Use ISODateTimeFormat as default for SIMPLE_DATE_FORMAT by @KKcorps in #9378
- Extract the common logic for upsert metadata manager by @Jackie-Jiang in #9435
- Make minion task metadata manager methods more generic by @saurabhd336 in #9436
- Always pass clientId to kafka's consumer properties by @navina in #9444
- Adaptive Server Selection by @vvivekiyer in #9311
- Refine IndexHandler methods a bit to make them reentrant by @klsince in #9440
- use MinionEventObserver to track finer grained task progress status on worker by @klsince in #9432
- Allow spaces in input file paths by @KKcorps in #9426
- Add support for gracefully handling the errors while transformations by @KKcorps in #9377
- Cache Deleted Segment Names in Server to Avoid SegmentMissingError by @ankitsultana in #9423
- Handle Invalid timestamps by @KKcorps in #9355
- refine minion worker event observer to track finer grained progress for tasks by @klsince in https://github.com/apache/pinot/p...
Apache Pinot Release 0.11.0
Summary
Apache Pinot 0.11.0 has introduced many new features to extend the query abilities, e.g. the Multi-Stage query engine enables Pinot to do distributed joins, more sql syntax(DML support), query functions and indexes(Text index, Timestamp index) supported for new use cases. And as always, more integrations with other systems(E.g. Spark3, Flink).
Note: there is a major upgrade for Apache Helix to 1.0.4, so please make sure you upgrade the system in the order of:
Helix Controller -> Pinot Controller -> Pinot Broker -> Pinot server
Multi-Stage Query Engine
The new multi-stage query engine (a.k.a V2 query engine) is designed to support more complex SQL semantics such as JOIN, OVER window, MATCH_RECOGNIZE and eventually, make Pinot support closer to full ANSI SQL semantics.
More to read: https://docs.pinot.apache.org/developers/advanced/v2-multi-stage-query-engine
Pause Stream Consumption on Apache Pinot
Pinot operators can pause realtime consumption of events while queries are being executed, and then resume consumption when ready to do so again.
More to read: https://medium.com/apache-pinot-developer-blog/pause-stream-consumption-on-apache-pinot-772a971ef403
Gap-filling function
The gapfilling functions allow users to interpolate data and perform powerful aggregations and data processing over time series data.
More to read: https://www.startree.ai/blog/gapfill-function-for-time-series-datasets-in-pinot
Add support for Spark 3.x (#8560)
Long waiting feature for segment generation on Spark 3.x.
Add Flink Pinot connector (#8233)
Similar to the Spark Pinot connector, this allows Flink users to dump data from the Flink application to Pinot.
Show running queries and cancel query by id (#9175)
This feature allows better fine-grained control on pinot queries.
Timestamp Index (#8343)
This allows users to have better query performance on the timestamp column for lower granularity. See: https://docs.pinot.apache.org/basics/indexing/timestamp-index
Native Text Indices (#8384)
Wanna search text in realtime? The new text indexing engine in Pinot supports the following capabilities:
- New operator: LIKE
select * FROM foo where text_col LIKE 'a%'
- New operator: CONTAINS
select * from foo where text_col CONTAINS 'bar'
- Native text index, built from the ground up, focusing on Pinot’s time series use cases and utilizing existing Pinot indices and structures(inverted index, bitmap storage).
- Real Time Text Index
Read more: https://medium.com/@atri.jiit/text-search-time-series-style-681af37ba42e
Adding DML definition and parse SQL InsertFile (#8557)
Now you can use INSERT INTO [database.]table FROM FILE dataDirURI OPTION ( k=v ) [, OPTION (k=v)]*
to load data into Pinot from a file using Minion. See: https://docs.pinot.apache.org/basics/data-import/from-query-console
Deduplication (#8708)
This feature supports enabling deduplication for realtime tables, via a top-level table config. At a high level, primaryKey (as defined in the table schema) hashes are stored into in-memory data structures, and each incoming row is validated against it. Duplicate rows are dropped.
The expectation while using this feature is for the stream to be partitioned by the primary key, strictReplicaGroup routing to be enabled, and the configured stream consumer type to be low level. These requirements are therefore mandated via table config API's input validations.
Functions support and changes:
- Add support for functions arrayConcatLong, arrayConcatFloat, arrayConcatDouble (#9131)
- Add support for regexpReplace scalar function (#9123)
- Add support for Base64 Encode/Decode Scalar Functions (#9114)
- Optimize like to regexp conversion to do not include unnecessary ^._ and ._$ (#8893)
- Support DISTINCT on multiple MV columns (#8873)
- Support DISTINCT on single MV column (#8857)
- Add histogram aggregation function (#8724)
- Optimize dateTimeConvert scalar function to only parse the format once (#8939)
- Support conjugates for scalar functions, add more scalar functions (#8582)
- add FIRSTWITHTIME aggregate function support #7647 (#8181)
- Add PercentileSmartTDigestAggregationFunction (#8565)
- Simplify the parameters for DistinctCountSmartHLLAggregationFunction (#8566)
- add scalar function for cast so it can be calculated at compile time (#8535)
- Scalable Gapfill Implementation for Avg/Count/Sum (#8647)
- Add commonly used math, string and date scalar functions in Pinot (#8304)
- Datetime transform functions (#8397)
- Scalar function for url encoding and decoding (#8378)
- Add support for IS NULL and NOT IS NULL in transform functions (#8264)
- Support st_contains using H3 index (#8498)
The full list of features introduced in this release
- add query cancel APIs on controller backed by those on brokers (#9276)
- Add an option to search input files recursively in ingestion job. The default is set to true to be backward compatible. (#9265)
- Adding endpoint to download local log files for each component (#9259)
- Add metrics to track controller segment download and upload requests in progress (#9258)
- add a freshness based consumption status checker (#9244)
- Force commit consuming segments (#9197)
- Adding kafka offset support for period and timestamp (#9193)
- Make upsert metadata manager pluggable (#9186)
- Adding logger utils and allow change logger level at runtime (#9180)
- Proper null handling in equality, inequality and membership operators for all SV column data types (#9173)
- support to show running queries and cancel query by id (#9171)
- Enhance upsert metadata handling (#9095)
- Proper null handling in Aggregation functions for SV data types (#9086)
- Add support for IAM role based credentials in Kinesis Plugin (#9071)
- Task genrator debug api (#9058)
- Add Segment Lineage List API #9005 (#9006)
- [colocated-join] Adds Support for instancePartitionsMap in Table Config (#8989)
- Support pause/resume consumption of realtime tables (#8986)
- #8970 Minion tab in Pinot UI (#8978)
- Add Protocol Buffer Stream Decoder (#8972)
- Update minion task metadata ZNode path (#8959)
- add /tasks/{taskType}/{tableNameWithType}/debug API (#8949)
- Defined a new broker metric for total query processing time (#8941)
- Proper null handling in SELECT, ORDER BY, DISTINCT, and GROUP BY (#8927)
- fixing REGEX OPTION parser (#8905)
- Enable key value byte stitching in PulsarMessageBatch (#8897)
- Add property to skip adding hadoop jars to package (#8888)
- Support DISTINCT on multiple MV columns (#8873)
- Implement Mutable FST Index (#8861)
- Support DISTINCT on single MV column (#8857)
- Add controller API for reload segment task status (#8828)
- Spark Connector, support for TIMESTAMP and BOOLEAN fields (#8825)
- Allow moveToFinalLocation in METADATA push based on config (#8823) (#8815)
- allow up to 4GB per bitmap index (#8796)
- Deprecate debug options and always use query options (#8768)
- Streamed segment download & untar with rate limiter to control disk usage (#8753)
- Improve the Explain Plan accuracy (#8738)
- allow to set https as the default scheme (#8729)
- Add histogram aggregation function (#8724)
- Allow table name with dots by a PinotConfiguration switch (#8713)
- Disable Groovy function by default (#8711)
- Deduplication (#8708)
- Add pluggable client auth provider (#8670)
- Adding pinot file system command (#8659)
- Allow broker to automatically rewrite expensive function to its approximate counterpart (#8655)
- allow to take data outside the time window by negating the window filter (#8640)
- Support BigDecimal raw value forward index; Support BigDecimal in many transforms and operators (#8622)
- Ingestion Aggregation Feature (#8611)
- Enable uploading segments to realtime tables (#8584)
- Package kafka 0.9 shaded jar to pinot-distribution (#8569)
- Simplify the parameters for DistinctCountSmartHLLAggregationFunction (#8566)
- Add PercentileSmartTDigestAggregationFunction (#8565)
- Add support for Spark 3.x (#8560)
- Adding DML definition and parse SQL InsertFile (#8557)
- endpoints to get and delete minion task metadata (#8551)
- Add query option to use more replica groups (#8550)
- Only discover public methods annotated with @ScalarFunction (#8544)
- Support single-valued BigDecimal in schema, type conversion, SQL statements and minimum set of transforms. (#8503)
- Add connection based FailureDetector (#8491)
- Add endpoints for some finer control on minion tasks (#8486)
- Add adhoc minion task creation endpoint (#8465)
- Rewrite PinotQuery based on expression hints at instance/segment level (#8451)
- Allow disabling dict generation for High cardinality columns (#8398)
- add segment size metric on segment push (#8387)
- Implement Native Text Operator (#8384)
- Change default memory allocation for consuming segments from on-heap to off-heap (#8380)
- New Pinot storage metrics for compressed tar.gz and table size w/o replicas (#8358)
- add a experiment API for upsert heap memory estimation (#8355)
- Timestamp type index (#8343)
- Upgrade Helix to 1.0.4 in Pinot (#8325)
- Allow overriding expression in query through query config (#8319)
- Always handle null time values (#8310)
- Add prefixesToRename config for renaming fields upon ingestion (#8273)
- Added multi column partitioning for offline table (#8255)
- Automatically update broker resource on broker changes (#8249)
Vulnerability fixs
Pinot has resolved all the high-level vulnerabilities issues:
- Add a new workflow to check vulnerabilities using trivy (#9044)
- Disable Groovy function by de...
0.10.0
Summary
This release introduces some new great features, performance enhancements, UI improvments, and bug fixes which are described in details in the following sections.
The release was cut from this commit fd9c58a.
Dependency Graph
The dependency graph for plug-and-play architecture that was introduced in release 0.3.0 has been extended and now it contains new nodes for Pinot Segment SPI.
SQL Improvements
- Implement NOT Operator (#8148)
- Add DistinctCountSmartHLLAggregationFunction which automatically store distinct values in Set or HyperLogLog based on cardinality (#8189)
- Add LEAST and GREATEST functions (#8100)
- Handle SELECT * with extra columns (#7959)
- Add FILTER clauses for aggregates (#7916)
- Add ST_Within function (#7990)
- Handle semicolon in query (#7861)
- Add EXPLAIN PLAN (#7568)
UI Enhancements
- Show Reported Size and Estimated Size in human readable format in UI (#8199)
- Make query console state URL based (#8194)
- Improve query console to not show query result when multiple columns have the same name (#8131)
- Improve Pinot dashboard tenant view to show correct amount of servers and brokers (#8115)
- Fix issue with opening new tabs from Pinot Dashboard (#8021)
- Fix issue with Query console going blank on syntax error (#8006)
- Make query stats always show even there's error (#7981)
- Implement OIDC auth workflow in UI (#7121)
- Add tooltip and modal for table status (#7899)
- Add option to wrap lines in custom code mirror (#7857)
- Add ability to comment out queries with cmd + / (#7841)
- Return exception when unavailable segments on empty broker response (#7823)
- Properly handle the case where segments are missing in externalview (#7803)
- Add TIMESTAMP to datetime column Type (#7746)
Performance Improvements
- Reuse regex matcher in dictionary based LIKE queries (#8261)
- Early terminate orderby when columns already sorted (#8228)
- Do not do another pass of Query Automaton Minimization (#8237)
- Improve RangeBitmap by upgrading RoaringBitmap (#8206)
- Optimize geometry serializer usage when literal is available (#8167)
- Improve performance of no-dictionary group by (#8195)
- Allocation free
DataBlockCache
lookups (#8140) - Prune unselected THEN statements in CaseTransformFunction (#8138)
- Aggregation delay conversion to double (#8139)
- Reduce object allocation rate in ExpressionContext or FunctionContext (#8124)
- Lock free DimensionDataTableManager (#8102)
- Improve json path performance during ingestion by upgrading JsonPath (#7819)
- Reduce allocations and speed up StringUtil.sanitizeString (#8013)
- Faster metric scans - ForwardIndexReader (#7920)
- Unpeel group by 3 ways to enable vectorization (#7949)
- Power of 2 fixed size chunks (#7934)
- Don't use mmap for compression except for huge chunks (#7931)
- Exit group-by marking loop early (#7935)
- Improve performance of base chunk forward index write (#7930)
- Cache JsonPaths to prevent compilation per segment (#7826)
- Use
LZ4
as default compression mode (#7797) - Peel off special case for 1 dimensional groupby (#7777)
- Bump roaringbitmap version to improve range queries performance (#7734)
Other Notable Features
- Adding NoopPinotMetricFactory and corresponding changes (#8270)
- Allow to specify fixed segment name for SegmentProcessorFramework (#8269)
- Move all prestodb dependencies into a separated module (#8266)
- Include docIds in Projection and Transform block (#8262)
- Automatically update broker resource on broker changes (#8249)
- Update ScalarFunction annotation from name to names to support function alias. (#8252)
- Implemented BoundedColumnValue partition function (#8224)
- Add copy recursive API to pinotFS (#8200)
- Add Support for Getting Live Brokers for a Table (without type suffix) (#8188)
- Pinot docker image - cache prometheus rules (#8241)
- In BrokerRequestToQueryContextConverter, remove unused filterExpressionContext (#8238)
- Adding retention period to segment delete REST API (#8122)
- Pinot docker image - upgrade prometheus and scope rulesets to components (#8227)
- Allow segment name postfix for SegmentProcessorFramework (#8230)
- Superset docker image - update pinotdb version in superset image (#8231)
- Add retention period to deleted segment files and allow table level overrides (#8176)
- Remove incubator from pinot and superset (#8223)
- Adding table config overrides for disabling groovy (#8196)
- Optimise sorted docId iteration order in mutable segments (#8213)
- Adding secure grpc query server support (#8207)
- Move Tls configs and utils from pinot-core to pinot-common (#8210)
- Reduce allocation rate in LookupTransformFunction (#8204)
- Allow subclass to customize what happens pre/post segment uploading (#8203)
- Enable controller service auto-discovery in Jersey framework (#8193)
- Add support for pushFileNamePattern in pushJobSpec (#8191)
- Add additionalMatchLabels to helm chart (#7177)
- Simulate rsvps after meetup.com retired the feed (#8180)
- Adding more checkstyle rules (#8197)
- Add persistence.extraVolumeMounts and persistence.extraVolumes to Kubernetes statefulsets (#7486)
- Adding scala profile for kafka 2.x build and remove root pom scala dependencies (#8174)
- Allow realtime data providers to accept non-kafka producers (#8190)
- Enhance revertReplaceSegments api (#8166)
- Adding broker level config for disabling Pinot queries with Groovy (#8159)
- Make presto driver query pinot server with SQL (#8186)
- Adding controller config for disabling Groovy in ingestionConfig (#8169)
- Adding main method for LaunchDataIngestionJobCommand for spark-submit command (#8168)
- Add auth token for segment replace rest APIs (#8146)
- Add allowRefresh option to UploadSegment (#8125)
- Add Ingress to Broker and Controller helm charts (#7997)
- Improve progress reporter in SegmentCreationMapper [(#8129)](https:...
Apache Pinot 0.9.3
This is a bug fixing release contains:
Update Log4j to 2.17.0 to address CVE-2021-45105 (#7933)
The release is based on the release 0.9.2 with the following cherry-picks:
Apache Pinot 0.9.2
Summary
This is a bug fixing release contains:
- Upgrade log4j to 2.16.0 to fix CVE-2021-45046 (#7903)
- Upgrade swagger-ui to 3.23.11 to fix CVE-2019-17495 (#7902)
- Fix the bug that RealtimeToOfflineTask failed to progress with large time bucket gaps (#7814).
The release is based on the release 0.9.1 with the following cherry-picks:
Apache Pinot 0.9.1
Summary
This release fixes the major issue of CVE-2021-44228 and a major bug fixing of pinot admin exit code issue(#7798).
The release is based on the release 0.9.0 with the following cherry-picks: