mirror of https://github.com/apache/druid.git
remove ThreadSafeAggregator
This commit is contained in:
parent
67f4bbae74
commit
c216eb7340
|
@ -80,9 +80,7 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
{
|
||||
private final long minTimestamp;
|
||||
private final QueryGranularity gran;
|
||||
|
||||
private final List<Function<InputRow, InputRow>> rowTransformers;
|
||||
|
||||
private final AggregatorFactory[] metrics;
|
||||
private final Map<String, Integer> metricIndexes;
|
||||
private final Map<String, String> metricTypes;
|
||||
|
@ -93,9 +91,7 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
private final LinkedHashMap<String, Integer> dimensionOrder;
|
||||
private final CopyOnWriteArrayList<String> dimensions;
|
||||
private final DimensionHolder dimValues;
|
||||
|
||||
private final Map<String, ColumnCapabilitiesImpl> columnCapabilities;
|
||||
|
||||
private final ConcurrentSkipListMap<TimeAndDims, Integer> facts;
|
||||
private final ResourceHolder<ByteBuffer> bufferHolder;
|
||||
private volatile AtomicInteger numEntries = new AtomicInteger();
|
||||
|
@ -130,8 +126,7 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
int currAggSize = 0;
|
||||
for (int i = 0; i < metrics.length; i++) {
|
||||
final AggregatorFactory agg = metrics[i];
|
||||
aggs[i] = new ThreadSafeAggregator(
|
||||
agg.factorizeBuffered(
|
||||
aggs[i] = agg.factorizeBuffered(
|
||||
new ColumnSelectorFactory()
|
||||
{
|
||||
@Override
|
||||
|
@ -271,7 +266,6 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
};
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
aggPositionOffsets[i] = currAggSize;
|
||||
currAggSize += agg.getMaxIntermediateSize();
|
||||
|
@ -861,45 +855,4 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class ThreadSafeAggregator implements BufferAggregator
|
||||
{
|
||||
|
||||
private final BufferAggregator delegate;
|
||||
|
||||
public ThreadSafeAggregator(BufferAggregator delegate)
|
||||
{
|
||||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void init(ByteBuffer buf, int position)
|
||||
{
|
||||
delegate.init(buf, position);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
delegate.aggregate(buf, position);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Object get(ByteBuffer buf, int position)
|
||||
{
|
||||
return delegate.get(buf, position);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized float getFloat(ByteBuffer buf, int position)
|
||||
{
|
||||
return delegate.getFloat(buf, position);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close()
|
||||
{
|
||||
delegate.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue