mirror of https://github.com/apache/druid.git
Merge pull request #549 from metamx/fix-sink
fix race condition with merge and persist and sink adding
This commit is contained in:
commit
e4db0728f2
|
@ -31,6 +31,7 @@ import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.metamx.common.Granularity;
|
import com.metamx.common.Granularity;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
|
import io.druid.data.input.InputRow;
|
||||||
import io.druid.query.Query;
|
import io.druid.query.Query;
|
||||||
import io.druid.query.QueryRunner;
|
import io.druid.query.QueryRunner;
|
||||||
import io.druid.segment.IndexIO;
|
import io.druid.segment.IndexIO;
|
||||||
|
@ -112,6 +113,16 @@ public class YeOldePlumberSchool implements PlumberSchool
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
public int add(InputRow row)
|
||||||
|
{
|
||||||
|
Sink sink = getSink(row.getTimestampFromEpoch());
|
||||||
|
if (sink == null) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
return sink.add(row);
|
||||||
|
}
|
||||||
|
|
||||||
public Sink getSink(long timestamp)
|
public Sink getSink(long timestamp)
|
||||||
{
|
{
|
||||||
if (theSink.getInterval().contains(timestamp)) {
|
if (theSink.getInterval().contains(timestamp)) {
|
||||||
|
|
|
@ -398,17 +398,15 @@ public class IndexTask extends AbstractFixedIntervalTask
|
||||||
final InputRow inputRow = firehose.nextRow();
|
final InputRow inputRow = firehose.nextRow();
|
||||||
|
|
||||||
if (shouldIndex(shardSpec, interval, inputRow)) {
|
if (shouldIndex(shardSpec, interval, inputRow)) {
|
||||||
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
|
int numRows = plumber.add(inputRow);
|
||||||
if (sink == null) {
|
if (numRows == -1) {
|
||||||
throw new NullPointerException(
|
throw new ISE(
|
||||||
String.format(
|
String.format(
|
||||||
"Was expecting non-null sink for timestamp[%s]",
|
"Was expecting non-null sink for timestamp[%s]",
|
||||||
new DateTime(inputRow.getTimestampFromEpoch())
|
new DateTime(inputRow.getTimestampFromEpoch())
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
int numRows = sink.add(inputRow);
|
|
||||||
metrics.incrementProcessed();
|
metrics.incrementProcessed();
|
||||||
|
|
||||||
if (numRows >= myRowFlushBoundary) {
|
if (numRows >= myRowFlushBoundary) {
|
||||||
|
|
|
@ -335,8 +335,8 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
|
int currCount = plumber.add(inputRow);
|
||||||
if (sink == null) {
|
if (currCount == -1) {
|
||||||
fireDepartment.getMetrics().incrementThrownAway();
|
fireDepartment.getMetrics().incrementThrownAway();
|
||||||
log.debug("Throwing away event[%s]", inputRow);
|
log.debug("Throwing away event[%s]", inputRow);
|
||||||
|
|
||||||
|
@ -348,11 +348,6 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (sink.isEmpty()) {
|
|
||||||
log.info("Task %s: New sink: %s", getId(), sink);
|
|
||||||
}
|
|
||||||
|
|
||||||
int currCount = sink.add(inputRow);
|
|
||||||
fireDepartment.getMetrics().incrementProcessed();
|
fireDepartment.getMetrics().incrementProcessed();
|
||||||
if (currCount >= tuningConfig.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) {
|
if (currCount >= tuningConfig.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) {
|
||||||
plumber.persist(firehose.commit());
|
plumber.persist(firehose.commit());
|
||||||
|
|
|
@ -43,7 +43,6 @@ import io.druid.query.SegmentDescriptor;
|
||||||
import io.druid.segment.indexing.DataSchema;
|
import io.druid.segment.indexing.DataSchema;
|
||||||
import io.druid.segment.indexing.RealtimeTuningConfig;
|
import io.druid.segment.indexing.RealtimeTuningConfig;
|
||||||
import io.druid.segment.realtime.plumber.Plumber;
|
import io.druid.segment.realtime.plumber.Plumber;
|
||||||
import io.druid.segment.realtime.plumber.Sink;
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.joda.time.Period;
|
import org.joda.time.Period;
|
||||||
|
@ -197,8 +196,8 @@ public class RealtimeManager implements QuerySegmentWalker
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
|
int currCount = plumber.add(inputRow);
|
||||||
if (sink == null) {
|
if (currCount == -1) {
|
||||||
metrics.incrementThrownAway();
|
metrics.incrementThrownAway();
|
||||||
log.debug("Throwing away event[%s]", inputRow);
|
log.debug("Throwing away event[%s]", inputRow);
|
||||||
|
|
||||||
|
@ -209,8 +208,6 @@ public class RealtimeManager implements QuerySegmentWalker
|
||||||
|
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
int currCount = sink.add(inputRow);
|
|
||||||
if (currCount >= config.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) {
|
if (currCount >= config.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) {
|
||||||
plumber.persist(firehose.commit());
|
plumber.persist(firehose.commit());
|
||||||
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
||||||
|
@ -225,13 +222,11 @@ public class RealtimeManager implements QuerySegmentWalker
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (RuntimeException e) {
|
catch (RuntimeException e) {
|
||||||
|
|
||||||
log.makeAlert(
|
log.makeAlert(
|
||||||
e,
|
e,
|
||||||
"RuntimeException aborted realtime processing[%s]",
|
"RuntimeException aborted realtime processing[%s]",
|
||||||
fireDepartment.getDataSchema().getDataSource()
|
fireDepartment.getDataSchema().getDataSource()
|
||||||
)
|
).emit();
|
||||||
.emit();
|
|
||||||
normalExit = false;
|
normalExit = false;
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.segment.realtime.plumber;
|
package io.druid.segment.realtime.plumber;
|
||||||
|
|
||||||
|
import io.druid.data.input.InputRow;
|
||||||
import io.druid.query.Query;
|
import io.druid.query.Query;
|
||||||
import io.druid.query.QueryRunner;
|
import io.druid.query.QueryRunner;
|
||||||
|
|
||||||
|
@ -30,7 +31,12 @@ public interface Plumber
|
||||||
*/
|
*/
|
||||||
public void startJob();
|
public void startJob();
|
||||||
|
|
||||||
public Sink getSink(long timestamp);
|
/**
|
||||||
|
* @param row - the row to insert
|
||||||
|
* @return - positive numbers indicate how many summarized rows exist in the index for that timestamp,
|
||||||
|
* -1 means a row was thrown away because it was too late
|
||||||
|
*/
|
||||||
|
public int add(InputRow row);
|
||||||
public <T> QueryRunner<T> getQueryRunner(Query<T> query);
|
public <T> QueryRunner<T> getQueryRunner(Query<T> query);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -21,6 +21,7 @@ import io.druid.client.ServerView;
|
||||||
import io.druid.common.guava.ThreadRenamingCallable;
|
import io.druid.common.guava.ThreadRenamingCallable;
|
||||||
import io.druid.common.guava.ThreadRenamingRunnable;
|
import io.druid.common.guava.ThreadRenamingRunnable;
|
||||||
import io.druid.concurrent.Execs;
|
import io.druid.concurrent.Execs;
|
||||||
|
import io.druid.data.input.InputRow;
|
||||||
import io.druid.query.MetricsEmittingQueryRunner;
|
import io.druid.query.MetricsEmittingQueryRunner;
|
||||||
import io.druid.query.Query;
|
import io.druid.query.Query;
|
||||||
import io.druid.query.QueryRunner;
|
import io.druid.query.QueryRunner;
|
||||||
|
@ -72,6 +73,7 @@ public class RealtimePlumber implements Plumber
|
||||||
|
|
||||||
private final DataSchema schema;
|
private final DataSchema schema;
|
||||||
private final RealtimeTuningConfig config;
|
private final RealtimeTuningConfig config;
|
||||||
|
|
||||||
private final RejectionPolicy rejectionPolicy;
|
private final RejectionPolicy rejectionPolicy;
|
||||||
private final FireDepartmentMetrics metrics;
|
private final FireDepartmentMetrics metrics;
|
||||||
private final ServiceEmitter emitter;
|
private final ServiceEmitter emitter;
|
||||||
|
@ -151,6 +153,16 @@ public class RealtimePlumber implements Plumber
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
public int add(InputRow row)
|
||||||
|
{
|
||||||
|
final Sink sink = getSink(row.getTimestampFromEpoch());
|
||||||
|
if (sink == null) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
return sink.add(row);
|
||||||
|
}
|
||||||
|
|
||||||
public Sink getSink(long timestamp)
|
public Sink getSink(long timestamp)
|
||||||
{
|
{
|
||||||
if (!rejectionPolicy.accept(timestamp)) {
|
if (!rejectionPolicy.accept(timestamp)) {
|
||||||
|
|
|
@ -50,7 +50,7 @@ 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 currIndex;
|
private volatile FireHydrant currHydrant;
|
||||||
|
|
||||||
private final Interval interval;
|
private final Interval interval;
|
||||||
private final DataSchema schema;
|
private final DataSchema schema;
|
||||||
|
@ -107,31 +107,35 @@ public class Sink implements Iterable<FireHydrant>
|
||||||
return interval;
|
return interval;
|
||||||
}
|
}
|
||||||
|
|
||||||
public FireHydrant getCurrIndex()
|
public FireHydrant getCurrHydrant()
|
||||||
{
|
{
|
||||||
return currIndex;
|
return currHydrant;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int add(InputRow row)
|
public int add(InputRow row)
|
||||||
{
|
{
|
||||||
if (currIndex == null) {
|
if (currHydrant == null) {
|
||||||
throw new IAE("No currIndex but given row[%s]", row);
|
throw new IAE("No currHydrant but given row[%s]", row);
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized (currIndex) {
|
synchronized (currHydrant) {
|
||||||
return currIndex.getIndex().add(row);
|
IncrementalIndex index = currHydrant.getIndex();
|
||||||
|
if (index == null) {
|
||||||
|
return -1; // the hydrant was swapped without being replaced
|
||||||
|
}
|
||||||
|
return index.add(row);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isEmpty()
|
public boolean isEmpty()
|
||||||
{
|
{
|
||||||
synchronized (currIndex) {
|
synchronized (currHydrant) {
|
||||||
return hydrants.size() == 1 && currIndex.getIndex().isEmpty();
|
return hydrants.size() == 1 && currHydrant.getIndex().isEmpty();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* If currIndex is A, creates a new index B, sets currIndex to B and returns A.
|
* If currHydrant is A, creates a new index B, sets currHydrant to B and returns A.
|
||||||
*
|
*
|
||||||
* @return the current index after swapping in a new one
|
* @return the current index after swapping in a new one
|
||||||
*/
|
*/
|
||||||
|
@ -142,8 +146,8 @@ public class Sink implements Iterable<FireHydrant>
|
||||||
|
|
||||||
public boolean swappable()
|
public boolean swappable()
|
||||||
{
|
{
|
||||||
synchronized (currIndex) {
|
synchronized (currHydrant) {
|
||||||
return currIndex.getIndex() != null && currIndex.getIndex().size() != 0;
|
return currHydrant.getIndex() != null && currHydrant.getIndex().size() != 0;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -189,15 +193,15 @@ public class Sink implements Iterable<FireHydrant>
|
||||||
);
|
);
|
||||||
|
|
||||||
FireHydrant old;
|
FireHydrant old;
|
||||||
if (currIndex == null) { // Only happens on initialization, cannot synchronize on null
|
if (currHydrant == null) { // Only happens on initialization, cannot synchronize on null
|
||||||
old = currIndex;
|
old = currHydrant;
|
||||||
currIndex = new FireHydrant(newIndex, hydrants.size(), getSegment().getIdentifier());
|
currHydrant = new FireHydrant(newIndex, hydrants.size(), getSegment().getIdentifier());
|
||||||
hydrants.add(currIndex);
|
hydrants.add(currHydrant);
|
||||||
} else {
|
} else {
|
||||||
synchronized (currIndex) {
|
synchronized (currHydrant) {
|
||||||
old = currIndex;
|
old = currHydrant;
|
||||||
currIndex = new FireHydrant(newIndex, hydrants.size(), getSegment().getIdentifier());
|
currHydrant = new FireHydrant(newIndex, hydrants.size(), getSegment().getIdentifier());
|
||||||
hydrants.add(currIndex);
|
hydrants.add(currHydrant);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -258,6 +258,21 @@ public class RealtimeManagerTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
public int add(InputRow row)
|
||||||
|
{
|
||||||
|
if (row == null) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
Sink sink = getSink(row.getTimestampFromEpoch());
|
||||||
|
|
||||||
|
if (sink == null) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
return sink.add(row);
|
||||||
|
}
|
||||||
|
|
||||||
public Sink getSink(long timestamp)
|
public Sink getSink(long timestamp)
|
||||||
{
|
{
|
||||||
if (sink.getInterval().contains(timestamp)) {
|
if (sink.getInterval().contains(timestamp)) {
|
||||||
|
|
|
@ -161,16 +161,6 @@ public class RealtimePlumberSchoolTest
|
||||||
EasyMock.verify(announcer, segmentPublisher, dataSegmentPusher, serverView, emitter);
|
EasyMock.verify(announcer, segmentPublisher, dataSegmentPusher, serverView, emitter);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testGetSink() throws Exception
|
|
||||||
{
|
|
||||||
final DateTime theTime = new DateTime("2013-01-01");
|
|
||||||
Sink sink = plumber.getSink(theTime.getMillis());
|
|
||||||
|
|
||||||
Assert.assertEquals(new Interval(String.format("%s/PT1H", theTime.toString())), sink.getInterval());
|
|
||||||
Assert.assertEquals(theTime.toString(), sink.getVersion());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testPersist() throws Exception
|
public void testPersist() throws Exception
|
||||||
{
|
{
|
||||||
|
|
|
@ -101,7 +101,7 @@ public class SinkTest
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|
||||||
FireHydrant currHydrant = sink.getCurrIndex();
|
FireHydrant currHydrant = sink.getCurrHydrant();
|
||||||
Assert.assertEquals(new Interval("2013-01-01/PT1M"), currHydrant.getIndex().getInterval());
|
Assert.assertEquals(new Interval("2013-01-01/PT1M"), currHydrant.getIndex().getInterval());
|
||||||
|
|
||||||
|
|
||||||
|
@ -143,8 +143,8 @@ public class SinkTest
|
||||||
);
|
);
|
||||||
|
|
||||||
Assert.assertEquals(currHydrant, swapHydrant);
|
Assert.assertEquals(currHydrant, swapHydrant);
|
||||||
Assert.assertNotSame(currHydrant, sink.getCurrIndex());
|
Assert.assertNotSame(currHydrant, sink.getCurrHydrant());
|
||||||
Assert.assertEquals(new Interval("2013-01-01/PT1M"), sink.getCurrIndex().getIndex().getInterval());
|
Assert.assertEquals(new Interval("2013-01-01/PT1M"), sink.getCurrHydrant().getIndex().getInterval());
|
||||||
|
|
||||||
Assert.assertEquals(2, Iterators.size(sink.iterator()));
|
Assert.assertEquals(2, Iterators.size(sink.iterator()));
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue