Release Notes for 1.1.0
This release comes with several features, including SQL, UI, and performance enhancements. Also included are bug fixes across multiple features such as the V2 multi-stage query engine, ingestion, storage format, and SQL support.
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')
returns ARRAY<INT>
Canonicalize SqlKind.OTHERS
and SqlKind.OTHER_FUNCTIONS
and support
concat
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)
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)
Bugfix for evaluation of chained literal functions (#12248)
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)
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 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)
Use server config pinot.server.consumption.rate.limit
to enable this feature
Server rate limiter is disabled by default (default value 0)
Supported in MergeRollupTask
and RealtimeToOfflineSegmentsTask
minion tasks
Use taskConfig segmentMapperFileSizeThresholdInBytes
to specify the threshold size
Security feature that makes the keystore/truststore swappable.
Auto-reloads keystore/truststore (without need for a restart) if they are local files
Adds support for deterministic and sticky routing for a query / table / broker. This setting would lead to same server / set of servers (for MultiStageReplicaGroupSelector
) being used for all queries of a given table.
Query option (takes precedence over fixed routing setting at table / broker config level)
SET "useFixedReplica"=true;
Table config (takes precedence over fixed routing setting at broker config level)
Broker conf - pinot.broker.use.fixed.replica=true
Use tableConfig dimensionTableConfig.errorOnDuplicatePrimaryKey=true
to enable this behavior
Disabled by default
Support to force-commit specific partitions of a realtime table.
Partitions can be specified to the forceCommit
API as a comma separated list of partition names or consuming segment names
Support to give the broker initial tags on startup.
Automatically updates brokerResource when broker joins the cluster for the first time
Broker tags are provided as comma-separated values in pinot.broker.instance.tags
StreamNative (the cloud SAAS offering of Pulsar) uses OAuth2 to authenticate clients to their Pulsar clusters.
For more information, see how to Configure OAuth2 authentication in Pulsar clients
Can be configured by adding the following properties to streamConfigs
:
Introduces a new table rebalance boolean config lowDiskMode.
Default value is false.
Applicable for rebalance with downtime=false.
When enabled, segments will first be offloaded from servers, then added to servers after offload is done. It may increase the total time of the rebalance, but can be useful when servers are low on disk space, and we want to scale up the cluster and rebalance the table to more servers.
#12112 adds the UI capability to toggle this option
Supports Vector Index on float array/multi-value columnz
Add predicate and function to retrieve topK closest vector. Example query
The function l2_distance
will return a double value where the first parameter is the embedding column and the second parameter is the search term embedding literal.
Since VectorSimilarity
is a predicate, once config the topK
, this predicate will return topk
rows per segment. Then if you are using this index with other predicate, you may not get expected number of rows since the records matching other predicate might not in the topk
rows.
Adds an upsert config deletedKeysTTL
which will remove deleted keys from in-memory hashmap and mark the validDocID as invalid after the deletedKeysTTL
threshold period.
Disabled by default. Enabled only if a valid value for deletedKeysTTL
is set.
Introduces the capability to specify a custom Lucene analyzer used by text index for indexing and search on an individual column basis.
Sample usage
Default Behavior falls back to using the standardAnalyzer
unless the luceneAnalyzerClass
property is specified.
Murmur3 support with optional fields seed
and variant
for the hash in functionConfig
field of columnPartitionMap.
Default value for seed
is 0.
Added support for 2 variants of Murmur3
: x86_32
and x64_32
configurable using the variant
field in functionConfig
. If no variant is provided we choose to keep the x86_32
variant as it was part of the original implementation.
Examples of functionConfig
;
Here there is no functionConfig configured, so the seed
value will be 0
and variant will be x86_32
.
Here the seed
is configured as 9001
but as no variant is provided, x86_32
will be picked up.
Here the variant
is mentioned so Murmur3 will use the x64_32
variant with 9001
as seed.
Note on users using Debezium
and Murmur3
as partitioning function :
The partitioning key should be set up on either of byte[]
, String
or long[]
columns.
On pinot variant
should be set as x64_32
and seed
should be set as 9001
.
Adds new MV dictionary encoded forward index format that only stores the unique MV entries.
This new index format can significantly reduce the index size when the MV entries repeat a lot
The new index format can be enabled during index creation, derived column creation, and segment reload
To enable the new index format, set the compression codec in the FieldConfig
:
Or use the new index JSON:
Adds support for 2 possible ways to handle null:
Table mode - which already exists
Column mode, which means that each column specifies its own nullability in the FieldSpec
Column mode can be enabled by the below config.
The default value for enableColumnBasedNullHandling
is false. When set to true, Pinot will ignore TableConfig.IndexingConfig.nullHandlingEnabled
and columns will be nullable if and only if FieldSpec.notNull
is false, which is also the default value.
Adds a new upsert config outOfOrderRecordColumn
When set to a non-null value, we check whether an event is OOO
or not and then accordingly update the corresponding column value to true / false.
This will help in tracking which event is out-of-order while using skipUpsert
Can be used to save space. For eg: when a functionColumnPairs
has a output type of bytes, such as when you use distinctcountrawhll.
Sample config
Supports instance assignment based pre-configured instance assignment map.
The assignment will always respect the mirrored servers in the pre-configured map
More details here
Sample table config
Adds dimension
as a valid option to table "type" in the /tables controller API
This patch adds a new config for upsert: dropOutOfOrderRecord
If set to true, pinot doesn't persist out-of-order events in the segment.
This feature is useful to
Save disk-usage
Avoid any confusion when using skipUpsert
for partial-upsert tables as nulls start showing up for columns where a previous non-null was encountered and we don't know if it's an out-of-order event or not.
New configs for the RebalanceChecker
periodic task:
controller.rebalance.checker.frequencyPeriod
: 5min by default ; -1 to disable
controller.rebalanceChecker.initialDelayInSeconds
: 2min+ by default
New configs added for RebalanceConfig
:
heartbeatIntervalInMs
: 300_000 i.e. 5min
heartbeatTimeoutInMs
: 3600_000 i.e. 1hr
maxAttempts
: 3 by default, i.e. the original run plus two retries
retryInitialDelayInMs
: 300_000 i.e. 5min, for exponential backoff w/ jitters
New metrics to monitor rebalance and its retries:
TABLE_REBALANCE_FAILURE("TableRebalanceFailure", false), emit from TableRebalancer.rebalanceTable()
TABLE_REBALANCE_EXECUTION_TIME_MS("tableRebalanceExecutionTimeMs", false), emit from TableRebalancer.rebalanceTable()
TABLE_REBALANCE_FAILURE_DETECTED("TableRebalanceFailureDetected", false), emit from RebalanceChecker
TABLE_REBALANCE_RETRY("TableRebalanceRetry", false), emit from RebalanceChecker
New restful API
DELETE /tables/{tableName}/rebalance
API to stop rebalance. In comparison, POST /tables/{tableName}/rebalance
was used to start one.
UltraLogLog
(#11835)UltraLogLog aggregations for Count Distinct (distinctCountULL
and distinctCountRawULL
)
UltraLogLog creation via Transform Function
UltraLogLog merging in MergeRollup
Support for UltraLogLog in Star-Tree indexes
Ingestion via transformation function
Extracting estimates via query aggregation functions
Segment rollup aggregation
StarTree aggregation
Broadly there are two configs that will enable this feature:
maxServerResponseSizeBytes: Maximum serialized response size across all servers for a query. This value is equally divided across all servers processing the query.
maxQueryResponseSizeBytes: Maximum length of the serialized response per server for a query
Configs are available as queryOption, tableConfig and Broker config. The priority of enforcement is as follows:
This is helpful when user has the entire JSON handy
UI still keeps Form Way to add Schema along with JSON view
Use option maxValueLength
in jsonIndexConfig
to restrict length of values
A value of 0 (or when the key is omitted) means there is no restriction
Supports serializing and writing MV columns in VarByteChunkForwardIndexWriterV4
Supports V4 reader that can be used to read SV var length, MV fixed length and MV var length buffers encoded with V4 writer
Approximation aggregation functions for estimating the frequencies of items a dataset in a memory efficient way. More details in Apache Datasketches library.
Table index api to get the aggregate index details of all segments for a table.
URL/tables/{tableName}/indexes
Response format
The lead controller rebalance delay is now configurable with controller.resource.rebalance.delay_ms
Changing rebalance configurations will now update the lead controller resource
Adds support for Pinot configuration through ENV variables with Dynamic mapping.
More details in issue: #10651
Sample configs through ENV
HLL++ has higher accuracy than HLL when cardinality of dimension is at 10k-100k.
More details here
Adds query rewriting logic to transform a "virtual" UDF, clpMatch
, into a boolean expression on the columns of a CLP-encoded field.
To use the rewriter, modify broker config to add org.apache.pinot.sql.parsers.rewriter.ClpRewriter
to pinot.broker.query.rewriter.class.names
.
DATETIMECONVERTWINDOWHOP
function (#11773)JSON_EXTRACT_INDEX
transform function to leverage json index for json value extraction (#11739)GenerateData
command support for generating data in JSON format (#11778)Support ARRAY function as a literal evaluation (#12278)
Support for ARRAY literal transform functions (#12118)
Theta Sketch Aggregation enhancements (#12042)
Adds configuration options for DistinctCountThetaSketchAggregationFunction
Respects ordering for existing Theta sketches to use "early-stop" optimisations for unions
Add query option override for Broker MinGroupTrimSize (#11984)
Support for 2 new scalar functions for bytes: toUUIDBytes
and fromUUIDBytes
(#11988)
Config option to make groupBy trim size configurable at Broker (#11958)
Pre-aggregation support for distinct count hll++ (#11747)
Add float type into literal thrift to preserve literal type conforming to SQL standards (#11697)
Enhancement to add query function override for Aggregate functions of multi valued columns (#11307)
Perf optimization in IN clause evaluation (#11557)
Add TextMatchFilterOptimizer to maximally push down text_match filters to Lucene (#12339
Async rendering of UI elements to load UI elements async resulting in faster page loads (#12210)
Make the table name link clickable in task details (#12253)
Swagger UI enhancements to resumeConsumption API call (#12200)
Adds support for CTRL key as a modifier for Query shortcuts (#12087)
UI enhancement to show partial index in reload (#11913)
UI improvement to add Links to Instance in Table and Segment View (#11807)
Fixes reload to use the right indexes API instead of fetching all segment metadata (#11793)
Enhancement to add toggle to hide/show query exceptions (#11611)
Enhancement to reduce the heap usage of String Dictionaries that are loaded on-heap (#12223)
Wire soft upsert delete for Compaction task (12330)
Upsert compaction debuggability APIs for validDocId metadata (#12275)
Make server resource classes configurable (#12324)
Shared aggregations for Startree index - mapping from aggregation used in the query to aggregation used to store pre-aggregated values (#12164)
Increased fetch timeout for Kineses to prevent stuck kinesis consumers
Metric to track table rebalance (#12270)
Allow server-level configs for upsert metadata (#18851)
Support to dynamically initialize Kafka client SSL configs (#12249)
Optimize segment metadata file creation without having to download full segment (#12255)
Allow string / numeric data type for deleteRecordColumn config (#12222)
Observability enhancement to add column name when JSON index building fails (#12151)
Creation of DateTimeGenerator
for DATE_TIME
field type columns (#12206)
Add singleton registry for all controller and minion metrics (#12119)
Support helm chart server separate liveness and readiness probe endpoints (#11800)
Observability enhancement to add metrics for Table Disabled and Consumption Paused (#12000)
Support for SegmentGenerationAndPushTask
to push segment to realtime table (#12084)
Enhancement to make the deep store upload retry async with configurable parallelism (#12017)
Optimizations in segment commit to not read partition group metadata (#11943)
Replace timer with scheduled executor service in IngestionDelayTracker to reduce number of threads (#11849)
Adds an option skipControllerCertValidation
to skip controller cert validation in AddTableCommand (#11967)
Adds instrumentation for DataTable Creation (#11942)
Improve performance of ZkBasicAuthAccessFactory by caching Bcrypt password (#11904)
Adds support to to fetch metadata for specific list of segments (#11949)
Allow user specify local temp directory for quickstart (#11961)
Optimization for server to directly return final result for queries hitting single server (#11938)
Explain plan optimization to early release AcquireReleaseColumnsSegmentOperator (#11945)
Observability metric to track query timeouts (#11892)
Add support for auth in QueryRunner (#11897)
Allow users to pass custom RecordTransformers to SegmentProcessorFramework (#11887)
Add isPartialResult flag to broker response (#11592)
Add new configs to Google Cloud Storage (GCS) connector: jsonKey
(#11890)
jsonKey
is the GCP credential key in string format (either in plain string or base64 encoded string). Refer Creating and managing service account keys to download the keys.
Performance enhancement to build segments in column orientation (#11776)
Disabled by default. Can be enabled by setting table config columnMajorSegmentBuilderEnabled
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)
Observability improvement to expose GRPC metrics (#11842)
Improvements to response format for reload API to be pretty printed (#11608)
Enhancements to support Java 21 (#11672)
Add more information in RequestContext class (#11708)
Support to read exact buffer byte ranges corresponding to a given forward index doc id (#11729)
Enhance Broker reducer to handle expression format change (#11762)
Capture build scans on ge.apache.org to benefit from deep build insights (#11767)
Performance enhancement in multiple places by updating initial capacity of HashMap (#11709)
Support for building indexes post segment file creation, allowing indexes that may depend on a completed segment to be built as part of the segment creation process (#11711)
Support excluding time values in SimpleSegmentNameGenerator (#11650)
Perf enhancement to reduce cpu usage by avoiding throwing an exception during query execution (#11715)
Added framework for supporting nulls in ScalarTransformFunctionWrapper in the future (#11653)
Observability change to metrics to export netty direct memory used and max (#11575)
Observability change to add a metric to measure total thread cpu time for a table (#11713)
Observability change to use SlidingTimeWindowArrayReservoir
in dropwizard metrics (#11695)
Minor improvements to upsert preload (#11694)
Observability changes to expose additional Realtime Ingestion Metrics (#11685)
Perf enhancement to remove the global lock in SegmentCompletionManager (#11679)
Enhancements to unify tmp file naming format and delete tmp files at a regular cadence by extending the ControllerPeriodicTask (#10815)
controller.realtime.segment.tmpFileAsyncDeletionEnabled
(default false
)
controller.realtime.segment.tmpFileRetentionInSeconds
(default 3600
)
Enhancements to allow override/force options when add schema (#11572)
Enhancement to handle direct memory OOM on brokers (#11496)
Enhancement to metadata API to return upsert partition to primary key count map for both controller and server APIs (#12334)
Enhancements to peer server segment download by retrying both peer discovery and download. (#12317)
Helper functions in StarTreeBuilderUtils and StarTreeV2BuilderConfig (#12361)
Perf optimizations to release all segments of a table in releaseAndRemoveAllSegments method (#12297)
Enhancement to Maintain pool selection for the minimizeDataMovement
instance partition assignment strategy (#11953)
Upsert enhancement to assign segments for with respect to ideal state (#11628)
Observability change to export Additional Upsert Metrics to Prom (#11660)
Observibility enhancement to add CPU metrics for minion purge task (#12337)
Add HttpHeaders in broker event listener requestContext (#12258)
Upsert bugfix in "rewind()" for CompactedPinotSegmentRecordReader (#12329)
Fix error message format for Preconditions.checks failures(#12327)
Fixes in upsert metadata manager (#12319)
Security fix to allow querying tables with table-type suffix (#12310)
Increased fetch timeout for Kineses to prevent stuck kinesis consumers(#12214)
Fixes lucene index errors when using QuickStart (#12289)
Null handling bugfix for sketch group-by queries (#12259)
Null pointer exception fixes in Controller SQL resource (#12211)
Bugfix for S3 connection pool error when AWS session tokens expire after an hour (#12221)
FileWriter fixes to append headerline only for required formats like csv (#12208)
Security bugfix for pulsar OAuth2 authentication (#12195)
Bugfix to appropriately compute "segment.flush.threshold.size" when force-committing realtime segments (#12188)
Fixes rebalance converge check that reports success before rebalance completes (#12182)
Fixes upsertPrimaryKeysCount
metric reporting when table is deleted (#12169)
Update LICENSE-binary for commons-configuration2 upgrade (#12165)
Improve error logging when preloading segments not exist on server (#12153)
Fixes to file access resource leaks (#12129)
Ingestion bugfix to avoid unnecessary transformers in CompositeTransformer (#12138)
Improve logging to print OS name during service statup (#12135)
Test fixes for ExprMinMaxRewriterTest.testQueryRewrite (#12047)
Fix default brokerUpdateFrequencyInMillis for connector (#12093)
Updates to README file (#12075)
Fix to remove unnecessary locking during segment preloading (#12077)
Fix bug with silently ignoring force commit call failures (#12044)
Upsert bugfix to allow optional segments that can be skipped by servers without failing the query (#11978)
Fix incorrect handling of consumer creation errors (#12045)
Fix the memory leak issue on CommonsConfigurationUtils
(#12056)
Fix rebalance on upsert table (#12054)
Add new Transformer to transform -0.0 and NaN (#12032)
Improve inverted index validation in table config to enhance user experience (#12043)
Fixes test flakiness by replacing HashSet/HashMap with LinkedHashSet/LinkedHashMap (#11941)
Flaky test fix for ServerRoutingStatsManagerTest.testQuerySubmitAndCompletionStats
(#12029)
Fix derived column from MV column (#12028)
Support for leveraging StarTree index in conjunction with filtered aggregations (#11886)
Improves tableConfig validation for enabling size based threshold for realtime tables (#12016)
Fix flaky PinotTenantRestletResourceTest (#12026)
Fix flaky Fix PinotTenantRestletResourceTest (#12019)
Fix the race condition of concurrent modification to segment data managers (#12004)
Fix the misuse of star-tree when all predicates are always false under OR (#12003)
Fix the test failures caused by instance drop failure (#12002)
Fix fromULL scalar function (#11995)
Fix to exclude module-info.class during shade operations (#11975)
Fix the wrong import for Preconditions (#11979)
Add check for illegal character '/' in taskName (#11955)
Bugfix to only register new segments when it's fully initalized by partitionUpsertMetadataManager
(#11964)
Obervability fix to add logs to track sequence of events for table creation (#11946)
Fix the NPE in minimizeDataMovement instance assignment strategy (#11952)
Fix to add catch all logging for exception during DQL/DML process (#11944)
Fix bug where we don't handle cases that a upsert table has both upsert deletion and upsert ttl configs (#11791)
Removing direct dependencies on commons-logging and replacing with jcl-over-slf4j (#11920)
Fix NPE for IN clause on constant STRING dictionary (#11930)
Fix flaky OfflineClusterIntegrationTest on server response size tests (#11926)
Avoid npe when checking mirror server set assignment (#11915)
Deprecate _segmentAssignmentStrategy
in favor of SegmentsValidationAndRetentionConfig
#11869
Bugfix to capture auth phase timing even if access is denied (#11884)
Bugfix to mark rows as invalid in case primary time column is out of range (#11907)
Fix to radomize server port to avoid port already bind issue (#11861)
Add LazyRow abstraction for previously indexed record (#11826)
Config Validation for upsert table to not assign COMPLETED segments to another server (#11852)
Bugfix to resolve dependency conflict in pinot-protobuf module (#11867)
Fix case of useMultistageEngine
property reference in JsonAsyncHttpPinotClientTransportFactory
(#11820)
Bugfix to add woodstox-core to pinot-s3 dependencies and fix stack trace (#11799)
Fix to move pinot-segment-local test from unit test suite 1 to 2 (#11865)
Observability fix to log upsert config when initializing the metadata manager (#11864)
Fix to improve tests when errors are received in the consumer thread (#11858)
Fix for flaky ArrayAgg test (#11860)
Fix for flaky tests in TupleSelectionTransformFunctionsTest (#11848)
Fix for arrayAgg null support (#11853)
Fix the bug of reading decimal value stored in int32 or int64 (#11840)
Remove duplicate pinot-integration-tests from unit test suite 2 (#11844)
Fix for a null handling error in queries (#11829)
Fix the way of fetching the segment zk metadata for task generators (#11832)
Make testInvalidateCachedControllerLeader times based on getMinInvalidateIntervalMs (#11815)
Update doap to reflect latest release (#11827)
Clean up integration test pom file (#11817)
Bugfix to exclude OFFLINE segments when reading server to segments map (#11818)
Add tests for zstd compressed parquet files (#11808)
Fix job submission time for reload and foce commit job (#11803)
Remove actually unsupported config that selectively enable nullable columns (#10653)
Fix LLCRealtimeClusterIntegrationTest.testReset (#11806)
Use expected version in api for table config read modify write change (#11782)
Move jobId out of rebalanceConfig (#11790)
Fix PeerServerSegmentFinder not respecting HTTPS port (#11752)
Enhanced geospatial v2 integration tests (#11741)
Add integration test for rebalance in upsert tables (#11568)
Fix trivy CI issue (#11757)
Cleanup rebalance configs by adding a RebalanceConfig class (#11730)
Fix a protobuf comment to be more precise (#11735)
Move scala dependencies to root pom (#11671)
Fix ProtoBuf inputformat plug-in handling for null values (#11723)
Bugfix where segment download URI is invalid after same CRC refresh using tar push (#11720)
Fix in TableCacheTest (#11717)
Add more test for broker jersey bounded thread pool (#11705)
Fix bug in gapfill with SumAvgGapfillProcessor. (#11714)
Bugfix to allow GcsPinotFS to work with granular permissions (#11655)
Fix default log4j2 config file path in helm chart (#11707)
Refactor code and doc occurrences of argmin/max -> exprmin/max (#11700)
Make constructor and functions public to be used from scheduler plugins (#11699)
Bugfix to change json_format to return java null when java null is received (#11673)
Fix the potential access to upsert metadata manager after it is closed (#11692)
Bugfix to use isOptional
instead of the deprecated hasOptional
Keyword (#11682)
Fix logging issue in RealtimeTableDataManager (#11693)
Cleanup some reader/writer logic for raw forward index (#11669)
Do not execute spotless in Java 21 (#11670)
Update license-maven-plugin (#11665)
Bugfix to allow deletion of local files with special characters (#11664)
Clean up CaseTransformFunction::constructStatementListLegacy. (#11339)
Bugfix to force FileChannel to commit data to disk (#11625)
Remove the old deprecated commit end without metadata (#11662)
Fix for a jackson vulnerability (#11619)
Refactor BasicAuthUtils from pinot-core to pinot-common and remove pinot-core dependency from pinot-jdbc-client (#11620)
Bugfix to support several extensions for different indexes (#11600)
Fix the alias handling in single-stage engine (#11610)
Fix to use constant null place holder (#11615)
Refactor to move all BlockValSet into the same package (#11616)
Remove deprecated Request class from pinot-java-client (#11614)
Refactoring to remove old thirdeye files. (#11609)
Testing fix to use builder method in integration test (#11564)
Fix the broken Pinot JDBC client. (#11606)
Bugfix to change the Forbidden error to Unauthorized (#11501)
Fix for schema add UI issue that passing wrong data in the request header (#11602)
Remove/Deprecate HLC handling code (#11590)
Fix the bug of using push time to identify new created segment (#11599)
Bugfix in CSVRecordReader when using line iterator (#11581)
Remove split commit and some deprecated config for real-time protocol on controller (#11663)Improved validation for single argument aggregation functions (#11556)
Fix to not emit lag once tabledatamanager shutdown (#11534)
Bugfix to fail reload if derived columns can't be created (#11559)
Fix the double unescape of property value (#12405)
Fix for the backward compatible issue that existing metadata may contain unescaped characters (#12393)
Skip invalid json string rather than throwing error during json indexing (#12238)
Fixing the multiple files concurrent write issue when reloading SSLFactory (#12384)
Fix memory leaking issue by making thread local variable static (#12242)
Bugfixfor Upsert compaction task generator (#12380)
Log information about SSLFactory renewal (#12357)
Fixing array literal usage for vector (#12365)
Fixing quickstart table baseballStats minion ingestion (#12371)
Fix backward compatible issue in DistinctCountThetaSketchAggregationFunction (#12347)
Bugfix to skip instead of throwing error on 'getValidDocIdMetadata' (#12360)
Fix to clean up segment metadata when the associated segment gets deleted from remote store (#12350)
Fix getBigDecimal() scale throwing rounding error (#12326)
Workaround fix for the problem of Helix sending 2 transitions for CONSUMING -> DROPPED (#12351)
Bugfix for making nonLeaderForTables exhaustive (#12345)
Remove split commit and some deprecated config for real-time protocol on controller (#11663)
Update the table config in quick start (#11652)
Deprecate k8s skaffold scripts and move helm to project root directory (#11648)
Fix NPE in SingleColumnKeySelector (#11644)
Simplify kafka build and remove old kafka 0.9 files (#11638)
Adding comments for docker image tags, make a hyper link of helmChart from root directory (#11646)
Improve the error response on controller. (#11624)
Simplify authrozation for table config get (#11640)
Bugfix to remove segments with empty download url in UpsertCompactionTask (#12320)
Test changes to make taskManager resources protected for derived classes to override in their setUp() method. (#12335)
Fix a race condition for upsert compaction (#12346). Notes on backward incompatibility below:
This PR is introducing backward incompatibility for UpsertCompactionTask. Previously, we allowed to configure the compaction task without the snapshot enabled. We found that using in-memory based validDocIds is a bit dangerous as it will not give us the consistency (e.g. fetching validDocIds bitmap while the server is restarting & updating validDocIds).
We now enforce the enableSnapshot=true
for UpsertCompactionTask if the advanced customer wants to run the compaction with the in-memory validDocId bitmap.
Also, we allow to configure invalidDocIdsType
to UpsertCompactionTask for advanced user.
snapshot
: Default validDocIds type. This indicates that the validDocIds bitmap is loaded from the snapshot from the Pinot segment. UpsertConfig's enableSnapshot
must be enabled for this type.
onHeap
: the validDocIds bitmap will be fetched from the server.
onHeapWithDelete
: the validDocIds bitmap will be fetched from the server. This will also take account into the deleted documents. UpsertConfig's deleteRecordColumn
must be provided for this type.
Removal of the feature flag allow.table.name.with.database
(#12402)
Error handling to throw exception when schema name doesn't match table name during table creation (#11591)
Incompatible API fix to remove table state update operation in GET call (#11621)
Use string to represent BigDecimal datatype in JSON response (#11716)
Single quoted literal will not have its type auto-derived to maintain SQL compatibility (#11763)
Change to not allow NaN as default value for Float and Double in Schemas (#11661)
Code cleanup and refactor that removes TableDataManagerConfig
(#12189)
Fix partition handling for consistency of values between query and segment (#12115)
Changes for migration to commons-configuration2 (#11985)
Cleanup to simplify the upsert metadata manager constructor (#12120)
Fixes typo in pom.xml (#11997)
JDBC Driver fixes to support Jetbrains Intellij/Datagrip database tooling (#11814)
Fix regression in ForwardIndexType for noDictionaryConfig and noDictionaryColumns (#11784)
Separate pr test scripts and codecov (#11804)
Bugfix to make reload status should only count online/consuming segments (#11787)
Fix flaky TableViewsTest (#11770)
Fix a flaky test (#11771)
Cleanup to fee more disk for trivy job (#11780)
Fix schema name in table config during controller startup (#11574)
Prevent NPE when attempt to fetch partition information fails (#11769)
Added UTs for null handling in CaseTransform
function. (#11721)
Bugfix to disallow peer download when replication is < 2 (#11469)
Updates to Docker image and GitHub action scripts (#12378)
Enhancements to queries test framework (#12215)
Update maven-jar-plugin and maven-enforcer-plugin version (#11637)
Update testng as the test provider explicitly instead of relying on the classpath. (#11612)
Update compatibility verifier version (#11684)
Upgrade Avro dependency to 1.10.2 (#11698)
Upgrade testng version to 7.8.0 (#11462)
Update lombok version and config (#11742)
Upgrading Apache Helix to 1.3.1 version (#11754)
Upgrade spark from 3.2 to 3.5 (#11702)
Added commons-configuration2 dependency. (#11792)
Upgrade confluent libraries to 7.2.6 to fix some errors related to optional proto fields (#11753)
Upgrade lucene to 9.8.0 and upgrade text index version (#11857)
Upgrade the PinotConfiguartion
to commons-configuartion2
(#11916)
Pre PinotConfig commons-configuartions2 upgrade (#11868)
Bump commons-codec:commons-codec from 1.15 to 1.16.0 (#12204)
Bump flink.version from 1.12.0 to 1.14.6 (#12202)
Bump com.yscope.clp:clp-ffi from 0.4.3 to 0.4.4 (#12203)
Bump org.apache.spark:spark-launcher_2.12 from 3.2.1 to 3.5.0 (#12199)
Bump io.grpc:grpc-context from 1.59.0 to 1.60.1 (#12198)
Bump com.azure:azure-core from 1.37.0 to 1.45.1 (#12193)
Bump org.freemarker:freemarker from 2.3.30 to 2.3.32 (#12192)
Bump com.google.auto.service:auto-service from 1.0.1 to 1.1.1 (#12183)
Bump dropwizard-metrics.version from 4.2.22 to 4.2.23 (#12178)
Bump org.apache.yetus:audience-annotations from 0.13.0 to 0.15.0 (#12170)
Bump com.gradle:common-custom-user-data-maven-extension (#12171)
Bump org.apache.httpcomponents:httpclient from 4.5.13 to 4.5.14 (#12172)
Bump org.glassfish.tyrus.bundles:tyrus-standalone-client (#12162)
Bump com.google.api.grpc:proto-google-common-protos (#12159)
Bump org.apache.datasketches:datasketches-java from 4.1.0 to 5.0.0 (#12161)
Bump org.apache.zookeeper:zookeeper from 3.6.3 to 3.7.2 (#12152)
Bump org.apache.commons:commons-collections4 from 4.1 to 4.4 (#12149)
Bump log4j.version from 2.20.0 to 2.22.0 (#12143)
Bump com.github.luben:zstd-jni from 1.5.5-6 to 1.5.5-11 (#12125)
Bump com.google.guava:guava from 32.0.1-jre to 32.1.3-jre (#12124)
Bump org.apache.avro:avro from 1.10.2 to 1.11.3 (#12116)
Bump org.apache.maven.plugins:maven-assembly-plugin from 3.1.1 to 3.6.0 (#12109)
Bump net.java.dev.javacc:javacc from 7.0.10 to 7.0.13 (#12103)
Bump com.azure:azure-identity from 1.8.1 to 1.11.1 (#12095)
Bump xml-apis:xml-apis from 1.4.01 to 2.0.2 (#12082)
Bump up the parquet version to 1.13.1 (#12076)
Bump io.grpc:grpc-context from 1.14.0 to 1.59.0 (#12034)
Bump org.reactivestreams:reactive-streams from 1.0.3 to 1.0.4 (#12033)
Bump org.codehaus.mojo:appassembler-maven-plugin from 1.10 to 2.1.0 (#12030)
Bump com.google.code.findbugs:jsr305 from 3.0.0 to 3.0.2 (#12031)
Bump org.jacoco:jacoco-maven-plugin from 0.8.9 to 0.8.11 (#12024)
Bump dropwizard-metrics.version from 4.2.2 to 4.2.22 (#12022)
Bump grpc.version from 1.53.0 to 1.59.0 (#12023)
Bump com.google.code.gson:gson from 2.2.4 to 2.10.1 (#12009)
Bump net.nicoulaj.maven.plugins:checksum-maven-plugin from 1.8 to 1.11 (#12008)
Bump circe.version from 0.14.2 to 0.14.6 (#12006)
Bump com.mercateo:test-clock from 1.0.2 to 1.0.4 (#12005)
Bump simpleclient_common.version from 0.8.1 to 0.16.0 (#11986)
Bump com.jayway.jsonpath:json-path from 2.7.0 to 2.8.0 (#11987)
Bump commons-net:commons-net from 3.1 to 3.10.0 (#11982)
Bump org.scalatest:scalatest-maven-plugin from 1.0 to 2.2.0 (#11973)
Bump io.netty:netty-bom from 4.1.94.Final to 4.1.100.Final (#11972)
Bump com.google.errorprone:error_prone_annotations from 2.3.4 to 2.23.0 (#11905)
Bump net.minidev:json-smart from 2.4.10 to 2.5.0 (#11875)
Bump org.yaml:snakeyaml from 2.0 to 2.2 (#11876)
Bump browserify-sign in /pinot-controller/src/main/resources (#11896)
Bump org.easymock:easymock from 4.2 to 5.2.0 (#11854)
Bump org.codehaus.mojo:exec-maven-plugin from 1.5.0 to 3.1.0 (#11856)
Bump com.github.luben:zstd-jni from 1.5.2-3 to 1.5.5-6 (#11855)
Bump aws.sdk.version from 2.20.94 to 2.20.137 (#11463)
Bump org.xerial.snappy:snappy-java from 1.1.10.1 to 1.1.10.4 (#11678)