mirror of
https://github.com/apache/druid.git
synced 2025-02-25 20:48:05 +00:00
code cleanups & formatting
This commit is contained in:
parent
eac776f1a7
commit
6dc69c2f30
@ -84,13 +84,13 @@ public class ApproximateHistogramFoldingAggregator implements Aggregator
|
|||||||
@Override
|
@Override
|
||||||
public float getFloat()
|
public float getFloat()
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException("ApproximateHistogramAggregator does not support getFloat()");
|
throw new UnsupportedOperationException("ApproximateHistogramFoldingAggregator does not support getFloat()");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getLong()
|
public long getLong()
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException("ApproximateHistogramAggregator does not support getLong()");
|
throw new UnsupportedOperationException("ApproximateHistogramFoldingAggregator does not support getLong()");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -63,13 +63,13 @@ public class HyperUniquesAggregator implements Aggregator
|
|||||||
@Override
|
@Override
|
||||||
public float getFloat()
|
public float getFloat()
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException("HyperUniquesAggregator does not support getFloat()");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getLong()
|
public long getLong()
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException("HyperUniquesAggregator does not support getLong()");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -62,7 +62,7 @@ public interface IncrementalIndex extends Iterable<Row>, Closeable
|
|||||||
|
|
||||||
boolean isEmpty();
|
boolean isEmpty();
|
||||||
|
|
||||||
ConcurrentNavigableMap<TimeAndDims,Integer> getSubMap(TimeAndDims start, TimeAndDims end);
|
ConcurrentNavigableMap<TimeAndDims, Integer> getSubMap(TimeAndDims start, TimeAndDims end);
|
||||||
|
|
||||||
Integer getMetricIndex(String columnName);
|
Integer getMetricIndex(String columnName);
|
||||||
|
|
||||||
|
@ -22,15 +22,11 @@ package io.druid.segment.incremental;
|
|||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.BiMap;
|
|
||||||
import com.google.common.collect.HashBiMap;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Iterators;
|
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.primitives.Ints;
|
|
||||||
import com.google.common.primitives.Longs;
|
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import io.druid.collections.ResourceHolder;
|
import io.druid.collections.ResourceHolder;
|
||||||
@ -63,8 +59,6 @@ import org.mapdb.DB;
|
|||||||
import org.mapdb.DBMaker;
|
import org.mapdb.DBMaker;
|
||||||
import org.mapdb.Serializer;
|
import org.mapdb.Serializer;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.io.Closeable;
|
|
||||||
import java.io.DataInput;
|
import java.io.DataInput;
|
||||||
import java.io.DataOutput;
|
import java.io.DataOutput;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
@ -80,7 +74,6 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.WeakHashMap;
|
import java.util.WeakHashMap;
|
||||||
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;
|
||||||
@ -143,7 +136,7 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||||||
@Override
|
@Override
|
||||||
public LongColumnSelector makeLongColumnSelector(final String columnName)
|
public LongColumnSelector makeLongColumnSelector(final String columnName)
|
||||||
{
|
{
|
||||||
if(columnName.equals(Column.TIME_COLUMN_NAME)){
|
if (columnName.equals(Column.TIME_COLUMN_NAME)) {
|
||||||
return new LongColumnSelector()
|
return new LongColumnSelector()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
@ -333,18 +326,18 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||||||
}
|
}
|
||||||
this.bufferHolder = bufferPool.take();
|
this.bufferHolder = bufferPool.take();
|
||||||
this.dimValues = new DimensionHolder();
|
this.dimValues = new DimensionHolder();
|
||||||
final DBMaker dbMaker = DBMaker.newMemoryDirectDB()
|
final DBMaker dbMaker = DBMaker.newMemoryDirectDB()
|
||||||
.transactionDisable()
|
.transactionDisable()
|
||||||
.asyncWriteEnable()
|
.asyncWriteEnable()
|
||||||
.cacheSoftRefEnable();
|
.cacheSoftRefEnable();
|
||||||
factsDb = dbMaker.make();
|
factsDb = dbMaker.make();
|
||||||
db = dbMaker.make();
|
db = dbMaker.make();
|
||||||
final TimeAndDimsSerializer timeAndDimsSerializer = new TimeAndDimsSerializer(this);
|
final TimeAndDimsSerializer timeAndDimsSerializer = new TimeAndDimsSerializer(this);
|
||||||
this.facts = factsDb.createTreeMap("__facts" + UUID.randomUUID())
|
this.facts = factsDb.createTreeMap("__facts" + UUID.randomUUID())
|
||||||
.keySerializer(timeAndDimsSerializer)
|
.keySerializer(timeAndDimsSerializer)
|
||||||
.comparator(timeAndDimsSerializer.getComparator())
|
.comparator(timeAndDimsSerializer.getComparator())
|
||||||
.valueSerializer(Serializer.INTEGER)
|
.valueSerializer(Serializer.INTEGER)
|
||||||
.make();
|
.make();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -366,6 +359,7 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public InputRow formatRow(InputRow row)
|
public InputRow formatRow(InputRow row)
|
||||||
{
|
{
|
||||||
for (Function<InputRow, InputRow> rowTransformer : rowTransformers) {
|
for (Function<InputRow, InputRow> rowTransformer : rowTransformers) {
|
||||||
@ -390,6 +384,7 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||||||
*
|
*
|
||||||
* @return the number of rows in the data set after adding the InputRow
|
* @return the number of rows in the data set after adding the InputRow
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public int add(InputRow row)
|
public int add(InputRow row)
|
||||||
{
|
{
|
||||||
row = formatRow(row);
|
row = formatRow(row);
|
||||||
@ -471,22 +466,24 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||||||
return numEntries.get();
|
return numEntries.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public boolean isEmpty()
|
public boolean isEmpty()
|
||||||
{
|
{
|
||||||
return numEntries.get() == 0;
|
return numEntries.get() == 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public int size()
|
public int size()
|
||||||
{
|
{
|
||||||
return numEntries.get();
|
return numEntries.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getMinTimeMillis()
|
private long getMinTimeMillis()
|
||||||
{
|
{
|
||||||
return facts.firstKey().getTimestamp();
|
return facts.firstKey().getTimestamp();
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getMaxTimeMillis()
|
private long getMaxTimeMillis()
|
||||||
{
|
{
|
||||||
return facts.lastKey().getTimestamp();
|
return facts.lastKey().getTimestamp();
|
||||||
}
|
}
|
||||||
@ -509,61 +506,61 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||||||
return retVal;
|
return retVal;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public AggregatorFactory[] getMetricAggs()
|
public AggregatorFactory[] getMetricAggs()
|
||||||
{
|
{
|
||||||
return metrics;
|
return metrics;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public List<String> getDimensions()
|
public List<String> getDimensions()
|
||||||
{
|
{
|
||||||
return dimensions;
|
return dimensions;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public String getMetricType(String metric)
|
public String getMetricType(String metric)
|
||||||
{
|
{
|
||||||
return metricTypes.get(metric);
|
return metricTypes.get(metric);
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getMinTimestamp()
|
@Override
|
||||||
{
|
|
||||||
return minTimestamp;
|
|
||||||
}
|
|
||||||
|
|
||||||
public QueryGranularity getGranularity()
|
|
||||||
{
|
|
||||||
return gran;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Interval getInterval()
|
public Interval getInterval()
|
||||||
{
|
{
|
||||||
return new Interval(minTimestamp, isEmpty() ? minTimestamp : gran.next(getMaxTimeMillis()));
|
return new Interval(minTimestamp, isEmpty() ? minTimestamp : gran.next(getMaxTimeMillis()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public DateTime getMinTime()
|
public DateTime getMinTime()
|
||||||
{
|
{
|
||||||
return isEmpty() ? null : new DateTime(getMinTimeMillis());
|
return isEmpty() ? null : new DateTime(getMinTimeMillis());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public DateTime getMaxTime()
|
public DateTime getMaxTime()
|
||||||
{
|
{
|
||||||
return isEmpty() ? null : new DateTime(getMaxTimeMillis());
|
return isEmpty() ? null : new DateTime(getMaxTimeMillis());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public DimDim getDimension(String dimension)
|
public DimDim getDimension(String dimension)
|
||||||
{
|
{
|
||||||
return isEmpty() ? null : dimValues.get(dimension);
|
return isEmpty() ? null : dimValues.get(dimension);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Integer getDimensionIndex(String dimension)
|
public Integer getDimensionIndex(String dimension)
|
||||||
{
|
{
|
||||||
return dimensionOrder.get(dimension);
|
return dimensionOrder.get(dimension);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public List<String> getMetricNames()
|
public List<String> getMetricNames()
|
||||||
{
|
{
|
||||||
return metricNames;
|
return metricNames;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Integer getMetricIndex(String metricName)
|
public Integer getMetricIndex(String metricName)
|
||||||
{
|
{
|
||||||
return metricIndexes.get(metricName);
|
return metricIndexes.get(metricName);
|
||||||
@ -592,16 +589,19 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||||||
return aggs[aggOffset].get(bufferHolder.get(), getMetricPosition(rowOffset, aggOffset));
|
return aggs[aggOffset].get(bufferHolder.get(), getMetricPosition(rowOffset, aggOffset));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public ColumnCapabilities getCapabilities(String column)
|
public ColumnCapabilities getCapabilities(String column)
|
||||||
{
|
{
|
||||||
return columnCapabilities.get(column);
|
return columnCapabilities.get(column);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public ConcurrentNavigableMap<TimeAndDims, Integer> getFacts()
|
public ConcurrentNavigableMap<TimeAndDims, Integer> getFacts()
|
||||||
{
|
{
|
||||||
return facts;
|
return facts;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public ConcurrentNavigableMap<TimeAndDims, Integer> getSubMap(TimeAndDims start, TimeAndDims end)
|
public ConcurrentNavigableMap<TimeAndDims, Integer> getSubMap(TimeAndDims start, TimeAndDims end)
|
||||||
{
|
{
|
||||||
return facts.subMap(start, end);
|
return facts.subMap(start, end);
|
||||||
@ -678,11 +678,6 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||||||
dimensions = Maps.newConcurrentMap();
|
dimensions = Maps.newConcurrentMap();
|
||||||
}
|
}
|
||||||
|
|
||||||
void reset()
|
|
||||||
{
|
|
||||||
dimensions.clear();
|
|
||||||
}
|
|
||||||
|
|
||||||
DimDim add(String dimension)
|
DimDim add(String dimension)
|
||||||
{
|
{
|
||||||
DimDim holder = dimensions.get(dimension);
|
DimDim holder = dimensions.get(dimension);
|
||||||
|
@ -27,11 +27,8 @@ import com.google.common.collect.ImmutableMap;
|
|||||||
import com.google.common.collect.Iterators;
|
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.primitives.Ints;
|
|
||||||
import com.google.common.primitives.Longs;
|
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import io.druid.collections.StupidPool;
|
|
||||||
import io.druid.data.input.InputRow;
|
import io.druid.data.input.InputRow;
|
||||||
import io.druid.data.input.MapBasedRow;
|
import io.druid.data.input.MapBasedRow;
|
||||||
import io.druid.data.input.Row;
|
import io.druid.data.input.Row;
|
||||||
@ -56,9 +53,6 @@ import io.druid.segment.serde.ComplexMetrics;
|
|||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.io.Closeable;
|
|
||||||
import java.nio.ByteBuffer;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
@ -161,14 +155,10 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
columnCapabilities.put(spatialDimension.getDimName(), capabilities);
|
columnCapabilities.put(spatialDimension.getDimName(), capabilities);
|
||||||
}
|
}
|
||||||
this.dimValues = new DimensionHolder();
|
this.dimValues = new DimensionHolder();
|
||||||
this.facts = createFactsTable();
|
this.facts = new ConcurrentSkipListMap<>();
|
||||||
this.deserializeComplexMetrics = deserializeComplexMetrics;
|
this.deserializeComplexMetrics = deserializeComplexMetrics;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected ConcurrentNavigableMap<TimeAndDims, Integer> createFactsTable() {
|
|
||||||
return new ConcurrentSkipListMap<>();
|
|
||||||
}
|
|
||||||
|
|
||||||
public OnheapIncrementalIndex(
|
public OnheapIncrementalIndex(
|
||||||
long minTimestamp,
|
long minTimestamp,
|
||||||
QueryGranularity gran,
|
QueryGranularity gran,
|
||||||
@ -207,6 +197,7 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public InputRow formatRow(InputRow row)
|
public InputRow formatRow(InputRow row)
|
||||||
{
|
{
|
||||||
for (Function<InputRow, InputRow> rowTransformer : rowTransformers) {
|
for (Function<InputRow, InputRow> rowTransformer : rowTransformers) {
|
||||||
@ -231,6 +222,7 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
*
|
*
|
||||||
* @return the number of rows in the data set after adding the InputRow
|
* @return the number of rows in the data set after adding the InputRow
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public int add(InputRow row)
|
public int add(InputRow row)
|
||||||
{
|
{
|
||||||
row = formatRow(row);
|
row = formatRow(row);
|
||||||
@ -287,7 +279,7 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
final TimeAndDims key = new TimeAndDims(Math.max(gran.truncate(row.getTimestampFromEpoch()), minTimestamp), dims);
|
final TimeAndDims key = new TimeAndDims(Math.max(gran.truncate(row.getTimestampFromEpoch()), minTimestamp), dims);
|
||||||
Integer rowOffset;
|
Integer rowOffset;
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
rowOffset = numEntries.get();
|
rowOffset = numEntries.get();
|
||||||
final Integer prev = facts.putIfAbsent(key, rowOffset);
|
final Integer prev = facts.putIfAbsent(key, rowOffset);
|
||||||
if (prev != null) {
|
if (prev != null) {
|
||||||
rowOffset = prev;
|
rowOffset = prev;
|
||||||
@ -301,7 +293,7 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
@Override
|
@Override
|
||||||
public LongColumnSelector makeLongColumnSelector(final String columnName)
|
public LongColumnSelector makeLongColumnSelector(final String columnName)
|
||||||
{
|
{
|
||||||
if(columnName.equals(Column.TIME_COLUMN_NAME)){
|
if (columnName.equals(Column.TIME_COLUMN_NAME)) {
|
||||||
return new LongColumnSelector()
|
return new LongColumnSelector()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
@ -459,11 +451,13 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
return numEntries.get();
|
return numEntries.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public boolean isEmpty()
|
public boolean isEmpty()
|
||||||
{
|
{
|
||||||
return numEntries.get() == 0;
|
return numEntries.get() == 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public int size()
|
public int size()
|
||||||
{
|
{
|
||||||
return numEntries.get();
|
return numEntries.get();
|
||||||
@ -487,12 +481,12 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
return aggList.get(rowOffset)[aggOffset].get();
|
return aggList.get(rowOffset)[aggOffset].get();
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getMinTimeMillis()
|
private long getMinTimeMillis()
|
||||||
{
|
{
|
||||||
return facts.firstKey().getTimestamp();
|
return facts.firstKey().getTimestamp();
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getMaxTimeMillis()
|
private long getMaxTimeMillis()
|
||||||
{
|
{
|
||||||
return facts.lastKey().getTimestamp();
|
return facts.lastKey().getTimestamp();
|
||||||
}
|
}
|
||||||
@ -515,31 +509,24 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
return retVal;
|
return retVal;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public AggregatorFactory[] getMetricAggs()
|
public AggregatorFactory[] getMetricAggs()
|
||||||
{
|
{
|
||||||
return metrics;
|
return metrics;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public List<String> getDimensions()
|
public List<String> getDimensions()
|
||||||
{
|
{
|
||||||
return dimensions;
|
return dimensions;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public String getMetricType(String metric)
|
public String getMetricType(String metric)
|
||||||
{
|
{
|
||||||
return metricTypes.get(metric);
|
return metricTypes.get(metric);
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getMinTimestamp()
|
|
||||||
{
|
|
||||||
return minTimestamp;
|
|
||||||
}
|
|
||||||
|
|
||||||
public QueryGranularity getGranularity()
|
|
||||||
{
|
|
||||||
return gran;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Interval getInterval()
|
public Interval getInterval()
|
||||||
{
|
{
|
||||||
return new Interval(minTimestamp, isEmpty() ? minTimestamp : gran.next(getMaxTimeMillis()));
|
return new Interval(minTimestamp, isEmpty() ? minTimestamp : gran.next(getMaxTimeMillis()));
|
||||||
@ -575,11 +562,6 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
return metricIndexes.get(metricName);
|
return metricIndexes.get(metricName);
|
||||||
}
|
}
|
||||||
|
|
||||||
Aggregator getAggregator(int rowOffset, int metricIndex)
|
|
||||||
{
|
|
||||||
return aggList.get(rowOffset)[metricIndex];
|
|
||||||
}
|
|
||||||
|
|
||||||
public ColumnCapabilities getCapabilities(String column)
|
public ColumnCapabilities getCapabilities(String column)
|
||||||
{
|
{
|
||||||
return columnCapabilities.get(column);
|
return columnCapabilities.get(column);
|
||||||
@ -601,6 +583,7 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
return iterableWithPostAggregations(null).iterator();
|
return iterableWithPostAggregations(null).iterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Iterable<Row> iterableWithPostAggregations(final List<PostAggregator> postAggs)
|
public Iterable<Row> iterableWithPostAggregations(final List<PostAggregator> postAggs)
|
||||||
{
|
{
|
||||||
return new Iterable<Row>()
|
return new Iterable<Row>()
|
||||||
@ -649,6 +632,7 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
@Override
|
@Override
|
||||||
public void close()
|
public void close()
|
||||||
{
|
{
|
||||||
|
// Nothing to close
|
||||||
}
|
}
|
||||||
|
|
||||||
class DimensionHolder
|
class DimensionHolder
|
||||||
@ -660,11 +644,6 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
dimensions = Maps.newConcurrentMap();
|
dimensions = Maps.newConcurrentMap();
|
||||||
}
|
}
|
||||||
|
|
||||||
void reset()
|
|
||||||
{
|
|
||||||
dimensions.clear();
|
|
||||||
}
|
|
||||||
|
|
||||||
DimDim add(String dimension)
|
DimDim add(String dimension)
|
||||||
{
|
{
|
||||||
DimDim holder = dimensions.get(dimension);
|
DimDim holder = dimensions.get(dimension);
|
||||||
@ -683,7 +662,8 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class DimDimImpl implements DimDim{
|
private static class DimDimImpl implements DimDim
|
||||||
|
{
|
||||||
private final Map<String, Integer> falseIds;
|
private final Map<String, Integer> falseIds;
|
||||||
private final Map<Integer, String> falseIdsReverse;
|
private final Map<Integer, String> falseIdsReverse;
|
||||||
private volatile String[] sortedVals = null;
|
private volatile String[] sortedVals = null;
|
||||||
@ -699,6 +679,7 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the interned String value to allow fast comparisons using `==` instead of `.equals()`
|
* Returns the interned String value to allow fast comparisons using `==` instead of `.equals()`
|
||||||
|
*
|
||||||
* @see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String)
|
* @see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String)
|
||||||
*/
|
*/
|
||||||
public String get(String str)
|
public String get(String str)
|
||||||
@ -772,7 +753,7 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||||||
|
|
||||||
public boolean compareCannonicalValues(String s1, String s2)
|
public boolean compareCannonicalValues(String s1, String s2)
|
||||||
{
|
{
|
||||||
return s1 ==s2;
|
return s1 == s2;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user