Improve the fallback strategy when the broker is unable to materialize the subquery's results as frames for estimating the bytes (#16679)

Better fallback strategy when the broker is unable to materialize the subquery's results as frames for estimating the bytes:
a. We don't touch the subquery sequence till we know that we can materialize the result as frames
This commit is contained in:
Laksh Singla 2024-07-12 21:49:12 +05:30 committed by GitHub
parent 197c54f673
commit 3a1b437056
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
6 changed files with 190 additions and 10 deletions

View File

@ -24,8 +24,11 @@ import com.google.common.collect.ImmutableMap;
import com.google.inject.Injector; import com.google.inject.Injector;
import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.config.NullHandling;
import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.Druids; 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.QueryDataSource;
import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
@ -33,6 +36,7 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.aggregation.cardinality.CardinalityAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.SketchQueryContext; import org.apache.druid.query.aggregation.datasketches.SketchQueryContext;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory; import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule; import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule;
@ -43,6 +47,7 @@ import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchTo
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToRankPostAggregator; import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToRankPostAggregator;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToStringPostAggregator; import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToStringPostAggregator;
import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchSqlAggregatorTest.DoublesSketchComponentSupplier; import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchSqlAggregatorTest.DoublesSketchComponentSupplier;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator;
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DefaultDimensionSpec;
@ -53,6 +58,7 @@ import org.apache.druid.segment.IndexBuilder;
import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.join.JoinType;
import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
@ -263,6 +269,143 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
); );
} }
@Test
public void testSubqueryWithNestedGroupBy()
{
final List<Object[]> expectedResults = ImmutableList.of(
new Object[]{946684800000L, "", 1L, "val1"},
new Object[]{946684800000L, "1", 1L, "val1"},
new Object[]{946684800000L, "10.1", 1L, "val1"},
new Object[]{946684800000L, "2", 1L, "val1"},
new Object[]{946684800000L, "abc", 1L, "val1"},
new Object[]{946684800000L, "def", 1L, "val1"}
);
testQuery(
"SELECT\n"
+ " MILLIS_TO_TIMESTAMP(946684800000) AS __time,\n"
+ " alias.\"user\",\n"
+ " alias.days,\n"
+ " (CASE WHEN alias.days < quantiles.first_quartile THEN 'val2' \n"
+ " WHEN alias.days >= quantiles.first_quartile AND alias.days < quantiles.third_quartile THEN 'val3' \n"
+ " WHEN alias.days >= quantiles.third_quartile THEN 'val1' END) AS val4\n"
+ "FROM (\n"
+ " SELECT\n"
+ " APPROX_QUANTILE_DS(alias.days, 0.25) AS first_quartile,\n"
+ " APPROX_QUANTILE_DS(alias.days, 0.75) AS third_quartile\n"
+ " FROM (\n"
+ " SELECT\n"
+ " dim1 \"user\",\n"
+ " COUNT(DISTINCT __time) AS days\n"
+ " FROM \"foo\"\n"
+ " GROUP BY 1\n"
+ " ) AS alias\n"
+ ") AS quantiles, (\n"
+ " SELECT\n"
+ " dim1 \"user\",\n"
+ " COUNT(DISTINCT __time) AS days\n"
+ " FROM \"foo\"\n"
+ " GROUP BY 1\n"
+ ") AS alias\n",
ImmutableMap.<String, Object>builder()
.putAll(QUERY_CONTEXT_DEFAULT)
.put(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000")
// Disallows the fallback to row based limiting
.put(QueryContexts.MAX_SUBQUERY_ROWS_KEY, "10")
.build(),
ImmutableList.of(
newScanQueryBuilder()
.dataSource(
JoinDataSource.create(
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
GroupByQuery.builder()
.setDataSource("foo")
.setQuerySegmentSpec(querySegmentSpec(Intervals.ETERNITY))
.setGranularity(Granularities.ALL)
.addDimension(new DefaultDimensionSpec(
"dim1",
"d0",
ColumnType.STRING
))
.addAggregator(new CardinalityAggregatorFactory(
"a0:a",
null,
Collections.singletonList(new DefaultDimensionSpec(
"__time",
"__time",
ColumnType.LONG
)),
false,
true
))
.setPostAggregatorSpecs(new HyperUniqueFinalizingPostAggregator(
"a0",
"a0:a"
))
.build()
)
)
.setQuerySegmentSpec(querySegmentSpec(Intervals.ETERNITY))
.setGranularity(Granularities.ALL)
.addAggregator(new DoublesSketchAggregatorFactory("_a0:agg", "a0", 128))
.setPostAggregatorSpecs(
new DoublesSketchToQuantilePostAggregator(
"_a0",
new FieldAccessPostAggregator("_a0:agg", "_a0:agg"),
0.25
),
new DoublesSketchToQuantilePostAggregator(
"_a1",
new FieldAccessPostAggregator("_a0:agg", "_a0:agg"),
0.75
)
)
.build()
),
new QueryDataSource(
GroupByQuery.builder()
.setDataSource("foo")
.setQuerySegmentSpec(querySegmentSpec(Intervals.ETERNITY))
.setGranularity(Granularities.ALL)
.addDimension(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))
.addAggregator(new CardinalityAggregatorFactory(
"a0",
null,
Collections.singletonList(new DefaultDimensionSpec(
"__time",
"__time",
ColumnType.LONG
)),
false,
true
))
.build()
),
"j0.",
"1",
JoinType.INNER,
null,
TestExprMacroTable.INSTANCE,
null
)
)
.intervals(querySegmentSpec(Intervals.ETERNITY))
.virtualColumns(
new ExpressionVirtualColumn("v0", "946684800000", ColumnType.LONG, TestExprMacroTable.INSTANCE),
new ExpressionVirtualColumn("v1", "case_searched((\"j0.a0\" < \"_a0\"),'val2',((\"j0.a0\" >= \"_a0\") && (\"j0.a0\" < \"_a1\")),'val3',(\"j0.a0\" >= \"_a1\"),'val1',null)", ColumnType.STRING, TestExprMacroTable.INSTANCE)
)
.columns("j0.a0", "j0.d0", "v0", "v1")
.build()
),
expectedResults
);
}
@Test @Test
public void testQuantileOnCastedString() public void testQuantileOnCastedString()
{ {

View File

@ -23,6 +23,7 @@ import org.apache.druid.error.DruidException;
import org.apache.druid.frame.Frame; import org.apache.druid.frame.Frame;
import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriter;
import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.frame.write.FrameWriterFactory;
import org.apache.druid.frame.write.UnsupportedColumnTypeException;
import org.apache.druid.java.util.common.Intervals; 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.Sequence;
import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Sequences;
@ -32,6 +33,7 @@ import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.segment.Cursor; import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.filter.BoundFilter; import org.apache.druid.segment.filter.BoundFilter;
import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.Filters;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -100,13 +102,18 @@ public class FrameCursorUtils
/** /**
* Writes a {@link Cursor} to a sequence of {@link Frame}. This method iterates over the rows of the cursor, * Writes a {@link Cursor} to a sequence of {@link Frame}. This method iterates over the rows of the cursor,
* and writes the columns to the frames. The iterable is lazy, and it traverses the required portion of the cursor * and writes the columns to the frames. The iterable is lazy, and it traverses the required portion of the cursor
* as required * as required.
* <p>
* If the type is missing from the signature, the method throws an exception without advancing/modifying/closing the
* cursor
*/ */
public static Iterable<Frame> cursorToFramesIterable( public static Iterable<Frame> cursorToFramesIterable(
final Cursor cursor, final Cursor cursor,
final FrameWriterFactory frameWriterFactory final FrameWriterFactory frameWriterFactory
) )
{ {
throwIfColumnsHaveUnknownType(frameWriterFactory.signature());
return () -> new Iterator<Frame>() return () -> new Iterator<Frame>()
{ {
@Override @Override
@ -158,7 +165,19 @@ public class FrameCursorUtils
final FrameWriterFactory frameWriterFactory final FrameWriterFactory frameWriterFactory
) )
{ {
return Sequences.simple(cursorToFramesIterable(cursor, frameWriterFactory)); return Sequences.simple(cursorToFramesIterable(cursor, frameWriterFactory));
} }
/**
* Throws {@link UnsupportedColumnTypeException} if the row signature has columns with unknown types. This is used to
* pre-determine if the frames can be materialized as rows, without touching the resource generating the frames.
*/
public static void throwIfColumnsHaveUnknownType(final RowSignature rowSignature)
{
for (int i = 0; i < rowSignature.size(); ++i) {
if (!rowSignature.getColumnType(i).isPresent()) {
throw new UnsupportedColumnTypeException(rowSignature.getColumnName(i), null);
}
}
}
} }

View File

@ -816,6 +816,8 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<ResultRow, GroupB
? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature) ? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature)
: rowSignature; : rowSignature;
FrameCursorUtils.throwIfColumnsHaveUnknownType(modifiedRowSignature);
FrameWriterFactory frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory( FrameWriterFactory frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory(
memoryAllocatorFactory, memoryAllocatorFactory,
modifiedRowSignature, modifiedRowSignature,

View File

@ -485,6 +485,8 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
RowSignature modifiedRowSignature = useNestedForUnknownTypes RowSignature modifiedRowSignature = useNestedForUnknownTypes
? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature) ? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature)
: rowSignature; : rowSignature;
FrameCursorUtils.throwIfColumnsHaveUnknownType(modifiedRowSignature);
FrameWriterFactory frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory( FrameWriterFactory frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory(
memoryAllocatorFactory, memoryAllocatorFactory,
modifiedRowSignature, modifiedRowSignature,

View File

@ -569,6 +569,8 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
RowSignature modifiedRowSignature = useNestedForUnknownTypes RowSignature modifiedRowSignature = useNestedForUnknownTypes
? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature) ? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature)
: rowSignature; : rowSignature;
FrameCursorUtils.throwIfColumnsHaveUnknownType(modifiedRowSignature);
FrameWriterFactory frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory( FrameWriterFactory frameWriterFactory = FrameWriters.makeColumnBasedFrameWriterFactory(
memoryAllocatorFactory, memoryAllocatorFactory,
rowSignature, rowSignature,

View File

@ -746,6 +746,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
{ {
Optional<Sequence<FrameSignaturePair>> framesOptional; Optional<Sequence<FrameSignaturePair>> framesOptional;
boolean startedAccumulating = false;
try { try {
framesOptional = toolChest.resultsAsFrames( framesOptional = toolChest.resultsAsFrames(
query, query,
@ -760,6 +761,9 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
Sequence<FrameSignaturePair> frames = framesOptional.get(); Sequence<FrameSignaturePair> frames = framesOptional.get();
List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>(); List<FrameSignaturePair> frameSignaturePairs = new ArrayList<>();
startedAccumulating = true;
frames.forEach( frames.forEach(
frame -> { frame -> {
limitAccumulator.addAndGet(frame.getFrame().numRows()); limitAccumulator.addAndGet(frame.getFrame().numRows());
@ -772,21 +776,29 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
} }
); );
return Optional.of(new FrameBasedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query))); return Optional.of(new FrameBasedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query)));
}
catch (ResourceLimitExceededException e) {
throw e;
} }
catch (UnsupportedColumnTypeException e) { catch (UnsupportedColumnTypeException e) {
subqueryStatsProvider.incrementSubqueriesFallingBackDueToUnsufficientTypeInfo(); subqueryStatsProvider.incrementSubqueriesFallingBackDueToUnsufficientTypeInfo();
log.debug(e, "Type info in signature insufficient to materialize rows as frames."); log.debug(e, "Type info in signature insufficient to materialize rows as frames.");
return Optional.empty(); return Optional.empty();
} }
catch (ResourceLimitExceededException e) {
throw e;
}
catch (Exception e) { catch (Exception e) {
subqueryStatsProvider.incrementSubqueriesFallingBackDueToUnknownReason(); if (startedAccumulating) {
log.debug(e, "Unable to materialize the results as frames due to an unhandleable exception " // If we have opened the resultSequence, we can't fall back safely as the resultSequence might hold some resources
+ "while conversion. Defaulting to materializing the results as rows"); // that we release on exception, and we need to throw the exception to disable the 'maxSubqueryBytes' configuration
return Optional.empty(); throw DruidException.defensive()
.build(
e,
"Unable to materialize the results as frames for estimating the byte footprint. "
+ "Please disable the 'maxSubqueryBytes' by setting it to 'disabled' in the query context or removing it altogether "
+ "from the query context and/or the server config."
);
} else {
return Optional.empty();
}
} }
} }