mirror of https://github.com/apache/druid.git
QueryableIndex: Close columns after failed vector cursor setup. (#16365)
* QueryableIndex: Close columns after failed vector cursor setup. If anything fails while setting up a vector cursor, the prior code in QueryableIndex would not close its ColumnCache and would therefore leak columns. Columns often contain references to buffers that must be closed. * Fix style.
This commit is contained in:
parent
fb7c84fb5d
commit
1b107ff695
|
@ -52,6 +52,7 @@ import org.apache.druid.segment.vector.QueryableIndexVectorColumnSelectorFactory
|
|||
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.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
@ -207,69 +208,76 @@ public class QueryableIndexCursorSequenceBuilder
|
|||
final Closer closer = Closer.create();
|
||||
final ColumnCache columnCache = new ColumnCache(index, closer);
|
||||
|
||||
final ColumnSelectorColumnIndexSelector bitmapIndexSelector = new ColumnSelectorColumnIndexSelector(
|
||||
index.getBitmapFactoryForDimensions(),
|
||||
virtualColumns,
|
||||
columnCache
|
||||
);
|
||||
// Wrap the remainder of cursor setup in a try, so if an error is encountered while setting it up, we don't
|
||||
// leak columns in the ColumnCache.
|
||||
try {
|
||||
final ColumnSelectorColumnIndexSelector bitmapIndexSelector = new ColumnSelectorColumnIndexSelector(
|
||||
index.getBitmapFactoryForDimensions(),
|
||||
virtualColumns,
|
||||
columnCache
|
||||
);
|
||||
|
||||
final int numRows = index.getNumRows();
|
||||
final FilterBundle filterBundle = makeFilterBundle(bitmapIndexSelector, numRows);
|
||||
final int numRows = index.getNumRows();
|
||||
final FilterBundle filterBundle = makeFilterBundle(bitmapIndexSelector, numRows);
|
||||
|
||||
NumericColumn timestamps = null;
|
||||
NumericColumn timestamps = null;
|
||||
|
||||
final int startOffset;
|
||||
final int endOffset;
|
||||
final int startOffset;
|
||||
final int endOffset;
|
||||
|
||||
if (interval.getStartMillis() > minDataTimestamp) {
|
||||
timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME);
|
||||
|
||||
startOffset = timeSearch(timestamps, interval.getStartMillis(), 0, index.getNumRows());
|
||||
} else {
|
||||
startOffset = 0;
|
||||
}
|
||||
|
||||
if (interval.getEndMillis() <= maxDataTimestamp) {
|
||||
if (timestamps == null) {
|
||||
if (interval.getStartMillis() > minDataTimestamp) {
|
||||
timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME);
|
||||
|
||||
startOffset = timeSearch(timestamps, interval.getStartMillis(), 0, index.getNumRows());
|
||||
} else {
|
||||
startOffset = 0;
|
||||
}
|
||||
|
||||
endOffset = timeSearch(timestamps, interval.getEndMillis(), startOffset, index.getNumRows());
|
||||
} else {
|
||||
endOffset = index.getNumRows();
|
||||
}
|
||||
if (interval.getEndMillis() <= maxDataTimestamp) {
|
||||
if (timestamps == null) {
|
||||
timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME);
|
||||
}
|
||||
|
||||
// filterBundle will only be null if the filter itself is null, otherwise check to see if the filter can use
|
||||
// an index
|
||||
final VectorOffset baseOffset =
|
||||
filterBundle == null || filterBundle.getIndex() == null
|
||||
? new NoFilterVectorOffset(vectorSize, startOffset, endOffset)
|
||||
: new BitmapVectorOffset(vectorSize, filterBundle.getIndex().getBitmap(), startOffset, endOffset);
|
||||
endOffset = timeSearch(timestamps, interval.getEndMillis(), startOffset, index.getNumRows());
|
||||
} else {
|
||||
endOffset = index.getNumRows();
|
||||
}
|
||||
|
||||
// baseColumnSelectorFactory using baseOffset is the column selector for filtering.
|
||||
final VectorColumnSelectorFactory baseColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset(
|
||||
columnCache,
|
||||
baseOffset
|
||||
);
|
||||
// filterBundle will only be null if the filter itself is null, otherwise check to see if the filter can use
|
||||
// an index
|
||||
final VectorOffset baseOffset =
|
||||
filterBundle == null || filterBundle.getIndex() == null
|
||||
? new NoFilterVectorOffset(vectorSize, startOffset, endOffset)
|
||||
: new BitmapVectorOffset(vectorSize, filterBundle.getIndex().getBitmap(), startOffset, endOffset);
|
||||
|
||||
// filterBundle will only be null if the filter itself is null, otherwise check to see if the filter needs to use
|
||||
// a value matcher
|
||||
if (filterBundle != null && filterBundle.getMatcherBundle() != null) {
|
||||
final VectorValueMatcher vectorValueMatcher = filterBundle.getMatcherBundle()
|
||||
.vectorMatcher(baseColumnSelectorFactory, baseOffset);
|
||||
final VectorOffset filteredOffset = FilteredVectorOffset.create(
|
||||
baseOffset,
|
||||
vectorValueMatcher
|
||||
);
|
||||
|
||||
// Now create the cursor and column selector that will be returned to the caller.
|
||||
final VectorColumnSelectorFactory filteredColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset(
|
||||
// baseColumnSelectorFactory using baseOffset is the column selector for filtering.
|
||||
final VectorColumnSelectorFactory baseColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset(
|
||||
columnCache,
|
||||
filteredOffset
|
||||
baseOffset
|
||||
);
|
||||
return new QueryableIndexVectorCursor(filteredColumnSelectorFactory, filteredOffset, vectorSize, closer);
|
||||
} else {
|
||||
return new QueryableIndexVectorCursor(baseColumnSelectorFactory, baseOffset, vectorSize, closer);
|
||||
|
||||
// filterBundle will only be null if the filter itself is null, otherwise check to see if the filter needs to use
|
||||
// a value matcher
|
||||
if (filterBundle != null && filterBundle.getMatcherBundle() != null) {
|
||||
final VectorValueMatcher vectorValueMatcher = filterBundle.getMatcherBundle()
|
||||
.vectorMatcher(baseColumnSelectorFactory, baseOffset);
|
||||
final VectorOffset filteredOffset = FilteredVectorOffset.create(
|
||||
baseOffset,
|
||||
vectorValueMatcher
|
||||
);
|
||||
|
||||
// Now create the cursor and column selector that will be returned to the caller.
|
||||
final VectorColumnSelectorFactory filteredColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset(
|
||||
columnCache,
|
||||
filteredOffset
|
||||
);
|
||||
return new QueryableIndexVectorCursor(filteredColumnSelectorFactory, filteredOffset, vectorSize, closer);
|
||||
} else {
|
||||
return new QueryableIndexVectorCursor(baseColumnSelectorFactory, baseOffset, vectorSize, closer);
|
||||
}
|
||||
}
|
||||
catch (Throwable t) {
|
||||
throw CloseableUtils.closeAndWrapInCatch(t, closer);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue