Commit Graph

13353 Commits

Author SHA1 Message Date
Clint Wylie b4bc9b6950
fix issue with auto columns with mix of scalar values and empty arrays (#15083) 2023-10-05 10:15:45 +05:30
Laksh Singla b8d03d36b0
Free up the resources when materializing the results as Frames (#15032)
Refactor the code to clean up the result sequences when materializing the results as Frames
2023-10-05 10:14:27 +05:30
Clint Wylie 3afe09a19d
urlencode nested serializer temp file names so they dont explode stuff (#15068)
Fixes a bug caused by #14919, which was just using the column name as part of a temp file name, which.. isn't very cool, my bad. Switched to use StringUtils.urlEncode so that ugly chars don't explode stuff. The modified test fails without the changes in this PR.
2023-10-05 10:13:45 +05:30
Laksh Singla 30cf76db99
Field writers for numerical arrays (#14900)
Row-based frames, and by extension, MSQ now supports numeric array types. This means that all queries consuming or producing arrays would also work with MSQ. Numeric arrays can also be ingested via MSQ. Post this patch, queries like, SELECT [1, 2] would work with MSQ since they consume a numeric array, instead of failing with an unsupported column type exception.
2023-10-04 23:16:47 +05:30
317brian 88476e0e83
docs: add note about transparent_reconnection for Avatica (#15066)
* add note about transparent_reconnection

* Update docs/api-reference/sql-jdbc.md
2023-10-04 09:52:48 -07:00
Zoltan Haindrich 90e4b25620
Fix lead/lag to be usable without offset (#15057) 2023-10-04 17:38:46 +05:30
Tejaswini Bandlamudi c888ac5d61
fix path of druid service IT logs (#15082) 2023-10-04 15:38:38 +05:30
Gian Merlino a9021e4cd7
Fix NPE with lenient aggregators merging in segmentMetadata. (#15078)
When merging analyses, lenient merging sets unmergeable aggregators
to null. Merging such a null aggregator record into a nonnull record
would potentially lead to NPE in getMergingFactory.

The new code only calls getMergingFactory if both the old and new
aggregators are nonnull; else, if either is null, then the merged
aggregator is also set to null.
2023-10-04 02:41:41 -07:00
Clint Wylie 632811b285
fix json compat layer to not rewrite v4 into v5 after segment merging (#14997) 2023-10-04 00:18:18 -07:00
Tejaswini Bandlamudi 28870c702a
Resolve reported CVEs (#15081) 2023-10-04 11:59:01 +05:30
Gian Merlino 2ed4fd1ae3
Compute broadcast-join segmentMapFn only once per worker. (#15007)
This patch introduces "processor managers" to processor factories, as a replacement for the sequence of processors. Processor managers can use the results of earlier processors to influence the creation of later processors, which provides us with the building block we need to ensure that broadcast join data is only read once.

In particular, when broadcast join is happening, the BaseFrameProcessorFactory now uses a ChainedProcessorManager to first run BroadcastJoinSegmentMapFnProcessor (in a single thread), and then run all of the regular processors (possibly multithreaded).
2023-10-04 11:47:00 +05:30
Vishesh Garg 7e8f3e69ef
Avoid intermediate offsets in bucketStart calculation logic to handle DST transition (#15038)
When moving timestamps by an offset using org.joda.time.chrono.ISOChronology library, if the new timestamp falls in Daylight Savings Time (DST) transition period, the library rounds it off to the nearest valid time. This can lead to incorrect final timestamp when calculated using intermediate offsets landing in DST transition, for e.g. +21D arrived at using +14D and +7D offset, where +14D lands in DST transition period. Since bucketStart values are calculated using this library, this behaviour can lead to incorrect bucketStart times.
2023-10-04 11:32:29 +05:30
Zoltan Haindrich 3342e03ea8
Windowing processing may have run into Exceptions when the whole table was processed (#15064)
Earlier when the query was processing the whole table; the planning may have ended with a NPE; as it was not possible to create a scanquery from it.
2023-10-04 11:27:11 +05:30
Xavier Léauté adef2069b1
Make unit tests pass with Java 21 (#15014)
This change updates dependencies as needed and fixes tests to remove code incompatible with Java 21
As a result all unit tests now pass with Java 21.

* update maven-shade-plugin to 3.5.0 and follow-up to #15042
  * explain why we need to override configuration when specifying outputFile
  * remove configuration from dependency management in favor of explicit overrides in each module.
* update to mockito to 5.5.0 for Java 21 support when running with Java 11+
  * continue using latest mockito 4.x (4.11.0) when running with Java 8  
  * remove need to mock private fields
* exclude incorrectly declared mockito dependency from pac4j-oidc
* remove mocking of ByteBuffer, since sealed classes can no longer be mocked in Java 21
* add JVM options workaround for system-rules junit plugin not supporting Java 18+
* exclude older versions of byte-buddy from assertj-core
* fix for Java 19 changes in floating point string representation
* fix missing InitializedNullHandlingTest
* update easymock to 5.2.0 for Java 21 compatibility
* update animal-sniffer-plugin to 1.23
* update nl.jqno.equalsverifier to 3.15.1
* update exec-maven-plugin to 3.1.0
2023-10-03 22:41:21 -07:00
Soumyava cb050282a0
Intervals are updated properly for Unnest queries (#15020)
Fixes a bug where the unnest queries were not updated with the correct intervals.
2023-10-04 02:52:10 +05:30
George Shiqi Wu 64754b6799
Allow users to pass task payload via deep storage instead of environment variable (#14887)
This change is meant to fix a issue where passing too large of a task payload to the mm-less task runner will cause the peon to fail to startup because the payload is passed (compressed) as a environment variable (TASK_JSON). In linux systems the limit for a environment variable is commonly 128KB, for windows systems less than this. Setting a env variable longer than this results in a bunch of "Argument list too long" errors.
2023-10-03 14:08:59 +05:30
Zoltan Haindrich f3d1c8b70e
Enable back testcases in CalciteWindowQueryTest (#15045)
Most of the testcases were disabled in CalciteWindowQueryTest during the Calcite-1.35 upgrade; there were some changes arising from the fact that the removal of DRUID_SUM had some unexpected sideffects:

SqlStdOperatorTable.SUM became the SUM operator
because of that SqlToRelConverter started rewriting windowed SUM -s into SUM0 -s
my opinion is that w.r.t to Druid this rewrite provides no real advantage - as SUM0 is serviced by SUM here
I believe that's not 100% correct in cases when it aggregates just null-s but that doesnt matter in this case
I propose to introduce back a local DRUID_SUM thing as an unchanged SUM and later when CALCITE-6020 is fixed ; we can drop that.
2023-10-03 10:18:44 +05:30
Soumyava 261f54dc04
coalesce on unnest row mismatch fix (#15019)
* coalesce on unnest row mismatch fix

* new example with coalesce over unnest with nested array columns

* New example with change in order which triggers the nvl

* new test plan update for useDefault=true
2023-10-02 17:26:50 -07:00
Pranav f1edd671fb
Exposing optional replaceMissingValueWith in lookup function and macros (#14956)
* Exposing optional replaceMissingValueWith in lookup function and macros

* args range validation

* Updating docs

* Addressing comments

* Update docs/querying/sql-scalar.md

Co-authored-by: Clint Wylie <cjwylie@gmail.com>

* Update docs/querying/sql-functions.md

Co-authored-by: Clint Wylie <cjwylie@gmail.com>

* Addressing comments

---------

Co-authored-by: Clint Wylie <cjwylie@gmail.com>
2023-10-02 17:09:23 -07:00
Parth Agrawal d038237ece
memcached cache: switch to AWS elasticache-java-cluster-client and add TLS support (#14827)
This PR updates the library used for Memcached client to AWS Elasticache Client : https://github.com/awslabs/aws-elasticache-cluster-client-memcached-for-java

This enables us to use the option of encrypting data in transit:
Amazon ElastiCache for Memcached now supports encryption of data in transit

For clusters running the Memcached engine, ElastiCache supports Auto Discovery—the ability for client programs to automatically identify all of the nodes in a cache cluster, and to initiate and maintain connections to all of these nodes.
Benefits of Auto Discovery - Amazon ElastiCache

AWS has forked spymemcached 2.12.1, and has since added all the patches included in 2.12.2 and 2.12.3 as part of the 1.2.0 release. So, this can now be considered as an equivalent drop-in replacement.

GitHub - awslabs/aws-elasticache-cluster-client-memcached-for-java: Amazon ElastiCache Cluster Client for Java - enhanced library to connect to ElastiCache clusters.
https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/services/elasticache/AmazonElastiCacheClient.html#AmazonElastiCacheClient--

How to enable TLS with Elasticache

On server side:
https://docs.aws.amazon.com/AmazonElastiCache/latest/mem-ug/in-transit-encryption-mc.html#in-transit-encryption-enable-existing-mc

On client side:
GitHub - awslabs/aws-elasticache-cluster-client-memcached-for-java: Amazon ElastiCache Cluster Client for Java - enhanced library to connect to ElastiCache clusters.
2023-10-02 12:51:05 -07:00
Zoltan Haindrich 2785e062d7
Correct quotation in drill query files (#15044) 2023-10-02 08:17:15 -07:00
Pranav 07c28f17ca
Fix missing format strings in calls to DruidException.build (#15056)
* Fix the NPE bug in nonStrictFormat

* using non null format string

* using Assert.assertThrows
2023-09-29 17:00:36 -07:00
YongGang 86087cee0a
Fix Peon not fail gracefully (#14880)
* fix Peon not fail gracefully

* move methods to Task interface

* fix checkstyle

* extract to interface

* check runThread nullability

* fix merge conflict

* minor refine

* minor refine

* fix unit test

* increase latch waiting time
2023-09-29 12:39:59 -07:00
Karan Kumar 2f1bcd6717
Adding `"segment/scan/active" metric for processing thread pool. (#15060) 2023-09-29 12:34:28 -07:00
Rishabh Singh ebb9724c26
Pass jvm option to write heap dump on out of memory (#15053) 2023-09-29 17:54:53 +05:30
Yuanli Han 9a4433bbad
Fix invalid segment path when using hdfs as the intermediate deepstore (#14984)
This PR fixes the invalid segment path when enabling druid_processing_intermediaryData_storage_type: "deepstore" and using hdfs as the deep store.
2023-09-29 12:53:46 +05:30
Zoltan Haindrich db71e28808
Enable SortProjectTransposeRule (#15002)
contains Enable already passing tests in DecoupledPlanningCalciteQueryTest #14996
enables a transpose rule to support a query plan in which the plan was in the shape:
Sort
  Project
     Aggregate
2023-09-29 10:49:03 +05:30
Zoltan Haindrich 5f3b310115
Build reliablity fixes (#15048)
* disable parallel builds; enable batch mode to get rid of transfer progress

* restore .m2 from setup-java if not found

* some change to sql

* add ws

* fix quote

* fix quote

* undo querytest change

* nullhandling in mvtest

* init more

* skip commitid plugin

* add-back 1.0C to build ; remove redundant skip-s from copy-resources; add comment
2023-09-28 12:27:52 -07:00
Tejaswini Bandlamudi fa61e654e4
fix uploading IT docker logs to GHA artifacts (#15046) 2023-09-28 15:25:52 +05:30
Zoltan Haindrich 022950a0c5
MV_FILTER_ONLY may run into Exceptions in case duplicate values were processed (#15012) 2023-09-27 19:19:42 +05:30
George Shiqi Wu fc929e6d25
upgrade shade (#15042)
Upgrade maven shade plugin to try to fix build failures

Sometimes we get maven shade errors in our integ tests becasue we don't run clean in between runs to clear the cache in order to speed them up. This can lead to the below error.
Error: Failed to execute goal org.apache.maven.plugins:maven-shade-plugin:3.2.4:shade (opentelemetry-extension) on project opentelemetry-emitter: Error creating shaded jar: duplicate entry: META-INF/services/org.apache.druid.opentelemetry.shaded.io.grpc.NameResolverProvider

See: https://issues.apache.org/jira/projects/MSHADE/issues/MSHADE-425?filter=allissues
An example run that failed: https://github.com/apache/druid/actions/runs/6301662092/job/17117142375?pr=14887

According to the ticket this is fixed by updating shade to 3.4.1.

When I updated to 3.4.1 I kept running into a different issue during static checks. (Caused by: java.lang.NoClassDefFoundError: com/github/rvesse/airline/parser/errors/ParseException)

I had to add the createDependencyReducedPom: false to get the build to pass.

The dependency reduced pom feature was added in 3.3.0 which we were not using before so setting it explicitly to false should not be a issue. https://issues.apache.org/jira/browse/MSHADE-36)
2023-09-27 18:19:21 +05:30
George Shiqi Wu 8e22a178cc
Support getTaskLocation for mixed task runner (#15033)
The KubernetesAndWorkerTaskRunner currently doesn't implement getTaskLocation, so tasks run by it will show a unknown TaskLocation in the druid console after a task has completed.

Fix bug in KubernetesAndWorkerTaskRunner that manifests as missing information in the druid Web Console.
2023-09-27 08:57:36 +05:30
Gian Merlino 3dabfead05
Fix getResultType for HLL, quantiles aggregators. (#15043)
The aggregators had incorrect types for getResultType when shouldFinalze
is false. They had the finalized type, but they should have had the
intermediate type.

Also includes a refactor of how ExprMacroTable is handled in tests, to make
it easier to add tests for this to the MSQ module. The bug was originally
noticed because the incorrect result types caused MSQ queries with DS_HLL
to behave erratically.
2023-09-27 08:51:14 +05:30
YongGang 7301e60a9c
Add metrics for number of segments generated per task in MSQ (#14980)
Add ingest/tombstones/count and ingest/segments/count metrics in MSQ.
2023-09-26 02:46:33 +05:30
Soumyava 75af741a96
Revert "SQL: Plan non-equijoin conditions as cross join followed by filter. (#14978)" (#15029)
This reverts commit 4f498e6469.
2023-09-25 11:35:44 -07:00
Abhishek Radhakrishnan ba6101ad75
Remove `EOFException` catch block from the Avro decoders (#15018)
* Remove stale comment since we're on avro version 1.11.1

* Update exception blocks. With 1.11.1, read() only throws IOException.

* Unit tests

* Cleanup and add more tests.
2023-09-25 08:38:41 -07:00
AmatyaAvadhanula f7a549123b
Commit segments only when they are covered by active locks (#15027)
* Commit segments only when they are covered by active locks
2023-09-25 13:45:42 +05:30
Tejaswini Bandlamudi 48b6d2abf9
skip org.owasp:dependency-check on extensions-contrib modules and suppress false-positive gRPC CVEs (#15026) 2023-09-25 12:14:42 +05:30
Gian Merlino 0850e615b2
Remove istrue, isfalse vectorized impls. (#14991)
These were added in #14977, but the implementations are incorrect, because they return null when the input arg is null. They should return false when the input is null. Remove them for now, rather than fixing them, since they're so new that they might as well never have existed.
2023-09-25 11:34:24 +05:30
Soumyava c184b5250f
Unnest now works on MSQ (#14886)
This entails:
    Removing the enableUnnest flag and additional machinery
    Updating the datasource plan and frame processors to support unnest
    Adding support in MSQ for UnnestDataSource and FilteredDataSource
    CalciteArrayTest now has a MSQ test component
    Additional tests for Unnest on MSQ
2023-09-25 09:19:21 +05:30
AmatyaAvadhanula c62193c4d7
Add support for concurrent batch Append and Replace (#14407)
Changes:
- Add task context parameter `taskLockType`. This determines the type of lock used by a batch task.
- Add new task actions for transactional replace and append of segments
- Add methods StorageCoordinator.commitAppendSegments and commitReplaceSegments
- Upgrade segments to appropriate versions when performing replace and append
- Add new metadata table `upgradeSegments` to track segments that need to be upgraded
- Add tests
2023-09-25 07:06:37 +05:30
Kashif Faraz d7c152c82c
Add a TaskReport for "kill" tasks (#15023)
- Add `KillTaskReport` that contains stats for `numSegmentsKilled`,
`numBatchesProcessed`, `numSegmentsMarkedAsUnused`
- Fix bug where exception message had no formatter but was was still being passed some args.
- Add some comments regarding deprecation of `markAsUnused` flag.
2023-09-23 07:44:27 +05:30
YongGang be3f93e3cf
Restore tasks when lifecycle start (#14909)
* K8s tasks restore should be from lifecycle start

* add test

* add more tests

* fix test

* wait tasks restore finish when start

* fix style

* revert previous change and add comment
2023-09-22 12:03:34 -07:00
Karan Kumar 5cee9f6148
Allow cancellation of MSQ tasks if they are waiting for segments to load (#15000)
With PR #14322 , MSQ insert/Replace q's will wait for segment to be loaded on the historical's before finishing.

The patch introduces a bug where in the main thread had a thread.sleep() which could not be interrupted via the cancel calls from the overlord.

This new patch addressed that problem by moving the thread.sleep inside a thread of its own. Thus the main thread is now waiting on the future object of this execution.

The cancel call can now shutdown the executor service via another method thus unblocking the main thread to proceed.
2023-09-22 11:21:04 +05:30
Kashif Faraz 409bffe7f2
Rename IMSC.announceHistoricalSegments to commitSegments (#15021)
This commit pulls out some changes from #14407 to simplify that PR.

Changes:
- Rename `IndexerMetadataStorageCoordinator.announceHistoricalSegments` to `commitSegments`
- Rename the overloaded method to `commitSegmentsAndMetadata`
- Fix some typos
2023-09-21 16:19:03 +05:30
Zoltan Haindrich e76962f453
Use annotation to mark DecoupleIgnore (#15005) 2023-09-21 12:36:52 +05:30
Laksh Singla ebb794632a
Allow users with STATE permissions to read and write the state APIs for querying with deep storage (#14944)
Currently, only the user who has submitted the async query has permission to interact with the status APIs for that async query. However, often we want an administrator to interact with these resources as well.
Druid handles these with the STATE resource traditionally, and if the requesting user has necessary permissions on it as well, alternatively, they should be allowed to interact with the status APIs, irrespective of whether they are the submitter of the query.
2023-09-21 06:55:07 +05:30
Pranav 883c2692d2
Adding new function decode_base64_utf8 and expr macro (#14943)
* Adding new function decode_base64_utf8 and expr macro

* using BaseScalarUnivariateMacroFunctionExpr

* Print stack trace in case of debug in ChainedExecutionQueryRunner

* fix static check
2023-09-20 17:06:34 -07:00
Xavier Léauté 22abc10f24
update RoaringBitmap to 0.9.49 (#15006)
* update RoaringBitmap to 0.9.49

update RoaringBitmap from 0.9.0 to 0.9.49

Many optimizations and improvements have gone into recent releases of
RoaringBitmap. It seems worthwhile to incorporate those.

* implement workaround for BatchIterator interface change

* add test case for BatchIteratorAdapter.advanceIfNeeded
2023-09-20 15:52:27 -07:00
Laksh Singla 82e809c8d0
fix (#15017) 2023-09-20 15:48:26 -07:00