Limit the subquery results by memory usage (#13952)

Users can now add a guardrail to prevent subquery’s results from exceeding the set number of bytes by setting druid.server.http.maxSubqueryRows in Broker's config or maxSubqueryRows in the query context. This feature is experimental for now and would default back to row-based limiting in case it fails to get the accurate size of the results consumed by the query.
This commit is contained in:
Laksh Singla 2023-06-26 18:12:28 +05:30 committed by GitHub
parent d7c9c2f367
commit 1647d5f4a0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
52 changed files with 4209 additions and 946 deletions

View File

@ -1840,6 +1840,19 @@ This strategy can be enabled by setting `druid.query.scheduler.laning.strategy=h
|--------|-----------|-------|
|`druid.query.scheduler.laning.maxLowPercent`|Maximum percent of the smaller number of `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, defining the number of HTTP threads that can be used by queries with a priority lower than 0. Value must be an integer in the range 1 to 100, and will be rounded up|No default, must be set if using this mode|
##### Guardrails for materialization of subqueries
Druid stores the subquery rows in temporary tables that live in the Java heap. It is a good practice to avoid large subqueries in Druid.
Therefore there are guardrails that are built in Druid to prevent the queries from generating subquery results which can exhaust the heap
space. They can be set on a cluster level or modified per query level as desired.
Note the following guardrails that can be set by the cluster admin to limit the subquery results:
1. `druid.server.http.maxSubqueryRows` in broker's config to set a default for the entire cluster or `maxSubqueryRows` in the query context to set an upper limit on the number of rows a subquery can generate
2. `druid.server.http.maxSubqueryBytes` in broker's config to set a default for the entire cluster or `maxSubqueryBytes` in the query context to set an upper limit on the number of bytes a subquery can generate
Note that limiting the subquery by bytes is a newer feature therefore it is experimental as it materializes the results differently.
If you choose to modify or set any of the above limits, you must also think about the heap size of all Brokers, Historicals, and task Peons that process data for the subqueries to accommodate the subquery results.
There is no formula to calculate the correct value. Trial and error is the best approach.
###### 'Manual' laning strategy
This laning strategy is best suited for cases where one or more external applications which query Druid are capable of manually deciding what lane a given query should belong to. Configured with a map of lane names to percent or exact max capacities, queries with a matching `lane` parameter in the [query context](../querying/query-context.md) will be subjected to those limits.
@ -1862,6 +1875,7 @@ Druid uses Jetty to serve HTTP requests. Each query being processed consumes a s
|`druid.server.http.defaultQueryTimeout`|Query timeout in millis, beyond which unfinished queries will be cancelled|300000|
|`druid.server.http.maxScatterGatherBytes`|Maximum number of bytes gathered from data processes such as Historicals and realtime processes to execute a query. Queries that exceed this limit will fail. This is an advance configuration that allows to protect in case Broker is under heavy load and not utilizing the data gathered in memory fast enough and leading to OOMs. This limit can be further reduced at query time using `maxScatterGatherBytes` in the context. Note that having large limit is not necessarily bad if broker is never under heavy concurrent load in which case data gathered is processed quickly and freeing up the memory used. Human-readable format is supported, see [here](human-readable-byte.md). |Long.MAX_VALUE|
|`druid.server.http.maxSubqueryRows`|Maximum number of rows from all subqueries per query. Druid stores the subquery rows in temporary tables that live in the Java heap. `druid.server.http.maxSubqueryRows` is a guardrail to prevent the system from exhausting available heap. When a subquery exceeds the row limit, Druid throws a resource limit exceeded exception: "Subquery generated results beyond maximum."<br /><br />It is a good practice to avoid large subqueries in Druid. However, if you choose to raise the subquery row limit, you must also increase the heap size of all Brokers, Historicals, and task Peons that process data for the subqueries to accommodate the subquery results.<br /><br />There is no formula to calculate the correct value. Trial and error is the best approach.|100000|
|`druid.server.http.maxSubqueryBytes`|Maximum number of bytes from all subqueries per query. Since the results are stored on the Java heap, `druid.server.http.maxSubqueryBytes` is a guardrail like `druid.server.http.maxSubqueryRows` to prevent the heap space from exhausting. When a subquery exceeds the byte limit, Druid throws a resource limit exceeded exception. A negative value for the guardrail indicates that Druid won't guardrail by memory. Check the docs for `druid.server.http.maxSubqueryRows` to see how to set the optimal value for a cluster. This is an experimental feature for now as this materializes the results in a different format.|-1|
|`druid.server.http.gracefulShutdownTimeout`|The maximum amount of time Jetty waits after receiving shutdown signal. After this timeout the threads will be forcefully shutdown. This allows any queries that are executing to complete(Only values greater than zero are valid).|`PT30S`|
|`druid.server.http.unannouncePropagationDelay`|How long to wait for ZooKeeper unannouncements to propagate before shutting down Jetty. This is a minimum and `druid.server.http.gracefulShutdownTimeout` does not start counting down until after this period elapses.|`PT0S` (do not wait)|
|`druid.server.http.maxQueryTimeout`|Maximum allowed value (in milliseconds) for `timeout` parameter. See [query-context](../querying/query-context.md) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE|

View File

@ -85,10 +85,15 @@ their "base" (bottom-leftmost) datasource, as described in the [join](#join) sec
the results are brought back to the Broker. Then, the Broker continues on with the rest of the query as if the subquery
was replaced with an inline datasource.
In most cases, Druid buffers subquery results in memory on the Broker before the rest of the query proceeds. Therefore, subqueries execute sequentially. The total number of rows buffered across all subqueries of a given query cannot exceed the [`druid.server.http.maxSubqueryRows`](../configuration/index.md) which defaults to 100000 rows. Otherwise Druid throws a resource limit exceeded exception: "Subquery generated results beyond maximum."
In most cases, Druid buffers subquery results in memory on the Broker before the rest of the query proceeds.
Therefore, subqueries execute sequentially. The total number of rows buffered across all subqueries of a given query
cannot exceed the [`druid.server.http.maxSubqueryRows`](../configuration/index.md) which defaults to 100000 rows, or the
[`druid.server.http.maxSubqueryBytes`](../configuration/index.md) if set. Otherwise, Druid throws a resource limit exceeded
exception.
There is one exception: if the outer query and all subqueries are the [groupBy](groupbyquery.md) type, then subquery
results can be processed in a streaming fashion and the `druid.server.http.maxSubqueryRows` limit does not apply.
results can be processed in a streaming fashion and the `druid.server.http.maxSubqueryRows` and `druid.server.http.maxSubqueryBytes`
limits do not apply.
### `join`

View File

@ -30,6 +30,7 @@ import org.apache.druid.msq.input.ReadableInputs;
import org.apache.druid.msq.input.table.SegmentWithDescriptor;
import org.apache.druid.query.InlineDataSource;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.segment.InlineSegmentWrangler;
import org.apache.druid.segment.SegmentWrangler;
import org.apache.druid.timeline.SegmentId;
@ -37,7 +38,7 @@ import java.util.function.Consumer;
/**
* Reads {@link InlineInputSlice} using {@link SegmentWrangler} (which is expected to contain an
* {@link org.apache.druid.segment.InlineSegmentWrangler}).
* {@link InlineSegmentWrangler}).
*/
public class InlineInputSliceReader implements InputSliceReader
{

View File

@ -149,13 +149,6 @@ public class CalciteSelectQueryMSQTest extends CalciteQueryTest
}
@Ignore
@Override
public void testMaxSubqueryRows()
{
}
@Ignore
@Override
public void testQueryWithMoreThanMaxNumericInFilter()

View File

@ -71,6 +71,10 @@ public class FrameReader
/**
* Create a reader for frames with a given {@link RowSignature}. The signature must exactly match the frames to be
* read, or else behavior is undefined.
* If the columnType is null, we store the data as {@link ColumnType#NESTED_DATA}. This can be done if we know that
* the data that we receive can be serded generically using the nested data. It is currently used in the brokers to
* store the data with unknown types into frames.
* @param signature signature used to generate the reader
*/
public static FrameReader create(final RowSignature signature)
{

View File

@ -19,10 +19,17 @@
package org.apache.druid.frame.segment;
import org.apache.druid.error.DruidException;
import org.apache.druid.frame.Frame;
import org.apache.druid.frame.write.FrameWriter;
import org.apache.druid.frame.write.FrameWriterFactory;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.filter.BoundDimFilter;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.filter.BoundFilter;
import org.apache.druid.segment.filter.Filters;
@ -30,6 +37,8 @@ import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.Iterator;
import java.util.NoSuchElementException;
public class FrameCursorUtils
{
@ -67,4 +76,63 @@ public class FrameCursorUtils
);
}
}
/**
* Writes a {@link Cursor} to a sequence of {@link Frame}. This method iterates over the rows of the cursor,
* and writes the columns to the frames
*
* @param cursor Cursor to write to the frame
* @param frameWriterFactory Frame writer factory to write to the frame.
* Determines the signature of the rows that are written to the frames
*/
public static Sequence<Frame> cursorToFrames(
Cursor cursor,
FrameWriterFactory frameWriterFactory
)
{
return Sequences.simple(
() -> new Iterator<Frame>()
{
@Override
public boolean hasNext()
{
return !cursor.isDone();
}
@Override
public Frame next()
{
// Makes sure that cursor contains some elements prior. This ensures if no row is written, then the row size
// is larger than the MemoryAllocators returned by the provided factory
if (!hasNext()) {
throw new NoSuchElementException();
}
boolean firstRowWritten = false;
Frame frame;
try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(cursor.getColumnSelectorFactory())) {
while (!cursor.isDone()) {
if (!frameWriter.addSelection()) {
break;
}
firstRowWritten = true;
cursor.advance();
}
if (!firstRowWritten) {
throw DruidException
.forPersona(DruidException.Persona.DEVELOPER)
.ofCategory(DruidException.Category.CAPACITY_EXCEEDED)
.build("Subquery's row size exceeds the frame size and therefore cannot write the subquery's "
+ "row to the frame. This is a non-configurable static limit that can only be modified by the "
+ "developer.");
}
frame = Frame.wrap(frameWriter.toByteArray());
}
return frame;
}
}
);
}
}

View File

@ -52,7 +52,7 @@ public class FrameQueryableIndex implements QueryableIndex
private final RowSignature signature;
private final List<FrameColumnReader> columnReaders;
FrameQueryableIndex(
public FrameQueryableIndex(
final Frame frame,
final RowSignature signature,
final List<FrameColumnReader> columnReaders

View File

@ -291,4 +291,15 @@ public class FrameWriterUtils
return true;
}
public static RowSignature replaceUnknownTypesWithNestedColumns(final RowSignature rowSignature)
{
RowSignature.Builder retBuilder = RowSignature.builder();
for (int i = 0; i < rowSignature.size(); ++i) {
String columnName = rowSignature.getColumnName(i);
ColumnType columnType = rowSignature.getColumnType(i).orElse(ColumnType.NESTED_DATA);
retBuilder.add(columnName, columnType);
}
return retBuilder.build();
}
}

View File

@ -32,6 +32,8 @@ import org.apache.druid.java.util.common.guava.Accumulator;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.query.FrameBasedInlineDataSourceSerializer;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.context.ResponseContextDeserializer;
import org.joda.time.DateTimeZone;
@ -52,6 +54,8 @@ public class DruidDefaultSerializersModule extends SimpleModule
JodaStuff.register(this);
addSerializer(FrameBasedInlineDataSource.class, new FrameBasedInlineDataSourceSerializer());
addDeserializer(
DateTimeZone.class,
new JsonDeserializer<DateTimeZone>()

View File

@ -0,0 +1,197 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query;
import org.apache.druid.frame.Frame;
import org.apache.druid.frame.read.FrameReader;
import org.apache.druid.frame.segment.FrameStorageAdapter;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.SegmentReference;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.RowSignature;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.stream.Collectors;
/**
* Represents an inline datasource where the rows are embedded within the DataSource object itself.
* <p>
* The rows are backed by a sequence of {@link FrameSignaturePair}, which contain the Frame representation of the rows
* represented by the datasource.
* <p>
* Note that the signature of the datasource can be different from the signatures of the constituent frames that it
* consists of. While fetching the iterables, it is the job of this class to make sure that the rows correspond to the
* {@link #rowSignature}. For frames that donot contain the columns present in the {@link #rowSignature}, they are
* populated with {@code null}.
*/
public class FrameBasedInlineDataSource implements DataSource
{
final List<FrameSignaturePair> frames;
final RowSignature rowSignature;
public FrameBasedInlineDataSource(
List<FrameSignaturePair> frames,
RowSignature rowSignature
)
{
this.frames = frames;
this.rowSignature = rowSignature;
}
public List<FrameSignaturePair> getFrames()
{
return frames;
}
public RowSignature getRowSignature()
{
return rowSignature;
}
public Sequence<Object[]> getRowsAsSequence()
{
final Sequence<Cursor> cursorSequence =
Sequences.simple(frames)
.flatMap(
frameSignaturePair -> {
Frame frame = frameSignaturePair.getFrame();
RowSignature frameSignature = frameSignaturePair.getRowSignature();
FrameReader frameReader = FrameReader.create(frameSignature);
return new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY)
.makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null);
}
);
return cursorSequence.flatMap(
(cursor) -> {
final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
final List<BaseObjectColumnValueSelector> selectors = rowSignature
.getColumnNames()
.stream()
.map(columnSelectorFactory::makeColumnValueSelector)
.collect(Collectors.toList());
return Sequences.simple(
() -> new Iterator<Object[]>()
{
@Override
public boolean hasNext()
{
return !cursor.isDone();
}
@Override
public Object[] next()
{
Object[] row = new Object[rowSignature.size()];
for (int i = 0; i < rowSignature.size(); ++i) {
row[i] = selectors.get(i).getObject();
}
cursor.advance();
return row;
}
}
);
}
);
}
@Override
public Set<String> getTableNames()
{
return Collections.emptySet();
}
@Override
public List<DataSource> getChildren()
{
return Collections.emptyList();
}
@Override
public DataSource withChildren(List<DataSource> children)
{
if (!children.isEmpty()) {
throw new IAE("Cannot accept children");
}
return this;
}
@Override
public boolean isCacheable(boolean isBroker)
{
return false;
}
@Override
public boolean isGlobal()
{
return true;
}
@Override
public boolean isConcrete()
{
return true;
}
@Override
public Function<SegmentReference, SegmentReference> createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc)
{
return Function.identity();
}
@Override
public DataSource withUpdatedDataSource(DataSource newSource)
{
return newSource;
}
@Override
public byte[] getCacheKey()
{
return null;
}
@Override
public DataSourceAnalysis getAnalysis()
{
return new DataSourceAnalysis(this, null, null, Collections.emptyList());
}
}

View File

@ -0,0 +1,103 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.jsontype.TypeSerializer;
import com.fasterxml.jackson.databind.ser.std.StdSerializer;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
/**
* Serializes {@link FrameBasedInlineDataSource} to the representation of {@link InlineDataSource}
* so that the servers' on wire transfer data doesn't change. {@link FrameBasedInlineDataSource} is currently limited
* to the brokers only and therefore this aids in conversion of the object to a representation that the data servers
* can recognize
*/
public class FrameBasedInlineDataSourceSerializer extends StdSerializer<FrameBasedInlineDataSource>
{
public FrameBasedInlineDataSourceSerializer()
{
super(FrameBasedInlineDataSource.class);
}
@Override
public void serialize(FrameBasedInlineDataSource value, JsonGenerator jg, SerializerProvider serializers)
throws IOException
{
jg.writeStartObject();
jg.writeStringField("type", "inline");
RowSignature rowSignature = value.getRowSignature();
jg.writeObjectField("columnNames", rowSignature.getColumnNames());
List<ColumnType> columnTypes = IntStream.range(0, rowSignature.size())
.mapToObj(i -> rowSignature.getColumnType(i).orElse(null))
.collect(Collectors.toList());
jg.writeObjectField("columnTypes", columnTypes);
jg.writeArrayFieldStart("rows");
value.getRowsAsSequence().forEach(row -> {
try {
JacksonUtils.writeObjectUsingSerializerProvider(jg, serializers, row);
}
catch (IOException e) {
// Ideally, this shouldn't be reachable.
// Wrap the IO exception in the runtime exception and propogate it forward
List<String> elements = new ArrayList<>();
for (Object o : row) {
elements.add(o.toString());
}
throw new RE(
e,
"Exception encountered while serializing [%s] in [%s]",
String.join(", ", elements),
FrameBasedInlineDataSource.class
);
}
});
jg.writeEndArray();
jg.writeEndObject();
}
/**
* Required because {@link DataSource} is polymorphic
*/
@Override
public void serializeWithType(
FrameBasedInlineDataSource value,
JsonGenerator jg,
SerializerProvider serializers,
TypeSerializer typeSer
) throws IOException
{
serialize(value, jg, serializers);
}
}

View File

@ -0,0 +1,52 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query;
import com.google.common.base.Preconditions;
import org.apache.druid.frame.Frame;
import org.apache.druid.segment.column.RowSignature;
/**
* Encapsulates a {@link Frame} and the {@link RowSignature} of the rows that are encapsulated in it
*/
public class FrameSignaturePair
{
final Frame frame;
final RowSignature rowSignature;
public FrameSignaturePair(
Frame frame,
RowSignature rowSignature
)
{
this.frame = Preconditions.checkNotNull(frame, "'frame' must be non null");
this.rowSignature = Preconditions.checkNotNull(rowSignature, "'rowSignature' must be non null");
}
public Frame getFrame()
{
return frame;
}
public RowSignature getRowSignature()
{
return rowSignature;
}
}

View File

@ -190,6 +190,7 @@ public class InlineDataSource implements DataSource
@JsonProperty("rows")
public List<Object[]> getRowsAsList()
{
Iterable<Object[]> rows = getRows();
return rows instanceof List ? ((List<Object[]>) rows) : Lists.newArrayList(rows);
}

View File

@ -0,0 +1,106 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.guava.Yielders;
import org.apache.druid.segment.RowAdapter;
import org.apache.druid.segment.RowBasedCursor;
import org.apache.druid.segment.RowWalker;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.RowSignature;
import java.util.Iterator;
/**
* Helper methods to create cursor from iterable of rows
*/
public class IterableRowsCursorHelper
{
/**
* Creates a cursor that iterates over all the rows generated by the iterable. Presence of __time column is not a
* necessity
*/
public static RowBasedCursor<Object[]> getCursorFromIterable(Iterable<Object[]> rows, RowSignature rowSignature)
{
RowAdapter<Object[]> rowAdapter = columnName -> {
if (rowSignature == null) {
return row -> null;
}
final int columnIndex = rowSignature.indexOf(columnName);
if (columnIndex < 0) {
return row -> null;
}
return row -> row[columnIndex];
};
RowWalker<Object[]> rowWalker = new RowWalker<>(Sequences.simple(rows), rowAdapter);
return new RowBasedCursor<>(
rowWalker,
rowAdapter,
null,
Intervals.ETERNITY, // Setting the interval to eternity ensures that we are iterating over all of the rows
VirtualColumns.EMPTY,
Granularities.ALL,
false,
rowSignature != null ? rowSignature : RowSignature.empty()
);
}
/**
* Creates a cursor that iterates over all the rows generated by the sequence. Presence of __time column is not a
* necessity
*/
public static RowBasedCursor<Object[]> getCursorFromSequence(Sequence<Object[]> rows, RowSignature rowSignature)
{
return getCursorFromIterable(
new Iterable<Object[]>()
{
Yielder<Object[]> yielder = Yielders.each(rows);
@Override
public Iterator<Object[]> iterator()
{
return new Iterator<Object[]>()
{
@Override
public boolean hasNext()
{
return !yielder.isDone();
}
@Override
public Object[] next()
{
Object[] retVal = yielder.get();
yielder = yielder.next(null);
return retVal;
}
};
}
},
rowSignature
);
}
}

View File

@ -357,6 +357,16 @@ public class QueryContext
return getInt(QueryContexts.MAX_SUBQUERY_ROWS_KEY, defaultSize);
}
public long getMaxSubqueryMemoryBytes(long defaultMemoryBytes)
{
return getLong(QueryContexts.MAX_SUBQUERY_BYTES_KEY, defaultMemoryBytes);
}
public boolean isUseNestedForUnknownTypeInSubquery(boolean defaultUseNestedForUnkownTypeInSubquery)
{
return getBoolean(QueryContexts.USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY, defaultUseNestedForUnkownTypeInSubquery);
}
public int getUncoveredIntervalsLimit()
{
return getUncoveredIntervalsLimit(QueryContexts.DEFAULT_UNCOVERED_INTERVALS_LIMIT);

View File

@ -54,6 +54,8 @@ public class QueryContexts
public static final String VECTORIZE_VIRTUAL_COLUMNS_KEY = "vectorizeVirtualColumns";
public static final String VECTOR_SIZE_KEY = "vectorSize";
public static final String MAX_SUBQUERY_ROWS_KEY = "maxSubqueryRows";
public static final String MAX_SUBQUERY_BYTES_KEY = "maxSubqueryBytes";
public static final String USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY = "useNestedForUnknownTypeInSubquery";
public static final String JOIN_FILTER_PUSH_DOWN_KEY = "enableJoinFilterPushDown";
public static final String JOIN_FILTER_REWRITE_ENABLE_KEY = "enableJoinFilterRewrite";
public static final String JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY = "enableJoinFilterRewriteValueColumnFilters";

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.JavaType;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.type.TypeFactory;
import com.google.common.base.Function;
import org.apache.druid.frame.allocation.MemoryAllocatorFactory;
import org.apache.druid.guice.annotations.ExtensionPoint;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.guava.Sequence;
@ -34,6 +35,7 @@ import org.apache.druid.timeline.LogicalSegment;
import javax.annotation.Nullable;
import java.util.Comparator;
import java.util.List;
import java.util.Optional;
import java.util.function.BinaryOperator;
/**
@ -318,7 +320,7 @@ public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultT
* query, each {@link org.apache.druid.query.topn.TopNResultValue} will generate a separate array for each of its
* {@code values}.
*
* By convention, the array form should include the __time column, if present, as a long (milliseconds since epoch).
* By convention, the array form should include the __time column, if present, as a long (milliseconds since epoch).
*
* @param resultSequence results of the form returned by {@link #mergeResults}
*
@ -330,4 +332,32 @@ public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultT
{
throw new UOE("Query type '%s' does not support returning results as arrays", query.getType());
}
/**
* Converts a sequence of this query's ResultType into a sequence of {@link FrameSignaturePair}. The array signature
* is the one give by {@link #resultArraySignature(Query)}. If the toolchest doesn't support this method, then it can
* return an empty optional. It is the duty of the callees to throw an appropriate exception in that case or use an
* alternative fallback approach
*
* Check documentation of {@link #resultsAsArrays(Query, Sequence)} as the behaviour of the rows represented by the
* frame sequence is identical.
*
* Each Frame has a separate {@link RowSignature} because for some query types like the Scan query, every
* column in the final result might not be present in the individual ResultType (and subsequently Frame). Therefore,
* this is done to preserve the space by not populating the column in that particular Frame and omitting it from its
* signature
* @param query Query being executed by the toolchest. Used to determine the rowSignature of the Frames
* @param resultSequence results of the form returned by {@link #mergeResults(QueryRunner)}
* @param memoryAllocatorFactory
* @param useNestedForUnknownTypes true if the unknown types in the results can be serded using complex types
*/
public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
QueryType query,
Sequence<ResultType> resultSequence,
MemoryAllocatorFactory memoryAllocatorFactory,
boolean useNestedForUnknownTypes
)
{
return Optional.empty();
}
}

View File

@ -36,6 +36,13 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.inject.Inject;
import org.apache.druid.data.input.Row;
import org.apache.druid.frame.Frame;
import org.apache.druid.frame.FrameType;
import org.apache.druid.frame.allocation.MemoryAllocatorFactory;
import org.apache.druid.frame.segment.FrameCursorUtils;
import org.apache.druid.frame.write.FrameWriterFactory;
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.MappedSequence;
@ -44,6 +51,8 @@ import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.query.CacheStrategy;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.FrameSignaturePair;
import org.apache.druid.query.IterableRowsCursorHelper;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryDataSource;
import org.apache.druid.query.QueryPlus;
@ -62,6 +71,7 @@ import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.query.groupby.resource.GroupByQueryResource;
import org.apache.druid.query.groupby.strategy.GroupByStrategy;
import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.column.RowSignature;
import org.joda.time.DateTime;
@ -73,6 +83,7 @@ import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.TreeMap;
import java.util.function.BinaryOperator;
@ -705,6 +716,40 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<ResultRow, GroupB
return resultSequence.map(ResultRow::getArray);
}
/**
* This returns a single frame containing the results of the group by query.
*/
@Override
public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
GroupByQuery query,
Sequence<ResultRow> resultSequence,
MemoryAllocatorFactory memoryAllocatorFactory,
boolean useNestedForUnknownTypes
)
{
RowSignature rowSignature = resultArraySignature(query);
RowSignature modifiedRowSignature = useNestedForUnknownTypes
? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature)
: rowSignature;
FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
FrameType.COLUMNAR,
memoryAllocatorFactory,
modifiedRowSignature,
new ArrayList<>()
);
Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(
resultsAsArrays(query, resultSequence),
rowSignature
);
Sequence<Frame> frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory);
return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature)));
}
/**
* This function checks the query for dimensions which can be optimized by applying the dimension extraction
* as the final step of the query instead of on every event.

View File

@ -0,0 +1,328 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.scan;
import com.google.common.base.Predicate;
import com.google.common.math.IntMath;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.ValueMatcher;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.IdLookup;
import org.apache.druid.segment.RowIdSupplier;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.data.IndexedInts;
import org.joda.time.DateTime;
import javax.annotation.Nullable;
import java.util.List;
/**
* Combines multiple cursors and iterates over them. It skips over the empty cursors
* The {@link DimensionSelector} and {@link ColumnValueSelector} it generates hold the reference to the original object
* because the cursor might be advanced independently after extracting out the {@link ColumnSelectorFactory} like in
* {@link org.apache.druid.frame.segment.FrameCursorUtils#cursorToFrames}. This ensures that the selectors always return
* the value pointed by the {@link #currentCursor}.
*/
public class ConcatCursor implements Cursor
{
private final List<Cursor> cursors;
private int currentCursor;
public ConcatCursor(
List<Cursor> cursors
)
{
this.cursors = cursors;
currentCursor = 0;
skipEmptyCursors();
}
@Override
public ColumnSelectorFactory getColumnSelectorFactory()
{
return new ColumnSelectorFactory()
{
@Override
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
{
return new DimensionSelector()
{
@Override
public IndexedInts getRow()
{
return cursors.get(currentCursor).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec).getRow();
}
@Override
public ValueMatcher makeValueMatcher(@Nullable String value)
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeDimensionSelector(dimensionSpec)
.makeValueMatcher(value);
}
@Override
public ValueMatcher makeValueMatcher(Predicate<String> predicate)
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeDimensionSelector(dimensionSpec)
.makeValueMatcher(predicate);
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeDimensionSelector(dimensionSpec)
.inspectRuntimeShape(inspector);
}
@Nullable
@Override
public Object getObject()
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeDimensionSelector(dimensionSpec)
.getObject();
}
@Override
public Class<?> classOfObject()
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeDimensionSelector(dimensionSpec)
.classOfObject();
}
@Override
public int getValueCardinality()
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeDimensionSelector(dimensionSpec)
.getValueCardinality();
}
@Nullable
@Override
public String lookupName(int id)
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeDimensionSelector(dimensionSpec)
.lookupName(id);
}
@Override
public boolean nameLookupPossibleInAdvance()
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeDimensionSelector(dimensionSpec)
.nameLookupPossibleInAdvance();
}
@Nullable
@Override
public IdLookup idLookup()
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeDimensionSelector(dimensionSpec)
.idLookup();
}
};
}
@Override
public ColumnValueSelector makeColumnValueSelector(String columnName)
{
return new ColumnValueSelector()
{
@Override
public double getDouble()
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeColumnValueSelector(columnName)
.getDouble();
}
@Override
public float getFloat()
{
return cursors.get(currentCursor).getColumnSelectorFactory().makeColumnValueSelector(columnName).getFloat();
}
@Override
public long getLong()
{
return cursors.get(currentCursor).getColumnSelectorFactory().makeColumnValueSelector(columnName).getLong();
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeColumnValueSelector(columnName)
.inspectRuntimeShape(inspector);
}
@Override
public boolean isNull()
{
return cursors.get(currentCursor).getColumnSelectorFactory().makeColumnValueSelector(columnName).isNull();
}
@Nullable
@Override
public Object getObject()
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeColumnValueSelector(columnName)
.getObject();
}
@Override
public Class classOfObject()
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.makeColumnValueSelector(columnName)
.classOfObject();
}
};
}
@Override
public ColumnCapabilities getColumnCapabilitiesWithDefault(String column, ColumnCapabilities defaultCapabilites)
{
return cursors.get(currentCursor)
.getColumnSelectorFactory()
.getColumnCapabilitiesWithDefault(column, defaultCapabilites);
}
@Nullable
@Override
public ExpressionType getType(String name)
{
return cursors.get(currentCursor).getColumnSelectorFactory().getType(name);
}
@Nullable
@Override
public ColumnCapabilities getColumnCapabilities(String column)
{
return cursors.get(currentCursor).getColumnSelectorFactory().getColumnCapabilities(column);
}
@Nullable
@Override
public RowIdSupplier getRowIdSupplier()
{
return cursors.get(currentCursor).getColumnSelectorFactory().getRowIdSupplier();
}
};
}
@Override
public DateTime getTime()
{
return cursors.get(currentCursor).getTime();
}
@Override
public void advance()
{
if (currentCursor < cursors.size()) {
cursors.get(currentCursor).advance();
advanceCursor();
}
}
@Override
public void advanceUninterruptibly()
{
if (currentCursor < cursors.size()) {
cursors.get(currentCursor).advanceUninterruptibly();
advanceCursor();
}
}
@Override
public boolean isDone()
{
return currentCursor == cursors.size();
}
@Override
public boolean isDoneOrInterrupted()
{
return isDone() || Thread.currentThread().isInterrupted();
}
@Override
public void reset()
{
while (currentCursor >= 0) {
if (currentCursor < cursors.size()) {
cursors.get(currentCursor).reset();
}
currentCursor = IntMath.checkedSubtract(currentCursor, 1);
}
currentCursor = 0;
skipEmptyCursors();
}
/**
* This method should be called whenever the currentCursor gets updated. It skips over the empty cursors so that the
* current pointer is pointing to a valid cursor
*/
private void skipEmptyCursors()
{
while (currentCursor < cursors.size() && cursors.get(currentCursor).isDone()) {
currentCursor = IntMath.checkedAdd(currentCursor, 1);
}
}
/**
* This method updates the current cursor. This is used to update the current cursor under question.
*/
private void advanceCursor()
{
if (cursors.get(currentCursor).isDone()) {
currentCursor = IntMath.checkedAdd(currentCursor, 1);
skipEmptyCursors();
}
}
}

View File

@ -39,7 +39,9 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.filter.Filters;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
@ -146,6 +148,7 @@ public class ScanQueryEngine
public Iterator<ScanResultValue> make()
{
final List<BaseObjectColumnValueSelector> columnSelectors = new ArrayList<>(allColumns.size());
final RowSignature.Builder rowSignatureBuilder = RowSignature.builder();
for (String column : allColumns) {
final BaseObjectColumnValueSelector selector;
@ -153,8 +156,20 @@ public class ScanQueryEngine
if (legacy && LEGACY_TIMESTAMP_KEY.equals(column)) {
selector = cursor.getColumnSelectorFactory()
.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME);
ColumnCapabilities columnCapabilities = cursor.getColumnSelectorFactory()
.getColumnCapabilities(ColumnHolder.TIME_COLUMN_NAME);
rowSignatureBuilder.add(
column,
columnCapabilities == null ? null : columnCapabilities.toColumnType()
);
} else {
selector = cursor.getColumnSelectorFactory().makeColumnValueSelector(column);
ColumnCapabilities columnCapabilities = cursor.getColumnSelectorFactory()
.getColumnCapabilities(column);
rowSignatureBuilder.add(
column,
columnCapabilities == null ? null : columnCapabilities.toColumnType()
);
}
columnSelectors.add(selector);
@ -191,7 +206,7 @@ public class ScanQueryEngine
throw new UOE("resultFormat[%s] is not supported", resultFormat.toString());
}
responseContext.addRowScanCount(offset - lastOffset);
return new ScanResultValue(segmentId.toString(), allColumns, events);
return new ScanResultValue(segmentId.toString(), allColumns, events, rowSignatureBuilder.build());
}
@Override

View File

@ -30,6 +30,7 @@ import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.column.RowSignature;
import java.io.IOException;
import java.util.ArrayList;
@ -111,7 +112,12 @@ public class ScanQueryLimitRowIterator implements CloseableIterator<ScanResultVa
// single batch length is <= Integer.MAX_VALUE, so this should not overflow
int numLeft = (int) (limit - count);
count = limit;
return new ScanResultValue(batch.getSegmentId(), batch.getColumns(), events.subList(0, numLeft));
return new ScanResultValue(
batch.getSegmentId(),
batch.getColumns(),
events.subList(0, numLeft),
batch.getRowSignature()
);
}
} else {
// Perform single-event ScanResultValue batching at the outer level. Each scan result value from the yielder
@ -119,15 +125,17 @@ public class ScanQueryLimitRowIterator implements CloseableIterator<ScanResultVa
int batchSize = query.getBatchSize();
List<Object> eventsToAdd = new ArrayList<>(batchSize);
List<String> columns = new ArrayList<>();
RowSignature rowSignature = null;
while (eventsToAdd.size() < batchSize && !yielder.isDone() && count < limit) {
ScanResultValue srv = yielder.get();
// Only replace once using the columns from the first event
columns = columns.isEmpty() ? srv.getColumns() : columns;
rowSignature = rowSignature == null ? srv.getRowSignature() : rowSignature;
eventsToAdd.add(Iterables.getOnlyElement((List<Object>) srv.getEvents()));
yielder = yielder.next(null);
count++;
}
return new ScanResultValue(null, columns, eventsToAdd);
return new ScanResultValue(null, columns, eventsToAdd, rowSignature);
}
}

View File

@ -117,7 +117,7 @@ public class ScanQueryOffsetSequence extends YieldingSequenceBase<ScanResultValu
skipped += toSkip;
return super.accumulate(
accumulated,
new ScanResultValue(result.getSegmentId(), result.getColumns(), newEvents)
new ScanResultValue(result.getSegmentId(), result.getColumns(), newEvents, result.getRowSignature())
);
}
} else {

View File

@ -22,27 +22,46 @@ package org.apache.druid.query.scan;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Function;
import com.google.common.base.Functions;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import com.google.common.collect.PeekingIterator;
import com.google.inject.Inject;
import org.apache.druid.frame.Frame;
import org.apache.druid.frame.FrameType;
import org.apache.druid.frame.allocation.MemoryAllocatorFactory;
import org.apache.druid.frame.segment.FrameCursorUtils;
import org.apache.druid.frame.write.FrameWriterFactory;
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.guava.BaseSequence;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.guava.Yielders;
import org.apache.druid.query.FrameSignaturePair;
import org.apache.druid.query.GenericQueryMetricsFactory;
import org.apache.druid.query.IterableRowsCursorHelper;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryMetrics;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.aggregation.MetricManipulationFn;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.utils.CloseableUtils;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, ScanQuery>
{
@ -187,13 +206,146 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
}
}
/**
* This batches the fetched {@link ScanResultValue}s which have similar signatures and are consecutives. In best case
* it would return a single frame, and in the worst case, it would return as many frames as the number of {@link ScanResultValue}
* passed.
*/
@Override
public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
final ScanQuery query,
final Sequence<ScanResultValue> resultSequence,
MemoryAllocatorFactory memoryAllocatorFactory,
boolean useNestedForUnknownTypes
)
{
final RowSignature defaultRowSignature = resultArraySignature(query);
Iterator<ScanResultValue> resultSequenceIterator = new Iterator<ScanResultValue>()
{
Yielder<ScanResultValue> yielder = Yielders.each(resultSequence);
@Override
public boolean hasNext()
{
return !yielder.isDone();
}
@Override
public ScanResultValue next()
{
ScanResultValue scanResultValue = yielder.get();
yielder = yielder.next(null);
return scanResultValue;
}
};
Iterable<Sequence<FrameSignaturePair>> retVal = () -> new Iterator<Sequence<FrameSignaturePair>>()
{
PeekingIterator<ScanResultValue> scanResultValuePeekingIterator = Iterators.peekingIterator(resultSequenceIterator);
@Override
public boolean hasNext()
{
return scanResultValuePeekingIterator.hasNext();
}
@Override
public Sequence<FrameSignaturePair> next()
{
final List<ScanResultValue> batch = new ArrayList<>();
final ScanResultValue scanResultValue = scanResultValuePeekingIterator.next();
batch.add(scanResultValue);
// If the rowSignature is not provided, assume that the scanResultValue can contain any number of the columns
// that appear in the original scan query
final RowSignature rowSignature = scanResultValue.getRowSignature() != null
? scanResultValue.getRowSignature()
: defaultRowSignature;
while (scanResultValuePeekingIterator.hasNext()) {
RowSignature nextRowSignature = scanResultValuePeekingIterator.peek().getRowSignature();
if (nextRowSignature == null) {
nextRowSignature = defaultRowSignature;
}
if (nextRowSignature != null && nextRowSignature.equals(rowSignature)) {
batch.add(scanResultValuePeekingIterator.next());
} else {
break;
}
}
return convertScanResultValuesToFrame(
batch,
rowSignature,
query,
memoryAllocatorFactory,
useNestedForUnknownTypes
);
}
};
return Optional.of(Sequences.concat(retVal));
}
private Sequence<FrameSignaturePair> convertScanResultValuesToFrame(
List<ScanResultValue> batch,
RowSignature rowSignature,
ScanQuery query,
MemoryAllocatorFactory memoryAllocatorFactory,
boolean useNestedForUnknownTypes
)
{
Preconditions.checkNotNull(rowSignature, "'rowSignature' must be provided");
List<Cursor> cursors = new ArrayList<>();
for (ScanResultValue scanResultValue : batch) {
final List rows = (List) scanResultValue.getEvents();
final Function<?, Object[]> mapper = getResultFormatMapper(query.getResultFormat(), rowSignature.getColumnNames());
final Iterable<Object[]> formattedRows = Lists.newArrayList(Iterables.transform(rows, (Function) mapper));
cursors.add(IterableRowsCursorHelper.getCursorFromIterable(
formattedRows,
rowSignature
));
}
RowSignature modifiedRowSignature = useNestedForUnknownTypes
? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature)
: rowSignature;
FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
FrameType.COLUMNAR,
memoryAllocatorFactory,
modifiedRowSignature,
new ArrayList<>()
);
Cursor concatCursor = new ConcatCursor(cursors);
Sequence<Frame> frames = FrameCursorUtils.cursorToFrames(
concatCursor,
frameWriterFactory
);
return frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature));
}
@Override
public Sequence<Object[]> resultsAsArrays(final ScanQuery query, final Sequence<ScanResultValue> resultSequence)
{
final List<String> fields = resultArraySignature(query).getColumnNames();
final Function<?, Object[]> mapper;
final Function<?, Object[]> mapper = getResultFormatMapper(query.getResultFormat(), resultArraySignature(query).getColumnNames());
switch (query.getResultFormat()) {
return resultSequence.flatMap(
result -> {
// Generics? Where we're going, we don't need generics.
final List rows = (List) result.getEvents();
final Iterable arrays = Iterables.transform(rows, (Function) mapper);
return Sequences.simple(arrays);
}
);
}
private Function<?, Object[]> getResultFormatMapper(ScanQuery.ResultFormat resultFormat, List<String> fields)
{
Function<?, Object[]> mapper;
switch (resultFormat) {
case RESULT_FORMAT_LIST:
mapper = (Map<String, Object> row) -> {
final Object[] rowArray = new Object[fields.size()];
@ -220,16 +372,8 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
};
break;
default:
throw new UOE("Unsupported resultFormat for array-based results: %s", query.getResultFormat());
throw new UOE("Unsupported resultFormat for array-based results: %s", resultFormat);
}
return resultSequence.flatMap(
result -> {
// Generics? Where we're going, we don't need generics.
final List rows = (List) result.getEvents();
final Iterable arrays = Iterables.transform(rows, (Function) mapper);
return Sequences.simple(arrays);
}
);
return mapper;
}
}

View File

@ -25,12 +25,14 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.RowSignature;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
public class ScanResultValue implements Comparable<ScanResultValue>
{
@ -45,17 +47,25 @@ public class ScanResultValue implements Comparable<ScanResultValue>
private final String segmentId;
private final List<String> columns;
private final Object events;
private final RowSignature rowSignature;
@JsonCreator
public ScanResultValue(
@JsonProperty("segmentId") @Nullable String segmentId,
@JsonProperty("columns") List<String> columns,
@JsonProperty("events") Object events
@JsonProperty("events") Object events,
@Nullable @JsonProperty("rowSignature") RowSignature rowSignature
)
{
this.segmentId = segmentId;
this.columns = columns;
this.events = events;
this.rowSignature = rowSignature;
}
public ScanResultValue(String segmentId, List<String> columns, Object events)
{
this(segmentId, columns, events, null);
}
@Nullable
@ -77,6 +87,14 @@ public class ScanResultValue implements Comparable<ScanResultValue>
return events;
}
@Nullable
@JsonProperty
public RowSignature getRowSignature()
{
return rowSignature;
}
public long getFirstEventTimestamp(ScanQuery.ResultFormat resultFormat)
{
if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) {
@ -101,7 +119,7 @@ public class ScanResultValue implements Comparable<ScanResultValue>
List<ScanResultValue> singleEventScanResultValues = new ArrayList<>();
List<Object> events = (List<Object>) this.getEvents();
for (Object event : events) {
singleEventScanResultValues.add(new ScanResultValue(segmentId, columns, Collections.singletonList(event)));
singleEventScanResultValues.add(new ScanResultValue(segmentId, columns, Collections.singletonList(event), rowSignature));
}
return singleEventScanResultValues;
}
@ -118,13 +136,16 @@ public class ScanResultValue implements Comparable<ScanResultValue>
ScanResultValue that = (ScanResultValue) o;
if (segmentId != null ? !segmentId.equals(that.segmentId) : that.segmentId != null) {
if (!Objects.equals(segmentId, that.segmentId)) {
return false;
}
if (columns != null ? !columns.equals(that.columns) : that.columns != null) {
if (!Objects.equals(columns, that.columns)) {
return false;
}
return events != null ? events.equals(that.events) : that.events == null;
if (!Objects.equals(rowSignature, that.rowSignature)) {
return false;
}
return Objects.equals(events, that.events);
}
@Override
@ -133,6 +154,7 @@ public class ScanResultValue implements Comparable<ScanResultValue>
int result = segmentId != null ? segmentId.hashCode() : 0;
result = 31 * result + (columns != null ? columns.hashCode() : 0);
result = 31 * result + (events != null ? events.hashCode() : 0);
result = 31 * result + (rowSignature != null ? rowSignature.hashCode() : 0);
return result;
}
@ -143,6 +165,7 @@ public class ScanResultValue implements Comparable<ScanResultValue>
"segmentId='" + segmentId + '\'' +
", columns=" + columns +
", events=" + events +
", rowSignature=" + rowSignature +
'}';
}

View File

@ -30,12 +30,21 @@ import com.google.common.collect.Maps;
import com.google.inject.Inject;
import org.apache.commons.lang.StringUtils;
import org.apache.druid.data.input.MapBasedRow;
import org.apache.druid.frame.Frame;
import org.apache.druid.frame.FrameType;
import org.apache.druid.frame.allocation.MemoryAllocatorFactory;
import org.apache.druid.frame.segment.FrameCursorUtils;
import org.apache.druid.frame.write.FrameWriterFactory;
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.CacheStrategy;
import org.apache.druid.query.FrameSignaturePair;
import org.apache.druid.query.IterableRowsCursorHelper;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
@ -49,18 +58,21 @@ import org.apache.druid.query.aggregation.MetricManipulationFn;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.RowAdapters;
import org.apache.druid.segment.RowBasedColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.joda.time.DateTime;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.BinaryOperator;
/**
@ -431,7 +443,6 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
)
{
final List<String> fields = resultArraySignature(query).getColumnNames();
return Sequences.map(
resultSequence,
result -> {
@ -451,6 +462,39 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
);
}
/**
* This returns a single frame containing the results of the timeseries query
*/
@Override
public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
TimeseriesQuery query,
Sequence<Result<TimeseriesResultValue>> resultSequence,
MemoryAllocatorFactory memoryAllocatorFactory,
boolean useNestedForUnknownTypes
)
{
final RowSignature rowSignature = resultArraySignature(query);
final Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(
resultsAsArrays(query, resultSequence),
rowSignature
);
RowSignature modifiedRowSignature = useNestedForUnknownTypes
? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature)
: rowSignature;
FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
FrameType.COLUMNAR,
memoryAllocatorFactory,
modifiedRowSignature,
new ArrayList<>()
);
Sequence<Frame> frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory);
// All frames are generated with the same signature therefore we can attach the row signature
return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature)));
}
private Function<Result<TimeseriesResultValue>, Result<TimeseriesResultValue>> makeComputeManipulatorFn(
final TimeseriesQuery query,
final MetricManipulationFn fn,

View File

@ -26,12 +26,21 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.inject.Inject;
import org.apache.druid.frame.Frame;
import org.apache.druid.frame.FrameType;
import org.apache.druid.frame.allocation.MemoryAllocatorFactory;
import org.apache.druid.frame.segment.FrameCursorUtils;
import org.apache.druid.frame.write.FrameWriterFactory;
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.BySegmentResultValue;
import org.apache.druid.query.CacheStrategy;
import org.apache.druid.query.FrameSignaturePair;
import org.apache.druid.query.IterableRowsCursorHelper;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryPlus;
@ -47,15 +56,18 @@ import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.column.RowSignature;
import org.joda.time.DateTime;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.BinaryOperator;
/**
@ -548,6 +560,38 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
);
}
/**
* This returns a single frame containing the rows of the topN query's results
*/
@Override
public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(
TopNQuery query,
Sequence<Result<TopNResultValue>> resultSequence,
MemoryAllocatorFactory memoryAllocatorFactory,
boolean useNestedForUnknownTypes
)
{
final RowSignature rowSignature = resultArraySignature(query);
final Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(
resultsAsArrays(query, resultSequence),
rowSignature
);
RowSignature modifiedRowSignature = useNestedForUnknownTypes
? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature)
: rowSignature;
FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
FrameType.COLUMNAR,
memoryAllocatorFactory,
rowSignature,
new ArrayList<>()
);
Sequence<Frame> frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory);
return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature)));
}
static class ThresholdAdjustingQueryRunner implements QueryRunner<Result<TopNResultValue>>
{
private final QueryRunner<Result<TopNResultValue>> runner;

View File

@ -48,7 +48,7 @@ public class RowBasedCursor<RowType> implements Cursor
private long rowId = 0;
RowBasedCursor(
public RowBasedCursor(
final RowWalker<RowType> rowWalker,
final RowAdapter<RowType> rowAdapter,
@Nullable final Filter filter,

View File

@ -41,7 +41,7 @@ public class RowWalker<T>
@Nullable // null = closed
private Yielder<T> rowYielder;
RowWalker(final Sequence<T> rowSequence, final RowAdapter<T> rowAdapter)
public RowWalker(final Sequence<T> rowSequence, final RowAdapter<T> rowAdapter)
{
this.rowSequence = rowSequence;
this.timestampFunction = rowAdapter.timestampFunction();

View File

@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList;
import it.unimi.dsi.fastutil.objects.Object2IntMap;
import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.query.InlineDataSource;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.dimension.DimensionSpec;
@ -44,7 +45,7 @@ import java.util.Optional;
* Type signature for a row in a Druid datasource or query result.
*
* @see org.apache.druid.query.QueryToolChest#resultArraySignature which returns signatures for query results
* @see org.apache.druid.query.InlineDataSource#getRowSignature which returns signatures for inline datasources
* @see InlineDataSource#getRowSignature which returns signatures for inline datasources
*/
public class RowSignature implements ColumnInspector
{

View File

@ -0,0 +1,64 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.segment.join;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.query.InlineDataSource;
import org.apache.druid.segment.join.table.FrameBasedIndexedTable;
import org.apache.druid.segment.join.table.IndexedTableJoinable;
import java.util.Optional;
import java.util.Set;
/**
* Creates a joinable from the {@link FrameBasedInlineDataSource}. This materializes the datasource to an
* {@link InlineDataSource}, before creating the joinable on it, which carries the overhead of this conversion.
*/
public class FrameBasedInlineJoinableFactory implements JoinableFactory
{
@Override
public boolean isDirectlyJoinable(DataSource dataSource)
{
return dataSource instanceof FrameBasedInlineDataSource;
}
@Override
public Optional<Joinable> build(DataSource dataSource, JoinConditionAnalysis condition)
{
FrameBasedInlineDataSource frameBasedInlineDataSource = (FrameBasedInlineDataSource) dataSource;
if (condition.canHashJoin()) {
final Set<String> rightKeyColumns = condition.getRightEquiConditionKeys();
return Optional.of(
new IndexedTableJoinable(
new FrameBasedIndexedTable(
frameBasedInlineDataSource,
rightKeyColumns,
DateTimes.nowUtc().toString()
)
)
);
}
return Optional.empty();
}
}

View File

@ -30,7 +30,8 @@ import java.util.Optional;
import java.util.Set;
/**
* A {@link JoinableFactory} for {@link InlineDataSource}. It works by building an {@link IndexedTable}.
* A {@link JoinableFactory} for {@link InlineDataSource}.
* It works by building an {@link IndexedTable}.
*
* It is not valid to pass any other DataSource type to the "build" method.
*/
@ -48,7 +49,7 @@ public class InlineJoinableFactory implements JoinableFactory
@Override
public Optional<Joinable> build(final DataSource dataSource, final JoinConditionAnalysis condition)
{
final InlineDataSource inlineDataSource = (InlineDataSource) dataSource;
InlineDataSource inlineDataSource = (InlineDataSource) dataSource;
if (condition.canHashJoin()) {
final Set<String> rightKeyColumns = condition.getRightEquiConditionKeys();

View File

@ -0,0 +1,336 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.segment.join.table;
import com.google.common.base.Preconditions;
import com.google.common.math.IntMath;
import org.apache.druid.frame.Frame;
import org.apache.druid.frame.read.FrameReader;
import org.apache.druid.frame.read.columnar.FrameColumnReader;
import org.apache.druid.frame.read.columnar.FrameColumnReaders;
import org.apache.druid.frame.segment.FrameStorageAdapter;
import org.apache.druid.frame.segment.columnar.FrameQueryableIndex;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.query.FrameSignaturePair;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.SimpleAscendingOffset;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.BaseColumn;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
public class FrameBasedIndexedTable implements IndexedTable
{
private static final Logger LOG = new Logger(FrameBasedIndexedTable.class);
private final Set<String> keyColumns;
private final RowSignature rowSignature;
private final String version;
private final List<IndexedTable.Index> keyColumnsIndexes;
private final int numRows;
private final List<QueryableIndex> frameQueryableIndexes = new ArrayList<>();
private final List<Integer> cumulativeRowCount = new ArrayList<>();
public FrameBasedIndexedTable(
final FrameBasedInlineDataSource frameBasedInlineDataSource,
final Set<String> keyColumns,
final String version
)
{
this.keyColumns = keyColumns;
this.version = version;
this.rowSignature = frameBasedInlineDataSource.getRowSignature();
int rowCount = 0;
for (FrameSignaturePair frameSignaturePair : frameBasedInlineDataSource.getFrames()) {
Frame frame = frameSignaturePair.getFrame();
RowSignature frameRowSignature = frameSignaturePair.getRowSignature();
frameQueryableIndexes.add(new FrameQueryableIndex(
frame,
frameRowSignature,
createColumnReaders(frameRowSignature)
));
rowCount += frame.numRows();
cumulativeRowCount.add(rowCount);
}
this.numRows = rowCount;
final ArrayList<RowBasedIndexBuilder> indexBuilders = new ArrayList<>(rowSignature.size());
final List<String> keyColumnNames = new ArrayList<>(keyColumns.size());
for (int i = 0; i < rowSignature.size(); i++) {
final RowBasedIndexBuilder m;
final String columnName = rowSignature.getColumnName(i);
if (keyColumns.contains(columnName)) {
final ColumnType keyType =
rowSignature.getColumnType(i).orElse(IndexedTableJoinMatcher.DEFAULT_KEY_TYPE);
m = new RowBasedIndexBuilder(keyType);
keyColumnNames.add(columnName);
} else {
m = null;
}
indexBuilders.add(m);
}
final Sequence<Cursor> cursors = Sequences.concat(
frameBasedInlineDataSource
.getFrames()
.stream()
.map(frameSignaturePair -> {
Frame frame = frameSignaturePair.getFrame();
RowSignature rowSignature = frameSignaturePair.getRowSignature();
FrameStorageAdapter frameStorageAdapter =
new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY);
return frameStorageAdapter.makeCursors(
null,
Intervals.ETERNITY,
VirtualColumns.EMPTY,
Granularities.ALL,
false,
null
);
})
.collect(Collectors.toList())
);
final Sequence<Integer> sequence = Sequences.map(
cursors,
cursor -> {
if (cursor == null) {
return 0;
}
int rowNumber = 0;
ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
// this should really be optimized to use dimension selectors where possible to populate indexes from bitmap
// indexes, but, an optimization for another day
final List<BaseObjectColumnValueSelector> selectors = keyColumnNames
.stream()
.map(columnSelectorFactory::makeColumnValueSelector)
.collect(Collectors.toList());
while (!cursor.isDone()) {
for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex < selectors.size(); keyColumnSelectorIndex++) {
final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex);
final int columnPosition = rowSignature.indexOf(keyColumnName);
final RowBasedIndexBuilder keyColumnIndexBuilder = indexBuilders.get(columnPosition);
keyColumnIndexBuilder.add(selectors.get(keyColumnSelectorIndex).getObject());
}
if (rowNumber % 100_000 == 0) {
if (rowNumber == 0) {
LOG.debug("Indexed first row for frame based datasource");
} else {
LOG.debug("Indexed row %s for frame based datasource", rowNumber);
}
}
rowNumber++;
cursor.advance();
}
return rowNumber;
}
);
Integer totalRows = sequence.accumulate(0, (accumulated, in) -> accumulated += in);
this.keyColumnsIndexes = indexBuilders.stream()
.map(builder -> builder != null ? builder.build() : null)
.collect(Collectors.toList());
LOG.info("Created FrameBasedIndexedTable with %s rows.", totalRows);
}
@Override
public String version()
{
return version;
}
@Override
public Set<String> keyColumns()
{
return keyColumns;
}
@Override
public RowSignature rowSignature()
{
return rowSignature;
}
@Override
public int numRows()
{
return numRows;
}
@Override
public Index columnIndex(int column)
{
return RowBasedIndexedTable.getKeyColumnIndex(column, keyColumnsIndexes);
}
@Override
public Reader columnReader(int column)
{
if (!rowSignature.contains(column)) {
throw new IAE("Column[%d] is not a valid column for the frame based datasource", column);
}
String columnName = rowSignature.getColumnName(column);
final SimpleAscendingOffset offset = new SimpleAscendingOffset(numRows());
final List<BaseObjectColumnValueSelector<?>> columnValueSelectors = new ArrayList<>();
final Set<Closeable> closeables = new HashSet<>();
for (QueryableIndex frameQueryableIndex : frameQueryableIndexes) {
ColumnHolder columnHolder = frameQueryableIndex.getColumnHolder(columnName);
if (columnHolder == null) {
columnValueSelectors.add(NilColumnValueSelector.instance());
} else {
BaseColumn baseColumn = columnHolder.getColumn();
columnValueSelectors.add(baseColumn.makeColumnValueSelector(offset));
closeables.add(baseColumn);
}
}
return new Reader()
{
@Nullable
@Override
public Object read(int row)
{
int frameIndex = binSearch(cumulativeRowCount, row);
if (frameIndex == frameQueryableIndexes.size()) {
throw new IndexOutOfBoundsException(
StringUtils.format("Requested row index [%d], Max row count [%d]", row, numRows())
);
}
// The offset needs to be changed as well
int adjustedOffset = frameIndex == 0
? row
: IntMath.checkedSubtract(row, cumulativeRowCount.get(frameIndex - 1));
offset.setCurrentOffset(adjustedOffset);
return columnValueSelectors.get(frameIndex).getObject();
}
@Override
public void close() throws IOException
{
for (Closeable c : closeables) {
c.close();
}
}
};
}
@Override
public boolean isCacheable()
{
return false;
}
@Override
public void close()
{
}
@Override
public Optional<Closeable> acquireReferences()
{
return Optional.of(
() -> {
}
);
}
private List<FrameColumnReader> createColumnReaders(RowSignature rowSignature)
{
final List<FrameColumnReader> columnReaders = new ArrayList<>(rowSignature.size());
for (int columnNumber = 0; columnNumber < rowSignature.size(); columnNumber++) {
ColumnType columnType = Preconditions.checkNotNull(
rowSignature.getColumnType(columnNumber).orElse(null),
"Type for column [%s]",
rowSignature.getColumnName(columnNumber)
);
columnReaders.add(FrameColumnReaders.create(columnNumber, columnType));
}
return columnReaders;
}
/**
* This method finds out the frame which contains the row indexed "row" from the cumulative array
* This is basically a binary search where we have to find the FIRST element which is STRICTLY GREATER than
* the "row" provided
* <p>
* Note: row is the index (therefore it is 0-indexed)
*/
private int binSearch(List<Integer> arr, int row)
{
int start = 0;
int end = arr.size();
while (start < end) {
// This will be rounded down in case (start + end) is odd. Therefore middle will always be less than
// end and will be equal to or greater than start
int middle = start + (end - start) / 2;
// If the "middle" satisfies the below predicate, then we can move the end backward because every element after
// middle would be satisfying the predicate
if (arr.get(middle) > row) {
end = middle;
} else {
start = middle + 1;
}
}
return start; // Note: at this point, end == start
}
}

View File

@ -0,0 +1,156 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.frame.Frame;
import org.apache.druid.frame.FrameType;
import org.apache.druid.frame.allocation.HeapMemoryAllocator;
import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory;
import org.apache.druid.frame.segment.FrameCursorUtils;
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import java.util.ArrayList;
public class FrameBasedInlineDataSourceSerializerTest
{
static {
NullHandling.initializeForTests();
}
private static final Interval INTERVAL = Intervals.of("2000/P1Y");
private static final RowSignature FOO_INLINE_SIGNATURE = RowSignature.builder()
.addTimeColumn()
.add("s", ColumnType.STRING)
.add("n", ColumnType.LONG)
.build();
private static final InlineDataSource FOO_INLINE = InlineDataSource.fromIterable(
ImmutableList.<Object[]>builder()
.add(new Object[]{INTERVAL.getStartMillis(), "x", 1})
.add(new Object[]{INTERVAL.getStartMillis(), "x", 2})
.add(new Object[]{INTERVAL.getStartMillis(), "y", 3})
.add(new Object[]{INTERVAL.getStartMillis(), "z", 4})
.build(),
FOO_INLINE_SIGNATURE
);
private static final RowSignature BAR_INLINE_SIGNATURE = RowSignature.builder()
.addTimeColumn()
.add("s", ColumnType.STRING)
.add("n", ColumnType.LONG)
.build();
private static final InlineDataSource BAR_INLINE = InlineDataSource.fromIterable(
ImmutableList.<Object[]>builder()
.add(new Object[]{INTERVAL.getStartMillis(), "a", 1})
.add(new Object[]{INTERVAL.getStartMillis(), "a", 2})
.add(new Object[]{INTERVAL.getStartMillis(), "b", 3})
.add(new Object[]{INTERVAL.getStartMillis(), "c", 4})
.build(),
BAR_INLINE_SIGNATURE
);
private static final RowSignature MULTI_VALUE_INLINE_SIGNATURE = RowSignature.builder()
.addTimeColumn()
.add("s", ColumnType.STRING_ARRAY)
.add("n", ColumnType.LONG)
.build();
private static final InlineDataSource MULTI_VALUE_INLINE = InlineDataSource.fromIterable(
ImmutableList.<Object[]>builder()
.add(new Object[]{INTERVAL.getStartMillis(), ImmutableList.of("a", "b"), 1})
.add(new Object[]{INTERVAL.getStartMillis(), ImmutableList.of("a", "c"), 2})
.add(new Object[]{INTERVAL.getStartMillis(), ImmutableList.of("b"), 3})
.add(new Object[]{INTERVAL.getStartMillis(), ImmutableList.of("c"), 4})
.build(),
MULTI_VALUE_INLINE_SIGNATURE
);
ObjectMapper objectMapper = new DefaultObjectMapper();
@Test
public void serialize() throws JsonProcessingException
{
assertConversionBetweenFrameBasedAndIterableBasedInlineDataSource(
convertToFrameBasedInlineDataSource(FOO_INLINE, FOO_INLINE_SIGNATURE),
FOO_INLINE
);
assertConversionBetweenFrameBasedAndIterableBasedInlineDataSource(
convertToFrameBasedInlineDataSource(BAR_INLINE, BAR_INLINE_SIGNATURE),
BAR_INLINE
);
assertConversionBetweenFrameBasedAndIterableBasedInlineDataSource(
convertToFrameBasedInlineDataSource(MULTI_VALUE_INLINE, MULTI_VALUE_INLINE_SIGNATURE),
MULTI_VALUE_INLINE
);
}
private FrameBasedInlineDataSource convertToFrameBasedInlineDataSource(
InlineDataSource inlineDataSource,
RowSignature rowSignature
)
{
Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(
inlineDataSource.getRows(),
rowSignature
);
RowSignature modifiedRowSignature = FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature);
Sequence<Frame> frames = FrameCursorUtils.cursorToFrames(
cursor,
FrameWriters.makeFrameWriterFactory(
FrameType.ROW_BASED,
new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
modifiedRowSignature,
new ArrayList<>()
)
);
return new FrameBasedInlineDataSource(
frames.map(frame -> new FrameSignaturePair(frame, rowSignature)).toList(),
modifiedRowSignature
);
}
private void assertConversionBetweenFrameBasedAndIterableBasedInlineDataSource(
FrameBasedInlineDataSource frameBasedInlineDataSource,
InlineDataSource inlineDataSource
) throws JsonProcessingException
{
String s = objectMapper.writeValueAsString(frameBasedInlineDataSource);
DataSource back = objectMapper.readValue(s, DataSource.class);
Assert.assertEquals(inlineDataSource, back);
}
}

View File

@ -0,0 +1,82 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query;
import com.google.common.collect.ImmutableList;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
public class IterableRowsCursorHelperTest
{
List<Object[]> rows = ImmutableList.of(
new Object[]{1, "a"},
new Object[]{3, "b"},
new Object[]{2, "b"}
);
RowSignature rowSignature = RowSignature.builder()
.add("dim1", ColumnType.LONG)
.add("dim2", ColumnType.STRING)
.build();
@Test
public void getCursorFromIterable()
{
Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(rows, rowSignature);
testCursorMatchesRowSequence(cursor, rowSignature, rows);
}
@Test
public void getCursorFromSequence()
{
Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(Sequences.simple(rows), rowSignature);
testCursorMatchesRowSequence(cursor, rowSignature, rows);
}
private void testCursorMatchesRowSequence(
Cursor cursor,
RowSignature expectedRowSignature,
List<Object[]> expectedRows
)
{
List<Object[]> actualRows = new ArrayList<>();
while (!cursor.isDone()) {
Object[] row = new Object[expectedRowSignature.size()];
for (int i = 0; i < expectedRowSignature.size(); ++i) {
ColumnValueSelector columnValueSelector = cursor.getColumnSelectorFactory()
.makeColumnValueSelector(expectedRowSignature.getColumnName(i));
row[i] = columnValueSelector.getObject();
}
actualRows.add(row);
cursor.advance();
}
QueryToolChestTestHelper.assertArrayResultsEquals(expectedRows, Sequences.simple(actualRows));
}
}

View File

@ -0,0 +1,287 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.scan;
import com.google.common.collect.ImmutableList;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.ListCursor;
import org.junit.Assert;
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
public class ConcatCursorTest
{
@Test
public void testConcatCursor()
{
Cursor dummyCursor1 = new ListCursor(new ArrayList<>());
Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b"));
Cursor dummyCursor2 = new ListCursor(new ArrayList<>());
Cursor cursor2 = new ListCursor(ImmutableList.of("c", "d"));
Cursor dummyCursor3 = new ListCursor(new ArrayList<>());
Cursor concatCursor = new ConcatCursor(ImmutableList.of(
dummyCursor1,
cursor1,
dummyCursor2,
cursor2,
dummyCursor3
));
List<Object> tempList = new ArrayList<>();
// Initial iteration
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b", "c", "d"), tempList);
// Check if reset() works after exhausting the cursor
concatCursor.reset();
tempList.clear();
for (int i = 0; i < 3; ++i) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b", "c"), tempList);
// Check if reset() works from the middle
concatCursor.reset();
tempList.clear();
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b", "c", "d"), tempList);
}
@Test
public void testConcatCursorOfEmptyCursors()
{
Cursor dummyCursor1 = new ListCursor(new ArrayList<>());
Cursor dummyCursor2 = new ListCursor(new ArrayList<>());
Cursor concatCursor = new ConcatCursor(ImmutableList.of(
dummyCursor1,
dummyCursor2
));
Assert.assertTrue(concatCursor.isDone());
}
@Test
public void testConcatCursorWhenBeginningCursorIsEmpty()
{
Cursor dummyCursor1 = new ListCursor(new ArrayList<>());
Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b"));
Cursor concatCursor = new ConcatCursor(ImmutableList.of(
dummyCursor1,
cursor1
));
List<Object> tempList = new ArrayList<>();
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b"), tempList);
// Check if reset() works after exhausting the cursor
concatCursor.reset();
tempList.clear();
for (int i = 0; i < 1; ++i) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a"), tempList);
// Check if reset() works from the middle
concatCursor.reset();
tempList.clear();
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b"), tempList);
}
@Test
public void testConcatCursorWhenEndingCursorIsEmpty()
{
Cursor dummyCursor1 = new ListCursor(new ArrayList<>());
Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b"));
Cursor concatCursor = new ConcatCursor(ImmutableList.of(
cursor1,
dummyCursor1
));
List<Object> tempList = new ArrayList<>();
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b"), tempList);
// Check if reset() works after exhausting the cursor
concatCursor.reset();
tempList.clear();
for (int i = 0; i < 1; ++i) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a"), tempList);
// Check if reset() works from the middle
concatCursor.reset();
tempList.clear();
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b"), tempList);
}
@Test
public void testConcatCursorWhenMultipleEmptyCursorsAtBeginning()
{
Cursor dummyCursor1 = new ListCursor(new ArrayList<>());
Cursor dummyCursor2 = new ListCursor(new ArrayList<>());
Cursor dummyCursor3 = new ListCursor(new ArrayList<>());
Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b"));
Cursor concatCursor = new ConcatCursor(ImmutableList.of(
dummyCursor1,
dummyCursor2,
dummyCursor3,
cursor1
));
List<Object> tempList = new ArrayList<>();
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b"), tempList);
// Check if reset() works after exhausting the cursor
concatCursor.reset();
tempList.clear();
for (int i = 0; i < 1; ++i) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a"), tempList);
// Check if reset() works from the middle
concatCursor.reset();
tempList.clear();
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b"), tempList);
}
@Test
public void testConcatCursorWhenMultipleEmptyCursorsAtEnd()
{
Cursor dummyCursor1 = new ListCursor(new ArrayList<>());
Cursor dummyCursor2 = new ListCursor(new ArrayList<>());
Cursor dummyCursor3 = new ListCursor(new ArrayList<>());
Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b"));
Cursor concatCursor = new ConcatCursor(ImmutableList.of(
cursor1,
dummyCursor1,
dummyCursor2,
dummyCursor3
));
List<Object> tempList = new ArrayList<>();
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b"), tempList);
// Check if reset() works after exhausting the cursor
concatCursor.reset();
tempList.clear();
for (int i = 0; i < 1; ++i) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a"), tempList);
// Check if reset() works from the middle
concatCursor.reset();
tempList.clear();
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b"), tempList);
}
@Test
public void testConcatCursorWhenMultipleEmptyCursorsAtTheMiddle()
{
Cursor dummyCursor1 = new ListCursor(new ArrayList<>());
Cursor dummyCursor2 = new ListCursor(new ArrayList<>());
Cursor dummyCursor3 = new ListCursor(new ArrayList<>());
Cursor cursor1 = new ListCursor(ImmutableList.of("a"));
Cursor cursor2 = new ListCursor(ImmutableList.of("b"));
Cursor concatCursor = new ConcatCursor(ImmutableList.of(
cursor1,
dummyCursor1,
dummyCursor2,
dummyCursor3,
cursor2
));
List<Object> tempList = new ArrayList<>();
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b"), tempList);
// Check if reset() works after exhausting the cursor
concatCursor.reset();
tempList.clear();
for (int i = 0; i < 1; ++i) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a"), tempList);
// Check if reset() works from the middle
concatCursor.reset();
tempList.clear();
while (!concatCursor.isDone()) {
tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject());
concatCursor.advance();
}
Assert.assertEquals(ImmutableList.of("a", "b"), tempList);
}
}

View File

@ -20,15 +20,24 @@
package org.apache.druid.query.scan;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.frame.allocation.HeapMemoryAllocator;
import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.DefaultGenericQueryMetricsFactory;
import org.apache.druid.query.Druids;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.query.FrameSignaturePair;
import org.apache.druid.query.QueryToolChestTestHelper;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.nested.NestedDataComplexTypeSerde;
import org.apache.druid.segment.nested.StructuredData;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.junit.Assert;
import org.junit.Test;
@ -40,12 +49,28 @@ import java.util.Map;
public class ScanQueryQueryToolChestTest
{
static {
NullHandling.initializeForTests();
ComplexMetrics.registerSerde(NestedDataComplexTypeSerde.TYPE_NAME, NestedDataComplexTypeSerde.INSTANCE);
}
// Expected results for the resultsAsArrays test methods.
private static final List<Object[]> ARRAY_RESULTS = ImmutableList.of(
private static final List<Object[]> ARRAY_RESULTS_1 = ImmutableList.of(
new Object[]{null, 3.2},
new Object[]{"x", "y"}
);
private static final List<Object[]> ARRAY_RESULTS_2 = ImmutableList.of(
new Object[]{"str1", 3.2},
new Object[]{"str2", 3.3}
);
private static final List<Object[]> ARRAY_RESULTS_3 = ImmutableList.of(
new Object[]{3.4, "str3"},
new Object[]{3.5, "str4"}
);
private final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest(
new ScanQueryConfig(),
new DefaultGenericQueryMetricsFactory()
@ -121,7 +146,7 @@ public class ScanQueryQueryToolChestTest
QueryToolChestTestHelper.assertArrayResultsEquals(
ImmutableList.of(new Object[]{}, new Object[]{}),
toolChest.resultsAsArrays(scanQuery, makeResults(ScanQuery.ResultFormat.RESULT_FORMAT_LIST))
toolChest.resultsAsArrays(scanQuery, makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_LIST))
);
}
@ -137,7 +162,7 @@ public class ScanQueryQueryToolChestTest
QueryToolChestTestHelper.assertArrayResultsEquals(
ImmutableList.of(new Object[]{}, new Object[]{}),
toolChest.resultsAsArrays(scanQuery, makeResults(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST))
toolChest.resultsAsArrays(scanQuery, makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST))
);
}
@ -153,8 +178,8 @@ public class ScanQueryQueryToolChestTest
.build();
QueryToolChestTestHelper.assertArrayResultsEquals(
ARRAY_RESULTS,
toolChest.resultsAsArrays(scanQuery, makeResults(ScanQuery.ResultFormat.RESULT_FORMAT_LIST))
ARRAY_RESULTS_1,
toolChest.resultsAsArrays(scanQuery, makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_LIST))
);
}
@ -170,22 +195,185 @@ public class ScanQueryQueryToolChestTest
.build();
QueryToolChestTestHelper.assertArrayResultsEquals(
ARRAY_RESULTS,
toolChest.resultsAsArrays(scanQuery, makeResults(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST))
ARRAY_RESULTS_1,
toolChest.resultsAsArrays(scanQuery, makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST))
);
}
@Test
public void test_resultsAsFrames_batchingWorksAsExpectedWithDistinctColumnTypes()
{
final ScanQuery scanQuery =
Druids.newScanQueryBuilder()
.dataSource("foo")
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000"))))
.columns("foo", "bar", "foo2", "bar2", "foo3", "bar3")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)
.build();
List<FrameSignaturePair> frames =
toolChest.resultsAsFrames(
scanQuery,
Sequences.concat(makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_LIST), results2(), results3()),
new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
true
).get().toList();
Assert.assertEquals(3, frames.size());
RowSignature resultRowSignature = RowSignature.builder()
.add("foo", null)
.add("bar", null)
.add("foo2", null)
.add("bar2", null)
.add("foo3", null)
.add("bar3", null)
.build();
Sequence<Object[]> rows = new FrameBasedInlineDataSource(frames, resultRowSignature).getRowsAsSequence();
QueryToolChestTestHelper.assertArrayResultsEquals(
ImmutableList.of(
new Object[]{null, StructuredData.wrap(3.2), null, null, null, null},
new Object[]{StructuredData.wrap("x"), StructuredData.wrap("y"), null, null, null, null},
new Object[]{null, null, "str1", 3.2, null, null},
new Object[]{null, null, "str2", 3.3, null, null},
new Object[]{null, null, null, null, 3.4, "str3"},
new Object[]{null, null, null, null, 3.5, "str4"}
),
rows
);
}
@Test
public void test_resultsAsFrames_batchingWorksAsExpectedWithMixedColumnTypes()
{
final ScanQuery scanQuery =
Druids.newScanQueryBuilder()
.dataSource("foo")
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000"))))
.columns("foo", "bar", "foo2", "bar2", "foo3", "bar3")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)
.build();
List<FrameSignaturePair> frames =
toolChest.resultsAsFrames(
scanQuery,
Sequences.concat(
results2(),
makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_LIST),
makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_LIST),
results3(),
results2(),
results2(),
results3()
),
new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
true
).get().toList();
Assert.assertEquals(5, frames.size());
RowSignature resultRowSignature = RowSignature.builder()
.add("foo", null)
.add("bar", null)
.add("foo2", null)
.add("bar2", null)
.add("foo3", null)
.add("bar3", null)
.build();
Sequence<Object[]> rows = new FrameBasedInlineDataSource(frames, resultRowSignature).getRowsAsSequence();
QueryToolChestTestHelper.assertArrayResultsEquals(
ImmutableList.of(
// results2
new Object[]{null, null, "str1", 3.2, null, null},
new Object[]{null, null, "str2", 3.3, null, null},
// results1
new Object[]{null, StructuredData.wrap(3.2), null, null, null, null},
new Object[]{StructuredData.wrap("x"), StructuredData.wrap("y"), null, null, null, null},
// results1
new Object[]{null, StructuredData.wrap(3.2), null, null, null, null},
new Object[]{StructuredData.wrap("x"), StructuredData.wrap("y"), null, null, null, null},
// results3
new Object[]{null, null, null, null, 3.4, "str3"},
new Object[]{null, null, null, null, 3.5, "str4"},
// results2
new Object[]{null, null, "str1", 3.2, null, null},
new Object[]{null, null, "str2", 3.3, null, null},
// results2
new Object[]{null, null, "str1", 3.2, null, null},
new Object[]{null, null, "str2", 3.3, null, null},
// results3
new Object[]{null, null, null, null, 3.4, "str3"},
new Object[]{null, null, null, null, 3.5, "str4"}
),
rows
);
}
@Test
public void test_resultsAsFrames_batchingWorksAsExpectedWithSameColumnTypes()
{
final ScanQuery scanQuery =
Druids.newScanQueryBuilder()
.dataSource("foo")
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000"))))
.columns("foo", "bar", "foo2", "bar2", "foo3", "bar3")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)
.build();
List<FrameSignaturePair> frames =
toolChest.resultsAsFrames(
scanQuery,
Sequences.concat(results2(), results2()),
new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
true
).get().toList();
Assert.assertEquals(1, frames.size());
RowSignature resultRowSignature = RowSignature.builder()
.add("foo", null)
.add("bar", null)
.add("foo2", null)
.add("bar2", null)
.add("foo3", null)
.add("bar3", null)
.build();
Sequence<Object[]> rows = new FrameBasedInlineDataSource(frames, resultRowSignature).getRowsAsSequence();
QueryToolChestTestHelper.assertArrayResultsEquals(
ImmutableList.of(
new Object[]{null, null, "str1", 3.2, null, null},
new Object[]{null, null, "str2", 3.3, null, null},
new Object[]{null, null, "str1", 3.2, null, null},
new Object[]{null, null, "str2", 3.3, null, null}
),
rows
);
}
/**
* Returns results that are a single ScanResultValue with two rows, each row having columns "foo" and "bar".
*/
private static Sequence<ScanResultValue> makeResults(final ScanQuery.ResultFormat resultFormat)
private static Sequence<ScanResultValue> makeResults1(final ScanQuery.ResultFormat resultFormat)
{
final List<Object> rows = new ArrayList<>();
// Generate rows in the manner of ScanQueryEngine.
switch (resultFormat) {
case RESULT_FORMAT_LIST:
ARRAY_RESULTS.forEach(arr -> {
ARRAY_RESULTS_1.forEach(arr -> {
final Map<String, Object> m = new HashMap<>();
m.put("foo", arr[0]);
m.put("bar", arr[1]);
@ -193,7 +381,7 @@ public class ScanQueryQueryToolChestTest
});
break;
case RESULT_FORMAT_COMPACTED_LIST:
ARRAY_RESULTS.forEach(arr -> rows.add(Arrays.asList(arr)));
ARRAY_RESULTS_1.forEach(arr -> rows.add(Arrays.asList(arr)));
break;
default:
throw new ISE("Cannot generate resultFormat '%s'", resultFormat);
@ -209,4 +397,66 @@ public class ScanQueryQueryToolChestTest
)
);
}
/**
* Returns results that are a single ScanResultValue with two rows, each row having columns "foo2" and "bar2". This
* generates results in the format of {@link ScanQuery.ResultFormat#RESULT_FORMAT_LIST}
*/
private static Sequence<ScanResultValue> results2()
{
final List<Object> rows = new ArrayList<>();
ARRAY_RESULTS_2.forEach(arr -> {
final Map<String, Object> m = new HashMap<>();
m.put("foo2", arr[0]);
m.put("bar2", arr[1]);
rows.add(m);
});
RowSignature.Builder rowSignatureBuilder = RowSignature.builder();
rowSignatureBuilder.add("foo2", ColumnType.STRING);
rowSignatureBuilder.add("bar2", ColumnType.DOUBLE);
return Sequences.simple(
ImmutableList.of(
new ScanResultValue(
null,
ImmutableList.of("foo2", "bar2"),
rows,
rowSignatureBuilder.build()
)
)
);
}
/**
* Returns results that are a single ScanResultValue with two rows, each row having columns "foo3" and "bar3". This
* generates results in the format of {@link ScanQuery.ResultFormat#RESULT_FORMAT_LIST}
*/
private static Sequence<ScanResultValue> results3()
{
final List<Object> rows = new ArrayList<>();
ARRAY_RESULTS_3.forEach(arr -> {
final Map<String, Object> m = new HashMap<>();
m.put("foo3", arr[0]);
m.put("bar3", arr[1]);
rows.add(m);
});
RowSignature.Builder rowSignatureBuilder = RowSignature.builder();
rowSignatureBuilder.add("foo3", ColumnType.DOUBLE);
rowSignatureBuilder.add("bar3", ColumnType.STRING);
return Sequences.simple(
ImmutableList.of(
new ScanResultValue(
null,
ImmutableList.of("foo3", "bar3"),
rows,
rowSignatureBuilder.build()
)
)
);
}
}

View File

@ -0,0 +1,372 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.segment.join.table;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import it.unimi.dsi.fastutil.ints.IntBidirectionalIterator;
import it.unimi.dsi.fastutil.ints.IntSortedSet;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.frame.Frame;
import org.apache.druid.frame.FrameType;
import org.apache.druid.frame.allocation.HeapMemoryAllocator;
import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory;
import org.apache.druid.frame.segment.FrameCursorUtils;
import org.apache.druid.frame.write.FrameWriterFactory;
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.query.FrameSignaturePair;
import org.apache.druid.query.IterableRowsCursorHelper;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
public class FrameBasedIndexedTableTest extends InitializedNullHandlingTest
{
private static final String STRING_COL_1 = "market";
private static final String LONG_COL_1 = "longNumericNull";
private static final String DOUBLE_COL_1 = "doubleNumericNull";
private static final String FLOAT_COL_1 = "floatNumericNull";
private static final String STRING_COL_2 = "partial_null_column";
private static final String MULTI_VALUE_COLUMN = "placementish";
private static final String NON_INDEXED_STRING_COL = "nonIndexedString";
private static final String NON_INDEXED_LONG_COL = "nonIndexedNumeric";
private static final String NON_INDEXED_DOUBLE_COL = "nonIndexedDouble";
private static final String NON_INDEXED_FLOAT_COL = "nonIndexedFloat";
private static final String DIM_NOT_EXISTS = "DIM_NOT_EXISTS";
private static final List<Object[]> DATASOURCE_ROWS =
ImmutableList.<Object[]>builder()
.add(
new Object[]{
"spot",
1L,
null,
3.1f,
"preferred",
new Object[]{"val1", "val2"},
"spot",
1L,
null,
3.1f
})
.add(new Object[]{
"total_market",
1L,
1.2d,
3.2f,
null,
new Object[]{"val1", "val2"},
"total_market",
1L,
1.2d,
3.2f
})
.add(new Object[]{
"spot",
2L,
1.3d,
3.1f,
"preferred",
new Object[]{"val1", "val2"},
"spot",
2L,
1.3d,
3.1f
})
.add(new Object[]{
"upfront",
1L,
1.5d,
3.5f,
"preferred",
new Object[]{"val1", "val2"},
"upfront",
1L,
1.5d,
3.5f
})
.add(new Object[]{
"total_market",
null,
1.1d,
3.1f,
null,
new Object[]{"val1", "val2"},
"total_market",
null,
1.1d,
3.1f
})
.add(new Object[]{
"upfront",
2L,
1.5d,
null,
"preferred",
new Object[]{"val1", "val2"},
"upfront",
2L,
1.5d,
null
})
.add(new Object[]{
"upfront",
4L,
1.1d,
3.9f,
"preferred",
new Object[]{"val1", "val2"},
"upfront",
4L,
1.1d,
3.9f
})
.add(new Object[]{
"total_market",
1L,
1.7d,
3.8f,
"preferred",
new Object[]{"val1", "val2"},
"total_market",
1L,
1.7d,
3.8f
})
.add(new Object[]{
"spot",
5L,
1.8d,
3.1f,
null,
new Object[]{"val1", "val2"},
"spot",
5L,
1.8d,
3.1f
})
.build();
private static final RowSignature ROW_SIGNATURE =
RowSignature.builder()
.add(STRING_COL_1, ColumnType.STRING)
.add(LONG_COL_1, ColumnType.LONG)
.add(DOUBLE_COL_1, ColumnType.DOUBLE)
.add(FLOAT_COL_1, ColumnType.FLOAT)
.add(STRING_COL_2, ColumnType.STRING)
.add(MULTI_VALUE_COLUMN, ColumnType.STRING_ARRAY)
.add(NON_INDEXED_STRING_COL, ColumnType.STRING)
.add(NON_INDEXED_LONG_COL, ColumnType.LONG)
.add(NON_INDEXED_DOUBLE_COL, ColumnType.DOUBLE)
.add(NON_INDEXED_FLOAT_COL, ColumnType.FLOAT)
.build();
private static final Set<String> KEY_COLUMNS = ImmutableSet.<String>builder()
.add(STRING_COL_1)
.add(STRING_COL_2)
.add(LONG_COL_1)
.add(DOUBLE_COL_1)
.add(FLOAT_COL_1)
.add(MULTI_VALUE_COLUMN)
.add(DIM_NOT_EXISTS)
.build();
private FrameBasedInlineDataSource dataSource;
private FrameBasedIndexedTable frameBasedIndexedTable;
@Before
public void setup()
{
Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(DATASOURCE_ROWS, ROW_SIGNATURE);
FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
FrameType.COLUMNAR,
new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
ROW_SIGNATURE,
new ArrayList<>()
);
Frame frame = Iterables.getOnlyElement(FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory).toList());
dataSource = new FrameBasedInlineDataSource(
ImmutableList.of(new FrameSignaturePair(frame, ROW_SIGNATURE)),
ROW_SIGNATURE
);
frameBasedIndexedTable = new FrameBasedIndexedTable(dataSource, KEY_COLUMNS, "test");
}
@Test
public void testInitShouldGenerateCorrectTable()
{
Assert.assertEquals(9, frameBasedIndexedTable.numRows());
}
@Test
public void testStringKeyColumn()
{
final String[] vals = new String[]{"spot", "total_market", "upfront"};
checkIndexAndReader(STRING_COL_1, vals);
}
@Test
public void testNullableStringKeyColumn()
{
final String[] vals = new String[]{null, "preferred"};
checkIndexAndReader(STRING_COL_2, vals);
}
@Test
public void testMultiValueStringKeyColumn()
{
final Object[] nonMatchingVals = new Object[]{ImmutableList.of("a", "preferred")};
checkIndexAndReader(MULTI_VALUE_COLUMN, new Object[0], nonMatchingVals);
}
@Test
public void testLongKeyColumn()
{
final Long[] vals = new Long[]{NullHandling.replaceWithDefault() ? 0L : null, 1L, 2L, 4L, 5L};
checkIndexAndReader(LONG_COL_1, vals);
}
@Test
public void testFloatKeyColumn()
{
final Float[] vals = new Float[]{NullHandling.replaceWithDefault() ? 0.0f : null, 3.1f, 3.2f, 3.5f, 3.8f, 3.9f};
checkIndexAndReader(FLOAT_COL_1, vals);
}
@Test
public void testDoubleKeyColumn()
{
final Double[] vals = new Double[]{
NullHandling.replaceWithDefault() ? 0.0 : null, 1.1d, 1.2d, 1.3d, 1.5d, 1.7d, 1.8d
};
checkIndexAndReader(DOUBLE_COL_1, vals);
}
@Test
public void testStringNonKeyColumn()
{
checkNonIndexedReader(NON_INDEXED_STRING_COL);
}
@Test
public void testLongNonKeyColumn()
{
checkNonIndexedReader(NON_INDEXED_LONG_COL);
}
@Test
public void testFloatNonKeyColumn()
{
checkNonIndexedReader(NON_INDEXED_FLOAT_COL);
}
@Test
public void testDoubleNonKeyColumn()
{
checkNonIndexedReader(NON_INDEXED_DOUBLE_COL);
}
@Test
public void testIsCacheable()
{
Assert.assertFalse(frameBasedIndexedTable.isCacheable());
}
private void checkIndexAndReader(String columnName, Object[] vals)
{
checkIndexAndReader(columnName, vals, new Object[0]);
}
private void checkIndexAndReader(String columnName, Object[] vals, Object[] nonmatchingVals)
{
checkColumnReader(columnName);
try (final Closer closer = Closer.create()) {
final int columnIndex = ROW_SIGNATURE.indexOf(columnName);
final IndexedTable.Reader reader = frameBasedIndexedTable.columnReader(columnIndex);
closer.register(reader);
final IndexedTable.Index valueIndex = frameBasedIndexedTable.columnIndex(columnIndex);
for (Object val : vals) {
final IntSortedSet valIndex = valueIndex.find(val);
if (val == null) {
Assert.assertEquals(0, valIndex.size());
} else {
Assert.assertTrue(valIndex.size() > 0);
final IntBidirectionalIterator rowIterator = valIndex.iterator();
while (rowIterator.hasNext()) {
Assert.assertEquals(val, reader.read(rowIterator.nextInt()));
}
}
}
for (Object val : nonmatchingVals) {
final IntSortedSet valIndex = valueIndex.find(val);
Assert.assertEquals(0, valIndex.size());
}
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
private void checkNonIndexedReader(String columnName)
{
// make sure it doesn't have an index since it isn't a key column
checkColumnReader(columnName);
int columnIndex = ROW_SIGNATURE.indexOf(columnName);
try {
Assert.assertNull(frameBasedIndexedTable.columnIndex(columnIndex));
}
catch (IAE iae) {
Assert.assertEquals(StringUtils.format("Column[%d] is not a key column", columnIndex), iae.getMessage());
}
}
private void checkColumnReader(String columnName)
{
int numRows = DATASOURCE_ROWS.size();
int columnNumber = ROW_SIGNATURE.indexOf(columnName);
IndexedTable.Reader reader = frameBasedIndexedTable.columnReader(columnNumber);
List<Object[]> originalRows = dataSource.getRowsAsSequence().toList();
for (int i = 0; i < numRows; ++i) {
Object original = originalRows.get(i)[columnNumber];
Assert.assertEquals(original, reader.read(i));
}
}
}

View File

@ -27,10 +27,12 @@ import com.google.inject.Scopes;
import com.google.inject.multibindings.MapBinder;
import com.google.inject.multibindings.Multibinder;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.query.GlobalTableDataSource;
import org.apache.druid.query.InlineDataSource;
import org.apache.druid.query.LookupDataSource;
import org.apache.druid.segment.join.BroadcastTableJoinableFactory;
import org.apache.druid.segment.join.FrameBasedInlineJoinableFactory;
import org.apache.druid.segment.join.InlineJoinableFactory;
import org.apache.druid.segment.join.JoinableFactory;
import org.apache.druid.segment.join.LookupJoinableFactory;
@ -50,6 +52,7 @@ public class JoinableFactoryModule implements Module
static final Map<Class<? extends DataSource>, Class<? extends JoinableFactory>> FACTORY_MAPPINGS =
ImmutableMap.of(
InlineDataSource.class, InlineJoinableFactory.class,
FrameBasedInlineDataSource.class, FrameBasedInlineJoinableFactory.class,
LookupDataSource.class, LookupJoinableFactory.class,
GlobalTableDataSource.class, BroadcastTableJoinableFactory.class
);

View File

@ -26,8 +26,10 @@ import com.google.inject.Module;
import com.google.inject.Scopes;
import com.google.inject.multibindings.MapBinder;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.query.InlineDataSource;
import org.apache.druid.query.LookupDataSource;
import org.apache.druid.segment.FrameBasedInlineSegmentWrangler;
import org.apache.druid.segment.InlineSegmentWrangler;
import org.apache.druid.segment.LookupSegmentWrangler;
import org.apache.druid.segment.MapSegmentWrangler;
@ -47,6 +49,7 @@ public class SegmentWranglerModule implements Module
static final Map<Class<? extends DataSource>, Class<? extends SegmentWrangler>> WRANGLER_MAPPINGS =
ImmutableMap.of(
InlineDataSource.class, InlineSegmentWrangler.class,
FrameBasedInlineDataSource.class, FrameBasedInlineSegmentWrangler.class,
LookupDataSource.class, LookupSegmentWrangler.class
);

View File

@ -0,0 +1,55 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.segment;
import org.apache.druid.frame.read.FrameReader;
import org.apache.druid.frame.segment.FrameSegment;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
public class FrameBasedInlineSegmentWrangler implements SegmentWrangler
{
private static final String SEGMENT_ID = "inline";
@Override
public Iterable<Segment> getSegmentsForIntervals(
DataSource dataSource,
Iterable<Interval> intervals
)
{
final FrameBasedInlineDataSource frameBasedInlineDataSource = (FrameBasedInlineDataSource) dataSource;
return () -> frameBasedInlineDataSource
.getFrames()
.stream()
.<Segment>map(
frameSignaturePair -> new FrameSegment(
frameSignaturePair.getFrame(),
FrameReader.create(frameSignaturePair.getRowSignature()),
SegmentId.dummy(SEGMENT_ID)
)
)
.iterator();
}
}

View File

@ -28,13 +28,18 @@ import org.apache.druid.client.CachingClusteredClient;
import org.apache.druid.client.DirectDruidClient;
import org.apache.druid.client.cache.Cache;
import org.apache.druid.client.cache.CacheConfig;
import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.FluentQueryRunnerBuilder;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.query.FrameSignaturePair;
import org.apache.druid.query.GlobalTableDataSource;
import org.apache.druid.query.InlineDataSource;
import org.apache.druid.query.PostProcessingOperator;
@ -65,9 +70,12 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Queue;
import java.util.Stack;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
/**
@ -81,6 +89,10 @@ import java.util.stream.Collectors;
*/
public class ClientQuerySegmentWalker implements QuerySegmentWalker
{
private static final Logger log = new Logger(ClientQuerySegmentWalker.class);
private static final int FRAME_SIZE = 8_000_000;
private final ServiceEmitter emitter;
private final QuerySegmentWalker clusterClient;
private final QuerySegmentWalker localClient;
@ -163,12 +175,19 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
final DataSource freeTradeDataSource = globalizeIfPossible(newQuery.getDataSource());
// do an inlining dry run to see if any inlining is necessary, without actually running the queries.
final int maxSubqueryRows = query.context().getMaxSubqueryRows(serverConfig.getMaxSubqueryRows());
final long maxSubqueryMemory = query.context().getMaxSubqueryMemoryBytes(serverConfig.getMaxSubqueryBytes());
final boolean useNestedForUnknownTypeInSubquery = query.context()
.isUseNestedForUnknownTypeInSubquery(serverConfig.isuseNestedForUnknownTypeInSubquery());
final DataSource inlineDryRun = inlineIfNecessary(
freeTradeDataSource,
toolChest,
new AtomicInteger(),
new AtomicLong(),
new AtomicBoolean(false),
maxSubqueryRows,
maxSubqueryMemory,
useNestedForUnknownTypeInSubquery,
true
);
@ -179,15 +198,21 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
}
// Now that we know the structure is workable, actually do the inlining (if necessary).
newQuery = newQuery.withDataSource(
inlineIfNecessary(
freeTradeDataSource,
toolChest,
new AtomicInteger(),
maxSubqueryRows,
false
)
AtomicLong memoryLimitAcc = new AtomicLong(0);
DataSource maybeInlinedDataSource = inlineIfNecessary(
freeTradeDataSource,
toolChest,
new AtomicInteger(),
memoryLimitAcc,
new AtomicBoolean(false),
maxSubqueryRows,
maxSubqueryMemory,
useNestedForUnknownTypeInSubquery,
false
);
newQuery = newQuery.withDataSource(maybeInlinedDataSource);
log.debug("Memory used by subqueries of query [%s] is [%d]", query, memoryLimitAcc.get());
if (canRunQueryUsingLocalWalker(newQuery)) {
// No need to decorate since LocalQuerySegmentWalker does its own.
@ -306,6 +331,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
* @param subqueryRowLimitAccumulator an accumulator for tracking the number of accumulated rows in all subqueries
* for a particular master query
* @param maxSubqueryRows Max rows that all the subqueries generated by a master query can have, combined
* @param useNestedForUnknownTypeInSubquery if true, then the null types are replaced by nested types while converting the results to frames
* @param dryRun if true, does not actually execute any subqueries, but will inline empty result sets.
*/
@SuppressWarnings({"rawtypes", "unchecked"}) // Subquery, toolchest, runner handling all use raw types
@ -313,7 +339,11 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
final DataSource dataSource,
@Nullable final QueryToolChest toolChestIfOutermost,
final AtomicInteger subqueryRowLimitAccumulator,
final AtomicLong subqueryMemoryLimitAccumulator,
final AtomicBoolean cannotMaterializeToFrames,
final int maxSubqueryRows,
final long maxSubqueryMemory,
final boolean useNestedForUnknownTypeInSubquery,
final boolean dryRun
)
{
@ -336,7 +366,17 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
if (current instanceof QueryDataSource) {
throw new ISE("Got a QueryDataSource[%s], should've walked it away in the loop above.", current);
}
current = inlineIfNecessary(current, null, subqueryRowLimitAccumulator, maxSubqueryRows, dryRun);
current = inlineIfNecessary(
current,
null,
subqueryRowLimitAccumulator,
subqueryMemoryLimitAccumulator,
cannotMaterializeToFrames,
maxSubqueryRows,
maxSubqueryMemory,
useNestedForUnknownTypeInSubquery,
dryRun
);
while (!stack.isEmpty()) {
current = stack.pop().withChildren(Collections.singletonList(current));
@ -350,7 +390,17 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
} else {
// Something happened during inlining that means the toolchest is no longer able to handle this subquery.
// We need to consider inlining it.
return inlineIfNecessary(current, toolChestIfOutermost, subqueryRowLimitAccumulator, maxSubqueryRows, dryRun);
return inlineIfNecessary(
current,
toolChestIfOutermost,
subqueryRowLimitAccumulator,
subqueryMemoryLimitAccumulator,
cannotMaterializeToFrames,
maxSubqueryRows,
maxSubqueryMemory,
useNestedForUnknownTypeInSubquery,
dryRun
);
}
} else if (canRunQueryUsingLocalWalker(subQuery) || canRunQueryUsingClusterWalker(subQuery)) {
// Subquery needs to be inlined. Assign it a subquery id and run it.
@ -372,7 +422,11 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
queryResults,
warehouse.getToolChest(subQuery),
subqueryRowLimitAccumulator,
maxSubqueryRows
subqueryMemoryLimitAccumulator,
cannotMaterializeToFrames,
maxSubqueryRows,
maxSubqueryMemory,
useNestedForUnknownTypeInSubquery
);
} else {
// Cannot inline subquery. Attempt to inline one level deeper, and then try again.
@ -383,14 +437,22 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
Iterables.getOnlyElement(dataSource.getChildren()),
null,
subqueryRowLimitAccumulator,
subqueryMemoryLimitAccumulator,
cannotMaterializeToFrames,
maxSubqueryRows,
maxSubqueryMemory,
useNestedForUnknownTypeInSubquery,
dryRun
)
)
),
toolChestIfOutermost,
subqueryRowLimitAccumulator,
subqueryMemoryLimitAccumulator,
cannotMaterializeToFrames,
maxSubqueryRows,
maxSubqueryMemory,
useNestedForUnknownTypeInSubquery,
dryRun
);
}
@ -399,7 +461,17 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
return dataSource.withChildren(
dataSource.getChildren()
.stream()
.map(child -> inlineIfNecessary(child, null, subqueryRowLimitAccumulator, maxSubqueryRows, dryRun))
.map(child -> inlineIfNecessary(
child,
null,
subqueryRowLimitAccumulator,
subqueryMemoryLimitAccumulator,
cannotMaterializeToFrames,
maxSubqueryRows,
maxSubqueryMemory,
useNestedForUnknownTypeInSubquery,
dryRun
))
.collect(Collectors.toList())
);
}
@ -560,7 +632,137 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
* If zero, this method will throw an error immediately.
* @throws ResourceLimitExceededException if the limit is exceeded
*/
private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSource(
private static <T, QueryType extends Query<T>> DataSource toInlineDataSource(
final QueryType query,
final Sequence<T> results,
final QueryToolChest<T, QueryType> toolChest,
final AtomicInteger limitAccumulator,
final AtomicLong memoryLimitAccumulator,
final AtomicBoolean cannotMaterializeToFrames,
final int limit,
long memoryLimit,
boolean useNestedForUnknownTypeInSubquery
)
{
final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
DataSource dataSource;
switch (ClientQuerySegmentWalkerUtils.getLimitType(memoryLimit, cannotMaterializeToFrames.get())) {
case ROW_LIMIT:
if (limitAccumulator.get() >= rowLimitToUse) {
throw ResourceLimitExceededException.withMessage(
"Cannot issue the query, subqueries generated results beyond maximum[%d] rows",
rowLimitToUse
);
}
dataSource = materializeResultsAsArray(
query,
results,
toolChest,
limitAccumulator,
limit
);
break;
case MEMORY_LIMIT:
if (memoryLimitAccumulator.get() >= memoryLimit) {
throw ResourceLimitExceededException.withMessage(
"Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
memoryLimit
);
}
Optional<DataSource> maybeDataSource = materializeResultsAsFrames(
query,
results,
toolChest,
limitAccumulator,
memoryLimitAccumulator,
memoryLimit,
useNestedForUnknownTypeInSubquery
);
if (!maybeDataSource.isPresent()) {
cannotMaterializeToFrames.set(true);
// Check if the previous row limit accumulator has exceeded the memory results
if (memoryLimitAccumulator.get() >= memoryLimit) {
throw ResourceLimitExceededException.withMessage(
"Cannot issue the query, subqueries generated results beyond maximum[%d] bytes",
memoryLimit
);
}
dataSource = materializeResultsAsArray(
query,
results,
toolChest,
limitAccumulator,
limit
);
} else {
dataSource = maybeDataSource.get();
}
break;
default:
throw new IAE("Only row based and memory based limiting is supported");
}
return dataSource;
}
/**
* This method materializes the query results as Frames. The method defaults back to materializing as rows in case
* one cannot materialize the results as frames
*/
private static <T, QueryType extends Query<T>> Optional<DataSource> materializeResultsAsFrames(
final QueryType query,
final Sequence<T> results,
final QueryToolChest<T, QueryType> toolChest,
final AtomicInteger limitAccumulator,
final AtomicLong memoryLimitAccumulator,
long memoryLimit,
boolean useNestedForUnknownTypeInSubquery
)
{
Optional<Sequence<FrameSignaturePair>> framesOptional;
try {
framesOptional = toolChest.resultsAsFrames(
query,
results,
new ArenaMemoryAllocatorFactory(FRAME_SIZE),
useNestedForUnknownTypeInSubquery
);
}
catch (Exception e) {
log.debug(e, "Unable to materialize the results as frames due to an unhandleable exception "
+ "while conversion. Defaulting to materializing the results as rows");
return Optional.empty();
}
if (!framesOptional.isPresent()) {
log.debug("Unable to materialize the results as frames. Defaulting to materializing the results as rows");
return Optional.empty();
}
Sequence<FrameSignaturePair> frames = framesOptional.get();
List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
frames.forEach(
frame -> {
limitAccumulator.addAndGet(frame.getFrame().numRows());
if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) {
throw ResourceLimitExceededException.withMessage(
"Subquery generated results beyond maximum[%d] bytes",
memoryLimit
);
}
frameSignaturePairs.add(frame);
}
);
return Optional.of(new FrameBasedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query)));
}
/**
* This method materializes the query results as {@code List<Objects[]>}
*/
private static <T, QueryType extends Query<T>> DataSource materializeResultsAsArray(
final QueryType query,
final Sequence<T> results,
final QueryToolChest<T, QueryType> toolChest,
@ -568,15 +770,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
final int limit
)
{
final int limitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
if (limitAccumulator.get() >= limitToUse) {
throw ResourceLimitExceededException.withMessage(
"Cannot issue subquery, maximum[%d] reached",
limitToUse
);
}
final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit;
final RowSignature signature = toolChest.resultArraySignature(query);
final ArrayList<Object[]> resultList = new ArrayList<>();
@ -584,17 +778,16 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
toolChest.resultsAsArrays(query, results).accumulate(
resultList,
(acc, in) -> {
if (limitAccumulator.getAndIncrement() >= limitToUse) {
if (limitAccumulator.getAndIncrement() >= rowLimitToUse) {
throw ResourceLimitExceededException.withMessage(
"Subquery generated results beyond maximum[%d]",
limitToUse
"Subquery generated results beyond maximum[%d] rows",
rowLimitToUse
);
}
acc.add(in);
return acc;
}
);
return InlineDataSource.fromIterable(resultList, signature);
}

View File

@ -0,0 +1,40 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.server;
public class ClientQuerySegmentWalkerUtils
{
public enum SubqueryResultLimit
{
ROW_LIMIT,
MEMORY_LIMIT
}
public static SubqueryResultLimit getLimitType(long memoryLimitBytes, boolean cannotMaterializeToFrames)
{
if (cannotMaterializeToFrames) {
return SubqueryResultLimit.ROW_LIMIT;
}
if (memoryLimitBytes > 0) {
return SubqueryResultLimit.MEMORY_LIMIT;
}
return SubqueryResultLimit.ROW_LIMIT;
}
}

View File

@ -43,6 +43,9 @@ import java.util.zip.Deflater;
public class ServerConfig
{
public static final int DEFAULT_GZIP_INFLATE_BUFFER_SIZE = 4096;
public static final long DEFAULT_MAX_SUBQUERY_BYTES = -1L;
private static final boolean DEFAULT_USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY = false;
/**
* The ServerConfig is normally created using {@link org.apache.druid.guice.JsonConfigProvider} binding.
@ -57,6 +60,8 @@ public class ServerConfig
long defaultQueryTimeout,
long maxScatterGatherBytes,
int maxSubqueryRows,
long maxSubqueryBytes,
boolean useNestedForUnknownTypeInSubquery,
long maxQueryTimeout,
int maxRequestHeaderSize,
@NotNull Period gracefulShutdownTimeout,
@ -78,6 +83,8 @@ public class ServerConfig
this.defaultQueryTimeout = defaultQueryTimeout;
this.maxScatterGatherBytes = HumanReadableBytes.valueOf(maxScatterGatherBytes);
this.maxSubqueryRows = maxSubqueryRows;
this.maxSubqueryBytes = maxSubqueryBytes;
this.useNestedForUnknownTypeInSubquery = useNestedForUnknownTypeInSubquery;
this.maxQueryTimeout = maxQueryTimeout;
this.maxRequestHeaderSize = maxRequestHeaderSize;
this.gracefulShutdownTimeout = gracefulShutdownTimeout;
@ -125,6 +132,12 @@ public class ServerConfig
@Min(1)
private int maxSubqueryRows = 100000;
@JsonProperty
private long maxSubqueryBytes = DEFAULT_MAX_SUBQUERY_BYTES;
@JsonProperty
private boolean useNestedForUnknownTypeInSubquery = DEFAULT_USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY;
@JsonProperty
@Min(1)
private long maxQueryTimeout = Long.MAX_VALUE;
@ -204,6 +217,16 @@ public class ServerConfig
return maxSubqueryRows;
}
public long getMaxSubqueryBytes()
{
return maxSubqueryBytes;
}
public boolean isuseNestedForUnknownTypeInSubquery()
{
return useNestedForUnknownTypeInSubquery;
}
public long getMaxQueryTimeout()
{
return maxQueryTimeout;
@ -280,6 +303,8 @@ public class ServerConfig
enableRequestLimit == that.enableRequestLimit &&
defaultQueryTimeout == that.defaultQueryTimeout &&
maxSubqueryRows == that.maxSubqueryRows &&
maxSubqueryBytes == that.maxSubqueryBytes &&
useNestedForUnknownTypeInSubquery == that.useNestedForUnknownTypeInSubquery &&
maxQueryTimeout == that.maxQueryTimeout &&
maxRequestHeaderSize == that.maxRequestHeaderSize &&
inflateBufferSize == that.inflateBufferSize &&
@ -307,6 +332,8 @@ public class ServerConfig
defaultQueryTimeout,
maxScatterGatherBytes,
maxSubqueryRows,
maxSubqueryBytes,
useNestedForUnknownTypeInSubquery,
maxQueryTimeout,
maxRequestHeaderSize,
gracefulShutdownTimeout,
@ -333,6 +360,8 @@ public class ServerConfig
", defaultQueryTimeout=" + defaultQueryTimeout +
", maxScatterGatherBytes=" + maxScatterGatherBytes +
", maxSubqueryRows=" + maxSubqueryRows +
", maxSubqueryBytes=" + maxSubqueryBytes +
", useNestedForUnknownTypeInSubquery=" + useNestedForUnknownTypeInSubquery +
", maxQueryTimeout=" + maxQueryTimeout +
", maxRequestHeaderSize=" + maxRequestHeaderSize +
", gracefulShutdownTimeout=" + gracefulShutdownTimeout +

View File

@ -153,6 +153,8 @@ public class CliIndexerServerModule implements Module
oldConfig.getDefaultQueryTimeout(),
oldConfig.getMaxScatterGatherBytes(),
oldConfig.getMaxSubqueryRows(),
oldConfig.getMaxSubqueryBytes(),
oldConfig.isuseNestedForUnknownTypeInSubquery(),
oldConfig.getMaxQueryTimeout(),
oldConfig.getMaxRequestHeaderSize(),
oldConfig.getGracefulShutdownTimeout(),

View File

@ -52,6 +52,8 @@ public class ServerConfigTest
defaultConfig.getDefaultQueryTimeout(),
defaultConfig.getMaxScatterGatherBytes(),
defaultConfig.getMaxSubqueryRows(),
defaultConfig.getMaxSubqueryBytes(),
defaultConfig.isuseNestedForUnknownTypeInSubquery(),
defaultConfig.getMaxQueryTimeout(),
defaultConfig.getMaxRequestHeaderSize(),
defaultConfig.getGracefulShutdownTimeout(),

View File

@ -34,6 +34,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.Druids;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.query.GlobalTableDataSource;
import org.apache.druid.query.InlineDataSource;
import org.apache.druid.query.JoinDataSource;
@ -62,6 +63,7 @@ import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.topn.TopNQuery;
import org.apache.druid.query.topn.TopNQueryBuilder;
import org.apache.druid.segment.FrameBasedInlineSegmentWrangler;
import org.apache.druid.segment.InlineSegmentWrangler;
import org.apache.druid.segment.MapSegmentWrangler;
import org.apache.druid.segment.ReferenceCountingSegment;
@ -72,6 +74,7 @@ import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.data.ComparableList;
import org.apache.druid.segment.data.ComparableStringArray;
import org.apache.druid.segment.join.FrameBasedInlineJoinableFactory;
import org.apache.druid.segment.join.InlineJoinableFactory;
import org.apache.druid.segment.join.JoinConditionAnalysis;
import org.apache.druid.segment.join.JoinType;
@ -348,7 +351,7 @@ public class ClientQuerySegmentWalkerTest
testQuery(
query,
ImmutableList.of(
new ArrayList<>(ImmutableList.of(
ExpectedQuery.cluster(subquery.withId(DUMMY_QUERY_ID).withSubQueryId("1.1")),
ExpectedQuery.local(
query.withDataSource(
@ -358,7 +361,7 @@ public class ClientQuerySegmentWalkerTest
)
)
)
),
)),
ImmutableList.of(new Object[]{Intervals.ETERNITY.getStartMillis(), 3L})
);
@ -798,7 +801,35 @@ public class ClientQuerySegmentWalkerTest
.withId(DUMMY_QUERY_ID);
expectedException.expect(ResourceLimitExceededException.class);
expectedException.expectMessage("Subquery generated results beyond maximum[2]");
expectedException.expectMessage("Subquery generated results beyond maximum[2] rows");
testQuery(query, ImmutableList.of(), ImmutableList.of());
}
@Test
public void testTimeseriesOnGroupByOnTableErrorTooLarge()
{
final GroupByQuery subquery =
GroupByQuery.builder()
.setDataSource(FOO)
.setGranularity(Granularities.ALL)
.setInterval(Collections.singletonList(INTERVAL))
.setDimensions(DefaultDimensionSpec.of("s"))
.build();
final TimeseriesQuery query =
(TimeseriesQuery) Druids.newTimeseriesQueryBuilder()
.dataSource(new QueryDataSource(subquery))
.granularity(Granularities.ALL)
.intervals(Intervals.ONLY_ETERNITY)
.aggregators(new CountAggregatorFactory("cnt"))
.context(ImmutableMap.of(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "1"))
.build()
.withId(DUMMY_QUERY_ID);
expectedException.expect(ResourceLimitExceededException.class);
expectedException.expectMessage("Subquery generated results beyond maximum[1] bytes");
testQuery(query, ImmutableList.of(), ImmutableList.of());
}
@ -1316,8 +1347,9 @@ public class ClientQuerySegmentWalkerTest
final SegmentWrangler segmentWrangler = new MapSegmentWrangler(
ImmutableMap.<Class<? extends DataSource>, SegmentWrangler>builder()
.put(InlineDataSource.class, new InlineSegmentWrangler())
.build()
.put(InlineDataSource.class, new InlineSegmentWrangler())
.put(FrameBasedInlineDataSource.class, new FrameBasedInlineSegmentWrangler())
.build()
);
final JoinableFactory globalFactory = new JoinableFactory()
@ -1336,11 +1368,12 @@ public class ClientQuerySegmentWalkerTest
};
final JoinableFactory joinableFactory = new MapJoinableFactory(
ImmutableSet.of(globalFactory, new InlineJoinableFactory()),
ImmutableSet.of(globalFactory, new InlineJoinableFactory(), new FrameBasedInlineJoinableFactory()),
ImmutableMap.<Class<? extends JoinableFactory>, Class<? extends DataSource>>builder()
.put(InlineJoinableFactory.class, InlineDataSource.class)
.put(globalFactory.getClass(), GlobalTableDataSource.class)
.build()
.put(InlineJoinableFactory.class, InlineDataSource.class)
.put(FrameBasedInlineJoinableFactory.class, FrameBasedInlineDataSource.class)
.put(globalFactory.getClass(), GlobalTableDataSource.class)
.build()
);
final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(joinableFactory);
@ -1450,8 +1483,9 @@ public class ClientQuerySegmentWalkerTest
ExpectedQuery(Query<?> query, ClusterOrLocal how)
{
Query<?> modifiedQuery;
// Need to blast various parameters that will vary and aren't important to test for.
this.query = query.withOverriddenContext(
modifiedQuery = query.withOverriddenContext(
ImmutableMap.<String, Object>builder()
.put(DirectDruidClient.QUERY_FAIL_TIME, 0L)
.put(QueryContexts.DEFAULT_TIMEOUT_KEY, 0L)
@ -1467,6 +1501,11 @@ public class ClientQuerySegmentWalkerTest
.build()
);
if (modifiedQuery.getDataSource() instanceof FrameBasedInlineDataSource) {
// Do this recursively for if the query's datasource is a query datasource
}
this.query = modifiedQuery;
this.how = how;
}

View File

@ -28,6 +28,7 @@ import org.apache.druid.query.DataSource;
import org.apache.druid.query.DefaultGenericQueryMetricsFactory;
import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.query.InlineDataSource;
import org.apache.druid.query.LookupDataSource;
import org.apache.druid.query.Query;
@ -70,6 +71,7 @@ import org.apache.druid.query.topn.TopNQueryRunnerFactory;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.SegmentWrangler;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.join.FrameBasedInlineJoinableFactory;
import org.apache.druid.segment.join.InlineJoinableFactory;
import org.apache.druid.segment.join.JoinableFactory;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
@ -372,8 +374,9 @@ public class QueryStackTests
ImmutableSet.Builder<JoinableFactory> setBuilder = ImmutableSet.builder();
ImmutableMap.Builder<Class<? extends JoinableFactory>, Class<? extends DataSource>> mapBuilder =
ImmutableMap.builder();
setBuilder.add(new InlineJoinableFactory());
setBuilder.add(new InlineJoinableFactory(), new FrameBasedInlineJoinableFactory());
mapBuilder.put(InlineJoinableFactory.class, InlineDataSource.class);
mapBuilder.put(FrameBasedInlineJoinableFactory.class, FrameBasedInlineDataSource.class);
if (lookupProvider != null) {
setBuilder.add(new LookupJoinableFactory(lookupProvider));
mapBuilder.put(LookupJoinableFactory.class, LookupDataSource.class);

View File

@ -239,6 +239,10 @@ public class BaseCalciteQueryTest extends CalciteTestBase
QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE
);
public static final Map<String, Object> QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT =
DEFAULT_QUERY_CONTEXT_BUILDER.put(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000")
.build();
// Add additional context to the given context map for when the
// timeseries query has timestamp_floor expression on the timestamp dimension
public static Map<String, Object> getTimeseriesContextWithFloorTime(

View File

@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
@ -42,7 +41,6 @@ import org.apache.druid.query.LookupDataSource;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryDataSource;
import org.apache.druid.query.ResourceLimitExceededException;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.UnionDataSource;
import org.apache.druid.query.aggregation.AggregatorFactory;
@ -139,52 +137,6 @@ import java.util.stream.Collectors;
public class CalciteQueryTest extends BaseCalciteQueryTest
{
@Test
public void testGroupByWithPostAggregatorReferencingTimeFloorColumnOnTimeseries()
{
cannotVectorize();
testQuery(
"SELECT TIME_FORMAT(\"date\", 'yyyy-MM'), SUM(x)\n"
+ "FROM (\n"
+ " SELECT\n"
+ " FLOOR(__time to hour) as \"date\",\n"
+ " COUNT(*) as x\n"
+ " FROM foo\n"
+ " GROUP BY 1\n"
+ ")\n"
+ "GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.HOUR)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
.build()
)
.setInterval(querySegmentSpec(Intervals.ETERNITY))
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_format(\"d0\",'yyyy-MM','UTC')",
ColumnType.STRING
)
)
.setGranularity(Granularities.ALL)
.addDimension(new DefaultDimensionSpec("v0", "_d0"))
.addAggregator(new LongSumAggregatorFactory("_a0", "a0"))
.build()
),
ImmutableList.of(
new Object[]{"2000-01", 3L},
new Object[]{"2001-01", 3L}
)
);
}
@Test
public void testInformationSchemaSchemata()
{
@ -4048,56 +4000,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testSelfJoin()
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
testQuery(
"SELECT COUNT(*) FROM druid.foo x, druid.foo y\n",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(
join(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns(
ImmutableList.of(
"__time",
"cnt",
"dim1",
"dim2",
"dim3",
"m1",
"m2",
"unique_dim1"
)
)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
"j0.",
"1",
JoinType.INNER
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{36L}
)
);
}
@Test
public void testGroupingWithNullInFilter()
{
@ -4131,82 +4033,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testTwoExactCountDistincts()
{
testQuery(
PLANNER_CONFIG_NO_HLL,
"SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(
newScanQueryBuilder()
.dataSource(
join(
new QueryDataSource(
GroupByQuery
.builder()
.setDataSource(
GroupByQuery
.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(
new FilteredAggregatorFactory(
new CountAggregatorFactory("a0"),
not(selector("d0", null, null))
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
new QueryDataSource(
GroupByQuery
.builder()
.setDataSource(
GroupByQuery
.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(
new FilteredAggregatorFactory(
new CountAggregatorFactory("a0"),
not(selector("d0", null, null))
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
"j0.",
"1",
JoinType.INNER
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("a0", "j0.a0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{NullHandling.sqlCompatible() ? 6L : 5L, NullHandling.sqlCompatible() ? 3L : 2L}
)
);
}
@Test
public void testGroupByNothingWithLiterallyFalseFilter()
{
@ -4549,58 +4375,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testViewAndJoin()
{
notMsqCompatible();
cannotVectorize();
Map<String, Object> queryContext = withLeftDirectAccessEnabled(QUERY_CONTEXT_DEFAULT);
testQuery(
"SELECT COUNT(*) FROM view.cview as a INNER JOIN druid.foo d on d.dim2 = a.dim2 WHERE a.dim1_firstchar <> 'z' ",
queryContext,
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(
join(
join(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim2")
.context(queryContext)
.build()
),
"j0.",
"(\"dim2\" == \"j0.dim2\")",
JoinType.INNER,
bound("dim2", "a", "a", false, false, null, null)
),
new QueryDataSource(
newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim2")
.context(queryContext)
.build()
),
"_j0.",
"('a' == \"_j0.dim2\")",
JoinType.INNER
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))))
.granularity(Granularities.ALL)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(withLeftDirectAccessEnabled(QUERY_CONTEXT_DEFAULT))
.build()
),
ImmutableList.of(
new Object[]{8L}
)
);
}
@Test
public void testCountStarWithLikeFilter()
{
@ -7181,157 +6955,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testUseTimeFloorInsteadOfGranularityOnJoinResult()
{
cannotVectorize();
testQuery(
"WITH main AS (SELECT * FROM foo LIMIT 2)\n"
+ "SELECT TIME_FLOOR(__time, 'PT1H') AS \"time\", dim1, COUNT(*)\n"
+ "FROM main\n"
+ "WHERE dim1 IN (SELECT dim1 FROM main GROUP BY 1 ORDER BY COUNT(*) DESC LIMIT 5)\n"
+ "GROUP BY 1, 2",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
join(
new QueryDataSource(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Intervals.ETERNITY))
.columns("__time", "dim1")
.limit(2)
.build()
),
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Intervals.ETERNITY))
.columns("dim1")
.limit(2)
.build()
)
)
.setInterval(querySegmentSpec(Intervals.ETERNITY))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0")))
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec(
"a0",
Direction.DESCENDING,
StringComparators.NUMERIC
)
),
5
)
)
.build()
),
"j0.",
"(\"dim1\" == \"j0.d0\")",
JoinType.INNER
)
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'PT1H',null,'UTC')",
ColumnType.LONG
)
)
.setDimensions(dimensions(
new DefaultDimensionSpec("v0", "d0", ColumnType.LONG),
new DefaultDimensionSpec("dim1", "d1")
))
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
NullHandling.sqlCompatible()
? ImmutableList.of(new Object[]{946684800000L, "", 1L}, new Object[]{946771200000L, "10.1", 1L})
: ImmutableList.of(new Object[]{946771200000L, "10.1", 1L})
);
}
@Test
public void testMinMaxAvgDailyCountWithLimit()
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
testQuery(
"SELECT * FROM ("
+ " SELECT max(cnt), min(cnt), avg(cnt), TIME_EXTRACT(max(t), 'EPOCH') last_time, count(1) num_days FROM (\n"
+ " SELECT TIME_FLOOR(__time, 'P1D') AS t, count(1) cnt\n"
+ " FROM \"foo\"\n"
+ " GROUP BY 1\n"
+ " )"
+ ") LIMIT 1\n",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC))
.intervals(querySegmentSpec(Filtration.eternity()))
.aggregators(new CountAggregatorFactory("a0"))
.context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
.build()
)
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(
useDefault ?
aggregators(
new LongMaxAggregatorFactory("_a0", "a0"),
new LongMinAggregatorFactory("_a1", "a0"),
new LongSumAggregatorFactory("_a2:sum", "a0"),
new CountAggregatorFactory("_a2:count"),
new LongMaxAggregatorFactory("_a3", "d0"),
new CountAggregatorFactory("_a4")
) : aggregators(
new LongMaxAggregatorFactory("_a0", "a0"),
new LongMinAggregatorFactory("_a1", "a0"),
new LongSumAggregatorFactory("_a2:sum", "a0"),
new FilteredAggregatorFactory(
new CountAggregatorFactory("_a2:count"),
not(selector("a0", null, null))
),
new LongMaxAggregatorFactory("_a3", "d0"),
new CountAggregatorFactory("_a4")
)
)
.setPostAggregatorSpecs(
ImmutableList.of(
new ArithmeticPostAggregator(
"_a2",
"quotient",
ImmutableList.of(
new FieldAccessPostAggregator(null, "_a2:sum"),
new FieldAccessPostAggregator(null, "_a2:count")
)
),
expressionPostAgg("p0", "timestamp_extract(\"_a3\",'EPOCH','UTC')")
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(new Object[]{1L, 1L, 1L, 978480000L, 6L})
);
}
@Test
public void testAvgDailyCountDistinct()
{
@ -7407,116 +7030,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testExactCountDistinctOfSemiJoinResult()
{
// Cannot vectorize due to extraction dimension spec.
cannotVectorize();
testQuery(
"SELECT COUNT(*)\n"
+ "FROM (\n"
+ " SELECT DISTINCT dim2\n"
+ " FROM druid.foo\n"
+ " WHERE SUBSTRING(dim2, 1, 1) IN (\n"
+ " SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n"
+ " ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n"
+ ")",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(
join(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
GroupByQuery
.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(not(selector("dim1", "", null)))
.setDimensions(
dimensions(
new ExtractionDimensionSpec(
"dim1",
"d0",
new SubstringDimExtractionFn(0, 1)
)
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
"j0.",
equalsCondition(
makeExpression("substring(\"dim2\", 0, 1)"),
DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")
),
JoinType.INNER
)
)
.setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01")))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{2L}
)
);
}
@Test
public void testMaxSubqueryRows()
{
notMsqCompatible();
expectedException.expect(ResourceLimitExceededException.class);
expectedException.expectMessage("Subquery generated results beyond maximum[2]");
testQuery(
PLANNER_CONFIG_DEFAULT,
ImmutableMap.of(QueryContexts.MAX_SUBQUERY_ROWS_KEY, 2),
"SELECT COUNT(*)\n"
+ "FROM druid.foo\n"
+ "WHERE SUBSTRING(dim2, 1, 1) IN (\n"
+ " SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n"
+ ")\n",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(),
ImmutableList.of()
);
}
@Test
public void testZeroMaxNumericInFilter()
{
expectedException.expect(UOE.class);
expectedException.expectMessage("[maxNumericInFilters] must be greater than 0");
testQuery(
PLANNER_CONFIG_DEFAULT,
ImmutableMap.of(QueryContexts.MAX_NUMERIC_IN_FILTERS, 0),
"SELECT COUNT(*)\n"
+ "FROM druid.numfoo\n"
+ "WHERE dim6 IN (\n"
+ "1,2,3\n"
+ ")\n",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(),
ImmutableList.of()
);
}
@Test
public void testHighestMaxNumericInFilter()
{
@ -7641,54 +7154,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter()
{
// Cannot vectorize topN operator.
cannotVectorize();
testQuery(
"SELECT\n"
+ " SUM(cnt),\n"
+ " COUNT(*)\n"
+ "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n"
+ "WHERE cnt > 0",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim2", "d0"))
.aggregators(new LongSumAggregatorFactory("a0", "cnt"))
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
.threshold(1)
.context(QUERY_CONTEXT_DEFAULT)
.build()
)
)
.setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC))
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("_a0", "a0"),
new CountAggregatorFactory("_a1")
))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
NullHandling.replaceWithDefault() ?
ImmutableList.of(
new Object[]{3L, 1L}
) :
ImmutableList.of(
new Object[]{2L, 1L}
)
);
}
@Test
public void testCompareExactAndApproximateCountDistinctUsingSubquery()
{
@ -11306,148 +10771,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
}
}
@Test
public void testUsingSubqueryAsFilterOnTwoColumns()
{
testQuery(
"SELECT __time, cnt, dim1, dim2 FROM druid.foo "
+ " WHERE (dim1, dim2) IN ("
+ " SELECT dim1, dim2 FROM ("
+ " SELECT dim1, dim2, COUNT(*)"
+ " FROM druid.foo"
+ " WHERE dim2 = 'abc'"
+ " GROUP BY dim1, dim2"
+ " HAVING COUNT(*) = 1"
+ " )"
+ " )",
ImmutableList.of(
newScanQueryBuilder()
.dataSource(
join(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(selector("dim2", "abc", null))
.setDimensions(dimensions(
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("dim2", "d1")
))
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setPostAggregatorSpecs(
ImmutableList.of(expressionPostAgg("p0", "'abc'"))
)
.setHavingSpec(having(selector("a0", "1", null)))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
"j0.",
StringUtils.format(
"(%s && %s)",
equalsCondition(makeColumnExpression("dim1"), makeColumnExpression("j0.d0")),
equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.p0"))
),
JoinType.INNER
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(expressionVirtualColumn("v0", "'abc'", ColumnType.STRING))
.columns("__time", "cnt", "dim1", "v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{timestamp("2001-01-02"), 1L, "def", "abc"}
)
);
}
@Test
public void testUsingSubqueryAsFilterWithInnerSort()
{
// Regression test for https://github.com/apache/druid/issues/4208
testQuery(
"SELECT dim1, dim2 FROM druid.foo\n"
+ " WHERE dim2 IN (\n"
+ " SELECT dim2\n"
+ " FROM druid.foo\n"
+ " GROUP BY dim2\n"
+ " ORDER BY dim2 DESC\n"
+ " )",
ImmutableList.of(
newScanQueryBuilder()
.dataSource(
join(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
"j0.",
equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.d0")),
JoinType.INNER
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim1", "dim2")
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
NullHandling.replaceWithDefault() ?
ImmutableList.of(
new Object[]{"", "a"},
new Object[]{"1", "a"},
new Object[]{"def", "abc"}
) :
ImmutableList.of(
new Object[]{"", "a"},
new Object[]{"2", ""},
new Object[]{"1", "a"},
new Object[]{"def", "abc"}
)
);
}
@Test
public void testUsingSubqueryWithLimit()
{
// Cannot vectorize scan query.
cannotVectorize();
testQuery(
"SELECT COUNT(*) AS cnt FROM ( SELECT * FROM druid.foo LIMIT 10 ) tmpA",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(expressionVirtualColumn("v0", "0", ColumnType.LONG))
.columns("v0")
.limit(10)
.context(QUERY_CONTEXT_DEFAULT)
.build()
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.build()
),
ImmutableList.of(
new Object[]{6L}
)
);
}
@Test
public void testUsingSubqueryWithoutLimit()
{
@ -13244,104 +12567,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testEmptyGroupWithOffsetDoesntInfiniteLoop()
{
notMsqCompatible();
testQuery(
"SELECT r0.c, r1.c\n"
+ "FROM (\n"
+ " SELECT COUNT(*) AS c\n"
+ " FROM \"foo\"\n"
+ " GROUP BY ()\n"
+ " OFFSET 1\n"
+ ") AS r0\n"
+ "LEFT JOIN (\n"
+ " SELECT COUNT(*) AS c\n"
+ " FROM \"foo\"\n"
+ " GROUP BY ()\n"
+ ") AS r1 ON TRUE LIMIT 10",
ImmutableList.of(
Druids.newScanQueryBuilder()
.dataSource(
join(
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(
aggregators(
new CountAggregatorFactory("a0")
)
)
.setLimitSpec(DefaultLimitSpec.builder().offset(1).limit(10).build())
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
new QueryDataSource(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(new CountAggregatorFactory("a0"))
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
"j0.",
"1",
JoinType.LEFT,
null
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("a0", "j0.a0")
.limit(10)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.legacy(false)
.build()
),
ImmutableList.of()
);
}
@Test
public void testJoinWithTimeDimension()
{
testQuery(
PLANNER_CONFIG_DEFAULT,
QUERY_CONTEXT_DEFAULT,
"SELECT count(*) FROM druid.foo t1 inner join druid.foo t2 on t1.__time = t2.__time",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(Druids.newTimeseriesQueryBuilder()
.dataSource(JoinDataSource.create(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
Druids.newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.columns("__time")
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()),
"j0.",
"(\"__time\" == \"j0.__time\")",
JoinType.INNER,
null,
ExprMacroTable.nil(),
CalciteTests.createJoinableFactoryWrapper()
))
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(QUERY_CONTEXT_DEFAULT)
.build()),
ImmutableList.of(new Object[]{6L})
);
}
@Test
public void testExpressionCounts()
{

View File

@ -0,0 +1,945 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.sql.calcite;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.Druids;
import org.apache.druid.query.JoinDataSource;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryDataSource;
import org.apache.druid.query.ResourceLimitExceededException;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.ExtractionDimensionSpec;
import org.apache.druid.query.extraction.SubstringDimExtractionFn;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.topn.DimensionTopNMetricSpec;
import org.apache.druid.query.topn.TopNQueryBuilder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.join.JoinType;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.joda.time.DateTimeZone;
import org.joda.time.Period;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Calcite tests which involve subqueries and materializing the intermediate results on {@link org.apache.druid.server.ClientQuerySegmentWalker}
* The tests are run with two different codepaths:
* 1. Where the memory limit is not set. The intermediate results are materialized as inline rows
* 2. Where the memory limit is set. The intermediate results are materialized as frames
*/
@RunWith(Parameterized.class)
public class CalciteSubqueryTest extends BaseCalciteQueryTest
{
public String testName;
public Map<String, Object> queryContext;
public CalciteSubqueryTest(
String testName,
Map<String, Object> queryContext
)
{
this.testName = testName;
this.queryContext = queryContext;
}
@Parameterized.Parameters(name = "{0}")
public static Iterable<Object[]> constructorFeeder()
{
final List<Object[]> constructors = new ArrayList<>();
constructors.add(
new Object[]{"without memory limit", QUERY_CONTEXT_DEFAULT}
);
constructors.add(
new Object[]{"with memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT}
);
return constructors;
}
@Test
public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter()
{
// Cannot vectorize topN operator.
cannotVectorize();
testQuery(
"SELECT\n"
+ " SUM(cnt),\n"
+ " COUNT(*)\n"
+ "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n"
+ "WHERE cnt > 0",
queryContext,
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim2", "d0"))
.aggregators(new LongSumAggregatorFactory("a0", "cnt"))
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
.threshold(1)
.build()
)
)
.setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC))
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("_a0", "a0"),
new CountAggregatorFactory("_a1")
))
.setContext(queryContext)
.build()
),
NullHandling.replaceWithDefault() ?
ImmutableList.of(
new Object[]{3L, 1L}
) :
ImmutableList.of(
new Object[]{2L, 1L}
)
);
}
@Test
public void testExactCountDistinctOfSemiJoinResult()
{
// Cannot vectorize due to extraction dimension spec.
cannotVectorize();
testQuery(
"SELECT COUNT(*)\n"
+ "FROM (\n"
+ " SELECT DISTINCT dim2\n"
+ " FROM druid.foo\n"
+ " WHERE SUBSTRING(dim2, 1, 1) IN (\n"
+ " SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n"
+ " ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n"
+ ")",
queryContext,
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(
join(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
GroupByQuery
.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(not(selector("dim1", "", null)))
.setDimensions(
dimensions(
new ExtractionDimensionSpec(
"dim1",
"d0",
new SubstringDimExtractionFn(0, 1)
)
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
"j0.",
equalsCondition(
DruidExpression.ofExpression(
ColumnType.STRING,
null,
args -> "substring(\"dim2\", 0, 1)",
Collections.emptyList()
),
DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")
),
JoinType.INNER
)
)
.setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01")))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{2L}
)
);
}
@Ignore("Merge buffers exceed the prescribed limit when the results are materialized as frames")
@Test
public void testTwoExactCountDistincts()
{
testQuery(
PLANNER_CONFIG_NO_HLL,
queryContext,
"SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(
newScanQueryBuilder()
.dataSource(
join(
new QueryDataSource(
GroupByQuery
.builder()
.setDataSource(
GroupByQuery
.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(
new FilteredAggregatorFactory(
new CountAggregatorFactory("a0"),
not(selector("d0", null, null))
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
new QueryDataSource(
GroupByQuery
.builder()
.setDataSource(
GroupByQuery
.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(
new FilteredAggregatorFactory(
new CountAggregatorFactory("a0"),
not(selector("d0", null, null))
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
"j0.",
"1",
JoinType.INNER
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("a0", "j0.a0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{NullHandling.sqlCompatible() ? 6L : 5L, NullHandling.sqlCompatible() ? 3L : 2L}
)
);
}
@Test
public void testViewAndJoin()
{
cannotVectorize();
Map<String, Object> queryContextModified = withLeftDirectAccessEnabled(queryContext);
testQuery(
"SELECT COUNT(*) FROM view.cview as a INNER JOIN druid.foo d on d.dim2 = a.dim2 WHERE a.dim1_firstchar <> 'z' ",
queryContextModified,
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(
join(
join(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim2")
.context(queryContextModified)
.build()
),
"j0.",
"(\"dim2\" == \"j0.dim2\")",
JoinType.INNER,
bound("dim2", "a", "a", false, false, null, null)
),
new QueryDataSource(
newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim2")
.context(queryContextModified)
.build()
),
"_j0.",
"('a' == \"_j0.dim2\")",
JoinType.INNER
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))))
.granularity(Granularities.ALL)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(queryContextModified)
.build()
),
ImmutableList.of(
new Object[]{8L}
)
);
}
@Test
public void testGroupByWithPostAggregatorReferencingTimeFloorColumnOnTimeseries()
{
cannotVectorize();
testQuery(
"SELECT TIME_FORMAT(\"date\", 'yyyy-MM'), SUM(x)\n"
+ "FROM (\n"
+ " SELECT\n"
+ " FLOOR(__time to hour) as \"date\",\n"
+ " COUNT(*) as x\n"
+ " FROM foo\n"
+ " GROUP BY 1\n"
+ ")\n"
+ "GROUP BY 1",
queryContext,
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.HOUR)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
.build()
)
.setInterval(querySegmentSpec(Intervals.ETERNITY))
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_format(\"d0\",'yyyy-MM','UTC')",
ColumnType.STRING
)
)
.setGranularity(Granularities.ALL)
.addDimension(new DefaultDimensionSpec("v0", "_d0"))
.addAggregator(new LongSumAggregatorFactory("_a0", "a0"))
.build()
),
ImmutableList.of(
new Object[]{"2000-01", 3L},
new Object[]{"2001-01", 3L}
)
);
}
@Test
public void testUsingSubqueryAsFilterWithInnerSort()
{
// Regression test for https://github.com/apache/druid/issues/4208
testQuery(
"SELECT dim1, dim2 FROM druid.foo\n"
+ " WHERE dim2 IN (\n"
+ " SELECT dim2\n"
+ " FROM druid.foo\n"
+ " GROUP BY dim2\n"
+ " ORDER BY dim2 DESC\n"
+ " )",
queryContext,
ImmutableList.of(
newScanQueryBuilder()
.dataSource(
join(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
"j0.",
equalsCondition(DruidExpression.ofColumn(ColumnType.STRING, "dim2"), DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")),
JoinType.INNER
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim1", "dim2")
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
NullHandling.replaceWithDefault() ?
ImmutableList.of(
new Object[]{"", "a"},
new Object[]{"1", "a"},
new Object[]{"def", "abc"}
) :
ImmutableList.of(
new Object[]{"", "a"},
new Object[]{"2", ""},
new Object[]{"1", "a"},
new Object[]{"def", "abc"}
)
);
}
@Test
public void testUsingSubqueryAsFilterOnTwoColumns()
{
testQuery(
"SELECT __time, cnt, dim1, dim2 FROM druid.foo "
+ " WHERE (dim1, dim2) IN ("
+ " SELECT dim1, dim2 FROM ("
+ " SELECT dim1, dim2, COUNT(*)"
+ " FROM druid.foo"
+ " WHERE dim2 = 'abc'"
+ " GROUP BY dim1, dim2"
+ " HAVING COUNT(*) = 1"
+ " )"
+ " )",
queryContext,
ImmutableList.of(
newScanQueryBuilder()
.dataSource(
join(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(selector("dim2", "abc", null))
.setDimensions(dimensions(
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("dim2", "d1")
))
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setPostAggregatorSpecs(
ImmutableList.of(expressionPostAgg("p0", "'abc'"))
)
.setHavingSpec(having(selector("a0", "1", null)))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
"j0.",
StringUtils.format(
"(%s && %s)",
equalsCondition(DruidExpression.ofColumn(ColumnType.STRING, "dim1"), DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")),
equalsCondition(DruidExpression.ofColumn(ColumnType.STRING, "dim2"), DruidExpression.ofColumn(ColumnType.STRING, "j0.p0"))
),
JoinType.INNER
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(expressionVirtualColumn("v0", "'abc'", ColumnType.STRING))
.columns("__time", "cnt", "dim1", "v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{timestamp("2001-01-02"), 1L, "def", "abc"}
)
);
}
@Test
public void testMinMaxAvgDailyCountWithLimit()
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
testQuery(
"SELECT * FROM ("
+ " SELECT max(cnt), min(cnt), avg(cnt), TIME_EXTRACT(max(t), 'EPOCH') last_time, count(1) num_days FROM (\n"
+ " SELECT TIME_FLOOR(__time, 'P1D') AS t, count(1) cnt\n"
+ " FROM \"foo\"\n"
+ " GROUP BY 1\n"
+ " )"
+ ") LIMIT 1\n",
queryContext,
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC))
.intervals(querySegmentSpec(Filtration.eternity()))
.aggregators(new CountAggregatorFactory("a0"))
.context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0"))
.build()
)
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(
useDefault ?
aggregators(
new LongMaxAggregatorFactory("_a0", "a0"),
new LongMinAggregatorFactory("_a1", "a0"),
new LongSumAggregatorFactory("_a2:sum", "a0"),
new CountAggregatorFactory("_a2:count"),
new LongMaxAggregatorFactory("_a3", "d0"),
new CountAggregatorFactory("_a4")
) : aggregators(
new LongMaxAggregatorFactory("_a0", "a0"),
new LongMinAggregatorFactory("_a1", "a0"),
new LongSumAggregatorFactory("_a2:sum", "a0"),
new FilteredAggregatorFactory(
new CountAggregatorFactory("_a2:count"),
not(selector("a0", null, null))
),
new LongMaxAggregatorFactory("_a3", "d0"),
new CountAggregatorFactory("_a4")
)
)
.setPostAggregatorSpecs(
ImmutableList.of(
new ArithmeticPostAggregator(
"_a2",
"quotient",
ImmutableList.of(
new FieldAccessPostAggregator(null, "_a2:sum"),
new FieldAccessPostAggregator(null, "_a2:count")
)
),
expressionPostAgg("p0", "timestamp_extract(\"_a3\",'EPOCH','UTC')")
)
)
.setContext(queryContext)
.build()
),
ImmutableList.of(new Object[]{1L, 1L, 1L, 978480000L, 6L})
);
}
@Test
public void testEmptyGroupWithOffsetDoesntInfiniteLoop()
{
testQuery(
"SELECT r0.c, r1.c\n"
+ "FROM (\n"
+ " SELECT COUNT(*) AS c\n"
+ " FROM \"foo\"\n"
+ " GROUP BY ()\n"
+ " OFFSET 1\n"
+ ") AS r0\n"
+ "LEFT JOIN (\n"
+ " SELECT COUNT(*) AS c\n"
+ " FROM \"foo\"\n"
+ " GROUP BY ()\n"
+ ") AS r1 ON TRUE LIMIT 10",
queryContext,
ImmutableList.of(
Druids.newScanQueryBuilder()
.dataSource(
join(
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(
aggregators(
new CountAggregatorFactory("a0")
)
)
.setLimitSpec(DefaultLimitSpec.builder().offset(1).limit(10).build())
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
new QueryDataSource(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(new CountAggregatorFactory("a0"))
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
"j0.",
"1",
JoinType.LEFT,
null
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("a0", "j0.a0")
.limit(10)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.legacy(false)
.build()
),
ImmutableList.of()
);
}
@Test
public void testMaxSubqueryRows()
{
if ("without memory limit".equals(testName)) {
expectedException.expect(ResourceLimitExceededException.class);
expectedException.expectMessage("Subquery generated results beyond maximum[1]");
Map<String, Object> modifiedQueryContext = new HashMap<>(queryContext);
modifiedQueryContext.put(QueryContexts.MAX_SUBQUERY_ROWS_KEY, 1);
testQuery(
"SELECT\n"
+ " SUM(cnt),\n"
+ " COUNT(*)\n"
+ "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 2) \n"
+ "WHERE cnt > 0",
modifiedQueryContext,
ImmutableList.of(),
ImmutableList.of()
);
} else {
// Since the results are materializable as frames, we are able to use the memory limit and donot rely on the
// row limit for the subquery
Map<String, Object> modifiedQueryContext = new HashMap<>(queryContext);
modifiedQueryContext.put(QueryContexts.MAX_SUBQUERY_ROWS_KEY, 1);
testQuery(
"SELECT\n"
+ " SUM(cnt),\n"
+ " COUNT(*)\n"
+ "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n"
+ "WHERE cnt > 0",
modifiedQueryContext,
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim2", "d0"))
.aggregators(new LongSumAggregatorFactory("a0", "cnt"))
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
.threshold(1)
.build()
)
)
.setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC))
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("_a0", "a0"),
new CountAggregatorFactory("_a1")
))
.setContext(queryContext)
.build()
),
NullHandling.replaceWithDefault() ?
ImmutableList.of(
new Object[]{3L, 1L}
) :
ImmutableList.of(
new Object[]{2L, 1L}
)
);
}
}
@Test
public void testZeroMaxNumericInFilter()
{
expectedException.expect(UOE.class);
expectedException.expectMessage("[maxNumericInFilters] must be greater than 0");
Map<String, Object> modifiedQueryContext = new HashMap<>(queryContext);
modifiedQueryContext.put(QueryContexts.MAX_NUMERIC_IN_FILTERS, 0);
testQuery(
PLANNER_CONFIG_DEFAULT,
modifiedQueryContext,
"SELECT COUNT(*)\n"
+ "FROM druid.numfoo\n"
+ "WHERE dim6 IN (\n"
+ "1,2,3\n"
+ ")\n",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(),
ImmutableList.of()
);
}
@Test
public void testUseTimeFloorInsteadOfGranularityOnJoinResult()
{
cannotVectorize();
testQuery(
"WITH main AS (SELECT * FROM foo LIMIT 2)\n"
+ "SELECT TIME_FLOOR(__time, 'PT1H') AS \"time\", dim1, COUNT(*)\n"
+ "FROM main\n"
+ "WHERE dim1 IN (SELECT dim1 FROM main GROUP BY 1 ORDER BY COUNT(*) DESC LIMIT 5)\n"
+ "GROUP BY 1, 2",
queryContext,
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
join(
new QueryDataSource(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Intervals.ETERNITY))
.columns("__time", "dim1")
.limit(2)
.build()
),
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Intervals.ETERNITY))
.columns("dim1")
.limit(2)
.build()
)
)
.setInterval(querySegmentSpec(Intervals.ETERNITY))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0")))
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec(
"a0",
OrderByColumnSpec.Direction.DESCENDING,
StringComparators.NUMERIC
)
),
5
)
)
.build()
),
"j0.",
"(\"dim1\" == \"j0.d0\")",
JoinType.INNER
)
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'PT1H',null,'UTC')",
ColumnType.LONG
)
)
.setDimensions(dimensions(
new DefaultDimensionSpec("v0", "d0", ColumnType.LONG),
new DefaultDimensionSpec("dim1", "d1")
))
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
NullHandling.sqlCompatible()
? ImmutableList.of(new Object[]{946684800000L, "", 1L}, new Object[]{946771200000L, "10.1", 1L})
: ImmutableList.of(new Object[]{946771200000L, "10.1", 1L})
);
}
@Test
public void testJoinWithTimeDimension()
{
testQuery(
PLANNER_CONFIG_DEFAULT,
queryContext,
"SELECT count(*) FROM druid.foo t1 inner join druid.foo t2 on t1.__time = t2.__time",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(Druids.newTimeseriesQueryBuilder()
.dataSource(JoinDataSource.create(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
Druids.newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.columns("__time")
.legacy(false)
.context(queryContext)
.build()),
"j0.",
"(\"__time\" == \"j0.__time\")",
JoinType.INNER,
null,
ExprMacroTable.nil(),
CalciteTests.createJoinableFactoryWrapper()
))
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(queryContext)
.build()),
ImmutableList.of(new Object[]{6L})
);
}
@Test
public void testUsingSubqueryWithLimit()
{
// Cannot vectorize scan query.
cannotVectorize();
testQuery(
"SELECT COUNT(*) AS cnt FROM ( SELECT * FROM druid.foo LIMIT 10 ) tmpA",
queryContext,
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(expressionVirtualColumn("v0", "0", ColumnType.LONG))
.columns("v0")
.limit(10)
.context(queryContext)
.build()
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.build()
),
ImmutableList.of(
new Object[]{6L}
)
);
}
@Test
public void testSelfJoin()
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
testQuery(
"SELECT COUNT(*) FROM druid.foo x, druid.foo y\n",
queryContext,
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(
join(
new TableDataSource(CalciteTests.DATASOURCE1),
new QueryDataSource(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns(
ImmutableList.of(
"__time",
"cnt",
"dim1",
"dim2",
"dim3",
"m1",
"m2",
"unique_dim1"
)
)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(queryContext)
.build()
),
"j0.",
"1",
JoinType.INNER
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(queryContext)
.build()
),
ImmutableList.of(
new Object[]{36L}
)
);
}
}

View File

@ -164,27 +164,6 @@ public class DecoupledPlanningCalciteQueryTest extends CalciteQueryTest
}
@Override
@Ignore
public void testSelfJoin()
{
}
@Override
@Ignore
public void testTwoExactCountDistincts()
{
}
@Override
@Ignore
public void testViewAndJoin()
{
}
@Override
@Ignore
public void testGroupByWithSortOnPostAggregationDefault()
@ -234,34 +213,6 @@ public class DecoupledPlanningCalciteQueryTest extends CalciteQueryTest
}
@Override
@Ignore
public void testUseTimeFloorInsteadOfGranularityOnJoinResult()
{
}
@Override
@Ignore
public void testMinMaxAvgDailyCountWithLimit()
{
}
@Override
@Ignore
public void testExactCountDistinctOfSemiJoinResult()
{
}
@Override
@Ignore
public void testMaxSubqueryRows()
{
}
@Override
@Ignore
public void testExactCountDistinctUsingSubqueryWithWherePushDown()
@ -276,27 +227,6 @@ public class DecoupledPlanningCalciteQueryTest extends CalciteQueryTest
}
@Override
@Ignore
public void testUsingSubqueryAsFilterOnTwoColumns()
{
}
@Override
@Ignore
public void testUsingSubqueryAsFilterWithInnerSort()
{
}
@Override
@Ignore
public void testUsingSubqueryWithLimit()
{
}
@Override
@Ignore
public void testPostAggWithTimeseries()
@ -324,20 +254,6 @@ public class DecoupledPlanningCalciteQueryTest extends CalciteQueryTest
}
@Override
@Ignore
public void testEmptyGroupWithOffsetDoesntInfiniteLoop()
{
}
@Override
@Ignore
public void testJoinWithTimeDimension()
{
}
@Override
@Ignore
public void testSubqueryTypeMismatchWithLiterals()

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Ordering;
import com.google.common.io.Closeables;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.FrameBasedInlineDataSource;
import org.apache.druid.query.InlineDataSource;
import org.apache.druid.query.LookupDataSource;
import org.apache.druid.query.Query;
@ -32,6 +33,7 @@ import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainer;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import org.apache.druid.segment.FrameBasedInlineSegmentWrangler;
import org.apache.druid.segment.InlineSegmentWrangler;
import org.apache.druid.segment.LookupSegmentWrangler;
import org.apache.druid.segment.MapSegmentWrangler;
@ -129,6 +131,7 @@ public class SpecificSegmentsQuerySegmentWalker implements QuerySegmentWalker, C
new MapSegmentWrangler(
ImmutableMap.<Class<? extends DataSource>, SegmentWrangler>builder()
.put(InlineDataSource.class, new InlineSegmentWrangler())
.put(FrameBasedInlineDataSource.class, new FrameBasedInlineSegmentWrangler())
.put(
LookupDataSource.class,
new LookupSegmentWrangler(LOOKUP_EXTRACTOR_FACTORY_CONTAINER_PROVIDER)