mirror of https://github.com/apache/druid.git
Remove boxing/unboxing in indexer (#4269)
* Remove boxing/unboxing in indexer * Fix rowIndex visibility * Cleanup
This commit is contained in:
parent
daa8ef8658
commit
d45dad2b44
|
@ -69,6 +69,9 @@ import io.druid.segment.incremental.IncrementalIndexAdapter;
|
|||
import io.druid.segment.serde.ComplexMetricColumnSerializer;
|
||||
import io.druid.segment.serde.ComplexMetricSerde;
|
||||
import io.druid.segment.serde.ComplexMetrics;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntSortedSet;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -90,7 +93,6 @@ import java.util.Map;
|
|||
import java.util.NoSuchElementException;
|
||||
import java.util.PriorityQueue;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -747,10 +749,14 @@ public class IndexMerger
|
|||
mergers.get(i).processMergedRow(dims[i]);
|
||||
}
|
||||
|
||||
for (Map.Entry<Integer, TreeSet<Integer>> comprisedRow : theRow.getComprisedRows().entrySet()) {
|
||||
final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getKey());
|
||||
Iterator<Int2ObjectMap.Entry<IntSortedSet>> rowsIterator = theRow.getComprisedRows().int2ObjectEntrySet().fastIterator();
|
||||
while (rowsIterator.hasNext()) {
|
||||
Int2ObjectMap.Entry<IntSortedSet> comprisedRow = rowsIterator.next();
|
||||
|
||||
for (Integer rowNum : comprisedRow.getValue()) {
|
||||
final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getIntKey());
|
||||
|
||||
for (IntIterator setIterator = comprisedRow.getValue().iterator(); setIterator.hasNext(); /* NOP */) {
|
||||
int rowNum = setIterator.nextInt();
|
||||
while (conversionBuffer.position() < rowNum) {
|
||||
conversionBuffer.put(INVALID_ROW);
|
||||
}
|
||||
|
@ -1216,9 +1222,12 @@ public class IndexMerger
|
|||
);
|
||||
|
||||
for (Rowboat rowboat : Arrays.asList(lhs, rhs)) {
|
||||
for (Map.Entry<Integer, TreeSet<Integer>> entry : rowboat.getComprisedRows().entrySet()) {
|
||||
for (Integer rowNum : entry.getValue()) {
|
||||
retVal.addRow(entry.getKey(), rowNum);
|
||||
Iterator<Int2ObjectMap.Entry<IntSortedSet>> entryIterator = rowboat.getComprisedRows().int2ObjectEntrySet().fastIterator();
|
||||
while (entryIterator.hasNext()) {
|
||||
Int2ObjectMap.Entry<IntSortedSet> entry = entryIterator.next();
|
||||
|
||||
for (IntIterator setIterator = entry.getValue().iterator(); setIterator.hasNext(); /* NOP */) {
|
||||
retVal.addRow(entry.getIntKey(), setIterator.nextInt());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -53,6 +53,9 @@ import io.druid.segment.serde.ComplexMetricSerde;
|
|||
import io.druid.segment.serde.ComplexMetrics;
|
||||
import io.druid.segment.serde.FloatGenericColumnPartSerde;
|
||||
import io.druid.segment.serde.LongGenericColumnPartSerde;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntSortedSet;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -64,10 +67,10 @@ import java.nio.ByteBuffer;
|
|||
import java.nio.IntBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
public class IndexMergerV9 extends IndexMerger
|
||||
{
|
||||
|
@ -452,16 +455,21 @@ public class IndexMergerV9 extends IndexMerger
|
|||
merger.processMergedRow(dims[i]);
|
||||
}
|
||||
|
||||
for (Map.Entry<Integer, TreeSet<Integer>> comprisedRow : theRow.getComprisedRows().entrySet()) {
|
||||
final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getKey());
|
||||
Iterator<Int2ObjectMap.Entry<IntSortedSet>> rowsIterator = theRow.getComprisedRows().int2ObjectEntrySet().fastIterator();
|
||||
while (rowsIterator.hasNext()) {
|
||||
Int2ObjectMap.Entry<IntSortedSet> comprisedRow = rowsIterator.next();
|
||||
|
||||
for (Integer rowNum : comprisedRow.getValue()) {
|
||||
final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getIntKey());
|
||||
|
||||
for (IntIterator setIterator = comprisedRow.getValue().iterator(); setIterator.hasNext(); /* NOP */) {
|
||||
int rowNum = setIterator.nextInt();
|
||||
while (conversionBuffer.position() < rowNum) {
|
||||
conversionBuffer.put(INVALID_ROW);
|
||||
}
|
||||
conversionBuffer.put(rowCount);
|
||||
}
|
||||
}
|
||||
|
||||
if ((++rowCount % 500000) == 0) {
|
||||
log.info("walked 500,000/%d rows in %,d millis.", rowCount, System.currentTimeMillis() - time);
|
||||
time = System.currentTimeMillis();
|
||||
|
|
|
@ -19,15 +19,14 @@
|
|||
|
||||
package io.druid.segment;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.common.primitives.Longs;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
|
||||
import it.unimi.dsi.fastutil.ints.IntRBTreeSet;
|
||||
import it.unimi.dsi.fastutil.ints.IntSortedSet;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.TreeSet;
|
||||
|
||||
public class Rowboat implements Comparable<Rowboat>
|
||||
{
|
||||
|
@ -35,7 +34,7 @@ public class Rowboat implements Comparable<Rowboat>
|
|||
private final Object[] dims;
|
||||
private final Object[] metrics;
|
||||
private final int rowNum;
|
||||
private final Map<Integer, TreeSet<Integer>> comprisedRows;
|
||||
private final Int2ObjectOpenHashMap<IntSortedSet> comprisedRows;
|
||||
private final DimensionHandler[] handlers;
|
||||
|
||||
public Rowboat(
|
||||
|
@ -52,7 +51,7 @@ public class Rowboat implements Comparable<Rowboat>
|
|||
this.rowNum = rowNum;
|
||||
this.handlers = handlers;
|
||||
|
||||
this.comprisedRows = Maps.newHashMap();
|
||||
this.comprisedRows = new Int2ObjectOpenHashMap<>();
|
||||
}
|
||||
|
||||
public long getTimestamp()
|
||||
|
@ -72,15 +71,15 @@ public class Rowboat implements Comparable<Rowboat>
|
|||
|
||||
public void addRow(int indexNum, int rowNum)
|
||||
{
|
||||
TreeSet<Integer> rowNums = comprisedRows.get(indexNum);
|
||||
IntSortedSet rowNums = comprisedRows.get(indexNum);
|
||||
if (rowNums == null) {
|
||||
rowNums = Sets.newTreeSet();
|
||||
rowNums = new IntRBTreeSet();
|
||||
comprisedRows.put(indexNum, rowNums);
|
||||
}
|
||||
rowNums.add(rowNum);
|
||||
}
|
||||
|
||||
public Map<Integer, TreeSet<Integer>> getComprisedRows()
|
||||
public Int2ObjectOpenHashMap<IntSortedSet> getComprisedRows()
|
||||
{
|
||||
return comprisedRows;
|
||||
}
|
||||
|
|
|
@ -19,12 +19,12 @@
|
|||
|
||||
package io.druid.segment;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.primitives.Ints;
|
||||
import io.druid.collections.bitmap.BitmapFactory;
|
||||
import io.druid.collections.bitmap.MutableBitmap;
|
||||
|
@ -43,6 +43,9 @@ import io.druid.segment.filter.BooleanValueMatcher;
|
|||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||
import it.unimi.dsi.fastutil.ints.IntArrays;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.objects.Object2IntRBTreeMap;
|
||||
import it.unimi.dsi.fastutil.objects.Object2IntSortedMap;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
|
@ -52,38 +55,13 @@ import java.util.Comparator;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
public class StringDimensionIndexer implements DimensionIndexer<Integer, int[], String>
|
||||
{
|
||||
public static final Function<Object, String> STRING_TRANSFORMER = new Function<Object, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(final Object o)
|
||||
{
|
||||
if (o == null) {
|
||||
return null;
|
||||
}
|
||||
if (o instanceof String) {
|
||||
return (String) o;
|
||||
}
|
||||
return o.toString();
|
||||
}
|
||||
};
|
||||
private static final Function<Object, String> STRING_TRANSFORMER = o -> o != null ? o.toString() : null;
|
||||
|
||||
public static final Comparator<String> UNENCODED_COMPARATOR = new Comparator<String>()
|
||||
{
|
||||
@Override
|
||||
public int compare(String o1, String o2)
|
||||
{
|
||||
if (o1 == null) {
|
||||
return o2 == null ? 0 : -1;
|
||||
}
|
||||
if (o2 == null) {
|
||||
return 1;
|
||||
}
|
||||
return o1.compareTo(o2);
|
||||
}
|
||||
};
|
||||
private static final Comparator<String> UNENCODED_COMPARATOR = Ordering.natural().nullsFirst();
|
||||
|
||||
private static class DimensionDictionary
|
||||
{
|
||||
|
@ -176,7 +154,7 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
|||
|
||||
public SortedDimensionDictionary(List<String> idToValue, int length)
|
||||
{
|
||||
Map<String, Integer> sortedMap = Maps.newTreeMap();
|
||||
Object2IntSortedMap<String> sortedMap = new Object2IntRBTreeMap<>();
|
||||
for (int id = 0; id < length; id++) {
|
||||
sortedMap.put(idToValue.get(id), id);
|
||||
}
|
||||
|
@ -184,7 +162,8 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
|||
this.idToIndex = new int[length];
|
||||
this.indexToId = new int[length];
|
||||
int index = 0;
|
||||
for (Integer id : sortedMap.values()) {
|
||||
for (IntIterator iterator = sortedMap.values().iterator(); iterator.hasNext();) {
|
||||
int id = iterator.nextInt();
|
||||
idToIndex[id] = index;
|
||||
indexToId[index] = id;
|
||||
index++;
|
||||
|
|
|
@ -25,7 +25,6 @@ import com.google.common.base.Function;
|
|||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
|
@ -37,9 +36,9 @@ import io.druid.data.input.Row;
|
|||
import io.druid.data.input.impl.DimensionSchema;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.data.input.impl.SpatialDimensionSchema;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
|
@ -205,14 +204,7 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
|
||||
// This is modified on add() in a critical section.
|
||||
private final ThreadLocal<InputRow> in = new ThreadLocal<>();
|
||||
private final Supplier<InputRow> rowSupplier = new Supplier<InputRow>()
|
||||
{
|
||||
@Override
|
||||
public InputRow get()
|
||||
{
|
||||
return in.get();
|
||||
}
|
||||
};
|
||||
private final Supplier<InputRow> rowSupplier = in::get;
|
||||
|
||||
/**
|
||||
* Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that
|
||||
|
@ -276,7 +268,7 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
columnCapabilities.put(dimName, capabilities);
|
||||
}
|
||||
|
||||
//__time capabilites
|
||||
//__time capabilities
|
||||
ColumnCapabilitiesImpl timeCapabilities = new ColumnCapabilitiesImpl();
|
||||
timeCapabilities.setType(ValueType.LONG);
|
||||
columnCapabilities.put(Column.TIME_COLUMN_NAME, timeCapabilities);
|
||||
|
@ -688,47 +680,41 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
|
||||
return Iterators.transform(
|
||||
getFacts().iterator(descending),
|
||||
new Function<Map.Entry<TimeAndDims, Integer>, Row>()
|
||||
{
|
||||
@Override
|
||||
public Row apply(final Map.Entry<TimeAndDims, Integer> input)
|
||||
{
|
||||
final TimeAndDims timeAndDims = input.getKey();
|
||||
final int rowOffset = input.getValue();
|
||||
timeAndDims -> {
|
||||
final int rowOffset = timeAndDims.getRowIndex();
|
||||
|
||||
Object[] theDims = timeAndDims.getDims();
|
||||
Object[] theDims = timeAndDims.getDims();
|
||||
|
||||
Map<String, Object> theVals = Maps.newLinkedHashMap();
|
||||
for (int i = 0; i < theDims.length; ++i) {
|
||||
Object dim = theDims[i];
|
||||
DimensionDesc dimensionDesc = dimensions.get(i);
|
||||
if (dimensionDesc == null) {
|
||||
continue;
|
||||
}
|
||||
String dimensionName = dimensionDesc.getName();
|
||||
DimensionHandler handler = dimensionDesc.getHandler();
|
||||
if (dim == null || handler.getLengthOfEncodedKeyComponent(dim) == 0) {
|
||||
theVals.put(dimensionName, null);
|
||||
continue;
|
||||
}
|
||||
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||
Object rowVals = indexer.convertUnsortedEncodedKeyComponentToActualArrayOrList(dim, DimensionIndexer.LIST);
|
||||
theVals.put(dimensionName, rowVals);
|
||||
Map<String, Object> theVals = Maps.newLinkedHashMap();
|
||||
for (int i = 0; i < theDims.length; ++i) {
|
||||
Object dim = theDims[i];
|
||||
DimensionDesc dimensionDesc = dimensions.get(i);
|
||||
if (dimensionDesc == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
AggregatorType[] aggs = getAggsForRow(rowOffset);
|
||||
for (int i = 0; i < aggs.length; ++i) {
|
||||
theVals.put(metrics[i].getName(), getAggVal(aggs[i], rowOffset, i));
|
||||
String dimensionName = dimensionDesc.getName();
|
||||
DimensionHandler handler = dimensionDesc.getHandler();
|
||||
if (dim == null || handler.getLengthOfEncodedKeyComponent(dim) == 0) {
|
||||
theVals.put(dimensionName, null);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (postAggs != null) {
|
||||
for (PostAggregator postAgg : postAggs) {
|
||||
theVals.put(postAgg.getName(), postAgg.compute(theVals));
|
||||
}
|
||||
}
|
||||
|
||||
return new MapBasedRow(timeAndDims.getTimestamp(), theVals);
|
||||
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||
Object rowVals = indexer.convertUnsortedEncodedKeyComponentToActualArrayOrList(dim, DimensionIndexer.LIST);
|
||||
theVals.put(dimensionName, rowVals);
|
||||
}
|
||||
|
||||
AggregatorType[] aggs = getAggsForRow(rowOffset);
|
||||
for (int i = 0; i < aggs.length; ++i) {
|
||||
theVals.put(metrics[i].getName(), getAggVal(aggs[i], rowOffset, i));
|
||||
}
|
||||
|
||||
if (postAggs != null) {
|
||||
for (PostAggregator postAgg : postAggs) {
|
||||
theVals.put(postAgg.getName(), postAgg.compute(theVals));
|
||||
}
|
||||
}
|
||||
|
||||
return new MapBasedRow(timeAndDims.getTimestamp(), theVals);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
@ -832,19 +818,40 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
|
||||
public static final class TimeAndDims
|
||||
{
|
||||
public static final int EMPTY_ROW_INDEX = -1;
|
||||
|
||||
private final long timestamp;
|
||||
private final Object[] dims;
|
||||
private final List<DimensionDesc> dimensionDescsList;
|
||||
|
||||
/**
|
||||
* rowIndex is not checked in {@link #equals} and {@link #hashCode} on purpose. TimeAndDims acts as a Map key
|
||||
* and "entry" object (rowIndex is the "value") at the same time. This is done to reduce object indirection and
|
||||
* improve locality, and avoid boxing of rowIndex as Integer, when stored in JDK collection:
|
||||
* {@link RollupFactsHolder} needs concurrent collections, that are not present in fastutil.
|
||||
*/
|
||||
private int rowIndex;
|
||||
|
||||
TimeAndDims(
|
||||
long timestamp,
|
||||
Object[] dims,
|
||||
List<DimensionDesc> dimensionDescsList
|
||||
)
|
||||
{
|
||||
this(timestamp, dims, dimensionDescsList, EMPTY_ROW_INDEX);
|
||||
}
|
||||
|
||||
TimeAndDims(
|
||||
long timestamp,
|
||||
Object[] dims,
|
||||
List<DimensionDesc> dimensionDescsList,
|
||||
int rowIndex
|
||||
)
|
||||
{
|
||||
this.timestamp = timestamp;
|
||||
this.dims = dims;
|
||||
this.dimensionDescsList = dimensionDescsList;
|
||||
this.rowIndex = rowIndex;
|
||||
}
|
||||
|
||||
public long getTimestamp()
|
||||
|
@ -857,6 +864,16 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
return dims;
|
||||
}
|
||||
|
||||
public int getRowIndex()
|
||||
{
|
||||
return rowIndex;
|
||||
}
|
||||
|
||||
private void setRowIndex(int rowIndex)
|
||||
{
|
||||
this.rowIndex = rowIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
@ -991,87 +1008,29 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
return true;
|
||||
}
|
||||
|
||||
public static class FactsEntry implements Map.Entry<TimeAndDims, Integer>
|
||||
{
|
||||
TimeAndDims key = null;
|
||||
Integer value = null;
|
||||
|
||||
public FactsEntry(TimeAndDims key, Integer value)
|
||||
{
|
||||
this.key = key;
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TimeAndDims getKey()
|
||||
{
|
||||
return key;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getValue()
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer setValue(Integer value)
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
FactsEntry that = (FactsEntry) o;
|
||||
|
||||
if (key != null ? !key.equals(that.key) : that.key != null) {
|
||||
return false;
|
||||
}
|
||||
return value != null ? value.equals(that.value) : that.value == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = key != null ? key.hashCode() : 0;
|
||||
result = 31 * result + (value != null ? value.hashCode() : 0);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
interface FactsHolder
|
||||
{
|
||||
/**
|
||||
* @return the previous value associated with the specified key, or
|
||||
* {@code null} if there was no mapping for the key.
|
||||
* @return the previous rowIndex associated with the specified key, or
|
||||
* {@code TimeAndDims#EMPTY_ROW_INDEX} if there was no mapping for the key.
|
||||
*/
|
||||
Integer getPriorIndex(TimeAndDims key);
|
||||
int getPriorIndex(TimeAndDims key);
|
||||
|
||||
long getMinTimeMillis();
|
||||
|
||||
long getMaxTimeMillis();
|
||||
|
||||
Iterable<Map.Entry<TimeAndDims, Integer>> entrySet();
|
||||
Iterator<TimeAndDims> iterator(boolean descending);
|
||||
|
||||
Iterator<Map.Entry<TimeAndDims, Integer>> iterator(boolean descending);
|
||||
|
||||
Iterable<Map.Entry<TimeAndDims, Integer>> timeRangeIterable(boolean descending, long timeStart, long timeEnd);
|
||||
Iterable<TimeAndDims> timeRangeIterable(boolean descending, long timeStart, long timeEnd);
|
||||
|
||||
Iterable<TimeAndDims> keySet();
|
||||
|
||||
/**
|
||||
* @return the previous value associated with the specified key, or
|
||||
* {@code null} if there was no mapping for the key.
|
||||
* @return the previous rowIndex associated with the specified key, or
|
||||
* {@code TimeAndDims#EMPTY_ROW_INDEX} if there was no mapping for the key.
|
||||
*/
|
||||
Integer putIfAbsent(TimeAndDims key, Integer rowIndex);
|
||||
int putIfAbsent(TimeAndDims key, int rowIndex);
|
||||
|
||||
void clear();
|
||||
}
|
||||
|
@ -1079,7 +1038,8 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
static class RollupFactsHolder implements FactsHolder
|
||||
{
|
||||
private final boolean sortFacts;
|
||||
private final ConcurrentMap<TimeAndDims, Integer> facts;
|
||||
// Can't use Set because we need to be able to get from collection
|
||||
private final ConcurrentMap<TimeAndDims, TimeAndDims> facts;
|
||||
private final List<DimensionDesc> dimensionDescsList;
|
||||
|
||||
public RollupFactsHolder(boolean sortFacts, Comparator<TimeAndDims> timeAndDimsComparator, List<DimensionDesc> dimensionDescsList)
|
||||
|
@ -1094,16 +1054,17 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
}
|
||||
|
||||
@Override
|
||||
public Integer getPriorIndex(TimeAndDims key)
|
||||
public int getPriorIndex(TimeAndDims key)
|
||||
{
|
||||
return facts.get(key);
|
||||
TimeAndDims timeAndDims = facts.get(key);
|
||||
return timeAndDims == null ? TimeAndDims.EMPTY_ROW_INDEX : timeAndDims.rowIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMinTimeMillis()
|
||||
{
|
||||
if (sortFacts) {
|
||||
return ((ConcurrentNavigableMap<TimeAndDims, Integer>) facts).firstKey().getTimestamp();
|
||||
return ((ConcurrentNavigableMap<TimeAndDims, TimeAndDims>) facts).firstKey().getTimestamp();
|
||||
} else {
|
||||
throw new UnsupportedOperationException("can't get minTime from unsorted facts data.");
|
||||
}
|
||||
|
@ -1113,39 +1074,33 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
public long getMaxTimeMillis()
|
||||
{
|
||||
if (sortFacts) {
|
||||
return ((ConcurrentNavigableMap<TimeAndDims, Integer>) facts).lastKey().getTimestamp();
|
||||
return ((ConcurrentNavigableMap<TimeAndDims, TimeAndDims>) facts).lastKey().getTimestamp();
|
||||
} else {
|
||||
throw new UnsupportedOperationException("can't get maxTime from unsorted facts data.");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Map.Entry<TimeAndDims, Integer>> entrySet()
|
||||
{
|
||||
return facts.entrySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Map.Entry<TimeAndDims, Integer>> iterator(boolean descending)
|
||||
public Iterator<TimeAndDims> iterator(boolean descending)
|
||||
{
|
||||
if (descending && sortFacts) {
|
||||
return ((ConcurrentNavigableMap<TimeAndDims, Integer>) facts).descendingMap().entrySet().iterator();
|
||||
return ((ConcurrentNavigableMap<TimeAndDims, TimeAndDims>) facts).descendingMap().keySet().iterator();
|
||||
}
|
||||
return entrySet().iterator();
|
||||
return keySet().iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Map.Entry<TimeAndDims, Integer>> timeRangeIterable(boolean descending, long timeStart, long timeEnd)
|
||||
public Iterable<TimeAndDims> timeRangeIterable(boolean descending, long timeStart, long timeEnd)
|
||||
{
|
||||
if (!sortFacts) {
|
||||
throw new UnsupportedOperationException("can't get timeRange from unsorted facts data.");
|
||||
}
|
||||
TimeAndDims start = new TimeAndDims(timeStart, new Object[]{}, dimensionDescsList);
|
||||
TimeAndDims end = new TimeAndDims(timeEnd, new Object[]{}, dimensionDescsList);
|
||||
ConcurrentNavigableMap<TimeAndDims, Integer> subMap =
|
||||
((ConcurrentNavigableMap<TimeAndDims, Integer>) facts).subMap(start, end);
|
||||
final Map<TimeAndDims, Integer> rangeMap = descending ? subMap.descendingMap() : subMap;
|
||||
return rangeMap.entrySet();
|
||||
ConcurrentNavigableMap<TimeAndDims, TimeAndDims> subMap =
|
||||
((ConcurrentNavigableMap<TimeAndDims, TimeAndDims>) facts).subMap(start, end);
|
||||
final Map<TimeAndDims, TimeAndDims> rangeMap = descending ? subMap.descendingMap() : subMap;
|
||||
return rangeMap.keySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1155,9 +1110,12 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
}
|
||||
|
||||
@Override
|
||||
public Integer putIfAbsent(TimeAndDims key, Integer rowIndex)
|
||||
public int putIfAbsent(TimeAndDims key, int rowIndex)
|
||||
{
|
||||
return facts.putIfAbsent(key, rowIndex);
|
||||
// setRowIndex() must be called before facts.putIfAbsent() for visibility of rowIndex from concurrent readers.
|
||||
key.setRowIndex(rowIndex);
|
||||
TimeAndDims prev = facts.putIfAbsent(key, key);
|
||||
return prev == null ? TimeAndDims.EMPTY_ROW_INDEX : prev.rowIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1170,37 +1128,30 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
static class PlainFactsHolder implements FactsHolder
|
||||
{
|
||||
private final boolean sortFacts;
|
||||
private final ConcurrentMap<Long, Deque<Map.Entry<TimeAndDims, Integer>>> facts;
|
||||
private final ConcurrentMap<Long, Deque<TimeAndDims>> facts;
|
||||
|
||||
public PlainFactsHolder(boolean sortFacts)
|
||||
{
|
||||
this.sortFacts = sortFacts;
|
||||
if (sortFacts) {
|
||||
this.facts = new ConcurrentSkipListMap<>(new Comparator<Long>()
|
||||
{
|
||||
@Override
|
||||
public int compare(Long lhs, Long rhs)
|
||||
{
|
||||
return Longs.compare(lhs, rhs);
|
||||
}
|
||||
});
|
||||
this.facts = new ConcurrentSkipListMap<>();
|
||||
} else {
|
||||
this.facts = new ConcurrentHashMap<>();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getPriorIndex(TimeAndDims key)
|
||||
public int getPriorIndex(TimeAndDims key)
|
||||
{
|
||||
// always return null to indicate that no prior key cause we always add new row
|
||||
return null;
|
||||
// always return EMPTY_ROW_INDEX to indicate that no prior key cause we always add new row
|
||||
return TimeAndDims.EMPTY_ROW_INDEX;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMinTimeMillis()
|
||||
{
|
||||
if (sortFacts) {
|
||||
return ((ConcurrentNavigableMap<Long, Deque<Map.Entry<TimeAndDims, Integer>>>) facts).firstKey();
|
||||
return ((ConcurrentNavigableMap<Long, Deque<TimeAndDims>>) facts).firstKey();
|
||||
} else {
|
||||
throw new UnsupportedOperationException("can't get minTime from unsorted facts data.");
|
||||
}
|
||||
|
@ -1210,93 +1161,65 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
public long getMaxTimeMillis()
|
||||
{
|
||||
if (sortFacts) {
|
||||
return ((ConcurrentNavigableMap<Long, Deque<Map.Entry<TimeAndDims, Integer>>>) facts).lastKey();
|
||||
return ((ConcurrentNavigableMap<Long, Deque<TimeAndDims>>) facts).lastKey();
|
||||
} else {
|
||||
throw new UnsupportedOperationException("can't get maxTime from unsorted facts data.");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Map.Entry<TimeAndDims, Integer>> entrySet()
|
||||
{
|
||||
return concat(facts.values(), false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Map.Entry<TimeAndDims, Integer>> iterator(boolean descending)
|
||||
public Iterator<TimeAndDims> iterator(boolean descending)
|
||||
{
|
||||
if (descending && sortFacts) {
|
||||
return concat(((ConcurrentNavigableMap<Long, Deque<Map.Entry<TimeAndDims, Integer>>>) facts)
|
||||
return concat(((ConcurrentNavigableMap<Long, Deque<TimeAndDims>>) facts)
|
||||
.descendingMap().values(), true).iterator();
|
||||
}
|
||||
return concat(facts.values(), false).iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Map.Entry<TimeAndDims, Integer>> timeRangeIterable(boolean descending, long timeStart, long timeEnd)
|
||||
public Iterable<TimeAndDims> timeRangeIterable(boolean descending, long timeStart, long timeEnd)
|
||||
{
|
||||
ConcurrentNavigableMap<Long, Deque<Map.Entry<TimeAndDims, Integer>>> subMap =
|
||||
((ConcurrentNavigableMap<Long, Deque<Map.Entry<TimeAndDims, Integer>>>) facts).subMap(timeStart, timeEnd);
|
||||
final Map<Long, Deque<Map.Entry<TimeAndDims, Integer>>> rangeMap = descending ? subMap.descendingMap() : subMap;
|
||||
ConcurrentNavigableMap<Long, Deque<TimeAndDims>> subMap =
|
||||
((ConcurrentNavigableMap<Long, Deque<TimeAndDims>>) facts).subMap(timeStart, timeEnd);
|
||||
final Map<Long, Deque<TimeAndDims>> rangeMap = descending ? subMap.descendingMap() : subMap;
|
||||
return concat(rangeMap.values(), descending);
|
||||
}
|
||||
|
||||
private Iterable<Map.Entry<TimeAndDims, Integer>> concat(
|
||||
final Iterable<Deque<Map.Entry<TimeAndDims, Integer>>> iterable,
|
||||
private Iterable<TimeAndDims> concat(
|
||||
final Iterable<Deque<TimeAndDims>> iterable,
|
||||
final boolean descending
|
||||
)
|
||||
{
|
||||
return new Iterable<Map.Entry<TimeAndDims, Integer>>()
|
||||
{
|
||||
@Override
|
||||
public Iterator<Map.Entry<TimeAndDims, Integer>> iterator()
|
||||
{
|
||||
return Iterators.concat(
|
||||
Iterators.transform(
|
||||
iterable.iterator(),
|
||||
new Function<Deque<Map.Entry<TimeAndDims, Integer>>, Iterator<Map.Entry<TimeAndDims, Integer>>>()
|
||||
{
|
||||
@Override
|
||||
public Iterator<Map.Entry<TimeAndDims, Integer>> apply(Deque<Map.Entry<TimeAndDims, Integer>> input)
|
||||
{
|
||||
return descending ? input.descendingIterator() : input.iterator();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
};
|
||||
return () -> Iterators.concat(
|
||||
Iterators.transform(
|
||||
iterable.iterator(),
|
||||
input -> descending ? input.descendingIterator() : input.iterator()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<TimeAndDims> keySet()
|
||||
{
|
||||
return Iterables.transform(
|
||||
entrySet(),
|
||||
new Function<Map.Entry<TimeAndDims, Integer>, TimeAndDims>()
|
||||
{
|
||||
@Override
|
||||
public TimeAndDims apply(Map.Entry<TimeAndDims, Integer> input)
|
||||
{
|
||||
return input.getKey();
|
||||
}
|
||||
}
|
||||
);
|
||||
return concat(facts.values(), false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer putIfAbsent(TimeAndDims key, Integer rowIndex)
|
||||
public int putIfAbsent(TimeAndDims key, int rowIndex)
|
||||
{
|
||||
Long time = key.getTimestamp();
|
||||
Deque<Map.Entry<TimeAndDims, Integer>> rows = facts.get(time);
|
||||
Deque<TimeAndDims> rows = facts.get(time);
|
||||
if (rows == null) {
|
||||
facts.putIfAbsent(time, new ConcurrentLinkedDeque<Map.Entry<TimeAndDims, Integer>>());
|
||||
facts.putIfAbsent(time, new ConcurrentLinkedDeque<>());
|
||||
// in race condition, rows may be put by other thread, so always get latest status from facts
|
||||
rows = facts.get(time);
|
||||
}
|
||||
rows.add(new FactsEntry(key, rowIndex));
|
||||
// always return null to indicate that we always add new row
|
||||
return null;
|
||||
// setRowIndex() must be called before rows.add() for visibility of rowIndex from concurrent readers.
|
||||
key.setRowIndex(rowIndex);
|
||||
rows.add(key);
|
||||
// always return EMPTY_ROW_INDEX to indicate that we always add new row
|
||||
return TimeAndDims.EMPTY_ROW_INDEX;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -179,17 +179,16 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
|||
* iterator() call to ensure the counter starts at 0.
|
||||
*/
|
||||
return Iterators.transform(
|
||||
index.getFacts().entrySet().iterator(),
|
||||
new Function<Map.Entry<IncrementalIndex.TimeAndDims, Integer>, Rowboat>()
|
||||
index.getFacts().keySet().iterator(),
|
||||
new Function<IncrementalIndex.TimeAndDims, Rowboat>()
|
||||
{
|
||||
int count = 0;
|
||||
|
||||
@Override
|
||||
public Rowboat apply(Map.Entry<IncrementalIndex.TimeAndDims, Integer> input)
|
||||
public Rowboat apply(IncrementalIndex.TimeAndDims timeAndDims)
|
||||
{
|
||||
final IncrementalIndex.TimeAndDims timeAndDims = input.getKey();
|
||||
final Object[] dimValues = timeAndDims.getDims();
|
||||
final int rowOffset = input.getValue();
|
||||
final int rowOffset = timeAndDims.getRowIndex();
|
||||
|
||||
Object[] dims = new Object[dimValues.length];
|
||||
for (IncrementalIndex.DimensionDesc dimension : dimensions) {
|
||||
|
|
|
@ -241,8 +241,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
{
|
||||
private final ValueMatcher filterMatcher = makeFilterMatcher(filter, this);
|
||||
private final int maxRowIndex;
|
||||
private Iterator<Map.Entry<IncrementalIndex.TimeAndDims, Integer>> baseIter;
|
||||
private Iterable<Map.Entry<IncrementalIndex.TimeAndDims, Integer>> cursorIterable;
|
||||
private Iterator<IncrementalIndex.TimeAndDims> baseIter;
|
||||
private Iterable<IncrementalIndex.TimeAndDims> cursorIterable;
|
||||
private boolean emptyRange;
|
||||
final DateTime time;
|
||||
int numAdvanced = -1;
|
||||
|
@ -278,8 +278,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
while (baseIter.hasNext()) {
|
||||
BaseQuery.checkInterrupted();
|
||||
|
||||
Map.Entry<IncrementalIndex.TimeAndDims, Integer> entry = baseIter.next();
|
||||
if (beyondMaxRowIndex(entry.getValue())) {
|
||||
IncrementalIndex.TimeAndDims entry = baseIter.next();
|
||||
if (beyondMaxRowIndex(entry.getRowIndex())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -306,8 +306,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
return;
|
||||
}
|
||||
|
||||
Map.Entry<IncrementalIndex.TimeAndDims, Integer> entry = baseIter.next();
|
||||
if (beyondMaxRowIndex(entry.getValue())) {
|
||||
IncrementalIndex.TimeAndDims entry = baseIter.next();
|
||||
if (beyondMaxRowIndex(entry.getRowIndex())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -358,8 +358,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
|
||||
boolean foundMatched = false;
|
||||
while (baseIter.hasNext()) {
|
||||
Map.Entry<IncrementalIndex.TimeAndDims, Integer> entry = baseIter.next();
|
||||
if (beyondMaxRowIndex(entry.getValue())) {
|
||||
IncrementalIndex.TimeAndDims entry = baseIter.next();
|
||||
if (beyondMaxRowIndex(entry.getRowIndex())) {
|
||||
numAdvanced++;
|
||||
continue;
|
||||
}
|
||||
|
@ -642,26 +642,26 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
|
||||
public static class EntryHolder
|
||||
{
|
||||
Map.Entry<IncrementalIndex.TimeAndDims, Integer> currEntry = null;
|
||||
IncrementalIndex.TimeAndDims currEntry = null;
|
||||
|
||||
public Map.Entry<IncrementalIndex.TimeAndDims, Integer> get()
|
||||
public IncrementalIndex.TimeAndDims get()
|
||||
{
|
||||
return currEntry;
|
||||
}
|
||||
|
||||
public void set(Map.Entry<IncrementalIndex.TimeAndDims, Integer> currEntry)
|
||||
public void set(IncrementalIndex.TimeAndDims currEntry)
|
||||
{
|
||||
this.currEntry = currEntry;
|
||||
}
|
||||
|
||||
public IncrementalIndex.TimeAndDims getKey()
|
||||
{
|
||||
return currEntry.getKey();
|
||||
return currEntry;
|
||||
}
|
||||
|
||||
public Integer getValue()
|
||||
public int getValue()
|
||||
{
|
||||
return currEntry.getValue();
|
||||
return currEntry.getRowIndex();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -189,8 +189,8 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
|
|||
int bufferOffset;
|
||||
|
||||
synchronized (this) {
|
||||
final Integer priorIndex = facts.getPriorIndex(key);
|
||||
if (null != priorIndex) {
|
||||
final int priorIndex = facts.getPriorIndex(key);
|
||||
if (TimeAndDims.EMPTY_ROW_INDEX != priorIndex) {
|
||||
final int[] indexAndOffset = indexAndOffsets.get(priorIndex);
|
||||
bufferIndex = indexAndOffset[0];
|
||||
bufferOffset = indexAndOffset[1];
|
||||
|
@ -236,7 +236,7 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
|
|||
}
|
||||
|
||||
// Last ditch sanity checks
|
||||
if (numEntries.get() >= maxRowCount && facts.getPriorIndex(key) == null) {
|
||||
if (numEntries.get() >= maxRowCount && facts.getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX) {
|
||||
throw new IndexSizeExceededException("Maximum number of rows [%d] reached", maxRowCount);
|
||||
}
|
||||
|
||||
|
@ -245,8 +245,8 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
|
|||
// note that indexAndOffsets must be updated before facts, because as soon as we update facts
|
||||
// concurrent readers get hold of it and might ask for newly added row
|
||||
indexAndOffsets.add(new int[]{bufferIndex, bufferOffset});
|
||||
final Integer prev = facts.putIfAbsent(key, rowIndex);
|
||||
if (null == prev) {
|
||||
final int prev = facts.putIfAbsent(key, rowIndex);
|
||||
if (TimeAndDims.EMPTY_ROW_INDEX == prev) {
|
||||
numEntries.incrementAndGet();
|
||||
} else {
|
||||
throw new ISE("WTF! we are in sychronized block.");
|
||||
|
|
|
@ -180,11 +180,11 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
|
|||
Supplier<InputRow> rowSupplier
|
||||
) throws IndexSizeExceededException
|
||||
{
|
||||
final Integer priorIndex = facts.getPriorIndex(key);
|
||||
final int priorIndex = facts.getPriorIndex(key);
|
||||
|
||||
Aggregator[] aggs;
|
||||
|
||||
if (null != priorIndex) {
|
||||
if (TimeAndDims.EMPTY_ROW_INDEX != priorIndex) {
|
||||
aggs = concurrentGet(priorIndex);
|
||||
doAggregate(metrics, aggs, rowContainer, row, reportParseExceptions);
|
||||
} else {
|
||||
|
@ -196,11 +196,11 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
|
|||
concurrentSet(rowIndex, aggs);
|
||||
|
||||
// Last ditch sanity checks
|
||||
if (numEntries.get() >= maxRowCount && facts.getPriorIndex(key) == null) {
|
||||
if (numEntries.get() >= maxRowCount && facts.getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX) {
|
||||
throw new IndexSizeExceededException("Maximum number of rows [%d] reached", maxRowCount);
|
||||
}
|
||||
final Integer prev = facts.putIfAbsent(key, rowIndex);
|
||||
if (null == prev) {
|
||||
final int prev = facts.putIfAbsent(key, rowIndex);
|
||||
if (TimeAndDims.EMPTY_ROW_INDEX == prev) {
|
||||
numEntries.incrementAndGet();
|
||||
} else {
|
||||
// We lost a race
|
||||
|
|
|
@ -169,11 +169,11 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark
|
|||
|
||||
|
||||
// Last ditch sanity checks
|
||||
if (numEntries.get() >= maxRowCount && getFacts().getPriorIndex(key) == null) {
|
||||
if (numEntries.get() >= maxRowCount && getFacts().getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX) {
|
||||
throw new IndexSizeExceededException("Maximum number of rows reached");
|
||||
}
|
||||
final Integer prev = getFacts().putIfAbsent(key, rowIndex);
|
||||
if (null == prev) {
|
||||
final int prev = getFacts().putIfAbsent(key, rowIndex);
|
||||
if (TimeAndDims.EMPTY_ROW_INDEX == prev) {
|
||||
numEntries.incrementAndGet();
|
||||
} else {
|
||||
// We lost a race
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.easymock.EasyMock;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
|
@ -83,8 +82,8 @@ public class OnheapIncrementalIndexTest
|
|||
public void run()
|
||||
{
|
||||
while (!Thread.interrupted()) {
|
||||
for (Map.Entry<IncrementalIndex.TimeAndDims, Integer> row : index.getFacts().entrySet()) {
|
||||
if (index.getMetricLongValue(row.getValue(), 0) != 1) {
|
||||
for (IncrementalIndex.TimeAndDims row : index.getFacts().keySet()) {
|
||||
if (index.getMetricLongValue(row.getRowIndex(), 0) != 1) {
|
||||
checkFailedCount.addAndGet(1);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue