changes:
* Added `CursorBuildSpec` which captures all of the 'interesting' stuff that goes into producing a cursor as a replacement for the method arguments of `CursorFactory.canVectorize`, `CursorFactory.makeCursor`, and `CursorFactory.makeVectorCursor`
* added new interface `CursorHolder` and new interface `CursorHolderFactory` as a replacement for `CursorFactory`, with method `makeCursorHolder`, which takes a `CursorBuildSpec` as an argument and replaces `CursorFactory.canVectorize`, `CursorFactory.makeCursor`, and `CursorFactory.makeVectorCursor`
* `CursorFactory.makeCursors` previously returned a `Sequence<Cursor>` corresponding to the query granularity buckets, with a separate `Cursor` per bucket. `CursorHolder.asCursor` instead returns a single `Cursor` (equivalent to 'ALL' granularity), and a new `CursorGranularizer` has been added for query engines to iterate over the cursor and divide into granularity buckets. This makes the non-vectorized engine behave the same way as the vectorized query engine (with its `VectorCursorGranularizer`), and simplifies a lot of stuff that has to read segments particularly if it does not care about bucketing the results into granularities.
* Deprecated `CursorFactory`, `CursorFactory.canVectorize`, `CursorFactory.makeCursors`, and `CursorFactory.makeVectorCursor`
* updated all `StorageAdapter` implementations to implement `makeCursorHolder`, transitioned direct `CursorFactory` implementations to instead implement `CursorMakerFactory`. `StorageAdapter` being a `CursorMakerFactory` is intended to be a transitional thing, ideally will not be released in favor of moving `CursorMakerFactory` to be fetched directly from `Segment`, however this PR was already large enough so this will be done in a follow-up.
* updated all query engines to use `makeCursorHolder`, granularity based engines to use `CursorGranularizer`.
Two performance enhancements:
1) Direct merging of input frames to output channels, without any
temporary files, if all input frames fit in memory.
2) When doing multi-level merging (now called "external mode"),
improve parallelism by boosting up the number of mergers in the
penultimate level.
To support direct merging, FrameChannelMerger is enhanced such that the
output partition min/max values are used to filter input frames. This
is necessary because all direct mergers read all input frames, but only
rows corresponding to a single output partition.
Some general refactors across Druid.
Switch to DruidExceptions
Add javadocs
Fix a bug in IntArrayColumns
Add a class for LongArrayColumns
Remove wireTransferable since it would never be called
Refactor DictionaryWriter to return the index written as a return value from write.
Refactors the SemanticCreator annotation.
Moves the interface to the semantic package.
Create a SemanticUtils to hold logic for storing semantic maps.
Add FrameMaker interface.
This PR fixes query correctness issues for MSQ window functions when using more than 1 worker (that is, maxNumTasks > 2).
Currently, we were keeping the shuffle spec of the previous stage when we didn't have any partition columns for window stage. This PR changes it to override the shuffle spec of the previous stage to MixShuffleSpec (if we have a window function with empty over clause) so that the window stage gets a single partition to work on.
A test has been added for a query which returned incorrect results prior to this change when using more than 1 workers.
This patch introduces an optional cluster configuration, druid.indexing.formats.stringMultiValueHandlingMode, allowing operators to override the default mode SORTED_SET for string dimensions. The possible values for the config are SORTED_SET, SORTED_ARRAY, or ARRAY (SORTED_SET is the default). Case insensitive values are allowed.
While this cluster property allows users to manage the multi-value handling mode for string dimension types, it's recommended to migrate to using real array types instead of MVDs.
This fixes a long-standing issue where compaction will honor the configured cluster wide property instead of rewriting it as the default SORTED_ARRAY always, even if the data was originally ingested with ARRAY or SORTED_SET.
* MSQ worker: Support in-memory shuffles.
This patch is a follow-up to #16168, adding worker-side support for
in-memory shuffles. Changes include:
1) Worker-side code now respects the same context parameter "maxConcurrentStages"
that was added to the controller in #16168. The parameter remains undocumented
for now, to give us a chance to more fully develop and test this functionality.
1) WorkerImpl is broken up into WorkerImpl, RunWorkOrder, and RunWorkOrderListener
to improve readability.
2) WorkerImpl has a new StageOutputHolder + StageOutputReader concept, which
abstract over memory-based or file-based stage results.
3) RunWorkOrder is updated to create in-memory stage output channels when
instructed to.
4) ControllerResource is updated to add /doneReadingInput/, so the controller
can tell when workers that sort, but do not gather statistics, are done reading
their inputs.
5) WorkerMemoryParameters is updated to consider maxConcurrentStages.
Additionally, WorkerChatHandler is split into WorkerResource, so as to match
ControllerChatHandler and ControllerResource.
* Updates for static checks, test coverage.
* Fixes.
* Remove exception.
* Changes from review.
* Address static check.
* Changes from review.
* Improvements to docs and method names.
* Update comments, add test.
* Additional javadocs.
* Fix throws.
* Fix worker stopping in tests.
* Fix stuck test.
* Round-robin iterator for datasources to kill.
Currently there's a fairness problem in the KillUnusedSegments duty
where the duty consistently selects the same set of datasources as discovered
from the metadata store or dynamic config params. This is a problem especially
when there are multiple unused. In a medium to large cluster, while we can increase
the task slots to increase the likelihood of broader coverage. This patch adds a simple
round-robin iterator to select datasources and has the following properties:
1. Starts with an initial random cursor position in an ordered list of candidates.
2. Consecutive {@code next()} iterations from {@link #getIterator()} are guaranteed to be deterministic
unless the set of candidates change when {@link #updateCandidates(Set)} is called.
3. Guarantees that no duplicate candidates are returned in two consecutive {@code next()} iterations.
* Renames in RoundRobinIteratorTest.
* Address review comments.
1. Clarify javadocs on the ordered list. Also flesh out the details a bit more.
2. Rename the test hooks to make intent clearer and fix typo.
3. Add NotThreadSafe annotation.
4. Remove one potentially noisy log that's in the path of iteration.
* Add null check to input candidates.
* More commentary.
* Addres review feedback: downgrade some new info logs to debug; invert condition.
Remove redundant comments.
Remove rendundant variable tracking.
* CircularList adjustments.
* Updates to CircularList and cleanup RoundRobinInterator.
* One more case and add more tests.
* Make advanceCursor private for now.
* Review comments.
* Coerce COMPLEX to number in numeric aggregators.
PR #15371 eliminated ObjectColumnSelector's built-in implementations of
numeric methods, which had been marked deprecated.
However, some complex types, like SpectatorHistogram, can be successfully coerced
to number. The documentation for spectator histograms encourages taking advantage of
this by aggregating complex columns with doubleSum and longSum. Currently, this
doesn't work properly for IncrementalIndex, where the behavior relied on those
deprecated ObjectColumnSelector methods.
This patch fixes the behavior by making two changes:
1) SimpleXYZAggregatorFactory (XYZ = type; base class for simple numeric aggregators;
all of these extend NullableNumericAggregatorFactory) use getObject for STRING
and COMPLEX. Previously, getObject was only used for STRING.
2) NullableNumericAggregatorFactory (base class for simple numeric aggregators)
has a new protected method "useGetObject". This allows the base class to
correctly check for null (using getObject or isNull).
The patch also adds a test for SpectatorHistogram + doubleSum + IncrementalIndex.
* Fix tests.
* Remove the special ColumnValueSelector.
* Add test.
* HashJoinEngine: Check for interruptions while walking left cursor.
Previously, the engine only checked for interruptions between emitting
joined rows. In scenarios where large numbers of left rows are skipped
completely (such as a highly selective INNER JOIN) this led to the
join cursor being insufficiently responsive to cancellation.
* Coverage.
Changes the WindowFrame internals / representation a bit; introduces dedicated frametypes for rows and groups which corresponds to the implemented processing methods
* more aggressive cancellation of broker parallel merge, more chill blocking queue timeouts
* wire parallel merge into query cancellation system
* oops
* style
* adjust metrics initialization
* fix timeout, fix cleanup to not block
* javadocs to clarify why cancellation future and gizmo are split
* cancelled -> canceled, simplify QueuePusher since it always takes a ResultBatch, non-static terminal marker to make stuff stop complaining about types, specialize tryOffer to be tryOfferTerminal so it wont be misused, add comments to clarify reason for non-blocking offers that might fail
For aggregators like StringFirst/Last, whose intermediate type isn't the same as the final type, using them in GroupBy, TopN or Timeseries subqueries causes a fallback when maxSubqueryBytes is set. This is because we assume that the finalization is not known, due to which the row signature cannot determine whether to use the intermediate or the final type, and it puts it as null. This PR figures out the finalization from the query context and uses the intermediate or the final type appropriately.
changes:
* moves value column serializer initialization, call to `writeValue` method to `GlobalDictionaryEncodedFieldColumnWriter.writeTo` instead of during `GlobalDictionaryEncodedFieldColumnWriter.addValue`. This shift means these numeric value columns are now done in the per field section that happens after serializing the nested column raw data, so only a single compression buffer and temp file will be needed at a time instead of the total number of nested literal fields present in the column. This should be especially helpful for complicated nested structures with thousands of columns as even those 64k compression buffers can add up pretty quickly to a sizeable chunk of direct memory.
changes:
* removes `druid.indexer.task.batchProcessingMode` in favor of always using `CLOSED_SEGMENT_SINKS` which uses `BatchAppenderator`. This was intended to become the default for native batch, but that was missed so `CLOSED_SEGMENTS` was the default (using `AppenderatorImpl`), however MSQ has been exclusively using `BatchAppenderator` with no problems so it seems safe to just roll it out as the only option for batch ingestion everywhere.
* with `batchProcessingMode` gone, there is no use for `AppenderatorImpl` so it has been removed
* implify `Appenderator` construction since there are only separate stream and batch versions now
* simplify tests since `batchProcessingMode` is gone
This PR aims to check if the complex column being queried aligns with the supported types in the aggregator and aggregator factories, and throws a user-friendly error message if they don't.
* Throw exception if DISTINCT used with window functions aggregate call
* Improve error message when unsupported aggregations are used with window functions
changes:
* removed `Firehose` and `FirehoseFactory` and remaining implementations which were mostly no longer used after #16602
* Moved `IngestSegmentFirehose` which was still used internally by Hadoop ingestion to `DatasourceRecordReader.SegmentReader`
* Rename `SQLFirehoseFactoryDatabaseConnector` to `SQLInputSourceDatabaseConnector` and similar renames for sub-classes
* Moved anything remaining in a 'firehose' package somewhere else
* Clean up docs on firehose stuff
Changes:
- Do not hold a reference to `TaskQueue` in `TaskStorageQueryAdapter`
- Use `TaskStorage` instead of `TaskStorageQueryAdapter` in `IndexerMetadataStorageAdapter`
- Rename `TaskStorageQueryAdapter` to `TaskQueryTool`
- Fix newly added task actions `RetrieveUpgradedFromSegmentIds` and `RetrieveUpgradedToSegmentIds`
by removing `isAudited` method.
Description:
Task action audit logging was first deprecated and disabled by default in Druid 0.13, #6368.
As called out in the original discussion #5859, there are several drawbacks to persisting task action audit logs.
- Only usage of the task audit logs is to serve the API `/indexer/v1/task/{taskId}/segments`
which returns the list of segments created by a task.
- The use case is really narrow and no prod clusters really use this information.
- There can be better ways of obtaining this information, such as the metric
`segment/added/bytes` which reports both the segment ID and task ID
when a segment is committed by a task. We could also include committed segment IDs in task reports.
- A task persisting several segments would bloat up the audit logs table putting unnecessary strain
on metadata storage.
Changes:
- Remove `TaskAuditLogConfig`
- Remove method `TaskAction.isAudited()`. No task action is audited anymore.
- Remove `SegmentInsertAction` as it is not used anymore. `SegmentTransactionalInsertAction`
is the new incarnation which has been in use for a while.
- Deprecate `MetadataStorageActionHandler.addLog()` and `getLogs()`. These are not used anymore
but need to be retained for backward compatibility of extensions.
- Do not create `druid_taskLog` metadata table anymore.
* When an ArrayList RAC creates a child RAC, the start and end offsets need to have the offset of parent's start offset
* Defaults the 2nd window bound to CURRENT ROW when only a single bound is specified
* Removes the windowingStrictValidation warning and throws a hard exception when Order By alongside RANGE clause is not provided with UNBOUNDED or CURRENT ROW as both bounds
Better fallback strategy when the broker is unable to materialize the subquery's results as frames for estimating the bytes:
a. We don't touch the subquery sequence till we know that we can materialize the result as frames
Description:
Compaction operations issued by the Coordinator currently run using the native query engine.
As majority of the advancements that we are making in batch ingestion are in MSQ, it is imperative
that we support compaction on MSQ to make Compaction more robust and possibly faster.
For instance, we have seen OOM errors in native compaction that MSQ could have handled by its
auto-calculation of tuning parameters.
This commit enables compaction on MSQ to remove the dependency on native engine.
Main changes:
* `DataSourceCompactionConfig` now has an additional field `engine` that can be one of
`[native, msq]` with `native` being the default.
* if engine is MSQ, `CompactSegments` duty assigns all available compaction task slots to the
launched `CompactionTask` to ensure full capacity is available to MSQ. This is to avoid stalling which
could happen in case a fraction of the tasks were allotted and they eventually fell short of the number
of tasks required by the MSQ engine to run the compaction.
* `ClientCompactionTaskQuery` has a new field `compactionRunner` with just one `engine` field.
* `CompactionTask` now has `CompactionRunner` interface instance with its implementations
`NativeCompactinRunner` and `MSQCompactionRunner` in the `druid-multi-stage-query` extension.
The objectmapper deserializes `ClientCompactionRunnerInfo` in `ClientCompactionTaskQuery` to the
`CompactionRunner` instance that is mapped to the specified type [`native`, `msq`].
* `CompactTask` uses the `CompactionRunner` instance it receives to create the indexing tasks.
* `CompactionTask` to `MSQControllerTask` conversion logic checks whether metrics are present in
the segment schema. If present, the task is created with a native group-by query; if not, the task is
issued with a scan query. The `storeCompactionState` flag is set in the context.
* Each created `MSQControllerTask` is launched in-place and its `TaskStatus` tracked to determine the
final status of the `CompactionTask`. The id of each of these tasks is the same as that of `CompactionTask`
since otherwise, the workers will be unable to determine the controller task's location for communication
(as they haven't been launched via the overlord).
changes:
* fixes a bug with unnest storage adapter not preserving underlying columns dictionary uniqueness when allowing dimension selector cursor
* fixes a bug with unnest on realtime segments with empty rows incorrectly specifying index 0 as the row dictionary value
* fix equality and typed in filter behavior for numeric match values on string columns
changes:
* EqualityFilter and TypedInfilter numeric match values against string columns will now cast strings to numeric values instead of converting the numeric values directly to string for pure string equality, which is consistent with the casts which are eaten in the SQL layer, as well as classic druid behavior
* added tests to cover numeric equality matching. Double match values in particular would fail to match the string values since `1.0` would become `'1.0'` which does not match `'1'`.
Updated javadoc for `ColumnIndexSupplier.as` to elaborate on the types of indexes callers might want to ask for from the method, as well as help implementors know what kinds of indexes they should implement to participate in filtering
* Defer more expressions in vectorized groupBy.
This patch adds a way for columns to provide GroupByVectorColumnSelectors,
which controls how the groupBy engine operates on them. This mechanism is used
by ExpressionVirtualColumn to provide an ExpressionDeferredGroupByVectorColumnSelector
that uses the inputs of an expression as the grouping key. The actual expression
evaluation is deferred until the grouped ResultRow is created.
A new context parameter "deferExpressionDimensions" allows users to control when
this deferred selector is used. The default is "fixedWidthNonNumeric", which is a
behavioral change from the prior behavior. Users can get the prior behavior by setting
this to "singleString".
* Fix style.
* Add deferExpressionDimensions to SqlExpressionBenchmark.
* Fix style.
* Fix inspections.
* Add more testing.
* Use valueOrDefault.
* Compute exprKeyBytes a bit lighter-weight.
MSQ cannot process null bytes in string fields, and the current workaround is to remove them using the REPLACE function. 'removeNullBytes' context parameter has been added which sanitizes the input string fields by removing these null bytes.
Changes:
- Rename `UsedSegmentChecker` to `PublishedSegmentsRetriever`
- Remove deprecated single `Interval` argument from `RetrieveUsedSegmentsAction`
as it is now unused and has been deprecated since #1988
- Return `Set` of segments instead of a `Collection` from `IndexerMetadataStorageCoordinator.retrieveUsedSegments()`
* first pass
* more changes
* fix tests and formatting
* fix kinesis failing tests
* fix kafka tests
* add dimension name to float parse errors
* double and convertToType handling of dimensionName can report parse errors with dimension name
* fix checkstyle issue
* fix tests
* more cases to have better parse exception messages
* fix test
* fix tests
* partially address comments
* annotate method parameter with nullable
* address comments
* fix tests
* let float, double, long dimensionIndexer pass dimensionName down to dimensionHandlerUtils
* fix compilation error and clean up formatting
* clean up whitespace
* address feedback. undo change, pass down report parse exception for convertToType
* fix test
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>
* Initial support for bootstrap segments.
- Adds a new API in the coordinator.
- All processes that have storage locations configured (including tasks)
talk to the coordinator if they can, and fetch bootstrap segments from it.
- Then load the segments onto the segment cache as part of startup.
- This addresses the segment bootstrapping logic required by processes before
they can start serving queries or ingesting.
This patch also lays the foundation to speed up upgrades.
* Fail open by default if there are any errors talking to the coordinator.
* Add test for failure scenario and cleanup logs.
* Cleanup and add debug log
* Assert the events so we know the list exactly.
* Revert RunRules test.
The rules aren't evaluated if there are no clusters.
* Revert RunRulesTest too.
* Remove debug info.
* Make the API POST and update log.
* Fix up UTs.
* Throw 503 from MetadataResource; clean up exception handling and DruidException.
* Remove unused logger, add verification of metrics and docs.
* Update error message
* Update server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
* Apply suggestions from code review
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
* Adjust test metric expectations with the rename.
* Add BootstrapSegmentResponse container in the response for future extensibility.
* Rename to BootstrapSegmentsInfo for internal consistency.
* Remove unused log.
* Use a member variable for broadcast segments instead of segmentAssigner.
* Minor cleanup
* Add test for loadable bootstrap segments and clarify comment.
* Review suggestions.
---------
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
* Turn invalid periods into user-facing exception providing more context.
The current exception is targeting the ADMIN persona. Catch that and turn
it into a USER persona instead. Also, provide more context in the error
message.
* Review comment: pass the wrapping expression and stringify.
* Update processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java
Co-authored-by: Clint Wylie <cjwylie@gmail.com>
---------
Co-authored-by: Clint Wylie <cjwylie@gmail.com>
Adds versions of
DruidException.defensive(String, Object...)
InvalidInput.exception(String, Object...)
InvalidInput.exception(Throwable, String, Object...)
the versions add a boolean as the first arg and only create and throw
an exception if it's false. It can be used similar to
Preconditions.checkState/checkArgument
* contains Make a full copy of the parser and apply our modifications to it #16503
* some minor api changes pair/entry
* some unnecessary aggregation was removed from a set of queries in `CalciteSubqueryTest`
* `AliasedOperatorConversion` was detecting `CHAR_LENGTH` as not a function ; I've removed the check
* the field it was using doesn't look maintained that much
* the `kind` is passed for the created `SqlFunction` so I don't think this check is actually needed
* some decoupled test cases become broken - will be fixed later
* some aggregate related changes: due to the fact that SUM() and COUNT() of no inputs are different
* upgrade avatica to 1.25.0
* `CalciteQueryTest#testExactCountDistinctWithFilter` is now executable
Closeapache/druid#16503
* fix NestedDataColumnIndexerV4 to not report cardinality
changes:
* fix issue similar to #16489 but for NestedDataColumnIndexerV4, which can report STRING type if it only processes a single type of values. this should be less common than the auto indexer problem
* fix some issues with sql benchmarks
* Optimise S3 storage writing for MSQ durable storage
* Get rid of static ConcurrentHashMap
* Fix static checks
* Fix tests
* Remove unused constructor parameter chunkValidation + relevant cleanup
* Assert etags as String instead of Integer
* Fix flaky test
* Inject executor service
* Make threadpool size dynamic based on number of cores
* Fix S3StorageDruidModuleTest
* Fix S3StorageConnectorProviderTest
* Fix injection issues
* Add S3UploadConfig to manage maximum number of concurrent chunks dynamically based on chunk size
* Address the minor review comments
* Refactor S3UploadConfig + ExecutorService into S3UploadManager
* Address review comments
* Make updateChunkSizeIfGreater() synchronized instead of recomputeMaxConcurrentNumChunks()
* Address the minor review comments
* Fix intellij-inspections check
* Refactor code to use futures for maxNumConcurrentChunks. Also use executor service with blocking queue for backpressure semantics.
* Update javadoc
* Get rid of cyclic dependency injection between S3UploadManager and S3OutputConfig
* Fix RetryableS3OutputStreamTest
* Remove unnecessary synchronization parts from RetryableS3OutputStream
* Update javadoc
* Add S3UploadManagerTest
* Revert back to S3StorageConnectorProvider extends S3OutputConfig
* Address Karan's review comments
* Address Kashif's review comments
* Change a log message to debug
* Address review comments
* Fix intellij-inspections check
* Fix checkstyle
---------
Co-authored-by: asdf2014 <asdf2014@apache.org>
* Fallback vectorization for FunctionExpr and BaseMacroFunctionExpr.
This patch adds FallbackVectorProcessor, a processor that adapts non-vectorizable
operations into vectorizable ones. It is used in FunctionExpr and BaseMacroFunctionExpr.
In addition:
- Identifiers are updated to offer getObjectVector for ARRAY and COMPLEX in addition
to STRING. ExprEvalObjectVector is updated to offer ARRAY and COMPLEX as well.
- In SQL tests, cannotVectorize now fails tests if an exception is not thrown. This makes
it easier to identify tests that can now vectorize.
- Fix a null-matcher bug in StringObjectVectorValueMatcher.
* Fix tests.
* Fixes.
* Fix tests.
* Fix test.
* Fix test.
* Simplify serialized form of JsonInputFormat.
Use JsonInclude for keepNullColumns, assumeNewlineDelimited, and
useJsonNodeReader. Because the default value of keepNullColumns is
variable, we store the original configured value rather than the
derived value, and include if the original value is nonnull.
* Fix test.
UnnestStorageAdapter and its cursors did not return capabilities correctly
for the output column. This patch fixes two problems:
1) UnnestStorageAdapter returned the capabilities of the unnest virtual
column prior to unnesting. It should return the post-unnest capabilities.
2) UnnestColumnValueSelectorCursor passed through isDictionaryEncoded from
the unnest virtual column. This is incorrect, because the dimension selector
created by this class never has a dictionary. This is the cause of #16543.
* Add interface method for returning canonical lookup name
* Address review comment
* Add test in LookupReferencesManagerTest for coverage check
* Add test in LookupSerdeModuleTest for coverage check
* 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
There are a few issues with using Jackson serialization in sending datasketches between controller and worker in MSQ. This caused a blowup due to holding multiple copies of the sketch being stored.
This PR aims to resolve this by switching to deserializing the sketch payload without Jackson.
The PR adds a new query parameter used during communication between controller and worker while fetching sketches, "sketchEncoding".
If the value of this parameter is OCTET, the sketch is returned as a binary encoding, done by ClusterByStatisticsSnapshotSerde.
If the value is not the above, the sketch is encoded by Jackson as before.
* fix issue with auto column grouping
changes:
* fixes bug where AutoTypeColumnIndexer reports incorrect cardinality, allowing it to incorrectly use array grouper algorithm for realtime queries producing incorrect results for strings
* fixes bug where auto LONG and DOUBLE type columns incorrectly report not having null values, resulting in incorrect null handling when grouping
* fix test
This brings them in line with the behavior of other numeric aggregations.
It is important because otherwise ClassCastExceptions can arise if comparing
different numeric types that may arise from deserialization.
* Fix ExpressionPredicateIndexSupplier numeric replace-with-default behavior.
In replace-with-default mode, null numeric values from the index should be
interpreted as zeroes by expressions. This makes the index supplier more
consistent with the behavior of the selectors created by the expression
virtual column.
* Fix test case.
* Speed up SQL IN using SCALAR_IN_ARRAY.
Main changes:
1) DruidSqlValidator now includes a rewrite of IN to SCALAR_IN_ARRAY, when the size of
the IN is above inFunctionThreshold. The default value of inFunctionThreshold
is 100. Users can restore the prior behavior by setting it to Integer.MAX_VALUE.
2) SearchOperatorConversion now generates SCALAR_IN_ARRAY when converting to a regular
expression, when the size of the SEARCH is above inFunctionExprThreshold. The default
value of inFunctionExprThreshold is 2. Users can restore the prior behavior by setting
it to Integer.MAX_VALUE.
3) ReverseLookupRule generates SCALAR_IN_ARRAY if the set of reverse-looked-up values is
greater than inFunctionThreshold.
* Revert test.
* Additional coverage.
* Update docs/querying/sql-query-context.md
Co-authored-by: Benedict Jin <asdf2014@apache.org>
* New test.
---------
Co-authored-by: Benedict Jin <asdf2014@apache.org>
Custom calcite rule mimicking AggregateProjectMergeRule to extend support to expressions.
The current calcite rule return null in such cases.
In addition, this removes the redundant references.
MSQ sorts the columns in a highly specialized manner by byte comparisons. As such the values are serialized differently. This works well for the primitive types and primitive arrays, however complex types cannot be serialized specially.
This PR adds the support for sorting the complex columns by deserializing the value from the field and comparing it via the type strategy. This is a lot slower than the byte comparisons, however, it's the only way to support sorting on complex columns that can have arbitrary serialization not optimized for MSQ.
The primitives and the arrays are still compared via the byte comparison, therefore this doesn't affect the performance of the queries supported before the patch. If there's a sorting key with mixed complex and primitive/primitive array types, for example: longCol1 ASC, longCol2 ASC, complexCol1 DESC, complexCol2 DESC, stringCol1 DESC, longCol3 DESC, longCol4 ASC, the comparison will happen like:
longCol1, longCol2 (ASC) - Compared together via byte-comparison, since both are byte comparable and need to be sorted in ascending order
complexCol1 (DESC) - Compared via deserialization, cannot be clubbed with any other field
complexCol2 (DESC) - Compared via deserialization, cannot be clubbed with any other field, even though the prior field was a complex column with the same order
stringCol1, longCol3 (DESC) - Compared together via byte-comparison, since both are byte comparable and need to be sorted in descending order
longCol4 (ASC) - Compared via byte-comparison, couldn't be coalesced with the previous fields as the direction was different
This way, we only deserialize the field wherever required
Changes:
- Remove deprecated `markAsUnused` parameter from `KillUnusedSegmentsTask`
- Allow `kill` task to use `REPLACE` lock when `useConcurrentLocks` is true
- Use `EXCLUSIVE` lock by default
* QueryableIndex: Close columns after failed vector cursor setup.
If anything fails while setting up a vector cursor, the prior code in
QueryableIndex would not close its ColumnCache and would therefore leak
columns. Columns often contain references to buffers that must be closed.
* Fix style.
* 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.
#16068 modified DimensionHandlerUtils to accept complex types to be dimensions. This had an unintended side effect of allowing complex types to be joined upon (which wasn't guarded explicitly, it doesn't work).
This PR modifies the IndexedTable to reject building the index on the complex types to prevent joining on complex types. The PR adds back the check in the same place, explicitly.
* Four changes to scalar_in_array as follow-ups to #16306:
1) Align behavior for `null` scalars to the behavior of the native `in` and `inType` filters: return `true` if the array itself contains null, else return `null`.
2) Rename the class to more closely match the function name.
3) Add a specialization for constant arrays, where we build a `HashSet`.
4) Use `castForEqualityComparison` to properly handle cross-type comparisons.
Additional tests verify comparisons between LONG and DOUBLE are now
handled properly.
* Fix spelling.
* Adjustments from review.
JSON parsing has this function "charsetFix" that fixes up strings
so they can round-trip through UTF-8 encoding without loss of
fidelity. It was originally introduced to fix a bug where strings
could be sorted, encoded, then decoded, and the resulting decoded
strings could end up no longer in sorted order (due to character
swaps during the encode operation).
The code has been in place for some time, and only applies to JSON.
I am not sure if it needs to apply to other formats; it's certainly
more difficult to get broken strings from other formats. It's easy
in JSON because you can write a JSON string like "foo\uD900".
At any rate, this patch does not revisit whether charsetFix should
be applied to all formats. It merely optimizes it for the JSON case.
The function works by using CharsetEncoder.canEncode, which is
a relatively slow method (just as expensive as actually encoding).
This patch adds a short-circuit to skip canEncode if all chars in
a string are in the basic multilingual plane (i.e. if no chars are
surrogates).
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.
Buffer aggregators can contain some cached objects within them, such as
Memory references or HLL Unions. Prior to this patch, various Grouper
implementations were not releasing this state when resetting their own
internal state, which could lead to excessive memory use.
This patch renames AggregatorAdapater#close to "reset", and updates
Grouper implementations to call this reset method whenever they reset
their internal state.
The base method on BufferAggregator and VectorAggregator remains named
"close", for compatibility with existing extensions, but the contract
is adjusted to say that the aggregator may be reused after the method
is called. All existing implementations in core already adhere to this
new contract, except for the ArrayOfDoubles build flavors, which are
updated in this patch to adhere.
Additionally, this patch harmonizes buffer sketch helpers to call their
clear method "clear" rather than a mix of "clear" and "close". (Others
were already using "clear".)
* Additional short circuiting knowledge in filter bundles.
Three updates:
1) The parameter "selectionRowCount" on "makeFilterBundle" is renamed
"applyRowCount", and redefined as an upper bound on rows remaining
after short-circuiting (rather than number of rows selected so far).
This definition works better for OR filters, which pass through the
FALSE set rather than the TRUE set to the next subfilter.
2) AndFilter uses min(applyRowCount, indexIntersectionSize) rather
than using selectionRowCount for the first subfilter and indexIntersectionSize
for each filter thereafter. This improves accuracy when the incoming
applyRowCount is smaller than the row count from the first few indexes.
3) OrFilter uses min(applyRowCount, totalRowCount - indexUnionSize) rather
than applyRowCount for subfilters. This allows an OR filter to pass
information about short-circuiting to its subfilters.
To help write tests for this, the patch also moves the sampled
wikiticker data file from sql to processing.
* Forbidden APIs.
* Forbidden APIs.
* Better comments.
* Fix inspection.
* Adjustments to tests.
Currently, export creates the files at the provided destination. The addition of the manifest file will provide a list of files created as part of the manifest. This will allow easier consumption of the data exported from Druid, especially for automated data pipelines
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
* Fix ORDER BY on certain GROUPING SETS.
DefaultLimitSpec (part of native groupBy) had a bug where it would assume
that results are naturally ordered by dimensions even when subtotalsSpec
is present. However, this is not necessarily the case. For certain
combinations of ORDER BY and GROUPING SETS, this would cause the ORDER BY
to be ignored.
* Fix test testGroupByWithSubtotalsSpecWithOrderLimitForcePushdown. Resorting was necessary.
* SQL tests: avoid mixing skip and cannot vectorize.
skipVectorize switches off vectorization tests completely, and
cannotVectorize turns vectorization tests into negative tests. It doesn't
make sense to use them together, so this patch makes it an error to do so,
and cleans up cases where both are mentioned.
This patch also has the effect of changing various tests from skipVectorize
to cannotVectorize, because in the past when both were mentioned,
skipVectorize would take priority.
* Fix bug with StringAnyAggregatorFactory attempting to vectorize when it cannt.
* Fix tests.
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.
Current Runtime Exceptions generated while writing frames only include the exception itself without including the name of the column they were encountered in. This patch introduces the further information in the error and makes it non-retryable.
* Allow typedIn to run in replace-with-default mode.
Useful when data servers, like Historicals, are running in replace-with-default
mode and the Broker is running in SQL-compatible mode, which can happen during
a rolling update that is applying a mode change.
The name of the combining filtered aggregator factory should be the same
as the name of the original factory. However, it wasn't the same in the
case where the original factory's name and the original delegate aggregator
were inconsistently named. In this scenario, we should use the name of
the original filtered aggregator, not the name of the original delegate
aggregator.
This PR creates an interface for ImmutableRTree and moved the existing implementation to new class which represent 32 bit implementation (stores coordinate as floats). This PR makes the ImmutableRTree extendable to create higher precision implementation as well (64 bit).
In all spatial bound filters, we accept float as input which might not be accurate in the case of high precision implementation of ImmutableRTree. This PR changed the bound filters to accepts the query bounds as double instead of float and it is backward compatible change as it compares double to existing float values in RTree. Previously it was comparing input float to RTree floats which can cause precision loss, now it is little better as it compares double to float which is still not 100% accurate.
There are no changes in the way that we query spatial dimension today except input bound parsing. There is little improvement in string filter predicate which now parse double strings instead of float and compares double to double which is 100% accurate but string predicate is only called when we dont have spatial index.
With allowing the interface to extend ImmutableRTree, we allow to create high precision (HP) implementation and defines new search strategies to perform HP search Iterable<ImmutableBitmap> search(ImmutableDoubleNode node, Bound bound);
With possible HP implementations, Radius bound filter can not really focus on accuracy, it is calculating Euclidean distance in comparing. As EARTH 🌍 is round and not flat, Euclidean distances are not accurate in geo system. This PR adds new param called 'radiusUnit' which allows you to specify units like meters, km, miles etc. It uses https://en.wikipedia.org/wiki/Haversine_formula to check if given geo point falls inside circle or not. Added a test that generates set of points inside and outside in RadiusBoundTest.
This PR aims to introduce Window functions on MSQ by doing the following:
Introduce a Window querykit for handling window queries along with its factory and a processor for window queries
If a window operator is present with a partition by clause, pushes the partition as a shuffle spec of the previous stage
In presence of empty OVER() clause lets all operators loose on a single rac
In presence of no empty OVER() clause, breaks down each window into individual stages
Associated machinery to handle window functions in MSQ
Introduced a separate hidden engine feature WINDOW_LEAF_OPERATOR which is set only for MSQ engine. In presence of this feature, the planner plans without the leaf operators by creating a window query over an inner scan query. In case of native this is set to false and the planner generates the leafOperators
Guardrails around materialization
Comprehensive UTs
Changes:
- Handle exceptions in the API and map them to a `Response` object with the appropriate error code.
- Replace `AuthorizationUtils.filterAuthorizedResources()` with `DatasourceResourceFilter`.
The endpoint is annotated consistent with other usages.
- Update `DatasourceResourceFilter` to remove the lambda and update javadocs.
The usages information is self-evident with an IDE.
- Adjust the invalid interval exception message.
- Break up the large unit test `testGetUnusedSegmentsInDataSource()` into smaller unit tests
for each test case. Also, validate the error codes.
* 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.
This commit allows to use the MV_FILTER_ONLY & MV_FILTER_NONE functions
with a non literal argument.
Currently `select mv_filter_only('mvd_dim', 'array_dim') from 'table'`
returns a `Unhandled Query Planning Failure`
This is being tackled and also considered for the cases where the `array_dim`
having null & empty values.
Changed classes:
* `MultiValueStringOperatorConversions`
* `ApplyFunction`
* `CalciteMultiValueStringQueryTest`
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.
changes:
* adds TypedInFilter which preserves matching sets in the native match value type
* SQL planner uses new TypedInFilter when druid.generic.useDefaultValueForNull=false (the default)
* SortMerge join support for IS NOT DISTINCT FROM.
The patch adds a "requiredNonNullKeyParts" field to the sortMerge
processor, which has the list of key parts that must be nonnull for
an equijoin condition to match. Conditions with SQL "=" are present in
the list; conditions with SQL "IS NOT DISTINCT FROM" are absent from
the list.
* Fix test.
* Update javadoc.
* 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
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`.
changes:
* fix issues with array_contains and array_overlap with null left side arguments
* modify singleThreaded stuff to allow optimizing Function similar to how we do for ExprMacro - removed SingleThreadSpecializable in favor of default impl of asSingleThreaded on Expr with clear javadocs that most callers shouldn't be calling it directly and should be using Expr.singleThreaded static method which uses a shuttle and delegates to asSingleThreaded instead
* add optimized 'singleThreaded' versions of array_contains and array_overlap
* add mv_harmonize_nulls native expression to use with MV_CONTAINS and MV_OVERLAP to allow them to behave consistently with filter rewrites, coercing null and [] into [null]
* fix bug with casting rhs argument for native array_contains and array_overlap expressions
ConcurrentGrouper kind of misuses ThreadLocal to hold a SpillingGrouper, and never calls remove() on it, which can result in large amounts of heap being retained as weak references even after grouping is finished. This PR calls keySerde.reset() on all of the Grouper.close() implementations that have a KeySerde and should free up a bunch of space that is no longer needed.
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:
Improve `SqlSegmentsMetadataManager`
- Break the loop in `populateUsedStatusLastUpdated` before going to sleep if there are no more segments to update
- Add comments and clean up logs
Refactor `SqlSegmentsMetadataManagerTest`
- Merge `SqlSegmentsMetadataManagerEmptyTest` into this test
- Add method `testPollEmpty`
- Shave a few seconds off of the tests by reducing poll duration
- Simplify creation of test segments
- Some renames here and there
- Remove unused methods
- Move `TestDerbyConnector.allowLastUsedFlagToBeNull` to this class
Other minor changes
- Add javadoc to `NoneShardSpec`
- Use lambda in `SqlSegmentMetadataPublisher`
While converting Sequence<ScanResultValue> to Sequence<Frames>, when maxSubqueryBytes is enabled, we batch the results to prevent creating a single frame per ScanResultValue. Batching requires peeking into the actual value, and checking if the row signature of the scan result’s value matches that of the previous value.
Since we can do this indefinitely (in the worst case all of them have the same signature), we keep fetching them and accumulating them in a list (on the heap). We don’t really know how much to batch before we actually write the value as frames.
The PR modifies the batching logic to not accumulate the results in an intermediary list
* `Expr#singleThreaded` which creates a singleThreaded version of the actual expression (caching ExprEval is allowed)
* `Expr#makeSingleThreaded` to make a whole subtree of expressions 'singleThreaded' - uses `Shuttle` to create the new expression tree
* `ConstantExpr#singleThreaded` creates a specialized `ConstantExpr` which does cache the `ExprEval`
* some `@Immutable` annotations were added to make it more likely to notice that there might be something off if a similar change will be made around here for some reason
Since #15175, the javadoc for ReadableFieldPointer is somewhat out of date. It says that
the pointer only points to the beginning of the field, but this is no longer true. This
patch updates the javadoc to be more accurate.
allow a hashjoin result to be converted to RowsAndColumns
added StorageAdapterRowsAndColumns
fix incorrect isConcrete() return values during early phase of planning
* 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.