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
|
public class QueryableIndexCursorFactory implements CursorFactory
|
||||||
{
|
{
|
||||||
private final QueryableIndex index;
|
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.index = index;
|
||||||
|
this.timeBoundaryInspector = timeBoundaryInspector;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor that creates a new {@link QueryableIndexTimeBoundaryInspector}.
|
||||||
|
*/
|
||||||
|
public QueryableIndexCursorFactory(QueryableIndex index)
|
||||||
|
{
|
||||||
|
this(index, QueryableIndexTimeBoundaryInspector.create(index));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -48,7 +62,11 @@ public class QueryableIndexCursorFactory implements CursorFactory
|
||||||
{
|
{
|
||||||
QueryableProjection<QueryableIndex> projection = index.getProjection(spec);
|
QueryableProjection<QueryableIndex> projection = index.getProjection(spec);
|
||||||
if (projection != null) {
|
if (projection != null) {
|
||||||
return new QueryableIndexCursorHolder(projection.getRowSelector(), projection.getCursorBuildSpec())
|
return new QueryableIndexCursorHolder(
|
||||||
|
projection.getRowSelector(),
|
||||||
|
projection.getCursorBuildSpec(),
|
||||||
|
timeBoundaryInspector
|
||||||
|
)
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
protected ColumnSelectorFactory makeColumnSelectorFactoryForOffset(
|
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
|
@Override
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.google.common.base.Supplier;
|
||||||
import com.google.common.base.Suppliers;
|
import com.google.common.base.Suppliers;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
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.io.Closer;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.druid.query.BaseQuery;
|
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.VectorCursor;
|
||||||
import org.apache.druid.segment.vector.VectorOffset;
|
import org.apache.druid.segment.vector.VectorOffset;
|
||||||
import org.apache.druid.utils.CloseableUtils;
|
import org.apache.druid.utils.CloseableUtils;
|
||||||
|
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -86,7 +86,8 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
|
|
||||||
public QueryableIndexCursorHolder(
|
public QueryableIndexCursorHolder(
|
||||||
QueryableIndex index,
|
QueryableIndex index,
|
||||||
CursorBuildSpec cursorBuildSpec
|
CursorBuildSpec cursorBuildSpec,
|
||||||
|
TimeBoundaryInspector timeBoundaryInspector
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.index = index;
|
this.index = index;
|
||||||
|
@ -109,6 +110,7 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
this.resourcesSupplier = Suppliers.memoize(
|
this.resourcesSupplier = Suppliers.memoize(
|
||||||
() -> new CursorResources(
|
() -> new CursorResources(
|
||||||
index,
|
index,
|
||||||
|
timeBoundaryInspector,
|
||||||
virtualColumns,
|
virtualColumns,
|
||||||
Cursors.getTimeOrdering(ordering),
|
Cursors.getTimeOrdering(ordering),
|
||||||
interval,
|
interval,
|
||||||
|
@ -159,9 +161,8 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
final CursorResources resources = resourcesSupplier.get();
|
final CursorResources resources = resourcesSupplier.get();
|
||||||
final FilterBundle filterBundle = resources.filterBundle;
|
final FilterBundle filterBundle = resources.filterBundle;
|
||||||
final int numRows = resources.numRows;
|
final int numRows = resources.numRows;
|
||||||
final long minDataTimestamp = resources.minDataTimestamp;
|
final long minDataTimestamp = resources.timeBoundaryInspector.getMinTime().getMillis();
|
||||||
final long maxDataTimestamp = resources.maxDataTimestamp;
|
final long maxDataTimestamp = resources.timeBoundaryInspector.getMaxTime().getMillis();
|
||||||
final NumericColumn timestamps = resources.timestamps;
|
|
||||||
final ColumnCache columnCache = resources.columnCache;
|
final ColumnCache columnCache = resources.columnCache;
|
||||||
final Order timeOrder = resources.timeOrder;
|
final Order timeOrder = resources.timeOrder;
|
||||||
|
|
||||||
|
@ -180,13 +181,13 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
|
|
||||||
if (timeOrder == Order.ASCENDING) {
|
if (timeOrder == Order.ASCENDING) {
|
||||||
for (; baseOffset.withinBounds(); baseOffset.increment()) {
|
for (; baseOffset.withinBounds(); baseOffset.increment()) {
|
||||||
if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) {
|
if (resources.getTimestampsColumn().getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else if (timeOrder == Order.DESCENDING) {
|
} else if (timeOrder == Order.DESCENDING) {
|
||||||
for (; baseOffset.withinBounds(); baseOffset.increment()) {
|
for (; baseOffset.withinBounds(); baseOffset.increment()) {
|
||||||
if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) {
|
if (resources.getTimestampsColumn().getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -197,14 +198,14 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
if (timeOrder == Order.ASCENDING) {
|
if (timeOrder == Order.ASCENDING) {
|
||||||
offset = new AscendingTimestampCheckingOffset(
|
offset = new AscendingTimestampCheckingOffset(
|
||||||
baseOffset,
|
baseOffset,
|
||||||
timestamps,
|
resources.getTimestampsColumn(),
|
||||||
timeEnd,
|
timeEnd,
|
||||||
maxDataTimestamp < timeEnd
|
maxDataTimestamp < timeEnd
|
||||||
);
|
);
|
||||||
} else if (timeOrder == Order.DESCENDING) {
|
} else if (timeOrder == Order.DESCENDING) {
|
||||||
offset = new DescendingTimestampCheckingOffset(
|
offset = new DescendingTimestampCheckingOffset(
|
||||||
baseOffset,
|
baseOffset,
|
||||||
timestamps,
|
resources.getTimestampsColumn(),
|
||||||
timeStart,
|
timeStart,
|
||||||
minDataTimestamp >= timeStart
|
minDataTimestamp >= timeStart
|
||||||
);
|
);
|
||||||
|
@ -244,9 +245,8 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
{
|
{
|
||||||
final CursorResources resources = resourcesSupplier.get();
|
final CursorResources resources = resourcesSupplier.get();
|
||||||
final FilterBundle filterBundle = resources.filterBundle;
|
final FilterBundle filterBundle = resources.filterBundle;
|
||||||
final long minDataTimestamp = resources.minDataTimestamp;
|
final long minDataTimestamp = resources.timeBoundaryInspector.getMinTime().getMillis();
|
||||||
final long maxDataTimestamp = resources.maxDataTimestamp;
|
final long maxDataTimestamp = resources.timeBoundaryInspector.getMaxTime().getMillis();
|
||||||
final NumericColumn timestamps = resources.timestamps;
|
|
||||||
final ColumnCache columnCache = resources.columnCache;
|
final ColumnCache columnCache = resources.columnCache;
|
||||||
final Order timeOrder = resources.timeOrder;
|
final Order timeOrder = resources.timeOrder;
|
||||||
|
|
||||||
|
@ -265,13 +265,13 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
final int endOffset;
|
final int endOffset;
|
||||||
|
|
||||||
if (timeOrder != Order.NONE && interval.getStartMillis() > minDataTimestamp) {
|
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 {
|
} else {
|
||||||
startOffset = 0;
|
startOffset = 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (timeOrder != Order.NONE && interval.getEndMillis() <= maxDataTimestamp) {
|
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 {
|
} else {
|
||||||
endOffset = index.getNumRows();
|
endOffset = index.getNumRows();
|
||||||
}
|
}
|
||||||
|
@ -660,17 +660,18 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
private static final class CursorResources implements Closeable
|
private static final class CursorResources implements Closeable
|
||||||
{
|
{
|
||||||
private final Closer closer;
|
private final Closer closer;
|
||||||
private final long minDataTimestamp;
|
private final TimeBoundaryInspector timeBoundaryInspector;
|
||||||
private final long maxDataTimestamp;
|
|
||||||
private final int numRows;
|
private final int numRows;
|
||||||
@Nullable
|
@Nullable
|
||||||
private final FilterBundle filterBundle;
|
private final FilterBundle filterBundle;
|
||||||
private final NumericColumn timestamps;
|
|
||||||
private final Order timeOrder;
|
private final Order timeOrder;
|
||||||
private final ColumnCache columnCache;
|
private final ColumnCache columnCache;
|
||||||
|
@MonotonicNonNull
|
||||||
|
private NumericColumn timestamps;
|
||||||
|
|
||||||
private CursorResources(
|
private CursorResources(
|
||||||
QueryableIndex index,
|
QueryableIndex index,
|
||||||
|
TimeBoundaryInspector timeBoundaryInspector,
|
||||||
VirtualColumns virtualColumns,
|
VirtualColumns virtualColumns,
|
||||||
Order timeOrder,
|
Order timeOrder,
|
||||||
Interval interval,
|
Interval interval,
|
||||||
|
@ -681,6 +682,7 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
{
|
{
|
||||||
this.closer = Closer.create();
|
this.closer = Closer.create();
|
||||||
this.columnCache = new ColumnCache(index, closer);
|
this.columnCache = new ColumnCache(index, closer);
|
||||||
|
this.timeBoundaryInspector = timeBoundaryInspector;
|
||||||
final ColumnSelectorColumnIndexSelector bitmapIndexSelector = new ColumnSelectorColumnIndexSelector(
|
final ColumnSelectorColumnIndexSelector bitmapIndexSelector = new ColumnSelectorColumnIndexSelector(
|
||||||
index.getBitmapFactoryForDimensions(),
|
index.getBitmapFactoryForDimensions(),
|
||||||
virtualColumns,
|
virtualColumns,
|
||||||
|
@ -688,14 +690,10 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
);
|
);
|
||||||
try {
|
try {
|
||||||
this.numRows = index.getNumRows();
|
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(
|
this.filterBundle = makeFilterBundle(
|
||||||
computeFilterWithIntervalIfNeeded(
|
computeFilterWithIntervalIfNeeded(
|
||||||
|
timeBoundaryInspector,
|
||||||
timeOrder,
|
timeOrder,
|
||||||
this.minDataTimestamp,
|
|
||||||
this.maxDataTimestamp,
|
|
||||||
interval,
|
interval,
|
||||||
filter
|
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
|
@Override
|
||||||
public void close() throws IOException
|
public void close() throws IOException
|
||||||
{
|
{
|
||||||
|
@ -781,20 +787,20 @@ public class QueryableIndexCursorHolder implements CursorHolder
|
||||||
*/
|
*/
|
||||||
@Nullable
|
@Nullable
|
||||||
private static Filter computeFilterWithIntervalIfNeeded(
|
private static Filter computeFilterWithIntervalIfNeeded(
|
||||||
|
final TimeBoundaryInspector timeBoundaryInspector,
|
||||||
final Order timeOrder,
|
final Order timeOrder,
|
||||||
final long minDataTimestamp,
|
|
||||||
final long maxDataTimestamp,
|
|
||||||
final Interval interval,
|
final Interval interval,
|
||||||
@Nullable final Filter filter
|
@Nullable final Filter filter
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (timeOrder == Order.NONE
|
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(
|
final RangeFilter timeFilter = new RangeFilter(
|
||||||
ColumnHolder.TIME_COLUMN_NAME,
|
ColumnHolder.TIME_COLUMN_NAME,
|
||||||
ColumnType.LONG,
|
ColumnType.LONG,
|
||||||
minDataTimestamp < interval.getStartMillis() ? interval.getStartMillis() : null,
|
timeBoundaryInspector.getMinTime().getMillis() < interval.getStartMillis() ? interval.getStartMillis() : null,
|
||||||
maxDataTimestamp >= interval.getEndMillis() ? interval.getEndMillis() : null,
|
timeBoundaryInspector.getMaxTime().getMillis() >= interval.getEndMillis() ? interval.getEndMillis() : null,
|
||||||
false,
|
false,
|
||||||
true,
|
true,
|
||||||
null
|
null
|
||||||
|
|
|
@ -46,8 +46,8 @@ public class QueryableIndexSegment implements Segment
|
||||||
public QueryableIndexSegment(QueryableIndex index, final SegmentId segmentId)
|
public QueryableIndexSegment(QueryableIndex index, final SegmentId segmentId)
|
||||||
{
|
{
|
||||||
this.index = index;
|
this.index = index;
|
||||||
this.cursorFactory = new QueryableIndexCursorFactory(index);
|
|
||||||
this.timeBoundaryInspector = QueryableIndexTimeBoundaryInspector.create(index);
|
this.timeBoundaryInspector = QueryableIndexTimeBoundaryInspector.create(index);
|
||||||
|
this.cursorFactory = new QueryableIndexCursorFactory(index, timeBoundaryInspector);
|
||||||
this.segmentId = segmentId;
|
this.segmentId = segmentId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue