Commit Graph

1450 Commits

Author SHA1 Message Date
Kashif Faraz 4df4896674
Refactor: Add common method in AbstractBatchIndexTask to create ingestion stats report (#16202)
Changes
-  No functional changes
- Add method `AbstractBatchIndexTask.buildIngestionStatsReport()` used in several batch tasks
- Add utility method `AbstractBatchIndexTask.addBuildSegmentStatsToReport()`
- Use boolean argument to represent a full report instead of the String `full` 
in internal methods. (REST API remains unchanged.)
- Rename `IngestionStatsAndErrorsTaskReportData` to `IngestionStatsAndErrors`
- Clean up some of the methods
2024-03-28 23:07:00 +05:30
Soumyava 524842a3bb
Window function on msq (#15470)
This PR aims to introduce Window functions on MSQ by doing the following:

    Introduce a Window querykit for handling window queries along with its factory and a processor for window queries
    If a window operator is present with a partition by clause, pushes the partition as a shuffle spec of the previous stage
    In presence of empty OVER() clause lets all operators loose on a single rac
    In presence of no empty OVER() clause, breaks down each window into individual stages
    Associated machinery to handle window functions in MSQ
    Introduced a separate hidden engine feature WINDOW_LEAF_OPERATOR which is set only for MSQ engine. In presence of this feature, the planner plans without the leaf operators by creating a window query over an inner scan query. In case of native this is set to false and the planner generates the leafOperators
    Guardrails around materialization
    Comprehensive UTs
2024-03-28 14:58:34 +05:30
Gian Merlino 7649957710
MSQ: Fix issue where AUTO assignment would not respect maxWorkerCount. (#16214)
WorkerAssignmentStrategy.AUTO was missing a check for maxWorkerCount
in the case where the inputs to a stage are not dynamically sliceable.
A common case here is when the inputs to a stage are other stages.
2024-03-28 14:40:31 +05:30
zachjsh 8370db106c
INSERT/REPLACE dimension target column types are validated against source input expressions (#15962)
* * address remaining comments from https://github.com/apache/druid/pull/15836

* *  address remaining comments from https://github.com/apache/druid/pull/15908

* * add test that exposes relational algebra issue

* * simplify test exposing issue

* * fix

* * add tests for sealed / non-sealed

* * update test descriptions

* * fix test failure when -Ddruid.generic.useDefaultValueForNull=true

* * check type assignment based on natice Druid types

* * add tests that cover missing jacoco coverage

* * add replace tests

* * add more tests and comments about column ordering

* * simplify tests

* * review comments

* * remove commented line

* * STRING family types should be validated as non-null
2024-03-25 12:34:07 -04:00
Aru Raghuwanshi 6e19ce5e69
Handle null values in `KafkaStringHeaderReader` (#16192) 2024-03-23 13:05:55 +05:30
Gian Merlino 2b23d0b5b5
MSQ: Controller checker should check for "closed" only. (#16161)
* MSQ: Controller checker should check for "closed" only.

Currently, the worker's controller checker will exit the worker if
the controller location is "closed" (no longer running) or if its location
is empty (i.e. location unknown).

This patch changes to only exit on "closed". We shouldn't exit on empty
location, because that may happen if the Overlord is slow to acknowledge the
location of a task.

* Fix test.
2024-03-19 19:25:48 -07:00
Gian Merlino c96b215dd6
SortMerge join support for IS NOT DISTINCT FROM. (#16003)
* SortMerge join support for IS NOT DISTINCT FROM.

The patch adds a "requiredNonNullKeyParts" field to the sortMerge
processor, which has the list of key parts that must be nonnull for
an equijoin condition to match. Conditions with SQL "=" are present in
the list; conditions with SQL "IS NOT DISTINCT FROM" are absent from
the list.

* Fix test.

* Update javadoc.
2024-03-19 12:02:13 -07:00
Zoltan Haindrich 1ad489a2ae
Fix build: newTempFolder (#16170) 2024-03-19 08:53:56 -07:00
Zoltan Haindrich 0a42342cef
Update Calcite*Test to use junit5 (#16106)
* Update Calcite*Test to use junit5

* change the way temp dirs are handled
* add openrewrite workflow to safeguard upgrade
* replace junitparamrunner with standard junit5 parametered tests
* update a few rules to junit5 api
* lots of boring changes

* cleanup QueryLogHook

* cleanup

* fix compile error: ARRAYS_DATASOURCE

* fix test

* remove enclosed

* empty

+TEST:TDigestSketchSqlAggregatorTest,HllSketchSqlAggregatorTest,DoublesSketchSqlAggregatorTest,ThetaSketchSqlAggregatorTest,ArrayOfDoublesSketchSqlAggregatorTest,BloomFilterSqlAggregatorTest,BloomDimFilterSqlTest,CatalogIngestionTest,CatalogQueryTest,FixedBucketsHistogramQuantileSqlAggregatorTest,QuantileSqlAggregatorTest,MSQArraysTest,MSQDataSketchesTest,MSQExportTest,MSQFaultsTest,MSQInsertTest,MSQLoadedSegmentTests,MSQParseExceptionsTest,MSQReplaceTest,MSQSelectTest,InsertLockPreemptedFaultTest,MSQWarningsTest,SqlMSQStatementResourcePostTest,SqlStatementResourceTest,CalciteSelectJoinQueryMSQTest,CalciteSelectQueryMSQTest,CalciteUnionQueryMSQTest,MSQTestBase,VarianceSqlAggregatorTest,SleepSqlTest,SqlRowTransformerTest,DruidAvaticaHandlerTest,DruidStatementTest,BaseCalciteQueryTest,CalciteArraysQueryTest,CalciteCorrelatedQueryTest,CalciteExplainQueryTest,CalciteExportTest,CalciteIngestionDmlTest,CalciteInsertDmlTest,CalciteJoinQueryTest,CalciteLookupFunctionQueryTest,CalciteMultiValueStringQueryTest,CalciteNestedDataQueryTest,CalciteParameterQueryTest,CalciteQueryTest,CalciteReplaceDmlTest,CalciteScanSignatureTest,CalciteSelectQueryTest,CalciteSimpleQueryTest,CalciteSubqueryTest,CalciteSysQueryTest,CalciteTableAppendTest,CalciteTimeBoundaryQueryTest,CalciteUnionQueryTest,CalciteWindowQueryTest,DecoupledPlanningCalciteJoinQueryTest,DecoupledPlanningCalciteQueryTest,DecoupledPlanningCalciteUnionQueryTest,DrillWindowQueryTest,DruidPlannerResourceAnalyzeTest,IngestTableFunctionTest,QueryTestRunner,SqlTestFrameworkConfig,SqlAggregationModuleTest,ExpressionsTest,GreatestExpressionTest,IPv4AddressMatchExpressionTest,IPv4AddressParseExpressionTest,IPv4AddressStringifyExpressionTest,LeastExpressionTest,TimeFormatOperatorConversionTest,CombineAndSimplifyBoundsTest,FiltrationTest,SqlQueryTest,CalcitePlannerModuleTest,CalcitesTest,DruidCalciteSchemaModuleTest,DruidSchemaNoDataInitTest,InformationSchemaTest,NamedDruidSchemaTest,NamedLookupSchemaTest,NamedSystemSchemaTest,RootSchemaProviderTest,SystemSchemaTest,CalciteTestBase,SqlResourceTest

* use @Nested

* add rule to remove enclosed; upgrade surefire

* remove enclosed

* cleanup

* add comment about surefire exclude
2024-03-19 04:05:12 -07:00
Adarsh Sanjeev a151bcfd12
Fix incorrect header names for certain export queries (#16096)
* Fix incorrect header names for certain queries

* Fix incorrect header names for certain queries

* Maintain upgrade compatibility

* Fix tests

* Change null handling
2024-03-19 15:11:04 +05:30
Gian Merlino 55c47fbcfd
MSQ: Fix NPE in getWorkerStats(). (#16159)
TaskTracker's status is null when TaskTrackers are first set up, and
stay null until the first status call comes back. This patch handles
that case and sets the status code to null in the WorkerStats object
in live reports.
2024-03-19 14:22:49 +05:30
Gian Merlino 8ee324c7e7
MSQ: Cancel workers more quickly. (#16158)
Prior to this patch, when canceled, workers would keep trying to contact
the controller: they would attempt to report an error, and if they were
in the midst of some other call (like a counters push) they would keep
trying it.

This can cause cancellation to be delayed, because the controller shuts
down its HTTP server before it cancels workers. Workers are then stuck
retrying calls to the controller that will never succeed. The retry loops
are broken when the controller gives up on them (one minute later) and
exits for real. Then, the controller failure detection logic on the worker
detects that the controller has failed, and the worker finally shuts down.

This patch speeds up worker cancellation by bypassing communication
with the controller. There is no real need for it. If the controller
canceled the workers, it isn't interested in further communications from
them. If the workers were canceled out-of-band, the controller can
detect this through worker monitoring and report it as a WorkerFailed
error.
2024-03-19 14:21:22 +05:30
Gian Merlino 36bc94c798
MSQ: Remove unnecessary snapshot deserialization code. (#16116)
Since #13205, a special deserializer module has no longer been necessary
to read key collector snapshots. This patch removes the unnecessary code.
2024-03-18 10:12:27 -07:00
Kashif Faraz 466057c61b
Remove deprecated DruidException, EntryExistsException (#14448)
Changes:
- Remove deprecated `DruidException` (old one) and `EntryExistsException`
- Use newly added comprehensive `DruidException` instead
- Update error message in `SqlMetadataStorageActionHandler` when max packet limit is violated.
- Factor out common code from several faults into `BaseFault`.
- Slightly update javadoc in `DruidException` to render it correctly
- Remove unused classes `SegmentToMove`, `SegmentToDrop`
- Move `ServletResourceUtils` from module `druid-processing` to `druid-server`
- Add utility method to build error Response from `DruidException`.
2024-03-15 21:29:11 +05:30
AlbericByte 33bb99cd0d
remove use log of log4j v1 (#15984) 2024-03-15 15:43:48 +05:30
Karan Kumar 5e603ac5ff
Adding more logging for s3 RetryableS3OutputStream (#16117)
Adding more logging for s3 RetryableS3OutputStream which would help us determine if the chunk size needs to be adjusted.
2024-03-14 11:35:57 +05:30
Gian Merlino 256160aba6
MSQ: Validate that strings and string arrays are not mixed. (#15920)
* MSQ: Validate that strings and string arrays are not mixed.

When multi-value strings and string arrays coexist in the same column,
it causes problems with "classic MVD" style queries such as:

  select * from wikipedia -- fails at runtime
  select count(*) from wikipedia where flags = 'B' -- fails at planning time
  select flags, count(*) from wikipedia group by 1 -- fails at runtime

To avoid these problems, this patch adds type verification for INSERT
and REPLACE. It is targeted: the only type changes that are blocked are
string-to-array and array-to-string. There is also a way to exclude
certain columns from the type checks, if the user really knows what
they're doing.

* Fixes.

* Tests and docs and error messages.

* More docs.

* Adjustments.

* Adjust message.

* Fix tests.

* Fix test in DV mode.
2024-03-13 15:37:27 -07:00
Gian Merlino 910124d4de
MSQ: Plan without implicit sorting. (#16073)
* MSQ: Plan without implicit sorting.

This patch adds an EngineFeature "GROUPBY_IMPLICITLY_SORTS" and sets
it true for native, false for MSQ. It's useful for two reasons:

1) In the future we'll likely want MSQ to hash-partition for GROUP BY
   instead of using a global sort, which would mean MSQ would not
   implicitly ORDER BY when there is a GROUP BY.

2) When doing REPLACE with MSQ, CLUSTERED BY is transformed to ORDER BY.
   We should retain that ORDER BY, as it may be a subset of the GROUP BY,
   and it is important to remember which fields the user wanted to include in
   range shard specs.

* Fix tests.

* Fix tests for real.

* Fix test.
2024-03-13 08:27:39 -07:00
Karan Kumar 84c5098473
Fix data race in getting results from MSQ select tasks. (#16107)
* Fix data race in getting results from MSQ select tasks.

* Add better logging

* Handling number overflow.
2024-03-13 08:58:18 +05:30
Zoltan Haindrich 8252d72e2a
Pull up literals in InputAccessor (#16033)
* Pull up literals in InputAccessor

* pull up literals in `InputAccessor`
* remove the need to pass `constants` of `Window`  operator

Fixes #15353

* update test

* enable relax_nulls
2024-03-12 09:14:31 -07:00
Vishesh Garg 2dd8b16467
Correct the API used to fetch the version for a GCS object (#16097)
Current API used to fetch the version for a GCS object is incorrect. This PR fixes that API.
2024-03-11 18:30:34 +05:30
Zoltan Haindrich 2eb7d7a89b
Calcite tests remove expected exception (#16046)
* Calcite tests remove expected exception

* update testcases using `expectedException` to utilize `assertThrows` instead
* remove `BaseCalciteQueryTest#expectedException`
* fixes `cannotVectorize` so it doesn't anymore stops further processing
* `msqIncompatible` is not anymore toggles a boolean - its an `Assume` instead

Fixes #15423

* cleanup

* move msqIncompat

* update test

* cleanup

* remove comment

* empty-commit

* empty-commit
2024-03-11 13:23:57 +05:30
Vishesh Garg b1c1937e94
Change last update timestamp granularity of GCS objects from seconds to milliseconds (#16083)
The previously used GCS API client library returned last update time for objects directly in milliseconds. The new library returns it in OffsetDateTime format which was being converted to seconds and stored against the object. This fix converts the time back to ms before storing it.
2024-03-09 07:54:33 +05:30
George Shiqi Wu 40ebaf83c9
Fix bug with mmless ingestion and compaction tasks on azure (#16065)
* Update azure behavior to match s3

* Add test

* Cleanup logic

* fix checkstyle

* Add comment
2024-03-08 15:42:44 -05:00
Jan Werner a7b2747e56
remove aws-sdk from ranger-extension (#16011)
Fixes # size blowup regression introduced in https://github.com/apache/druid/pull/15443

This PR removes the transitive dependency of ranger-plugins-audit to reduce the size of the compiled artifacts

* add aws-logs-sdk to ensure that all the transitive dependencies are satisfied
* replace aws-bundle-sdk with aws-logs-sdk
* add additional guidance on ranger update, add dependency ignore to satisfy dependency analyzer
* add aws-sdk-logs to list of ignored dependencies to satisfy the maven plugin
* align aws-sdk versions
2024-03-08 07:35:29 -08:00
AmatyaAvadhanula 5871b81a78
Fix race in BaseNodeRoleWatcher tests (#16064)
* Fix race in BaseNodeRoleWatcher tests

* Make non static
2024-03-07 13:41:16 -08:00
Vishesh Garg bed5d9c3b2
Remove exception on failure response from GCS delete API (#16047)
* Throw 404 Exception on failure response from GCS delete API

* Replace String.format

* Apply suggestions from code review

Co-authored-by: Abhishek Radhakrishnan <abhishek.rb19@gmail.com>

* Remove exception for file not found and fix tests

* Add warn log and fix intellij inspection errors

* More intellij inspection fixes

* * Change to debug log
* change runtime exception class for code coverage
* Add file paths for batch delete failures

* Move failedPaths computation to inside isDebugEnabled flag

* Correct handling of StorageException

* Address review comments

* Remove unused exceptions

* Address code coverage and review comments

* Minor corrections

---------

Co-authored-by: Abhishek Radhakrishnan <abhishek.rb19@gmail.com>
2024-03-07 17:57:17 +05:30
AmatyaAvadhanula c2841425f4
Handle uninitialized cache in Node role watchers (#15726)
BaseNodeRoleWatcher counts down cacheInitialized after a timeout, but also sets some flag that it was a timed-out initialization. and call nodeViewInitializationTimedOut (new method on listeners) instead of nodeViewInitialized. Then listeners can do what is most appropriate with this information.
2024-03-06 16:00:24 +05:30
Adarsh Sanjeev ddd9da2e09
Make servedSegments nullable to maintain compatibility (#16034)
* Make servedSegments nullable to maintain compatibility
2024-03-06 11:39:24 +05:30
zachjsh 720f1e834a
Add support for AzureDNSZone enabled storage accounts used for deep storage (#16016)
* Add support for AzureDNSZone enabled storage accounts used for deep storage

Added a new config to AzureAccountConfig

`storageAccountEndpointSuffix`

which allows the user to specify a storage account endpoint suffix where the underlying
storage account is enabled for AzureDNSZone. The previous config `endpointSuffix`, did not allow
support for such accounts. The previous config has been deprecated in favor of this new config. Also
fixed an issue where `managedIdentityClientId` was not being set properly

* * address review comments

* * add back azure government link and docs
2024-03-04 16:13:28 -05:00
Gian Merlino 930655ff18
Move retries into DataSegmentPusher implementations. (#15938)
* Move retries into DataSegmentPusher implementations.

The individual implementations know better when they should and should
not retry. They can also generate better error messages.

The inspiration for this patch was a situation where EntityTooLarge was
generated by the S3DataSegmentPusher, and retried uselessly by the
retry harness in PartialSegmentMergeTask.

* Fix missing var.

* Adjust imports.

* Tests, comments, style.

* Remove unused import.
2024-03-04 10:36:21 -08:00
Adarsh Sanjeev 93eeb05eaf
Revert explain attributes change to old behaviour. (#16004)
* Revert explain attributes change

* Fix tests

* Fix tests

* Rename function
2024-03-04 15:56:02 +05:30
Gian Merlino 8d3ed31015
MSQ: Nicer error when sortMerge join falls back to broadcast. (#16002)
* MSQ: Nicer error when sortMerge join falls back to broadcast.

In certain cases, joins run as broadcast even when the user hinted
that they wanted sortMerge. This happens when the sortMerge algorithm
is unable to process the join, because it isn't a direct comparison
between two fields on the LHS and RHS.

When this happens, the error message from BroadcastTablesTooLargeFault
is quite confusing, since it mentions that you should try sortMerge
to fix it. But the user may have already configured sortMerge.

This patch fixes it by having two error messages, based on whether
broadcast join was used as a primary selection or as a fallback selection.

* Style.

* Better message.
2024-03-01 13:16:39 -08:00
Kashif Faraz f757231420
Use cache for password hash while validating LDAP password (#15993) 2024-02-28 18:33:33 +05:30
Adarsh Sanjeev d2c2036ea2
Optimize MSQ realtime queries (#15399)
Currently, while reading results from realtime tasks, requests are sent on a segment level. This is slightly wasteful, as when contacting a data servers, it is possible to transfer results for all segments which it is hosting, instead of only one segment at a time.

One change this PR makes is to group the segments on the basis of servers. This reduces the number of queries to data servers made. Since we don't have access to the number of rows for realtime segments, the grouping is done with a fixed estimated number of rows for each realtime segment.
2024-02-28 11:32:14 +05:30
Karan Kumar 5bb5b41b18
Adding task pending time in MSQ reports (#15966)
Added a new field pendingMs in MSQ task reports. This helps in figuring out the exact run time of the MSQ worker tasks.
    Fixed data races.
2024-02-27 14:41:28 +05:30
Laksh Singla 17e4f3ac60
Refactor GroupBy and TopN code to relax the constraint of dimensions being comparable (#15559)
The code in the groupBy engine and the topN engine assume that the dimensions are comparable and can call dimA.compareTo(dimB) to sort the dimensions and group them together.
This works well for the primitive dimensions, because they are Comparable, however falls apart when the dimensions can be arrays (or in future scenarios complex columns). In cases when the dimensions are not comparable, Druid resorts to having a wrapper type ComparableStringArray and ComparableList, which is a Comparable, based on the list comparator.
2024-02-27 11:39:29 +05:30
AmatyaAvadhanula e2b7289dea
Try to fetch the task status for an active from memory (#15724)
* Reduce metadata calls to fetch the status for an active task
2024-02-26 13:53:05 +05:30
Zoltan Haindrich 06deda9415
ScanAndSort query fails with NPE for simple queries (#15914)
* some stuff

* add dummy fields

* draft-fix

* rename test

* cleanup

* add null

* cleanup

* cleanup

* add test

* updates

* move check tp constructore

* cleanup

* updates/etc

* fix some more

* add rowSignatureMode

* checkstyle/etc

* override

* missing msqIncompat

* fix test

* fixes

* undo

* updates

* remove param
2024-02-24 15:33:50 -08:00
Adithya Chakilam 1f443d218c
Enable partition stats on streaming task completion report (#15930)
Changes:
- Add visibility into number of records processed by each streaming task per partition
- Add field `recordsProcessed` to `IngestionStatsAndErrorsTaskReportData`
- Populate number of records processed per partition in `SeekableStreamIndexTaskRunner`
2024-02-23 16:29:03 +05:30
zachjsh 8ebf237576
Move INSERT & REPLACE validation to the Calcite validator (#15908)
This PR contains a portion of the changes from the inactive draft PR for integrating the catalog with the Calcite planner https://github.com/apache/druid/pull/13686 from @paul-rogers, Refactoring the IngestHandler and subclasses to produce a validated SqlInsert instance node instead of the previous Insert source node. The SqlInsert node is then validated in the calcite validator. The validation that is implemented as part of this pr, is only that for the source node, and some of the validation that was previously done in the ingest handlers. As part of this change, the partitionedBy clause can be supplied by the table catalog metadata if it exists, and can be omitted from the ingest time query in this case.
2024-02-22 14:01:59 -05:00
Clint Wylie fe2ba8cc28
fix return type inference of parse_long, which can also be null if string is not parseable into a long (#15909)
* fix return type inference of parse_long, which can also be null if string is not parseable into a long

* fix msq test
2024-02-15 08:45:34 -08:00
Parth Agrawal 495e66f2e7
CVE Fix: Update json-path version (#15772)
Apache Druid brings the dependency json-path which is affected by CVE-2023-51074.
Its latest version 2.9.0 fixes the above CVE.

Append function has been added to json-path and so the unit test to check for the append function not present has been updated.

---------

Co-authored-by: Xavier Léauté <xvrl@apache.org>
2024-02-14 20:58:27 -08:00
YongGang 19ed5c863f
Enhance rolling Supervisor restarts at taskDuration (#15859) 2024-02-14 15:44:34 -08:00
Gian Merlino 0f6a895372
Rework ExprMacro base classes to simplify implementations. (#15622)
* Rework ExprMacro base classes to simplify implementations.

This patch removes BaseScalarUnivariateMacroFunctionExpr, adds
BaseMacroFunctionExpr at the top of the hierarchy (a suitable base class
for ExprMacros that take either arrays or scalars), and adds an
implementation for "visit" to BaseMacroFunctionExpr.

The effect on implementations is generally cleaner code:

- Exprs no longer need to implement "visit".
- Exprs no longer need to implement "stringify", even if they don't
  use all of their args at runtime, because BaseMacroFunctionExpr has
  access to even unused args.
- Exprs that accept arrays can extend BaseMacroFunctionExpr and
  inherit a bunch of useful methods. The only one they need to
  implement themselves that scalar exprs don't is "supplyAnalyzeInputs".

* Make StringDecodeBase64UTFExpression a static class.

* Remove unused import.

* Formatting, annotation changes.
2024-02-12 15:50:45 -08:00
Lasse Mammen 4255711b3e
fix: handle BOOKMARK events in kubernetes pod discovery (#15819) 2024-02-09 18:50:04 +05:30
Gian Merlino 21a97f4c61
Fix HllSketchHolderObjectStrategy#isSafeToConvertToNullSketch. (#15860)
* Fix HllSketchHolderObjectStrategy#isSafeToConvertToNullSketch.

The prior code from #15162 was reading only the low-order byte of an int
representing the size of a coupon set. As a result, it would erroneously
believe that a coupon set with a multiple of 256 elements was empty.
2024-02-08 08:14:28 +05:30
Adarsh Sanjeev 514b3b4d01
Add export capabilities to MSQ with SQL syntax (#15689)
* Add test

* Parser changes to support export statements

* Fix builds

* Address comments

* Add frame processor

* Address review comments

* Fix builds

* Update syntax

* Webconsole workaround

* Refactor

* Refactor

* Change export file path

* Update docs

* Remove webconsole changes

* Fix spelling mistake

* Parser changes, add tests

* Parser changes, resolve build warnings

* Fix failing test

* Fix failing test

* Fix IT tests

* Add tests

* Cleanup

* Fix unparse

* Fix forbidden API

* Update docs

* Update docs

* Address review comments

* Address review comments

* Fix tests

* Address review comments

* Fix insert unparse

* Add external write resource action

* Fix tests

* Add resource check to overlord resource

* Fix tests

* Add IT

* Update syntax

* Update tests

* Update permission

* Address review comments

* Address review comments

* Address review comments

* Add tests

* Add check for runtime parameter for bucket and path

* Add check for runtime parameter for bucket and path

* Add tests

* Update docs

* Fix NPE

* Update docs, remove deadcode

* Fix formatting
2024-02-07 22:08:50 +05:30
Pramod Immaneni 59bca0951a
Parallelize storage of incremental segments (#13982)
During ingestion, incremental segments are created in memory for the different time chunks and persisted to disk when certain thresholds are reached (max number of rows, max memory, incremental persist period etc). In the case where there are a lot of dimension and metrics (1000+) it was observed that the creation/serialization of incremental segment file format for persistence and persisting the file took a while and it was blocking ingestion of new data. This affected the real-time ingestion. This serialization and persistence can be parallelized across the different time chunks. This update aims to do that.

The patch adds a simple configuration parameter to the ingestion tuning configuration to specify number of persistence threads. The default value is 1 if it not specified which makes it the same as it is today.
2024-02-07 10:43:05 +05:30
Laksh Singla ee78a0367d
Fix serialization bug in PassthroughAggregatorFactory (#15830)
PassthroughAggregatorFactory overrides a deprecated method in the AggregatorFactory, on which it relies on for serializing one of its fields complexTypeName. This was accidentally removed, leading to a bug in the factory, where the type name doesn't get serialized properly, and places null in the type name. This PR revives that method with a different name and adds tests for the same.
2024-02-05 15:11:10 +05:30