* Hook up PodTemplateTaskAdapter
* Make task adapter TYPE parameters final
* Rename adapters types
* Include specified adapter name in exception message
* Documentation for sidecarSupport deprecation
* Fix order
* Set TASK_ID as environment variable in PodTemplateTaskAdapter (#13969)
* Update docs/development/extensions-contrib/k8s-jobs.md
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
* Hook up PodTemplateTaskAdapter
* Make task adapter TYPE parameters final
* Rename adapters types
* Include specified adapter name in exception message
* Documentation for sidecarSupport deprecation
* Fix order
* fix spelling errors
---------
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
* Refactoring and bug fixes on top of unnest. The filter now is passed inside the unnest cursors. Added tests for scenarios such as
1. filter on unnested column which involves a left filter rewrite
2. filter on unnested virtual column which pushes the filter to the right only and involves no rewrite
3. not filters
4. SQL functions applied on top of unnested column
5. null present in first row of the column to be unnested
* Pod template task adapter
* Use getBaseTaskDirPaths
* Remove unused task from getEnv
* Use Optional.ifPresent() instead of Optional.map()
* Pass absolute path
* Don't pass task to getEnv
* Assert the correct adapter is created
* Javadocs and Comments
* Add exception message to assertions
* Add segment generator counters to reports
* Remove unneeded annotation
* Fix checkstyle and coverage
* Add persist and merged as new metrics
* Address review comments
* Fix checkstyle
* Create metrics class to handle updating counters
* Address review comments
* Add rowsPushed as a new metrics
changes:
* fixes inconsistent handling of byte[] values between ExprEval.bestEffortOf and ExprEval.ofType, which could cause byte[] values to end up as java toString values instead of base64 encoded strings in ingest time transforms
* improved ExpressionTransform binding to re-use ExprEval.bestEffortOf when evaluating a binding instead of throwing it away
* improved ExpressionTransform array handling, added RowFunction.evalDimension that returns List<String> to back Row.getDimension and remove the automatic coercing of array types that would typically happen to expression transforms unless using Row.getDimension
* added some tests for ExpressionTransform with array inputs
* improved ExpressionPostAggregator to use partial type information from decoration
* migrate some test uses of InputBindings.forMap to use other methods
Changes:
- Set `useRoundRobinSegmentAssignment` in coordinator dynamic config to `true` by default.
- Set `batchSegmentAllocation` in `TaskLockConfig` (used in Overlord runtime properties) to `true` by default.
Broken test appears unrelated to this PR
* make druidapi pip installable
* include druidapi in prerequisites
* add license to setup.py
* updates from Paul's review
* note about editable install
* Apply suggestions from code review
Co-authored-by: 317brian <53799971+317brian@users.noreply.github.com>
* update install instructions
* found unrelated typos
* standardize install cmd with pip
---------
Co-authored-by: 317brian <53799971+317brian@users.noreply.github.com>
* Lower default maxRowsInMemory for realtime ingestion.
The thinking here is that for best ingestion throughput, we want
intermediate persists to be as big as possible without using up all
available memory. So, we rely mainly on maxBytesInMemory. The default
maxRowsInMemory (1 million) is really just a safety: in case we have
a large number of very small rows, we don't want to get overwhelmed
by per-row overheads.
However, maximum ingestion throughput isn't necessarily the primary
goal for realtime ingestion. Query performance is also important. And
because query performance is not as good on the in-memory dataset, it's
helpful to keep it from growing too large. 150k seems like a reasonable
balance here. It means that for a typical 5 million row segment, we
won't trigger more than 33 persists due to this limit, which is a
reasonable number of persists.
* Update tests.
* Update server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
* Fix test.
* Fix link.
---------
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
* This makes the zookeeper connection retry count configurable. This is presently hardcoded to 29 tries which ends up taking a long time for the druid node to shutdown in case of ZK connectivity loss.
Having a shorter retry count helps k8s deployments to fail fast. In situations where the underlying k8s node loses network connectivity or is no longer able to talk to zookeeper, failing fast can trigger pod restarts which can then reassign the pod to a healthy k8s node.
Existing behavior is preserved, but users can override this property if needed.
* Removing intermediateSuperSorterStorageMaxLocalBytes, maxInputBytesPerWorker, composedIntermediateSuperSorterStorageEnabled, clusterStatisticsMergeMode from docs
* Adding documentation in the context class.
*
1. Handling deletion/creation of container created during the previously run test in AzureTestUtil.java.
2. Adding/updating log messages and comments in Azure and GCS deep storage tests.
* Adds new implementation of 'frontCoded' string encoding strategy, which writes out a v1 FrontCodedIndexed which stores buckets on a prefix of the previous value instead of the first value in the bucket
* Refactoring and bug fixes on top of unnest. The filter now is passed inside the unnest cursors. Added tests for scenarios such as
1. filter on unnested column which involves a left filter rewrite
2. filter on unnested virtual column which pushes the filter to the right only and involves no rewrite
3. not filters
4. SQL functions applied on top of unnested column
5. null present in first row of the column to be unnested
* Various changes and fixes to UNNEST.
Native changes:
1) UnnestDataSource: Replace "column" and "outputName" with "virtualColumn".
This enables pushing expressions into the datasource. This in turn
allows us to do the next thing...
2) UnnestStorageAdapter: Logically apply query-level filters and virtual
columns after the unnest operation. (Physically, filters are pulled up,
when possible.) This is beneficial because it allows filters and
virtual columns to reference the unnested column, and because it is
consistent with how the join datasource works.
3) Various documentation updates, including declaring "unnest" as an
experimental feature for now.
SQL changes:
1) Rename DruidUnnestRel (& Rule) to DruidUnnestRel (& Rule). The rel
is simplified: it only handles the UNNEST part of a correlated join.
Constant UNNESTs are handled with regular inline rels.
2) Rework DruidCorrelateUnnestRule to focus on pulling Projects from
the left side up above the Correlate. New test testUnnestTwice verifies
that this works even when two UNNESTs are stacked on the same table.
3) Include ProjectCorrelateTransposeRule from Calcite to encourage
pushing mappings down below the left-hand side of the Correlate.
4) Add a new CorrelateFilterLTransposeRule and CorrelateFilterRTransposeRule
to handle pulling Filters up above the Correlate. New tests
testUnnestWithFiltersOutside and testUnnestTwiceWithFilters verify
this behavior.
5) Require a context feature flag for SQL UNNEST, since it's undocumented.
As part of this, also cleaned up how we handle feature flags in SQL.
They're now hooked into EngineFeatures, which is useful because not
all engines support all features.
You can now add additional configuration files to be copied to the final conf directory on startup when running in a containerized environment. Useful for running on Kubernetes and needing to add more files with a config map. To specify the path where the configMap is mounted, utilize the DRUID_ADDITIONAL_CONF_DIR environment variable.
* As a follow up to #13893, this PR improves the comments added along with examples for the code, as well as adds handling for an edge case where the generated tombstone boundaries were overshooting the bounds of MIN_TIME (or MAX_TIME).
With SuperSorter using the PartitionedOutputChannels for sorting, it might OOM on inputs of reasonable size because the channel consists of both the writable frame channel and the frame allocator, both of which are not required once the output channel has been written to.
This change adds a readOnly to the output channel which contains only the readable channel, due to which unnecessary memory references to the writable channel and the memory allocator are lost once the output channel has been written to, preventing the OOM.
* Window planning: use collation traits, improve subquery logic.
SQL changes:
1) Attach RelCollation (sorting) trait to any PartialDruidQuery
that ends in AGGREGATE or AGGREGATE_PROJECT. This allows planning to
take advantage of the fact that Druid sorts by dimensions when
doing aggregations.
2) Windowing: inspect RelCollation trait from input, and insert naiveSort
if, and only if, necessary.
3) Windowing: add support for Project after Window, when the Project
is a simple mapping. Helps eliminate subqueries.
4) DruidRules: update logic for considering subqueries to reflect that
subqueries are not required to be GroupBys, and that we have a bunch
of new Stages now. With all of this evolution that has happened, the
old logic didn't quite make sense.
Native changes:
1) Use merge sort (stable) rather than quicksort when sorting
RowsAndColumns. Makes it easier to write test cases for plans that
involve re-sorting the data.
* Changes from review.
* Mark the bad test as failing.
* Additional update.
* Fix failingTest.
* Fix tests.
* Mark a var final.
* 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.
* use custom case operator conversion instead of direct operator conversion, to produce native nvl expression for SQL NVL and 2 argument COALESCE, and add optimization for certain case filters from coalesce and nvl statements
* Sort-merge join and hash shuffles for MSQ.
The main changes are in the processing, multi-stage-query, and sql modules.
processing module:
1) Rename SortColumn to KeyColumn, replace boolean descending with KeyOrder.
This makes it nicer to model hash keys, which use KeyOrder.NONE.
2) Add nullability checkers to the FieldReader interface, and an
"isPartiallyNullKey" method to FrameComparisonWidget. The join
processor uses this to detect null keys.
3) Add WritableFrameChannel.isClosed and OutputChannel.isReadableChannelReady
so callers can tell which OutputChannels are ready for reading and which
aren't.
4) Specialize FrameProcessors.makeCursor to return FrameCursor, a random-access
implementation. The join processor uses this to rewind when it needs to
replay a set of rows with a particular key.
5) Add MemoryAllocatorFactory, which is embedded inside FrameWriterFactory
instead of a particular MemoryAllocator. This allows FrameWriterFactory
to be shared in more scenarios.
multi-stage-query module:
1) ShuffleSpec: Add hash-based shuffles. New enum ShuffleKind helps callers
figure out what kind of shuffle is happening. The change from SortColumn
to KeyColumn allows ClusterBy to be used for both hash-based and sort-based
shuffling.
2) WorkerImpl: Add ability to handle hash-based shuffles. Refactor the logic
to be more readable by moving the work-order-running code to the inner
class RunWorkOrder, and the shuffle-pipeline-building code to the inner
class ShufflePipelineBuilder.
3) Add SortMergeJoinFrameProcessor and factory.
4) WorkerMemoryParameters: Adjust logic to reserve space for output frames
for hash partitioning. (We need one frame per partition.)
sql module:
1) Add sqlJoinAlgorithm context parameter; can be "broadcast" or
"sortMerge". With native, it must always be "broadcast", or it's a
validation error. MSQ supports both. Default is "broadcast" in
both engines.
2) Validate that MSQs do not use broadcast join with RIGHT or FULL join,
as results are not correct for broadcast join with those types. Allow
this in native for two reasons: legacy (the docs caution against it,
but it's always been allowed), and the fact that it actually *does*
generate correct results in native when the join is processed on the
Broker. It is much less likely that MSQ will plan in such a way that
generates correct results.
3) Remove subquery penalty in DruidJoinQueryRel when using sort-merge
join, because subqueries are always required, so there's no reason
to penalize them.
4) Move previously-disabled join reordering and manipulation rules to
FANCY_JOIN_RULES, and enable them when using sort-merge join. Helps
get to better plans where projections and filters are pushed down.
* Work around compiler problem.
* Updates from static analysis.
* Fix @param tag.
* Fix declared exception.
* Fix spelling.
* Minor adjustments.
* wip
* Merge fixups
* fixes
* Fix CalciteSelectQueryMSQTest
* Empty keys are sortable.
* Address comments from code review. Rename mux -> mix.
* Restore inspection config.
* Restore original doc.
* Reorder imports.
* Adjustments
* Fix.
* Fix imports.
* Adjustments from review.
* Update header.
* Adjust docs.