()
diff --git a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java
new file mode 100644
index 00000000000..a0b7bda150d
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java
@@ -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.
+ *
+ * The rows are backed by a sequence of {@link FrameSignaturePair}, which contain the Frame representation of the rows
+ * represented by the datasource.
+ *
+ * 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 frames;
+ final RowSignature rowSignature;
+
+ public FrameBasedInlineDataSource(
+ List frames,
+ RowSignature rowSignature
+ )
+ {
+ this.frames = frames;
+ this.rowSignature = rowSignature;
+ }
+
+ public List getFrames()
+ {
+ return frames;
+ }
+
+ public RowSignature getRowSignature()
+ {
+ return rowSignature;
+ }
+
+ public Sequence getRowsAsSequence()
+ {
+
+ final Sequence 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 selectors = rowSignature
+ .getColumnNames()
+ .stream()
+ .map(columnSelectorFactory::makeColumnValueSelector)
+ .collect(Collectors.toList());
+
+ return Sequences.simple(
+ () -> new Iterator()
+ {
+ @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 getTableNames()
+ {
+ return Collections.emptySet();
+ }
+
+ @Override
+ public List getChildren()
+ {
+ return Collections.emptyList();
+ }
+
+ @Override
+ public DataSource withChildren(List 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 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());
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializer.java b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializer.java
new file mode 100644
index 00000000000..0a4feaca9bd
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializer.java
@@ -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
+{
+ 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 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 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);
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/FrameSignaturePair.java b/processing/src/main/java/org/apache/druid/query/FrameSignaturePair.java
new file mode 100644
index 00000000000..ef279ea73b8
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/FrameSignaturePair.java
@@ -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;
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java
index 3fb015d369e..a14eb63fe6f 100644
--- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java
+++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java
@@ -190,6 +190,7 @@ public class InlineDataSource implements DataSource
@JsonProperty("rows")
public List getRowsAsList()
{
+ Iterable rows = getRows();
return rows instanceof List ? ((List) rows) : Lists.newArrayList(rows);
}
diff --git a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java
new file mode 100644
index 00000000000..c5bb271213e
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java
@@ -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 getCursorFromIterable(Iterable rows, RowSignature rowSignature)
+ {
+ RowAdapter 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 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 getCursorFromSequence(Sequence rows, RowSignature rowSignature)
+ {
+ return getCursorFromIterable(
+ new Iterable()
+ {
+ Yielder yielder = Yielders.each(rows);
+
+ @Override
+ public Iterator iterator()
+ {
+ return new Iterator()
+ {
+ @Override
+ public boolean hasNext()
+ {
+ return !yielder.isDone();
+ }
+
+ @Override
+ public Object[] next()
+ {
+ Object[] retVal = yielder.get();
+ yielder = yielder.next(null);
+ return retVal;
+ }
+ };
+ }
+ },
+ rowSignature
+ );
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/QueryContext.java b/processing/src/main/java/org/apache/druid/query/QueryContext.java
index 624bc1cb3d1..147ebbec18e 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryContext.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryContext.java
@@ -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);
diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java
index f3106d9a601..ea21987bd18 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java
@@ -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";
diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java
index ee400f814fb..2657f4f49f7 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java
@@ -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> resultsAsFrames(
+ QueryType query,
+ Sequence resultSequence,
+ MemoryAllocatorFactory memoryAllocatorFactory,
+ boolean useNestedForUnknownTypes
+ )
+ {
+ return Optional.empty();
+ }
}
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
index 173e6babd0d..d13998c8340 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
@@ -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> resultsAsFrames(
+ GroupByQuery query,
+ Sequence 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 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.
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java b/processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java
new file mode 100644
index 00000000000..71ba425d457
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java
@@ -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 cursors;
+ private int currentCursor;
+
+ public ConcatCursor(
+ List 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 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();
+ }
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
index efcbe51c0c4..2246e349cbb 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java
@@ -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 make()
{
final List 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
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java
index 68b311e83a3..785b71ed42d 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java
@@ -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 eventsToAdd = new ArrayList<>(batchSize);
List 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) srv.getEvents()));
yielder = yielder.next(null);
count++;
}
- return new ScanResultValue(null, columns, eventsToAdd);
+ return new ScanResultValue(null, columns, eventsToAdd, rowSignature);
}
}
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryOffsetSequence.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryOffsetSequence.java
index 6685a40d973..57bf6672d70 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryOffsetSequence.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryOffsetSequence.java
@@ -117,7 +117,7 @@ public class ScanQueryOffsetSequence extends YieldingSequenceBase
{
@@ -187,13 +206,146 @@ public class ScanQueryQueryToolChest extends QueryToolChest> resultsAsFrames(
+ final ScanQuery query,
+ final Sequence resultSequence,
+ MemoryAllocatorFactory memoryAllocatorFactory,
+ boolean useNestedForUnknownTypes
+ )
+ {
+ final RowSignature defaultRowSignature = resultArraySignature(query);
+ Iterator resultSequenceIterator = new Iterator()
+ {
+ Yielder 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> retVal = () -> new Iterator>()
+ {
+ PeekingIterator scanResultValuePeekingIterator = Iterators.peekingIterator(resultSequenceIterator);
+
+ @Override
+ public boolean hasNext()
+ {
+ return scanResultValuePeekingIterator.hasNext();
+ }
+
+ @Override
+ public Sequence next()
+ {
+ final List 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 convertScanResultValuesToFrame(
+ List batch,
+ RowSignature rowSignature,
+ ScanQuery query,
+ MemoryAllocatorFactory memoryAllocatorFactory,
+ boolean useNestedForUnknownTypes
+ )
+ {
+ Preconditions.checkNotNull(rowSignature, "'rowSignature' must be provided");
+
+ List cursors = new ArrayList<>();
+
+ for (ScanResultValue scanResultValue : batch) {
+ final List rows = (List) scanResultValue.getEvents();
+ final Function, Object[]> mapper = getResultFormatMapper(query.getResultFormat(), rowSignature.getColumnNames());
+ final Iterable 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 frames = FrameCursorUtils.cursorToFrames(
+ concatCursor,
+ frameWriterFactory
+ );
+
+ return frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature));
+ }
+
@Override
public Sequence resultsAsArrays(final ScanQuery query, final Sequence resultSequence)
{
- final List 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 fields)
+ {
+ Function, Object[]> mapper;
+
+ switch (resultFormat) {
case RESULT_FORMAT_LIST:
mapper = (Map row) -> {
final Object[] rowArray = new Object[fields.size()];
@@ -220,16 +372,8 @@ public class ScanQueryQueryToolChest extends QueryToolChest {
- // 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;
}
}
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java
index aa368102dd4..ab65d3fc6c3 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java
@@ -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
{
@@ -45,17 +47,25 @@ public class ScanResultValue implements Comparable
private final String segmentId;
private final List columns;
private final Object events;
+ private final RowSignature rowSignature;
@JsonCreator
public ScanResultValue(
@JsonProperty("segmentId") @Nullable String segmentId,
@JsonProperty("columns") List 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 columns, Object events)
+ {
+ this(segmentId, columns, events, null);
}
@Nullable
@@ -77,6 +87,14 @@ public class ScanResultValue implements Comparable
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
List singleEventScanResultValues = new ArrayList<>();
List events = (List) 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 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
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
"segmentId='" + segmentId + '\'' +
", columns=" + columns +
", events=" + events +
+ ", rowSignature=" + rowSignature +
'}';
}
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
index 5a4417aa719..54a17d267d1 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
@@ -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 fields = resultArraySignature(query).getColumnNames();
-
return Sequences.map(
resultSequence,
result -> {
@@ -451,6 +462,39 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest> resultsAsFrames(
+ TimeseriesQuery query,
+ Sequence> 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 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> makeComputeManipulatorFn(
final TimeseriesQuery query,
final MetricManipulationFn fn,
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
index a7785a01be1..0109c9828e9 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
@@ -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> resultsAsFrames(
+ TopNQuery query,
+ Sequence> 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 frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory);
+
+ return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature)));
+ }
+
static class ThresholdAdjustingQueryRunner implements QueryRunner>
{
private final QueryRunner> runner;
diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java
index c0c95251177..eb7a870c172 100644
--- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java
@@ -48,7 +48,7 @@ public class RowBasedCursor implements Cursor
private long rowId = 0;
- RowBasedCursor(
+ public RowBasedCursor(
final RowWalker rowWalker,
final RowAdapter rowAdapter,
@Nullable final Filter filter,
diff --git a/processing/src/main/java/org/apache/druid/segment/RowWalker.java b/processing/src/main/java/org/apache/druid/segment/RowWalker.java
index 9f5c89a6437..d6241f197e8 100644
--- a/processing/src/main/java/org/apache/druid/segment/RowWalker.java
+++ b/processing/src/main/java/org/apache/druid/segment/RowWalker.java
@@ -41,7 +41,7 @@ public class RowWalker
@Nullable // null = closed
private Yielder rowYielder;
- RowWalker(final Sequence rowSequence, final RowAdapter rowAdapter)
+ public RowWalker(final Sequence rowSequence, final RowAdapter rowAdapter)
{
this.rowSequence = rowSequence;
this.timestampFunction = rowAdapter.timestampFunction();
diff --git a/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java b/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java
index 770d3632bd2..0fa8935533d 100644
--- a/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java
+++ b/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java
@@ -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
{
diff --git a/processing/src/main/java/org/apache/druid/segment/join/FrameBasedInlineJoinableFactory.java b/processing/src/main/java/org/apache/druid/segment/join/FrameBasedInlineJoinableFactory.java
new file mode 100644
index 00000000000..b35bda644a8
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/join/FrameBasedInlineJoinableFactory.java
@@ -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 build(DataSource dataSource, JoinConditionAnalysis condition)
+ {
+ FrameBasedInlineDataSource frameBasedInlineDataSource = (FrameBasedInlineDataSource) dataSource;
+
+ if (condition.canHashJoin()) {
+ final Set rightKeyColumns = condition.getRightEquiConditionKeys();
+ return Optional.of(
+ new IndexedTableJoinable(
+ new FrameBasedIndexedTable(
+ frameBasedInlineDataSource,
+ rightKeyColumns,
+ DateTimes.nowUtc().toString()
+ )
+ )
+ );
+ }
+
+ return Optional.empty();
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java b/processing/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java
index 4eee53fde5d..a265e15279f 100644
--- a/processing/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java
+++ b/processing/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java
@@ -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 build(final DataSource dataSource, final JoinConditionAnalysis condition)
{
- final InlineDataSource inlineDataSource = (InlineDataSource) dataSource;
+ InlineDataSource inlineDataSource = (InlineDataSource) dataSource;
if (condition.canHashJoin()) {
final Set rightKeyColumns = condition.getRightEquiConditionKeys();
diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java
new file mode 100644
index 00000000000..a7de99905c3
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java
@@ -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 keyColumns;
+ private final RowSignature rowSignature;
+ private final String version;
+ private final List keyColumnsIndexes;
+ private final int numRows;
+ private final List frameQueryableIndexes = new ArrayList<>();
+ private final List cumulativeRowCount = new ArrayList<>();
+
+
+ public FrameBasedIndexedTable(
+ final FrameBasedInlineDataSource frameBasedInlineDataSource,
+ final Set 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 indexBuilders = new ArrayList<>(rowSignature.size());
+ final List 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 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 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 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 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> columnValueSelectors = new ArrayList<>();
+ final Set 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 acquireReferences()
+ {
+ return Optional.of(
+ () -> {
+ }
+ );
+ }
+
+ private List createColumnReaders(RowSignature rowSignature)
+ {
+ final List 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
+ *
+ * Note: row is the index (therefore it is 0-indexed)
+ */
+ private int binSearch(List 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
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java b/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java
new file mode 100644
index 00000000000..fbbc089255b
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java
@@ -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.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.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.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 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);
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java
new file mode 100644
index 00000000000..1acaceabbd6
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java
@@ -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 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 expectedRows
+ )
+ {
+ List 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));
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/query/scan/ConcatCursorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ConcatCursorTest.java
new file mode 100644
index 00000000000..afe7a95cfdc
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/query/scan/ConcatCursorTest.java
@@ -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 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 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 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 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 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 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);
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java
index bdd9b01838c..63c5b381921 100644
--- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java
+++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java
@@ -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 ARRAY_RESULTS = ImmutableList.of(
+ private static final List ARRAY_RESULTS_1 = ImmutableList.of(
new Object[]{null, 3.2},
new Object[]{"x", "y"}
);
+ private static final List ARRAY_RESULTS_2 = ImmutableList.of(
+ new Object[]{"str1", 3.2},
+ new Object[]{"str2", 3.3}
+ );
+
+ private static final List 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 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 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 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 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 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 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 makeResults(final ScanQuery.ResultFormat resultFormat)
+ private static Sequence makeResults1(final ScanQuery.ResultFormat resultFormat)
{
final List 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 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 results2()
+ {
+ final List rows = new ArrayList<>();
+
+ ARRAY_RESULTS_2.forEach(arr -> {
+ final Map 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 results3()
+ {
+ final List rows = new ArrayList<>();
+
+ ARRAY_RESULTS_3.forEach(arr -> {
+ final Map 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()
+ )
+ )
+ );
+ }
}
diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java
new file mode 100644
index 00000000000..bbb5704a561
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java
@@ -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 DATASOURCE_ROWS =
+ ImmutableList.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 KEY_COLUMNS = ImmutableSet.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 originalRows = dataSource.getRowsAsSequence().toList();
+ for (int i = 0; i < numRows; ++i) {
+ Object original = originalRows.get(i)[columnNumber];
+ Assert.assertEquals(original, reader.read(i));
+ }
+ }
+}
diff --git a/server/src/main/java/org/apache/druid/guice/JoinableFactoryModule.java b/server/src/main/java/org/apache/druid/guice/JoinableFactoryModule.java
index e8f803dc773..102f1b349b3 100644
--- a/server/src/main/java/org/apache/druid/guice/JoinableFactoryModule.java
+++ b/server/src/main/java/org/apache/druid/guice/JoinableFactoryModule.java
@@ -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 JoinableFactory>> FACTORY_MAPPINGS =
ImmutableMap.of(
InlineDataSource.class, InlineJoinableFactory.class,
+ FrameBasedInlineDataSource.class, FrameBasedInlineJoinableFactory.class,
LookupDataSource.class, LookupJoinableFactory.class,
GlobalTableDataSource.class, BroadcastTableJoinableFactory.class
);
diff --git a/server/src/main/java/org/apache/druid/guice/SegmentWranglerModule.java b/server/src/main/java/org/apache/druid/guice/SegmentWranglerModule.java
index 93b7090a083..a6cc1ce987b 100644
--- a/server/src/main/java/org/apache/druid/guice/SegmentWranglerModule.java
+++ b/server/src/main/java/org/apache/druid/guice/SegmentWranglerModule.java
@@ -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 SegmentWrangler>> WRANGLER_MAPPINGS =
ImmutableMap.of(
InlineDataSource.class, InlineSegmentWrangler.class,
+ FrameBasedInlineDataSource.class, FrameBasedInlineSegmentWrangler.class,
LookupDataSource.class, LookupSegmentWrangler.class
);
diff --git a/server/src/main/java/org/apache/druid/segment/FrameBasedInlineSegmentWrangler.java b/server/src/main/java/org/apache/druid/segment/FrameBasedInlineSegmentWrangler.java
new file mode 100644
index 00000000000..407bbf750fa
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/segment/FrameBasedInlineSegmentWrangler.java
@@ -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 getSegmentsForIntervals(
+ DataSource dataSource,
+ Iterable intervals
+ )
+ {
+ final FrameBasedInlineDataSource frameBasedInlineDataSource = (FrameBasedInlineDataSource) dataSource;
+
+ return () -> frameBasedInlineDataSource
+ .getFrames()
+ .stream()
+ .map(
+ frameSignaturePair -> new FrameSegment(
+ frameSignaturePair.getFrame(),
+ FrameReader.create(frameSignaturePair.getRowSignature()),
+ SegmentId.dummy(SEGMENT_ID)
+ )
+ )
+ .iterator();
+
+ }
+}
diff --git a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java
index 61f7817e2fd..d66055229bc 100644
--- a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java
+++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java
@@ -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 > InlineDataSource toInlineDataSource(
+ private static > DataSource toInlineDataSource(
+ final QueryType query,
+ final Sequence results,
+ final QueryToolChest 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 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 > Optional materializeResultsAsFrames(
+ final QueryType query,
+ final Sequence results,
+ final QueryToolChest toolChest,
+ final AtomicInteger limitAccumulator,
+ final AtomicLong memoryLimitAccumulator,
+ long memoryLimit,
+ boolean useNestedForUnknownTypeInSubquery
+ )
+ {
+ Optional> 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 frames = framesOptional.get();
+ List 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}
+ */
+ private static > DataSource materializeResultsAsArray(
final QueryType query,
final Sequence results,
final QueryToolChest 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 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);
}
diff --git a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java
new file mode 100644
index 00000000000..f185a5a5326
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java
@@ -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;
+ }
+}
diff --git a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java
index d6b62822115..9b72670c3fd 100644
--- a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java
+++ b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java
@@ -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 +
diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java
index 491ee4fd98b..687ca2ef548 100644
--- a/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java
+++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java
@@ -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(),
diff --git a/server/src/test/java/org/apache/druid/initialization/ServerConfigTest.java b/server/src/test/java/org/apache/druid/initialization/ServerConfigTest.java
index f6161e6fa57..96745e163a0 100644
--- a/server/src/test/java/org/apache/druid/initialization/ServerConfigTest.java
+++ b/server/src/test/java/org/apache/druid/initialization/ServerConfigTest.java
@@ -52,6 +52,8 @@ public class ServerConfigTest
defaultConfig.getDefaultQueryTimeout(),
defaultConfig.getMaxScatterGatherBytes(),
defaultConfig.getMaxSubqueryRows(),
+ defaultConfig.getMaxSubqueryBytes(),
+ defaultConfig.isuseNestedForUnknownTypeInSubquery(),
defaultConfig.getMaxQueryTimeout(),
defaultConfig.getMaxRequestHeaderSize(),
defaultConfig.getGracefulShutdownTimeout(),
diff --git a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java
index 967d9342d78..a8aaf10da00 100644
--- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java
+++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java
@@ -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., 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 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.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;
}
diff --git a/server/src/test/java/org/apache/druid/server/QueryStackTests.java b/server/src/test/java/org/apache/druid/server/QueryStackTests.java
index 7011a0283df..1c23edf3b9a 100644
--- a/server/src/test/java/org/apache/druid/server/QueryStackTests.java
+++ b/server/src/test/java/org/apache/druid/server/QueryStackTests.java
@@ -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 setBuilder = ImmutableSet.builder();
ImmutableMap.Builder, 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);
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
index b93228e076f..2ba9ce6dc73 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
@@ -239,6 +239,10 @@ public class BaseCalciteQueryTest extends CalciteTestBase
QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE
);
+ public static final Map 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 getTimeseriesContextWithFloorTime(
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
index 792c25a5e1d..fcf2c9dbb2b 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
@@ -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 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()
{
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java
new file mode 100644
index 00000000000..40c04f35807
--- /dev/null
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java
@@ -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 queryContext;
+
+ public CalciteSubqueryTest(
+ String testName,
+ Map queryContext
+ )
+ {
+ this.testName = testName;
+ this.queryContext = queryContext;
+ }
+
+ @Parameterized.Parameters(name = "{0}")
+ public static Iterable constructorFeeder()
+ {
+ final List 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 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 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 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 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}
+ )
+ );
+ }
+}
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java
index dfd1acad0cf..0c667325b4d 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java
@@ -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()
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java
index 6bc8e6ca728..b43a6515956 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java
@@ -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., SegmentWrangler>builder()
.put(InlineDataSource.class, new InlineSegmentWrangler())
+ .put(FrameBasedInlineDataSource.class, new FrameBasedInlineSegmentWrangler())
.put(
LookupDataSource.class,
new LookupSegmentWrangler(LOOKUP_EXTRACTOR_FACTORY_CONTAINER_PROVIDER)