Commit Graph

2227 Commits

Author SHA1 Message Date
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
AmatyaAvadhanula f3d69f30e6
Associate pending segments with the tasks that requested them (#16144)
Changes:
- Add column `task_allocator_id` to `pendingSegments` metadata table.
- Add column `upgraded_from_segment_id` to `pendingSegments` metadata table.
- Add interface `PendingSegmentAllocatingTask` and implement it by all tasks which
can allocate pending segments.
- Use `taskAllocatorId` to identify the task (and its sub-tasks or replicas) to which
a pending segment has been allocated.
- Perform active cleanup of pending segments in `TaskLockbox` once there are no
active tasks for the corresponding task allocator id.
- When committing APPEND segments, also commit all upgraded pending segments
corresponding to that task allocator id.
- When committing REPLACE segments, upgrade all overlapping pending segments in
the same transaction.
2024-04-17 09:06:31 +05:30
Kashif Faraz 81d7b6ebe1
Fix OverlordClient to read reports as a concrete `ReportMap` (#16226)
Follow up to #16217 

Changes:
- Update `OverlordClient.getReportAsMap()` to return `TaskReport.ReportMap`
- Move the following classes to `org.apache.druid.indexer.report` in the `druid-processing` module
  - `TaskReport`
  - `KillTaskReport`
  - `IngestionStatsAndErrorsTaskReport`
  - `TaskContextReport`
  - `TaskReportFileWriter`
  - `SingleFileTaskReportFileWriter`
  - `TaskReportSerdeTest`
- Remove `MsqOverlordResourceTestClient` as it had only one method
which is already present in `OverlordResourceTestClient` itself
2024-04-15 08:00:59 +05:30
YongGang da9feb4430
Introduce TaskContextReport for reporting task context (#16041)
Changes:
- Add `TaskContextEnricher` interface to improve task management and monitoring
- Invoke `enrichContext` in `TaskQueue.add()` whenever a new task is submitted to the Overlord
- Add `TaskContextReport` to write out task context information in reports
2024-04-12 08:57:49 +05:30
Vishesh Garg 3d595cfab1
Add storeCompactionState flag support to msq (#15965)
Compaction in the native engine by default records the state of compaction for each segment in the lastCompactionState segment field. This PR adds support for doing the same in the MSQ engine, targeted for future cases such as REPLACE and compaction done via MSQ.

Note that this PR doesn't implicitly store the compaction state for MSQ replace tasks; it is stored with flag "storeCompactionState": true in the query context.
2024-04-09 16:47:47 +05:30
Abhishek Radhakrishnan 75fb57ed6e
Update error messages when supervisor's checkpoint state is invalid (#16208)
* Update error message when topic messages.

Suggest resetting the supervisor when the topic changes instead of changing
the supervisor name which is actually making a new supervisor.

* Update server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java

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

* Cleanup

* Remove log and include oldCommitMetadataFromDb

* Fix test

---------

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
2024-04-03 10:34:17 -07:00
AmatyaAvadhanula 218513ad55
Use created time from metadata store in list tasks (#16228) 2024-04-03 09:03:32 +05:30
Kashif Faraz 0de44d91f1
Cleanup serialiazation of TaskReportMap (#16217)
* Build task reports in AbstractBatchIndexTask

* Minor cleanup

* Apply suggestions from code review by @abhishekrb

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

* Cleanup IndexTaskTest

* Fix formatting

* Fix coverage

* Cleanup serialization of TaskReport map

* Replace occurrences of Map<String, TaskReport>

* Return TaskReport.ReportMap for live reports, fix test comparisons

* Address test failures

---------

Co-authored-by: Abhishek Radhakrishnan <abhishek.rb19@gmail.com>
2024-04-01 11:53:24 -07:00
Adithya Chakilam 463010bb29
Populate segment stats for non-parallel compaction jobs (#16171)
* Populate segment stats for non-parallel compaction jobs

* fix

* add-tests

* comments

* update-test

* comments
2024-03-29 09:40:55 -04:00
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
Adithya Chakilam a65b2d4f41
Visibility into LagBased AutoScaler desired task count (#16199)
* Visibility into skipped scale notices

* comments

* change to emit always instead of just skips

* fix failing test

* comments

* Add couple more tests
2024-03-27 13:08:00 -04:00
Gian Merlino 58a8a23243
Avoid conversion to String in JsonReader, JsonNodeReader. (#15693)
* Avoid conversion to String in JsonReader, JsonNodeReader.

These readers were running UTF-8 decode on the provided entity to
convert it to a String, then parsing the String as JSON. The patch
changes them to parse the provided entity's input stream directly.

In order to preserve the nice error messages that include parse errors,
the readers now need to open the entity again on the error path, to
re-read the data. To make this possible, the InputEntity#open contract
is tightened to require the ability to re-open entities, and existing
InputEntity implementations are updated to allow re-opening.

This patch also renames JsonLineReaderBenchmark to JsonInputFormatBenchmark,
updates it to benchmark all three JSON readers, and adds a case that reads
fields out of the parsed row (not just creates it).

* Fixes for static analysis.

* Implement intermediateRowAsString in JsonReader.

* Enhanced JsonInputFormatBenchmark.

Renames JsonLineReaderBenchmark to JsonInputFormatBenchmark, and enhances it to
test various readers (JsonReader, JsonLineReader, JsonNodeReader) as well as
to test with/without field discovery.
2024-03-26 08:16:05 -07:00
Abhishek Radhakrishnan 95595ba4f5
Fix handling an empty list of versions (#16198)
* Differentiate null and empty lists of segment IDs and versions.

Treat them differently so the. Segment IDs and versions can be An empty list,
in which case, the queries should just not return anything. Versions are optional, so
they can be null, which just indicates nothing, so the queries should return segments with
all possible versions. Segment IDs cannot be null as indicated by the absence of @Nullable
annotation.

* Update javadocs and add empty versions test to kill task.

* Add test for RetrieveSegmentsActions as well.
2024-03-25 17:51:24 -07:00
Kashif Faraz e7dc00b86d
Refactor: Simplify creation input row filter predicate in various batch tasks (#16196)
Changes:
- Simplify method `AbstractBatchIndexTask.defaultRowFilter()` and rename
- Add method `allowNonNullWithinInputIntervalsOf()`
- Add javadocs
2024-03-26 04:54:07 +05:30
Kashif Faraz 82f443340d
Clean up TaskQueueTest (#16187)
Changes:
- Remove redundant code from `TaskQueueTest`
- Use lambdas in `TaskQueue`
- Simplify error message when `TaskQueue` is full
2024-03-25 09:40:01 +05:30
AmatyaAvadhanula cfa2a901b3
Redact passwords from tasks fetched from the TaskQueue (#16182)
* Redact passwords from tasks fetched from the TaskQueue
2024-03-23 14:22:11 +05:30
Rushikesh Bankar 3d8b0ffae8
Add indexer level task metrics to provide more visibility in the task distribution (#15991)
Changes:

Add the following indexer level task metrics:
- `worker/task/running/count`
- `worker/task/assigned/count`
- `worker/task/completed/count`

These metrics will provide more visibility into the tasks distribution across indexers
(We often see a task skew issue across indexers and with this issue it would be easier
to catch the imbalance)
2024-03-21 11:08:01 +05:30
Abhishek Radhakrishnan fa8e511492
Add versions to `markUsed` and `markUnused` APIs (#16141)
* Mark used and unused APIs by versions.

* remove the conditional invocations.

* isValid() and test updates.

* isValid() and tests.

* Remove warning logs for invalid user requests. Also, downgrade visibility.

* Update resp message, etc.

* tests and some cleanup.

* Docs draft

* Clarify docs

* Update server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java

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

* Review comments

* Remove default interface methods only used in tests and update docs.

* Clarify javadocs and @Nullable.

* Add more tests.

* Parameterized versions.

---------

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
2024-03-19 09:22:25 -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
Abhishek Agarwal 7d307df6e9
Fix metric emission in the segment generation phase (#16146)
Fix metric emission in the segment generation phase
2024-03-18 14:38:18 +05:30
Abhishek Radhakrishnan 3eefc47722
Refactor tests and code clean up (#16129)
* Add update() in TestDerbyConnectorRule

* use common function.

* fixup build.

* fixup indentations.

* Revert "fixup indentations."

This reverts commit a9d6b73e79.

* fixup indentataions.

* Remove Thread.sleep() by directly calling updateUsedStatusLastUpdated.

* another indentation slip.

* Move common segment initialization to setup().

* Fix for checkstyle.

* review comments: indentation fixes, type.

* Wrapper class for Segments table

* Add KillUnusedSegmentsTaskBuilder in test class

* Remove javadocs for self-explanatory methods.
2024-03-15 10:13:14 -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
Abhishek Radhakrishnan fb7bb0953d
Kill segments by versions (#15994)
* Kill task version support.

Kill tasks by default kill all versions of unused segments in the specified
interval. Users wanting to delete specific versions (for example, data compliance
reasons) and keep rest of the versions can specify the optional version in the
kill task payload.

* Formatting changes.

* Multi version tests in RetrieveSegmentsActionsTest

Sort of like method-level parameterized tests.

* Address review feedback

* Accept a list of versions instead of a single version.

Support multiple versions.

* Tests for multiple versions.

* Update docs

* Cleanup

* Address review comments.

Retain the old interface method and make it default and route it to
the method with nullable versions variant. Update usages to use the
default method where versions doesn't matter.

* Remove versions from retreive used segments action.

* Some updates.

* Apply suggestions from code review

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

* /s/actual/observed/g

* minor test cleanup

* WIP: Test fixes and updates. Also add test for kill by version with used load spec.

Checkpoint.

---------

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
2024-03-13 09:37:30 +05:30
Abhishek Radhakrishnan 0a615f16de
Fix bug where numSegmentsKilled is reported incorrectly. Also, add a unit test. (#16103) 2024-03-12 10:02:54 +05:30
Abhishek Radhakrishnan 8084f2206b
Remove `@JsonIgnore` annotations for private members of `TaskAction` classes (#16099)
* Remove @JsonIgnore annotations for private members

* checkstyle fix - removed unused imports.
2024-03-12 00:12:36 +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
Adithya Chakilam 564c44ed85
Add stats segmentsRead and segmentsPublished to compaction task reports (#15947)
Changes:
- Add visibility into number of segments read/published by each parallel compaction
- Add new fields `segmentsRead`, `segmentsPublished` to `IngestionStatsAndErrorsTaskReportData`
- Update `ParallelIndexSupervisorTask` to populate the new stats
2024-03-07 09:37:23 +05:30
Adithya Chakilam ae022cc0c9
fixup!: #15981 Missing completion reports on index_parallel tasks (#16042)
* initial commit

* comments

* typo

* comments

* comments

* remove var

* initialize global var early

* remove new line

* small test fix

* same fix another test
2024-03-06 13:58:34 -05:00
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