* 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
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>
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
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.
* 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
* 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
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.
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`
Changes:
- Remove deprecated `markAsUnused` parameter from `KillUnusedSegmentsTask`
- Allow `kill` task to use `REPLACE` lock when `useConcurrentLocks` is true
- Use `EXCLUSIVE` lock by default
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.
* 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.
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.
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()`
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`
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.
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.
Changes:
- Use non-static fields to track task counts in `ForkingTaskRunner`
- Update assertions in `ForkingTaskRunnerTest` to ensure that the tests are idempotent
Tries to address the comments made on #16284 after merged.
Changes:
- Remove method `Supervisor.getLagMetric()`
- Add method `Supervisor.computeLagForAutoScaler()`
- Remove classes `LagMetric` and `LagMetricTest`
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.
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
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
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.
* 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>
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
* 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.
* 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.
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)
* 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>
* 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
* 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.
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`.
* 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>
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.
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
* initial commit
* comments
* typo
* comments
* comments
* remove var
* initialize global var early
* remove new line
* small test fix
* same fix another test
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.
* 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.
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`
* Globally disable AUTO_CLOSE_JSON_CONTENT.
This JsonGenerator feature is on by default. It causes problems with code
like this:
try (JsonGenerator jg = ...) {
jg.writeStartArray();
for (x : xs) {
jg.writeObject(x);
}
jg.writeEndArray();
}
If a jg.writeObject call fails due to some problem with the data it's
reading, the JsonGenerator will write the end array marker automatically
when closed as part of the try-with-resources. If the generator is writing
to a stream where the reader does not have some other mechanism to realize
that an exception was thrown, this leads the reader to believe that the
array is complete when it actually isn't.
Prior to this patch, we disabled AUTO_CLOSE_JSON_CONTENT for JSON-wrapped
SQL result formats in #11685, which fixed an issue where such results
could be erroneously interpreted as complete. This patch fixes a similar
issue with task reports, and all similar issues that may exist elsewhere,
by disabling the feature globally.
* Update test.
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.
* Merge hydrant runners flatly for realtime queries.
Prior to this patch, we have two layers of mergeRunners for realtime
queries: one for each Sink (a logical segment) and one across all
Sinks. This is done because to keep metrics and results grouped by Sink,
given that each FireHydrant within a Sink has its own separate storage
adapter.
However, it costs extra memory usage due to the extra layer of
materialization. This is especially pronounced for groupBy queries,
which only use their merge buffers at the top layer of merging. The
lower layer of merging materializes ResultRows directly into the heap,
which can cause heap exhaustion if there are enough ResultRows.
This patch changes to a single layer of merging when bySegment: false,
just like Historicals. To accommodate that, segment metrics like
query/segment/time are now per-FireHydrant instead of per-Sink.
Two layers of merging are retained when bySegment: true. This isn't
common, because it's typically only used when segment level caching
is enabled on the Broker, which is off by default.
* Use SinkQueryRunners.
* Remove unused method.
* Kill tasks should honor the buffer period of unused segments.
- The coordinator duty KillUnusedSegments determines an umbrella interval
for each datasource to determine the kill interval. There can be multiple unused
segments in an umbrella interval with different used_status_last_updated timestamps.
For example, consider an unused segment that is 30 days old and one that is 1 hour old. Currently
the kill task after the 30-day mark would kill both the unused segments and not retain the 1-hour
old one.
- However, when a kill task is instantiated with this umbrella interval, it’d kill
all the unused segments regardless of the last updated timestamp. We need kill
tasks and RetrieveUnusedSegmentsAction to honor the bufferPeriod to avoid killing
unused segments in the kill interval prematurely.
* Clarify default behavior in docs.
* test comments
* fix canDutyRun()
* small updates.
* checkstyle
* forbidden api fix
* doc fix, unused import, codeql scan error, and cleanup logs.
* Address review comments
* Rename maxUsedFlagLastUpdatedTime to maxUsedStatusLastUpdatedTime
This is consistent with the column name `used_status_last_updated`.
* Apply suggestions from code review
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
* Make period Duration type
* Remove older variants of runKilLTask() in OverlordClient interface
* Test can now run without waiting for canDutyRun().
* Remove previous variants of retrieveUnusedSegments from internal metadata storage coordinator interface.
Removes the following interface methods in favor of a new method added:
- retrieveUnusedSegmentsForInterval(String, Interval)
- retrieveUnusedSegmentsForInterval(String, Interval, Integer)
* Chain stream operations
* cleanup
* Pass in the lastUpdatedTime to markUnused test function and remove sleep.
---------
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
* Clear "lineSplittable" for JSON when using KafkaInputFormat.
JsonInputFormat has a "withLineSplittable" method that can be used to
control whether JSON is read line-by-line, or as a whole. The intent
is that in streaming ingestion, "lineSplittable" is false (although it
can be overridden by "assumeNewlineDelimited"), and in batch ingestion,
lineSplittable is true.
When a "json" format is wrapped by a "kafka" format, this isn't set
properly. This patch updates KafkaInputFormat to set this on an
underlying "json" format.
The tests for KafkaInputFormat were overriding the "lineSplittable"
parameter explicitly, which wasn't really fair, because that made them
unrealistic to what happens in production. Now they omit the parameter
and get the production behavior.
* Add test.
* Fix test coverage.
* Faster parsing: reduce String usage, list-based input rows.
Three changes:
1) Reworked FastLineIterator to optionally avoid generating Strings
entirely, and reduce copying somewhat. Benefits the line-oriented
JSON, CSV, delimited (TSV), and regex formats.
2) In the delimited (TSV) format, when the delimiter is a single byte,
split on UTF-8 bytes directly.
3) In CSV and delimited (TSV) formats, use list-based input rows when
the column list is provided upfront by the user.
* Fix style.
* Fix inspections.
* Restore validation.
* Remove fastutil-extra.
* Exception type.
* Fixes for error messages.
* Fixes for null handling.
Changes:
- Add new task context flag useConcurrentLocks.
- This can be set for an individual task or at a cluster level using `druid.indexer.task.default.context`.
- When set to true, any appending task would use an APPEND lock and any other
ingestion task would use a REPLACE lock when using time chunk locking.
- If false (default), we fall back on the context flag taskLockType and then useSharedLock.
Changes:
- Handle exception in deletePendingSegments API and map to correct HTTP status code
- Clean up exception message using `DruidException`
- Add unit tests
Currently, If 2 tasks are consuming from the same partitions, try to publish the segment and update the metadata, the second task can fail because the end offset stored in the metadata store doesn't match with the start offset of the second task. We can fix this by retrying instead of failing.
AFAIK apart from the above issue, the metadata mismatch can happen in 2 scenarios:
- when we update the input topic name for the data source
- when we run 2 replicas of ingestion tasks(1 replica will publish and 1 will fail as the first replica has already updated the metadata).
Implemented the comparable function to compare the last committed end offset and new Sequence start offset. And return a specific error msg for this.
Add retry logic on indexers to retry for this specific error msg.
Updated the existing test case.
The initial step in optimizing segment metadata was to centralize the construction of datasource schema in the Coordinator (#14985). Subsequently, our goal is to eliminate the requirement for regularly executing queries to obtain segment schema information. This task encompasses addressing both realtime and finalized segments.
This modification specifically addresses the issue with realtime segments. Tasks will now routinely communicate the schema for realtime segments during the segment announcement process. The Coordinator will identify the schema alongside the segment announcement and subsequently update the schema for realtime segments in the metadata cache.
Changes
- Audit create or update of a supervisor spec. The purpose of the audit is
to track which user made change to a supervisor and when.
- The audit entry does not contain the entire spec or even a diff of the changes
as this is already captured in the `druid_supervisors` metadata table.
Description
When batchAllocationWaitTime is set to 0, the segment allocation queue is polled continuously even when it is empty. This would take up cpu cycles unnecessarily.
Some existing race conditions would also become more frequent when the batchAllocationWaitTime is 0. This PR tries to better address those race conditions as well.
Changes
Do not reschedule a poll if queue is empty
When a new batch is added to queue, schedule a poll
Simplify keyToBatch map
Handle race conditions better
As soon as a batch starts getting processed, do not add any more requests to it
* Allow empty inserts and replace.
- Introduce a new query context failOnEmptyInsert which defaults to false.
- When this context is false (default), MSQE will now allow empty inserts and replaces.
- When this context is true, MSQE will throw the existing InsertCannotBeEmpty MSQ fault.
- For REPLACE ALL over an ALL grain segment, the query will generate a tombstone spanning eternity
which will be removed eventually be the coordinator.
- Add unit tests in MSQInsertTest, MSQReplaceTest to test the new default behavior (i.e., when failOnEmptyInsert = false)
- Update unit tests in MSQFaultsTest to test the non-default behavior (i.e., when failOnEmptyInsert = true)
* Ignore test to see if it's the culprit for OOM
* Add heap dump config
* Bump up -Xmx from 1500 MB to 2048 MB
* Add steps to tarball and collect hprof dump to GHA action
* put back mx to 1500MB to trigger the failure
* add the step to reusable unit test workflow as well
* Revert the temp heap dump & @Ignore changes since max heap size is increased
* Minor updates
* Review comments
1. Doc suggestions
2. Add tests for empty insert and replace queries with ALL grain and limit in the
default failOnEmptyInsert mode (=false). Add similar tests to MSQFaultsTest with
failOnEmptyInsert = true, so the query does fail with an InsertCannotBeEmpty fault.
3. Nullable annotation and javadocs
* Add comment
replace_limit.patch
Currently in the realtime ingestion (Kafka/Kinesis) case, after publishing the segments, upon acknowledgement from the coordinator that the segments are already placed in some historicals, the peon would unannounce the segments (basically saying the segments are not in this peon anymore to the whole cluster) and drop the segments from cache and sink timeline in one shot.
The in transit queries from the brokers that still thinks the segments are in the peon can get a NullPointer exception when the peon is unsetting the hydrants in the sinks.
The fix would let the peon to wait for a configurable delay period before dropping segments, remove segments from cache etc after the peon unannounce the segments.
This delayed approach is similar to how the historicals handle segments moving out.
Changes
- Add `log` implementation for `AuditManager` alongwith `SQLAuditManager`
- `LoggingAuditManager` simply logs the audit event. Thus, it returns empty for
all `fetchAuditHistory` calls.
- Add new config `druid.audit.manager.type` which can take values `log`, `sql` (default)
- Add new config `druid.audit.manager.logLevel` which can take values `DEBUG`, `INFO`, `WARN`.
This gets activated only if `type` is `log`.
- Remove usage of `ConfigSerde` from `AuditManager` as audit is not just limited to configs
- Add `AuditSerdeHelper` for a single implementation of serialization/deserialization of
audit payload and other utility methods.
The segment allocation algorithm reuses an already allocated pending segment if the new allocation request is made for the same parameters:
datasource
sequence name
same interval
same value of skipSegmentLineageCheck (false for batch append, true for streaming append)
same previous segment id (used only when skipSegmentLineageCheck = false)
The above parameters can thus uniquely identify a pending segment (enforced by the UNIQUE constraint on the sequence_name_prev_id_sha1 column in druid_pendingSegments metadata table).
This reuse is done in order to
allow replica tasks (in case of streaming ingestion) to use the same set of segment IDs.
allow re-run of a failed batch task to use the same segment ID and prevent unnecessary allocations
* Fixing failing compaction/parallel index jobs during upgrade due to new actions not available on the overlord.
* Fixing build
* Removing extra space.
* Fixing json getter.
* Review comments.
Changes:
- Fix log `Got end of partition marker for partition [%s] from task [%s] in discoverTasks`
by fixing order of args
- Simplify in-line classes by using lambda
- Update kill task message from `Task [%s] failed to respond to [set end offsets]
in a timely manner, killing task` to `Failed to set end offsets, killing task`
- Clean up tests