update IncrementalIndex to support unsorted facts map that can be used in groupBy merging to improve performance

This commit is contained in:
Himanshu Gupta 2016-03-01 10:10:50 -06:00
parent 1e49092ce7
commit 02dfd5cd80
8 changed files with 65 additions and 22 deletions

View File

@ -126,6 +126,7 @@ public class IncrementalIndexAddRowsBenchmark
aggs, aggs,
false, false,
false, false,
true,
maxRows maxRows
); );
} }

View File

@ -90,6 +90,7 @@ public class GroupByQueryHelper
aggs.toArray(new AggregatorFactory[aggs.size()]), aggs.toArray(new AggregatorFactory[aggs.size()]),
false, false,
true, true,
true,
Math.min(query.getContextValue(CTX_KEY_MAX_RESULTS, config.getMaxResults()), config.getMaxResults()), Math.min(query.getContextValue(CTX_KEY_MAX_RESULTS, config.getMaxResults()), config.getMaxResults()),
bufferPool bufferPool
); );
@ -102,6 +103,7 @@ public class GroupByQueryHelper
aggs.toArray(new AggregatorFactory[aggs.size()]), aggs.toArray(new AggregatorFactory[aggs.size()]),
false, false,
true, true,
true,
Math.min(query.getContextValue(CTX_KEY_MAX_RESULTS, config.getMaxResults()), config.getMaxResults()) Math.min(query.getContextValue(CTX_KEY_MAX_RESULTS, config.getMaxResults()), config.getMaxResults())
); );
} }

View File

