mirror of https://github.com/apache/druid.git
update IncrementalIndex to support unsorted facts map that can be used in groupBy merging to improve performance
This commit is contained in:
parent
1e49092ce7
commit
02dfd5cd80
|
@ -126,6 +126,7 @@ public class IncrementalIndexAddRowsBenchmark
|
||||||
aggs,
|
aggs,
|
||||||
false,
|
false,
|
||||||
false,
|
false,
|
||||||
|
true,
|
||||||
maxRows
|
maxRows
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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>()
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -99,6 +99,7 @@ public class MultiValuedDimensionTest
|
||||||
},
|
},
|
||||||
true,
|
true,
|
||||||
true,
|
true,
|
||||||
|
true,
|
||||||
5000
|
5000
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue