mirror of https://github.com/apache/druid.git
Improve performance of IndexMergerV9 (#3440)
* Improve performance of StringDimensionMergerV9 and StringDimensionMergerLegacy by avoiding primitive int boxing by using IntIterator in IndexedInts instead of Iterator<Integer>; Extract some common logic for V9 and Legacy mergers; Minor improvements to resource handling in StringDimensionMergerV9 * Don't mask index in MergeIntIterator.makeQueueElement() * DRY conversion RoaringBitmap's IntIterator to fastutil's IntIterator * Do implement skip(n) in IntIterators extending AbstractIntIterator because original implementation is not reliable * Use Test(expected=Exception.class) instead of try { } catch (Exception e) { /* ignore */ }
This commit is contained in:
parent
ddc856214d
commit
85ac8eff90
5
pom.xml
5
pom.xml
|
@ -574,6 +574,11 @@
|
||||||
<artifactId>commons-math3</artifactId>
|
<artifactId>commons-math3</artifactId>
|
||||||
<version>3.6.1</version>
|
<version>3.6.1</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>it.unimi.dsi</groupId>
|
||||||
|
<artifactId>fastutil</artifactId>
|
||||||
|
<version>7.0.13</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
<!-- Test Scope -->
|
<!-- Test Scope -->
|
||||||
<dependency>
|
<dependency>
|
||||||
|
|
|
@ -39,6 +39,10 @@
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>bytebuffer-collections</artifactId>
|
<artifactId>bytebuffer-collections</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>it.unimi.dsi</groupId>
|
||||||
|
<artifactId>fastutil</artifactId>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>emitter</artifactId>
|
<artifactId>emitter</artifactId>
|
||||||
|
|
|
@ -50,14 +50,14 @@ import io.druid.segment.ObjectColumnSelector;
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
import io.druid.segment.column.ColumnCapabilities;
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -530,31 +530,30 @@ public class RowBasedGrouperHelper
|
||||||
public IndexedInts getRow()
|
public IndexedInts getRow()
|
||||||
{
|
{
|
||||||
final List<String> dimensionValues = row.get().getDimension(dimension);
|
final List<String> dimensionValues = row.get().getDimension(dimension);
|
||||||
final ArrayList<Integer> vals = Lists.newArrayList();
|
|
||||||
if (dimensionValues != null) {
|
final int dimensionValuesSize = dimensionValues != null ? dimensionValues.size() : 0;
|
||||||
for (int i = 0; i < dimensionValues.size(); ++i) {
|
|
||||||
vals.add(i);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return new IndexedInts()
|
return new IndexedInts()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public int size()
|
public int size()
|
||||||
{
|
{
|
||||||
return vals.size();
|
return dimensionValuesSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int get(int index)
|
public int get(int index)
|
||||||
{
|
{
|
||||||
return vals.get(index);
|
if (index < 0 || index >= dimensionValuesSize) {
|
||||||
|
throw new IndexOutOfBoundsException("index: " + index);
|
||||||
|
}
|
||||||
|
return index;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return vals.iterator();
|
return IntIterators.fromTo(0, dimensionValuesSize);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -27,6 +27,7 @@ import io.druid.segment.data.IndexedIntsIterator;
|
||||||
import io.druid.segment.data.IndexedIterable;
|
import io.druid.segment.data.IndexedIterable;
|
||||||
import io.druid.segment.data.IndexedMultivalue;
|
import io.druid.segment.data.IndexedMultivalue;
|
||||||
import io.druid.segment.data.WritableSupplier;
|
import io.druid.segment.data.WritableSupplier;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
@ -206,7 +207,7 @@ public class CompressedVSizeIndexedSupplier implements WritableSupplier<IndexedM
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return new IndexedIntsIterator(this);
|
return new IndexedIntsIterator(this);
|
||||||
}
|
}
|
||||||
|
|
|
@ -60,13 +60,12 @@ import io.druid.segment.column.ValueType;
|
||||||
import io.druid.segment.data.BitmapSerdeFactory;
|
import io.druid.segment.data.BitmapSerdeFactory;
|
||||||
import io.druid.segment.data.CompressedObjectStrategy;
|
import io.druid.segment.data.CompressedObjectStrategy;
|
||||||
import io.druid.segment.data.CompressionFactory;
|
import io.druid.segment.data.CompressionFactory;
|
||||||
import io.druid.segment.data.LongSupplierSerializer;
|
|
||||||
import io.druid.segment.data.GenericIndexed;
|
import io.druid.segment.data.GenericIndexed;
|
||||||
import io.druid.segment.data.IOPeon;
|
import io.druid.segment.data.IOPeon;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
|
||||||
import io.druid.segment.data.IndexedIterable;
|
import io.druid.segment.data.IndexedIterable;
|
||||||
import io.druid.segment.data.ListIndexed;
|
import io.druid.segment.data.ListIndexed;
|
||||||
|
import io.druid.segment.data.LongSupplierSerializer;
|
||||||
import io.druid.segment.data.TmpFileIOPeon;
|
import io.druid.segment.data.TmpFileIOPeon;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexAdapter;
|
import io.druid.segment.incremental.IncrementalIndexAdapter;
|
||||||
|
@ -1189,7 +1188,10 @@ public class IndexMerger
|
||||||
{
|
{
|
||||||
private final AggregatorFactory[] metricAggs;
|
private final AggregatorFactory[] metricAggs;
|
||||||
|
|
||||||
public AggFactoryStringIndexed(AggregatorFactory[] metricAggs) {this.metricAggs = metricAggs;}
|
public AggFactoryStringIndexed(AggregatorFactory[] metricAggs)
|
||||||
|
{
|
||||||
|
this.metricAggs = metricAggs;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Class<? extends String> getClazz()
|
public Class<? extends String> getClazz()
|
||||||
|
|
|
@ -30,13 +30,6 @@ import com.google.common.io.Closer;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.metamx.collections.bitmap.BitmapFactory;
|
|
||||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
|
||||||
import com.metamx.collections.bitmap.MutableBitmap;
|
|
||||||
import com.metamx.collections.spatial.ImmutableRTree;
|
|
||||||
import com.metamx.collections.spatial.RTree;
|
|
||||||
import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy;
|
|
||||||
import com.metamx.common.ByteBufferUtils;
|
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.io.smoosh.FileSmoosher;
|
import com.metamx.common.io.smoosh.FileSmoosher;
|
||||||
import com.metamx.common.io.smoosh.SmooshedWriter;
|
import com.metamx.common.io.smoosh.SmooshedWriter;
|
||||||
|
|
|
@ -0,0 +1,197 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import com.metamx.common.IAE;
|
||||||
|
import com.metamx.common.guava.MergeIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.AbstractIntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||||
|
import it.unimi.dsi.fastutil.longs.LongHeaps;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.NoSuchElementException;
|
||||||
|
|
||||||
|
public final class IntIteratorUtils
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* Implements {@link IntIterator#skip(int)}.
|
||||||
|
*/
|
||||||
|
public static int skip(IntIterator it, int n)
|
||||||
|
{
|
||||||
|
if (n < 0) {
|
||||||
|
throw new IAE("n: " + n);
|
||||||
|
}
|
||||||
|
int skipped = 0;
|
||||||
|
while (skipped < n && it.hasNext()) {
|
||||||
|
it.next();
|
||||||
|
skipped++;
|
||||||
|
}
|
||||||
|
return skipped;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Merges several iterators of ascending {@code int} values into a single iterator of ascending {@code int} values.
|
||||||
|
* It isn't checked if the given source iterators are actually ascending, if they are not, the order of values in the
|
||||||
|
* returned iterator is undefined.
|
||||||
|
* <p>
|
||||||
|
* This is similar to what {@link MergeIterator} does with simple {@link java.util.Iterator}s.
|
||||||
|
*
|
||||||
|
* @param iterators iterators to merge, must return ascending values
|
||||||
|
*/
|
||||||
|
public static IntIterator mergeAscending(List<IntIterator> iterators)
|
||||||
|
{
|
||||||
|
if (iterators.isEmpty()) {
|
||||||
|
return IntIterators.EMPTY_ITERATOR;
|
||||||
|
}
|
||||||
|
if (iterators.size() == 1) {
|
||||||
|
return iterators.get(0);
|
||||||
|
}
|
||||||
|
return new MergeIntIterator(iterators);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class is designed mostly after {@link MergeIterator}. {@code MergeIterator} uses a priority queue of wrapper
|
||||||
|
* "peeking" iterators. Peeking wrappers are not available in fastutil for specialized iterators like IntIterator, so
|
||||||
|
* they should be implemented manually in the druid codebase. Instead, another approach is taken: a priority queue
|
||||||
|
* of primitive long values is used, in long values the high 32-bits is the last value from some iterator, and the low
|
||||||
|
* 32 bits is the index of this iterator in the given list (copied to array, to avoid indirection during the
|
||||||
|
* iteration) of source iterators. Since values are in the high bits, the composed longs are still compared using the
|
||||||
|
* natural order. So this approach avoids indirections and implementing PeekingIntIterator.
|
||||||
|
* <p>
|
||||||
|
* Instead of {@link it.unimi.dsi.fastutil.longs.LongHeapPriorityQueue}, a priority queue is implemented on lower
|
||||||
|
* level, to avoid heap array shuffling on each iteration with dequeue and than enqueue, when merged iterators tend
|
||||||
|
* to stay in the head of the heap for at least several iterations.
|
||||||
|
*/
|
||||||
|
static final class MergeIntIterator extends AbstractIntIterator
|
||||||
|
{
|
||||||
|
|
||||||
|
private final IntIterator[] iterators;
|
||||||
|
private final long[] pQueue;
|
||||||
|
private int pQueueSize;
|
||||||
|
|
||||||
|
private static long makeQueueElement(int value, int index)
|
||||||
|
{
|
||||||
|
// Don't have to mask index because this is a Java array index => positive => no sign bit extension
|
||||||
|
return index | (((long) value) << 32);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static int value(long queueElement)
|
||||||
|
{
|
||||||
|
return (int) (queueElement >>> 32);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static int iterIndex(long queueElement)
|
||||||
|
{
|
||||||
|
return (int) queueElement;
|
||||||
|
}
|
||||||
|
|
||||||
|
MergeIntIterator(List<IntIterator> iterators)
|
||||||
|
{
|
||||||
|
this.iterators = iterators.toArray(new IntIterator[0]);
|
||||||
|
pQueue = new long[iterators.size()];
|
||||||
|
pQueueSize = 0;
|
||||||
|
|
||||||
|
for (int iterIndex = 0; iterIndex < this.iterators.length; iterIndex++) {
|
||||||
|
IntIterator iter = this.iterators[iterIndex];
|
||||||
|
if (iter != null && iter.hasNext()) {
|
||||||
|
pQueue[pQueueSize] = makeQueueElement(iter.nextInt(), iterIndex);
|
||||||
|
pQueueSize++;
|
||||||
|
LongHeaps.upHeap(pQueue, pQueueSize, pQueueSize - 1, null);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasNext()
|
||||||
|
{
|
||||||
|
return pQueueSize != 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int nextInt()
|
||||||
|
{
|
||||||
|
if (!hasNext()) {
|
||||||
|
throw new NoSuchElementException();
|
||||||
|
}
|
||||||
|
|
||||||
|
long queueHead = pQueue[0];
|
||||||
|
int retVal = value(queueHead);
|
||||||
|
int iterIndex = iterIndex(queueHead);
|
||||||
|
IntIterator retIt = iterators[iterIndex];
|
||||||
|
|
||||||
|
if (retIt.hasNext()) {
|
||||||
|
// stay in the head, likely no elements will be moved in the heap
|
||||||
|
pQueue[0] = makeQueueElement(retIt.nextInt(), iterIndex);
|
||||||
|
LongHeaps.downHeap(pQueue, pQueueSize, 0, null);
|
||||||
|
} else {
|
||||||
|
pQueueSize--;
|
||||||
|
if (pQueueSize != 0) {
|
||||||
|
pQueue[0] = pQueue[pQueueSize];
|
||||||
|
LongHeaps.downHeap(pQueue, pQueueSize, 0, null);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return retVal;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int skip(int n)
|
||||||
|
{
|
||||||
|
return IntIteratorUtils.skip(this, n);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static IntIterator fromRoaringBitmapIntIterator(org.roaringbitmap.IntIterator iterator)
|
||||||
|
{
|
||||||
|
return new RoaringBitmapDelegatingIntIterator(iterator);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class RoaringBitmapDelegatingIntIterator extends AbstractIntIterator
|
||||||
|
{
|
||||||
|
private final org.roaringbitmap.IntIterator delegate;
|
||||||
|
|
||||||
|
private RoaringBitmapDelegatingIntIterator(org.roaringbitmap.IntIterator delegate)
|
||||||
|
{
|
||||||
|
this.delegate = delegate;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasNext()
|
||||||
|
{
|
||||||
|
return delegate.hasNext();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int nextInt()
|
||||||
|
{
|
||||||
|
return delegate.next();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int skip(int n)
|
||||||
|
{
|
||||||
|
return IntIteratorUtils.skip(this, n);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private IntIteratorUtils() {}
|
||||||
|
}
|
|
@ -22,9 +22,10 @@ package io.druid.segment;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.collect.Iterators;
|
import com.google.common.collect.Iterators;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
|
||||||
|
|
||||||
public class NullDimensionSelector implements DimensionSelector
|
public class NullDimensionSelector implements DimensionSelector
|
||||||
{
|
{
|
||||||
|
@ -41,8 +42,8 @@ public class NullDimensionSelector implements DimensionSelector
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator() {
|
public IntIterator iterator() {
|
||||||
return Iterators.singletonIterator(0);
|
return IntIterators.singleton(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -24,13 +24,10 @@ import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Predicates;
|
import com.google.common.base.Predicates;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterators;
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||||
import com.metamx.common.IAE;
|
|
||||||
import com.metamx.common.UOE;
|
|
||||||
import com.metamx.common.guava.CloseQuietly;
|
import com.metamx.common.guava.CloseQuietly;
|
||||||
import com.metamx.common.guava.Sequence;
|
import com.metamx.common.guava.Sequence;
|
||||||
import com.metamx.common.guava.Sequences;
|
import com.metamx.common.guava.Sequences;
|
||||||
|
@ -59,6 +56,7 @@ import io.druid.segment.data.Offset;
|
||||||
import io.druid.segment.filter.AndFilter;
|
import io.druid.segment.filter.AndFilter;
|
||||||
import io.druid.segment.filter.BooleanValueMatcher;
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
import io.druid.segment.filter.Filters;
|
import io.druid.segment.filter.Filters;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.roaringbitmap.IntIterator;
|
import org.roaringbitmap.IntIterator;
|
||||||
|
@ -66,7 +64,6 @@ import org.roaringbitmap.IntIterator;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -535,9 +532,9 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public it.unimi.dsi.fastutil.ints.IntIterator iterator()
|
||||||
{
|
{
|
||||||
return Iterators.singletonIterator(column.getSingleValueRow(cursorOffset.getOffset()));
|
return IntIterators.singleton(column.getSingleValueRow(cursorOffset.getOffset()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -23,9 +23,10 @@ import com.google.common.collect.Iterators;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
public class SingleScanTimeDimSelector implements DimensionSelector
|
public class SingleScanTimeDimSelector implements DimensionSelector
|
||||||
|
@ -108,9 +109,9 @@ public class SingleScanTimeDimSelector implements DimensionSelector
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return Iterators.singletonIterator(dimensionValueIndex);
|
return IntIterators.singleton(dimensionValueIndex);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -37,11 +37,14 @@ import io.druid.segment.data.IndexedIterable;
|
||||||
import io.druid.segment.filter.BooleanValueMatcher;
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntArrayList;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntList;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntLists;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -352,14 +355,13 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
||||||
}
|
}
|
||||||
|
|
||||||
int nullId = getEncodedValue(null, false);
|
int nullId = getEncodedValue(null, false);
|
||||||
List<Integer> valsTmp = null;
|
IntList valsTmp = null;
|
||||||
if ((indices == null || indices.length == 0) && nullId > -1) {
|
if ((indices == null || indices.length == 0) && nullId > -1) {
|
||||||
if (nullId < maxId) {
|
if (nullId < maxId) {
|
||||||
valsTmp = new ArrayList<>(1);
|
valsTmp = IntLists.singleton(nullId);
|
||||||
valsTmp.add(nullId);
|
|
||||||
}
|
}
|
||||||
} else if (indices != null && indices.length > 0) {
|
} else if (indices != null && indices.length > 0) {
|
||||||
valsTmp = new ArrayList<>(indices.length);
|
valsTmp = new IntArrayList(indices.length);
|
||||||
for (int i = 0; i < indices.length; i++) {
|
for (int i = 0; i < indices.length; i++) {
|
||||||
int id = indices[i];
|
int id = indices[i];
|
||||||
if (id < maxId) {
|
if (id < maxId) {
|
||||||
|
@ -368,7 +370,7 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final List<Integer> vals = valsTmp == null ? Collections.EMPTY_LIST : valsTmp;
|
final IntList vals = valsTmp == null ? IntLists.EMPTY_LIST : valsTmp;
|
||||||
return new IndexedInts()
|
return new IndexedInts()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -384,7 +386,7 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return vals.iterator();
|
return vals.iterator();
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,9 +19,6 @@
|
||||||
|
|
||||||
package io.druid.segment;
|
package io.druid.segment;
|
||||||
|
|
||||||
import com.google.common.collect.Iterables;
|
|
||||||
import com.google.common.collect.Lists;
|
|
||||||
import com.google.common.collect.Ordering;
|
|
||||||
import com.google.common.io.ByteSink;
|
import com.google.common.io.ByteSink;
|
||||||
import com.google.common.io.ByteStreams;
|
import com.google.common.io.ByteStreams;
|
||||||
import com.google.common.io.Closer;
|
import com.google.common.io.Closer;
|
||||||
|
@ -29,14 +26,12 @@ import com.google.common.io.Files;
|
||||||
import com.google.common.io.OutputSupplier;
|
import com.google.common.io.OutputSupplier;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.metamx.collections.bitmap.BitmapFactory;
|
import com.metamx.collections.bitmap.BitmapFactory;
|
||||||
import com.metamx.collections.bitmap.MutableBitmap;
|
|
||||||
import com.metamx.collections.spatial.ImmutableRTree;
|
import com.metamx.collections.spatial.ImmutableRTree;
|
||||||
import com.metamx.collections.spatial.RTree;
|
import com.metamx.collections.spatial.RTree;
|
||||||
import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy;
|
import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy;
|
||||||
import com.metamx.common.ByteBufferUtils;
|
import com.metamx.common.ByteBufferUtils;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import io.druid.collections.CombiningIterable;
|
|
||||||
import io.druid.common.guava.FileOutputSupplier;
|
import io.druid.common.guava.FileOutputSupplier;
|
||||||
import io.druid.common.utils.SerializerUtils;
|
import io.druid.common.utils.SerializerUtils;
|
||||||
import io.druid.segment.column.ColumnCapabilities;
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
|
@ -100,12 +95,13 @@ public class StringDimensionMergerLegacy extends StringDimensionMergerV9 impleme
|
||||||
{
|
{
|
||||||
final SerializerUtils serializerUtils = new SerializerUtils();
|
final SerializerUtils serializerUtils = new SerializerUtils();
|
||||||
long dimStartTime = System.currentTimeMillis();
|
long dimStartTime = System.currentTimeMillis();
|
||||||
|
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
|
||||||
|
|
||||||
String bmpFilename = String.format("%s.inverted", dimensionName);
|
String bmpFilename = String.format("%s.inverted", dimensionName);
|
||||||
bitmapWriter = new GenericIndexedWriter<>(
|
bitmapWriter = new GenericIndexedWriter<>(
|
||||||
ioPeon,
|
ioPeon,
|
||||||
bmpFilename,
|
bmpFilename,
|
||||||
indexSpec.getBitmapSerdeFactory().getObjectStrategy()
|
bitmapSerdeFactory.getObjectStrategy()
|
||||||
);
|
);
|
||||||
bitmapWriter.open();
|
bitmapWriter.open();
|
||||||
|
|
||||||
|
@ -125,21 +121,20 @@ public class StringDimensionMergerLegacy extends StringDimensionMergerV9 impleme
|
||||||
Indexed<String> dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY);
|
Indexed<String> dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY);
|
||||||
log.info("Starting dimension[%s] with cardinality[%,d]", dimensionName, dimVals.size());
|
log.info("Starting dimension[%s] with cardinality[%,d]", dimensionName, dimVals.size());
|
||||||
|
|
||||||
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
|
|
||||||
final BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory();
|
final BitmapFactory bmpFactory = bitmapSerdeFactory.getBitmapFactory();
|
||||||
|
|
||||||
RTree tree = null;
|
RTree tree = null;
|
||||||
spatialWriter = null;
|
spatialWriter = null;
|
||||||
boolean hasSpatial = capabilities.hasSpatialIndexes();
|
boolean hasSpatial = capabilities.hasSpatialIndexes();
|
||||||
spatialIoPeon = new TmpFileIOPeon();
|
spatialIoPeon = new TmpFileIOPeon();
|
||||||
if (hasSpatial) {
|
if (hasSpatial) {
|
||||||
BitmapFactory bmpFactory = bitmapSerdeFactory.getBitmapFactory();
|
|
||||||
String spatialFilename = String.format("%s.spatial", dimensionName);
|
String spatialFilename = String.format("%s.spatial", dimensionName);
|
||||||
spatialWriter = new ByteBufferWriter<ImmutableRTree>(
|
spatialWriter = new ByteBufferWriter<>(
|
||||||
spatialIoPeon, spatialFilename, new IndexedRTree.ImmutableRTreeObjectStrategy(bmpFactory)
|
spatialIoPeon, spatialFilename, new IndexedRTree.ImmutableRTreeObjectStrategy(bmpFactory)
|
||||||
);
|
);
|
||||||
spatialWriter.open();
|
spatialWriter.open();
|
||||||
tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory);
|
tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bmpFactory), bmpFactory);
|
||||||
}
|
}
|
||||||
|
|
||||||
IndexSeeker[] dictIdSeeker = toIndexSeekers(adapters, dimConversions, dimensionName);
|
IndexSeeker[] dictIdSeeker = toIndexSeekers(adapters, dimConversions, dimensionName);
|
||||||
|
@ -147,46 +142,19 @@ public class StringDimensionMergerLegacy extends StringDimensionMergerV9 impleme
|
||||||
//Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result.
|
//Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result.
|
||||||
for (int dictId = 0; dictId < dimVals.size(); dictId++) {
|
for (int dictId = 0; dictId < dimVals.size(); dictId++) {
|
||||||
progress.progress();
|
progress.progress();
|
||||||
List<Iterable<Integer>> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size());
|
mergeBitmaps(
|
||||||
for (int j = 0; j < adapters.size(); ++j) {
|
segmentRowNumConversions,
|
||||||
int seekedDictId = dictIdSeeker[j].seek(dictId);
|
dimVals,
|
||||||
if (seekedDictId != IndexSeeker.NOT_EXIST) {
|
bmpFactory,
|
||||||
convertedInverteds.add(
|
tree,
|
||||||
new ConvertingIndexedInts(
|
hasSpatial,
|
||||||
adapters.get(j).getBitmapIndex(dimensionName, seekedDictId), segmentRowNumConversions.get(j)
|
dictIdSeeker,
|
||||||
)
|
dictId,
|
||||||
);
|
adapters,
|
||||||
}
|
dimensionName,
|
||||||
}
|
nullRowsBitmap,
|
||||||
|
bitmapWriter
|
||||||
MutableBitmap bitset = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap();
|
|
||||||
for (Integer row : CombiningIterable.createSplatted(
|
|
||||||
convertedInverteds,
|
|
||||||
Ordering.<Integer>natural().nullsFirst()
|
|
||||||
)) {
|
|
||||||
if (row != IndexMerger.INVALID_ROW) {
|
|
||||||
bitset.add(row);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if ((dictId == 0) && (Iterables.getFirst(dimVals, "") == null)) {
|
|
||||||
bitset.or(nullRowsBitmap);
|
|
||||||
}
|
|
||||||
|
|
||||||
bitmapWriter.write(
|
|
||||||
bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(bitset)
|
|
||||||
);
|
);
|
||||||
|
|
||||||
if (hasSpatial) {
|
|
||||||
String dimVal = dimVals.get(dictId);
|
|
||||||
if (dimVal != null) {
|
|
||||||
List<String> stringCoords = Lists.newArrayList(SPLITTER.split(dimVal));
|
|
||||||
float[] coords = new float[stringCoords.size()];
|
|
||||||
for (int j = 0; j < coords.length; j++) {
|
|
||||||
coords[j] = Float.valueOf(stringCoords.get(j));
|
|
||||||
}
|
|
||||||
tree.insert(coords, bitset);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
log.info("Completed dimension[%s] in %,d millis.", dimensionName, System.currentTimeMillis() - dimStartTime);
|
log.info("Completed dimension[%s] in %,d millis.", dimensionName, System.currentTimeMillis() - dimStartTime);
|
||||||
|
|
|
@ -19,12 +19,9 @@
|
||||||
|
|
||||||
package io.druid.segment;
|
package io.druid.segment;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
|
||||||
import com.google.common.base.Splitter;
|
import com.google.common.base.Splitter;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Iterators;
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Ordering;
|
|
||||||
import com.google.common.io.ByteStreams;
|
import com.google.common.io.ByteStreams;
|
||||||
import com.google.common.io.Closer;
|
import com.google.common.io.Closer;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
|
@ -37,7 +34,6 @@ import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy;
|
||||||
import com.metamx.common.ByteBufferUtils;
|
import com.metamx.common.ByteBufferUtils;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import io.druid.collections.CombiningIterable;
|
|
||||||
import io.druid.segment.column.ColumnCapabilities;
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.column.ColumnDescriptor;
|
import io.druid.segment.column.ColumnDescriptor;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
|
@ -57,8 +53,10 @@ import io.druid.segment.data.ListIndexed;
|
||||||
import io.druid.segment.data.VSizeIndexedIntsWriter;
|
import io.druid.segment.data.VSizeIndexedIntsWriter;
|
||||||
import io.druid.segment.data.VSizeIndexedWriter;
|
import io.druid.segment.data.VSizeIndexedWriter;
|
||||||
import io.druid.segment.serde.DictionaryEncodedColumnPartSerde;
|
import io.druid.segment.serde.DictionaryEncodedColumnPartSerde;
|
||||||
|
import it.unimi.dsi.fastutil.ints.AbstractIntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterable;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileOutputStream;
|
import java.io.FileOutputStream;
|
||||||
|
@ -67,7 +65,6 @@ import java.nio.IntBuffer;
|
||||||
import java.nio.MappedByteBuffer;
|
import java.nio.MappedByteBuffer;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||||
|
@ -231,7 +228,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||||
@Override
|
@Override
|
||||||
public int[] convertSegmentRowValuesToMergedRowValues(int[] segmentRow, int segmentIndexNumber)
|
public int[] convertSegmentRowValuesToMergedRowValues(int[] segmentRow, int segmentIndexNumber)
|
||||||
{
|
{
|
||||||
int[] dimVals = (int[]) segmentRow;
|
int[] dimVals = segmentRow;
|
||||||
// For strings, convert missing values to null/empty if conversion flag is set
|
// For strings, convert missing values to null/empty if conversion flag is set
|
||||||
// But if bitmap/dictionary is not used, always convert missing to 0
|
// But if bitmap/dictionary is not used, always convert missing to 0
|
||||||
if (dimVals == null) {
|
if (dimVals == null) {
|
||||||
|
@ -255,7 +252,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||||
@Override
|
@Override
|
||||||
public void processMergedRow(int[] rowValues) throws IOException
|
public void processMergedRow(int[] rowValues) throws IOException
|
||||||
{
|
{
|
||||||
int[] vals = (int[]) rowValues;
|
int[] vals = rowValues;
|
||||||
if (vals == null || vals.length == 0) {
|
if (vals == null || vals.length == 0) {
|
||||||
nullRowsBitmap.add(rowCount);
|
nullRowsBitmap.add(rowCount);
|
||||||
} else if (hasNull && vals.length == 1 && (vals[0]) == 0) {
|
} else if (hasNull && vals.length == 1 && (vals[0]) == 0) {
|
||||||
|
@ -283,18 +280,21 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||||
bitmapWriter = new GenericIndexedWriter<>(
|
bitmapWriter = new GenericIndexedWriter<>(
|
||||||
ioPeon,
|
ioPeon,
|
||||||
bmpFilename,
|
bmpFilename,
|
||||||
indexSpec.getBitmapSerdeFactory().getObjectStrategy()
|
bitmapSerdeFactory.getObjectStrategy()
|
||||||
);
|
);
|
||||||
bitmapWriter.open();
|
bitmapWriter.open();
|
||||||
|
|
||||||
// write dim values to one single file because we need to read it
|
// write dim values to one single file because we need to read it
|
||||||
File dimValueFile = IndexIO.makeDimFile(outDir, dimensionName);
|
File dimValueFile = IndexIO.makeDimFile(outDir, dimensionName);
|
||||||
try(FileOutputStream fos = new FileOutputStream(dimValueFile)) {
|
try (FileOutputStream fos = new FileOutputStream(dimValueFile)) {
|
||||||
ByteStreams.copy(dictionaryWriter.combineStreams(), fos);
|
ByteStreams.copy(dictionaryWriter.combineStreams(), fos);
|
||||||
}
|
}
|
||||||
|
|
||||||
final MappedByteBuffer dimValsMapped = Files.map(dimValueFile);
|
final MappedByteBuffer dimValsMapped = Files.map(dimValueFile);
|
||||||
try (Closeable dimValsMappedUnmapper = new Closeable()
|
try (
|
||||||
|
Closeable toCloseEncodedValueWriter = encodedValueWriter;
|
||||||
|
Closeable toCloseBitmapWriter = bitmapWriter;
|
||||||
|
Closeable dimValsMappedUnmapper = new Closeable()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public void close()
|
public void close()
|
||||||
|
@ -308,11 +308,10 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||||
RTree tree = null;
|
RTree tree = null;
|
||||||
boolean hasSpatial = capabilities.hasSpatialIndexes();
|
boolean hasSpatial = capabilities.hasSpatialIndexes();
|
||||||
if (hasSpatial) {
|
if (hasSpatial) {
|
||||||
BitmapFactory bitmapFactory = indexSpec.getBitmapSerdeFactory().getBitmapFactory();
|
|
||||||
spatialWriter = new ByteBufferWriter<>(
|
spatialWriter = new ByteBufferWriter<>(
|
||||||
ioPeon,
|
ioPeon,
|
||||||
String.format("%s.spatial", dimensionName),
|
String.format("%s.spatial", dimensionName),
|
||||||
new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapFactory)
|
new IndexedRTree.ImmutableRTreeObjectStrategy(bmpFactory)
|
||||||
);
|
);
|
||||||
spatialWriter.open();
|
spatialWriter.open();
|
||||||
tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bmpFactory), bmpFactory);
|
tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bmpFactory), bmpFactory);
|
||||||
|
@ -323,45 +322,19 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||||
//Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result.
|
//Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result.
|
||||||
for (int dictId = 0; dictId < dimVals.size(); dictId++) {
|
for (int dictId = 0; dictId < dimVals.size(); dictId++) {
|
||||||
progress.progress();
|
progress.progress();
|
||||||
List<Iterable<Integer>> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size());
|
mergeBitmaps(
|
||||||
for (int j = 0; j < adapters.size(); ++j) {
|
segmentRowNumConversions,
|
||||||
int seekedDictId = dictIdSeeker[j].seek(dictId);
|
dimVals,
|
||||||
if (seekedDictId != IndexSeeker.NOT_EXIST) {
|
bmpFactory,
|
||||||
convertedInverteds.add(
|
tree,
|
||||||
new ConvertingIndexedInts(
|
hasSpatial,
|
||||||
adapters.get(j).getBitmapIndex(dimensionName, seekedDictId), segmentRowNumConversions.get(j)
|
dictIdSeeker,
|
||||||
)
|
dictId,
|
||||||
);
|
adapters,
|
||||||
}
|
dimensionName,
|
||||||
}
|
nullRowsBitmap,
|
||||||
|
bitmapWriter
|
||||||
MutableBitmap bitset = bmpFactory.makeEmptyMutableBitmap();
|
);
|
||||||
for (Integer row : CombiningIterable.createSplatted(
|
|
||||||
convertedInverteds,
|
|
||||||
Ordering.<Integer>natural().nullsFirst()
|
|
||||||
)) {
|
|
||||||
if (row != IndexMerger.INVALID_ROW) {
|
|
||||||
bitset.add(row);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
ImmutableBitmap bitmapToWrite = bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(bitset);
|
|
||||||
if ((dictId == 0) && (Iterables.getFirst(dimVals, "") == null)) {
|
|
||||||
bitmapToWrite = bmpFactory.makeImmutableBitmap(nullRowsBitmap).union(bitmapToWrite);
|
|
||||||
}
|
|
||||||
bitmapWriter.write(bitmapToWrite);
|
|
||||||
|
|
||||||
if (hasSpatial) {
|
|
||||||
String dimVal = dimVals.get(dictId);
|
|
||||||
if (dimVal != null) {
|
|
||||||
List<String> stringCoords = Lists.newArrayList(SPLITTER.split(dimVal));
|
|
||||||
float[] coords = new float[stringCoords.size()];
|
|
||||||
for (int j = 0; j < coords.length; j++) {
|
|
||||||
coords[j] = Float.valueOf(stringCoords.get(j));
|
|
||||||
}
|
|
||||||
tree.insert(coords, bitset);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (hasSpatial) {
|
if (hasSpatial) {
|
||||||
|
@ -377,10 +350,70 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||||
System.currentTimeMillis() - dimStartTime
|
System.currentTimeMillis() - dimStartTime
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
bitmapWriter.close();
|
static void mergeBitmaps(
|
||||||
encodedValueWriter.close();
|
List<IntBuffer> segmentRowNumConversions,
|
||||||
|
Indexed<String> dimVals,
|
||||||
|
BitmapFactory bmpFactory,
|
||||||
|
RTree tree,
|
||||||
|
boolean hasSpatial,
|
||||||
|
IndexSeeker[] dictIdSeeker,
|
||||||
|
int dictId,
|
||||||
|
List<IndexableAdapter> adapters,
|
||||||
|
String dimensionName,
|
||||||
|
MutableBitmap nullRowsBitmap,
|
||||||
|
GenericIndexedWriter<ImmutableBitmap> bitmapWriter
|
||||||
|
) throws IOException
|
||||||
|
{
|
||||||
|
List<ConvertingIndexedInts> 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)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
MutableBitmap mergedIndexes = bmpFactory.makeEmptyMutableBitmap();
|
||||||
|
List<IntIterator> convertedInvertedIndexesIterators = new ArrayList<>(convertedInvertedIndexesToMerge.size());
|
||||||
|
for (ConvertingIndexedInts convertedInvertedIndexes : convertedInvertedIndexesToMerge) {
|
||||||
|
convertedInvertedIndexesIterators.add(convertedInvertedIndexes.iterator());
|
||||||
|
}
|
||||||
|
|
||||||
|
// Merge ascending index iterators into a single one, remove duplicates, and add to the mergedIndexes bitmap.
|
||||||
|
// Merge is needed, because some compacting MutableBitmap implementations are very inefficient when bits are
|
||||||
|
// added not in the ascending order.
|
||||||
|
int prevRow = IndexMerger.INVALID_ROW;
|
||||||
|
for (IntIterator mergeIt = IntIteratorUtils.mergeAscending(convertedInvertedIndexesIterators);
|
||||||
|
mergeIt.hasNext(); ) {
|
||||||
|
int row = mergeIt.nextInt();
|
||||||
|
if (row != prevRow && row != IndexMerger.INVALID_ROW) {
|
||||||
|
mergedIndexes.add(row);
|
||||||
|
}
|
||||||
|
prevRow = row;
|
||||||
|
}
|
||||||
|
|
||||||
|
if ((dictId == 0) && (Iterables.getFirst(dimVals, "") == null)) {
|
||||||
|
mergedIndexes.or(nullRowsBitmap);
|
||||||
|
}
|
||||||
|
|
||||||
|
bitmapWriter.write(bmpFactory.makeImmutableBitmap(mergedIndexes));
|
||||||
|
|
||||||
|
if (hasSpatial) {
|
||||||
|
String dimVal = dimVals.get(dictId);
|
||||||
|
if (dimVal != null) {
|
||||||
|
List<String> stringCoords = Lists.newArrayList(SPLITTER.split(dimVal));
|
||||||
|
float[] coords = new float[stringCoords.size()];
|
||||||
|
for (int j = 0; j < coords.length; j++) {
|
||||||
|
coords[j] = Float.valueOf(stringCoords.get(j));
|
||||||
|
}
|
||||||
|
tree.insert(coords, mergedIndexes);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -500,7 +533,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class ConvertingIndexedInts implements Iterable<Integer>
|
public static class ConvertingIndexedInts implements IntIterable
|
||||||
{
|
{
|
||||||
private final IndexedInts baseIndex;
|
private final IndexedInts baseIndex;
|
||||||
private final IntBuffer conversionBuffer;
|
private final IntBuffer conversionBuffer;
|
||||||
|
@ -525,19 +558,29 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return Iterators.transform(
|
final IntIterator baseIterator = baseIndex.iterator();
|
||||||
baseIndex.iterator(),
|
return new AbstractIntIterator()
|
||||||
new Function<Integer, Integer>()
|
{
|
||||||
{
|
@Override
|
||||||
@Override
|
public boolean hasNext()
|
||||||
public Integer apply(@Nullable Integer input)
|
{
|
||||||
{
|
return baseIterator.hasNext();
|
||||||
return conversionBuffer.get(input);
|
}
|
||||||
}
|
|
||||||
}
|
@Override
|
||||||
);
|
public int nextInt()
|
||||||
|
{
|
||||||
|
return conversionBuffer.get(baseIterator.nextInt());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int skip(int n)
|
||||||
|
{
|
||||||
|
return IntIteratorUtils.skip(this, n);
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,11 +19,12 @@
|
||||||
|
|
||||||
package io.druid.segment.data;
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class ArrayBasedIndexedInts implements IndexedInts
|
public class ArrayBasedIndexedInts implements IndexedInts
|
||||||
{
|
{
|
||||||
private final int[] expansion;
|
private final int[] expansion;
|
||||||
|
@ -43,7 +44,7 @@ public class ArrayBasedIndexedInts implements IndexedInts
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return new IndexedIntsIterator(this);
|
return new IndexedIntsIterator(this);
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,11 +21,11 @@ package io.druid.segment.data;
|
||||||
|
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||||
import org.roaringbitmap.IntIterator;
|
import io.druid.segment.IntIteratorUtils;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -82,30 +82,9 @@ public class BitmapCompressedIndexedInts implements IndexedInts, Comparable<Immu
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return new Iterator<Integer>()
|
return IntIteratorUtils.fromRoaringBitmapIntIterator(immutableBitmap.iterator());
|
||||||
{
|
|
||||||
IntIterator baseIterator = immutableBitmap.iterator();
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean hasNext()
|
|
||||||
{
|
|
||||||
return baseIterator.hasNext();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Integer next()
|
|
||||||
{
|
|
||||||
return baseIterator.next();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void remove()
|
|
||||||
{
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -27,6 +27,7 @@ import com.metamx.common.guava.CloseQuietly;
|
||||||
import io.druid.collections.ResourceHolder;
|
import io.druid.collections.ResourceHolder;
|
||||||
import io.druid.collections.StupidResourceHolder;
|
import io.druid.collections.StupidResourceHolder;
|
||||||
import io.druid.segment.CompressedPools;
|
import io.druid.segment.CompressedPools;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
@ -293,7 +294,7 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier<IndexedIn
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return new IndexedIntsIterator(this);
|
return new IndexedIntsIterator(this);
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,6 +28,7 @@ import com.metamx.common.guava.CloseQuietly;
|
||||||
import io.druid.collections.ResourceHolder;
|
import io.druid.collections.ResourceHolder;
|
||||||
import io.druid.collections.StupidResourceHolder;
|
import io.druid.collections.StupidResourceHolder;
|
||||||
import io.druid.segment.CompressedPools;
|
import io.druid.segment.CompressedPools;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
@ -366,7 +367,7 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier<Inde
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return new IndexedIntsIterator(this);
|
return new IndexedIntsIterator(this);
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,10 +19,10 @@
|
||||||
|
|
||||||
package io.druid.segment.data;
|
package io.druid.segment.data;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -47,9 +47,9 @@ public class EmptyIndexedInts implements IndexedInts
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return ImmutableList.<Integer>of().iterator();
|
return IntIterators.EMPTY_ITERATOR;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,12 +19,14 @@
|
||||||
|
|
||||||
package io.druid.segment.data;
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterable;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get a int an index (array or list lookup abstraction without boxing).
|
* Get a int an index (array or list lookup abstraction without boxing).
|
||||||
*/
|
*/
|
||||||
public interface IndexedInts extends Iterable<Integer>, Closeable
|
public interface IndexedInts extends IntIterable, Closeable
|
||||||
{
|
{
|
||||||
int size();
|
int size();
|
||||||
int get(int index);
|
int get(int index);
|
||||||
|
|
|
@ -20,11 +20,12 @@
|
||||||
package io.druid.segment.data;
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
|
||||||
import java.util.Iterator;
|
import io.druid.segment.IntIteratorUtils;
|
||||||
|
import it.unimi.dsi.fastutil.ints.AbstractIntIterator;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class IndexedIntsIterator implements Iterator<Integer>
|
public class IndexedIntsIterator extends AbstractIntIterator
|
||||||
{
|
{
|
||||||
private final IndexedInts baseInts;
|
private final IndexedInts baseInts;
|
||||||
private final int size;
|
private final int size;
|
||||||
|
@ -47,14 +48,13 @@ public class IndexedIntsIterator implements Iterator<Integer>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Integer next()
|
public int nextInt() {
|
||||||
{
|
|
||||||
return baseInts.get(currIndex++);
|
return baseInts.get(currIndex++);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void remove()
|
public int skip(int n)
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
return IntIteratorUtils.skip(this, n);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,11 +22,11 @@ package io.druid.segment.data;
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import io.druid.collections.IntList;
|
import io.druid.collections.IntList;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.nio.IntBuffer;
|
import java.nio.IntBuffer;
|
||||||
import java.util.Iterator;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -86,7 +86,7 @@ public class IntBufferIndexedInts implements IndexedInts, Comparable<IntBufferIn
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return new IndexedIntsIterator(this);
|
return new IndexedIntsIterator(this);
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,8 +19,9 @@
|
||||||
|
|
||||||
package io.druid.segment.data;
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -44,7 +45,7 @@ public class ListBasedIndexedInts implements IndexedInts
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return new IndexedIntsIterator(this);
|
return new IndexedIntsIterator(this);
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,11 +22,11 @@ package io.druid.segment.data;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.nio.channels.WritableByteChannel;
|
import java.nio.channels.WritableByteChannel;
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -178,7 +178,7 @@ public class VSizeIndexedInts implements IndexedInts, Comparable<VSizeIndexedInt
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return new IndexedIntsIterator(this);
|
return new IndexedIntsIterator(this);
|
||||||
}
|
}
|
||||||
|
|
|
@ -60,6 +60,8 @@ import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.serde.ComplexMetricExtractor;
|
import io.druid.segment.serde.ComplexMetricExtractor;
|
||||||
import io.druid.segment.serde.ComplexMetricSerde;
|
import io.druid.segment.serde.ComplexMetricSerde;
|
||||||
import io.druid.segment.serde.ComplexMetrics;
|
import io.druid.segment.serde.ComplexMetrics;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
@ -227,31 +229,29 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
public IndexedInts getRow()
|
public IndexedInts getRow()
|
||||||
{
|
{
|
||||||
final List<String> dimensionValues = in.get().getDimension(dimension);
|
final List<String> dimensionValues = in.get().getDimension(dimension);
|
||||||
final ArrayList<Integer> vals = Lists.newArrayList();
|
final int dimensionValuesSize = dimensionValues != null ? dimensionValues.size() : 0;
|
||||||
if (dimensionValues != null) {
|
|
||||||
for (int i = 0; i < dimensionValues.size(); ++i) {
|
|
||||||
vals.add(i);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return new IndexedInts()
|
return new IndexedInts()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public int size()
|
public int size()
|
||||||
{
|
{
|
||||||
return vals.size();
|
return dimensionValuesSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int get(int index)
|
public int get(int index)
|
||||||
{
|
{
|
||||||
return vals.get(index);
|
if (index < 0 || index >= dimensionValuesSize) {
|
||||||
|
throw new IndexOutOfBoundsException("index: " + index);
|
||||||
|
}
|
||||||
|
return index;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return vals.iterator();
|
return IntIterators.fromTo(0, dimensionValuesSize);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -28,6 +28,7 @@ import com.metamx.common.logger.Logger;
|
||||||
import io.druid.segment.DimensionHandler;
|
import io.druid.segment.DimensionHandler;
|
||||||
import io.druid.segment.DimensionIndexer;
|
import io.druid.segment.DimensionIndexer;
|
||||||
import io.druid.segment.IndexableAdapter;
|
import io.druid.segment.IndexableAdapter;
|
||||||
|
import io.druid.segment.IntIteratorUtils;
|
||||||
import io.druid.segment.Metadata;
|
import io.druid.segment.Metadata;
|
||||||
import io.druid.segment.Rowboat;
|
import io.druid.segment.Rowboat;
|
||||||
import io.druid.segment.column.ColumnCapabilities;
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
|
@ -35,8 +36,8 @@ import io.druid.segment.data.EmptyIndexedInts;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.data.ListIndexed;
|
import io.druid.segment.data.ListIndexed;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.roaringbitmap.IntIterator;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -285,30 +286,9 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return new Iterator<Integer>()
|
return IntIteratorUtils.fromRoaringBitmapIntIterator(bitmapIndex.iterator());
|
||||||
{
|
|
||||||
final IntIterator baseIter = bitmapIndex.iterator();
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean hasNext()
|
|
||||||
{
|
|
||||||
return baseIter.hasNext();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Integer next()
|
|
||||||
{
|
|
||||||
return baseIter.next();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void remove()
|
|
||||||
{
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -34,21 +34,20 @@ import io.druid.query.aggregation.BufferAggregator;
|
||||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.dimension.ExtractionDimensionSpec;
|
import io.druid.query.dimension.ExtractionDimensionSpec;
|
||||||
import io.druid.query.dimension.LookupDimensionSpec;
|
|
||||||
import io.druid.query.dimension.RegexFilteredDimensionSpec;
|
import io.druid.query.dimension.RegexFilteredDimensionSpec;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.extraction.JavaScriptExtractionFn;
|
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||||
import io.druid.query.extraction.RegexDimExtractionFn;
|
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||||
import io.druid.query.extraction.UpperExtractionFn;
|
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
import io.druid.segment.data.IndexedInts;
|
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.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -136,9 +135,9 @@ public class CardinalityAggregatorTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<Integer> iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return Iterators.forArray(column.get(p));
|
return IntIterators.asIntIterator(Iterators.forArray(column.get(p)));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -0,0 +1,53 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntListIterator;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import static io.druid.segment.IntIteratorUtils.skip;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
|
||||||
|
public class IntIteratorUtilsTest
|
||||||
|
{
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSkip()
|
||||||
|
{
|
||||||
|
assertEquals(0, skip(IntIterators.EMPTY_ITERATOR, 5));
|
||||||
|
assertEquals(0, skip(IntIterators.EMPTY_ITERATOR, 0));
|
||||||
|
|
||||||
|
IntListIterator it = IntIterators.fromTo(0, 10);
|
||||||
|
assertEquals(3, skip(it, 3));
|
||||||
|
assertEquals(3, it.nextInt());
|
||||||
|
assertEquals(6, skip(it, 100));
|
||||||
|
assertEquals(0, skip(it, 100));
|
||||||
|
assertFalse(it.hasNext());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = IllegalArgumentException.class)
|
||||||
|
public void testNegativeSkipArgument()
|
||||||
|
{
|
||||||
|
skip(IntIterators.fromTo(0, 10), -1);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,142 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntArrayList;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntList;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntLists;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.NoSuchElementException;
|
||||||
|
import java.util.concurrent.ThreadLocalRandom;
|
||||||
|
|
||||||
|
import static io.druid.segment.IntIteratorUtils.mergeAscending;
|
||||||
|
import static it.unimi.dsi.fastutil.ints.IntIterators.EMPTY_ITERATOR;
|
||||||
|
import static java.lang.Integer.MAX_VALUE;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
|
public class MergeIntIteratorTest
|
||||||
|
{
|
||||||
|
@Test(expected = NoSuchElementException.class)
|
||||||
|
public void testNoIterators()
|
||||||
|
{
|
||||||
|
IntIterator it = mergeAscending(Collections.<IntIterator>emptyList());
|
||||||
|
assertEmpty(it);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = NoSuchElementException.class)
|
||||||
|
public void testMergeEmptyIterators()
|
||||||
|
{
|
||||||
|
IntIterator it = mergeAscending(Arrays.<IntIterator>asList(EMPTY_ITERATOR, EMPTY_ITERATOR));
|
||||||
|
assertEmpty(it);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void assertEmpty(IntIterator it)
|
||||||
|
{
|
||||||
|
assertFalse(it.hasNext());
|
||||||
|
try {
|
||||||
|
it.next();
|
||||||
|
fail("expected NoSuchElementException on it.next() after it.hasNext() = false");
|
||||||
|
}
|
||||||
|
catch (NoSuchElementException ignore) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
// expected to fail with NoSuchElementException
|
||||||
|
it.nextInt();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check for some possible corner cases, because {@link io.druid.segment.IntIteratorUtils.MergeIntIterator} is
|
||||||
|
* implemented using packing ints within longs, that is prone to some overflow or sign bit extension bugs
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testOverflow()
|
||||||
|
{
|
||||||
|
List<IntList> lists = Lists.newArrayList(
|
||||||
|
IntLists.singleton(Integer.MIN_VALUE),
|
||||||
|
IntLists.singleton(Integer.MIN_VALUE),
|
||||||
|
IntLists.singleton(-1),
|
||||||
|
IntLists.singleton(0),
|
||||||
|
IntLists.singleton(MAX_VALUE)
|
||||||
|
);
|
||||||
|
for (int i = 0; i < lists.size() + 1; i++) {
|
||||||
|
assertAscending(mergeAscending(iteratorsFromLists(lists)));
|
||||||
|
Collections.rotate(lists, 1);
|
||||||
|
}
|
||||||
|
Collections.shuffle(lists);
|
||||||
|
assertAscending(mergeAscending(iteratorsFromLists(lists)));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static List<IntIterator> iteratorsFromLists(List<IntList> lists)
|
||||||
|
{
|
||||||
|
ArrayList<IntIterator> r = new ArrayList<>();
|
||||||
|
for (IntList list : lists) {
|
||||||
|
r.add(list.iterator());
|
||||||
|
}
|
||||||
|
return r;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void smokeTest()
|
||||||
|
{
|
||||||
|
ThreadLocalRandom r = ThreadLocalRandom.current();
|
||||||
|
for (int i = 0; i < 1000; i++) {
|
||||||
|
int numIterators = r.nextInt(1, 11);
|
||||||
|
List<IntList> lists = new ArrayList<>(numIterators);
|
||||||
|
for (int j = 0; j < numIterators; j++) {
|
||||||
|
lists.add(new IntArrayList());
|
||||||
|
}
|
||||||
|
for (int j = 0; j < 50; j++) {
|
||||||
|
lists.get(r.nextInt(numIterators)).add(j);
|
||||||
|
}
|
||||||
|
for (int j = 0; j < lists.size() + 1; j++) {
|
||||||
|
assertAscending(mergeAscending(iteratorsFromLists(lists)));
|
||||||
|
Collections.rotate(lists, 1);
|
||||||
|
}
|
||||||
|
for (int j = 0; j < 10; j++) {
|
||||||
|
Collections.shuffle(lists);
|
||||||
|
assertAscending(mergeAscending(iteratorsFromLists(lists)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void assertAscending(IntIterator it)
|
||||||
|
{
|
||||||
|
if (!it.hasNext()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
int prev = it.nextInt();
|
||||||
|
for (; it.hasNext(); ) {
|
||||||
|
int current = it.nextInt();
|
||||||
|
if (prev > current) {
|
||||||
|
Assert.fail("not ascending: " + prev + ", then " + current);
|
||||||
|
}
|
||||||
|
prev = current;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue