Commit Graph

1579 Commits

Author SHA1 Message Date
Vishesh Garg 197c54f673
Auto-Compaction using Multi-Stage Query Engine (#16291)
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).
2024-07-12 16:40:20 +05:30
Sree Charan Manamala eb981d855f
Correct aggregators violating names (#16615)
In case of few aggregators for example BloomSqlAggregator, BaseVarianceSqlAggregator etc, the aggName is being updated from a0 to a0:agg, breaching the contract as we would expect the aggName as the name which is passed. This is causing a mismatch while creating a column accessor.

This commit aims to correct those violating sql aggregators.
2024-07-12 09:18:09 +02:00
Adarsh Sanjeev 7c625356c5
Add logging for sketches on workers (#16697)
Improve the logging of sketches on workers.
2024-07-09 14:37:43 +05:30
Adarsh Sanjeev af5399cd9d
Fixes a bug when running queries with a limit clause (#16643)
Add a shuffling based on the resultShuffleSpecFactory after a limit processor depending on the query destination. LimitFrameProcessors currently do not update the partition boosting column, so we also add the boost column to the previous stage, if one is required.
2024-07-09 14:29:12 +05:30
Kashif Faraz 7c6f2b1e20
Minor log cleanup in K8sDruidNodeDiscoveryProvider (#16701) 2024-07-08 18:32:39 +05:30
Abhishek Radhakrishnan 35b970935f
Better error handling when retrieving Avro schemas from registry (#16684)
* Handle RestClientException separately, instead of returning a generic error.

- Add tests
- Clean up the tests; remove the legacy expected exception pattern
- Better test assertions

* Rename tests

* checkstyle fixes
2024-07-02 16:48:34 -07:00
Akshat Jain 34c80ee3de
Add MSQ engine support for window function drill tests (#16665)
* Add MSQ engine support for window function drill tests

* Address review comments

* Revert formatting changes in TestDataBuilder
2024-06-28 11:14:17 +05:30
Abhishek Radhakrishnan 82117e8101
Add MSQ query context `maxNumSegments` (#16637)
* Add MSQ query context maxNumSegments.

- Default is MAX_INT (unbounded).
- When set and if a time chunk contains more number of segments than set in the
  query context, the MSQ task will fail with TooManySegments fault.

* Fixup hashCode().

* Rename and checkpoint.

* Add some insert and replace happy and sad path tests.

* Update error msg.

* Commentary

* Adjust the default to be null (meaning no max bound on number of segments).

Also fix formatter.

* Fix CodeQL warnings and minor cleanup.

* Assert on maxNumSegments tuning config.

* Minor test cleanup.

* Use null default for the MultiStageQueryContext as well

* Review feedback

* Review feedback

* Move logic to common function getPartitionsByBucket shared by INSERT and REPLACE.

* Rename to validateNumSegmentsPerBucketOrThrow() for consistency.

* Add segmentGranularity to error message.
2024-06-26 09:29:51 -07:00
Laksh Singla 71b3b5ab5d
Add query context parameter to remove null bytes when writing frames (#16579)
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.
2024-06-26 15:00:30 +05:30
Tom 52c9929019
Column name in parse exceptions (#16529)
* 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
2024-06-25 13:42:52 -07:00
Kashif Faraz f1043d20bc
Support csv input format in Kafka ingestion with header (#16630)
* Support ListBasedInputRow in Kafka ingestion with header
* Fix up buildBlendedEventMap
* Add new test for KafkaInputFormat with csv value and headers
* Do not use forbidden APIs
* Move utility method to TestUtils
2024-06-25 11:50:01 +05:30
Clint Wylie 37a50e6803
Remove index_realtime and index_realtime_appenderator tasks (#16602)
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>
2024-06-24 20:13:33 -07:00
Adarsh Sanjeev 1a883ba1f7
Fix complex columns with export (#16572)
This PR fixes a few bugs with MSQ export. The main change is calling SqlResults#coerce before writing the column. This allows sketches and json to be correctly deserialized. The format of the exported complex columns are similar to those produced by Async MSQ queries with CSV format.

Notes:

    Fix printing of complex columns during export. Sketches and JSON are now correctly formatted during export.
    Fix an NPE if the writer has not been initialized. Empty export queries will create an empty file at the location.
    Fix a bug with counters for MSQ export, where rows were reported for only the first partition.
2024-06-24 09:03:30 +05:30
Akshat Jain 641f739a47
Fix flaky test in RetryableS3OutputStreamTest (#16639)
As part of #16481, we have started uploading the chunks in parallel.
That means that it's not necessary for the part that finished uploading last
to be less than or equal to the chunkSize (as the final part could've been uploaded earlier).

This made a test in RetryableS3OutputStreamTest flaky where we were
asserting that the final part should be smaller than chunk size.

This commit fixes the test, and also adds another test where the file size
is such that all chunk sizes would be of equal size.
2024-06-24 08:13:47 +05:30
Akshat Jain cd438b1918
Emit metrics for S3UploadThreadPool (#16616)
* Emit metrics for S3UploadThreadPool

* Address review comments

* Revert unnecessary formatting change

* Revert unnecessary formatting change in metrics.md file

* Address review comments

* Add metric for task duration

* Minor fix in metrics.md

* Add s3Key and uploadId in the log message

* Address review comments

* Create new instance of ServiceMetricEvent.Builder for thread safety

* Address review comments

* Address review comments
2024-06-21 11:36:47 +05:30
Zoltan Haindrich ac19b148c2
Upgrade calcite to 1.37.0 (#16504)
* 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

Close apache/druid#16503
2024-06-13 08:47:50 +02:00
Andreas Maechler 40ba429c5f
More validation for Azure account config (#16561)
* Mark `account` as NotNull

* Remove account test

Handled by annotation now

* Cleanup account config

* Mark container as not-null.
2024-06-07 13:24:15 -07:00
Andreas Maechler e6a82e8a11
Only create container in `AzureStorage` for write operations (#16558)
* Remove unused constants

* Refactor getBlockBlobLength

* Better link

* Upper-case log

* Mark defaultStorageAccount nullable

This is the case if you do not use Azure for deep-storage but ingest from Azure blobs.

* Do not always create a new container if it doesn't exist

Specifically, only create a container if uploading a blob or writing a blob stream

* Add lots of comments, group methods

* Revert "Mark defaultStorageAccount nullable"

* Add mockito for junit

* Add extra test

* Add comment

Thanks George.

* Pass blockSize as Long

* Test more branches...
2024-06-07 09:47:51 -07:00
Akshat Jain 03a38be446
Optimize S3 storage writing for MSQ durable storage (#16481)
* 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>
2024-06-07 11:33:16 +05:30
Gian Merlino 277006446d
Fallback vectorization for FunctionExpr and BaseMacroFunctionExpr. (#16366)
* 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.
2024-06-05 20:03:02 -07:00
Gian Merlino 2534a42539
Fix serde for ArrayOfDoublesSketchConstantPostAggregator. (#16550)
* Fix serde for ArrayOfDoublesSketchConstantPostAggregator.

The version originally added in #13819 was missing an annotation for
the "value" property. Fixes #16539.

Line endings for ArrayOfDoublesSketchConstantPostAggregator.java are changed
from \r\n to \n.

Adds a serde test, and improves various other datasketches post-aggregator
serde tests to deserialize into PostAggregator. This verifies that the type
information is set up correctly.

* Fix excessive imports.

* Fix equals, hashCode.
2024-06-05 20:01:51 -07:00
Abhishek Radhakrishnan b9ba286423
Fix task bootstrapping & simplify segment load/drop flows (#16475)
* 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
2024-06-04 10:44:46 -07:00
Andreas Maechler b0f2a07c40
Add README with link to docs (#16540) 2024-06-04 07:41:01 -07:00
Andreas Maechler 02caa50fd0
Remove unused interface from Azure extension (#16541) 2024-06-04 08:21:26 +05:30
Andreas Maechler 6c7443c93a
Update Azure extension tests to JUnit 5 (#16521)
Changes:
- Loosely followed the steps in the migration guide at
https://junit.org/junit5/docs/current/user-guide/#migrating-from-junit4
- Updated POM to add JUnit 5 dependencies
- Updated imports to JUnit 5 packages
- Updated annotations (Lifecycle annotations like `@BeforeEach`)
- Updated exception testing (`assertThrows`)
- Updated temporary path handling (use `@TempDir` annotation)
- Various other updates (replace other `Rule` usages, make sure to use JUnit 5 assertions)
2024-06-04 08:19:48 +05:30
Karan Kumar d0916865d0
Fix race in AzureClient factory fetch (#16525)
* Fix race in AzureClient factory fetch

* Fixing forbidden check.

* Renaming variable.
2024-06-01 22:50:44 +05:30
Sree Charan Manamala 27cfe12f4a
Enable reordering of window operators (#16482)
This commit aims to enable the re-ordering of window operators in order to optimise
the sort and partition operators.
Example : 
```
SELECT m1, m2,
SUM(m1) OVER(PARTITION BY m2) as sum1,
SUM(m2) OVER() as sum2
from numFoo
GROUP BY m1,m2
```

In order to compute this query, we can order the operators as to first compute the operators
corresponding to sum2 and then place the operators corresponding to sum1 which would
help us in reducing one sort operator if we order our operators by sum1 and then sum2.
2024-05-29 12:17:12 +05:30
Adarsh Sanjeev 21f725f33e
Add octet streaming of sketchs in MSQ (#16269)
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.
2024-05-28 18:12:38 +05:30
Akshat Jain ddfd62d9a9
Disable loading lookups by default in CompactionTask (#16420)
This PR updates CompactionTask to not load any lookups by default, unless transformSpec is present.

If transformSpec is present, we will make the decision based on context values, loading all lookups by default. This is done to ensure backward compatibility since transformSpec can reference lookups.
If transform spec is not present and no context value is passed, we donot load any lookup.

This behavior can be overridden by supplying lookupLoadingMode and lookupsToLoad in the task context.
2024-05-15 11:39:23 +05:30
Codegass 621525a5cb
Refactor: Clean up `DecimalParquetInputTest` using Assume (#16436) 2024-05-14 21:13:07 +05:30
Adarsh Sanjeev 18a4722d11
Resolve a bug where datasketches would not downsample sketches sufficiently (#16119)
* Fix sketch memory issue

* Rename function

* Add unit test

* Revert downsampling change
2024-05-14 10:23:57 +05:30
Akshat Jain d1100a6f63
Add retries for building S3 client (#16438)
* Add retries for building S3 client

* Use S3Utils instead of RetryUtils

* Add test
2024-05-13 16:32:06 -07:00
Laksh Singla 4bfc186153
Support sorting on complex columns in MSQ (#16322)
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
2024-05-13 15:07:05 +05:30
Igor Berman d0f3fdab37
Allow using different lock types for kill task, remove markAsUnused parameter (#16362)
Changes:
- Remove deprecated `markAsUnused` parameter from `KillUnusedSegmentsTask`
- Allow `kill` task to use `REPLACE` lock when `useConcurrentLocks` is true
- Use `EXCLUSIVE` lock by default
2024-05-10 06:37:36 +05:30
Adarsh Sanjeev 30f3cf5017
Add more info in MSQ export log message (#16363) 2024-05-09 13:02:19 +05:30
Zoltan Haindrich 1811674753
Enable quidem tests to use different suppliers (#16382)
* enable quidem uri support for `druidtest:///?ComponentSupplier=Nested` and similar
* changes the way `SqlTestFrameworkConfig` is being applied; all options will have their own annotation (its kinda impossible to detect that an annotation has a set value or its the default)
* enables hierarchical processing of config annotation (was needed to enable class level supplier annotation)
* moves uri processing related string2config stuff into `SqlTestFrameworkConfig`
2024-05-09 09:21:02 +02:00
Akshat Jain 775d654a6c
Load only the required lookups for MSQ tasks (#16358)
With this PR changes, MSQ tasks (MSQControllerTask and MSQWorkerTask) only load the required lookups during querying and ingestion, based on the value of CTX_LOOKUPS_TO_LOAD key in the query context.
2024-05-09 11:21:54 +05:30
Adarsh Sanjeev f82cc34e5b
Maintain a connection while exporting results with MSQ (#16381)
* Maintain a connection while exporting results with MSQ

* Fix checkstyle

* Fix checkstyle

* Move initialization from constructor

* Add null check

* Address review comments
2024-05-08 11:34:20 +05:30
Adarsh Sanjeev 269e035e76
Add validation for reindex with realtime sources (#16390)
Add validation for reindex with realtime sources.

With the addition of concurrent compaction, it is possible to ingest data while querying from realtime sources with MSQ into the same datasource. This could potentially lead to issues if the interval that is ingested into is replaced by an MSQ job, which has queried only some of the data from the realtime task.

This PR adds validation to check that the datasource being ingested into is not being queried from, if the query includes realtime sources.
2024-05-07 10:32:15 +05:30
Alberic Liu 92fb0ff718
upgrade mysql:mysql-connector-java to 8.2.0 (#16024)
* upgrade mysql:mysql-connector-java to 8.2.0

* fix the check errors

* remove unused comment
2024-05-06 21:58:37 +08:00
zachjsh fb7c84fb5d
Catalog clustering keys fixes (#16351)
* * add another catalog clustering columns unit test

* * dissallow clusterKeys with descending order

* * make more clear that clustering is re-written into ingest node
whether a catalog table or not

* * when partitionedBy is stored in catalog, user shouldnt need to specify
it in order to specify clustering

* * fix intellij inspection failure
2024-05-03 14:02:56 -04:00
Jan Werner b16401323b
update dependencies to address CVEs (#16374)
update dependencies to address new batch of CVEs:
- Azure POM from 1.2.19 to 1.2.23 to update transitive dependency nimbus-jose-jwt to address:  CVE-2023-52428
- commons-configuration2 from 2.8.0 to 2.10.1 to address: CVE-2024-29131 CVE-2024-29133
- bcpkix-jdk18on from 1.76 to 1.78.1 to address: CVE-2024-30172 CVE-2024-30171 CVE-2024-29857
2024-05-02 21:35:21 -07:00
Gian Merlino 5d1950d451
MSQ controller: Support in-memory shuffles; towards JVM reuse. (#16168)
* 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.
2024-04-30 21:30:27 -07:00
Adarsh Sanjeev fb63520de9
Add tests for ProcessorManager (#16327)
* Add tests for ProcessorManager
2024-04-30 09:35:26 +05:30
Adithya Chakilam f8015eb02a
Add config lagAggregate to LagBasedAutoScalerConfig (#16334)
Changes:
- Add new config `lagAggregate` to `LagBasedAutoScalerConfig`
- Add field `aggregateForScaling` to `LagStats`
- Use the new field/config to determine which aggregate to use to compute lag
- Remove method `Supervisor.computeLagForAutoScaler()`
2024-04-29 22:20:41 +05:30
Adarsh Sanjeev 9a2d7c28bc
Prepare master branch for 31.0.0 release (#16333) 2024-04-26 09:22:43 +05:30
Arun Ramani 126a0c219a
Surface lock revocation exceptions in task status (#16325) 2024-04-26 08:39:44 +05:30
AmatyaAvadhanula 31eee7d51e
Check for handoff of upgraded segments (#16162)
Changes:
1) Check for handoff of upgraded realtime segments.
2) Drop sink only when all associated realtime segments have been abandoned.
3) Delete pending segments upon commit to prevent unnecessary upgrades and
partition space exhaustion when a concurrent replace happens. This also prevents
potential data duplication.
4) Register pending segment upgrade only on those tasks to which the segment is associated.
2024-04-25 22:03:38 +05:30
Jakub Matyszewski 5061507541
pacj4: add UserProfile attributes to AuthenticationResult context (#16109)
I'm adding OIDC context to the AuthenticationResult returned by pac4j extension. I wanted to use this context as input in OpenPolicyAgent authorization. Since AuthenticationResult already accepts context as a parameter it felt okay to pass the profile attributes there.
2024-04-25 12:10:12 +05:30
Zoltan Haindrich 9c0bd56f5b
Make QueryComponentSupliers independent from test classes (#16275) 2024-04-25 02:12:07 -04:00
Gian Merlino 8a5cc976a9
ArrayOfDoublesSketchBuildAggregator: Fix NPE in get() for empty sketch. (#16330)
Fixes a bug introduced in #16296, where the sketch might not be
initialized if get() is called without calling aggregate(). Also adds
a test for this case.
2024-04-25 00:59:59 -04:00
Laksh Singla 6bca406d31
Grouping on complex columns aka unifying GroupBy strategies (#16068)
Users can pass complex types as dimensions to the group by queries. For example:

SELECT nested_col1, count(*) FROM foo GROUP BY nested_col1
2024-04-24 23:00:14 +05:30
Rishabh Singh e30790e013
Introduce Segment Schema Publishing and Polling for Efficient Datasource Schema Building (#15817)
Issue: #14989

The initial step in optimizing segment metadata was to centralize the construction of datasource schema in the Coordinator (#14985). Thereafter, we addressed the problem of publishing schema for realtime segments (#15475). Subsequently, our goal is to eliminate the requirement for regularly executing queries to obtain segment schema information.

This is the final change which involves publishing segment schema for finalized segments from task and periodically polling them in the Coordinator.
2024-04-24 22:22:53 +05:30
Sree Charan Manamala 080476f9ea
WINDOWING - Fix 2 nodes with same digest causing mapping issue (#16301)
Fixes the mapping issue in window fucntions where 2 nodes get the same reference.
2024-04-24 16:45:02 +05:30
Gian Merlino 274ccbfd85
Reset buffer aggregators when resetting Groupers. (#16296)
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".)
2024-04-24 05:39:24 -04:00
Parth Agrawal f1d24c868f
[CVE Fixes] Update version of Nimbus.jose.jwt (#16320)
* Update version of nimbus.jose.jwt.version

* update licenses.yaml
2024-04-23 15:11:54 +05:30
Vishesh Garg 173a206829
Fix incorrect check of InvalidFieldException to InvalidFieldFault while generating MSQ Error Report (#16273)
InvalidFieldFault is incorrectly checked as InvalidFieldException in mapQueryColumnNameToOutputColumnName. This fixes the bug.
2024-04-22 15:18:49 +05:30
Laksh Singla b9bbde5c0a
Fix deadlock that can occur while merging group by results (#15420)
This PR prevents such a deadlock from happening by acquiring the merge buffers in a single place and passing it down to the runner that might need it.
2024-04-22 14:10:44 +05:30
Adithya Chakilam cff5d1e369
Add method Supervisor.computeLagForAutoScaler (#16314)
Tries to address the comments made on #16284 after merged.

Changes:
- Remove method `Supervisor.getLagMetric()`
- Add method `Supervisor.computeLagForAutoScaler()`
- Remove classes `LagMetric` and `LagMetricTest`
2024-04-20 07:57:50 +05:30
Akshat Jain 79e48c6b45
Fix NPE while loading lookups from empty JDBC source (#16307) 2024-04-18 21:52:02 +05:30
zachjsh 3f2dd46ede
Catalog table should not need explicit segment granularity set (#16278)
* * fix

* * fix

* * address review comments

* * fix

* * simplify tests

* * fix complex type nullability issue

* * fix and update test

* * address review comments

* * address test review comments

* * fix checkstyle

* * fix checkstyle

* * fix failing test
2024-04-17 11:46:24 -04:00
zachjsh 2351f038eb
Kafka with topicPattern can ignore old offsets spuriously (#16190)
* * fix

* * simplify

* * simplify tests

* * update matches function definition for Kafka Datasource Metadata

* * add matchesOld

* * override matches and plus for kafka based metadata / sequence numbers

* * implement minus
* add tests

* * fix failing tests

* * remove TODO comments

* * simplfy and add comments

* * remove unused variable in tests

* * remove unneeded function

* * add serde tests

* * more stuff

* * address review comments

* * remove unneeded code.
2024-04-17 10:00:17 -04:00
Adithya Chakilam 34237bc112
Consider max lag for kinesis while autoscaling (#16284)
* Consider max lag for kinesis while autoscaling

* add test for coverage

* test folder
2024-04-17 15:05:05 +05:30
AmatyaAvadhanula f3d69f30e6
Associate pending segments with the tasks that requested them (#16144)
Changes:
- Add column `task_allocator_id` to `pendingSegments` metadata table.
- Add column `upgraded_from_segment_id` to `pendingSegments` metadata table.
- Add interface `PendingSegmentAllocatingTask` and implement it by all tasks which
can allocate pending segments.
- Use `taskAllocatorId` to identify the task (and its sub-tasks or replicas) to which
a pending segment has been allocated.
- Perform active cleanup of pending segments in `TaskLockbox` once there are no
active tasks for the corresponding task allocator id.
- When committing APPEND segments, also commit all upgraded pending segments
corresponding to that task allocator id.
- When committing REPLACE segments, upgrade all overlapping pending segments in
the same transaction.
2024-04-17 09:06:31 +05:30
zachjsh a5428e75ff
INSERT/REPLACE complex target column types are validated against source input expressions (#16223)
* * fix

* * fix

* * address review comments

* * fix

* * simplify tests

* * fix complex type nullability issue

* * address review comments

* * address test review comments

* * fix checkstyle
2024-04-16 17:20:35 -04:00
AmatyaAvadhanula ad6bd62140
Handle task location fetch from overlord during rolling upgrades (#16227)
Bug:
#15724 introduced a bug where a rolling upgrade would cause all task locations
returned by the Overlord on an older version to be unknown.

Fix:
If the new API fails, fall back to single task status API which always returns a valid task location.
2024-04-16 21:01:37 +05:30
YongGang 6964297b53
Remove the unused Controller context reference from Worker (#16285) 2024-04-16 08:34:24 +05:30
Adarsh Sanjeev 3df00aef9d
Add manifest file for MSQ export (#15953)
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
2024-04-15 11:37:31 +05:30
Kashif Faraz 81d7b6ebe1
Fix OverlordClient to read reports as a concrete `ReportMap` (#16226)
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
2024-04-15 08:00:59 +05:30
YongGang da9feb4430
Introduce TaskContextReport for reporting task context (#16041)
Changes:
- Add `TaskContextEnricher` interface to improve task management and monitoring
- Invoke `enrichContext` in `TaskQueue.add()` whenever a new task is submitted to the Overlord
- Add `TaskContextReport` to write out task context information in reports
2024-04-12 08:57:49 +05:30
Gian Merlino 9f358f5f4a
SQL tests: avoid mixing skip and cannot vectorize. (#16251)
* 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.
2024-04-11 15:06:11 -07:00
Vishesh Garg 3d595cfab1
Add storeCompactionState flag support to msq (#15965)
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.
2024-04-09 16:47:47 +05:30
Vishesh Garg 9a4fb58543
Record column name for exceptions while writing frames in RowBasedFrameWriter (#16130)
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.
2024-04-09 15:39:10 +05:30
Adarsh Sanjeev e2e0cb905c
Add reasoning for choosing shardSpec to the MSQ report (#16175)
This PR logs the segment type and reason chosen. It also adds it to the query report, to be displayed in the UI.

This PR adds a new section to the reports, segmentReport. This contains the segment type created, if the query is an ingestion, and null otherwise.
2024-04-09 11:32:02 +05:30
Gian Merlino 5e5cf9af99
Reduce upload buffer size in GoogleTaskLogs. (#16236)
* Reduce upload buffer size in GoogleTaskLogs.

Use a 1MB upload buffer, rather than the default of 15 MB in the API client. This is
mainly because MMs may upload logs in parallel, and typically have small heaps. The
default-sized 15 MB buffers add up quickly and can cause a MM to run out of memory.

* Make bufferSize a nullable Integer. Add tests.
2024-04-08 12:54:31 -07:00
Parag Jain f55c9e58a8
add google as external storage for msq export (#16051)
Support for exporting msq results to gcs bucket. This is essentially copying the logic of s3 export for gs, originally done by @adarshsanjeev in this PR - #15689
2024-04-05 12:10:10 +05:30
Soumyava 4bea865697
Restore context flag for window functions (#16229) 2024-04-03 13:57:13 +05:30
zachjsh 9b52c909e0
fix complex types returning UNKNOWN as their SQL type inference (#16216)
* * fix

* * fix

* * address review comments
2024-04-02 14:36:01 -04:00
Kashif Faraz 0de44d91f1
Cleanup serialiazation of TaskReportMap (#16217)
* Build task reports in AbstractBatchIndexTask

* Minor cleanup

* Apply suggestions from code review by @abhishekrb

Co-authored-by: Abhishek Radhakrishnan <abhishek.rb19@gmail.com>

* Cleanup IndexTaskTest

* Fix formatting

* Fix coverage

* Cleanup serialization of TaskReport map

* Replace occurrences of Map<String, TaskReport>

* Return TaskReport.ReportMap for live reports, fix test comparisons

* Address test failures

---------

Co-authored-by: Abhishek Radhakrishnan <abhishek.rb19@gmail.com>
2024-04-01 11:53:24 -07:00
Kashif Faraz 4df4896674
Refactor: Add common method in AbstractBatchIndexTask to create ingestion stats report (#16202)
Changes
-  No functional changes
- Add method `AbstractBatchIndexTask.buildIngestionStatsReport()` used in several batch tasks
- Add utility method `AbstractBatchIndexTask.addBuildSegmentStatsToReport()`
- Use boolean argument to represent a full report instead of the String `full` 
in internal methods. (REST API remains unchanged.)
- Rename `IngestionStatsAndErrorsTaskReportData` to `IngestionStatsAndErrors`
- Clean up some of the methods
2024-03-28 23:07:00 +05:30
Soumyava 524842a3bb
Window function on msq (#15470)
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
2024-03-28 14:58:34 +05:30
Gian Merlino 7649957710
MSQ: Fix issue where AUTO assignment would not respect maxWorkerCount. (#16214)
WorkerAssignmentStrategy.AUTO was missing a check for maxWorkerCount
in the case where the inputs to a stage are not dynamically sliceable.
A common case here is when the inputs to a stage are other stages.
2024-03-28 14:40:31 +05:30
zachjsh 8370db106c
INSERT/REPLACE dimension target column types are validated against source input expressions (#15962)
* * address remaining comments from https://github.com/apache/druid/pull/15836

* *  address remaining comments from https://github.com/apache/druid/pull/15908

* * add test that exposes relational algebra issue

* * simplify test exposing issue

* * fix

* * add tests for sealed / non-sealed

* * update test descriptions

* * fix test failure when -Ddruid.generic.useDefaultValueForNull=true

* * check type assignment based on natice Druid types

* * add tests that cover missing jacoco coverage

* * add replace tests

* * add more tests and comments about column ordering

* * simplify tests

* * review comments

* * remove commented line

* * STRING family types should be validated as non-null
2024-03-25 12:34:07 -04:00
Aru Raghuwanshi 6e19ce5e69
Handle null values in `KafkaStringHeaderReader` (#16192) 2024-03-23 13:05:55 +05:30
Gian Merlino 2b23d0b5b5
MSQ: Controller checker should check for "closed" only. (#16161)
* MSQ: Controller checker should check for "closed" only.

Currently, the worker's controller checker will exit the worker if
the controller location is "closed" (no longer running) or if its location
is empty (i.e. location unknown).

This patch changes to only exit on "closed". We shouldn't exit on empty
location, because that may happen if the Overlord is slow to acknowledge the
location of a task.

* Fix test.
2024-03-19 19:25:48 -07:00
Gian Merlino c96b215dd6
SortMerge join support for IS NOT DISTINCT FROM. (#16003)
* 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.
2024-03-19 12:02:13 -07:00
Zoltan Haindrich 1ad489a2ae
Fix build: newTempFolder (#16170) 2024-03-19 08:53:56 -07:00
Zoltan Haindrich 0a42342cef
Update Calcite*Test to use junit5 (#16106)
* Update Calcite*Test to use junit5

* change the way temp dirs are handled
* add openrewrite workflow to safeguard upgrade
* replace junitparamrunner with standard junit5 parametered tests
* update a few rules to junit5 api
* lots of boring changes

* cleanup QueryLogHook

* cleanup

* fix compile error: ARRAYS_DATASOURCE

* fix test

* remove enclosed

* empty

+TEST:TDigestSketchSqlAggregatorTest,HllSketchSqlAggregatorTest,DoublesSketchSqlAggregatorTest,ThetaSketchSqlAggregatorTest,ArrayOfDoublesSketchSqlAggregatorTest,BloomFilterSqlAggregatorTest,BloomDimFilterSqlTest,CatalogIngestionTest,CatalogQueryTest,FixedBucketsHistogramQuantileSqlAggregatorTest,QuantileSqlAggregatorTest,MSQArraysTest,MSQDataSketchesTest,MSQExportTest,MSQFaultsTest,MSQInsertTest,MSQLoadedSegmentTests,MSQParseExceptionsTest,MSQReplaceTest,MSQSelectTest,InsertLockPreemptedFaultTest,MSQWarningsTest,SqlMSQStatementResourcePostTest,SqlStatementResourceTest,CalciteSelectJoinQueryMSQTest,CalciteSelectQueryMSQTest,CalciteUnionQueryMSQTest,MSQTestBase,VarianceSqlAggregatorTest,SleepSqlTest,SqlRowTransformerTest,DruidAvaticaHandlerTest,DruidStatementTest,BaseCalciteQueryTest,CalciteArraysQueryTest,CalciteCorrelatedQueryTest,CalciteExplainQueryTest,CalciteExportTest,CalciteIngestionDmlTest,CalciteInsertDmlTest,CalciteJoinQueryTest,CalciteLookupFunctionQueryTest,CalciteMultiValueStringQueryTest,CalciteNestedDataQueryTest,CalciteParameterQueryTest,CalciteQueryTest,CalciteReplaceDmlTest,CalciteScanSignatureTest,CalciteSelectQueryTest,CalciteSimpleQueryTest,CalciteSubqueryTest,CalciteSysQueryTest,CalciteTableAppendTest,CalciteTimeBoundaryQueryTest,CalciteUnionQueryTest,CalciteWindowQueryTest,DecoupledPlanningCalciteJoinQueryTest,DecoupledPlanningCalciteQueryTest,DecoupledPlanningCalciteUnionQueryTest,DrillWindowQueryTest,DruidPlannerResourceAnalyzeTest,IngestTableFunctionTest,QueryTestRunner,SqlTestFrameworkConfig,SqlAggregationModuleTest,ExpressionsTest,GreatestExpressionTest,IPv4AddressMatchExpressionTest,IPv4AddressParseExpressionTest,IPv4AddressStringifyExpressionTest,LeastExpressionTest,TimeFormatOperatorConversionTest,CombineAndSimplifyBoundsTest,FiltrationTest,SqlQueryTest,CalcitePlannerModuleTest,CalcitesTest,DruidCalciteSchemaModuleTest,DruidSchemaNoDataInitTest,InformationSchemaTest,NamedDruidSchemaTest,NamedLookupSchemaTest,NamedSystemSchemaTest,RootSchemaProviderTest,SystemSchemaTest,CalciteTestBase,SqlResourceTest

* use @Nested

* add rule to remove enclosed; upgrade surefire

* remove enclosed

* cleanup

* add comment about surefire exclude
2024-03-19 04:05:12 -07:00
Adarsh Sanjeev a151bcfd12
Fix incorrect header names for certain export queries (#16096)
* Fix incorrect header names for certain queries

* Fix incorrect header names for certain queries

* Maintain upgrade compatibility

* Fix tests

* Change null handling
2024-03-19 15:11:04 +05:30
Gian Merlino 55c47fbcfd
MSQ: Fix NPE in getWorkerStats(). (#16159)
TaskTracker's status is null when TaskTrackers are first set up, and
stay null until the first status call comes back. This patch handles
that case and sets the status code to null in the WorkerStats object
in live reports.
2024-03-19 14:22:49 +05:30
Gian Merlino 8ee324c7e7
MSQ: Cancel workers more quickly. (#16158)
Prior to this patch, when canceled, workers would keep trying to contact
the controller: they would attempt to report an error, and if they were
in the midst of some other call (like a counters push) they would keep
trying it.

This can cause cancellation to be delayed, because the controller shuts
down its HTTP server before it cancels workers. Workers are then stuck
retrying calls to the controller that will never succeed. The retry loops
are broken when the controller gives up on them (one minute later) and
exits for real. Then, the controller failure detection logic on the worker
detects that the controller has failed, and the worker finally shuts down.

This patch speeds up worker cancellation by bypassing communication
with the controller. There is no real need for it. If the controller
canceled the workers, it isn't interested in further communications from
them. If the workers were canceled out-of-band, the controller can
detect this through worker monitoring and report it as a WorkerFailed
error.
2024-03-19 14:21:22 +05:30
Gian Merlino 36bc94c798
MSQ: Remove unnecessary snapshot deserialization code. (#16116)
Since #13205, a special deserializer module has no longer been necessary
to read key collector snapshots. This patch removes the unnecessary code.
2024-03-18 10:12:27 -07:00
Kashif Faraz 466057c61b
Remove deprecated DruidException, EntryExistsException (#14448)
Changes:
- Remove deprecated `DruidException` (old one) and `EntryExistsException`
- Use newly added comprehensive `DruidException` instead
- Update error message in `SqlMetadataStorageActionHandler` when max packet limit is violated.
- Factor out common code from several faults into `BaseFault`.
- Slightly update javadoc in `DruidException` to render it correctly
- Remove unused classes `SegmentToMove`, `SegmentToDrop`
- Move `ServletResourceUtils` from module `druid-processing` to `druid-server`
- Add utility method to build error Response from `DruidException`.
2024-03-15 21:29:11 +05:30
AlbericByte 33bb99cd0d
remove use log of log4j v1 (#15984) 2024-03-15 15:43:48 +05:30
Karan Kumar 5e603ac5ff
Adding more logging for s3 RetryableS3OutputStream (#16117)
Adding more logging for s3 RetryableS3OutputStream which would help us determine if the chunk size needs to be adjusted.
2024-03-14 11:35:57 +05:30
Gian Merlino 256160aba6
MSQ: Validate that strings and string arrays are not mixed. (#15920)
* MSQ: Validate that strings and string arrays are not mixed.

When multi-value strings and string arrays coexist in the same column,
it causes problems with "classic MVD" style queries such as:

  select * from wikipedia -- fails at runtime
  select count(*) from wikipedia where flags = 'B' -- fails at planning time
  select flags, count(*) from wikipedia group by 1 -- fails at runtime

To avoid these problems, this patch adds type verification for INSERT
and REPLACE. It is targeted: the only type changes that are blocked are
string-to-array and array-to-string. There is also a way to exclude
certain columns from the type checks, if the user really knows what
they're doing.

* Fixes.

* Tests and docs and error messages.

* More docs.

* Adjustments.

* Adjust message.

* Fix tests.

* Fix test in DV mode.
2024-03-13 15:37:27 -07:00
Gian Merlino 910124d4de
MSQ: Plan without implicit sorting. (#16073)
* MSQ: Plan without implicit sorting.

This patch adds an EngineFeature "GROUPBY_IMPLICITLY_SORTS" and sets
it true for native, false for MSQ. It's useful for two reasons:

1) In the future we'll likely want MSQ to hash-partition for GROUP BY
   instead of using a global sort, which would mean MSQ would not
   implicitly ORDER BY when there is a GROUP BY.

2) When doing REPLACE with MSQ, CLUSTERED BY is transformed to ORDER BY.
   We should retain that ORDER BY, as it may be a subset of the GROUP BY,
   and it is important to remember which fields the user wanted to include in
   range shard specs.

* Fix tests.

* Fix tests for real.

* Fix test.
2024-03-13 08:27:39 -07:00
Karan Kumar 84c5098473
Fix data race in getting results from MSQ select tasks. (#16107)
* Fix data race in getting results from MSQ select tasks.

* Add better logging

* Handling number overflow.
2024-03-13 08:58:18 +05:30
Zoltan Haindrich 8252d72e2a
Pull up literals in InputAccessor (#16033)
* Pull up literals in InputAccessor

* pull up literals in `InputAccessor`
* remove the need to pass `constants` of `Window`  operator

Fixes #15353

* update test

* enable relax_nulls
2024-03-12 09:14:31 -07:00
Vishesh Garg 2dd8b16467
Correct the API used to fetch the version for a GCS object (#16097)
Current API used to fetch the version for a GCS object is incorrect. This PR fixes that API.
2024-03-11 18:30:34 +05:30