mirror of https://github.com/apache/druid.git
QueryableIndexSegment: Re-use time boundary inspector. (#17397)
This patch re-uses timeBoundaryInspector for each cursor holder, which enables caching of minDataTimestamp and maxDataTimestamp. Fixes a performance regression introduced in #16533, where these fields stopped being cached across cursors. Prior to that patch, they were cached in the QueryableIndexStorageAdapter.
This commit is contained in:
parent
d8e4be654f
commit
9c25226e06
|
@ -37,10 +37,24 @@ import java.util.List;
|
|||
public class QueryableIndexCursorFactory implements CursorFactory
|
||||
{
|
||||
private final QueryableIndex index;
|
||||
private final TimeBoundaryInspector timeBoundaryInspector;
|
||||
|
||||
public QueryableIndexCursorFactory(QueryableIndex index)
|
||||
/**
|
||||
* Constructor that accepts a {@link QueryableIndexTimeBoundaryInspector} that is in use elsewhere, promoting
|
||||
* efficient re-use.
|
||||
*/
|
||||
public QueryableIndexCursorFactory(QueryableIndex index, TimeBoundaryInspector timeBoundaryInspector)
|
||||
{
|
||||
this.index = index;
|
||||
this.timeBoundaryInspector = timeBoundaryInspector;
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor that creates a new {@link QueryableIndexTimeBoundaryInspector}.
|
||||
*/
|
||||
public QueryableIndexCursorFactory(QueryableIndex index)
|
||||
{
|
||||
this(index, QueryableIndexTimeBoundaryInspector.create(index));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -48,7 +62,11 @@ public class QueryableIndexCursorFactory implements CursorFactory
|
|||
{
|
||||
QueryableProjection<QueryableIndex> projection = index.getProjection(spec);
|
||||
if (projection != null) {
|
||||
return new QueryableIndexCursorHolder(projection.getRowSelector(), projection.getCursorBuildSpec())
|
||||
return new QueryableIndexCursorHolder(
|
||||
projection.getRowSelector(),
|
||||
projection.getCursorBuildSpec(),
|
||||
timeBoundaryInspector
|
||||
)
|
||||
{
|
||||
@Override
|
||||
protected ColumnSelectorFactory makeColumnSelectorFactoryForOffset(
|
||||
|
@ -86,7 +104,7 @@ public class QueryableIndexCursorFactory implements CursorFactory
|
|||
}
|
||||
};
|
||||
}
|
||||
return new QueryableIndexCursorHolder(index, CursorBuildSpec.builder(spec).build());
|
||||
return new QueryableIndexCursorHolder(index, CursorBuildSpec.builder(spec).build(), timeBoundaryInspector);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.google.common.base.Supplier;
|
|||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.query.BaseQuery;
|
||||
|
@ -58,6 +57,7 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
|||
import org.apache.druid.segment.vector.VectorCursor;
|
||||
import org.apache.druid.segment.vector.VectorOffset;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -86,7 +86,8 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
|
||||
public QueryableIndexCursorHolder(
|
||||
QueryableIndex index,
|
||||
CursorBuildSpec cursorBuildSpec
|
||||
CursorBuildSpec cursorBuildSpec,
|
||||
TimeBoundaryInspector timeBoundaryInspector
|
||||
)
|
||||
{
|
||||
this.index = index;
|
||||
|
@ -109,6 +110,7 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
this.resourcesSupplier = Suppliers.memoize(
|
||||
() -> new CursorResources(
|
||||
index,
|
||||
timeBoundaryInspector,
|
||||
virtualColumns,
|
||||
Cursors.getTimeOrdering(ordering),
|
||||
interval,
|
||||
|
@ -159,9 +161,8 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
final CursorResources resources = resourcesSupplier.get();
|
||||
final FilterBundle filterBundle = resources.filterBundle;
|
||||
final int numRows = resources.numRows;
|
||||
final long minDataTimestamp = resources.minDataTimestamp;
|
||||
final long maxDataTimestamp = resources.maxDataTimestamp;
|
||||
final NumericColumn timestamps = resources.timestamps;
|
||||
final long minDataTimestamp = resources.timeBoundaryInspector.getMinTime().getMillis();
|
||||
final long maxDataTimestamp = resources.timeBoundaryInspector.getMaxTime().getMillis();
|
||||
final ColumnCache columnCache = resources.columnCache;
|
||||
final Order timeOrder = resources.timeOrder;
|
||||
|
||||
|
@ -180,13 +181,13 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
|
||||
if (timeOrder == Order.ASCENDING) {
|
||||
for (; baseOffset.withinBounds(); baseOffset.increment()) {
|
||||
if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) {
|
||||
if (resources.getTimestampsColumn().getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
} else if (timeOrder == Order.DESCENDING) {
|
||||
for (; baseOffset.withinBounds(); baseOffset.increment()) {
|
||||
if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) {
|
||||
if (resources.getTimestampsColumn().getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -197,14 +198,14 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
if (timeOrder == Order.ASCENDING) {
|
||||
offset = new AscendingTimestampCheckingOffset(
|
||||
baseOffset,
|
||||
timestamps,
|
||||
resources.getTimestampsColumn(),
|
||||
timeEnd,
|
||||
maxDataTimestamp < timeEnd
|
||||
);
|
||||
} else if (timeOrder == Order.DESCENDING) {
|
||||
offset = new DescendingTimestampCheckingOffset(
|
||||
baseOffset,
|
||||
timestamps,
|
||||
resources.getTimestampsColumn(),
|
||||
timeStart,
|
||||
minDataTimestamp >= timeStart
|
||||
);
|
||||
|
@ -244,9 +245,8 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
{
|
||||
final CursorResources resources = resourcesSupplier.get();
|
||||
final FilterBundle filterBundle = resources.filterBundle;
|
||||
final long minDataTimestamp = resources.minDataTimestamp;
|
||||
final long maxDataTimestamp = resources.maxDataTimestamp;
|
||||
final NumericColumn timestamps = resources.timestamps;
|
||||
final long minDataTimestamp = resources.timeBoundaryInspector.getMinTime().getMillis();
|
||||
final long maxDataTimestamp = resources.timeBoundaryInspector.getMaxTime().getMillis();
|
||||
final ColumnCache columnCache = resources.columnCache;
|
||||
final Order timeOrder = resources.timeOrder;
|
||||
|
||||
|
@ -265,13 +265,13 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
final int endOffset;
|
||||
|
||||
if (timeOrder != Order.NONE && interval.getStartMillis() > minDataTimestamp) {
|
||||
startOffset = timeSearch(timestamps, interval.getStartMillis(), 0, index.getNumRows());
|
||||
startOffset = timeSearch(resources.getTimestampsColumn(), interval.getStartMillis(), 0, index.getNumRows());
|
||||
} else {
|
||||
startOffset = 0;
|
||||
}
|
||||
|
||||
if (timeOrder != Order.NONE && interval.getEndMillis() <= maxDataTimestamp) {
|
||||
endOffset = timeSearch(timestamps, interval.getEndMillis(), startOffset, index.getNumRows());
|
||||
endOffset = timeSearch(resources.getTimestampsColumn(), interval.getEndMillis(), startOffset, index.getNumRows());
|
||||
} else {
|
||||
endOffset = index.getNumRows();
|
||||
}
|
||||
|
@ -660,17 +660,18 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
private static final class CursorResources implements Closeable
|
||||
{
|
||||
private final Closer closer;
|
||||
private final long minDataTimestamp;
|
||||
private final long maxDataTimestamp;
|
||||
private final TimeBoundaryInspector timeBoundaryInspector;
|
||||
private final int numRows;
|
||||
@Nullable
|
||||
private final FilterBundle filterBundle;
|
||||
private final NumericColumn timestamps;
|
||||
private final Order timeOrder;
|
||||
private final ColumnCache columnCache;
|
||||
@MonotonicNonNull
|
||||
private NumericColumn timestamps;
|
||||
|
||||
private CursorResources(
|
||||
QueryableIndex index,
|
||||
TimeBoundaryInspector timeBoundaryInspector,
|
||||
VirtualColumns virtualColumns,
|
||||
Order timeOrder,
|
||||
Interval interval,
|
||||
|
@ -681,6 +682,7 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
{
|
||||
this.closer = Closer.create();
|
||||
this.columnCache = new ColumnCache(index, closer);
|
||||
this.timeBoundaryInspector = timeBoundaryInspector;
|
||||
final ColumnSelectorColumnIndexSelector bitmapIndexSelector = new ColumnSelectorColumnIndexSelector(
|
||||
index.getBitmapFactoryForDimensions(),
|
||||
virtualColumns,
|
||||
|
@ -688,14 +690,10 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
);
|
||||
try {
|
||||
this.numRows = index.getNumRows();
|
||||
this.timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME);
|
||||
this.minDataTimestamp = DateTimes.utc(timestamps.getLongSingleValueRow(0)).getMillis();
|
||||
this.maxDataTimestamp = DateTimes.utc(timestamps.getLongSingleValueRow(timestamps.length() - 1)).getMillis();
|
||||
this.filterBundle = makeFilterBundle(
|
||||
computeFilterWithIntervalIfNeeded(
|
||||
timeBoundaryInspector,
|
||||
timeOrder,
|
||||
this.minDataTimestamp,
|
||||
this.maxDataTimestamp,
|
||||
interval,
|
||||
filter
|
||||
),
|
||||
|
@ -711,6 +709,14 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
}
|
||||
}
|
||||
|
||||
public NumericColumn getTimestampsColumn()
|
||||
{
|
||||
if (timestamps == null) {
|
||||
timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME);
|
||||
}
|
||||
return timestamps;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
|
@ -781,20 +787,20 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
|||
*/
|
||||
@Nullable
|
||||
private static Filter computeFilterWithIntervalIfNeeded(
|
||||
final TimeBoundaryInspector timeBoundaryInspector,
|
||||
final Order timeOrder,
|
||||
final long minDataTimestamp,
|
||||
final long maxDataTimestamp,
|
||||
final Interval interval,
|
||||
@Nullable final Filter filter
|
||||
)
|
||||
{
|
||||
if (timeOrder == Order.NONE
|
||||
&& (minDataTimestamp < interval.getStartMillis() || maxDataTimestamp >= interval.getEndMillis())) {
|
||||
&& (timeBoundaryInspector.getMinTime().getMillis() < interval.getStartMillis()
|
||||
|| timeBoundaryInspector.getMaxTime().getMillis() >= interval.getEndMillis())) {
|
||||
final RangeFilter timeFilter = new RangeFilter(
|
||||
ColumnHolder.TIME_COLUMN_NAME,
|
||||
ColumnType.LONG,
|
||||
minDataTimestamp < interval.getStartMillis() ? interval.getStartMillis() : null,
|
||||
maxDataTimestamp >= interval.getEndMillis() ? interval.getEndMillis() : null,
|
||||
timeBoundaryInspector.getMinTime().getMillis() < interval.getStartMillis() ? interval.getStartMillis() : null,
|
||||
timeBoundaryInspector.getMaxTime().getMillis() >= interval.getEndMillis() ? interval.getEndMillis() : null,
|
||||
false,
|
||||
true,
|
||||
null
|
||||
|
|
|
@ -46,8 +46,8 @@ public class QueryableIndexSegment implements Segment
|
|||
public QueryableIndexSegment(QueryableIndex index, final SegmentId segmentId)
|
||||
{
|
||||
this.index = index;
|
||||
this.cursorFactory = new QueryableIndexCursorFactory(index);
|
||||
this.timeBoundaryInspector = QueryableIndexTimeBoundaryInspector.create(index);
|
||||
this.cursorFactory = new QueryableIndexCursorFactory(index, timeBoundaryInspector);
|
||||
this.segmentId = segmentId;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue