remove ThreadSafeAggregator

This commit is contained in:
nishantmonu51 2014-08-21 23:56:07 +05:30
parent 67f4bbae74
commit c216eb7340
1 changed files with 111 additions and 158 deletions

View File

@ -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();
}
}
}