Commit Graph

3255 Commits

Author SHA1 Message Date
Kashif Faraz f694066965
When removeNullBytes is set, length calculations did not take into account null bytes. (#17232) (#17266)
* When replaceNullBytes is set, length calculations did not take into account null bytes.

Co-authored-by: Karan Kumar <karankumar1100@gmail.com>
2024-10-07 20:51:54 +05:30
Abhishek Agarwal 52441c005c
add druid.expressions.allowVectorizeFallback and default to false (#17248) (#17260)
changes:

adds ExpressionProcessing.allowVectorizeFallback() and ExpressionProcessingConfig.allowVectorizeFallback(), defaulting to false until few remaining bugs can be fixed (mostly complex types and some odd interactions with mixed types)
add cannotVectorizeUnlessFallback functions to make it easy to toggle the default of this config, and easy to know what to delete when we remove it in the future

Co-authored-by: Clint Wylie <cwylie@apache.org>
2024-10-06 12:29:16 +05:30
Clint Wylie 7b3fc4e768
backport projections (#17257)
* abstract `IncrementalIndex` cursor stuff to prepare for using different "views" of the data based on the cursor build spec (#17064)

* abstract `IncrementalIndex` cursor stuff to prepare to allow for possibility of using different "views" of the data based on the cursor build spec
changes:
* introduce `IncrementalIndexRowSelector` interface to capture how `IncrementalIndexCursor` and `IncrementalIndexColumnSelectorFactory` read data
* `IncrementalIndex` implements `IncrementalIndexRowSelector`
* move `FactsHolder` interface to separate file
* other minor refactorings

* add DataSchema.Builder to tidy stuff up a bit (#17065)

* add DataSchema.Builder to tidy stuff up a bit

* fixes

* fixes

* more style fixes

* review stuff

* Projections prototype (#17214)
2024-10-05 23:03:41 +05:30
Kashif Faraz f27a1dc651
[Backport] Dart: Smoother handling of stage early exit (#17228) (#17069) (#17256)
* MSQ: Properly report errors that occur when starting up RunWorkOrder. (#17069)
* Dart: Smoother handling of stage early-exit. (#17228)
---------
Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-10-05 17:29:34 +05:30
Kashif Faraz 10528a6d9e
[Backport] Patches (#17039) (#17173) (#17216) (#17224) (#17230) (#17238) (#17251)
* SQL: Use regular filters for time filtering in subqueries. (#17173)
* RunWorkOrder: Account for two simultaneous statistics collectors. (#17216)
* DartTableInputSpecSlicer: Fix for TLS workers. (#17224)
* Upgrade avro - minor version (#17230)
* SuperSorter: Don't set allDone if it's already set. (#17238)
* Decoupled planning: improve join support (#17039)
---------
Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
Co-authored-by: Zoltan Haindrich <kirk@rxd.hu>
2024-10-05 09:22:57 +05:30
Clint Wylie 0ffdbaa6eb
read metadata in SimpleQueryableIndex if available to compute segment ordering (#17181) (#17191) 2024-10-04 21:15:27 +05:30
Kashif Faraz 5fbdc387c3
[Backport] MSQ profile for Brokers and Historicals. (#17140) (#17072) (#17244)
Backport for the following patches
* MSQ profile for Brokers and Historicals. (#17140)
* Remove workerId parameter from postWorkerError. (#17072)
---------
Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-10-04 18:45:46 +05:30
Kashif Faraz 50eb7321d2
[Backport] Move TerminalStageSpecFactory packages (#17049) and log.warn on ArrayIngestMode.MVD (#17164) (#17242)
* Move TerminalStageSpecFactory packages. (#17049)
* log.warn anytime a column is relying on ArrayIngestMode.MVD (#17164)
---------
Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
Co-authored-by: Clint Wylie <cwylie@apache.org>
2024-10-04 14:21:28 +05:30
Kashif Faraz b275ffed8d
[Backport] Changes required for Dart (#17046) (#17047) (#17048) (#17066) (#17074) (#17076) (#17077) (#17193) (#17243)
Backport the following patches for a clean backport of Dart changes
1. Add "targetPartitionsPerWorker" setting for MSQ. (#17048)
2. MSQ: Improved worker cancellation. (#17046)
3. Add "includeAllCounters()" to WorkerContext. (#17047)
4. MSQ: Include worker context maps in WorkOrders. (#17076)
5. TableInputSpecSlicer changes to support running on Brokers. (#17074)
6. Fix call to MemoryIntrospector in IndexerControllerContext. (#17066)
7. MSQ: Add QueryKitSpec to encapsulate QueryKit params. (#17077)
8. MSQ: Use task context flag useConcurrentLocks to determine task lock type (#17193)
2024-10-04 11:36:05 +05:30
Kashif Faraz 491087fbe3
Modify DataSegmentProvider to also return DataSegment (#17021) (#17217)
Currently, TaskDataSegmentProvider fetches the DataSegment from the Coordinator while loading the segment, but just discards it later. This PR refactors this to also return the DataSegment so that it can be used by workers without a separate fetch.

Co-authored-by: Adarsh Sanjeev <adarshsanjeev@gmail.com>
2024-10-02 14:34:27 +05:30
Kashif Faraz 4a8008aebb
Use the whole frame when writing rows. (#17094) (#17195)
This patch makes the following adjustments to enable writing larger
single rows to frames:
1) RowBasedFrameWriter: Max out allocation size on the final doubling.
   i.e., if the final allocation "naturally" would be 1 MiB but the
   max frame size is 900 KiB, use 900 KiB rather than failing the 1 MiB
   allocation.
2) AppendableMemory: In reserveAdditional, release the last block if it
   is empty. This eliminates waste when a frame writer uses a
   successive-doubling approach to find the right allocation size.
3) ArenaMemoryAllocator: Reclaim memory from the last allocation when
   the last allocation is closed.

Prior to these changes, a single row could be much smaller than the
frame size and still fail to be added to the frame.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-10-02 07:44:33 +05:30
Kashif Faraz 23b9039a02
MSQ: Rework memory management. (#17057) (#17210)
This patch reworks memory management to better support multi-threaded
workers running in shared JVMs. There are two main changes.

First, processing buffers and threads are moved from a per-JVM model to
a per-worker model. This enables queries to hold processing buffers
without blocking other concurrently-running queries. Changes:

- Introduce ProcessingBuffersSet and ProcessingBuffers to hold the
  per-worker and per-work-order processing buffers (respectively). On Peons,
  this is the JVM-wide processing pool. On Indexers, this is a per-worker
  pool of on-heap buffers. (This change fixes a bug on Indexers where
  excessive processing buffers could be used if MSQ tasks ran concurrently
  with realtime tasks.)

- Add "bufferPool" argument to GroupingEngine#process so a per-worker pool
  can be passed in.

- Add "druid.msq.task.memory.maxThreads" property, which controls the
  maximum number of processing threads to use per task. This allows usage of
  multiple processing buffers per task if admins desire.

- IndexerWorkerContext acquires processingBuffers when creating the FrameContext
  for a work order, and releases them when closing the FrameContext.

- Add "usesProcessingBuffers()" to FrameProcessorFactory so workers know
  how many sets of processing buffers are needed to run a given query.

Second, adjustments to how WorkerMemoryParameters slices up bundles, to
favor more memory for sorting and segment generation. Changes:

- Instead of using same-sized bundles for processing and for sorting,
  workers now use minimally-sized processing bundles (just enough to read
  inputs plus a little overhead). The rest is devoted to broadcast data
  buffering, sorting, and segment-building.

- Segment-building is now limited to 1 concurrent segment per work order.
  This allows each segment-building action to use more memory. Note that
  segment-building is internally multi-threaded to a degree. (Build and
  persist can run concurrently.)

- Simplify frame size calculations by removing the distinction between
  "standard" and "large" frames. The new default frame size is the same
  as the old "standard" frames, 1 MB. The original goal of of the large
  frames was to reduce the number of temporary files during sorting, but
  I think we can achieve the same thing by simply merging a larger number
  of standard frames at once.

- Remove the small worker adjustment that was added in #14117 to account
  for an extra frame involved in writing to durable storage. Instead,
  account for the extra frame whenever we are actually using durable storage.

- Cap super-sorter parallelism using the number of output partitions, rather
  than using a hard coded cap at 4. Note that in practice, so far, this cap
  has not been relevant for tasks because they have only been using a single
  processing thread anyway.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-10-01 19:50:24 +05:30
Kashif Faraz 9b192bd5a2
add CursorHolder.isPreAggregated method to allow cursors on pre-aggregated data (#17058) (#17205)
changes:
* CursorHolder.isPreAggregated method indicates that a cursor has pre-aggregated data for all AggregatorFactory specified in a CursorBuildSpec. If true, engines should rewrite the query to use AggregatorFactory.getCombiningAggreggator, and column selector factories will provide selectors with the aggregator interediate type for the aggregator factory name
* Added groupby, timeseries, and topN support for CursorHolder.isPreAggregated
* Added synthetic test since no CursorHolder implementations support isPreAggregated at this point in time

Co-authored-by: Clint Wylie <cwylie@apache.org>
2024-10-01 15:33:16 +05:30
Kashif Faraz 7638d29c40
PostJoinCursor should never advance without interruption (#17099) (#17196)
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
2024-09-30 20:37:18 +05:30
Kashif Faraz 25f2447fb2
Add test for exceptions in FutureUtils.transformAsync. (#17106) (#17197)
Adds an additional test case to FutureUtilsTest.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-09-30 20:36:56 +05:30
Kashif Faraz f4ad5d001b
FrameChannelMerger: Fix incorrect behavior of finished(). (#17088) (#17194)
Previously, the processor used "remainingChannels" to track the number of
non-null entries of currentFrame. Now, "remainingChannels" tracks the
number of channels that are unfinished.

The difference is subtle. In the previous code, when an input channel
was blocked upon exiting nextFrame(), the "currentFrames" entry would be
null, and therefore the "remainingChannels" variable would be decremented.
After the next await and call to populateCurrentFramesAndTournamentTree(),
"remainingChannels" would be incremented if the channel had become
unblocked after awaiting.

This means that finished(), which returned true if remainingChannels was
zero, would not be reliable if called between nextFrame() and the
next await + populateCurrentFramesAndTournamentTree().

This patch changes things such that finished() is always reliable. This
fixes a regression introduced in PR #16911, which added a call to
finished() that was, at that time, unsafe.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-09-30 19:47:10 +05:30
Kashif Faraz 8c7c422d8d
[Backport] Speed up FrameFileTest, SuperSorterTest. (#17068) (#17190)
These are two heavily parameterized tests that, together, account for
about 60% of runtime in the test suite.

FrameFileTest changes:
1) Cache frame files in a static, rather than building the frame file
   for each parameterization of the test.
2) Adjust TestArrayCursorFactory to cache the signature, rather than
   re-creating it on each call to getColumnCapabilities.

SuperSorterTest changes:
1) Dramatically reduce the number of tests that run with
   "maxRowsPerFrame" = 1. These are particularly slow due to writing so
   many small files. Some still run, since it's useful to test edge cases,
   but much fewer than before.
2) Reduce the "maxActiveProcessors" axis of the test from [1, 2, 4] to
   [1, 3]. The aim is to reduce the number of cases while still getting
   good coverage of the feature.
3) Reduce the "maxChannelsPerProcessor" axis of the test from [2, 3, 8]
   to [2, 7]. The aim is to reduce the number of cases while still getting
   good coverage of the feature.
4) Use in-memory input channels rather than file channels.
5) Defer formatting of assertion failure messages until they are needed.
6) Cache the cursor factory and its signature in a static.
7) Cache sorted test rows (used for verification) in a static.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-09-30 16:49:55 +05:30
Cece Mei c31da4f7a7
Incorporate `estimatedComputeCost` into all `BitmapColumnIndex` classes. (#17125) (#17172)
changes:
* filter index processing is now automatically ordered based on estimated 'cost', which is approximated based on how many expected bitmap operations are required to construct the bitmap used for the 'offset'
* cursorAutoArrangeFilters context flag now defaults to true, but can be set to false to disable cost based filter index sorting
2024-09-30 02:24:05 -07:00
Clint Wylie 2e5d30993e
fix a mistake in CursorGranularizer to check doneness after advance (#17175) (#17176)
Fixes a mistake introduced in #16533 which can result in CursorGranularizer incorrectly trying to get values from a selector after calling cursor.advance because of a missing check for cursor.isDone
2024-09-30 02:21:11 -07:00
Clint Wylie 4123f2ca90
add multi-value string object vector matcher and expression vector object selectors (#17162) (#17165) 2024-09-30 02:20:30 -07:00
Clint Wylie 51bea56948
add VirtualColumns.findEquivalent and VirtualColumn.EquivalenceKey (#17084) (#17166) 2024-09-27 17:54:06 -07:00
Cece Mei a0c842e98b
Create a FilterBundle.Builder class and use it to construct FilterBundle. (#17055) (#17159) 2024-09-25 18:04:32 -07:00
Akshat Jain 627752922c
MSQ window functions: Reject MVDs during window processing (#17036) (#17127)
* MSQ window functions: Reject MVDs during window processing

* MSQ window functions: Reject MVDs during window processing

* Remove parameterization from MSQWindowTest
2024-09-25 15:00:02 +05:30
Clint Wylie 1096728fa4
use CastToObjectVectorProcessor for cast to string (#17148) (#17149) 2024-09-24 21:15:45 -07:00
Clint Wylie cf00b4cd24
various fixes and improvements to vectorization fallback (#17098) (#17142)
changes:
* add `ApplyFunction` support to vectorization fallback, allowing many of the remaining expressions to be vectorized
* add `CastToObjectVectorProcessor` so that vector engine can correctly cast any type
* add support for array and complex vector constants
* reduce number of cases which can block vectorization in expression planner to be unknown inputs (such as unknown multi-valuedness)
* fix array constructor expression, apply map expression to make actual evaluated type match the output type inference
* fix bug in array_contains where something like array_contains([null], 'hello') would return true if the array was a numeric array since the non-null string value would cast to a null numeric
* fix isNull/isNotNull to correctly handle any type of input argument
2024-09-24 16:40:49 -07:00
Sree Charan Manamala 0c58f88ded
Add serde for ColumnBasedRowsAndColumns to fix window queries without group by (#16658) (#17111)
Register a Ser-De for RowsAndColumns so that the window operator query running on leaf operators would be transferred properly on the wire. Would fix the empty response given by window queries without group by on the native engine.

(cherry picked from commit bb1c3c1749)
2024-09-20 11:34:35 +02:00
Laksh Singla 2f13cd2500
Support maxSubqueryBytes for window functions (#16800) (#17085)
Window queries now acknowledge maxSubqueryBytes.
2024-09-19 19:31:17 +05:30
Sree Charan Manamala 11727af2a6
Fix String Frame Readers to read String Arrays correctly (#16885) (#17103)
While writing to a frame, String arrays are written by setting the multivalue byte.
But while reading, it was hardcoded to false.

(cherry picked from commit c7c3307e61)
2024-09-19 09:02:12 +05:30
Clint Wylie c462e103b6
transition away from StorageAdapter (#16985) (#17024)
* transition away from StorageAdapter
changes:
* CursorHolderFactory has been renamed to CursorFactory and moved off of StorageAdapter, instead fetched directly from the segment via 'asCursorFactory'. The previous deprecated CursorFactory interface has been merged into StorageAdapter
* StorageAdapter is no longer used by any engines or tests and has been marked as deprecated with default implementations of all methods that throw exceptions indicating the new methods to call instead
* StorageAdapter methods not covered by CursorFactory (CursorHolderFactory prior to this change) have been moved into interfaces which are retrieved by Segment.as, the primary classes are the previously existing Metadata, as well as new interfaces PhysicalSegmentInspector and TopNOptimizationInspector
* added UnnestSegment and FilteredSegment that extend WrappedSegmentReference since their StorageAdapter implementations were previously provided by WrappedSegmentReference
* added PhysicalSegmentInspector which covers some of the previous StorageAdapter functionality which was primarily used for segment metadata queries and other metadata uses, and is implemented for QueryableIndexSegment and IncrementalIndexSegment
* added TopNOptimizationInspector to cover the oddly specific StorageAdapter.hasBuiltInFilters implementation, which is implemented for HashJoinSegment, UnnestSegment, and FilteredSegment
* Updated all engines and tests to no longer use StorageAdapter
2024-09-09 21:43:41 -07:00
Sree Charan Manamala 51fe3c08ab
Window Functions : Reject MVDs during window processing (#17002)
This commit aims to reject MVDs in window processing as we do not support them.
Earlier to this commit, query running a window aggregate partitioned by an MVD column would fail with ClassCastException
2024-09-09 12:07:54 +05:30
Clint Wylie b0f36c1b89
fix bug with CastOperatorConversion with types which cannot be mapped to native druid types (#17011) 2024-09-06 17:07:32 -07:00
Gian Merlino 175636b28f
Frame writers: Coerce numeric and array types in certain cases. (#16994)
This patch adds "TypeCastSelectors", which is used when writing frames to
perform two coercions:

- When a numeric type is desired and the underlying type is non-numeric or
  unknown, the underlying selector is wrapped, "getObject" is called and the
  result is coerced using "ExprEval.ofType". This differs from the prior
  behavior where the primitive methods like "getLong", "getDouble", etc, would
  be called directly. This fixes an issue where a column would be read as
  all-zeroes when its SQL type is numeric and its physical type is string, which
  can happen when evolving a column's type from string to number.

-  When an array type is desired, the underlying selector is wrapped,
   "getObject" is called, and the result is coerced to Object[]. This coercion
   replaces some earlier logic from #15917.
2024-09-05 17:20:00 -07:00
Kashif Faraz ba6f804f48
Fix compaction status API response (#17006)
Description:
#16768 introduces new compaction APIs on the Overlord `/compact/status` and `/compact/progress`.
But the corresponding `OverlordClient` methods do not return an object compatible with the actual
endpoints defined in `OverlordCompactionResource`.

This patch ensures that the objects are compatible.

Changes:
- Add `CompactionStatusResponse` and `CompactionProgressResponse`
- Use these as the return type in `OverlordClient` methods and as the response entity in `OverlordCompactionResource`
- Add `SupervisorCleanupModule` bound on the Coordinator to perform cleanup of supervisors.
Without this module, Coordinator cannot deserialize compaction supervisors.
2024-09-05 23:22:01 +05:30
Clint Wylie 57bf053dc9
remove compiler warnings about unqualified calls to yield() (#16995) 2024-09-03 20:04:30 -07:00
Gian Merlino 57c4b552d9
Fix logical merge conflict in SuperSorterTest. (#16993)
Logical merge conflict between #16911 and #16914.
2024-09-03 16:14:59 -04:00
Gian Merlino 786c959e9e
MSQ: Add limitHint to global-sort shuffles. (#16911)
* MSQ: Add limitHint to global-sort shuffles.

This allows pushing down limits into the SuperSorter.

* Test fixes.

* Add limitSpec to ScanQueryKit. Fix SuperSorter tracking.
2024-09-03 09:05:29 -07:00
Sree Charan Manamala 619d8ef964
Window Functions : Numeric Arrays Frame Column Writers - fix class cast exception (#16983)
Fix ClassCastException in ArrayFrameCoulmnWriters
2024-09-03 11:44:52 +05:30
Zoltan Haindrich 32e8e074ae
Planning could have failed if UNION ALL operator was completely removed (#16946) 2024-09-02 04:37:10 -04:00
Kashif Faraz fe3d589ff9
Run compaction as a supervisor on Overlord (#16768)
Description
-----------
Auto-compaction currently poses several challenges as it:
1. may get stuck on a failing interval.
2. may get stuck on the latest interval if more data keeps coming into it.
3. always picks the latest interval regardless of the level of compaction in it.
4. may never pick a datasource if its intervals are not very recent.
5. requires setting an explicit period which does not cater to the changing needs of a Druid cluster.

This PR introduces various improvements to compaction scheduling to tackle the above problems.

Change Summary
--------------
1. Run compaction for a datasource as a supervisor of type `autocompact` on Overlord.
2. Make compaction policy extensible and configurable.
3. Track status of recently submitted compaction tasks and pass this info to policy.
4. Add `/simulate` API on both Coordinator and Overlord to run compaction simulations.
5. Redirect compaction status APIs to the Overlord when compaction supervisors are enabled.
2024-09-02 07:53:13 +05:30
Parag Jain 6eb42e8d5a
fix extraction of timeseries results from result level cache (#16895)
* fix extraction of timeseries results from result level cache

* remove unneded import

* add test
2024-09-01 00:25:55 +05:30
Virushade 0217c8c541
Change Inspection Profile to set "Method is identical to its super method" as error (#16976)
* Make IntelliJ's MethodIsIdenticalToSuperMethod an error

* Change codebase to follow new IntelliJ inspection

* Restore non-short-circuit boolean expressions to pass tests
2024-08-31 09:37:34 +05:30
Gian Merlino caf8ce3e0b
MSQ: Add CPU and thread usage counters. (#16914)
* MSQ: Add CPU and thread usage counters.

The main change adds "cpu" and "wall" counters. The "cpu" counter measures
CPU time (using JvmUtils.getCurrentThreadCpuTime) taken up by processors
in processing threads. The "wall" counter measures the amount of wall time
taken up by processors in those same processing threads. Both counters are
broken down by type of processor.

This patch also includes changes to support adding new counters. Due to an
oversight in the original design, older deserializers are not forwards-compatible;
they throw errors when encountering an unknown counter type. To manage this,
the following changes are made:

1) The defaultImpl NilQueryCounterSnapshot is added to QueryCounterSnapshot's
   deserialization configuration. This means that any unrecognized counter types
   will be read as "nil" by deserializers. Going forward, once all servers are
   on the latest code, this is enough to enable easily adding new counters.

2) A new context parameter "includeAllCounters" is added, which defaults to "false".
   When this parameter is set "false", only legacy counters are included. When set
   to "true", all counters are included. This is currently undocumented. In a future
   version, we should set the default to "true", and at that time, include a release
   note that people updating from versions prior to Druid 31 should set this to
   "false" until their upgrade is complete.

* Style, coverage.

* Fix.
2024-08-30 20:02:30 -07:00
Kashif Faraz d5b64ba2e3
Improve exception handling in extension druid-pac4j (#16979)
Changes:
- Simplify exception handling in `CryptoService` by just catching a `Exception`
- Throw a `DruidException` as the exception is user facing
- Log the exception for easier debugging
- Add a test to verify thrown exception
2024-08-30 12:32:49 +05:30
Akshat Jain fbd305af0f
MSQ WF: Batch multiple PARTITION BY keys for processing (#16823)
Currently, if we have a query with window function having PARTITION BY xyz, and we have a million unique values for xyz each having 1 row, we'd end up creating a million individual RACs for processing, each having a single row. This is unnecessary, and we can batch the PARTITION BY keys together for processing, and process them only when we can't batch further rows to adhere to maxRowsMaterialized config.

The previous iteration of this PR was simplifying WindowOperatorQueryFrameProcessor to run all operators on all the rows instead of creating smaller RACs per partition by key. That approach was discarded in favor of the batching approach, and the details are summarized here: #16823 (comment).
2024-08-28 11:32:47 +05:30
Pranav 0caf383102
Fix buffer capacity race condition in spatial (#16931) 2024-08-27 00:36:29 -07:00
Clint Wylie f8301a314f
generic block compressed complex columns (#16863)
changes:
* Adds new `CompressedComplexColumn`, `CompressedComplexColumnSerializer`, `CompressedComplexColumnSupplier` based on `CompressedVariableSizedBlobColumn` used by JSON columns
* Adds `IndexSpec.complexMetricCompression` which can be used to specify compression for the generic compressed complex column. Defaults to uncompressed because compressed columns are not backwards compatible.
* Adds new definition of `ComplexMetricSerde.getSerializer` which accepts an `IndexSpec` argument when creating a serializer. The old signature has been marked `@Deprecated` and has a default implementation that returns `null`, but it will be used by the default implementation of the new version if it is implemented to return a non-null value. The default implementation of the new method will use a `CompressedComplexColumnSerializer` if `IndexSpec.complexMetricCompression` is not null/none/uncompressed, or will use `LargeColumnSupportedComplexColumnSerializer` otherwise.
* Removed all duplicate generic implementations of `ComplexMetricSerde.getSerializer` and `ComplexMetricSerde.deserializeColumn` into default implementations `ComplexMetricSerde` instead of being copied all over the place. The default implementation of `deserializeColumn` will check if the first byte indicates that the new compression was used, otherwise will use the `GenericIndexed` based supplier.
* Complex columns with custom serializers/deserializers are unaffected and may continue doing whatever it is they do, either with specialized compression or whatever else, this new stuff is just to provide generic implementations built around `ObjectStrategy`.
* add ObjectStrategy.readRetainsBufferReference so CompressedComplexColumn only copies on read if required
* add copyValueOnRead flag down to CompressedBlockReader to avoid buffer duplicate if the value needs copied anyway
2024-08-27 00:34:41 -07:00
Gian Merlino ed3dbd6242
MSQ: Fix validation of time position in collations. (#16961)
* MSQ: Fix validation of time position in collations.

It is possible for the collation to refer to a field that isn't mapped,
such as when the DML includes "CLUSTERED BY some_function(some_field)".
In this case, the collation refers to a projected column that is not
part of the field mappings. Prior to this patch, that would lead to an
out of bounds list access on fieldMappings.

This patch fixes the problem by identifying the position of __time in
the fieldMappings first, rather than retrieving each collation field
from fieldMappings.

Fixes a bug introduced in #16849.

* Fix test. Better warning message.
2024-08-27 00:02:32 -07:00
Gian Merlino 5d2ed33b89
Place __time in signatures according to sort order. (#16958)
* Place __time in signatures according to sort order.

Updates a variety of places to put __time in row signatures according
to its position in the sort order, rather than always first, including:

- InputSourceSampler.
- ScanQueryEngine (in the default signature when "columns" is empty).
- Various StorageAdapters, which also have the effect of reordering
  the column order in segmentMetadata queries, and therefore in SQL
  schemas as well.

Follow-up to #16849.

* Fix compilation.

* Additional fixes.

* Fix.

* Fix style.

* Omit nonexistent columns from the row signature.

* Fix tests.
2024-08-26 21:45:51 -07:00
Gian Merlino 0603d5153d
Segments sorted by non-time columns. (#16849)
* Segments primarily sorted by non-time columns.

Currently, segments are always sorted by __time, followed by the sort
order provided by the user via dimensionsSpec or CLUSTERED BY. Sorting
by __time enables efficient execution of queries involving time-ordering
or granularity. Time-ordering is a simple matter of reading the rows in
stored order, and granular cursors can be generated in streaming fashion.

However, for various workloads, it's better for storage footprint and
query performance to sort by arbitrary orders that do not start with __time.
With this patch, users can sort segments by such orders.

For spec-based ingestion, users add "useExplicitSegmentSortOrder: true" to
dimensionsSpec. The "dimensions" list determines the sort order. To
define a sort order that includes "__time", users explicitly
include a dimension named "__time".

For SQL-based ingestion, users set the context parameter
"useExplicitSegmentSortOrder: true". The CLUSTERED BY clause is then
used as the explicit segment sort order.

In both cases, when the new "useExplicitSegmentSortOrder" parameter is
false (the default), __time is implicitly prepended to the sort order,
as it always was prior to this patch.

The new parameter is experimental for two main reasons. First, such
segments can cause errors when loaded by older servers, due to violating
their expectations that timestamps are always monotonically increasing.
Second, even on newer servers, not all queries can run on non-time-sorted
segments. Scan queries involving time-ordering and any query involving
granularity will not run. (To partially mitigate this, a currently-undocumented
SQL feature "sqlUseGranularity" is provided. When set to false the SQL planner
avoids using "granularity".)

Changes on the write path:

1) DimensionsSpec can now optionally contain a __time dimension, which
   controls the placement of __time in the sort order. If not present,
   __time is considered to be first in the sort order, as it has always
   been.

2) IncrementalIndex and IndexMerger are updated to sort facts more
   flexibly; not always by time first.

3) Metadata (stored in metadata.drd) gains a "sortOrder" field.

4) MSQ can generate range-based shard specs even when not all columns are
   singly-valued strings. It merely stops accepting new clustering key
   fields when it encounters the first one that isn't a singly-valued
   string. This is useful because it enables range shard specs on
   "someDim" to be created for clauses like "CLUSTERED BY someDim, __time".

Changes on the read path:

1) Add StorageAdapter#getSortOrder so query engines can tell how a
   segment is sorted.

2) Update QueryableIndexStorageAdapter, IncrementalIndexStorageAdapter,
   and VectorCursorGranularizer to throw errors when using granularities
   on non-time-ordered segments.

3) Update ScanQueryEngine to throw an error when using the time-ordering
  "order" parameter on non-time-ordered segments.

4) Update TimeBoundaryQueryRunnerFactory to perform a segment scan when
   running on a non-time-ordered segment.

5) Add "sqlUseGranularity" context parameter that causes the SQL planner
   to avoid using granularities other than ALL.

Other changes:

1) Rename DimensionsSpec "hasCustomDimensions" to "hasFixedDimensions"
   and change the meaning subtly: it now returns true if the DimensionsSpec
   represents an unchanging list of dimensions, or false if there is
   some discovery happening. This is what call sites had expected anyway.

* Fixups from CI.

* Fixes.

* Fix missing arg.

* Additional changes.

* Fix logic.

* Fixes.

* Fix test.

* Adjust test.

* Remove throws.

* Fix styles.

* Fix javadocs.

* Cleanup.

* Smoother handling of null ordering.

* Fix tests.

* Missed a spot on the merge.

* Fixups.

* Avoid needless Filters.and.

* Add timeBoundaryInspector to test.

* Fix tests.

* Fix FrameStorageAdapterTest.

* Fix various tests.

* Use forceSegmentSortByTime instead of useExplicitSegmentSortOrder.

* Pom fix.

* Fix doc.
2024-08-23 08:24:43 -07:00
Adarsh Sanjeev e2516d9a67
WriteOutBytes improvements
This PR generally improves the working of WriteOutBytes and WriteOutMedium. Some analysis of usage of TmpFileSegmentWriteOutMedium shows that they periodically get used for very small things. The overhead of creating a tmp file is actually very large. To improve the performance in these cases, this PR modifies TmpFileSegmentWriteOutMedium to return a heap-based WriteOutBytes that falls back to making a tmp file when it actually fills up.
---------
Co-authored-by: imply-cheddar <eric.tschetter@imply.io>
2024-08-23 11:32:30 +05:30