@ -71,6 +71,7 @@ import java.util.Comparator;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentNavigableMap;
import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -340,6 +341,7 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
private final AggregatorType[] aggs; private final AggregatorType[] aggs;
private final boolean deserializeComplexMetrics; private final boolean deserializeComplexMetrics;
private final boolean reportParseExceptions; private final boolean reportParseExceptions;
private final boolean sortFacts;
private final Metadata metadata; private final Metadata metadata;
private final Map<String, MetricDesc> metricDescs; private final Map<String, MetricDesc> metricDescs;
@ -374,7 +376,8 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
public IncrementalIndex( public IncrementalIndex(
final IncrementalIndexSchema incrementalIndexSchema, final IncrementalIndexSchema incrementalIndexSchema,
final boolean deserializeComplexMetrics, final boolean deserializeComplexMetrics,
final boolean reportParseExceptions final boolean reportParseExceptions,
final boolean sortFacts
) )
{ {
this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); this.minTimestamp = incrementalIndexSchema.getMinTimestamp();
@ -383,6 +386,7 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
this.rowTransformers = new CopyOnWriteArrayList<>(); this.rowTransformers = new CopyOnWriteArrayList<>();
this.deserializeComplexMetrics = deserializeComplexMetrics; this.deserializeComplexMetrics = deserializeComplexMetrics;
this.reportParseExceptions = reportParseExceptions; this.reportParseExceptions = reportParseExceptions;
this.sortFacts = sortFacts;
this.metadata = new Metadata().setAggregators(getCombiningAggregators(metrics)); this.metadata = new Metadata().setAggregators(getCombiningAggregators(metrics));
@ -441,7 +445,7 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
// use newDimDim() to create a DimDim, makeDimDim() provides the subclass-specific implementation // use newDimDim() to create a DimDim, makeDimDim() provides the subclass-specific implementation
protected abstract DimDim makeDimDim(String dimension, Object lock); protected abstract DimDim makeDimDim(String dimension, Object lock);
public abstract ConcurrentNavigableMap<TimeAndDims, Integer> getFacts(); public abstract ConcurrentMap<TimeAndDims, Integer> getFacts();
public abstract boolean canAppendRow(); public abstract boolean canAppendRow();
@ -673,12 +677,20 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
private long getMinTimeMillis() private long getMinTimeMillis()
{ {
return getFacts().firstKey().getTimestamp(); if (sortFacts) {
return ((ConcurrentNavigableMap<TimeAndDims, Integer>) getFacts()).firstKey().getTimestamp();
} else {
throw new UnsupportedOperationException("can't get minTime from unsorted facts data.");
}
} }
private long getMaxTimeMillis() private long getMaxTimeMillis()
{ {
return getFacts().lastKey().getTimestamp(); if (sortFacts) {
return ((ConcurrentNavigableMap<TimeAndDims, Integer>) getFacts()).lastKey().getTimestamp();
} else {
throw new UnsupportedOperationException("can't get maxTime from unsorted facts data.");
}
} }
private int[] getDimVals(final DimDim dimLookup, final List<Comparable> dimValues) private int[] getDimVals(final DimDim dimLookup, final List<Comparable> dimValues)
@ -831,7 +843,11 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
public ConcurrentNavigableMap<TimeAndDims, Integer> getSubMap(TimeAndDims start, TimeAndDims end) public ConcurrentNavigableMap<TimeAndDims, Integer> getSubMap(TimeAndDims start, TimeAndDims end)
{ {
return getFacts().subMap(start, end); if (sortFacts) {
return ((ConcurrentNavigableMap<TimeAndDims, Integer>) getFacts()).subMap(start, end);
} else {
throw new UnsupportedOperationException("can't get subMap from unsorted facts data.");
}
} }
public Metadata getMetadata() public Metadata getMetadata()
@ -862,7 +878,14 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
public Iterator<Row> iterator() public Iterator<Row> iterator()
{ {
final List<DimensionDesc> dimensions = getDimensions(); final List<DimensionDesc> dimensions = getDimensions();
final ConcurrentNavigableMap<TimeAndDims, Integer> facts = descending ? getFacts().descendingMap() : getFacts();
Map<TimeAndDims, Integer> facts = null;
if (descending && sortFacts) {
facts = ((ConcurrentNavigableMap<TimeAndDims, Integer>) getFacts()).descendingMap();
} else {
facts = getFacts();
}
return Iterators.transform( return Iterators.transform(
facts.entrySet().iterator(), facts.entrySet().iterator(),
new Function<Map.Entry<TimeAndDims, Integer>, Row>() new Function<Map.Entry<TimeAndDims, Integer>, Row>()

View File

@ -38,7 +38,8 @@ import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -51,7 +52,7 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
private final List<ResourceHolder<ByteBuffer>> aggBuffers = new ArrayList<>(); private final List<ResourceHolder<ByteBuffer>> aggBuffers = new ArrayList<>();
private final List<int[]> indexAndOffsets = new ArrayList<>(); private final List<int[]> indexAndOffsets = new ArrayList<>();
private final ConcurrentNavigableMap<TimeAndDims, Integer> facts; private final ConcurrentMap<TimeAndDims, Integer> facts;
private final AtomicInteger indexIncrement = new AtomicInteger(0); private final AtomicInteger indexIncrement = new AtomicInteger(0);
@ -71,14 +72,20 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
IncrementalIndexSchema incrementalIndexSchema, IncrementalIndexSchema incrementalIndexSchema,
boolean deserializeComplexMetrics, boolean deserializeComplexMetrics,
boolean reportParseExceptions, boolean reportParseExceptions,
boolean sortFacts,
int maxRowCount, int maxRowCount,
StupidPool<ByteBuffer> bufferPool StupidPool<ByteBuffer> bufferPool
) )
{ {
super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions); super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions, sortFacts);
this.maxRowCount = maxRowCount; this.maxRowCount = maxRowCount;
this.bufferPool = bufferPool; this.bufferPool = bufferPool;
this.facts = new ConcurrentSkipListMap<>(dimsComparator());
if (sortFacts) {
this.facts = new ConcurrentSkipListMap<>(dimsComparator());
} else {
this.facts = new ConcurrentHashMap<>();
}
//check that stupid pool gives buffers that can hold at least one row's aggregators //check that stupid pool gives buffers that can hold at least one row's aggregators
ResourceHolder<ByteBuffer> bb = bufferPool.take(); ResourceHolder<ByteBuffer> bb = bufferPool.take();
@ -100,6 +107,7 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
final AggregatorFactory[] metrics, final AggregatorFactory[] metrics,
boolean deserializeComplexMetrics, boolean deserializeComplexMetrics,
boolean reportParseExceptions, boolean reportParseExceptions,
boolean sortFacts,
int maxRowCount, int maxRowCount,
StupidPool<ByteBuffer> bufferPool StupidPool<ByteBuffer> bufferPool
) )
@ -111,6 +119,7 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
.build(), .build(),
deserializeComplexMetrics, deserializeComplexMetrics,
reportParseExceptions, reportParseExceptions,
sortFacts,
maxRowCount, maxRowCount,
bufferPool bufferPool
); );
@ -131,13 +140,14 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
.build(), .build(),
true, true,
true, true,
true,
maxRowCount, maxRowCount,
bufferPool bufferPool
); );
} }
@Override @Override
public ConcurrentNavigableMap<TimeAndDims, Integer> getFacts() public ConcurrentMap<TimeAndDims, Integer> getFacts()
{ {
return facts; return facts;
} }

View File

@ -33,13 +33,11 @@ import io.druid.segment.DimensionSelector;
import io.druid.segment.FloatColumnSelector; import io.druid.segment.FloatColumnSelector;
import io.druid.segment.LongColumnSelector; import io.druid.segment.LongColumnSelector;
import io.druid.segment.ObjectColumnSelector; import io.druid.segment.ObjectColumnSelector;
import io.druid.segment.column.ValueType;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentNavigableMap;
import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -48,7 +46,7 @@ import java.util.concurrent.atomic.AtomicInteger;
public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator> public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
{ {
private final ConcurrentHashMap<Integer, Aggregator[]> aggregators = new ConcurrentHashMap<>(); private final ConcurrentHashMap<Integer, Aggregator[]> aggregators = new ConcurrentHashMap<>();
private final ConcurrentNavigableMap<TimeAndDims, Integer> facts; private final ConcurrentMap<TimeAndDims, Integer> facts;
private final AtomicInteger indexIncrement = new AtomicInteger(0); private final AtomicInteger indexIncrement = new AtomicInteger(0);
protected final int maxRowCount; protected final int maxRowCount;
private volatile Map<String, ColumnSelectorFactory> selectors; private volatile Map<String, ColumnSelectorFactory> selectors;
@ -59,12 +57,18 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
IncrementalIndexSchema incrementalIndexSchema, IncrementalIndexSchema incrementalIndexSchema,
boolean deserializeComplexMetrics, boolean deserializeComplexMetrics,
boolean reportParseExceptions, boolean reportParseExceptions,
boolean sortFacts,
int maxRowCount int maxRowCount
) )
{ {
super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions); super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions, sortFacts);
this.maxRowCount = maxRowCount; this.maxRowCount = maxRowCount;
this.facts = new ConcurrentSkipListMap<>(dimsComparator());
if (sortFacts) {
this.facts = new ConcurrentSkipListMap<>(dimsComparator());
} else {
this.facts = new ConcurrentHashMap<>();
}
} }
public OnheapIncrementalIndex( public OnheapIncrementalIndex(
@ -73,6 +77,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
final AggregatorFactory[] metrics, final AggregatorFactory[] metrics,
boolean deserializeComplexMetrics, boolean deserializeComplexMetrics,
boolean reportParseExceptions, boolean reportParseExceptions,
boolean sortFacts,
int maxRowCount int maxRowCount
) )
{ {
@ -83,6 +88,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
.build(), .build(),
deserializeComplexMetrics, deserializeComplexMetrics,
reportParseExceptions, reportParseExceptions,
sortFacts,
maxRowCount maxRowCount
); );
} }
@ -101,6 +107,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
.build(), .build(),
true, true,
true, true,
true,
maxRowCount maxRowCount
); );
} }
@ -111,11 +118,11 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
int maxRowCount int maxRowCount
) )
{ {
this(incrementalIndexSchema, true, reportParseExceptions, maxRowCount); this(incrementalIndexSchema, true, reportParseExceptions, true, maxRowCount);
} }
@Override @Override
public ConcurrentNavigableMap<TimeAndDims, Integer> getFacts() public ConcurrentMap<TimeAndDims, Integer> getFacts()
{ {
return facts; return facts;
} }

View File

@ -99,6 +99,7 @@ public class MultiValuedDimensionTest
}, },
true, true,
true, true,
true,
5000 5000
); );

View File

@ -71,7 +71,6 @@ import io.druid.segment.QueryableIndexSegment;
import io.druid.segment.Segment; import io.druid.segment.Segment;
import io.druid.segment.TestHelper; import io.druid.segment.TestHelper;
import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IndexSizeExceededException;
import io.druid.segment.incremental.OnheapIncrementalIndex; import io.druid.segment.incremental.OnheapIncrementalIndex;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.commons.io.LineIterator; import org.apache.commons.io.LineIterator;
@ -311,7 +310,7 @@ public class AggregationTestHelper
List<File> toMerge = new ArrayList<>(); List<File> toMerge = new ArrayList<>();
try { try {
index = new OnheapIncrementalIndex(minTimestamp, gran, metrics, deserializeComplexMetrics, true, maxRowCount); index = new OnheapIncrementalIndex(minTimestamp, gran, metrics, deserializeComplexMetrics, true, true, maxRowCount);
while (rows.hasNext()) { while (rows.hasNext()) {
Object row = rows.next(); Object row = rows.next();
if (!index.canAppendRow()) { if (!index.canAppendRow()) {
@ -319,7 +318,7 @@ public class AggregationTestHelper
toMerge.add(tmp); toMerge.add(tmp);
indexMerger.persist(index, tmp, new IndexSpec()); indexMerger.persist(index, tmp, new IndexSpec());
index.close(); index.close();
index = new OnheapIncrementalIndex(minTimestamp, gran, metrics, deserializeComplexMetrics, true, maxRowCount); index = new OnheapIncrementalIndex(minTimestamp, gran, metrics, deserializeComplexMetrics, true, true, maxRowCount);
} }
if (row instanceof String && parser instanceof StringInputRowParser) { if (row instanceof String && parser instanceof StringInputRowParser) {
//Note: this is required because StringInputRowParser is InputRowParser<ByteBuffer> as opposed to //Note: this is required because StringInputRowParser is InputRowParser<ByteBuffer> as opposed to

View File

@ -120,7 +120,7 @@ public class IngestSegmentFirehoseTest
IncrementalIndex index = null; IncrementalIndex index = null;
try { try {
index = new OnheapIncrementalIndex(0, QueryGranularity.NONE, aggregators, true, true, 5000); index = new OnheapIncrementalIndex(0, QueryGranularity.NONE, aggregators, true, true, true, 5000);
for (String line : rows) { for (String line : rows) {
index.add(parser.parse(line)); index.add(parser.parse(line));
} }