mirror of https://github.com/apache/druid.git
configure buffer sizes
This commit is contained in:
parent
5bdc4a761a
commit
a7e19ad892
|
@ -617,7 +617,11 @@ public class IndexGeneratorJob implements Jobby
|
||||||
|
|
||||||
private IncrementalIndex makeIncrementalIndex(Bucket theBucket, AggregatorFactory[] aggs)
|
private IncrementalIndex makeIncrementalIndex(Bucket theBucket, AggregatorFactory[] aggs)
|
||||||
{
|
{
|
||||||
//TODO: review this, add a config for batch ingestion
|
int aggsSize = 0;
|
||||||
|
for (AggregatorFactory agg : aggs) {
|
||||||
|
aggsSize += agg.getMaxIntermediateSize();
|
||||||
|
}
|
||||||
|
int bufferSize = aggsSize * config.getSchema().getTuningConfig().getRowFlushBoundary();
|
||||||
return new IncrementalIndex(
|
return new IncrementalIndex(
|
||||||
new IncrementalIndexSchema.Builder()
|
new IncrementalIndexSchema.Builder()
|
||||||
.withMinTimestamp(theBucket.time.getMillis())
|
.withMinTimestamp(theBucket.time.getMillis())
|
||||||
|
@ -625,7 +629,7 @@ public class IndexGeneratorJob implements Jobby
|
||||||
.withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity())
|
.withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity())
|
||||||
.withMetrics(aggs)
|
.withMetrics(aggs)
|
||||||
.build(),
|
.build(),
|
||||||
new OffheapBufferPool(1024 * 1024 * 1024)
|
new OffheapBufferPool(bufferSize)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -75,7 +75,6 @@ public class TaskToolbox
|
||||||
private final SegmentLoader segmentLoader;
|
private final SegmentLoader segmentLoader;
|
||||||
private final ObjectMapper objectMapper;
|
private final ObjectMapper objectMapper;
|
||||||
private final File taskWorkDir;
|
private final File taskWorkDir;
|
||||||
private final StupidPool<ByteBuffer> indexPool;
|
|
||||||
|
|
||||||
public TaskToolbox(
|
public TaskToolbox(
|
||||||
TaskConfig config,
|
TaskConfig config,
|
||||||
|
@ -93,8 +92,7 @@ public class TaskToolbox
|
||||||
MonitorScheduler monitorScheduler,
|
MonitorScheduler monitorScheduler,
|
||||||
SegmentLoader segmentLoader,
|
SegmentLoader segmentLoader,
|
||||||
ObjectMapper objectMapper,
|
ObjectMapper objectMapper,
|
||||||
final File taskWorkDir,
|
final File taskWorkDir
|
||||||
StupidPool<ByteBuffer> indexPool
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.config = config;
|
this.config = config;
|
||||||
|
@ -113,7 +111,6 @@ public class TaskToolbox
|
||||||
this.segmentLoader = segmentLoader;
|
this.segmentLoader = segmentLoader;
|
||||||
this.objectMapper = objectMapper;
|
this.objectMapper = objectMapper;
|
||||||
this.taskWorkDir = taskWorkDir;
|
this.taskWorkDir = taskWorkDir;
|
||||||
this.indexPool = indexPool;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public TaskConfig getConfig()
|
public TaskConfig getConfig()
|
||||||
|
@ -216,7 +213,4 @@ public class TaskToolbox
|
||||||
return taskWorkDir;
|
return taskWorkDir;
|
||||||
}
|
}
|
||||||
|
|
||||||
public StupidPool<ByteBuffer> getIndexPool(){
|
|
||||||
return indexPool;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -60,7 +60,6 @@ public class TaskToolboxFactory
|
||||||
private final MonitorScheduler monitorScheduler;
|
private final MonitorScheduler monitorScheduler;
|
||||||
private final SegmentLoaderFactory segmentLoaderFactory;
|
private final SegmentLoaderFactory segmentLoaderFactory;
|
||||||
private final ObjectMapper objectMapper;
|
private final ObjectMapper objectMapper;
|
||||||
private final StupidPool<ByteBuffer> bufferPool;
|
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public TaskToolboxFactory(
|
public TaskToolboxFactory(
|
||||||
|
@ -96,7 +95,6 @@ public class TaskToolboxFactory
|
||||||
this.monitorScheduler = monitorScheduler;
|
this.monitorScheduler = monitorScheduler;
|
||||||
this.segmentLoaderFactory = segmentLoaderFactory;
|
this.segmentLoaderFactory = segmentLoaderFactory;
|
||||||
this.objectMapper = objectMapper;
|
this.objectMapper = objectMapper;
|
||||||
this.bufferPool = bufferPool;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public TaskToolbox build(Task task)
|
public TaskToolbox build(Task task)
|
||||||
|
@ -119,8 +117,7 @@ public class TaskToolboxFactory
|
||||||
monitorScheduler,
|
monitorScheduler,
|
||||||
segmentLoaderFactory.manufacturate(taskWorkDir),
|
segmentLoaderFactory.manufacturate(taskWorkDir),
|
||||||
objectMapper,
|
objectMapper,
|
||||||
taskWorkDir,
|
taskWorkDir
|
||||||
bufferPool
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -70,7 +70,6 @@ public class YeOldePlumberSchool implements PlumberSchool
|
||||||
private final String version;
|
private final String version;
|
||||||
private final DataSegmentPusher dataSegmentPusher;
|
private final DataSegmentPusher dataSegmentPusher;
|
||||||
private final File tmpSegmentDir;
|
private final File tmpSegmentDir;
|
||||||
private final StupidPool<ByteBuffer> bufferPool;
|
|
||||||
|
|
||||||
private static final Logger log = new Logger(YeOldePlumberSchool.class);
|
private static final Logger log = new Logger(YeOldePlumberSchool.class);
|
||||||
|
|
||||||
|
@ -79,16 +78,13 @@ public class YeOldePlumberSchool implements PlumberSchool
|
||||||
@JsonProperty("interval") Interval interval,
|
@JsonProperty("interval") Interval interval,
|
||||||
@JsonProperty("version") String version,
|
@JsonProperty("version") String version,
|
||||||
@JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher,
|
@JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher,
|
||||||
@JacksonInject("tmpSegmentDir") File tmpSegmentDir,
|
@JacksonInject("tmpSegmentDir") File tmpSegmentDir
|
||||||
//TODO: review this global annotation
|
|
||||||
@JacksonInject @Global StupidPool<ByteBuffer> bufferPool
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.interval = interval;
|
this.interval = interval;
|
||||||
this.version = version;
|
this.version = version;
|
||||||
this.dataSegmentPusher = dataSegmentPusher;
|
this.dataSegmentPusher = dataSegmentPusher;
|
||||||
this.tmpSegmentDir = tmpSegmentDir;
|
this.tmpSegmentDir = tmpSegmentDir;
|
||||||
this.bufferPool = bufferPool;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -105,7 +101,7 @@ public class YeOldePlumberSchool implements PlumberSchool
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
// There can be only one.
|
// There can be only one.
|
||||||
final Sink theSink = new Sink(interval, schema, config, version, bufferPool);
|
final Sink theSink = new Sink(interval, schema, config, version);
|
||||||
|
|
||||||
// Temporary directory to hold spilled segments.
|
// Temporary directory to hold spilled segments.
|
||||||
final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getIdentifier());
|
final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getIdentifier());
|
||||||
|
|
|
@ -32,6 +32,7 @@ import io.druid.indexing.common.TaskLock;
|
||||||
import io.druid.indexing.common.TaskStatus;
|
import io.druid.indexing.common.TaskStatus;
|
||||||
import io.druid.indexing.common.TaskToolbox;
|
import io.druid.indexing.common.TaskToolbox;
|
||||||
import io.druid.indexing.common.actions.SegmentInsertAction;
|
import io.druid.indexing.common.actions.SegmentInsertAction;
|
||||||
|
import io.druid.offheap.OffheapBufferPool;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.segment.IndexMerger;
|
import io.druid.segment.IndexMerger;
|
||||||
import io.druid.segment.IndexableAdapter;
|
import io.druid.segment.IndexableAdapter;
|
||||||
|
@ -83,7 +84,7 @@ public class DeleteTask extends AbstractFixedIntervalTask
|
||||||
0,
|
0,
|
||||||
QueryGranularity.NONE,
|
QueryGranularity.NONE,
|
||||||
new AggregatorFactory[0],
|
new AggregatorFactory[0],
|
||||||
toolbox.getIndexPool()
|
new OffheapBufferPool(0)
|
||||||
);
|
);
|
||||||
try {
|
try {
|
||||||
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(getInterval(), empty);
|
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(getInterval(), empty);
|
||||||
|
|
|
@ -385,8 +385,7 @@ public class IndexTask extends AbstractFixedIntervalTask
|
||||||
interval,
|
interval,
|
||||||
version,
|
version,
|
||||||
wrappedDataSegmentPusher,
|
wrappedDataSegmentPusher,
|
||||||
tmpDir,
|
tmpDir
|
||||||
toolbox.getIndexPool()
|
|
||||||
).findPlumber(schema, new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec), metrics);
|
).findPlumber(schema, new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec), metrics);
|
||||||
|
|
||||||
// rowFlushBoundary for this job
|
// rowFlushBoundary for this job
|
||||||
|
|
|
@ -308,7 +308,6 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
segmentPublisher,
|
segmentPublisher,
|
||||||
toolbox.getNewSegmentServerView(),
|
toolbox.getNewSegmentServerView(),
|
||||||
toolbox.getQueryExecutorService(),
|
toolbox.getQueryExecutorService(),
|
||||||
toolbox.getIndexPool(),
|
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -54,20 +54,21 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
|
||||||
private final GroupByQueryEngine engine;
|
private final GroupByQueryEngine engine;
|
||||||
private final Supplier<GroupByQueryConfig> config;
|
private final Supplier<GroupByQueryConfig> config;
|
||||||
private final GroupByQueryQueryToolChest toolChest;
|
private final GroupByQueryQueryToolChest toolChest;
|
||||||
private final StupidPool<ByteBuffer> bufferPool;
|
@Global
|
||||||
|
StupidPool<ByteBuffer> computationBufferPool;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public GroupByQueryRunnerFactory(
|
public GroupByQueryRunnerFactory(
|
||||||
GroupByQueryEngine engine,
|
GroupByQueryEngine engine,
|
||||||
Supplier<GroupByQueryConfig> config,
|
Supplier<GroupByQueryConfig> config,
|
||||||
GroupByQueryQueryToolChest toolChest,
|
GroupByQueryQueryToolChest toolChest,
|
||||||
@Global StupidPool<ByteBuffer> bufferPool
|
@Global StupidPool<ByteBuffer> computationBufferPool
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.engine = engine;
|
this.engine = engine;
|
||||||
this.config = config;
|
this.config = config;
|
||||||
this.toolChest = toolChest;
|
this.toolChest = toolChest;
|
||||||
this.bufferPool = bufferPool;
|
this.computationBufferPool = computationBufferPool;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -123,7 +124,13 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
return new GroupByParallelQueryRunner(queryExecutor, new RowOrdering(), config, bufferPool, queryRunners);
|
return new GroupByParallelQueryRunner(
|
||||||
|
queryExecutor,
|
||||||
|
new RowOrdering(),
|
||||||
|
config,
|
||||||
|
computationBufferPool,
|
||||||
|
queryRunners
|
||||||
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -58,7 +58,6 @@ public class GreaterThanHavingSpec implements HavingSpec
|
||||||
public boolean eval(Row row)
|
public boolean eval(Row row)
|
||||||
{
|
{
|
||||||
float metricValue = row.getFloatMetric(aggregationName);
|
float metricValue = row.getFloatMetric(aggregationName);
|
||||||
|
|
||||||
return Float.compare(metricValue, value.floatValue()) > 0;
|
return Float.compare(metricValue, value.floatValue()) > 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -297,6 +297,9 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
if (!facts.containsKey(key)) {
|
if (!facts.containsKey(key)) {
|
||||||
int rowOffset = totalAggSize * numEntries.getAndIncrement();
|
int rowOffset = totalAggSize * numEntries.getAndIncrement();
|
||||||
|
if (rowOffset + totalAggSize > bufferHolder.get().limit()) {
|
||||||
|
throw new ISE("Buffer Full cannot add more rows current rowSize : %d", numEntries.get());
|
||||||
|
}
|
||||||
for (int i = 0; i < aggs.length; i++) {
|
for (int i = 0; i < aggs.length; i++) {
|
||||||
aggs[i].init(bufferHolder.get(), getMetricPosition(rowOffset, i));
|
aggs[i].init(bufferHolder.get(), getMetricPosition(rowOffset, i));
|
||||||
}
|
}
|
||||||
|
|
|
@ -64,8 +64,7 @@ public class FlushingPlumber extends RealtimePlumber
|
||||||
ServiceEmitter emitter,
|
ServiceEmitter emitter,
|
||||||
QueryRunnerFactoryConglomerate conglomerate,
|
QueryRunnerFactoryConglomerate conglomerate,
|
||||||
DataSegmentAnnouncer segmentAnnouncer,
|
DataSegmentAnnouncer segmentAnnouncer,
|
||||||
ExecutorService queryExecutorService,
|
ExecutorService queryExecutorService
|
||||||
StupidPool<ByteBuffer> bufferPool
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(
|
super(
|
||||||
|
@ -78,8 +77,7 @@ public class FlushingPlumber extends RealtimePlumber
|
||||||
queryExecutorService,
|
queryExecutorService,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null
|
||||||
bufferPool
|
|
||||||
);
|
);
|
||||||
|
|
||||||
this.flushDuration = flushDuration;
|
this.flushDuration = flushDuration;
|
||||||
|
|
|
@ -54,7 +54,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
|
||||||
private final QueryRunnerFactoryConglomerate conglomerate;
|
private final QueryRunnerFactoryConglomerate conglomerate;
|
||||||
private final DataSegmentAnnouncer segmentAnnouncer;
|
private final DataSegmentAnnouncer segmentAnnouncer;
|
||||||
private final ExecutorService queryExecutorService;
|
private final ExecutorService queryExecutorService;
|
||||||
private final StupidPool<ByteBuffer> bufferPool;
|
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public FlushingPlumberSchool(
|
public FlushingPlumberSchool(
|
||||||
|
@ -63,8 +62,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
|
||||||
@JacksonInject QueryRunnerFactoryConglomerate conglomerate,
|
@JacksonInject QueryRunnerFactoryConglomerate conglomerate,
|
||||||
@JacksonInject DataSegmentAnnouncer segmentAnnouncer,
|
@JacksonInject DataSegmentAnnouncer segmentAnnouncer,
|
||||||
@JacksonInject @Processing ExecutorService queryExecutorService,
|
@JacksonInject @Processing ExecutorService queryExecutorService,
|
||||||
//TODO: define separate index pool
|
|
||||||
@JacksonInject @Global StupidPool<ByteBuffer> bufferPool,
|
|
||||||
// Backwards compatible
|
// Backwards compatible
|
||||||
@JsonProperty("windowPeriod") Period windowPeriod,
|
@JsonProperty("windowPeriod") Period windowPeriod,
|
||||||
@JsonProperty("basePersistDirectory") File basePersistDirectory,
|
@JsonProperty("basePersistDirectory") File basePersistDirectory,
|
||||||
|
@ -82,7 +79,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
queryExecutorService,
|
queryExecutorService,
|
||||||
bufferPool,
|
|
||||||
windowPeriod,
|
windowPeriod,
|
||||||
basePersistDirectory,
|
basePersistDirectory,
|
||||||
segmentGranularity,
|
segmentGranularity,
|
||||||
|
@ -96,7 +92,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
|
||||||
this.conglomerate = conglomerate;
|
this.conglomerate = conglomerate;
|
||||||
this.segmentAnnouncer = segmentAnnouncer;
|
this.segmentAnnouncer = segmentAnnouncer;
|
||||||
this.queryExecutorService = queryExecutorService;
|
this.queryExecutorService = queryExecutorService;
|
||||||
this.bufferPool = bufferPool;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -116,8 +111,7 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
|
||||||
emitter,
|
emitter,
|
||||||
conglomerate,
|
conglomerate,
|
||||||
segmentAnnouncer,
|
segmentAnnouncer,
|
||||||
queryExecutorService,
|
queryExecutorService
|
||||||
bufferPool
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -99,7 +99,6 @@ public class RealtimePlumber implements Plumber
|
||||||
private volatile ExecutorService persistExecutor = null;
|
private volatile ExecutorService persistExecutor = null;
|
||||||
private volatile ExecutorService mergeExecutor = null;
|
private volatile ExecutorService mergeExecutor = null;
|
||||||
private volatile ScheduledExecutorService scheduledExecutor = null;
|
private volatile ScheduledExecutorService scheduledExecutor = null;
|
||||||
private final StupidPool<ByteBuffer> bufferPool;
|
|
||||||
|
|
||||||
|
|
||||||
public RealtimePlumber(
|
public RealtimePlumber(
|
||||||
|
@ -112,9 +111,7 @@ public class RealtimePlumber implements Plumber
|
||||||
ExecutorService queryExecutorService,
|
ExecutorService queryExecutorService,
|
||||||
DataSegmentPusher dataSegmentPusher,
|
DataSegmentPusher dataSegmentPusher,
|
||||||
SegmentPublisher segmentPublisher,
|
SegmentPublisher segmentPublisher,
|
||||||
FilteredServerView serverView,
|
FilteredServerView serverView
|
||||||
StupidPool<ByteBuffer> bufferPool
|
|
||||||
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.schema = schema;
|
this.schema = schema;
|
||||||
|
@ -128,7 +125,6 @@ public class RealtimePlumber implements Plumber
|
||||||
this.dataSegmentPusher = dataSegmentPusher;
|
this.dataSegmentPusher = dataSegmentPusher;
|
||||||
this.segmentPublisher = segmentPublisher;
|
this.segmentPublisher = segmentPublisher;
|
||||||
this.serverView = serverView;
|
this.serverView = serverView;
|
||||||
this.bufferPool = bufferPool;
|
|
||||||
|
|
||||||
log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy());
|
log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy());
|
||||||
}
|
}
|
||||||
|
@ -193,7 +189,7 @@ public class RealtimePlumber implements Plumber
|
||||||
segmentGranularity.increment(new DateTime(truncatedTime))
|
segmentGranularity.increment(new DateTime(truncatedTime))
|
||||||
);
|
);
|
||||||
|
|
||||||
retVal = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval), bufferPool);
|
retVal = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval));
|
||||||
|
|
||||||
try {
|
try {
|
||||||
segmentAnnouncer.announceSegment(retVal.getSegment());
|
segmentAnnouncer.announceSegment(retVal.getSegment());
|
||||||
|
@ -544,7 +540,7 @@ public class RealtimePlumber implements Plumber
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
Sink currSink = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval), hydrants, bufferPool);
|
Sink currSink = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval), hydrants);
|
||||||
sinks.put(sinkInterval.getStartMillis(), currSink);
|
sinks.put(sinkInterval.getStartMillis(), currSink);
|
||||||
sinkTimeline.add(
|
sinkTimeline.add(
|
||||||
currSink.getInterval(),
|
currSink.getInterval(),
|
||||||
|
|
|
@ -23,14 +23,9 @@ import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Supplier;
|
|
||||||
import com.metamx.common.Granularity;
|
import com.metamx.common.Granularity;
|
||||||
import com.metamx.common.logger.Logger;
|
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import io.druid.client.FilteredServerView;
|
import io.druid.client.FilteredServerView;
|
||||||
import io.druid.client.ServerView;
|
|
||||||
import io.druid.collections.StupidPool;
|
|
||||||
import io.druid.guice.annotations.Global;
|
|
||||||
import io.druid.guice.annotations.Processing;
|
import io.druid.guice.annotations.Processing;
|
||||||
import io.druid.query.QueryRunnerFactoryConglomerate;
|
import io.druid.query.QueryRunnerFactoryConglomerate;
|
||||||
import io.druid.segment.indexing.DataSchema;
|
import io.druid.segment.indexing.DataSchema;
|
||||||
|
@ -42,9 +37,7 @@ import io.druid.server.coordination.DataSegmentAnnouncer;
|
||||||
import org.joda.time.Period;
|
import org.joda.time.Period;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.nio.ByteBuffer;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -57,8 +50,6 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
private final SegmentPublisher segmentPublisher;
|
private final SegmentPublisher segmentPublisher;
|
||||||
private final FilteredServerView serverView;
|
private final FilteredServerView serverView;
|
||||||
private final ExecutorService queryExecutorService;
|
private final ExecutorService queryExecutorService;
|
||||||
private final StupidPool<ByteBuffer> bufferPool;
|
|
||||||
|
|
||||||
// Backwards compatible
|
// Backwards compatible
|
||||||
private final Period windowPeriod;
|
private final Period windowPeriod;
|
||||||
private final File basePersistDirectory;
|
private final File basePersistDirectory;
|
||||||
|
@ -76,8 +67,6 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
@JacksonInject SegmentPublisher segmentPublisher,
|
@JacksonInject SegmentPublisher segmentPublisher,
|
||||||
@JacksonInject FilteredServerView serverView,
|
@JacksonInject FilteredServerView serverView,
|
||||||
@JacksonInject @Processing ExecutorService executorService,
|
@JacksonInject @Processing ExecutorService executorService,
|
||||||
//TODO: define separate index pool
|
|
||||||
@JacksonInject @Global StupidPool<ByteBuffer> bufferPool,
|
|
||||||
// Backwards compatible
|
// Backwards compatible
|
||||||
@JsonProperty("windowPeriod") Period windowPeriod,
|
@JsonProperty("windowPeriod") Period windowPeriod,
|
||||||
@JsonProperty("basePersistDirectory") File basePersistDirectory,
|
@JsonProperty("basePersistDirectory") File basePersistDirectory,
|
||||||
|
@ -100,7 +89,6 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
this.versioningPolicy = versioningPolicy;
|
this.versioningPolicy = versioningPolicy;
|
||||||
this.rejectionPolicyFactory = rejectionPolicyFactory;
|
this.rejectionPolicyFactory = rejectionPolicyFactory;
|
||||||
this.maxPendingPersists = maxPendingPersists;
|
this.maxPendingPersists = maxPendingPersists;
|
||||||
this.bufferPool = bufferPool;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Deprecated
|
@Deprecated
|
||||||
|
@ -159,8 +147,7 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
queryExecutorService,
|
queryExecutorService,
|
||||||
dataSegmentPusher,
|
dataSegmentPusher,
|
||||||
segmentPublisher,
|
segmentPublisher,
|
||||||
serverView,
|
serverView
|
||||||
bufferPool
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -29,8 +29,7 @@ import com.metamx.common.ISE;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import io.druid.collections.StupidPool;
|
import io.druid.collections.StupidPool;
|
||||||
import io.druid.data.input.InputRow;
|
import io.druid.data.input.InputRow;
|
||||||
import io.druid.data.input.impl.SpatialDimensionSchema;
|
import io.druid.offheap.OffheapBufferPool;
|
||||||
import io.druid.guice.annotations.Global;
|
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
|
@ -52,31 +51,25 @@ import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
public class Sink implements Iterable<FireHydrant>
|
public class Sink implements Iterable<FireHydrant>
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(Sink.class);
|
private static final Logger log = new Logger(Sink.class);
|
||||||
|
|
||||||
private volatile FireHydrant currHydrant;
|
|
||||||
|
|
||||||
private final Interval interval;
|
private final Interval interval;
|
||||||
private final DataSchema schema;
|
private final DataSchema schema;
|
||||||
private final RealtimeTuningConfig config;
|
private final RealtimeTuningConfig config;
|
||||||
private final String version;
|
private final String version;
|
||||||
private final CopyOnWriteArrayList<FireHydrant> hydrants = new CopyOnWriteArrayList<FireHydrant>();
|
private final CopyOnWriteArrayList<FireHydrant> hydrants = new CopyOnWriteArrayList<FireHydrant>();
|
||||||
private final StupidPool<ByteBuffer> bufferPool;
|
private volatile FireHydrant currHydrant;
|
||||||
|
|
||||||
|
|
||||||
public Sink(
|
public Sink(
|
||||||
Interval interval,
|
Interval interval,
|
||||||
DataSchema schema,
|
DataSchema schema,
|
||||||
RealtimeTuningConfig config,
|
RealtimeTuningConfig config,
|
||||||
String version,
|
String version
|
||||||
StupidPool<ByteBuffer> bufferPool
|
|
||||||
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.schema = schema;
|
this.schema = schema;
|
||||||
this.config = config;
|
this.config = config;
|
||||||
this.interval = interval;
|
this.interval = interval;
|
||||||
this.version = version;
|
this.version = version;
|
||||||
this.bufferPool = bufferPool;
|
|
||||||
|
|
||||||
makeNewCurrIndex(interval.getStartMillis(), schema);
|
makeNewCurrIndex(interval.getStartMillis(), schema);
|
||||||
}
|
}
|
||||||
|
@ -86,15 +79,13 @@ public class Sink implements Iterable<FireHydrant>
|
||||||
DataSchema schema,
|
DataSchema schema,
|
||||||
RealtimeTuningConfig config,
|
RealtimeTuningConfig config,
|
||||||
String version,
|
String version,
|
||||||
List<FireHydrant> hydrants,
|
List<FireHydrant> hydrants
|
||||||
StupidPool<ByteBuffer> bufferPool
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.schema = schema;
|
this.schema = schema;
|
||||||
this.config = config;
|
this.config = config;
|
||||||
this.interval = interval;
|
this.interval = interval;
|
||||||
this.version = version;
|
this.version = version;
|
||||||
this.bufferPool = bufferPool;
|
|
||||||
|
|
||||||
for (int i = 0; i < hydrants.size(); ++i) {
|
for (int i = 0; i < hydrants.size(); ++i) {
|
||||||
final FireHydrant hydrant = hydrants.get(i);
|
final FireHydrant hydrant = hydrants.get(i);
|
||||||
|
@ -187,6 +178,11 @@ public class Sink implements Iterable<FireHydrant>
|
||||||
|
|
||||||
private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema)
|
private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema)
|
||||||
{
|
{
|
||||||
|
int aggsSize = 0;
|
||||||
|
for (AggregatorFactory agg : schema.getAggregators()) {
|
||||||
|
aggsSize += agg.getMaxIntermediateSize();
|
||||||
|
}
|
||||||
|
int bufferSize = aggsSize * config.getMaxRowsInMemory();
|
||||||
IncrementalIndex newIndex = new IncrementalIndex(
|
IncrementalIndex newIndex = new IncrementalIndex(
|
||||||
new IncrementalIndexSchema.Builder()
|
new IncrementalIndexSchema.Builder()
|
||||||
.withMinTimestamp(minTimestamp)
|
.withMinTimestamp(minTimestamp)
|
||||||
|
@ -194,7 +190,7 @@ public class Sink implements Iterable<FireHydrant>
|
||||||
.withSpatialDimensions(schema.getParser())
|
.withSpatialDimensions(schema.getParser())
|
||||||
.withMetrics(schema.getAggregators())
|
.withMetrics(schema.getAggregators())
|
||||||
.build(),
|
.build(),
|
||||||
bufferPool
|
new OffheapBufferPool(bufferSize)
|
||||||
);
|
);
|
||||||
|
|
||||||
FireHydrant old;
|
FireHydrant old;
|
||||||
|
|
|
@ -74,7 +74,7 @@ public class FireDepartmentTest
|
||||||
new RealtimeIOConfig(
|
new RealtimeIOConfig(
|
||||||
null,
|
null,
|
||||||
new RealtimePlumberSchool(
|
new RealtimePlumberSchool(
|
||||||
null, null, null, null, null, null, null, TestQueryRunners.pool, null, null, null, null, null, 0
|
null, null, null, null, null, null, null, null, null, null, null, null, 0
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
new RealtimeTuningConfig(
|
new RealtimeTuningConfig(
|
||||||
|
|
|
@ -118,7 +118,7 @@ public class RealtimeManagerTest
|
||||||
null,
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
plumber = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema, tuningConfig, new DateTime().toString(), TestQueryRunners.pool));
|
plumber = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema, tuningConfig, new DateTime().toString()));
|
||||||
|
|
||||||
realtimeManager = new RealtimeManager(
|
realtimeManager = new RealtimeManager(
|
||||||
Arrays.<FireDepartment>asList(
|
Arrays.<FireDepartment>asList(
|
||||||
|
|
|
@ -145,7 +145,6 @@ public class RealtimePlumberSchoolTest
|
||||||
segmentPublisher,
|
segmentPublisher,
|
||||||
serverView,
|
serverView,
|
||||||
MoreExecutors.sameThreadExecutor(),
|
MoreExecutors.sameThreadExecutor(),
|
||||||
TestQueryRunners.pool,
|
|
||||||
new Period("PT10m"),
|
new Period("PT10m"),
|
||||||
tmpDir,
|
tmpDir,
|
||||||
Granularity.HOUR,
|
Granularity.HOUR,
|
||||||
|
|
|
@ -56,7 +56,7 @@ public class SinkTest
|
||||||
final Interval interval = new Interval("2013-01-01/2013-01-02");
|
final Interval interval = new Interval("2013-01-01/2013-01-02");
|
||||||
final String version = new DateTime().toString();
|
final String version = new DateTime().toString();
|
||||||
RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig(
|
RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig(
|
||||||
1,
|
100,
|
||||||
new Period("P1Y"),
|
new Period("P1Y"),
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
@ -65,7 +65,7 @@ public class SinkTest
|
||||||
null,
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
final Sink sink = new Sink(interval, schema, tuningConfig, version, TestQueryRunners.pool);
|
final Sink sink = new Sink(interval, schema, tuningConfig, version);
|
||||||
|
|
||||||
sink.add(
|
sink.add(
|
||||||
new InputRow()
|
new InputRow()
|
||||||
|
|
Loading…
Reference in New Issue