Commit Graph

459 Commits

Author SHA1 Message Date
Gian Merlino 69aac6c8dd
Direct UTF-8 access for "in" filters. (#12517)
* Direct UTF-8 access for "in" filters.

Directly related:

1) InDimFilter: Store stored Strings (in ValuesSet) plus sorted UTF-8
   ByteBuffers (in valuesUtf8). Use valuesUtf8 whenever possible. If
   necessary, the input set is copied into a ValuesSet. Much logic is
   simplified, because we always know what type the values set will be.
   I think that there won't even be an efficiency loss in most cases.
   InDimFilter is most frequently created by deserialization, and this
   patch updates the JsonCreator constructor to deserialize
   directly into a ValuesSet.

2) Add Utf8ValueSetIndex, which InDimFilter uses to avoid UTF-8 decodes
   during index lookups.

3) Add unsigned comparator to ByteBufferUtils and use it in
   GenericIndexed.BYTE_BUFFER_STRATEGY. This is important because UTF-8
   bytes can be compared as bytes if, and only if, the comparison
   is unsigned.

4) Add specialization to GenericIndexed.singleThreaded().indexOf that
   avoids needless ByteBuffer allocations.

5) Clarify that objects returned by ColumnIndexSupplier.as are not
   thread-safe. DictionaryEncodedStringIndexSupplier now calls
   singleThreaded() on all relevant GenericIndexed objects, saving
   a ByteBuffer allocation per access.

Also:

1) Fix performance regression in LikeFilter: since #12315, it applied
   the suffix matcher to all values in range even for type MATCH_ALL.

2) Add ObjectStrategy.canCompare() method. This fixes LikeFilterBenchmark,
   which was broken due to calls to strategy.compare in
   GenericIndexed.fromIterable.

* Add like-filter implementation tests.

* Add in-filter implementation tests.

* Add tests, fix issues.

* Fix style.

* Adjustments from review.
2022-05-20 01:51:28 -07:00
Gian Merlino 4631cff2a9
Free ByteBuffers in tests and fix some bugs. (#12521)
* Ensure ByteBuffers allocated in tests get freed.

Many tests had problems where a direct ByteBuffer would be allocated
and then not freed. This is bad because it causes flaky tests.

To fix this:

1) Add ByteBufferUtils.allocateDirect(size), which returns a ResourceHolder.
   This makes it easy to free the direct buffer. Currently, it's only used
   in tests, because production code seems OK.

2) Update all usages of ByteBuffer.allocateDirect (off-heap) in tests either
   to ByteBuffer.allocate (on-heap, which are garbaged collected), or to
   ByteBufferUtils.allocateDirect (wherever it seemed like there was a good
   reason for the buffer to be off-heap). Make sure to close all direct
   holders when done.

* Changes based on CI results.

* A different approach.

* Roll back BitmapOperationTest stuff.

* Try additional surefire memory.

* Revert "Roll back BitmapOperationTest stuff."

This reverts commit 49f846d9e3.

* Add TestBufferPool.

* Revert Xmx change in tests.

* Better behaved NestedQueryPushDownTest. Exit tests on OOME.

* Fix TestBufferPool.

* Remove T1C from ARM tests.

* Somewhat safer.

* Fix tests.

* Fix style stuff.

* Additional debugging.

* Reset null / expr configs better.

* ExpressionLambdaAggregatorFactory thread-safety.

* Alter forkNode to try to get better info when a JVM crashes.

* Fix buffer retention in ExpressionLambdaAggregatorFactory.

* Remove unused import.
2022-05-19 07:42:29 -07:00
Lucas Capistrant deb69d1bc0
Allow coordinator to be configured to kill segments in future (#10877)
Allow a Druid cluster to kill segments whose interval_end is a date in the future. This can be done by setting druid.coordinator.kill.durationToRetain to a negative period. For example PT-24H would allow segments to be killed if their interval_end date was 24 hours or less into the future at the time that the kill task is generated by the system.

A cluster operator can also disregard the druid.coordinator.kill.durationToRetain entirely by setting a new configuration, druid.coordinator.kill.ignoreDurationToRetain=true. This ignores interval_end date when looking for segments to kill, and instead is capable of killing any segment marked unused. This new configuration is off by default, and a cluster operator should fully understand and accept the risks if they enable it.
2022-05-11 07:35:15 +05:30
Tejaswini Bandlamudi 1d1f53e7d5
Improve error messages when URI points to a file that doesn't exist (#12490) 2022-05-01 11:26:16 +05:30
Gian Merlino 529b983ad0
GroupBy: Reduce allocations by reusing entry and key holders. (#12474)
* GroupBy: Reduce allocations by reusing entry and key holders.

Two main changes:

1) Reuse Entry objects returned by various implementations of
   Grouper.iterator.

2) Reuse key objects contained within those Entry objects.

This is allowed by the contract, which states that entries must be
processed and immediately discarded. However, not all call sites
respected this, so this patch also updates those call sites.

One particularly sneaky way that the old code retained entries too long
is due to Guava's MergingIterator and CombiningIterator. Internally,
these both advance to the next value prior to returning the current
value. So, this patch addresses that in two ways:

1) For merging, we have our own implementation MergeIterator already,
   although it had the same problem. So, this patch updates our
   implementation to return the current item prior to advancing to the
   next item. It also adds a forbidden-api entry to ensure that this
   safer implementation is used instead of Guava's.

2) For combining, we address the problem in a different way: by copying
   the key when creating the new, combined entry.

* Attempt to fix test.

* Remove unused import.
2022-04-28 23:21:13 -07:00
Frank Chen df074f2f96
Improve exception message for native binary operators (#12335)
* Improve exception message

* Update message
2022-04-28 10:20:16 +08:00
Gian Merlino 7b89682bbe
DimensionRangeShardSpec speed boost. (#12477)
* DimensionRangeShardSpec speed boost.

Calling isEmpty() and equals() on RangeSets is expensive, because these
fall back on default implementations that call size(). And size() is
_also_ a default implementation that iterates the entire collection.

* Fix and test from code review.
2022-04-27 14:20:35 -07:00
Gian Merlino a2bad0b3a2
Reduce allocations due to Jackson serialization. (#12468)
* Reduce allocations due to Jackson serialization.

This patch attacks two sources of allocations during Jackson
serialization:

1) ObjectMapper.writeValue and JsonGenerator.writeObject create a new
   DefaultSerializerProvider instance for each call. It has lots of
   fields and creates pressure on the garbage collector. So, this patch
   adds helper functions in JacksonUtils that enable reuse of
   SerializerProvider objects and updates various call sites to make
   use of this.

2) GroupByQueryToolChest copies the ObjectMapper for every query to
   install a special module that supports backwards compatibility with
   map-based rows. This isn't needed if resultAsArray is set and
   all servers are running Druid 0.16.0 or later. This release was a
   while ago. So, this patch disables backwards compatibility by default,
   which eliminates the need to copy the heavyweight ObjectMapper. The
   patch also introduces a configuration option that allows admins to
   explicitly enable backwards compatibility.

* Add test.

* Update additional call sites and add to forbidden APIs.
2022-04-27 14:17:26 -07:00
Gian Merlino 72d15ab321
JvmMonitor: Handle more generation and collector scenarios. (#12469)
* JvmMonitor: Handle more generation and collector scenarios.

ZGC on Java 11 only has a generation 1 (there is no 0). This causes
a NullPointerException when trying to extract the spacesCount for
generation 0. In addition, ZGC on Java 15 has a collector number 2
but no spaces in generation 2, which breaks the assumption that
collectors always have same-numbered spaces.

This patch adjusts things to be more robust, enabling the JvmMonitor
to work properly for ZGC on both Java 11 and 15.

* Test adjustments.

* Improve surefire arglines.

* Need a placeholder
2022-04-27 11:18:40 -07:00
Gian Merlino e7e49ec9c8
For the various Yielder objects, don't create new Yielders and instead mutate state. (#12475)
Co-authored-by: imply-cheddar <86940447+imply-cheddar@users.noreply.github.com>
2022-04-27 10:52:20 -07:00
Abhishek Agarwal 2fe053c5cb
Bump up the versions (#12480) 2022-04-27 14:28:20 +05:30
Will Xu 4868ef9529
Enable Arm builds (#12451)
This PR enables ARM builds on Travis. I've ported over the changes from @martin-g on reducing heap requirements for some of the tests to ensure they run well on Travis arm instances.
2022-04-26 20:14:40 +05:30
Didip Kerabat 2473de2552
Metrics for shenandoah based on this source code: 554caf33a0/src/hotspot/share/gc/shenandoah/shenandoahMonitoringSupport.cpp (L65) (#12369)
Co-authored-by: Didip Kerabat <didip@apple.com>
2022-04-22 11:44:05 -07:00
Tejaswini Bandlamudi 177e1856cd
Fix GCS based ingestion if bucket name contains underscores (#12445)
GCP allows bucket names to contain underscores. When a location in such a bucket
is mapped to `java.net.URI`, `URI.getHost()` returns null. `URI.getHost()` is used as
the bucket name in `CloudObjectLocation`, leading to an NPE.

This commit uses `URI.getAuthority()` as the bucket name if `URI.getHost()` is null.
2022-04-21 09:22:35 +05:30
Agustin Gonzalez 0460d45e92
Make tombstones ingestible by having them return an empty result set. (#12392)
* Make tombstones ingestible by having them return an empty result set.

* Spotbug

* Coverage

* Coverage

* Remove unnecessary exception (checkstyle)

* Fix integration test and add one more to test dropExisting set to false over tombstones

* Force dropExisting to true in auto-compaction when the interval contains only tombstones

* Checkstyle, fix unit test

* Changed flag by mistake, fixing it

* Remove method from interface since this method is specific to only DruidSegmentInputentity

* Fix typo

* Adapt to latest code

* Update comments when only tombstones to compact

* Move empty iterator to a new DruidTombstoneSegmentReader

* Code review feedback

* Checkstyle

* Review feedback

* Coverage
2022-04-15 09:08:06 -07:00
hqx871 a22d413725
Use binary search to improve DimensionRangeShardSpec lookup (#12417)
If there are many shards, mapper of IndexGeneratorJob seems to spend a lot of time in calling
DimensionRangeShardSpec.isInChunk to lookup target shard. This can be significantly improved
by using binary search instead of comparing an input row to every shardSpec.

Changes:
* Add `BaseDimensionRangeShardSpec` which provides a binary-search-based
   implementation for `createLookup`
* `DimensionRangeShardSpec`, `SingleDimensionShardSpec`, and 
   `DimensionRangeBucketShardSpec` now extend `BaseDimensionRangeShardSpec`
2022-04-15 21:37:06 +05:30
Clint Wylie 5824ab9608
fix issue with boolean expression input (#12429) 2022-04-13 16:34:01 -07:00
Jihoon Son 5e5625f3ae
Fix indexMerger to respect the includeAllDimensions flag (#12428)
* Fix indexMerger to respect flag includeAllDimensions flag; jsonInputFormat should set keepNullColumns if useFieldDiscovery is set

* address comments
2022-04-13 12:43:11 -07:00
Maytas Monsereenusorn 8edea5a82d
Add a new flag for ingestion to preserve existing metrics (#12185)
* add impl

* add impl

* fix checkstyle

* add impl

* add unit test

* fix stuff

* fix stuff

* fix stuff

* add unit test

* add more unit tests

* add more unit tests

* add IT

* add IT

* add IT

* add IT

* add ITs

* address comments

* fix test

* fix test

* fix test

* address comments

* address comments

* address comments

* fix conflict

* fix checkstyle

* address comments

* fix test

* fix checkstyle

* fix test

* fix test

* fix IT
2022-04-08 11:02:02 -07:00
somu-imply a1ea658115
Introducing a new config to ignore nulls while computing String Cardinality (#12345)
* Counting nulls in String cardinality with a config

* Adding tests for the new config

* Wrapping the vectorize part to allow backward compatibility

* Adding different tests, cleaning the code and putting the check at the proper position, handling hasRow() and hasValue() changes

* Updating testcase and code

* Adding null handling test to improve coverage

* Checkstyle fix

* Adding 1 more change in docs

* Making docs clearer
2022-03-29 14:31:36 -07:00
Maytas Monsereenusorn dbb9518f50
Fix auto compaction by adjusting compaction task's interval to align with segmentGranularity when segmentGranularity is set (#12334)
* add impl

* add ITs

* address comments

* address comments

* address comments

* fix failure

* fix checkstyle

* fix checkstyle
2022-03-18 12:46:16 -07:00
Xavier Léauté c33fa11669
improve test compatibility with Java 17 and remove deprecated methods (#12341)
* remove use of reflection in EnvironmentVariableDynamicConfigProvider for Java 17 compatibility
* fix mocks mock objects not getting closed properly, causing issues with Java 17
* remove use of deprecated methods and rules in tests
2022-03-18 08:19:28 -07:00
Jihoon Son 5e23674fe5
Fix a race condition in the '/tasks' Overlord API (#12330)
* finds complete and active tasks from the same snapshot

* overlord resource

* unit test

* integration test

* javadoc and cleanup

* more cleanup

* fix test and add more
2022-03-17 10:47:45 +09:00
Dr. Sizzles 69f928f50e
Adding k8s support for human readable parsing (#12316)
* Adding k8s support for human readable parsing

* Update docs/configuration/human-readable-byte.md

Co-authored-by: Frank Chen <frankchen@apache.org>

* Update docs/configuration/human-readable-byte.md

Co-authored-by: Frank Chen <frankchen@apache.org>

* Update core/src/main/java/org/apache/druid/java/util/common/HumanReadableBytes.java

Co-authored-by: Frank Chen <frankchen@apache.org>

* Changes per review

Co-authored-by: Rahul Gidwani <r_gidwani@apple.com>
Co-authored-by: Frank Chen <frankchen@apache.org>
2022-03-16 11:18:47 +08:00
Xavier Léauté 5d02a91faa
upgrade Error Prone to 2.11 (requires Java 11) (#12306)
The latest version of Error Prone now requires Java 11. Upgrading means we can
remove a lot of the maven profile complexity required to run checks with Java 8.
This also requires switching our strict build to use Java 11.

* update error-prone to 2.11
* remove need for specific maven profiles for Java 8 and Java 15
* fix additional Error Prone warnings with Java 11
* update strict build to use Java 11
2022-03-14 19:40:48 -07:00
AmatyaAvadhanula 7bf1d8c5c0
Facilitate lazy initialization of connections to mitigate overwhelming of Coordinator (#12298)
Add config for eager / lazy connection initialization in ResourcePool

Description
Currently, when multiple tasks are launched, each of them eagerly initializes a full pool's worth of connections to the coordinator.

While this is acceptable when the parameter for number of eagerConnections (== maxSize) is small, this can be problematic in environments where it's a large value (say 1000) and multiple tasks are launched simultaneously, which can cause a large number of connections to be created to the coordinator, thereby overwhelming it.

Patch
Nodes like the broker may require eager initialization of resources and do not create connections with the Coordinator.
It is unnecessary to do this with other types of nodes.

A config parameter eagerInitialization is added, which when set to true, initializes the max permissible connections when ResourcePool is initialized.

If set to false, lazy initialization of connection resources takes place.

NOTE: All nodes except the broker have this new parameter set to false in the quickstart as part of this PR

Algorithm
The current implementation relies on the creation of maxSize resources eagerly.

The new implementation's behaviour is as follows:

If a resource has been previously created and is available, lend it.
Else if the number of created resources is less than the allowed parameter, create and lend it.
Else, wait for one of the lent resources to be returned.
2022-03-09 23:17:43 +05:30
Agustin Gonzalez abe76ccb90
Batch ingestion replace (#12137)
* Tombstone support for replace functionality

* A used segment interval is the interval of a current used segment that overlaps any of the input intervals for the spec

* Update compaction test to match replace behavior

* Adapt ITAutoCompactionTest to work with tombstones rather than dropping segments. Add support for tombstones in the broker.

* Style plus simple queriableindex test

* Add segment cache loader tombstone test

* Add more tests

* Add a method to the LogicalSegment to test whether it has any data

* Test filter with some empty logical segments

* Refactor more compaction/dropexisting tests

* Code coverage

* Support for all empty segments

* Skip tombstones when looking-up broker's timeline. Discard changes made to tool chest to avoid empty segments since they will no longer have empty segments after lookup because we are skipping over them.

* Fix null ptr when segment does not have a queriable index

* Add support for empty replace interval (all input data has been filtered out)

* Fixed coverage & style

* Find tombstone versions from lock versions

* Test failures & style

* Interner was making this fail since the two segments were consider equal due to their id's being equal

* Cleanup tombstone version code

* Force timeChunkLock whenever replace (i.e. dropExisting=true) is being used

* Reject replace spec when input intervals are empty

* Documentation

* Style and unit test

* Restore test code deleted by mistake

* Allocate forces TIME_CHUNK locking and uses lock versions. TombstoneShardSpec added.

* Unused imports. Dead code. Test coverage.

* Coverage.

* Prevent killer from throwing an exception for tombstones. This is the killer used in the peon for killing segments.

* Fix OmniKiller + more test coverage.

* Tombstones are now marked using a shard spec

* Drop a segment factory.json in the segment cache for tombstones

* Style

* Style + coverage

* style

* Add TombstoneLoadSpec.class to mapper in test

* Update core/src/main/java/org/apache/druid/segment/loading/TombstoneLoadSpec.java

Typo

Co-authored-by: Jonathan Wei <jon-wei@users.noreply.github.com>

* Update docs/configuration/index.md

Missing

Co-authored-by: Jonathan Wei <jon-wei@users.noreply.github.com>

* Typo

* Integrated replace with an existing test since the replace part was redundant and more importantly, the test file was very close or exceeding the 10 min default "no output" CI Travis threshold.

* Range does not work with multi-dim

Co-authored-by: Jonathan Wei <jon-wei@users.noreply.github.com>
2022-03-08 20:07:02 -07:00
Gian Merlino 28f8bcce9b
Always reopen stream in FileUtils.copyLarge, RetryingInputStream. (#12307)
* Always reopen stream in FileUtils.copyLarge, RetryingInputStream.

When an InputStream throws an exception from one of its read methods,
we should assume it's bad and reopen it.

The main changes here are:

- In FileUtils.copyLarge, replace InputStream with InputStreamSupplier.
- In RetryingInputStream, collapse retryCondition and resetCondition
  into a single condition. Also, make it required, since every usage
  is passing in a specific condition anyway.

* Test fixes.

* Fix read impl.
2022-03-05 14:39:14 -08:00
Laksh Singla 3f709db173
Make ParseExceptions more informative (#12259)
This PR aims to make the ParseExceptions in Druid more informative, by adding additional information (metadata) to the ParseException, which can contain additional information about the exception. For example - the path of the file generating the issue, the line number (where it can be easily fetched - like CsvReader)

Following changes are addressed in this PR:

A new class CloseableIteratorWithMetadata has been created which is like CloseableIterator but also has a metadata method that returns a context Map<String, Object> about the current element returned by next().
IntermediateRowParsingReader#read() now attaches the InputEntity and the "record number" which created the exception (while parsing them), and IntermediateRowParsingReader#sample attaches the InputEntity (but not the "record number").
TextReader (and its subclasses), which is a specific implementation of the IntermediateRowParsingReader also include the line number which caused the generation of the error.
This will also help in triaging the issues when InputSourceReader generates ParseException because it can point to the specific InputEntity which caused the exception (while trying to read it).
2022-02-28 22:31:15 +05:30
Xavier Léauté d105519558
Replace use of PowerMock with Mockito (#12282)
Mockito now supports all our needs and plays much better with recent Java versions.
Migrating to Mockito also simplifies running the kind of tests that required PowerMock in the past. 

* replace all uses of powermock with mockito-inline
* upgrade mockito to 4.3.1 and fix use of deprecated methods
* import mockito bom to align all our mockito dependencies
* add powermock to forbidden-apis to avoid accidentally reintroducing it in the future
2022-02-27 22:47:09 -08:00
Xavier Léauté 1434197ee1
update airline dependency to 2.x (#12270)
* upgrade Airline to Airline 2
  https://github.com/airlift/airline is no longer maintained, updating to
  https://github.com/rvesse/airline (Airline 2) to use an actively
  maintained version, while minimizing breaking changes.

  Note, this is a backwards incompatible change, and extensions relying on
  the CliCommandCreator extension point will also need to be updated.

* fix dependency checks where jakarta.inject is now resolved first instead
  of javax.inject, due to Airline 2 using jakarta
2022-02-27 15:19:28 -08:00
Jihoon Son e5ad862665
A new includeAllDimension flag for dimensionsSpec (#12276)
* includeAllDimensions in dimensionsSpec

* doc

* address comments

* unused import and doc spelling
2022-02-25 18:27:48 -08:00
Karan Kumar b86f2d4c2e
Performance fixes in proto readers (#12267) 2022-02-24 23:21:48 +05:30
somu-imply 033989eb1d
Adding vectorized time_shift (#12254)
* Adding vectorized time_shift

* Vectorize time shift, addressing review comments

* Remove an unused import
2022-02-11 14:44:52 -08:00
Clint Wylie 3ee66bb492
allow optimizing sql expressions and virtual columns (#12241)
* rework sql planner expression and virtual column handling

* simplify a bit

* add back and deprecate old methods, more tests, fix multi-value string coercion bug and associated tests

* spotbugs

* fix bugs with multi-value string array expression handling

* javadocs and adjust test

* better

* fix tests
2022-02-09 14:55:50 -08:00
Jihoon Son ab3d994a17
Lazy instantiation for segmentKillers, segmentMovers, and segmentArchivers (#12207)
* working

* Lazily load segmentKillers, segmentMovers, and segmentArchivers

* more tests

* test-jar plugin

* more coverage

* lazy client

* clean up changes

* checkstyle

* i did not change the branch condition

* adjust failure rate to run tests faster

* javadocs

* checkstyle
2022-02-08 13:02:06 -08:00
Clint Wylie ae71e05fc5
array_concat_agg and array_agg support for array inputs (#12226)
* array_concat_agg and array_agg support for array inputs
changes:
* added array_concat_agg to aggregate arrays into a single array
* added array_agg support for array inputs to make nested array
* added 'shouldAggregateNullInputs' and 'shouldCombineAggregateNullInputs' to fix a correctness issue with STRING_AGG and ARRAY_AGG when merging results, with dual purpose of being an optimization for aggregating

* fix test

* tie capabilities type to legacy mode flag about coercing arrays to strings

* oops

* better javadoc
2022-02-07 19:59:30 -08:00
Gian Merlino de82c611de
Harmonize implementations of "visit" for Exprs from ExprMacros. (#12230)
* Harmonize implementations of "visit" for Exprs from ExprMacros.

Many of them had bugs where they would not visit all of the original
arguments. I don't think this has user-visible consequences right now,
but it's possible it would in a future world where "visit" is used
for more stuff than it is today.

So, this patch all updates all implementations to a more consistent
style that emphasizes reapplying the macro to the shuttled args.

* Test fixes, test coverage, PR review comments.
2022-02-04 08:08:54 -08:00
tejaswini-imply 290130b1fa
Fix bug while adding `Range` header in HttpEntity (#12215)
Changes:
- Add `Range` header to the request before opening the connection
- Use header `Content-Range` instead of `Accept-Ranges` as `Content-Range` is guaranteed to be populated if the server is returning a partial response
2022-02-04 18:17:51 +05:30
Clint Wylie f9b406c8f2
add backwards compatibility mode for multi-value string array null value coercion (#12210) 2022-01-31 22:38:15 -08:00
Karan Kumar 96b3498a40
Grouping on arrays as arrays (#12078)
* init multiValue column group by

* Changing sorting to Lexicographic as default

* Adding initial tests

* 1.Fixing test cases adding
2.Optimized inmem structs

* Linking SQL layer to native layer

* Adding multiDimension support to group by column strategy

* 1. Removing array coercion in Calcite layer
2. Removing ResultRowDeserializer

* 1. Supporting all primitive array types
2. Removing dimension spec as part of columnSelector

* 1. Supporting all primitive array types
2. Removing dimension spec as part of columnSelector

* 1. Checkstyle things
2. Removing flag

* Minor naming things

* CheckStyle Things

* Fixing test case

* Fixing hashing

* 1. Adding the MV function
2. Added few test cases

* 1. Adding MV function test cases

* Adding Selector strategy function test cases

* Fixing ClientQuerySegmentWalkerTest

* Adding GroupByQueryRunnerTest test cases

* Fixing test cases

* Adding few more test cases

* Fixing Exception asset statement and intellij inspection

* Adding null compatibility tests

* Review comments

* Fixing few failing tests

* Fixing few failing tests

* Do no convert to topN Q incase of group by on array

* Fixing checkstyle

* Fixing differences between jdk's class cast exception message

* 1. Fixing ordering if the grouping key is an array

* Fixing DefaultLimitSpec

* Fixing CalciteArraysQueryTest

* Dummy commit for LGTM

* changes:
* only coerce multi-value string null values when `ExpressionPlan.Trait.NEEDS_APPLIED` is set
* correct return type inference for ARRAY_APPEND,ARRAY_PREPEND,ARRAY_SLICE,ARRAY_CONCAT
* fix bug with ExprEval.ofType when actual type of object from binding doesn't match its claimed type

* Review comments

* Fixing test cases

* Fixing spot bugs

* Fixing strict compile

Co-authored-by: Clint Wylie <cwylie@apache.org>
2022-01-25 20:30:56 -08:00
Maytas Monsereenusorn bd7fe45da0
Support adding metrics in Auto Compaction (#12125)
* add impl

* add impl

* add unit tests

* add unit tests

* add unit tests

* add unit tests

* add unit tests

* add integration tests

* add integration tests

* fix LGTM

* fix test

* remove doc
2022-01-17 20:19:31 -08:00
Clint Wylie 1dba089a62
fix array type strategy write size tracking (#12150)
* fix array type strategy write size tracking

* fix checkstyle
2022-01-13 10:22:40 -08:00
Xavier Léauté e56ea31697
follow-up to fix formatting broken in #12147 (#12148)
follow-up to #12147 to fix the build
2022-01-12 20:59:32 -08:00
Xavier Léauté 168187e6df
avoid unnecessary String.format calls in IdUtils.validateId (#12147)
Based on profiling data, about 25% of the time de-serializing DataSchema
is spent on formatting strings in validateId.

This can add up quickly, especially when de-serializing task information
in the overlord, where in can consume almost 2% of CPU if there are many
tasks.

Since the formatting is unnecessary unless the checks fail, we can
leverage the built-in formatting of Preconditions.checkArgument instead
to avoid the cost.
2022-01-12 16:34:40 -08:00
Clint Wylie 7cf9192765
fix delegated smoosh writer and some new facilities for segment writeout medium (#12132)
* fix delegated smoosh writer and some new facilities for segment writeout medium
changes:
* fixed issue with delegated `SmooshedWriter` when writing files that look like paths, causing `NoSuchFileException` exceptions when attempting to open a channel to the file
* `FileSmoosher.addWithSmooshedWriter` when _not_ delegating now checks that it is still open when closing, making it a no-op if already closed (allowing column serializers to add additional files and avoid delegated mode if they are finished writing out their own content and ned to add additional files)
* add `makeChildWriteOutMedium` to `SegmentWriteOutMedium` interface, which allows users of a shared medium to clean up `WriteOutBytes` if they fully control the lifecycle. there are no callers of this yet, adding for future functionality
* `OnHeapByteBufferWriteOutBytes` now can be marked as not open so it `OnHeapMemorySegmentWriteOutMedium` can now behave identically to other medium implementations

* fix to address nit - use AtomicLong
2022-01-10 22:25:19 -08:00
Clint Wylie e583033231
add 'TypeStrategy' to types (#11888)
* add TypeStrategy - value comparators and binary serialization for any TypeSignature
2022-01-10 17:12:14 -08:00
AmatyaAvadhanula c0b1514177
Segment pruning for multi-dim partitioning given query domain (#12046)
Segment pruning for multi-dim partitioning for a given query

DimensionRangeShardSpec#possibleInDomain has been modified to enhance pruning when multi-dim partitioning is used.

Idea
While iterating through each dimension,

If query domain doesn't overlap with the set of permissible values in the segment, the segment is pruned.
If the overlap happens on a boundary, consider the next dimensions.
If there is an overlap within the segment boundaries, the segment cannot be pruned.
2021-12-17 12:44:43 +05:30
Suneet Saldanha 25ac04e067
MySqlFirehoseDatabaseConnector uses configured driver class name (#12049) 2021-12-09 20:58:55 -08:00
Frank Chen 58245b4617
Support JsonPath functions in JsonPath expressions (#11722)
* Add jsonPath functions support

* Add jsonPath function test for Avro

* Add jsonPath function length() to Orc

* Add jsonPath function length() to Parquet

* Add more tests to ORC format

* update doc

* Fix exception during ingestion

* Add IT test case

* Revert "Fix exception during ingestion"

This reverts commit 5a5484b9ea.

* update IT test case

* Add 'keys()'

* Commit IT test case

* Fix UT
2021-12-10 10:53:23 +08:00