mirror of https://github.com/apache/druid.git
code cleanups & formatting
This commit is contained in:
parent
eac776f1a7
commit
6dc69c2f30
|
@ -84,13 +84,13 @@ public class ApproximateHistogramFoldingAggregator implements Aggregator
|
|||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
throw new UnsupportedOperationException("ApproximateHistogramAggregator does not support getFloat()");
|
||||
throw new UnsupportedOperationException("ApproximateHistogramFoldingAggregator does not support getFloat()");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
throw new UnsupportedOperationException("ApproximateHistogramAggregator does not support getLong()");
|
||||
throw new UnsupportedOperationException("ApproximateHistogramFoldingAggregator does not support getLong()");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -63,13 +63,13 @@ public class HyperUniquesAggregator implements Aggregator
|
|||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
throw new UnsupportedOperationException("HyperUniquesAggregator does not support getFloat()");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
throw new UnsupportedOperationException("HyperUniquesAggregator does not support getLong()");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,15 +22,11 @@ package io.druid.segment.incremental;
|
|||
|
||||
import com.google.common.base.Function;
|
||||
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.ImmutableMap;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
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.ISE;
|
||||
import io.druid.collections.ResourceHolder;
|
||||
|
@ -63,8 +59,6 @@ import org.mapdb.DB;
|
|||
import org.mapdb.DBMaker;
|
||||
import org.mapdb.Serializer;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
@ -80,7 +74,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.WeakHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ConcurrentNavigableMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
@ -366,6 +359,7 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRow formatRow(InputRow row)
|
||||
{
|
||||
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
|
||||
*/
|
||||
@Override
|
||||
public int add(InputRow row)
|
||||
{
|
||||
row = formatRow(row);
|
||||
|
@ -471,22 +466,24 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||
return numEntries.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty()
|
||||
{
|
||||
return numEntries.get() == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return numEntries.get();
|
||||
}
|
||||
|
||||
public long getMinTimeMillis()
|
||||
private long getMinTimeMillis()
|
||||
{
|
||||
return facts.firstKey().getTimestamp();
|
||||
}
|
||||
|
||||
public long getMaxTimeMillis()
|
||||
private long getMaxTimeMillis()
|
||||
{
|
||||
return facts.lastKey().getTimestamp();
|
||||
}
|
||||
|
@ -509,61 +506,61 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory[] getMetricAggs()
|
||||
{
|
||||
return metrics;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDimensions()
|
||||
{
|
||||
return dimensions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getMetricType(String metric)
|
||||
{
|
||||
return metricTypes.get(metric);
|
||||
}
|
||||
|
||||
public long getMinTimestamp()
|
||||
{
|
||||
return minTimestamp;
|
||||
}
|
||||
|
||||
public QueryGranularity getGranularity()
|
||||
{
|
||||
return gran;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval(minTimestamp, isEmpty() ? minTimestamp : gran.next(getMaxTimeMillis()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DateTime getMinTime()
|
||||
{
|
||||
return isEmpty() ? null : new DateTime(getMinTimeMillis());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DateTime getMaxTime()
|
||||
{
|
||||
return isEmpty() ? null : new DateTime(getMaxTimeMillis());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimDim getDimension(String dimension)
|
||||
{
|
||||
return isEmpty() ? null : dimValues.get(dimension);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getDimensionIndex(String dimension)
|
||||
{
|
||||
return dimensionOrder.get(dimension);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getMetricNames()
|
||||
{
|
||||
return metricNames;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getMetricIndex(String metricName)
|
||||
{
|
||||
return metricIndexes.get(metricName);
|
||||
|
@ -592,16 +589,19 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||
return aggs[aggOffset].get(bufferHolder.get(), getMetricPosition(rowOffset, aggOffset));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities(String column)
|
||||
{
|
||||
return columnCapabilities.get(column);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ConcurrentNavigableMap<TimeAndDims, Integer> getFacts()
|
||||
{
|
||||
return facts;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ConcurrentNavigableMap<TimeAndDims, Integer> getSubMap(TimeAndDims start, TimeAndDims end)
|
||||
{
|
||||
return facts.subMap(start, end);
|
||||
|
@ -678,11 +678,6 @@ public class OffheapIncrementalIndex implements IncrementalIndex
|
|||
dimensions = Maps.newConcurrentMap();
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
dimensions.clear();
|
||||
}
|
||||
|
||||
DimDim add(String 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.Lists;
|
||||
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.ISE;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
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.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
|
@ -161,14 +155,10 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||
columnCapabilities.put(spatialDimension.getDimName(), capabilities);
|
||||
}
|
||||
this.dimValues = new DimensionHolder();
|
||||
this.facts = createFactsTable();
|
||||
this.facts = new ConcurrentSkipListMap<>();
|
||||
this.deserializeComplexMetrics = deserializeComplexMetrics;
|
||||
}
|
||||
|
||||
protected ConcurrentNavigableMap<TimeAndDims, Integer> createFactsTable() {
|
||||
return new ConcurrentSkipListMap<>();
|
||||
}
|
||||
|
||||
public OnheapIncrementalIndex(
|
||||
long minTimestamp,
|
||||
QueryGranularity gran,
|
||||
|
@ -207,6 +197,7 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRow formatRow(InputRow row)
|
||||
{
|
||||
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
|
||||
*/
|
||||
@Override
|
||||
public int add(InputRow row)
|
||||
{
|
||||
row = formatRow(row);
|
||||
|
@ -459,11 +451,13 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||
return numEntries.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty()
|
||||
{
|
||||
return numEntries.get() == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return numEntries.get();
|
||||
|
@ -487,12 +481,12 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||
return aggList.get(rowOffset)[aggOffset].get();
|
||||
}
|
||||
|
||||
public long getMinTimeMillis()
|
||||
private long getMinTimeMillis()
|
||||
{
|
||||
return facts.firstKey().getTimestamp();
|
||||
}
|
||||
|
||||
public long getMaxTimeMillis()
|
||||
private long getMaxTimeMillis()
|
||||
{
|
||||
return facts.lastKey().getTimestamp();
|
||||
}
|
||||
|
@ -515,31 +509,24 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory[] getMetricAggs()
|
||||
{
|
||||
return metrics;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDimensions()
|
||||
{
|
||||
return dimensions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getMetricType(String metric)
|
||||
{
|
||||
return metricTypes.get(metric);
|
||||
}
|
||||
|
||||
public long getMinTimestamp()
|
||||
{
|
||||
return minTimestamp;
|
||||
}
|
||||
|
||||
public QueryGranularity getGranularity()
|
||||
{
|
||||
return gran;
|
||||
}
|
||||
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval(minTimestamp, isEmpty() ? minTimestamp : gran.next(getMaxTimeMillis()));
|
||||
|
@ -575,11 +562,6 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||
return metricIndexes.get(metricName);
|
||||
}
|
||||
|
||||
Aggregator getAggregator(int rowOffset, int metricIndex)
|
||||
{
|
||||
return aggList.get(rowOffset)[metricIndex];
|
||||
}
|
||||
|
||||
public ColumnCapabilities getCapabilities(String column)
|
||||
{
|
||||
return columnCapabilities.get(column);
|
||||
|
@ -601,6 +583,7 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||
return iterableWithPostAggregations(null).iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Row> iterableWithPostAggregations(final List<PostAggregator> postAggs)
|
||||
{
|
||||
return new Iterable<Row>()
|
||||
|
@ -649,6 +632,7 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||
@Override
|
||||
public void close()
|
||||
{
|
||||
// Nothing to close
|
||||
}
|
||||
|
||||
class DimensionHolder
|
||||
|
@ -660,11 +644,6 @@ public class OnheapIncrementalIndex implements IncrementalIndex
|
|||
dimensions = Maps.newConcurrentMap();
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
dimensions.clear();
|
||||
}
|
||||
|
||||
DimDim add(String 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<Integer, String> falseIdsReverse;
|
||||
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()`
|
||||
*
|
||||
* @see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String)
|
||||
*/
|
||||
public String get(String str)
|
||||
|
|
Loading…
Reference in New Issue