Fixes an issue where sql query request logs do not include the default query context
values set via `druid.query.default.context.xyz` runtime properties.
# Change summary
* Inject `DefaultQueryConfig` into `SqlLifecycleFactory`
* Add params from `DefaultQueryConfig` to the query context in `SqlLifecycle`
# Description
- This change does not affect query execution. This is because the
`DefaultQueryConfig` was already being used in `QueryLifecycle`,
which is initialized when the SQL is translated to a native query.
- This also handles any potential use case where a context parameter should be
handled at the SQL stage itself.
* SqlSegmentsMetadataQuery: Fix OVERLAPS for wide target segments.
Segments with endpoints prior to year 0 or after year 9999 may overlap
the search intervals but not match the generated SQL conditions. So, we
need to add an additional OR condition to catch these.
I checked a real, live MySQL metadata store to confirm that the query
still uses metadata store indexes. It does.
* Add comments.
Often users are submitting queries, and ingestion specs that work only if the relevant extension is not loaded. However, the error is too technical for the users and doesn't suggest them to check for missing extensions. This PR modifies the error message so users can at least check their settings before assuming that the error is because of a bug.
* Service stdout log files, move logs to log/.
Two changes that make log behavior cleaner:
1) Redirect messages from the Java runtime to their own log files.
Otherwise, they would get jumbled up in the output of the all-in-one
start command.
2) Use log/ instead of bin/log/ for the default log directory. Makes them
easier to find.
Additionally, add documentation about how to avoid the reflective
access warnings in Java 11.
* Spelling.
* See if code formatting affects spelling.
* Small addition to Multitenancy considerations doc
* Update docs/querying/multitenancy.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update multitenancy.md
Edit suggested by @kfaraz
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
fixes an issue caused by a test modification in #12408 that was closing buffers allocated by the compression strategy instead of allowing the closer to do it
RowBasedColumnSelectorFactory inherited strange behavior from
Rows.objectToStrings for nulls that appear in lists: instead of being
left as a null, it is replaced with the string "null". Some callers may
need compatibility with this strange behavior, but it should be opt-in.
Query-time call sites are changed to opt-out of this behavior, since it
is not consistent with query-time expectations. The IncrementalIndex
ingestion-time call site retains the old behavior, as this is traditionally
when Rows.objectToStrings would be used.
* Add RowIdSupplier to ColumnSelectorFactory.
This enables virtual columns to cache their outputs in case they are
called multiple times on the same underlying row. This is common for
numeric selectors, where the common pattern is to call isNull() and
then follow with getLong(), getFloat(), or getDouble(). Here, output
caching reduces the number of expression evals by half.
* Fix tests.
* Adding zstandard compression library
* 1. Took @clintropolis's advice to have ZStandard decompressor use the byte array when the buffers are not direct.
2. Cleaned up checkstyle issues.
* Fixing zstandard version to latest stable version in pom's and updating license files
* Removing zstd from benchmarks and adding to processing (poms)
* fix the intellij inspection issue
* Removing the prefix v for the version in the license check for ztsd
* Fixing license checks
Co-authored-by: Rahul Gidwani <r_gidwani@apple.com>
Adds a default implementation of getQueryContext, which was added to the Query interface in #12396. Query is marked with @ExtensionPoint, and lately we have been trying to be less volatile on these interfaces by providing default implementations to be more chill for extension writers.
The way this default implementation is done in this PR is a bit strange due to the way that getQueryContext is used (mutated with system default and system generated keys); the default implementation has a specific object that it returns, and I added another temporary default method isLegacyContext that checks if the getQueryContext returns that object or not. If not, callers fall back to using getContext and withOverriddenContext to set these default and system values.
I am open to other ideas as well, but this way should work at least without exploding, and added some tests to ensure that it is wired up correctly for QueryLifecycle, including the context authorization stuff.
The added test shows the strange behavior if query context authorization is enabled, mainly that the system default and system generated query context keys also need to be granted as permissions for things to function correctly. This is not great, so I mentioned it in the javadocs as well. Not sure if it needs to be called out anywhere else.
Description
Fixes a bug when running q's like
SELECT cntarray,
Count(*)
FROM (SELECT dim1,
dim2,
Array_agg(cnt) AS cntarray
FROM (SELECT dim1,
dim2,
dim3,
Count(*) AS cnt
FROM foo
GROUP BY 1,
2,
3)
GROUP BY 1,
2)
GROUP BY 1
This generates an error:
org.apache.druid.java.util.common.ISE: Unable to convert type [Ljava.lang.Object; to org.apache.druid.segment.data.ComparableList
at org.apache.druid.segment.DimensionHandlerUtils.convertToList(DimensionHandlerUtils.java:405) ~[druid-xx]
Because it's an array of numbers it looks like it does the convertToList call, which looks like:
@Nullable
public static ComparableList convertToList(Object obj)
{
if (obj == null) {
return null;
}
if (obj instanceof List) {
return new ComparableList((List) obj);
}
if (obj instanceof ComparableList) {
return (ComparableList) obj;
}
throw new ISE("Unable to convert type %s to %s", obj.getClass().getName(), ComparableList.class.getName());
}
I.e. it doesn't know about arrays. Added the array handling as part of this PR.
* Emit state of replace and append for native batch tasks
* Emit count of one depending on batch ingestion mode (APPEND, OVERWRITE, REPLACE)
* Add metric to compaction job
* Avoid null ptr exc when null emitter
* Coverage
* Emit tombstone & segment counts
* Tasks need a type
* Spelling
* Integrate BatchIngestionMode in batch ingestion tasks functionality
* Typos
* Remove batch ingestion type from metric since it is already in a dimension. Move IngestionMode to AbstractTask to facilitate having mode as a dimension. Add metrics to streaming. Add missing coverage.
* Avoid inner class referenced by sub-class inspection. Refactor computation of IngestionMode to make it more robust to null IOConfig and fix test.
* Spelling
* Avoid polluting the Task interface
* Rename computeCompaction methods to avoid ambiguous java compiler error if they are passed null. Other minor cleanup.
In the case that the clustered by is before the partitioned by for an sql query, the error message is a bit confusing.
insert into foo select * from bar clustered by dim1 partitioned by all
Error: SQL parse failed
Encountered "PARTITIONED" at line 1, column 88.
Was expecting one of: <EOF> "," ... "ASC" ... "DESC" ... "NULLS" ... "." ... "NOT" ... "IN" ... "<" ... "<=" ... ">" ... ">=" ... "=" ... "<>" ... "!=" ... "BETWEEN" ... "LIKE" ... "SIMILAR" ... "+" ... "-" ... "*" ... "/" ... "%" ... "||" ... "AND" ... "OR" ... "IS" ... "MEMBER" ... "SUBMULTISET" ... "CONTAINS" ... "OVERLAPS" ... "EQUALS" ... "PRECEDES" ... "SUCCEEDS" ... "IMMEDIATELY" ... "MULTISET" ... "[" ... "FORMAT" ... "(" ... Less...
org.apache.calcite.sql.parser.SqlParseException
This is a bit confusing and adding a check could be added to throw a more user friendly message stating that the order should be reversed.
Add error message for incorrectly ordered clause in sql.
* ConcurrentGrouper: Add option to always slice up merge buffers thread-locally.
Normally, the ConcurrentGrouper shares merge buffers across processing
threads until spilling starts, and then switches to a thread-local model.
This minimizes memory use and reduces likelihood of spilling, which is
good, but it creates thread contention. The new mergeThreadLocal option
causes a query to start in thread-local mode immediately, and allows us
to experiment with the relative performance of the two modes.
* Fix grammar in docs.
* Fix race in ConcurrentGrouper.
* Fix issue with timeouts.
* Remove unused import.
* Add "tradeoff" to dictionary.
* Deal with potential cardinality estimate being negative and add logging
* Fix typo in name
* Refine and minimize logging
* Make it info based on code review
* Create a named constant for the magic number
Issue:
Even though `CompactionTuningConfig` allows a `maxColumnsToMerge` config
(to optimize memory usage, particulary for datasources with many dimensions),
the corresponding client object `ClientCompactionTaskQueryTuningConfig`
(used by the coordinator duty `CompactSegments` to trigger auto-compaction)
does not contain this field. Thus, the value of `maxColumnsToMerge` specified
in any datasource compaction config is ignored.
Changes:
- Add field `maxColumnsToMerge` in `ClientCompactionTaskQueryTuningConfig`
and `UserCompactionTaskQueryTuningConfig`
- Fix tests
* 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.
* RemoteTaskRunner: Fix NPE in streamTaskReports.
It is possible for a work item to drop out of runningTasks after the
ZkWorker is retrieved. In this case, the current code would throw
an NPE.
* Additional tests and additional fixes.
* Fix import.
* SQL: Add is_active to sys.segments, update examples and docs.
is_active is short for:
(is_published = 1 AND is_overshadowed = 0) OR is_realtime = 1
It's important because this represents "all the segments that should
be queryable, whether or not they actually are right now". Most of the
time, this is the set of segments that people will want to look at.
The web console already adds this filter to a lot of its queries,
proving its usefulness.
This patch also reworks the caveat at the bottom of the sys.segments
section, so its information is mixed into the description of each result
field. This should make it more likely for people to see the information.
* Wording updates.
* Adjustments for spellcheck.
* Adjust IT.
1) Align "Assigning task" log messages between RTR and HRTR.
2) Remove confusing reference to "Coordinator".
3) Move "Not assigning task" message from INFO to DEBUG. It's not super
important to see this message: we mainly want to see what _does_ get
assigned.
4) Reword "Task switched from pending to running" message to better
match the structure of the "Assigning task" message from the same
method.
* Add builder for TaskToolbox.
The main purpose of this change is to make it easier to create
TaskToolboxes in tests. However, the builder is used in production
too, by TaskToolboxFactory.
* Fix imports, adjust formatting.
* Fix import.
* 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.