Commit Graph

676 Commits

Author SHA1 Message Date
YongGang 46dbc74053
Support Dynamic Peon Pod Template Selection in K8s extension (#16510)
* initial commit

* add Javadocs

* refine JSON input config

* more test and fix build

* extract existing behavior as default strategy

* change template mapping fallback

* add docs

* update doc

* fix doc

* address comments

* define Matcher interface

* fix test coverage

* use lower case for endpoint path

* update Json name

* add more tests

* refactoring Selector class
2024-06-12 15:27:10 -07:00
Zoltan Haindrich f8645de341
Remove incorrect utf8 conversion of ResultCache keys (#16569) 2024-06-12 13:12:05 -07:00
Bünyamin 30c59042e0
Add new metrics from v30 to prometheus-emitter (#16345)
Co-authored-by: asdf2014 <asdf2014@apache.org>
2024-06-05 10:51:48 +05:30
Atul Mohan b53d75758f
IcebergInputSource : Add option to toggle case sensitivity while reading columns from iceberg catalog (#16496)
* Toggle case sensitivity while reading columns from iceberg

* Fix tests

* Drop case check and set unconditionally
2024-05-31 10:18:52 -07:00
Abhishek Radhakrishnan 75937c98e8
Upgrade delta kernel from 3.1.0 to 3.2.0 (#16513)
Upstream release: https://github.com/delta-io/delta/releases/tag/v3.2.0

- Upgrade kernel dependency to 3.2.0
- Notable breaking changes introduced in upstream that affects the Druid extension:
 - Rename TableClient -> Engine
 - Rename DefaultTableClient -> DefaultEngine
 - Exceptions moved to a separate package
 - Table.getPath() doesn't throw TableNotFoundException. Instead the exception is thrown
   when getting snapshot info from the Table object
2024-05-29 10:46:30 -07:00
Benedict Jin cb7c2c1e37
Downgrade the version of Apache Curator from 5.5.0 to 5.3.0 to avoid a bug in the new version (#16425) 2024-05-10 15:08:33 +05:30
Zoltan Haindrich 1811674753
Enable quidem tests to use different suppliers (#16382)
* enable quidem uri support for `druidtest:///?ComponentSupplier=Nested` and similar
* changes the way `SqlTestFrameworkConfig` is being applied; all options will have their own annotation (its kinda impossible to detect that an annotation has a set value or its the default)
* enables hierarchical processing of config annotation (was needed to enable class level supplier annotation)
* moves uri processing related string2config stuff into `SqlTestFrameworkConfig`
2024-05-09 09:21:02 +02:00
Abhishek Radhakrishnan 1d7595f3f7
Support for filters in the Druid Delta Lake connector (#16288)
* Delta Lake support for filters.

* Updates

* cleanup comments

* Docs

* Remmove Enclosed runner

* Rename

* Cleanup test

* Serde test for the Delta input source and fix jackson annotation.

* Updates and docs.

* Update error messages to be clearer

* Fixes

* Handle NumberFormatException to provide a nicer error message.

* Apply suggestions from code review

Co-authored-by: 317brian <53799971+317brian@users.noreply.github.com>

* Doc fixes based on feedback

* Yes -> yes in docs; reword slightly.

* Update docs/ingestion/input-sources.md

Co-authored-by: Laksh Singla <lakshsingla@gmail.com>

* Update docs/ingestion/input-sources.md

Co-authored-by: Laksh Singla <lakshsingla@gmail.com>

* Documentation, javadoc and more updates.

* Not with an or expression end-to-end test.

* Break up =, >, >=, <, <= into its own types instead of sub-classing.

---------

Co-authored-by: 317brian <53799971+317brian@users.noreply.github.com>
Co-authored-by: Laksh Singla <lakshsingla@gmail.com>
2024-04-29 11:31:36 -07:00
Bünyamin 9aef8e02ef
Expend coverage for default mapping (#16340) 2024-04-27 17:39:07 +05:30
Adarsh Sanjeev 9a2d7c28bc
Prepare master branch for 31.0.0 release (#16333) 2024-04-26 09:22:43 +05:30
Kashif Faraz 4b6748bdc9
Update default value of useMaxMemoryEstimates for Hadoop jobs (#16280) 2024-04-26 08:07:21 +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
Zoltan Haindrich 9c0bd56f5b
Make QueryComponentSupliers independent from test classes (#16275) 2024-04-25 02:12:07 -04:00
Bünyamin e74da6a6b6
Add new metrics for prometheus emitter (#16329) 2024-04-25 07:16:24 +05:30
Laksh Singla 6bca406d31
Grouping on complex columns aka unifying GroupBy strategies (#16068)
Users can pass complex types as dimensions to the group by queries. For example:

SELECT nested_col1, count(*) FROM foo GROUP BY nested_col1
2024-04-24 23:00:14 +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
Laksh Singla b9bbde5c0a
Fix deadlock that can occur while merging group by results (#15420)
This PR prevents such a deadlock from happening by acquiring the merge buffers in a single place and passing it down to the runner that might need it.
2024-04-22 14:10:44 +05:30
Hardik Bajaj 0bf5e7745d
Add configurable parameters for statsd client (#16283)
Statsd client sometimes drops metrics when this queueSize of statsd client with max unprocessed messages is completely full. This causes some high cardinality metrics like per partition lag being droppped.
There are multiple parameters of statsdclient that can be initialized and can help increase the load/capacity of client to not to drop metrics more frequently.
Properties like queueSize, poolSize, processorWorkers and senderWorkers will now be configurable at runtime
2024-04-17 18:35:31 +05:30
Zoltan Haindrich 1df41db46d
Migrate to use docker compose v2 (#16232)
https://github.com/actions/runner-images/issues/9557
2024-04-03 12:32:55 +02:00
zachjsh 9b52c909e0
fix complex types returning UNKNOWN as their SQL type inference (#16216)
* * fix

* * fix

* * address review comments
2024-04-02 14:36:01 -04:00
Tapajit Chandra Paul dbef348249
Include heartbeat and zk-connected metric in PrometheusEmitter (#16209) 2024-04-01 15:12:06 +05:30
Sensor f99501179d
remove junit-jupiter-api version to prevent inconsistent version (#16210) 2024-03-27 17:48:14 -07:00
Kashif Faraz 323d67a0ac
Add errorCode to failure type `InternalServerError` (#16186)
Changes:
- Use error code `internalServerError` for failures of this type
- Remove the error code argument from `InternalServerError.exception()` methods
thus fixing a bug in the callers.
2024-03-24 04:24:09 +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
YongGang e7cf8299ce
Expose Kinesis lag metrics (#16172) 2024-03-19 19:42:10 -04: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
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
Clint Wylie 313da98879
decouple column serializer compression closers from SegmentWriteoutMedium to optionally allow serializers to release direct memory allocated for compression earlier than when segment is completed (#16076) 2024-03-11 12:28:04 -07:00
Sensor ba3d4daf45
Fix field names in PeonCommandContext (#16067) 2024-03-09 08:46:51 +05:30
dependabot[bot] 775c1180ae
Bump redis.clients:jedis from 5.0.2 to 5.1.2 (#16074)
Bumps [redis.clients:jedis](https://github.com/redis/jedis) from 5.0.2 to 5.1.2.
- [Release notes](https://github.com/redis/jedis/releases)
- [Commits](https://github.com/redis/jedis/compare/v5.0.2...v5.1.2)

---
updated-dependencies:
- dependency-name: redis.clients:jedis
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
2024-03-08 07:40:37 -08:00
George Shiqi Wu 80cab51d50
Fix bug with cancelling pending tasks when running kubernetes ingestion. (#16036)
* Fix bug

* Add new test
2024-03-07 15:48:14 -05:00
Sensor 4e9b758661
Support CPU resource configurable for Kubernates job under MoK Mode (#16008)
* support CPU resource configurable for Kubernates job

* update property doc

* fix test name

* refine doc format
2024-03-04 10:12:09 -05:00
Jamie 80942d5754
Feature: add support for ingesting from rabbitmq super streams (#14137)
* Add support for ingesting from Rabbit MQ Super Streams
2024-02-22 10:50:37 +05:30
Suneet Saldanha cbc53d53b4
Update k8sTaskRunner log message (#15871) 2024-02-21 14:34:00 +08:00
George Shiqi Wu 2c0d1128f8
Fix pod template reading logic (#15915)
* Fix pod template reading

* PR changes

* Fix unit tests
2024-02-20 11:13:51 -05:00
Tom f224035c7e
Fix Flakiness in KafkaEmitterTest (#15907)
* thrust of the fix to allow for the json values to be out of order

The existing problem is that toMap doesn't turn some values into json primitive
values, for example segmentMetadata just has DateTime objects for it's time in
the EventMap, but Alert event converts those into strings when calling toMap.
This creates an issue because when we check the emitted events the mapper
deserializing the string value for dateTime leaves it as a string in the
EventMap. So the question is do we alter the events toMap() to return string/map
version of objects or to make the expected events do a round trip of
eventMap -> string -> eventMap to turn everything into json primitives

* fix issue by making toMap events convert Objects into strings, or maps

* fix linting errors

* use method of using mapper to round trip expected data to make it have same type
as those of the events emitted

* remove unnecessary comment
2024-02-15 10:01:55 +05:30
Abhishek Radhakrishnan c324e37751
Add javadocs to `KafkaEmitterTest` & fix flaky test (#15898)
* Address review comment: add test javadocs

* Fix flaky assertion failure.

Use ConcurrentHashMap instead of HashMap because the producer callback
can trigger concurrently and override the map initialization.

* fixup intellij inspection
2024-02-14 11:52:06 -08:00
Tom c225c19f81
fix copy paste issue in earlier PR (#15890) 2024-02-12 19:49:19 -05:00
Abhishek Radhakrishnan 51fd79ee58
Clean up kafka emitter tests, add more validations and code coverage. (#15878)
* Clean up kafka emitter tests a bit and add more validations.

The test wasn't validating what events were sent, but simply the dropped counters, which
aren't that useful.
Additionally, this module has fewer tests, so folks often run into code coverage issue
in this extension. Hopefully this change helps with that too.

* Change things to feed-based rather than topic-based.

* Another test for shared topic

* Switch to DruidException, add test dependencies and sad path config tests.

* missing test dependency

* minor renames.

* Add more tests - to test unknown events and drop when queue is full
2024-02-12 16:22:19 -05:00
Tom 11a8624ef1
allow for kafka-emitter to have extra dimensions be set for each event it emits (#15845)
* allow for kafka-emitter to have extra dimensions be set for each event it emits

* fix checktsyle issue in kafkaemitterconfig

* make changes to fix docs, and cleanup copy paste error in #toString()

* undo formatting to markdown table

* add more branches so test passes

* fix checkstyle issue
2024-02-08 22:55:24 -08:00
Abhishek Radhakrishnan 1a5b57df84
Update `groupId` for delta-lake and iceberg extensions (#15843)
* Update the group id to org.apache.druid.extensions.contrib for contrib exts.

* Note iceberg and delta lake extensions in extensions.md

* properties and shell backticks

* Update groupId in distribution/pom.xml

* remove delta-lake from dist.

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

The patch adds a simple configuration parameter to the ingestion tuning configuration to specify number of persistence threads. The default value is 1 if it not specified which makes it the same as it is today.
2024-02-07 10:43:05 +05:30
Abhishek Radhakrishnan 1affa35b29
Bump up Delta Lake Kernel to 3.1.0 (#15842)
This patch bumps Delta Lake Kernel dependency from 3.0.0 to 3.1.0, which released last week - please see https://github.com/delta-io/delta/releases/tag/v3.1.0 for release notes.

There were a few "breaking" API changes in 3.1.0, you can find the rationale for some of those changes here.

Next-up in this extension: add and expose filter predicates.
2024-02-06 21:25:17 +05:30
Atul Mohan 2e46a98024
Add range filtering support for iceberg ingestion (#15782)
* Add range filtering support for iceberg ingestion

* Docs formatting

* Spelling
2024-02-01 23:32:30 -08:00
Abhishek Radhakrishnan 9f95a691f7
Extension to read and ingest Delta Lake tables (#15755)
* something

* test commit

* compilation fix

* more compilation fixes (fixme placeholders)

* Comment out druid-kereberos build since it conflicts with newly added transitive deps from delta-lake

Will need to sort out the dependencies later.

* checkpoint

* remove snapshot schema since we can get schema from the row

* iterator bug fix

* json json json

* sampler flow

* empty impls for read(InputStats) and sample()

* conversion?

* conversion, without timestamp

* Web console changes to show Delta Lake

* Asset bug fix and tile load

* Add missing pieces to input source info, etc.

* fix stuff

* Use a different delta lake asset

* Delta lake extension dependencies

* Cleanup

* Add InputSource, module init and helper code to process delta files.

* Test init

* Checkpoint changes

* Test resources and updates

* some fixes

* move to the correct package

* More tests

* Test cleanup

* TODOs

* Test updates

* requirements and javadocs

* Adjust dependencies

* Update readme

* Bump up version

* fixup typo in deps

* forbidden api and checkstyle checks

* Trim down dependencies

* new lines

* Fixup Intellij inspections.

* Add equals() and hashCode()

* chain splits, intellij inspections

* review comments and todo placeholder

* fix up some docs

* null table path and test dependencies. Fixup broken link.

* run prettify

* Different test; fixes

* Upgrade pyspark and delta-spark to latest (3.5.0 and 3.0.0) and regenerate tests

* yank the old test resource.

* add a couple of sad path tests

* Updates to readme based on latest.

* Version support

* Extract Delta DateTime converstions to DeltaTimeUtils class and add test

* More comprehensive split tests.

* Some test renames.

* Cleanup and update instructions.

* add pruneSchema() optimization for table scans.

* Oops, missed the parquet files.

* Update default table and rename schema constants.

* Test setup and misc changes.

* Add class loader logic as the context class loader is unaware about extension classes

* change some table client creation logic.

* Add hadoop-aws, hadoop-common and related exclusions.

* Remove org.apache.hadoop:hadoop-common

* Apply suggestions from code review

Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>

* Add entry to .spelling to fix docs static check

---------

Co-authored-by: abhishekagarwal87 <1477457+abhishekagarwal87@users.noreply.github.com>
Co-authored-by: Laksh Singla <lakshsingla@gmail.com>
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
2024-01-30 21:53:50 -08:00
Zoltan Haindrich 2eba20d724
Fix minor build issues and stabilize intellij-inspections runs (#15747)
* Possibly stabilize intellij-inspections

* remove `integration-tests-ex/cases` from excluded projects from initial build
* enable ErrorProne's `CheckedExceptionNotThrown` to get earlier errors than intellij-inspections

* fix ddsketch pom.xml

* fix spellcheck
2024-01-24 15:17:33 +05:30
Hiroshi Fukada 3fe3a65344
New: Add DDSketch in extensions-contrib (#15049)
* New: Add DDSketch-Druid extension

- Based off of http://www.vldb.org/pvldb/vol12/p2195-masson.pdf and uses
 the corresponding https://github.com/DataDog/sketches-java library
- contains tests for post building and using aggregation/post
  aggregation.
- New aggregator: `ddSketch`
- New post aggregators: `quantileFromDDSketch` and
  `quantilesFromDDSketch`

* Fixing easy CodeQL warnings/errors

* Fixing docs, and dependencies

Also moved aggregator ids to AggregatorUtil and PostAggregatorIds

* Adding more Docs and better null/empty handling for aggregators

* Fixing docs, and pom version

* DDSketch documentation format and wording
2024-01-23 20:17:07 +05:30
Karan Kumar c4990f56d6
Prepare main branch for next 30.0.0 release. (#15707) 2024-01-23 15:55:54 +05:30
Zoltan Haindrich d6a12c4389
Add ability to enable ResultCache in tests (#15465) 2024-01-22 09:02:59 -05:00
zachjsh 9d4e8053a4
Kinesis adaptive memory management (#15360)
### Description

Our Kinesis consumer works by using the [GetRecords API](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html) in some number of `fetchThreads`, each fetching some number of records (`recordsPerFetch`) and each inserting into a shared buffer that can hold a `recordBufferSize` number of records. The logic is described in our documentation at: https://druid.apache.org/docs/27.0.0/development/extensions-core/kinesis-ingestion/#determine-fetch-settings 

There is a problem with the logic that this pr fixes: the memory limits rely on a hard-coded “estimated record size” that is `10 KB` if `deaggregate: false` and `1 MB` if `deaggregate: true`. There have been cases where a supervisor had `deaggregate: true` set even though it wasn’t needed, leading to under-utilization of memory and poor ingestion performance.

Users don’t always know if their records are aggregated or not. Also, even if they could figure it out, it’s better to not have to. So we’d like to eliminate the `deaggregate` parameter, which means we need to do memory management more adaptively based on the actual record sizes.

We take advantage of the fact that GetRecords doesn’t return more than 10MB (https://docs.aws.amazon.com/streams/latest/dev/service-sizes-and-limits.html ):

This pr: 

eliminates `recordsPerFetch`, always use the max limit of 10000 records (the default limit if not set)

eliminate `deaggregate`, always have it true

cap `fetchThreads` to ensure that if each fetch returns the max (`10MB`) then we don't exceed our budget (`100MB` or `5% of heap`). In practice this means `fetchThreads` will never be more than `10`. Tasks usually don't have that many processors available to them anyway, so in practice I don't think this will change the number of threads for too many deployments

add `recordBufferSizeBytes` as a bytes-based limit rather than records-based limit for the shared queue. We do know the byte size of kinesis records by at this point. Default should be `100MB` or `10% of heap`, whichever is smaller.

add `maxBytesPerPoll` as a bytes-based limit for how much data we poll from shared buffer at a time. Default is `1000000` bytes.

deprecate `recordBufferSize`, use `recordBufferSizeBytes` instead. Warning is logged if `recordBufferSize` is specified

deprecate `maxRecordsPerPoll`, use `maxBytesPerPoll` instead. Warning is logged if maxRecordsPerPoll` is specified

Fixed issue that when the record buffer is full, the fetchRecords logic throws away the rest of the GetRecords result after `recordBufferOfferTimeout` and starts a new shard iterator. This seems excessively churny. Instead,  wait an unbounded amount of time for queue to stop being full. If the queue remains full, we’ll end up right back waiting for it after the restarted fetch.

There was also a call to `newQ::offer` without check in `filterBufferAndResetBackgroundFetch`, which seemed like it could cause data loss. Now checking return value here, and failing if false.

### Release Note

Kinesis ingestion memory tuning config has been greatly simplified, and a more adaptive approach is now taken for the configuration. Here is a summary of the changes made:

eliminates `recordsPerFetch`, always use the max limit of 10000 records (the default limit if not set)

eliminate `deaggregate`, always have it true

cap `fetchThreads` to ensure that if each fetch returns the max (`10MB`) then we don't exceed our budget (`100MB` or `5% of heap`). In practice this means `fetchThreads` will never be more than `10`. Tasks usually don't have that many processors available to them anyway, so in practice I don't think this will change the number of threads for too many deployments

add `recordBufferSizeBytes` as a bytes-based limit rather than records-based limit for the shared queue. We do know the byte size of kinesis records by at this point. Default should be `100MB` or `10% of heap`, whichever is smaller.

add `maxBytesPerPoll` as a bytes-based limit for how much data we poll from shared buffer at a time. Default is `1000000` bytes.

deprecate `recordBufferSize`, use `recordBufferSizeBytes` instead. Warning is logged if `recordBufferSize` is specified

deprecate `maxRecordsPerPoll`, use `maxBytesPerPoll` instead. Warning is logged if maxRecordsPerPoll` is specified
2024-01-19 14:30:21 -05:00