mirror of
https://github.com/apache/druid.git
synced 2025-02-17 07:25:02 +00:00
Remove IndexedInts.iterator() (#4811)
* Remove IndexedInts.iterator() * Retain IndexedInts.iterator(), but don't extend Iterable * Add BitmapValues * Fix tests
This commit is contained in:
parent
47ebc48059
commit
a9d8539802
@ -22,6 +22,7 @@ package io.druid.query.aggregation.distinctcount;
|
||||
import io.druid.collections.bitmap.MutableBitmap;
|
||||
import io.druid.query.aggregation.Aggregator;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
|
||||
public class DistinctCountAggregator implements Aggregator
|
||||
{
|
||||
@ -41,7 +42,9 @@ public class DistinctCountAggregator implements Aggregator
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
for (final Integer index : selector.getRow()) {
|
||||
IndexedInts row = selector.getRow();
|
||||
for (int i = 0; i < row.size(); i++) {
|
||||
int index = row.get(i);
|
||||
mutableBitmap.add(index);
|
||||
}
|
||||
}
|
||||
|
@ -24,6 +24,7 @@ import io.druid.collections.bitmap.WrappedRoaringBitmap;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
|
||||
|
||||
@ -51,7 +52,9 @@ public class DistinctCountBufferAggregator implements BufferAggregator
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
MutableBitmap mutableBitmap = getMutableBitmap(buf, position);
|
||||
for (final Integer index : selector.getRow()) {
|
||||
IndexedInts row = selector.getRow();
|
||||
for (int i = 0; i < row.size(); i++) {
|
||||
int index = row.get(i);
|
||||
mutableBitmap.add(index);
|
||||
}
|
||||
buf.putLong(position, mutableBitmap.size());
|
||||
|
@ -24,7 +24,6 @@ import io.druid.hll.HyperLogLogCollector;
|
||||
import io.druid.query.aggregation.cardinality.CardinalityAggregator;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
@ -62,8 +61,9 @@ public class StringCardinalityAggregatorColumnSelectorStrategy implements Cardin
|
||||
@Override
|
||||
public void hashValues(DimensionSelector dimSelector, HyperLogLogCollector collector)
|
||||
{
|
||||
for (IntIterator rowIt = dimSelector.getRow().iterator(); rowIt.hasNext(); ) {
|
||||
int index = rowIt.nextInt();
|
||||
IndexedInts row = dimSelector.getRow();
|
||||
for (int i = 0; i < row.size(); i++) {
|
||||
int index = row.get(i);
|
||||
final String value = dimSelector.lookupName(index);
|
||||
collector.add(CardinalityAggregator.hashFn.hashUnencodedChars(nullToSpecial(value)).asBytes());
|
||||
}
|
||||
|
@ -196,8 +196,9 @@ public class GroupByQueryEngine
|
||||
newKey.putInt(MISSING_VALUE);
|
||||
unaggregatedBuffers = updateValues(newKey, dims.subList(1, dims.size()));
|
||||
} else {
|
||||
for (Integer dimValue : row) {
|
||||
for (int i = 0; i < row.size(); i++) {
|
||||
ByteBuffer newKey = key.duplicate();
|
||||
int dimValue = row.get(i);
|
||||
newKey.putInt(dimValue);
|
||||
unaggregatedBuffers = updateValues(newKey, dims.subList(1, dims.size()));
|
||||
}
|
||||
|
@ -25,11 +25,9 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.data.CompressedObjectStrategy;
|
||||
import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.data.IndexedIntsIterator;
|
||||
import io.druid.segment.data.IndexedIterable;
|
||||
import io.druid.segment.data.IndexedMultivalue;
|
||||
import io.druid.segment.data.WritableSupplier;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
@ -206,12 +204,6 @@ public class CompressedVSizeIndexedSupplier implements WritableSupplier<IndexedM
|
||||
// no-op
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return new IndexedIntsIterator(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
|
@ -20,8 +20,8 @@
|
||||
package io.druid.segment;
|
||||
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.data.BitmapValues;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.Map;
|
||||
@ -43,7 +43,7 @@ public interface IndexableAdapter
|
||||
|
||||
Iterable<Rowboat> getRows();
|
||||
|
||||
IndexedInts getBitmapIndex(String dimension, int dictId);
|
||||
BitmapValues getBitmapValues(String dimension, int dictId);
|
||||
|
||||
String getMetricType(String metric);
|
||||
|
||||
|
@ -43,7 +43,7 @@ public final class IntIteratorUtils
|
||||
}
|
||||
int skipped = 0;
|
||||
while (skipped < n && it.hasNext()) {
|
||||
it.next();
|
||||
it.nextInt();
|
||||
skipped++;
|
||||
}
|
||||
return skipped;
|
||||
|
@ -39,10 +39,9 @@ import io.druid.segment.column.IndexedDoublesGenericColumn;
|
||||
import io.druid.segment.column.IndexedFloatsGenericColumn;
|
||||
import io.druid.segment.column.IndexedLongsGenericColumn;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.data.BitmapCompressedIndexedInts;
|
||||
import io.druid.segment.data.EmptyIndexedInts;
|
||||
import io.druid.segment.data.ImmutableBitmapValues;
|
||||
import io.druid.segment.data.BitmapValues;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.data.IndexedIterable;
|
||||
import io.druid.segment.data.ListIndexed;
|
||||
import org.joda.time.Interval;
|
||||
@ -300,23 +299,6 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||
};
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
IndexedInts getBitmapIndex(String dimension, String value)
|
||||
{
|
||||
final Column column = input.getColumn(dimension);
|
||||
|
||||
if (column == null) {
|
||||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||
}
|
||||
|
||||
final BitmapIndex bitmaps = column.getBitmapIndex();
|
||||
if (bitmaps == null) {
|
||||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||
}
|
||||
|
||||
return new BitmapCompressedIndexedInts(bitmaps.getBitmap(bitmaps.getIndex(value)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getMetricType(String metric)
|
||||
{
|
||||
@ -347,25 +329,42 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getBitmapIndex(String dimension, int dictId)
|
||||
public BitmapValues getBitmapValues(String dimension, int dictId)
|
||||
{
|
||||
final Column column = input.getColumn(dimension);
|
||||
if (column == null) {
|
||||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||
return BitmapValues.EMPTY;
|
||||
}
|
||||
|
||||
final BitmapIndex bitmaps = column.getBitmapIndex();
|
||||
if (bitmaps == null) {
|
||||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||
return BitmapValues.EMPTY;
|
||||
}
|
||||
|
||||
if (dictId >= 0) {
|
||||
return new BitmapCompressedIndexedInts(bitmaps.getBitmap(dictId));
|
||||
return new ImmutableBitmapValues(bitmaps.getBitmap(dictId));
|
||||
} else {
|
||||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||
return BitmapValues.EMPTY;
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
BitmapValues getBitmapIndex(String dimension, String value)
|
||||
{
|
||||
final Column column = input.getColumn(dimension);
|
||||
|
||||
if (column == null) {
|
||||
return BitmapValues.EMPTY;
|
||||
}
|
||||
|
||||
final BitmapIndex bitmaps = column.getBitmapIndex();
|
||||
if (bitmaps == null) {
|
||||
return BitmapValues.EMPTY;
|
||||
}
|
||||
|
||||
return new ImmutableBitmapValues(bitmaps.getBitmap(bitmaps.getIndex(value)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Metadata getMetadata()
|
||||
{
|
||||
|
@ -22,8 +22,8 @@ package io.druid.segment;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Iterables;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.data.BitmapValues;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.Map;
|
||||
@ -90,9 +90,9 @@ public class RowboatFilteringIndexAdapter implements IndexableAdapter
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getBitmapIndex(String dimension, int dictId)
|
||||
public BitmapValues getBitmapValues(String dimension, int dictId)
|
||||
{
|
||||
return baseAdapter.getBitmapIndex(dimension, dictId);
|
||||
return baseAdapter.getBitmapValues(dimension, dictId);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -40,6 +40,7 @@ import io.druid.segment.column.ColumnDescriptor;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.data.ArrayIndexed;
|
||||
import io.druid.segment.data.BitmapSerdeFactory;
|
||||
import io.druid.segment.data.BitmapValues;
|
||||
import io.druid.segment.data.ByteBufferWriter;
|
||||
import io.druid.segment.data.CompressedObjectStrategy;
|
||||
import io.druid.segment.data.CompressedVSizeIndexedV3Writer;
|
||||
@ -48,7 +49,6 @@ import io.druid.segment.data.GenericIndexed;
|
||||
import io.druid.segment.data.GenericIndexedWriter;
|
||||
import io.druid.segment.data.IOPeon;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.data.IndexedIntsWriter;
|
||||
import io.druid.segment.data.IndexedRTree;
|
||||
import io.druid.segment.data.VSizeIndexedIntsWriter;
|
||||
@ -58,6 +58,7 @@ import it.unimi.dsi.fastutil.ints.AbstractIntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterable;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
@ -370,13 +371,14 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||
GenericIndexedWriter<ImmutableBitmap> bitmapWriter
|
||||
) throws IOException
|
||||
{
|
||||
List<ConvertingIndexedInts> convertedInvertedIndexesToMerge = Lists.newArrayListWithCapacity(adapters.size());
|
||||
List<ConvertingBitmapValues> convertedInvertedIndexesToMerge = Lists.newArrayListWithCapacity(adapters.size());
|
||||
for (int j = 0; j < adapters.size(); ++j) {
|
||||
int seekedDictId = dictIdSeeker[j].seek(dictId);
|
||||
if (seekedDictId != IndexSeeker.NOT_EXIST) {
|
||||
convertedInvertedIndexesToMerge.add(
|
||||
new ConvertingIndexedInts(
|
||||
adapters.get(j).getBitmapIndex(dimensionName, seekedDictId), segmentRowNumConversions.get(j)
|
||||
new ConvertingBitmapValues(
|
||||
adapters.get(j).getBitmapValues(dimensionName, seekedDictId),
|
||||
segmentRowNumConversions.get(j)
|
||||
)
|
||||
);
|
||||
}
|
||||
@ -384,7 +386,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||
|
||||
MutableBitmap mergedIndexes = bmpFactory.makeEmptyMutableBitmap();
|
||||
List<IntIterator> convertedInvertedIndexesIterators = new ArrayList<>(convertedInvertedIndexesToMerge.size());
|
||||
for (ConvertingIndexedInts convertedInvertedIndexes : convertedInvertedIndexesToMerge) {
|
||||
for (ConvertingBitmapValues convertedInvertedIndexes : convertedInvertedIndexesToMerge) {
|
||||
convertedInvertedIndexesIterators.add(convertedInvertedIndexes.iterator());
|
||||
}
|
||||
|
||||
@ -538,34 +540,27 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||
}
|
||||
}
|
||||
|
||||
public static class ConvertingIndexedInts implements IntIterable
|
||||
public static class ConvertingBitmapValues implements IntIterable
|
||||
{
|
||||
private final IndexedInts baseIndex;
|
||||
private final BitmapValues baseValues;
|
||||
private final IntBuffer conversionBuffer;
|
||||
|
||||
public ConvertingIndexedInts(
|
||||
IndexedInts baseIndex,
|
||||
IntBuffer conversionBuffer
|
||||
)
|
||||
ConvertingBitmapValues(BitmapValues baseValues, IntBuffer conversionBuffer)
|
||||
{
|
||||
this.baseIndex = baseIndex;
|
||||
this.baseValues = baseValues;
|
||||
this.conversionBuffer = conversionBuffer;
|
||||
}
|
||||
|
||||
public int size()
|
||||
{
|
||||
return baseIndex.size();
|
||||
}
|
||||
|
||||
public int get(int index)
|
||||
{
|
||||
return conversionBuffer.get(baseIndex.get(index));
|
||||
return baseValues.size();
|
||||
}
|
||||
|
||||
@Nonnull
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
final IntIterator baseIterator = baseIndex.iterator();
|
||||
final IntIterator baseIterator = baseValues.iterator();
|
||||
return new AbstractIntIterator()
|
||||
{
|
||||
@Override
|
||||
|
@ -22,8 +22,6 @@ package io.druid.segment.data;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import it.unimi.dsi.fastutil.ints.IntArrays;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
@ -76,12 +74,6 @@ public final class ArrayBasedIndexedInts implements IndexedInts
|
||||
return expansion[index];
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return IntIterators.wrap(expansion, 0, size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
|
@ -1,101 +0,0 @@
|
||||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.segment.data;
|
||||
|
||||
import com.google.common.collect.Ordering;
|
||||
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.IntIteratorUtils;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class BitmapCompressedIndexedInts implements IndexedInts, Comparable<ImmutableBitmap>
|
||||
{
|
||||
private static final Ordering<ImmutableBitmap> COMPARATOR = new Ordering<ImmutableBitmap>()
|
||||
{
|
||||
@Override
|
||||
public int compare(
|
||||
ImmutableBitmap set, ImmutableBitmap set1
|
||||
)
|
||||
{
|
||||
if (set.isEmpty() && set1.isEmpty()) {
|
||||
return 0;
|
||||
}
|
||||
if (set.isEmpty()) {
|
||||
return -1;
|
||||
}
|
||||
if (set1.isEmpty()) {
|
||||
return 1;
|
||||
}
|
||||
return set.compareTo(set1);
|
||||
}
|
||||
}.nullsFirst();
|
||||
|
||||
private final ImmutableBitmap immutableBitmap;
|
||||
|
||||
public BitmapCompressedIndexedInts(ImmutableBitmap immutableBitmap)
|
||||
{
|
||||
this.immutableBitmap = immutableBitmap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(@Nullable ImmutableBitmap otherBitmap)
|
||||
{
|
||||
return COMPARATOR.compare(immutableBitmap, otherBitmap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return immutableBitmap.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int get(int index)
|
||||
{
|
||||
throw new UnsupportedOperationException("This is really slow, so it's just not supported.");
|
||||
}
|
||||
|
||||
public ImmutableBitmap getImmutableBitmap()
|
||||
{
|
||||
return immutableBitmap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return IntIteratorUtils.fromRoaringBitmapIntIterator(immutableBitmap.iterator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("immutableBitmap", immutableBitmap);
|
||||
}
|
||||
}
|
@ -0,0 +1,48 @@
|
||||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.segment.data;
|
||||
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||
|
||||
/**
|
||||
* Doesn't extend {@link it.unimi.dsi.fastutil.ints.IntIterable} to avoid accidential for-each iteration with boxing.
|
||||
*/
|
||||
public interface BitmapValues
|
||||
{
|
||||
BitmapValues EMPTY = new BitmapValues()
|
||||
{
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return IntIterators.EMPTY_ITERATOR;
|
||||
}
|
||||
};
|
||||
|
||||
int size();
|
||||
|
||||
IntIterator iterator();
|
||||
}
|
@ -29,7 +29,6 @@ import io.druid.java.util.common.guava.CloseQuietly;
|
||||
import io.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.CompressedPools;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
@ -311,12 +310,6 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier<IndexedIn
|
||||
return buffer.get(buffer.position() + bufferIndex);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return new IndexedIntsIterator(this);
|
||||
}
|
||||
|
||||
protected void loadBuffer(int bufferNum)
|
||||
{
|
||||
CloseQuietly.close(holder);
|
||||
|
@ -31,7 +31,6 @@ import io.druid.java.util.common.guava.CloseQuietly;
|
||||
import io.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.CompressedPools;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
@ -374,12 +373,6 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier<Inde
|
||||
buffer.getInt(pos) & littleEndianMask;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return new IndexedIntsIterator(this);
|
||||
}
|
||||
|
||||
protected void loadBuffer(int bufferNum)
|
||||
{
|
||||
CloseQuietly.close(holder);
|
||||
|
@ -20,8 +20,6 @@
|
||||
package io.druid.segment.data;
|
||||
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
@ -47,12 +45,6 @@ public class EmptyIndexedInts implements IndexedInts
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return IntIterators.EMPTY_ITERATOR;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
|
@ -19,43 +19,30 @@
|
||||
|
||||
package io.druid.segment.data;
|
||||
|
||||
|
||||
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.segment.IntIteratorUtils;
|
||||
import it.unimi.dsi.fastutil.ints.AbstractIntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IndexedIntsIterator extends AbstractIntIterator
|
||||
public class ImmutableBitmapValues implements BitmapValues
|
||||
{
|
||||
private final IndexedInts baseInts;
|
||||
private final int size;
|
||||
private final ImmutableBitmap immutableBitmap;
|
||||
|
||||
private int currIndex = 0;
|
||||
|
||||
public IndexedIntsIterator(
|
||||
IndexedInts baseInts
|
||||
)
|
||||
public ImmutableBitmapValues(ImmutableBitmap immutableBitmap)
|
||||
{
|
||||
this.baseInts = baseInts;
|
||||
|
||||
size = baseInts.size();
|
||||
this.immutableBitmap = immutableBitmap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
public int size()
|
||||
{
|
||||
return currIndex < size;
|
||||
return immutableBitmap.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextInt()
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return baseInts.get(currIndex++);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int skip(int n)
|
||||
{
|
||||
return IntIteratorUtils.skip(this, n);
|
||||
return IntIteratorUtils.fromRoaringBitmapIntIterator(immutableBitmap.iterator());
|
||||
}
|
||||
}
|
@ -21,17 +21,28 @@ package io.druid.segment.data;
|
||||
|
||||
import io.druid.query.monomorphicprocessing.CalledFromHotLoop;
|
||||
import io.druid.query.monomorphicprocessing.HotLoopCallee;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterable;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.function.IntConsumer;
|
||||
|
||||
/**
|
||||
* Get a int an index (array or list lookup abstraction without boxing).
|
||||
*
|
||||
* Doesn't extend {@link Iterable} (or {@link it.unimi.dsi.fastutil.ints.IntIterable} to avoid accidential
|
||||
* for-each iteration with boxing.
|
||||
*/
|
||||
public interface IndexedInts extends IntIterable, Closeable, HotLoopCallee
|
||||
public interface IndexedInts extends Closeable, HotLoopCallee
|
||||
{
|
||||
@CalledFromHotLoop
|
||||
int size();
|
||||
@CalledFromHotLoop
|
||||
int get(int index);
|
||||
|
||||
default void forEach(IntConsumer action)
|
||||
{
|
||||
int size = size();
|
||||
for (int i = 0; i < size; i++) {
|
||||
action.accept(get(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -21,8 +21,6 @@ package io.druid.segment.data;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
@ -72,12 +70,6 @@ public class RangeIndexedInts implements IndexedInts
|
||||
return index;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return IntIterators.fromTo(0, size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
|
@ -20,8 +20,6 @@
|
||||
package io.druid.segment.data;
|
||||
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
@ -49,12 +47,6 @@ public final class SingleIndexedInt implements IndexedInts
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return IntIterators.singleton(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
|
@ -23,7 +23,6 @@ import com.google.common.collect.Lists;
|
||||
import com.google.common.primitives.Ints;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
@ -177,12 +176,6 @@ public class VSizeIndexedInts implements IndexedInts, Comparable<VSizeIndexedInt
|
||||
return 1 + 1 + 4 + buffer.remaining();
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return new IndexedIntsIterator(this);
|
||||
}
|
||||
|
||||
public void writeToChannel(WritableByteChannel channel) throws IOException
|
||||
{
|
||||
channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes}));
|
||||
|
@ -20,8 +20,6 @@
|
||||
package io.druid.segment.data;
|
||||
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
@ -56,12 +54,6 @@ public class ZeroIndexedInts implements IndexedInts
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return IntIterators.singleton(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
|
@ -25,7 +25,6 @@ import com.google.common.collect.Maps;
|
||||
import io.druid.collections.bitmap.BitmapFactory;
|
||||
import io.druid.collections.bitmap.MutableBitmap;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.DimensionHandler;
|
||||
import io.druid.segment.DimensionIndexer;
|
||||
import io.druid.segment.IndexableAdapter;
|
||||
@ -33,14 +32,12 @@ import io.druid.segment.IntIteratorUtils;
|
||||
import io.druid.segment.Metadata;
|
||||
import io.druid.segment.Rowboat;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.data.EmptyIndexedInts;
|
||||
import io.druid.segment.data.BitmapValues;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.data.ListIndexed;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@ -223,31 +220,31 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getBitmapIndex(String dimension, int index)
|
||||
public BitmapValues getBitmapValues(String dimension, int index)
|
||||
{
|
||||
DimensionAccessor accessor = accessors.get(dimension);
|
||||
if (accessor == null) {
|
||||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||
return BitmapValues.EMPTY;
|
||||
}
|
||||
ColumnCapabilities capabilities = accessor.dimensionDesc.getCapabilities();
|
||||
DimensionIndexer indexer = accessor.dimensionDesc.getIndexer();
|
||||
|
||||
if (!capabilities.hasBitmapIndexes()) {
|
||||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||
return BitmapValues.EMPTY;
|
||||
}
|
||||
|
||||
final int id = (Integer) indexer.getUnsortedEncodedValueFromSorted(index);
|
||||
if (id < 0 || id >= indexer.getCardinality()) {
|
||||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||
return BitmapValues.EMPTY;
|
||||
}
|
||||
|
||||
MutableBitmap bitmapIndex = accessor.invertedIndexes[id];
|
||||
|
||||
if (bitmapIndex == null) {
|
||||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||
return BitmapValues.EMPTY;
|
||||
}
|
||||
|
||||
return new BitmapIndexedInts(bitmapIndex);
|
||||
return new MutableBitmapValues(bitmapIndex);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -262,12 +259,11 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||
return index.getCapabilities(column);
|
||||
}
|
||||
|
||||
static class BitmapIndexedInts implements IndexedInts
|
||||
static class MutableBitmapValues implements BitmapValues
|
||||
{
|
||||
|
||||
private final MutableBitmap bitmapIndex;
|
||||
|
||||
BitmapIndexedInts(MutableBitmap bitmapIndex)
|
||||
MutableBitmapValues(MutableBitmap bitmapIndex)
|
||||
{
|
||||
this.bitmapIndex = bitmapIndex;
|
||||
}
|
||||
@ -278,29 +274,11 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||
return bitmapIndex.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int get(int index)
|
||||
{
|
||||
// Slow for concise bitmaps, but is fast with roaring bitmaps, so it's just not supported.
|
||||
throw new UnsupportedOperationException("Not supported.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return IntIteratorUtils.fromRoaringBitmapIntIterator(bitmapIndex.iterator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("bitmapIndex", bitmapIndex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -48,8 +48,6 @@ import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.DimensionSelectorUtils;
|
||||
import io.druid.segment.IdLookup;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
@ -142,12 +140,6 @@ public class CardinalityAggregatorTest
|
||||
return column.get(p)[i];
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntIterator iterator()
|
||||
{
|
||||
return IntIterators.asIntIterator(Iterators.forArray(column.get(p)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
|
@ -25,8 +25,6 @@ import io.druid.segment.data.IndexedInts;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
public class ConstantDimensionSelectorTest
|
||||
{
|
||||
private final DimensionSelector NULL_SELECTOR = DimensionSelectorUtils.constantSelector(null);
|
||||
@ -46,11 +44,6 @@ public class ConstantDimensionSelectorTest
|
||||
IndexedInts row = NULL_SELECTOR.getRow();
|
||||
Assert.assertEquals(1, row.size());
|
||||
Assert.assertEquals(0, row.get(0));
|
||||
|
||||
Iterator<Integer> iter = row.iterator();
|
||||
Assert.assertEquals(true, iter.hasNext());
|
||||
Assert.assertEquals(0, iter.next().intValue());
|
||||
Assert.assertEquals(false, iter.hasNext());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -49,14 +49,15 @@ import io.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import io.druid.segment.column.DictionaryEncodedColumn;
|
||||
import io.druid.segment.column.SimpleDictionaryEncodedColumn;
|
||||
import io.druid.segment.data.BitmapSerdeFactory;
|
||||
import io.druid.segment.data.BitmapValues;
|
||||
import io.druid.segment.data.CompressedObjectStrategy;
|
||||
import io.druid.segment.data.CompressionFactory;
|
||||
import io.druid.segment.data.IncrementalIndexTest;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexAdapter;
|
||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import io.druid.segment.incremental.IndexSizeExceededException;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
@ -1520,12 +1521,13 @@ public class IndexMergerTestBase
|
||||
checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d9", "921"));
|
||||
}
|
||||
|
||||
private void checkBitmapIndex(ArrayList<Integer> expected, IndexedInts real)
|
||||
private void checkBitmapIndex(ArrayList<Integer> expected, BitmapValues real)
|
||||
{
|
||||
Assert.assertEquals(expected.size(), real.size());
|
||||
int i = 0;
|
||||
for (Object index : real) {
|
||||
Assert.assertEquals(expected.get(i++), index);
|
||||
for (IntIterator iterator = real.iterator(); iterator.hasNext(); ) {
|
||||
int index = iterator.nextInt();
|
||||
Assert.assertEquals(expected.get(i++), (Integer) index);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -19,11 +19,11 @@
|
||||
|
||||
package io.druid.segment;
|
||||
|
||||
import io.druid.segment.data.BitmapValues;
|
||||
import io.druid.segment.data.CompressedObjectStrategy;
|
||||
import io.druid.segment.data.CompressionFactory;
|
||||
import io.druid.segment.data.ConciseBitmapSerdeFactory;
|
||||
import io.druid.segment.data.IncrementalIndexTest;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
@ -69,10 +69,10 @@ public class QueryableIndexIndexableAdapterTest
|
||||
IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index);
|
||||
String dimension = "dim1";
|
||||
//null is added to all dimensions with value
|
||||
IndexedInts indexedInts = adapter.getBitmapIndex(dimension, 0);
|
||||
BitmapValues bitmapValues = adapter.getBitmapValues(dimension, 0);
|
||||
for (int i = 0; i < adapter.getDimValueLookup(dimension).size(); i++) {
|
||||
indexedInts = adapter.getBitmapIndex(dimension, i);
|
||||
Assert.assertEquals(1, indexedInts.size());
|
||||
bitmapValues = adapter.getBitmapValues(dimension, i);
|
||||
Assert.assertEquals(1, bitmapValues.size());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -115,11 +115,8 @@ public class CompressedVSizeIndexedSupplierTest
|
||||
final IndexedInts vSizeIndexedInts = iterator.next();
|
||||
|
||||
Assert.assertEquals(ints.length, vSizeIndexedInts.size());
|
||||
Iterator<Integer> valsIterator = vSizeIndexedInts.iterator();
|
||||
int j = 0;
|
||||
while (valsIterator.hasNext()) {
|
||||
Assert.assertEquals((Integer) ints[j], valsIterator.next());
|
||||
j++;
|
||||
for (int i = 0; i < vSizeIndexedInts.size(); i++) {
|
||||
Assert.assertEquals(ints[i], vSizeIndexedInts.get(i));
|
||||
}
|
||||
row++;
|
||||
}
|
||||
|
@ -22,11 +22,11 @@ package io.druid.segment.incremental;
|
||||
import io.druid.segment.IndexSpec;
|
||||
import io.druid.segment.IndexableAdapter;
|
||||
import io.druid.segment.Rowboat;
|
||||
import io.druid.segment.data.BitmapValues;
|
||||
import io.druid.segment.data.CompressedObjectStrategy;
|
||||
import io.druid.segment.data.CompressionFactory;
|
||||
import io.druid.segment.data.ConciseBitmapSerdeFactory;
|
||||
import io.druid.segment.data.IncrementalIndexTest;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
@ -55,8 +55,8 @@ public class IncrementalIndexAdapterTest
|
||||
);
|
||||
String dimension = "dim1";
|
||||
for (int i = 0; i < adapter.getDimValueLookup(dimension).size(); i++) {
|
||||
IndexedInts indexedInts = adapter.getBitmapIndex(dimension, i);
|
||||
Assert.assertEquals(1, indexedInts.size());
|
||||
BitmapValues bitmapValues = adapter.getBitmapValues(dimension, i);
|
||||
Assert.assertEquals(1, bitmapValues.size());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -459,9 +459,7 @@ public class IncrementalIndexStorageAdapterTest
|
||||
// and then, cursoring continues in the other thread
|
||||
while (!cursor.isDone()) {
|
||||
IndexedInts row = dimSelector.getRow();
|
||||
for (int i : row) {
|
||||
Assert.assertTrue(i < cardinality);
|
||||
}
|
||||
row.forEach(i -> Assert.assertTrue(i < cardinality));
|
||||
cursor.advance();
|
||||
rowNumInCursor++;
|
||||
}
|
||||
@ -593,17 +591,11 @@ public class IncrementalIndexStorageAdapterTest
|
||||
// and then, cursoring continues in the other thread
|
||||
while (!cursor.isDone()) {
|
||||
IndexedInts rowA = dimSelector1A.getRow();
|
||||
for (int i : rowA) {
|
||||
Assert.assertTrue(i < cardinalityA);
|
||||
}
|
||||
rowA.forEach(i -> Assert.assertTrue(i < cardinalityA));
|
||||
IndexedInts rowB = dimSelector1B.getRow();
|
||||
for (int i : rowB) {
|
||||
Assert.assertTrue(i < cardinalityA);
|
||||
}
|
||||
rowB.forEach(i -> Assert.assertTrue(i < cardinalityA));
|
||||
IndexedInts rowC = dimSelector1C.getRow();
|
||||
for (int i : rowC) {
|
||||
Assert.assertTrue(i < cardinalityA);
|
||||
}
|
||||
rowC.forEach(i -> Assert.assertTrue(i < cardinalityA));
|
||||
IndexedInts rowD = dimSelector2D.getRow();
|
||||
// no null id, so should get empty dims array
|
||||
Assert.assertEquals(0, rowD.size());
|
||||
|
Loading…
x
Reference in New Issue
Block a user