mirror of https://github.com/apache/druid.git
optimize numeric column null value checking for low filter selectivity (more rows) (#8822)
* use peekable iterator for numeric column selector null checking instead of bitmap.get for those sweet sweet nanoseconds * remove unused method * slight optimization i think * remove clone from wrappers since we do not use and is confusing * fixes and tests * int instead of Integer * fix it * fixes, more tests * fix
This commit is contained in:
parent
80dbf44fca
commit
9ed9a80b9d
|
@ -0,0 +1,196 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.benchmark;
|
||||
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.collections.bitmap.WrappedImmutableConciseBitmap;
|
||||
import org.apache.druid.collections.bitmap.WrappedImmutableRoaringBitmap;
|
||||
import org.apache.druid.extendedset.intset.ConciseSet;
|
||||
import org.apache.druid.extendedset.intset.ImmutableConciseSet;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
import org.openjdk.jmh.annotations.Measurement;
|
||||
import org.openjdk.jmh.annotations.Mode;
|
||||
import org.openjdk.jmh.annotations.OutputTimeUnit;
|
||||
import org.openjdk.jmh.annotations.Param;
|
||||
import org.openjdk.jmh.annotations.Scope;
|
||||
import org.openjdk.jmh.annotations.Setup;
|
||||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.annotations.Timeout;
|
||||
import org.openjdk.jmh.annotations.Warmup;
|
||||
import org.openjdk.jmh.infra.Blackhole;
|
||||
import org.roaringbitmap.IntIterator;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
import org.roaringbitmap.buffer.MutableRoaringBitmap;
|
||||
|
||||
import java.util.BitSet;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@State(Scope.Benchmark)
|
||||
@Fork(value = 1)
|
||||
@Warmup(iterations = 5, time = 10, timeUnit = TimeUnit.SECONDS)
|
||||
@Measurement(iterations = 20, time = 10, timeUnit = TimeUnit.SECONDS)
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
@Timeout(time = 30, timeUnit = TimeUnit.SECONDS)
|
||||
public class NullHandlingBitmapGetVsIteratorBenchmark
|
||||
{
|
||||
@Param({
|
||||
//"concise",
|
||||
"roaring"
|
||||
})
|
||||
String bitmapType;
|
||||
|
||||
@Param({"500000"})
|
||||
private int numRows;
|
||||
|
||||
@Param({
|
||||
"0.001",
|
||||
"0.01",
|
||||
"0.1",
|
||||
"0.25",
|
||||
"0.5",
|
||||
"0.75",
|
||||
"0.99999"
|
||||
})
|
||||
private double filterMatch;
|
||||
|
||||
@Param({
|
||||
"0",
|
||||
"0.1",
|
||||
"0.25",
|
||||
"0.5",
|
||||
"0.75",
|
||||
"0.99"
|
||||
})
|
||||
private double nullDensity;
|
||||
|
||||
ImmutableBitmap bitmap;
|
||||
BitSet pretendFilterOffsets;
|
||||
|
||||
@Setup
|
||||
public void setup()
|
||||
{
|
||||
pretendFilterOffsets = new BitSet(numRows);
|
||||
switch (bitmapType) {
|
||||
case "concise":
|
||||
ConciseSet conciseSet = new ConciseSet();
|
||||
for (int i = 0; i < numRows; i++) {
|
||||
double rando = ThreadLocalRandom.current().nextDouble(0.0, 1.0);
|
||||
if (filterMatch == 1.0 || rando < filterMatch) {
|
||||
pretendFilterOffsets.set(i);
|
||||
}
|
||||
if (rando < nullDensity) {
|
||||
conciseSet.add(i);
|
||||
}
|
||||
}
|
||||
bitmap = new WrappedImmutableConciseBitmap(ImmutableConciseSet.newImmutableFromMutable(conciseSet));
|
||||
break;
|
||||
case "roaring":
|
||||
MutableRoaringBitmap roaringBitmap = new MutableRoaringBitmap();
|
||||
|
||||
for (int i = 0; i < numRows; i++) {
|
||||
double rando = ThreadLocalRandom.current().nextDouble(0.0, 1.0);
|
||||
if (filterMatch == 1.0 || rando < filterMatch) {
|
||||
pretendFilterOffsets.set(i);
|
||||
}
|
||||
if (rando < nullDensity) {
|
||||
roaringBitmap.add(i);
|
||||
}
|
||||
}
|
||||
bitmap = new WrappedImmutableRoaringBitmap(roaringBitmap.toImmutableRoaringBitmap());
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void get(final Blackhole blackhole)
|
||||
{
|
||||
for (int i = pretendFilterOffsets.nextSetBit(0); i >= 0; i = pretendFilterOffsets.nextSetBit(i + 1)) {
|
||||
final boolean isNull = bitmap.get(i);
|
||||
if (isNull) {
|
||||
blackhole.consume(isNull);
|
||||
} else {
|
||||
blackhole.consume(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void iterator(final Blackhole blackhole)
|
||||
{
|
||||
IntIterator nullIterator = bitmap.iterator();
|
||||
int offsetMark = -1;
|
||||
int nullMark = -1;
|
||||
for (int i = pretendFilterOffsets.nextSetBit(0); i >= 0; i = pretendFilterOffsets.nextSetBit(i + 1)) {
|
||||
// this is totally useless, hopefully this doesn't get optimized out, try to mimic what the selector is doing
|
||||
if (i < offsetMark) {
|
||||
nullMark = -1;
|
||||
nullIterator = bitmap.iterator();
|
||||
}
|
||||
offsetMark = i;
|
||||
while (nullMark < i && nullIterator.hasNext()) {
|
||||
nullMark = nullIterator.next();
|
||||
}
|
||||
final boolean isNull = nullMark == offsetMark;
|
||||
if (isNull) {
|
||||
blackhole.consume(isNull);
|
||||
} else {
|
||||
blackhole.consume(offsetMark);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void peekableIterator(final Blackhole blackhole)
|
||||
{
|
||||
PeekableIntIterator nullIterator = bitmap.peekableIterator();
|
||||
int offsetMark = -1;
|
||||
int nullMark = -1;
|
||||
for (int i = pretendFilterOffsets.nextSetBit(0); i >= 0; i = pretendFilterOffsets.nextSetBit(i + 1)) {
|
||||
// this is totally useless, hopefully this doesn't get optimized out, try to mimic what the selector is doing
|
||||
if (i < offsetMark) {
|
||||
nullMark = -1;
|
||||
nullIterator = bitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = i;
|
||||
if (nullMark < i) {
|
||||
nullIterator.advanceIfNeeded(i);
|
||||
if (nullIterator.hasNext()) {
|
||||
nullMark = nullIterator.next();
|
||||
}
|
||||
}
|
||||
final boolean isNull = nullMark == offsetMark;
|
||||
if (isNull) {
|
||||
blackhole.consume(isNull);
|
||||
} else {
|
||||
blackhole.consume(offsetMark);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,43 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.collections.bitmap;
|
||||
|
||||
import org.apache.druid.extendedset.intset.IntSet;
|
||||
|
||||
public class ConcisePeekableIteratorAdapter extends PeekableIteratorAdapter<IntSet.IntIterator>
|
||||
{
|
||||
ConcisePeekableIteratorAdapter(IntSet.IntIterator iterator)
|
||||
{
|
||||
super(iterator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void advanceIfNeeded(int i)
|
||||
{
|
||||
if (mark < i) {
|
||||
baseIterator.skipAllBefore(i);
|
||||
if (baseIterator.hasNext()) {
|
||||
mark = baseIterator.next();
|
||||
} else {
|
||||
mark = NOT_SET;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -21,6 +21,7 @@ package org.apache.druid.collections.bitmap;
|
|||
|
||||
import org.roaringbitmap.BatchIterator;
|
||||
import org.roaringbitmap.IntIterator;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
/**
|
||||
* This class is meant to represent a simple wrapper around an immutable bitmap
|
||||
|
@ -33,6 +34,14 @@ public interface ImmutableBitmap
|
|||
*/
|
||||
IntIterator iterator();
|
||||
|
||||
/**
|
||||
* @return a peekable iterator which can skip to a position
|
||||
*/
|
||||
default PeekableIntIterator peekableIterator()
|
||||
{
|
||||
return new PeekableIteratorAdapter<>(iterator());
|
||||
}
|
||||
|
||||
/**
|
||||
* @return a batched iterator over the set bits of this bitmap
|
||||
*/
|
||||
|
|
|
@ -0,0 +1,81 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.collections.bitmap;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.roaringbitmap.IntIterator;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
public class PeekableIteratorAdapter<TIntIterator extends IntIterator> implements PeekableIntIterator
|
||||
{
|
||||
static final int NOT_SET = -1;
|
||||
final TIntIterator baseIterator;
|
||||
int mark = NOT_SET;
|
||||
|
||||
PeekableIteratorAdapter(TIntIterator iterator)
|
||||
{
|
||||
this.baseIterator = Preconditions.checkNotNull(iterator, "iterator");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void advanceIfNeeded(int i)
|
||||
{
|
||||
while (mark < i && baseIterator.hasNext()) {
|
||||
mark = baseIterator.next();
|
||||
}
|
||||
if (mark < i) {
|
||||
mark = NOT_SET;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int peekNext()
|
||||
{
|
||||
if (mark == NOT_SET) {
|
||||
mark = baseIterator.next();
|
||||
}
|
||||
return mark;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PeekableIntIterator clone()
|
||||
{
|
||||
throw new UnsupportedOperationException(
|
||||
"PeekableIteratorAdapter.clone is not implemented, but this should not happen"
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
return mark != NOT_SET || baseIterator.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int next()
|
||||
{
|
||||
if (mark != NOT_SET) {
|
||||
final int currentBit = mark;
|
||||
mark = NOT_SET;
|
||||
return currentBit;
|
||||
}
|
||||
return baseIterator.next();
|
||||
}
|
||||
}
|
|
@ -19,9 +19,11 @@
|
|||
|
||||
package org.apache.druid.collections.bitmap;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.druid.extendedset.intset.ConciseSet;
|
||||
import org.apache.druid.extendedset.intset.ImmutableConciseSet;
|
||||
import org.roaringbitmap.IntIterator;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
public class WrappedConciseBitmap implements MutableBitmap
|
||||
{
|
||||
|
@ -48,7 +50,8 @@ public class WrappedConciseBitmap implements MutableBitmap
|
|||
this.bitmap = conciseSet;
|
||||
}
|
||||
|
||||
ConciseSet getBitmap()
|
||||
@VisibleForTesting
|
||||
public ConciseSet getBitmap()
|
||||
{
|
||||
return bitmap;
|
||||
}
|
||||
|
@ -109,6 +112,12 @@ public class WrappedConciseBitmap implements MutableBitmap
|
|||
return bitmap.iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PeekableIntIterator peekableIterator()
|
||||
{
|
||||
return new ConcisePeekableIteratorAdapter(bitmap.iterator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty()
|
||||
{
|
||||
|
|
|
@ -19,9 +19,9 @@
|
|||
|
||||
package org.apache.druid.collections.bitmap;
|
||||
|
||||
|
||||
import org.apache.druid.extendedset.intset.ImmutableConciseSet;
|
||||
import org.roaringbitmap.IntIterator;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
import java.nio.IntBuffer;
|
||||
|
||||
|
@ -76,6 +76,12 @@ public class WrappedImmutableConciseBitmap implements ImmutableBitmap
|
|||
return bitmap.iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PeekableIntIterator peekableIterator()
|
||||
{
|
||||
return new ConcisePeekableIteratorAdapter(bitmap.iterator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.collections.bitmap;
|
|||
|
||||
import org.roaringbitmap.BatchIterator;
|
||||
import org.roaringbitmap.IntIterator;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -77,6 +78,12 @@ public class WrappedImmutableRoaringBitmap implements ImmutableBitmap
|
|||
return bitmap.getIntIterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PeekableIntIterator peekableIterator()
|
||||
{
|
||||
return bitmap.getIntIterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BatchIterator batchIterator()
|
||||
{
|
||||
|
|
|
@ -19,7 +19,9 @@
|
|||
|
||||
package org.apache.druid.collections.bitmap;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.roaringbitmap.IntIterator;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
import org.roaringbitmap.RoaringBitmap;
|
||||
import org.roaringbitmap.RoaringBitmapWriter;
|
||||
import org.roaringbitmap.buffer.MutableRoaringBitmap;
|
||||
|
@ -56,7 +58,8 @@ public class WrappedRoaringBitmap implements MutableBitmap
|
|||
this.compressRunOnSerialization = compressRunOnSerialization;
|
||||
}
|
||||
|
||||
ImmutableBitmap toImmutableBitmap()
|
||||
@VisibleForTesting
|
||||
public ImmutableBitmap toImmutableBitmap()
|
||||
{
|
||||
MutableRoaringBitmap bitmap = writer.get().clone();
|
||||
if (compressRunOnSerialization) {
|
||||
|
@ -146,6 +149,12 @@ public class WrappedRoaringBitmap implements MutableBitmap
|
|||
return writer.get().getIntIterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PeekableIntIterator peekableIterator()
|
||||
{
|
||||
return writer.get().getIntIterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty()
|
||||
{
|
||||
|
|
|
@ -29,7 +29,7 @@ public class SimpleAscendingOffset extends Offset
|
|||
private final int initialOffset;
|
||||
private int currentOffset;
|
||||
|
||||
SimpleAscendingOffset(int rowCount)
|
||||
public SimpleAscendingOffset(int rowCount)
|
||||
{
|
||||
this(0, rowCount);
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector;
|
|||
import org.apache.druid.segment.vector.ReadableVectorOffset;
|
||||
import org.apache.druid.segment.vector.VectorSelectorUtils;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
|
@ -94,10 +95,27 @@ public interface ColumnarDoubles extends Closeable
|
|||
} else {
|
||||
class HistoricalDoubleColumnSelectorWithNulls implements DoubleColumnSelector, HistoricalColumnSelector<Double>
|
||||
{
|
||||
private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator();
|
||||
private int nullMark = -1;
|
||||
private int offsetMark = -1;
|
||||
|
||||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
return nullValueBitmap.get(offset.getOffset());
|
||||
final int i = offset.getOffset();
|
||||
if (i < offsetMark) {
|
||||
// offset was reset, reset iterator state
|
||||
nullMark = -1;
|
||||
nullIterator = nullValueBitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = i;
|
||||
if (nullMark < i) {
|
||||
nullIterator.advanceIfNeeded(offsetMark);
|
||||
if (nullIterator.hasNext()) {
|
||||
nullMark = nullIterator.next();
|
||||
}
|
||||
}
|
||||
return nullMark == offsetMark;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -137,6 +155,9 @@ public interface ColumnarDoubles extends Closeable
|
|||
|
||||
private int id = ReadableVectorOffset.NULL_ID;
|
||||
|
||||
private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator();
|
||||
private int offsetMark = -1;
|
||||
|
||||
@Nullable
|
||||
private boolean[] nullVector = null;
|
||||
|
||||
|
@ -168,12 +189,21 @@ public interface ColumnarDoubles extends Closeable
|
|||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
if (offset.getStartOffset() < offsetMark) {
|
||||
nullIterator = nullValueBitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = offset.getStartOffset() + offset.getCurrentVectorSize();
|
||||
ColumnarDoubles.this.get(doubleVector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
||||
} else {
|
||||
ColumnarDoubles.this.get(doubleVector, offset.getOffsets(), offset.getCurrentVectorSize());
|
||||
final int[] offsets = offset.getOffsets();
|
||||
if (offsets[offsets.length - 1] < offsetMark) {
|
||||
nullIterator = nullValueBitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = offsets[offsets.length - 1];
|
||||
ColumnarDoubles.this.get(doubleVector, offsets, offset.getCurrentVectorSize());
|
||||
}
|
||||
|
||||
nullVector = VectorSelectorUtils.populateNullVector(nullVector, offset, nullValueBitmap);
|
||||
nullVector = VectorSelectorUtils.populateNullVector(nullVector, offset, nullIterator);
|
||||
|
||||
id = offset.getId();
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.druid.segment.vector.BaseFloatVectorValueSelector;
|
|||
import org.apache.druid.segment.vector.ReadableVectorOffset;
|
||||
import org.apache.druid.segment.vector.VectorSelectorUtils;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
|
@ -94,10 +95,27 @@ public interface ColumnarFloats extends Closeable
|
|||
} else {
|
||||
class HistoricalFloatColumnSelectorwithNulls implements FloatColumnSelector, HistoricalColumnSelector<Float>
|
||||
{
|
||||
private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator();
|
||||
private int nullMark = -1;
|
||||
private int offsetMark = -1;
|
||||
|
||||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
return nullValueBitmap.get(offset.getOffset());
|
||||
final int i = offset.getOffset();
|
||||
if (i < offsetMark) {
|
||||
// offset was reset, reset iterator state
|
||||
nullMark = -1;
|
||||
nullIterator = nullValueBitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = i;
|
||||
if (nullMark < i) {
|
||||
nullIterator.advanceIfNeeded(offsetMark);
|
||||
if (nullIterator.hasNext()) {
|
||||
nullMark = nullIterator.next();
|
||||
}
|
||||
}
|
||||
return nullMark == offsetMark;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -137,6 +155,9 @@ public interface ColumnarFloats extends Closeable
|
|||
|
||||
private int id = ReadableVectorOffset.NULL_ID;
|
||||
|
||||
private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator();
|
||||
private int offsetMark = -1;
|
||||
|
||||
@Nullable
|
||||
private boolean[] nullVector = null;
|
||||
|
||||
|
@ -168,12 +189,21 @@ public interface ColumnarFloats extends Closeable
|
|||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
if (offset.getStartOffset() < offsetMark) {
|
||||
nullIterator = nullValueBitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = offset.getStartOffset() + offset.getCurrentVectorSize();
|
||||
ColumnarFloats.this.get(floatVector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
||||
} else {
|
||||
ColumnarFloats.this.get(floatVector, offset.getOffsets(), offset.getCurrentVectorSize());
|
||||
final int[] offsets = offset.getOffsets();
|
||||
if (offsets[offsets.length - 1] < offsetMark) {
|
||||
nullIterator = nullValueBitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = offsets[offsets.length - 1];
|
||||
ColumnarFloats.this.get(floatVector, offsets, offset.getCurrentVectorSize());
|
||||
}
|
||||
|
||||
nullVector = VectorSelectorUtils.populateNullVector(nullVector, offset, nullValueBitmap);
|
||||
nullVector = VectorSelectorUtils.populateNullVector(nullVector, offset, nullIterator);
|
||||
|
||||
id = offset.getId();
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.druid.segment.vector.BaseLongVectorValueSelector;
|
|||
import org.apache.druid.segment.vector.ReadableVectorOffset;
|
||||
import org.apache.druid.segment.vector.VectorSelectorUtils;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
|
@ -94,10 +95,27 @@ public interface ColumnarLongs extends Closeable
|
|||
} else {
|
||||
class HistoricalLongColumnSelectorWithNulls implements LongColumnSelector, HistoricalColumnSelector<Long>
|
||||
{
|
||||
private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator();
|
||||
private int nullMark = -1;
|
||||
private int offsetMark = -1;
|
||||
|
||||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
return nullValueBitmap.get(offset.getOffset());
|
||||
final int i = offset.getOffset();
|
||||
if (i < offsetMark) {
|
||||
// offset was reset, reset iterator state
|
||||
nullMark = -1;
|
||||
nullIterator = nullValueBitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = i;
|
||||
if (nullMark < i) {
|
||||
nullIterator.advanceIfNeeded(offsetMark);
|
||||
if (nullIterator.hasNext()) {
|
||||
nullMark = nullIterator.next();
|
||||
}
|
||||
}
|
||||
return nullMark == offsetMark;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -137,6 +155,9 @@ public interface ColumnarLongs extends Closeable
|
|||
|
||||
private int id = ReadableVectorOffset.NULL_ID;
|
||||
|
||||
private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator();
|
||||
private int offsetMark = -1;
|
||||
|
||||
@Nullable
|
||||
private boolean[] nullVector = null;
|
||||
|
||||
|
@ -168,12 +189,21 @@ public interface ColumnarLongs extends Closeable
|
|||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
if (offset.getStartOffset() < offsetMark) {
|
||||
nullIterator = nullValueBitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = offset.getStartOffset() + offset.getCurrentVectorSize();
|
||||
ColumnarLongs.this.get(longVector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
||||
} else {
|
||||
ColumnarLongs.this.get(longVector, offset.getOffsets(), offset.getCurrentVectorSize());
|
||||
final int[] offsets = offset.getOffsets();
|
||||
if (offsets[offsets.length - 1] < offsetMark) {
|
||||
nullIterator = nullValueBitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = offsets[offsets.length - 1];
|
||||
ColumnarLongs.this.get(longVector, offsets, offset.getCurrentVectorSize());
|
||||
}
|
||||
|
||||
nullVector = VectorSelectorUtils.populateNullVector(nullVector, offset, nullValueBitmap);
|
||||
nullVector = VectorSelectorUtils.populateNullVector(nullVector, offset, nullIterator);
|
||||
|
||||
id = offset.getId();
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
package org.apache.druid.segment.vector;
|
||||
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
|
@ -32,10 +32,10 @@ public class VectorSelectorUtils
|
|||
public static boolean[] populateNullVector(
|
||||
@Nullable final boolean[] nullVector,
|
||||
final ReadableVectorOffset offset,
|
||||
final ImmutableBitmap nullValueBitmap
|
||||
final PeekableIntIterator nullIterator
|
||||
)
|
||||
{
|
||||
if (nullValueBitmap.isEmpty()) {
|
||||
if (!nullIterator.hasNext()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -47,14 +47,33 @@ public class VectorSelectorUtils
|
|||
retVal = new boolean[offset.getMaxVectorSize()];
|
||||
}
|
||||
|
||||
// Probably not super efficient to call "get" so much, but, no worse than the non-vectorized version.
|
||||
if (offset.isContiguous()) {
|
||||
final int startOffset = offset.getStartOffset();
|
||||
nullIterator.advanceIfNeeded(startOffset);
|
||||
if (!nullIterator.hasNext()) {
|
||||
return null;
|
||||
}
|
||||
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
||||
retVal[i] = nullValueBitmap.get(i + offset.getStartOffset());
|
||||
final int row = i + startOffset;
|
||||
nullIterator.advanceIfNeeded(row);
|
||||
if (!nullIterator.hasNext()) {
|
||||
break;
|
||||
}
|
||||
retVal[i] = row == nullIterator.peekNext();
|
||||
}
|
||||
} else {
|
||||
final int[] currentOffsets = offset.getOffsets();
|
||||
nullIterator.advanceIfNeeded(currentOffsets[0]);
|
||||
if (!nullIterator.hasNext()) {
|
||||
return null;
|
||||
}
|
||||
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
||||
retVal[i] = nullValueBitmap.get(offset.getOffsets()[i]);
|
||||
final int row = currentOffsets[i];
|
||||
nullIterator.advanceIfNeeded(row);
|
||||
if (!nullIterator.hasNext()) {
|
||||
break;
|
||||
}
|
||||
retVal[i] = row == nullIterator.peekNext();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,99 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.collections.bitmap;
|
||||
|
||||
import org.apache.druid.collections.IntSetTestUtility;
|
||||
import org.apache.druid.extendedset.intset.ImmutableConciseSet;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
public class BitmapPeekableIteratorTest
|
||||
{
|
||||
@Test
|
||||
public void testBitSet()
|
||||
{
|
||||
final WrappedBitSetBitmap bitmap = new WrappedBitSetBitmap();
|
||||
populate(bitmap);
|
||||
assertPeekable(bitmap.peekableIterator());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConciseMutable()
|
||||
{
|
||||
final WrappedConciseBitmap bitmap = new WrappedConciseBitmap();
|
||||
populate(bitmap);
|
||||
assertPeekable(bitmap.peekableIterator());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConciseImmutable()
|
||||
{
|
||||
final WrappedConciseBitmap bitmap = new WrappedConciseBitmap();
|
||||
populate(bitmap);
|
||||
final ImmutableBitmap immutable = new WrappedImmutableConciseBitmap(
|
||||
ImmutableConciseSet.newImmutableFromMutable(bitmap.getBitmap())
|
||||
);
|
||||
assertPeekable(immutable.peekableIterator());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRoaringMutable()
|
||||
{
|
||||
WrappedRoaringBitmap bitmap = new WrappedRoaringBitmap();
|
||||
populate(bitmap);
|
||||
assertPeekable(bitmap.peekableIterator());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRoaringImmutable()
|
||||
{
|
||||
WrappedRoaringBitmap bitmap = new WrappedRoaringBitmap();
|
||||
populate(bitmap);
|
||||
assertPeekable(bitmap.toImmutableBitmap().peekableIterator());
|
||||
}
|
||||
|
||||
private void populate(MutableBitmap bitmap)
|
||||
{
|
||||
for (int i : IntSetTestUtility.getSetBits()) {
|
||||
bitmap.add(i);
|
||||
}
|
||||
}
|
||||
|
||||
private void assertPeekable(PeekableIntIterator iterator)
|
||||
{
|
||||
// extra calls are to make sure things that are not expected to have apparent side-effects have none
|
||||
Assert.assertTrue(iterator.hasNext());
|
||||
int mark = -1;
|
||||
for (int i : IntSetTestUtility.getSetBits()) {
|
||||
Assert.assertTrue(iterator.hasNext());
|
||||
iterator.advanceIfNeeded(i);
|
||||
Assert.assertTrue(iterator.hasNext());
|
||||
iterator.advanceIfNeeded(i); // this should do nothing
|
||||
Assert.assertTrue(iterator.hasNext());
|
||||
if (iterator.hasNext()) {
|
||||
Assert.assertEquals(i, iterator.peekNext());
|
||||
mark = iterator.next();
|
||||
}
|
||||
Assert.assertEquals(i, mark);
|
||||
}
|
||||
Assert.assertFalse(iterator.hasNext());
|
||||
}
|
||||
}
|
|
@ -46,7 +46,7 @@ public class WrappedRoaringBitmapTest
|
|||
new RoaringBitmapFactory(false)
|
||||
},
|
||||
new RoaringBitmapFactory[] {
|
||||
new RoaringBitmapFactory(false)
|
||||
new RoaringBitmapFactory(true)
|
||||
}
|
||||
);
|
||||
}
|
||||
|
|
|
@ -4292,19 +4292,6 @@ public class TopNQueryRunnerTest
|
|||
return runWithMerge(query, ResponseContext.createEmpty());
|
||||
}
|
||||
|
||||
private Sequence<Result<TopNResultValue>> runWithPreMergeAndMerge(TopNQuery query, ResponseContext context)
|
||||
{
|
||||
final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
|
||||
);
|
||||
final QueryRunner<Result<TopNResultValue>> _runner = new FinalizeResultsQueryRunner(
|
||||
chest.mergeResults(chest.preMergeQueryDecoration(runner)),
|
||||
chest
|
||||
);
|
||||
return _runner.run(QueryPlus.wrap(query), context);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTopNWithExtractionFilterNoExistingValue()
|
||||
{
|
||||
|
|
|
@ -84,18 +84,6 @@ public class TestIndex
|
|||
"indexMin",
|
||||
"indexMaxPlusTen"
|
||||
};
|
||||
public static final String[] DIMENSIONS = new String[]{
|
||||
"market",
|
||||
"quality",
|
||||
"qualityLong",
|
||||
"qualityFloat",
|
||||
"qualityDouble",
|
||||
"qualityNumericString",
|
||||
"placement",
|
||||
"placementish",
|
||||
"partial_null_column",
|
||||
"null_column"
|
||||
};
|
||||
|
||||
public static final List<DimensionSchema> DIMENSION_SCHEMAS = Arrays.asList(
|
||||
new StringDimensionSchema("market"),
|
||||
|
|
|
@ -0,0 +1,328 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.data;
|
||||
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.collections.bitmap.WrappedRoaringBitmap;
|
||||
import org.apache.druid.segment.BaseNullableColumnValueSelector;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.SimpleAscendingOffset;
|
||||
import org.apache.druid.segment.column.DoublesColumn;
|
||||
import org.apache.druid.segment.column.FloatsColumn;
|
||||
import org.apache.druid.segment.column.LongsColumn;
|
||||
import org.apache.druid.segment.vector.NoFilterVectorOffset;
|
||||
import org.apache.druid.segment.vector.VectorOffset;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
||||
public class NumericNullColumnSelectorTest
|
||||
{
|
||||
private final int seed = 1337;
|
||||
private final Random rando = new Random(seed);
|
||||
private final int numBitmaps = 32;
|
||||
private final int numRows = 1024;
|
||||
private final int vectorSize = 128;
|
||||
private final SimpleAscendingOffset offset = new SimpleAscendingOffset(numRows);
|
||||
private final NoFilterVectorOffset vectorOffset = new NoFilterVectorOffset(vectorSize, 0, numRows);
|
||||
private final NoFilterOffsetThatCanBeMangledToTestOverlapping anotherVectorOffset =
|
||||
new NoFilterOffsetThatCanBeMangledToTestOverlapping(vectorSize, 0, numRows);
|
||||
private ImmutableBitmap[] bitmaps;
|
||||
|
||||
@Before
|
||||
public void setup()
|
||||
{
|
||||
bitmaps = new ImmutableBitmap[numBitmaps];
|
||||
for (int bitmap = 0; bitmap < numBitmaps; bitmap++) {
|
||||
WrappedRoaringBitmap mutable = new WrappedRoaringBitmap();
|
||||
for (int i = 0; i < numRows; i++) {
|
||||
if (rando.nextDouble() > 0.2) {
|
||||
mutable.add(i);
|
||||
}
|
||||
}
|
||||
bitmaps[bitmap] = mutable.toImmutableBitmap();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLongSelectorWithNullsCanResetOffset()
|
||||
{
|
||||
for (ImmutableBitmap bitmap : bitmaps) {
|
||||
ColumnarLongs longs = new ColumnarLongs()
|
||||
{
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return numRows;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long get(int index)
|
||||
{
|
||||
return ThreadLocalRandom.current().nextLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
LongsColumn columnWithNulls = LongsColumn.create(longs, bitmap);
|
||||
ColumnValueSelector<?> selector = columnWithNulls.makeColumnValueSelector(offset);
|
||||
assertOffsetCanReset(selector, bitmap, offset);
|
||||
VectorValueSelector vectorSelector = columnWithNulls.makeVectorValueSelector(vectorOffset);
|
||||
assertVectorOffsetCanReset(vectorSelector, bitmap, vectorOffset);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFloatSelectorWithNullsCanResetOffset()
|
||||
{
|
||||
for (ImmutableBitmap bitmap : bitmaps) {
|
||||
ColumnarFloats floats = new ColumnarFloats()
|
||||
{
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return numRows;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float get(int index)
|
||||
{
|
||||
return ThreadLocalRandom.current().nextFloat();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
FloatsColumn columnWithNulls = FloatsColumn.create(floats, bitmap);
|
||||
ColumnValueSelector<?> selector = columnWithNulls.makeColumnValueSelector(offset);
|
||||
assertOffsetCanReset(selector, bitmap, offset);
|
||||
VectorValueSelector vectorSelector = columnWithNulls.makeVectorValueSelector(vectorOffset);
|
||||
assertVectorOffsetCanReset(vectorSelector, bitmap, vectorOffset);
|
||||
VectorValueSelector anotherSelector = columnWithNulls.makeVectorValueSelector(anotherVectorOffset);
|
||||
assertVectorChillWhenOffsetsOverlap(anotherSelector, bitmap, anotherVectorOffset);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDoubleSelectorWithNullsCanResetOffset()
|
||||
{
|
||||
for (ImmutableBitmap bitmap : bitmaps) {
|
||||
ColumnarDoubles doubles = new ColumnarDoubles()
|
||||
{
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return numRows;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double get(int index)
|
||||
{
|
||||
return ThreadLocalRandom.current().nextDouble();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
DoublesColumn columnWithNulls = DoublesColumn.create(doubles, bitmap);
|
||||
ColumnValueSelector<?> selector = columnWithNulls.makeColumnValueSelector(offset);
|
||||
assertOffsetCanReset(selector, bitmap, offset);
|
||||
VectorValueSelector vectorSelector = columnWithNulls.makeVectorValueSelector(vectorOffset);
|
||||
assertVectorOffsetCanReset(vectorSelector, bitmap, vectorOffset);
|
||||
}
|
||||
}
|
||||
|
||||
private static void assertOffsetCanReset(
|
||||
BaseNullableColumnValueSelector selector,
|
||||
ImmutableBitmap bitmap,
|
||||
SimpleAscendingOffset readItAll
|
||||
)
|
||||
{
|
||||
boolean encounteredNull = false;
|
||||
while (readItAll.withinBounds()) {
|
||||
Assert.assertEquals(bitmap.get(readItAll.getOffset()), selector.isNull());
|
||||
encounteredNull |= selector.isNull();
|
||||
readItAll.increment();
|
||||
}
|
||||
readItAll.reset();
|
||||
Assert.assertTrue(encounteredNull);
|
||||
encounteredNull = false;
|
||||
while (readItAll.withinBounds()) {
|
||||
Assert.assertEquals(bitmap.get(readItAll.getOffset()), selector.isNull());
|
||||
encounteredNull |= selector.isNull();
|
||||
readItAll.increment();
|
||||
}
|
||||
Assert.assertTrue(encounteredNull);
|
||||
readItAll.reset();
|
||||
}
|
||||
|
||||
private static void assertVectorOffsetCanReset(
|
||||
VectorValueSelector selector,
|
||||
ImmutableBitmap bitmap,
|
||||
NoFilterVectorOffset readAllVectors
|
||||
)
|
||||
{
|
||||
boolean encounteredNull = false;
|
||||
boolean nullVector[];
|
||||
|
||||
// read it all, advancing offset
|
||||
while (!readAllVectors.isDone()) {
|
||||
nullVector = selector.getNullVector();
|
||||
for (int i = readAllVectors.getStartOffset(); i < readAllVectors.getCurrentVectorSize(); i++) {
|
||||
Assert.assertEquals(bitmap.get(readAllVectors.getStartOffset() + i), nullVector[i]);
|
||||
encounteredNull |= nullVector[i];
|
||||
}
|
||||
readAllVectors.advance();
|
||||
}
|
||||
// reset and read it all again to make sure matches
|
||||
readAllVectors.reset();
|
||||
Assert.assertTrue(encounteredNull);
|
||||
encounteredNull = false;
|
||||
while (!readAllVectors.isDone()) {
|
||||
nullVector = selector.getNullVector();
|
||||
for (int i = readAllVectors.getStartOffset(); i < readAllVectors.getCurrentVectorSize(); i++) {
|
||||
Assert.assertEquals(bitmap.get(readAllVectors.getStartOffset() + i), nullVector[i]);
|
||||
encounteredNull |= nullVector[i];
|
||||
}
|
||||
readAllVectors.advance();
|
||||
}
|
||||
Assert.assertTrue(encounteredNull);
|
||||
readAllVectors.reset();
|
||||
}
|
||||
|
||||
public static void assertVectorChillWhenOffsetsOverlap(
|
||||
VectorValueSelector selector,
|
||||
ImmutableBitmap bitmap,
|
||||
NoFilterOffsetThatCanBeMangledToTestOverlapping readAllVectors
|
||||
)
|
||||
{
|
||||
boolean encounteredNull = false;
|
||||
boolean nullVector[];
|
||||
|
||||
// test overlapping reads (should reset iterator anyway)
|
||||
readAllVectors.mangleOffset(0);
|
||||
nullVector = selector.getNullVector();
|
||||
for (int i = readAllVectors.getStartOffset(); i < readAllVectors.getCurrentVectorSize(); i++) {
|
||||
Assert.assertEquals(bitmap.get(readAllVectors.getStartOffset() + i), nullVector[i]);
|
||||
encounteredNull |= nullVector[i];
|
||||
}
|
||||
Assert.assertTrue(encounteredNull);
|
||||
// this can't currently happen, but we want to protect selectors in case offsets ever try to overlap
|
||||
readAllVectors.mangleOffset(1);
|
||||
|
||||
nullVector = selector.getNullVector();
|
||||
for (int i = readAllVectors.getStartOffset(); i < readAllVectors.getCurrentVectorSize(); i++) {
|
||||
Assert.assertEquals(bitmap.get(readAllVectors.getStartOffset() + i), nullVector[i]);
|
||||
encounteredNull |= nullVector[i];
|
||||
}
|
||||
readAllVectors.reset();
|
||||
|
||||
Assert.assertTrue(encounteredNull);
|
||||
}
|
||||
|
||||
private static class NoFilterOffsetThatCanBeMangledToTestOverlapping implements VectorOffset
|
||||
{
|
||||
private final int maxVectorSize;
|
||||
private final int start;
|
||||
private final int end;
|
||||
private int theOffset;
|
||||
|
||||
NoFilterOffsetThatCanBeMangledToTestOverlapping(final int maxVectorSize, final int start, final int end)
|
||||
{
|
||||
this.maxVectorSize = maxVectorSize;
|
||||
this.start = start;
|
||||
this.end = end;
|
||||
reset();
|
||||
}
|
||||
|
||||
public void mangleOffset(int replacement)
|
||||
{
|
||||
theOffset = replacement;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getId()
|
||||
{
|
||||
return theOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void advance()
|
||||
{
|
||||
theOffset += maxVectorSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isDone()
|
||||
{
|
||||
return theOffset >= end;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isContiguous()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return maxVectorSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
return Math.min(maxVectorSize, end - theOffset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getStartOffset()
|
||||
{
|
||||
return theOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] getOffsets()
|
||||
{
|
||||
throw new UnsupportedOperationException("no filter");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset()
|
||||
{
|
||||
theOffset = start;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,124 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.vector;
|
||||
|
||||
import org.apache.druid.collections.IntSetTestUtility;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.collections.bitmap.MutableBitmap;
|
||||
import org.apache.druid.collections.bitmap.WrappedBitSetBitmap;
|
||||
import org.apache.druid.collections.bitmap.WrappedConciseBitmap;
|
||||
import org.apache.druid.collections.bitmap.WrappedImmutableConciseBitmap;
|
||||
import org.apache.druid.collections.bitmap.WrappedRoaringBitmap;
|
||||
import org.apache.druid.extendedset.intset.ImmutableConciseSet;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
public class VectorSelectorUtilsTest
|
||||
{
|
||||
@Test
|
||||
public void testBitSetNullVector()
|
||||
{
|
||||
final WrappedBitSetBitmap bitmap = new WrappedBitSetBitmap();
|
||||
populate(bitmap);
|
||||
assertNullVector(bitmap);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConciseMutableNullVector()
|
||||
{
|
||||
final WrappedConciseBitmap bitmap = new WrappedConciseBitmap();
|
||||
populate(bitmap);
|
||||
assertNullVector(bitmap);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConciseImmutableNullVector()
|
||||
{
|
||||
final WrappedConciseBitmap bitmap = new WrappedConciseBitmap();
|
||||
populate(bitmap);
|
||||
final ImmutableBitmap immutable = new WrappedImmutableConciseBitmap(
|
||||
ImmutableConciseSet.newImmutableFromMutable(bitmap.getBitmap())
|
||||
);
|
||||
assertNullVector(immutable);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRoaringMutableNullVector()
|
||||
{
|
||||
WrappedRoaringBitmap bitmap = new WrappedRoaringBitmap();
|
||||
populate(bitmap);
|
||||
assertNullVector(bitmap);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRoaringImmutableNullVector()
|
||||
{
|
||||
WrappedRoaringBitmap bitmap = new WrappedRoaringBitmap();
|
||||
populate(bitmap);
|
||||
assertNullVector(bitmap.toImmutableBitmap());
|
||||
}
|
||||
|
||||
public static void populate(MutableBitmap bitmap)
|
||||
{
|
||||
for (int i : IntSetTestUtility.getSetBits()) {
|
||||
bitmap.add(i);
|
||||
}
|
||||
}
|
||||
|
||||
private void assertNullVector(ImmutableBitmap bitmap)
|
||||
{
|
||||
PeekableIntIterator iterator = bitmap.peekableIterator();
|
||||
Set<Integer> nulls = IntSetTestUtility.getSetBits();
|
||||
final int vectorSize = 32;
|
||||
final boolean[] nullVector = new boolean[vectorSize];
|
||||
ReadableVectorOffset someOffset = new NoFilterVectorOffset(vectorSize, 0, vectorSize);
|
||||
|
||||
VectorSelectorUtils.populateNullVector(nullVector, someOffset, iterator);
|
||||
for (int i = 0; i < vectorSize; i++) {
|
||||
Assert.assertEquals(nulls.contains(i), nullVector[i]);
|
||||
}
|
||||
|
||||
iterator = bitmap.peekableIterator();
|
||||
final int smallerVectorSize = 8;
|
||||
boolean[] smallVector = null;
|
||||
for (int offset = 0; offset < smallerVectorSize * 4; offset += smallerVectorSize) {
|
||||
ReadableVectorOffset smallOffset = new NoFilterVectorOffset(smallerVectorSize, offset, offset + smallerVectorSize);
|
||||
smallVector = VectorSelectorUtils.populateNullVector(smallVector, smallOffset, iterator);
|
||||
for (int i = 0; i < smallerVectorSize; i++) {
|
||||
if (smallVector == null) {
|
||||
Assert.assertFalse(nulls.contains(offset + i));
|
||||
} else {
|
||||
Assert.assertEquals(nulls.contains(offset + i), smallVector[i]);
|
||||
}
|
||||
}
|
||||
smallVector = null;
|
||||
}
|
||||
|
||||
iterator = bitmap.peekableIterator();
|
||||
ReadableVectorOffset allTheNulls = new BitmapVectorOffset(8, bitmap, 0, 22);
|
||||
smallVector = VectorSelectorUtils.populateNullVector(smallVector, allTheNulls, iterator);
|
||||
for (int i = 0; i < nulls.size(); i++) {
|
||||
Assert.assertTrue(smallVector[i]);
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue