Skip to content

Commit

Permalink
changes:
Browse files Browse the repository at this point in the history
* CursorMaker no longer handles query granularity directly
* Sequence<Cursor> of makeCursors is now just a Cursor from makeCursor
* added CursorGranularizer to bucket queries by granularity instead, updated all engines that support query granularity to use this instead (topN still needs some more work)
* remove Cursor.getTime
* remove implementations of CursorFactory methods that are not asCursorMaker, replacing with defaults that throw exceptions
  • Loading branch information
clintropolis committed Jul 26, 2024
1 parent f693e3e commit 8184213
Show file tree
Hide file tree
Showing 97 changed files with 3,927 additions and 3,970 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,9 @@
package org.apache.druid.benchmark;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import org.apache.druid.common.config.NullHandling;
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.io.Closer;
import org.apache.druid.js.JavaScriptConfig;
import org.apache.druid.query.aggregation.BufferAggregator;
Expand All @@ -36,6 +34,7 @@
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorMaker;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.column.ValueType;
Expand All @@ -60,7 +59,6 @@
import org.openjdk.jmh.infra.Blackhole;

import java.nio.ByteBuffer;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;

Expand Down Expand Up @@ -169,25 +167,22 @@ private double compute(final Function<ColumnSelectorFactory, BufferAggregator> a
.setInterval(index.getDataInterval())
.setGranularity(Granularities.ALL)
.build();
final Sequence<Cursor> cursors = adapter.asCursorMaker(buildSpec).makeCursors();
try (final CursorMaker maker = adapter.asCursorMaker(buildSpec)) {
final Cursor cursor = maker.makeCursor();

final List<Double> results = cursors
.map(cursor -> {
final BufferAggregator bufferAggregator = aggregatorFactory.apply(cursor.getColumnSelectorFactory());
bufferAggregator.init(aggregationBuffer, 0);
final BufferAggregator bufferAggregator = aggregatorFactory.apply(cursor.getColumnSelectorFactory());
bufferAggregator.init(aggregationBuffer, 0);

while (!cursor.isDone()) {
bufferAggregator.aggregate(aggregationBuffer, 0);
cursor.advance();
}
while (!cursor.isDone()) {
bufferAggregator.aggregate(aggregationBuffer, 0);
cursor.advance();
}

final Double dbl = (Double) bufferAggregator.get(aggregationBuffer, 0);
bufferAggregator.close();
return dbl;
})
.toList();
final Double dbl = (Double) bufferAggregator.get(aggregationBuffer, 0);
bufferAggregator.close();

return Iterables.getOnlyElement(results);
return dbl;
}
}

private static class NativeBufferAggregator implements BufferAggregator
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import org.apache.druid.common.config.NullHandling;
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.io.Closer;
import org.apache.druid.math.expr.ExpressionProcessing;
import org.apache.druid.query.expression.TestExprMacroTable;
Expand All @@ -34,6 +33,7 @@
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorMaker;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.column.ValueType;
Expand All @@ -58,7 +58,6 @@
import org.openjdk.jmh.infra.Blackhole;

import java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeUnit;

@State(Scope.Benchmark)
Expand Down Expand Up @@ -151,17 +150,16 @@ public void expressionFilter(Blackhole blackhole)
.setInterval(index.getDataInterval())
.setGranularity(Granularities.ALL)
.build();
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec).makeCursors();
try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) {
final Cursor cursor = maker.makeCursor();

final List<?> results = cursors
.map(cursor -> {
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x");
consumeString(cursor, selector, blackhole);
return null;
})
.toList();

blackhole.consume(results);
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x");
while (!cursor.isDone()) {
blackhole.consume(selector.getObject());
cursor.advance();
}
}
}

@Benchmark
Expand All @@ -172,23 +170,13 @@ public void nativeFilter(Blackhole blackhole)
.setInterval(index.getDataInterval())
.setGranularity(Granularities.ALL)
.build();
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec).makeCursors();
final List<?> results = cursors
.map(cursor -> {
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x");
consumeString(cursor, selector, blackhole);
return null;
})
.toList();

blackhole.consume(results);
}

private void consumeString(final Cursor cursor, final ColumnValueSelector selector, final Blackhole blackhole)
{
while (!cursor.isDone()) {
blackhole.consume(selector.getLong());
cursor.advance();
try (final CursorMaker maker = new QueryableIndexStorageAdapter(index).asCursorMaker(buildSpec)) {
final Cursor cursor = maker.makeCursor();
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x");
while (!cursor.isDone()) {
blackhole.consume(selector.getObject());
cursor.advance();
}
}
}
}
Loading

0 comments on commit 8184213

Please sign in to comment.