* 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
### 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
* Undocument unused segments retrieval API.
* Mark API deprecated and unstable. Note that it'll be removed.
* Cleanup .spelling entries
* Remove the Unstable annotation
* Add SpectatorHistogram extension
* Clarify documentation
Cleanup comments
* Use ColumnValueSelector directly
so that we support being queried as a Number using longSum or doubleSum aggregators as well as a histogram.
When queried as a Number, we're returning the count of entries in the histogram.
* Apply suggestions from code review
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
* Fix references
* Fix spelling
* Update docs/development/extensions-contrib/spectator-histogram.md
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
---------
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
### Description
This pr adds an api for retrieving unused segments for a particular datasource. The api supports pagination by the addition of `limit` and `lastSegmentId` parameters. The resulting unused segments are returned with optional `sortOrder`, `ASC` or `DESC` with respect to the matching segments `id`, `start time`, and `end time`, or not returned in any guarenteed order if `sortOrder` is not specified
`GET /druid/coordinator/v1/datasources/{dataSourceName}/unusedSegments?interval={interval}&limit={limit}&lastSegmentId={lastSegmentId}&sortOrder={sortOrder}`
Returns a list of unused segments for a datasource in the cluster contained within an optionally specified interval.
Optional parameters for limit and lastSegmentId can be given as well, to limit results and enable paginated results.
The results may be sorted in either ASC, or DESC order depending on specifying the sortOrder parameter.
`dataSourceName`: The name of the datasource
`interval`: the specific interval to search for unused segments for.
`limit`: the maximum number of unused segments to return information about. This property helps to
support pagination
`lastSegmentId`: the last segment id from which to search for results. All segments returned are > this segment
lexigraphically if sortOrder is null or ASC, or < this segment lexigraphically if sortOrder is DESC.
`sortOrder`: Specifies the order with which to return the matching segments by start time, end time. A null
value indicates that order does not matter.
This PR has:
- [x] been self-reviewed.
- [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
- [x] added documentation for new or modified features or behaviors.
- [ ] a release note entry in the PR description.
- [x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
- [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
- [x] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
- [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
- [ ] added integration tests.
- [x] been tested in a test Druid cluster.
This PR revives #14978 with a few more bells and whistles. Instead of an unconditional cross-join, we will now split the join condition such that some conditions are now evaluated post-join. To decide what sub-condition goes where, I have refactored DruidJoinRule class to extract unsupported sub-conditions. We build a postJoinFilter out of these unsupported sub-conditions and push to the join.
This patch introduces a param snapshotTime in the iceberg inputsource spec that allows the user to ingest data files associated with the most recent snapshot as of the given time. This helps the user ingest data based on older snapshots by specifying the associated snapshot time.
This patch also upgrades the iceberg core version to 1.4.1
* Add system fields to input sources.
Main changes:
1) The SystemField enum defines system fields "__file_uri", "__file_path",
and "__file_bucket". They are associated with each input entity.
2) The SystemFieldInputSource interface can be added to any InputSource
to make it system-field-capable. It sets up serialization of a list
of configured "systemFields" in the JSON form of the input source, and
provides a method getSystemFieldValue for computing the value of each
system field. Cloud object, HDFS, HTTP, and Local now have this.
* Fix various LocalInputSource calls.
* Fix style stuff.
* Fixups.
* Fix tests and coverage.
* better documentation for the differences between arrays and mvds
* add outputType to ExpressionPostAggregator to make docs true
* add output coercion if outputType is defined on ExpressionPostAgg
* updated post-aggregations.md to be consistent with aggregations.md and filters.md and use tables
* SQL: Plan non-equijoin conditions as cross join followed by filter.
Druid has previously refused to execute joins with non-equality-based
conditions. This was well-intentioned: the idea was to push people to
write their queries in a different, hopefully more performant way.
But as we're moving towards fuller SQL support, it makes more sense to
allow these conditions to go through with the best plan we can come up
with: a cross join followed by a filter. In some cases this will allow
the query to run, and people will be happy with that. In other cases,
it will run into resource limits during execution. But we should at
least give the query a chance.
This patch also updates the documentation to explain how people can
tell whether their queries are being planned this way.
* cartesian is a word.
* Adjust tests.
* Update docs/querying/datasource.md
Co-authored-by: Benedict Jin <asdf2014@apache.org>
---------
Co-authored-by: Benedict Jin <asdf2014@apache.org>
Currently, after an MSQ query, the web console is responsible for waiting for the segments to load. It does so by checking if there are any segments loading into the datasource ingested into, which can cause some issues, like in cases where the segments would never be loaded, or would end up waiting for other ingests as well.
This PR shifts this responsibility to the controller, which would have the list of segments created.
* Add new configurable buffer period to create gap between mark unused and kill of segment
* Changes after testing
* fixes and improvements
* changes after initial self review
* self review changes
* update sql statement that was lacking last_used
* shore up some code in SqlMetadataConnector after self review
* fix derby compatibility and improve testing/docs
* fix checkstyle violations
* Fixes post merge with master
* add some unit tests to improve coverage
* ignore test coverage on new UpdateTools cli tool
* another attempt to ignore UpdateTables in coverage check
* change column name to used_flag_last_updated
* fix a method signature after column name switch
* update docs spelling
* Update spelling dictionary
* Fixing up docs/spelling and integrating altering tasks table with my alteration code
* Update NULL values for used_flag_last_updated in the background
* Remove logic to allow segs with null used_flag_last_updated to be killed regardless of bufferPeriod
* remove unneeded things now that the new column is automatically updated
* Test new background row updater method
* fix broken tests
* fix create table statement
* cleanup DDL formatting
* Revert adding columns to entry table by default
* fix compilation issues after merge with master
* discovered and fixed metastore inserts that were breaking integration tests
* fixup forgotten insert by using pattern of sharing now timestamp across columns
* fix issue introduced by merge
* fixup after merge with master
* add some directions to docs in the case of segment table validation issues
* Add supervisor /resetOffsets API.
- Add a new endpoint /druid/indexer/v1/supervisor/<supervisorId>/resetOffsets
which accepts DataSourceMetadata as a body parameter.
- Update logs, unit tests and docs.
* Add a new interface method for backwards compatibility.
* Rename
* Adjust tests and javadocs.
* Use CoreInjectorBuilder instead of deprecated makeInjectorWithModules
* UT fix
* Doc updates.
* remove extraneous debugging logs.
* Remove the boolean setting; only ResetHandle() and resetInternal()
* Relax constraints and add a new ResetOffsetsNotice; cleanup old logic.
* A separate ResetOffsetsNotice and some cleanup.
* Minor cleanup
* Add a check & test to verify that sequence numbers are only of type SeekableStreamEndSequenceNumbers
* Add unit tests for the no op implementations for test coverage
* CodeQL fix
* checkstyle from merge conflict
* Doc changes
* DOCUSAURUS code tabs fix. Thanks, Brian!
In this PR, I have gotten rid of multiTopic parameter and instead added a topicPattern parameter. Kafka supervisor will pass topicPattern or topic as the stream name to the core ingestion engine. There is validation to ensure that only one of topic or topicPattern will be set. This new setting is easier to understand than overloading the topic field that earlier could be interpreted differently depending on the value of some other field.
This adds a new contrib extension: druid-iceberg-extensions which can be used to ingest data stored in Apache Iceberg format. It adds a new input source of type iceberg that connects to a catalog and retrieves the data files associated with an iceberg table and provides these data file paths to either an S3 or HDFS input source depending on the warehouse location.
Two important dependencies associated with Apache Iceberg tables are:
Catalog : This extension supports reading from either a Hive Metastore catalog or a Local file-based catalog. Support for AWS Glue is not available yet.
Warehouse : This extension supports reading data files from either HDFS or S3. Adapters for other cloud object locations should be easy to add by extending the AbstractInputSourceAdapter.
* Add aggregatorMergeStrategy property to SegmentMetadaQuery.
- Adds a new property aggregatorMergeStrategy to segmentMetadata query.
aggregatorMergeStrategy currently supports three types of merge strategies -
the legacy strict and lenient strategies, and the new latest strategy.
- The latest strategy considers the latest aggregator from the latest segment
by time order when there's a conflict when merging aggregators from different
segments.
- Deprecate lenientAggregatorMerge property; The API validates that both the new
and old properties are not set, and returns an exception.
- When merging segments as part of segmentMetadata query, the segments have a more
elaborate id -- <datasource>_<interval>_merged_<partition_number> format, similar to
the name format that segments usually contain. Previously it was simply "merged".
- Adjust unit tests to test the latest strategy, to assert the returned complete
SegmentAnalysis object instead of just the aggregators for completeness.
* Don't explicitly set strict strategy in tests
* Apply suggestions from code review
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
* Update docs/querying/segmentmetadataquery.md
* Apply suggestions from code review
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
---------
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
Description:
Druid allows a configuration of load rules that may cause a used segment to not be loaded
on any historical. This status is not tracked in the sys.segments table on the broker, which
makes it difficult to determine if the unavailability of a segment is expected and if we should
not wait for it to be loaded on a server after ingestion has finished.
Changes:
- Track replication factor in `SegmentReplicantLookup` during evaluation of load rules
- Update API `/druid/coordinator/v1metadata/segments` to return replication factor
- Add column `replication_factor` to the sys.segments virtual table and populate it in
`MetadataSegmentView`
- If this column is 0, the segment is not assigned to any historical and will not be loaded.
* Add INFORMATION_SCHEMA.ROUTINES to expose Druid operators and functions.
* checkstyle
* remove IS_DETERMISITIC.
* test
* cleanup test
* remove logs and simplify
* fixup unit test
* Add docs for INFORMATION_SCHEMA.ROUTINES table.
* Update test and add another SQL query.
* add stuff to .spelling and checkstyle fix.
* Add more tests for custom operators.
* checkstyle and comment.
* Some naming cleanup.
* Add FUNCTION_ID
* The different Calcite function syntax enums get translated to FUNCTION
* Update docs.
* Cleanup markdown table.
* fixup test.
* fixup intellij inspection
* Review comment: nullable column; add a function to determine function syntax.
* More tests; add non-function syntax operators.
* More unit tests. Also add a separate test for DruidOperatorTable.
* actually just validate non-zero count.
* switch up the order
* checkstyle fixes.
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
Co-authored-by: Victoria Lim <lim.t.victoria@gmail.com>
* Be able to load segments on Peons
This change introduces a new config on WorkerConfig
that indicates how many bytes of each storage
location to use for storage of a task. Said config
is divided up amongst the locations and slots
and then used to set TaskConfig.tmpStorageBytesPerTask
The Peons use their local task dir and
tmpStorageBytesPerTask as their StorageLocations for
the SegmentManager such that they can accept broadcast
segments.
* Allow for Log4J to be configured for peons but still ensure console logging is enforced
This change will allow for log4j to be configured for peons but require console logging is still
configured for them to ensure peon logs are saved to deep storage.
Also fixed the test ConsoleLoggingEnforcementTest to use a valid appender for the non console
Config as the previous config was incorrect and would never return a logger.
* fix checkstyle
* add warning to logger when it overwrites all loggers to be console
* optimize calls for altering logging config for ConsoleLoggingEnforcementConfigurationFactory
add getName to the druid logger class
* update docs, and error message
* edit docs to be more clear
* fix checkstyle issues
* CI fixes - LoggerTest code coverage and fix spelling issue for logging docs
* "maxResultsSize" has been removed from the S3OutputConfig and a default "chunkSize" of 100MiB is now present. This change primarily affects users who wish to use durable storage for MSQ jobs.
This PR is a follow-up to #13819 so that the Tuple sketch functionality can be used in SQL for both ingestion using Multi-Stage Queries (MSQ) and also for analytic queries against Tuple sketch columns.
Document how to report security issues on the security overview page, so we can link this page from the homepage. That should make all the other important security information easier to find as well.
Expands the OIDC based auth in Druid by adding a JWT Authenticator that validates ID Tokens associated with a request. The existing pac4j authenticator works for authenticating web users while accessing the console, whereas this authenticator is for validating Druid API requests made by Direct clients. Services already supporting OIDC can attach their ID tokens to the Druid requests
under the Authorization request header.
* Improve memory efficiency of WrappedRoaringBitmap.
Two changes:
1) Use an int[] for sizes 4 or below.
2) Remove the boolean compressRunOnSerialization. Doesn't save much
space, but it does save a little, and it isn't adding a ton of value
to have it be configurable. It was originally configurable in case
anything broke when enabling it, but it's been a while and nothing
has broken.
* Slight adjustment.
* Adjust for inspection.
* Updates.
* Update snaps.
* Update test.
* Adjust test.
* Fix snaps.
* Fix lookup docs
* Fix spelling
* Apply suggestions from code review
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
---------
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
Much improved table functions
* Revises properties, definitions in the catalog
* Adds a "table function" abstraction to model such functions
* Specific functions for HTTP, inline, local and S3.
* Extended SQL types in the catalog
* Restructure external table definitions to use table functions
* EXTEND syntax for Druid's extern table function
* Support for array-valued table function parameters
* Support for array-valued SQL query parameters
* Much new documentation
* Zero-copy local deep storage.
This is useful for local deep storage, since it reduces disk usage and
makes Historicals able to load segments instantaneously.
Two changes:
1) Introduce "druid.storage.zip" parameter for local storage, which defaults
to false. This changes default behavior from writing an index.zip to writing
a regular directory. This is safe to do even during a rolling update, because
the older code actually already handled unzipped directories being present
on local deep storage.
2) In LocalDataSegmentPuller and LocalDataSegmentPusher, use hard links
instead of copies when possible. (Generally this is possible when the
source and destination directory are on the same filesystem.)
* Druid automated quickstart
* remove conf/druid/single-server/quickstart/_common/historical/jvm.config
* Minor changes in python script
* Add lower bound memory for some services
* Additional runtime properties for services
* Update supervise script to accept command arguments, corresponding changes in druid-quickstart.py
* File end newline
* Limit the ability to start multiple instances of a service, documentation changes
* simplify script arguments
* restore changes in medium profile
* run-druid refactor
* compute and pass middle manager runtime properties to run-druid
supervise script changes to process java opts array
use argparse, leave free memory, logging
* Remove extra quotes from mm task javaopts array
* Update logic to compute minimum memory
* simplify run-druid
* remove debug options from run-druid
* resolve the config_path provided
* comment out service specific runtime properties which are computed in the code
* simplify run-druid
* clean up docs, naming changes
* Throw ValueError exception on illegal state
* update docs
* rename args, compute_only -> compute, run_zk -> zk
* update help documentation
* update help documentation
* move task memory computation into separate method
* Add validation checks
* remove print
* Add validations
* remove start-druid bash script, rename start-druid-main
* Include tasks in lower bound memory calculation
* Fix test
* 256m instead of 256g
* caffeine cache uses 5% of heap
* ensure min task count is 2, task count is monotonic
* update configs and documentation for runtime props in conf/druid/single-server/quickstart
* Update docs
* Specify memory argument for each profile in single-server.md
* Update middleManager runtime.properties
* Move quickstart configs to conf/druid/base, add bash launch script, support python2
* Update supervise script
* rename base config directory to auto
* rename python script, changes to pass repeated args to supervise
* remove exmaples/conf/druid/base dir
* add docs
* restore changes in conf dir
* update start-druid-auto
* remove hashref for commands in supervise script
* start-druid-main java_opts array is comma separated
* update entry point script name in python script
* Update help docs
* documentation changes
* docs changes
* update docs
* add support for running indexer
* update supported services list
* update help
* Update python.md
* remove dir
* update .spelling
* Remove dependency on psutil and pathlib
* update docs
* Update get_physical_memory method
* Update help docs
* update docs
* update method to get physical memory on python
* udpate spelling
* update .spelling
* minor change
* Minor change
* memory comptuation for indexer
* update start-druid
* Update python.md
* Update single-server.md
* Update python.md
* run python3 --version to check if python is installed
* Update supervise script
* start-druid: echo message if python not found
* update anchor text
* minor change
* Update condition in supervise script
* JVM not jvm in docs
* initial commit for jdbc tutorial
(cherry picked from commit 04c4adad71e5436b76c3425fe369df03aaaf0acb)
* add commentary
* address comments from charles
* add query context to example
* fix typo
* add links
* Apply suggestions from code review
Co-authored-by: Frank Chen <frankchen@apache.org>
* fix datatype
* address feedback
* add parameterize to spelling file. the past tense version was already there
Co-authored-by: Frank Chen <frankchen@apache.org>
* Use standard library to correctly glob and stop at the correct folder structure when filtering cloud objects.
Removed:
import org.apache.commons.io.FilenameUtils;
Add:
import java.nio.file.FileSystems;
import java.nio.file.PathMatcher;
import java.nio.file.Paths;
* Forgot to update CloudObjectInputSource as well.
* Fix tests.
* Removed unused exceptions.
* Able to reduced user mistakes, by removing the protocol and the bucket on filter.
* add 1 more test.
* add comment on filterWithoutProtocolAndBucket
* Fix lint issue.
* Fix another lint issue.
* Replace all mention of filter -> objectGlob per convo here:
https://github.com/apache/druid/pull/13027#issuecomment-1266410707
* fix 1 bad constructor.
* Fix the documentation.
* Don’t do anything clever with the object path.
* Remove unused imports.
* Fix spelling error.
* Fix incorrect search and replace.
* Addressing Gian’s comment.
* add filename on .spelling
* Fix documentation.
* fix documentation again
Co-authored-by: Didip Kerabat <didip@apple.com>
* Fix typo
* Fix some spacing
* Add missing fields
* Cleanup table spacing
* Remove durable storage docs again
Thanks Brian for pointing out previous discussions.
* Update docs/multi-stage-query/reference.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Mark codes as code
* And even more codes as code
* Another set of spaces
* Combine `ColumnTypeNotSupported`
Thanks Karan.
* More whitespaces and typos
* Add spelling and fix links
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Compaction: Fetch segments one at a time on main task; skip when possible.
Compact tasks include the ability to fetch existing segments and determine
reasonable defaults for granularitySpec, dimensionsSpec, and metricsSpec.
This is a useful feature that makes compact tasks work well even when the
user running the compaction does not have a clear idea of what they want
the compacted segments to be like.
However, this comes at a cost: it takes time, and disk space, to do all
of these fetches. This patch improves the situation in two ways:
1) When segments do need to be fetched, download them one at a time and
delete them when we're done. This still takes time, but minimizes the
required disk space.
2) Don't fetch segments on the main compact task when they aren't needed.
If the user provides a full granularitySpec, dimensionsSpec, and
metricsSpec, we can skip it.
* Adjustments.
* Changes from code review.
* Fix logic for determining rollup.
* Support for middle manager less druid, tasks launch as k8s jobs
* Fixing forking task runner test
* Test cleanup, dependency cleanup, intellij inspections cleanup
* Changes per PR review
Add configuration option to disable http/https proxy for the k8s client
Update the docs to provide more detail about sidecar support
* Removing un-needed log lines
* Small changes per PR review
* Upon task completion we callback to the overlord to update the status / locaiton, for slower k8s clusters, this reduces locking time significantly
* Merge conflict fix
* Fixing tests and docs
* update tiny-cluster.yaml
changed `enableTaskLevelLogPush` to `encapsulatedTask`
* Apply suggestions from code review
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
* Minor changes per PR request
* Cleanup, adding test to AbstractTask
* Add comment in peon.sh
* Bumping code coverage
* More tests to make code coverage happy
* Doh a duplicate dependnecy
* Integration test setup is weird for k8s, will do this in a different PR
* Reverting back all integration test changes, will do in anotbher PR
* use StringUtils.base64 instead of Base64
* Jdk is nasty, if i compress in jdk 11 in jdk 17 the decompressed result is different
Co-authored-by: Rahul Gidwani <r_gidwani@apple.com>
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
* add FrontCodedIndexed for delta string encoding
* now for actual segments
* fix indexOf
* fixes and thread safety
* add bucket size 4, which seems generally better
* fixes
* fixes maybe
* update indexes to latest interfaces
* utf8 support
* adjust
* oops
* oops
* refactor, better, faster
* more test
* fixes
* revert
* adjustments
* fix prefixing
* more chill
* sql nested benchmark too
* refactor
* more comments and javadocs
* better get
* remove base class
* fix
* hot rod
* adjust comments
* faster still
* minor adjustments
* spatial index support
* spotbugs
* add isSorted to Indexed to strengthen indexOf contract if set, improve javadocs, add docs
* fix docs
* push into constructor
* use base buffer instead of copy
* oops