Commit Graph

2256 Commits

Author SHA1 Message Date
Rushikesh Bankar c8323d1a7c
Add indexer task success and failure metrics (#16829)
This PR adds indexer-level task metrics-

"indexer/task/failed/count"
"indexer/task/success/count"

the current "worker/task/completed/count" metric shows all the tasks completed irrespective of success or failure status so these metrics would help us get more visibility into the status of the completed tasks
2024-08-05 16:21:27 +05:30
AmatyaAvadhanula 92a40d8169
Add API to fetch conflicting task locks (#16799)
* Add API to fetch conflicting active locks
2024-07-30 11:40:48 +05:30
Vishesh Garg e9ea243d97
Enable compaction ITs on MSQ engine (#16778)
Follow-up to #16291, this commit enables a subset of existing native compaction ITs on the MSQ engine.

In the process, the following changes have been introduced in the MSQ compaction flow:
- Populate `metricsSpec` in `CompactionState` from `querySpec` in `MSQControllerTask` instead of `dataSchema`
- Add check for pre-rolled-up segments having `AggregatorFactory` with different input and output column names
- Fix passing missing cluster-by clause in scan queries
- Add annotation of `CompactionState` to tombstone segments
2024-07-30 09:34:46 +05:30
Kashif Faraz caedeb66cd
Add API to update compaction engine (#16803)
Changes:
- Add API `/druid/coordinator/v1/config/compaction/global` to update cluster level compaction config
- Add class `CompactionConfigUpdateRequest`
- Fix bug in `CoordinatorCompactionConfig` which caused compaction engine to not be persisted.
Use json field name `engine` instead of `compactionEngine` because JSON field names must align
with the getter name.
- Update MSQ validation error messages
- Complete overhaul of `CoordinatorCompactionConfigResourceTest` to remove unnecessary mocking
and add more meaningful tests.
- Add `TuningConfigBuilder` to easily build tuning configs for tests.
- Add `DatasourceCompactionConfigBuilder`
2024-07-27 09:14:51 +05:30
Clint Wylie 02b8738c00
remove batchProcessingMode from task config, remove AppenderatorImpl (#16765)
changes:
* removes `druid.indexer.task.batchProcessingMode` in favor of always using `CLOSED_SEGMENT_SINKS` which uses `BatchAppenderator`. This was intended to become the default for native batch, but that was missed so `CLOSED_SEGMENTS` was the default (using `AppenderatorImpl`), however MSQ has been exclusively using `BatchAppenderator` with no problems so it seems safe to just roll it out as the only option for batch ingestion everywhere.
* with `batchProcessingMode` gone, there is no use for `AppenderatorImpl` so it has been removed
* implify `Appenderator` construction since there are only separate stream and batch versions now
* simplify tests since `batchProcessingMode` is gone
2024-07-22 13:56:44 -07:00
Clint Wylie a34a06e192
remove Firehose and FirehoseFactory (#16758)
changes:
* removed `Firehose` and `FirehoseFactory` and remaining implementations which were mostly no longer used after #16602
* Moved `IngestSegmentFirehose` which was still used internally by Hadoop ingestion to `DatasourceRecordReader.SegmentReader`
* Rename `SQLFirehoseFactoryDatabaseConnector` to `SQLInputSourceDatabaseConnector` and similar renames for sub-classes
* Moved anything remaining in a 'firehose' package somewhere else
* Clean up docs on firehose stuff
2024-07-19 14:37:21 -07:00
Kashif Faraz b1edf4a5b4
Refactor: Clean up Overlord guice dependencies (#16752)
Description:
Overlord guice dependencies are currently a little difficult to plug into.
This was encountered while working on a separate PR where a class needed to depend
on `TaskMaster.getTaskQueue()`  to query some task related info but this class itself
needs to be a dependency of `TaskMaster` so that it can be registered to the leader lifecycle.

The approach taken here is to simply decouple the leadership lifecycle of the overlord from
manipulation or querying of its state.

Changes:
- No functional change
- Add new class `DruidOverlord` to contain leadership logic after the model of `DruidCoordinator`
- The new class `DruidOverlord` should not be a dependency of any class with the exception of
REST endpoint `*Resource` classes.
- All classes that need to listen to leadership changes must be a dependency of `DruidOverlord`
so that they can be registered to the leadership lifecycle.
- Move all querying logic from `OverlordResource` to `TaskQueryTool` so that other classes can
leverage this logic too (required for follow up PR).
- Update tests
2024-07-19 17:30:23 +05:30
Clint Wylie 35b876436b
remove native scan query legacy mode (#16659) 2024-07-18 23:33:27 -07:00
Kashif Faraz 89066b72cf
Fix bug in TaskStorageQueryAdapter (#16750)
Changes:
- Do not hold a reference to `TaskQueue` in `TaskStorageQueryAdapter`
- Use `TaskStorage` instead of `TaskStorageQueryAdapter` in `IndexerMetadataStorageAdapter`
- Rename `TaskStorageQueryAdapter` to `TaskQueryTool`
- Fix newly added task actions `RetrieveUpgradedFromSegmentIds` and `RetrieveUpgradedToSegmentIds`
by removing `isAudited` method.
2024-07-17 23:17:41 +05:30
Kashif Faraz 9f6ce6ddc0
Remove task action audit logging and druid_taskLog metadata table (#16309)
Description:
Task action audit logging was first deprecated and disabled by default in Druid 0.13, #6368.

As called out in the original discussion #5859, there are several drawbacks to persisting task action audit logs. 
- Only usage of the task audit logs is to serve the API `/indexer/v1/task/{taskId}/segments`
which returns the list of segments created by a task.
- The use case is really narrow and no prod clusters really use this information.
- There can be better ways of obtaining this information, such as the metric
`segment/added/bytes` which reports both the segment ID and task ID
when a segment is committed by a task. We could also include committed segment IDs in task reports.
- A task persisting several segments would bloat up the audit logs table putting unnecessary strain
on metadata storage.

Changes:
- Remove `TaskAuditLogConfig`
- Remove method `TaskAction.isAudited()`. No task action is audited anymore.
- Remove `SegmentInsertAction` as it is not used anymore. `SegmentTransactionalInsertAction`
is the new incarnation which has been in use for a while.
- Deprecate `MetadataStorageActionHandler.addLog()` and `getLogs()`. These are not used anymore
but need to be retained for backward compatibility of extensions.
- Do not create `druid_taskLog` metadata table anymore.
2024-07-17 17:09:00 +05:30
Kashif Faraz 01d67ae543
Allow CompactionSegmentIterator to have custom priority (#16737)
Changes:
- Break `NewestSegmentFirstIterator` into two parts
  - `DatasourceCompactibleSegmentIterator` - this contains all the code from `NewestSegmentFirstIterator`
  but now handles a single datasource and allows a priority to be specified
  - `PriorityBasedCompactionSegmentIterator` - contains separate iterator for each datasource and
  combines the results into a single queue to be used by a compaction search policy
- Update `NewestSegmentFirstPolicy` to use the above new classes
- Cleanup `CompactionStatistics` and `AutoCompactionSnapshot`
- Cleanup `CompactSegments`
- Remove unused methods from `Tasks`
- Remove unneeded `TasksTest`
- Move tests from `NewestSegmentFirstIteratorTest` to `CompactionStatusTest`
and `DatasourceCompactibleSegmentIteratorTest`
2024-07-16 19:54:49 +05:30
AmatyaAvadhanula d6c760f7ce
Do not kill segments with referenced load specs from deep storage (#16667)
Do not kill segments with referenced load specs from deep storage
2024-07-15 14:07:53 +05:30
Kashif Faraz 656667ee89
Tests: Add utility class TuningConfigBuilder to make IndexTask tests more readable and concise (#16732)
Changes:
- No functional change
- Add class `TuningConfigBuilder` to build `IndexTuningConfig`, `CompactionTuningConfig`
- Remove old class `ParallelIndexTestingFactory.TuningConfigBuilder`
- Remove some unused fields and methods
2024-07-15 10:13:06 +05:30
Kashif Faraz a618c5dd0d
Refactor: Miscellaneous batch task cleanup (#16730)
Changes
- No functional change
- Remove unused method `IndexTuningConfig.withPartitionsSpec()`
- Remove unused method `ParallelIndexTuningConfig.withPartitionsSpec()`
- Remove redundant method `CompactTask.emitIngestionModeMetrics()`
- Remove Clock argument from `CompactionTask.createDataSchemasForInterval()` as it was only needed
for one test which was just verifying the value passed by the test itself. The code now uses a `Stopwatch`
instead and test simply verifies that the metric has been emitted.
- Other minor cleanup changes
2024-07-13 08:12:51 +05:30
Vishesh Garg 197c54f673
Auto-Compaction using Multi-Stage Query Engine (#16291)
Description:
Compaction operations issued by the Coordinator currently run using the native query engine.
As majority of the advancements that we are making in batch ingestion are in MSQ, it is imperative
that we support compaction on MSQ to make Compaction more robust and possibly faster. 
For instance, we have seen OOM errors in native compaction that MSQ could have handled by its
auto-calculation of tuning parameters. 

This commit enables compaction on MSQ to remove the dependency on native engine. 

Main changes:
* `DataSourceCompactionConfig` now has an additional field `engine` that can be one of 
`[native, msq]` with `native` being the default.
*  if engine is MSQ, `CompactSegments` duty assigns all available compaction task slots to the
launched `CompactionTask` to ensure full capacity is available to MSQ. This is to avoid stalling which
could happen in case a fraction of the tasks were allotted and they eventually fell short of the number
of tasks required by the MSQ engine to run the compaction.
* `ClientCompactionTaskQuery` has a new field `compactionRunner` with just one `engine` field.
* `CompactionTask` now has `CompactionRunner` interface instance with its implementations
`NativeCompactinRunner` and `MSQCompactionRunner` in the `druid-multi-stage-query` extension.
The objectmapper deserializes `ClientCompactionRunnerInfo` in `ClientCompactionTaskQuery` to the
`CompactionRunner` instance that is mapped to the specified type [`native`, `msq`]. 
* `CompactTask` uses the `CompactionRunner` instance it receives to create the indexing tasks.
* `CompactionTask` to `MSQControllerTask` conversion logic checks whether metrics are present in 
the segment schema. If present, the task is created with a native group-by query; if not, the task is
issued with a scan query. The `storeCompactionState` flag is set in the context.
* Each created `MSQControllerTask` is launched in-place and its `TaskStatus` tracked to determine the
final status of the `CompactionTask`. The id of each of these tasks is the same as that of `CompactionTask`
since otherwise, the workers will be unable to determine the controller task's location for communication
(as they haven't been launched via the overlord).
2024-07-12 16:40:20 +05:30
Kashif Faraz d9bd02256a
Refactor: Rename UsedSegmentChecker and cleanup task actions (#16644)
Changes:
- Rename `UsedSegmentChecker` to `PublishedSegmentsRetriever`
- Remove deprecated single `Interval` argument from `RetrieveUsedSegmentsAction`
as it is now unused and has been deprecated since #1988 
- Return `Set` of segments instead of a `Collection` from `IndexerMetadataStorageCoordinator.retrieveUsedSegments()`
2024-06-26 10:48:59 +05:30
Tom 52c9929019
Column name in parse exceptions (#16529)
* first pass

* more changes

* fix tests and formatting

* fix kinesis failing tests

* fix kafka tests

* add dimension name to float parse errors

* double and convertToType handling of dimensionName can report parse errors with dimension name

* fix checkstyle issue

* fix tests

* more cases to have better parse exception messages

* fix test

* fix tests

* partially address comments

* annotate method parameter with nullable

* address comments

* fix tests

* let float, double, long dimensionIndexer pass dimensionName down to dimensionHandlerUtils

* fix compilation error and clean up formatting

* clean up whitespace

* address feedback. undo change, pass down report parse exception for convertToType

* fix test
2024-06-25 13:42:52 -07:00
Kashif Faraz f1043d20bc
Support csv input format in Kafka ingestion with header (#16630)
* Support ListBasedInputRow in Kafka ingestion with header
* Fix up buildBlendedEventMap
* Add new test for KafkaInputFormat with csv value and headers
* Do not use forbidden APIs
* Move utility method to TestUtils
2024-06-25 11:50:01 +05:30
Clint Wylie 37a50e6803
Remove index_realtime and index_realtime_appenderator tasks (#16602)
index_realtime tasks were removed from the documentation in #13107. Even
at that time, they weren't really documented per se— just mentioned. They
existed solely to support Tranquility, which is an obsolete ingestion
method that predates migration of Druid to ASF and is no longer being
maintained. Tranquility docs were also de-linked from the sidebars and
the other doc pages in #11134. Only a stub remains, so people with
links to the page can see that it's no longer recommended.

index_realtime_appenderator tasks existed in the code base, but were
never documented, nor as far as I am aware were they used for any purpose.

This patch removes both task types completely, as well as removes all
supporting code that was otherwise unused. It also updates the stub
doc for Tranquility to be firmer that it is not compatible. (Previously,
the stub doc said it wasn't recommended, and pointed out that it is
built against an ancient 0.9.2 version of Druid.)

ITUnionQueryTest has been migrated to the new integration tests framework and updated to use Kafka ingestion.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-06-24 20:13:33 -07:00
Kashif Faraz 0fe6a2af68
Fix replica task failures with metadata inconsistency while running concurrent append replace (#16614)
Changes:
- Add new task action `RetrieveSegmentsByIdAction`
- Use new task action to retrieve segments irrespective of their visibility
- During rolling upgrades, this task action would fail as Overlord would be on old version
- If new action fails, fall back to just fetching used segments as before
2024-06-24 09:56:04 +05:30
Maytas Monsereenusorn d6c7d868cd
Fix peon startup with non string property value (#16612) 2024-06-16 07:48:44 +05:30
Kashif Faraz e4fdf1055b
Update default value of `druid.indexer.tasklock.batchAllocationWaitTime` to zero (#16578)
Update default value of druid.indexer.tasklock.batchAllocationWaitTime to 0.
Thus, a segment allocation request is processed immediately unless there are already some requests queued before this one. While in queue, a segment allocation request may get clubbed together with other similar requests into a batch to reduce load on the metadata store.
2024-06-10 20:07:23 +05:30
Kashif Faraz e4f59e00b2
Fix backwards compatibility with centralized schema config in partial_index_merge tasks (#16556)
* Handle null values of centralized schema config in PartialMergeTask

* Fix checkstyle

* Do not pass centralized schema config from supervisor task to sub-tasks

* Do not pass ObjectMapper in constructor of task

* Fix logs

* Fix tests
2024-06-06 13:44:04 +05:30
Abhishek Radhakrishnan b9ba286423
Fix task bootstrapping & simplify segment load/drop flows (#16475)
* Fix task bootstrap locations.

* Remove dependency of SegmentCacheManager from SegmentLoadDropHandler.

- The load drop handler code talks to the local cache manager via
SegmentManager.

* Clean up unused imports and stuff.

* Test fixes.

* Intellij inspections and test bind.

* Clean up dependencies some more

* Extract test load spec and factory to its own class.

* Cleanup test util

* Pull SegmentForTesting out to TestSegmentUtils.

* Fix up.

* Minor changes to infoDir

* Replace server announcer mock and verify that.

* Add tests.

* Update javadocs.

* Address review comments.

* Separate methods for download and bootstrap load

* Clean up return types and exception handling.

* No callback for loadSegment().

* Minor cleanup

* Pull out the test helpers into its own static class so it can have better state control.

* LocalCacheManager stuff

* Fix build.

* Fix build.

* Address some CI warnings.

* Minor updates to javadocs and test code.

* Address some CodeQL test warnings and checkstyle fix.

* Pass a Consumer<DataSegment> instead of boolean & rename variables.

* Small updates

* Remove one test constructor.

* Remove the other constructor that wasn't initializing fully and update usages.

* Cleanup withInfoDir() builder and unnecessary test hooks.

* Remove mocks and elaborate on comments.

* Commentary

* Fix a few Intellij inspection warnings.

* Suppress corePoolSize intellij-inspect warning.

The intellij-inspect tool doesn't seem to correctly inspect
lambda usages. See ScheduledExecutors.

* Update docs and add more tests.

* Use hamcrest for asserting order on expectation.

* Shutdown bootstrap exec.

* Fix checkstyle
2024-06-04 10:44:46 -07:00
Kashif Faraz 1974a38bc9
Clean up allocation and supervisor logs for easier debugging (#16535)
Changes:
- Use string taskGroup consistently to easily search for a task group
- Clean up other logs
- No change in any logic
2024-06-03 16:41:04 +05:30
George Shiqi Wu 0936798122
Add limit to task payload size (#16512)
* Add limit to task payload size

* Change to a warning

* Remove test

* Fix unit tests

* Optionally throw alert

* PR comments

* Update indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>

* PR comments

* Reject large payloads

* Update docs/configuration/index.md

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>

* Update indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>

---------

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
2024-05-31 09:17:36 -07:00
George Shiqi Wu ed9881df88
Cleanup logic from handoff API (#16457)
* Cleanup logic from handoff API

* Fix test

* Fix checkstyle

* Update docs
2024-05-16 08:42:44 -07:00
Akshat Jain ddfd62d9a9
Disable loading lookups by default in CompactionTask (#16420)
This PR updates CompactionTask to not load any lookups by default, unless transformSpec is present.

If transformSpec is present, we will make the decision based on context values, loading all lookups by default. This is done to ensure backward compatibility since transformSpec can reference lookups.
If transform spec is not present and no context value is passed, we donot load any lookup.

This behavior can be overridden by supplying lookupLoadingMode and lookupsToLoad in the task context.
2024-05-15 11:39:23 +05:30
George Shiqi Wu c1bf4fed90
API for stopping streaming tasks early (#16310)
* Try stopping task early

* Fix checkstyle

* Add unit test

* Add a couple more tests

* PR changes

* Use notice

* fix checkstyle

* PR changes

* Update indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java

Co-authored-by: Suneet Saldanha <suneet@apache.org>

* Update indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java

Co-authored-by: Suneet Saldanha <suneet@apache.org>

* Change payload

* Remove quotes

---------

Co-authored-by: Suneet Saldanha <suneet@apache.org>
2024-05-14 06:39:50 -07:00
Kashif Faraz 3b84751233
Remove unused task action SegmentLockReleaseAction (#16422)
Changes:
- Remove `SegmentLockReleaseAction` as it is not used anywhere.
It is not even registered as a known sub-type of `TaskAction`.
- Minor refactor in `TaskLockbox`. No functional change.
- Remove `ExpectedException` from `TaskLockboxTest`
2024-05-10 06:38:29 +05:30
Igor Berman d0f3fdab37
Allow using different lock types for kill task, remove markAsUnused parameter (#16362)
Changes:
- Remove deprecated `markAsUnused` parameter from `KillUnusedSegmentsTask`
- Allow `kill` task to use `REPLACE` lock when `useConcurrentLocks` is true
- Use `EXCLUSIVE` lock by default
2024-05-10 06:37:36 +05:30
Rishabh Singh c61c3785a0
Followup changes to 15817 (Segment schema publishing and polling) (#16368)
* Fix build

* Nit changes in KillUnreferencedSegmentSchema

* Replace reference to the abbreviation SMQ with Metadata Query, rename inTransit maps in schema cache

* nitpicks

* Remove reference to smq abbreviation from integration-tests

* Remove reference to smq abbreviation from integration-tests

* minor change

* Update index.md

* Add delimiter while computing schema fingerprint hash
2024-05-03 19:13:52 +05:30
AmatyaAvadhanula 5fae20d287
Do not allocate ids conflicting with existing segment ids (#16380)
* Do not allocate ids conflicting with existing segment ids

* Parameterized tests

* Add doc and retain test for coverage
2024-05-03 19:09:48 +05:30
AmatyaAvadhanula b7ae78296a
Allow different timechunk lock types to coexist in a task group (#16369)
Description:
All the streaming ingestion tasks for a given datasource share the same lock for a given interval.
Changing lock types in the supervisor can lead to segment allocation errors due to lock conflicts
for the new tasks while the older tasks are still running.

Fix:
Allow locks of different types (EXCLUSIVE, SHARED, APPEND, REPLACE) to co-exist if they have
the same interval and the same task group.
2024-05-02 19:54:43 +05:30
Gian Merlino 5d1950d451
MSQ controller: Support in-memory shuffles; towards JVM reuse. (#16168)
* MSQ controller: Support in-memory shuffles; towards JVM reuse.

This patch contains two controller changes that make progress towards a
lower-latency MSQ.

First, support for in-memory shuffles. The main feature of in-memory shuffles,
as far as the controller is concerned, is that they are not fully buffered. That
means that whenever a producer stage uses in-memory output, its consumer must run
concurrently. The controller determines which stages run concurrently, and when
they start and stop.

"Leapfrogging" allows any chain of sort-based stages to use in-memory shuffles
even if we can only run two stages at once. For example, in a linear chain of
stages 0 -> 1 -> 2 where all do sort-based shuffles, we can use in-memory shuffling
for each one while only running two at once. (When stage 1 is done reading input
and about to start writing its output, we can stop 0 and start 2.)

1) New OutputChannelMode enum attached to WorkOrders that tells workers
   whether stage output should be in memory (MEMORY), or use local or durable
   storage.

2) New logic in the ControllerQueryKernel to determine which stages can use
   in-memory shuffling (ControllerUtils#computeStageGroups) and to launch them
   at the appropriate time (ControllerQueryKernel#createNewKernels).

3) New "doneReadingInput" method on Controller (passed down to the stage kernels)
   which allows stages to transition to POST_READING even if they are not
   gathering statistics. This is important because it enables "leapfrogging"
   for HASH_LOCAL_SORT shuffles, and for GLOBAL_SORT shuffles with 1 partition.

4) Moved result-reading from ControllerContext#writeReports to new QueryListener
   interface, which ControllerImpl feeds results to row-by-row while the query
   is still running. Important so we can read query results from the final
   stage using an in-memory channel.

5) New class ControllerQueryKernelConfig holds configs that control kernel
   behavior (such as whether to pipeline, maximum number of concurrent stages,
   etc). Generated by the ControllerContext.

Second, a refactor towards running workers in persistent JVMs that are able to
cache data across queries. This is helpful because I believe we'll want to reuse
JVMs and cached data for latency reasons.

1) Move creation of WorkerManager and TableInputSpecSlicer to the
   ControllerContext, rather than ControllerImpl. This allows managing workers and
   work assignment differently when JVMs are reusable.

2) Lift the Controller Jersey resource out from ControllerChatHandler to a
   reusable resource.

3) Move memory introspection to a MemoryIntrospector interface, and introduce
   ControllerMemoryParameters that uses it. This makes it easier to run MSQ in
   process types other than Indexer and Peon.

Both of these areas will have follow-ups that make similar changes on the
worker side.

* Address static checks.

* Address static checks.

* Fixes.

* Report writer tests.

* Adjustments.

* Fix reports.

* Review updates.

* Adjust name.

* Small changes.
2024-04-30 21:30:27 -07:00
AmatyaAvadhanula 42e99bf912
Add new index on datasource and task_allocator_id for pending segments (#16355)
* Add pending segments index on datasource and task_allocator_id

* Use both datasource and task_allocator_id in queries
2024-04-30 15:48:16 +05:30
Laksh Singla e695e52d3f
Improve code flow in the First/Last vector aggregators and unify the numeric aggregators with the String implementations (#16230)
This PR fixes the first and last vector aggregators and improves their readability. Following changes are introduced

    The folding is broken in the vectorized versions. We consider time before checking the folded object.

    If the numerical aggregator gets passed any other object type for some other reason (like String), then the aggregator considers it to be folded, even though it shouldn’t be. We should convert these objects to the desired type, and aggregate them properly.

    The aggregators must properly use generics. This would minimize the ClassCastException issues that can happen with mixed segment types. We are unifying the string first/last aggregators with numeric versions as well.

    The aggregators must aggregate null values (https://github.com/apache/druid/blob/master/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstLastUtils.java#L55-L56 ). The aggregator should only ignore pairs with time == null, and not value == null

    Time nullity is ignored when trying to vectorize the data.

    String versions initialized with DateTimes.MIN that is equal to Long.MIN / 2. This can cause incorrect results in case the user enters a custom time column. NOTE: This is still present because it would require a larger refactor in all of the versions.

    There is a difference in what users might expect from the results because the code flow is changed (for example, the direction of the for loops, etc), however, this will only change the results, and not the contract set by first/last aggregators, which is that if multiple values have the same timestamp, then any of them can get picked.

    If the column is non-existent, the users might expect a change in the timestamp from DateTime.MAX to Long.MAX, because the code incorrectly used DateTime.MAX to initialize the aggregator, however, in case of a custom timestamp column, this might not be the case. The SQL query might be prohibited from using any Long since it requires a cast to the timestamp function that can fail, but AFAICT native queries don't have such limitations.
2024-04-30 15:13:14 +05:30
Alberic Liu 736a2ab7c1
update code style for task type (#16343)
* update code style for task type

* address the comments
2024-04-29 14:42:55 -07:00
Adithya Chakilam f8015eb02a
Add config lagAggregate to LagBasedAutoScalerConfig (#16334)
Changes:
- Add new config `lagAggregate` to `LagBasedAutoScalerConfig`
- Add field `aggregateForScaling` to `LagStats`
- Use the new field/config to determine which aggregate to use to compute lag
- Remove method `Supervisor.computeLagForAutoScaler()`
2024-04-29 22:20:41 +05:30
Akshat Jain 9d2cae40c3
Add support for selective loading of lookups in the task layer (#16328)
Changes:
- Add `LookupLoadingSpec` to support 3 modes of lookup loading: ALL, NONE, ONLY_REQUIRED
- Add method `Task.getLookupLoadingSpec()`
- Do not load any lookups for `KillUnusedSegmentsTask`
2024-04-29 07:19:59 +05:30
Adarsh Sanjeev 9a2d7c28bc
Prepare master branch for 31.0.0 release (#16333) 2024-04-26 09:22:43 +05:30
Arun Ramani 126a0c219a
Surface lock revocation exceptions in task status (#16325) 2024-04-26 08:39:44 +05:30
AmatyaAvadhanula 31eee7d51e
Check for handoff of upgraded segments (#16162)
Changes:
1) Check for handoff of upgraded realtime segments.
2) Drop sink only when all associated realtime segments have been abandoned.
3) Delete pending segments upon commit to prevent unnecessary upgrades and
partition space exhaustion when a concurrent replace happens. This also prevents
potential data duplication.
4) Register pending segment upgrade only on those tasks to which the segment is associated.
2024-04-25 22:03:38 +05:30
Rishabh Singh e30790e013
Introduce Segment Schema Publishing and Polling for Efficient Datasource Schema Building (#15817)
Issue: #14989

The initial step in optimizing segment metadata was to centralize the construction of datasource schema in the Coordinator (#14985). Thereafter, we addressed the problem of publishing schema for realtime segments (#15475). Subsequently, our goal is to eliminate the requirement for regularly executing queries to obtain segment schema information.

This is the final change which involves publishing segment schema for finalized segments from task and periodically polling them in the Coordinator.
2024-04-24 22:22:53 +05:30
Kashif Faraz 1dabb02843
Fix `ForkingTaskRunnerTest` (#16323)
Changes:
- Use non-static fields to track task counts in `ForkingTaskRunner`
- Update assertions in `ForkingTaskRunnerTest` to ensure that the tests are idempotent
2024-04-24 14:05:05 +05:30
AmatyaAvadhanula 08b5a8b88e
Ignore append locks for compaction when using concurrent locks (#16316)
* Ignore append locks for compaction when using concurrent locks
2024-04-22 23:26:45 +05:30
Adithya Chakilam cff5d1e369
Add method Supervisor.computeLagForAutoScaler (#16314)
Tries to address the comments made on #16284 after merged.

Changes:
- Remove method `Supervisor.getLagMetric()`
- Add method `Supervisor.computeLagForAutoScaler()`
- Remove classes `LagMetric` and `LagMetricTest`
2024-04-20 07:57:50 +05:30
YongGang 6974498d98
Improve error message when task fails before becoming ready (#16286) 2024-04-18 08:15:41 +05:30
zachjsh 2351f038eb
Kafka with topicPattern can ignore old offsets spuriously (#16190)
* * fix

* * simplify

* * simplify tests

* * update matches function definition for Kafka Datasource Metadata

* * add matchesOld

* * override matches and plus for kafka based metadata / sequence numbers

* * implement minus
* add tests

* * fix failing tests

* * remove TODO comments

* * simplfy and add comments

* * remove unused variable in tests

* * remove unneeded function

* * add serde tests

* * more stuff

* * address review comments

* * remove unneeded code.
2024-04-17 10:00:17 -04:00
Adithya Chakilam 34237bc112
Consider max lag for kinesis while autoscaling (#16284)
* Consider max lag for kinesis while autoscaling

* add test for coverage

* test folder
2024-04-17 15:05:05 +05:30