Refactoring of ReferenceCountingSegment and FireHydrant (#4154)

* Refactoring of ReferenceCountingSegment and FireHydrant

* Address comment

* Fix FireHydrant.closeSegment()

* Address comment

* Added comments to ReferenceCountingSegment
This commit is contained in:
Roman Leventov 2017-09-12 14:28:35 -05:00 committed by Himanshu
parent c0be050242
commit 832cc293ef
10 changed files with 248 additions and 240 deletions

View File

@ -19,12 +19,10 @@
package io.druid.query; package io.druid.query;
import io.druid.java.util.common.guava.CloseQuietly;
import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
import io.druid.segment.ReferenceCountingSegment; import io.druid.segment.ReferenceCountingSegment;
import java.io.Closeable;
import java.util.Map; import java.util.Map;
/** /**
@ -49,16 +47,20 @@ public class ReferenceCountingSegmentQueryRunner<T> implements QueryRunner<T>
@Override @Override
public Sequence<T> run(final QueryPlus<T> queryPlus, Map<String, Object> responseContext) public Sequence<T> run(final QueryPlus<T> queryPlus, Map<String, Object> responseContext)
{ {
final Closeable closeable = adapter.increment(); if (adapter.increment()) {
if (closeable != null) {
try { try {
final Sequence<T> baseSequence = factory.createRunner(adapter).run(queryPlus, responseContext); final Sequence<T> baseSequence = factory.createRunner(adapter).run(queryPlus, responseContext);
return Sequences.withBaggage(baseSequence, closeable); return Sequences.withBaggage(baseSequence, adapter.decrementOnceCloseable());
} }
catch (RuntimeException e) { catch (Throwable t) {
CloseQuietly.close(closeable); try {
throw e; adapter.decrement();
}
catch (Exception e) {
t.addSuppressed(e);
}
throw t;
} }
} else { } else {
// Segment was closed before we had a chance to increment the reference count // Segment was closed before we had a chance to increment the reference count

View File

@ -19,23 +19,48 @@
package io.druid.segment; package io.druid.segment;
import com.google.common.base.Preconditions;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.util.concurrent.Phaser;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
/**
* ReferenceCountingSegment allows to call {@link #close()} before some other "users", which called {@link
* #increment()}, has not called {@link #decrement()} yet, and the wrapped {@link Segment} won't be actually closed
* until that. So ReferenceCountingSegment implements something like automatic reference count-based resource
* management.
*/
public class ReferenceCountingSegment extends AbstractSegment public class ReferenceCountingSegment extends AbstractSegment
{ {
private static final EmittingLogger log = new EmittingLogger(ReferenceCountingSegment.class); private static final EmittingLogger log = new EmittingLogger(ReferenceCountingSegment.class);
private final Segment baseSegment; private final Segment baseSegment;
private final AtomicBoolean closed = new AtomicBoolean(false);
private final Object lock = new Object(); private final Phaser referents = new Phaser(1)
{
private volatile int numReferences = 0; @Override
private volatile boolean isClosed = false; protected boolean onAdvance(int phase, int registeredParties)
{
Preconditions.checkState(registeredParties == 0);
// Ensure that onAdvance() doesn't throw exception, otherwise termination won't happen
try {
baseSegment.close();
}
catch (Exception e) {
try {
log.error(e, "Exception while closing segment[%s]", baseSegment.getIdentifier());
}
catch (Exception e2) {
// ignore
}
}
// Always terminate.
return true;
}
};
public ReferenceCountingSegment(Segment baseSegment) public ReferenceCountingSegment(Segment baseSegment)
{ {
@ -44,141 +69,78 @@ public class ReferenceCountingSegment extends AbstractSegment
public Segment getBaseSegment() public Segment getBaseSegment()
{ {
synchronized (lock) { return !isClosed() ? baseSegment : null;
if (isClosed) {
return null;
}
return baseSegment;
}
} }
public int getNumReferences() public int getNumReferences()
{ {
return numReferences; return Math.max(referents.getRegisteredParties() - 1, 0);
} }
public boolean isClosed() public boolean isClosed()
{ {
return isClosed; return referents.isTerminated();
} }
@Override @Override
public String getIdentifier() public String getIdentifier()
{ {
synchronized (lock) { return !isClosed() ? baseSegment.getIdentifier() : null;
if (isClosed) {
return null;
}
return baseSegment.getIdentifier();
}
} }
@Override @Override
public Interval getDataInterval() public Interval getDataInterval()
{ {
synchronized (lock) { return !isClosed() ? baseSegment.getDataInterval() : null;
if (isClosed) {
return null;
}
return baseSegment.getDataInterval();
}
} }
@Override @Override
public QueryableIndex asQueryableIndex() public QueryableIndex asQueryableIndex()
{ {
synchronized (lock) { return !isClosed() ? baseSegment.asQueryableIndex() : null;
if (isClosed) {
return null;
}
return baseSegment.asQueryableIndex();
}
} }
@Override @Override
public StorageAdapter asStorageAdapter() public StorageAdapter asStorageAdapter()
{ {
synchronized (lock) { return !isClosed() ? baseSegment.asStorageAdapter() : null;
if (isClosed) {
return null;
}
return baseSegment.asStorageAdapter();
}
} }
@Override @Override
public void close() throws IOException public void close()
{ {
synchronized (lock) { if (closed.compareAndSet(false, true)) {
if (isClosed) { referents.arriveAndDeregister();
log.info("Failed to close, %s is closed already", baseSegment.getIdentifier()); } else {
return; log.warn("close() is called more than once on ReferenceCountingSegment");
}
} }
if (numReferences > 0) { public boolean increment()
log.info("%d references to %s still exist. Decrementing.", numReferences, baseSegment.getIdentifier()); {
// Negative return from referents.register() means the Phaser is terminated.
return referents.register() >= 0;
}
/**
* Returns a {@link Closeable} which action is to call {@link #decrement()} only once. If close() is called on the
* returned Closeable object for the second time, it won't call {@link #decrement()} again.
*/
public Closeable decrementOnceCloseable()
{
AtomicBoolean decremented = new AtomicBoolean(false);
return () -> {
if (decremented.compareAndSet(false, true)) {
decrement(); decrement();
} else { } else {
log.info("Closing %s", baseSegment.getIdentifier()); log.warn("close() is called more than once on ReferenceCountingSegment.decrementOnceCloseable()");
innerClose();
}
}
}
public Closeable increment()
{
synchronized (lock) {
if (isClosed) {
return null;
}
numReferences++;
final AtomicBoolean decrementOnce = new AtomicBoolean(false);
return new Closeable()
{
@Override
public void close() throws IOException
{
if (decrementOnce.compareAndSet(false, true)) {
decrement();
}
} }
}; };
} }
}
private void decrement() public void decrement()
{ {
synchronized (lock) { referents.arriveAndDeregister();
if (isClosed) {
return;
}
if (--numReferences < 0) {
try {
innerClose();
}
catch (Exception e) {
log.error("Unable to close queryable index %s", getIdentifier());
}
}
}
}
private void innerClose() throws IOException
{
synchronized (lock) {
log.info("Closing %s, numReferences: %d", baseSegment.getIdentifier(), numReferences);
isClosed = true;
baseSegment.close();
}
} }
@Override @Override

View File

@ -83,17 +83,14 @@ public class ReferenceCountingSegmentTest
public void testMultipleClose() throws Exception public void testMultipleClose() throws Exception
{ {
Assert.assertFalse(segment.isClosed()); Assert.assertFalse(segment.isClosed());
final Closeable closeable = segment.increment(); Assert.assertTrue(segment.increment());
Assert.assertTrue(segment.getNumReferences() == 1); Assert.assertEquals(1, segment.getNumReferences());
Closeable closeable = segment.decrementOnceCloseable();
closeable.close(); closeable.close();
closeable.close(); closeable.close();
exec.submit( exec.submit(
new Runnable() () -> {
{
@Override
public void run()
{
try { try {
closeable.close(); closeable.close();
} }
@ -101,19 +98,14 @@ public class ReferenceCountingSegmentTest
throw Throwables.propagate(e); throw Throwables.propagate(e);
} }
} }
} ).get();
); Assert.assertEquals(0, segment.getNumReferences());
Assert.assertTrue(segment.getNumReferences() == 0);
Assert.assertFalse(segment.isClosed()); Assert.assertFalse(segment.isClosed());
segment.close(); segment.close();
segment.close(); segment.close();
exec.submit( exec.submit(
new Runnable() () -> {
{
@Override
public void run()
{
try { try {
segment.close(); segment.close();
} }
@ -121,18 +113,17 @@ public class ReferenceCountingSegmentTest
throw Throwables.propagate(e); throw Throwables.propagate(e);
} }
} }
} ).get();
);
Assert.assertTrue(segment.getNumReferences() == 0); Assert.assertEquals(0, segment.getNumReferences());
Assert.assertTrue(segment.isClosed()); Assert.assertTrue(segment.isClosed());
segment.increment(); segment.increment();
segment.increment(); segment.increment();
segment.increment(); segment.increment();
Assert.assertTrue(segment.getNumReferences() == 0); Assert.assertEquals(0, segment.getNumReferences());
segment.close(); segment.close();
Assert.assertTrue(segment.getNumReferences() == 0); Assert.assertEquals(0, segment.getNumReferences());
} }
} }

View File

@ -19,46 +19,39 @@
package io.druid.segment.realtime; package io.druid.segment.realtime;
import com.google.common.base.Throwables;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.java.util.common.Pair; import io.druid.java.util.common.Pair;
import io.druid.segment.IncrementalIndexSegment; import io.druid.segment.IncrementalIndexSegment;
import io.druid.segment.ReferenceCountingSegment; import io.druid.segment.ReferenceCountingSegment;
import io.druid.segment.Segment; import io.druid.segment.Segment;
import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndex;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.util.concurrent.atomic.AtomicReference;
/** /**
*/ */
public class FireHydrant public class FireHydrant
{ {
private final int count; private final int count;
private final Object swapLock = new Object(); private final AtomicReference<ReferenceCountingSegment> adapter;
private volatile IncrementalIndex index; private volatile IncrementalIndex index;
private volatile ReferenceCountingSegment adapter;
public FireHydrant( public FireHydrant(IncrementalIndex index, int count, String segmentIdentifier)
IncrementalIndex index,
int count,
String segmentIdentifier
)
{ {
this.index = index; this.index = index;
this.adapter = new ReferenceCountingSegment(new IncrementalIndexSegment(index, segmentIdentifier)); this.adapter = new AtomicReference<>(
new ReferenceCountingSegment(new IncrementalIndexSegment(index, segmentIdentifier))
);
this.count = count; this.count = count;
} }
public FireHydrant( public FireHydrant(Segment adapter, int count)
Segment adapter,
int count
)
{ {
this.index = null; this.index = null;
this.adapter = new ReferenceCountingSegment(adapter); this.adapter = new AtomicReference<>(new ReferenceCountingSegment(adapter));
this.count = count; this.count = count;
} }
@ -67,9 +60,35 @@ public class FireHydrant
return index; return index;
} }
public Segment getSegment() public String getSegmentIdentifier()
{ {
return adapter; return adapter.get().getIdentifier();
}
public Interval getSegmentDataInterval()
{
return adapter.get().getDataInterval();
}
public ReferenceCountingSegment getIncrementedSegment()
{
ReferenceCountingSegment segment = adapter.get();
while (true) {
if (segment.increment()) {
return segment;
}
// segment.increment() returned false, means it is closed. Since close() in swapSegment() happens after segment
// swap, the new segment should already be visible.
ReferenceCountingSegment newSegment = adapter.get();
if (segment == newSegment) {
throw new ISE("segment.close() is called somewhere outside FireHydrant.swapSegment()");
}
if (newSegment == null) {
throw new ISE("FireHydrant was 'closed' by swapping segment to null while acquiring a segment");
}
segment = newSegment;
// Spin loop.
}
} }
public int getCount() public int getCount()
@ -82,37 +101,41 @@ public class FireHydrant
return index == null; return index == null;
} }
public void swapSegment(Segment newAdapter) public void swapSegment(@Nullable Segment newSegment)
{ {
synchronized (swapLock) { while (true) {
if (adapter != null && newAdapter != null && !newAdapter.getIdentifier().equals(adapter.getIdentifier())) { ReferenceCountingSegment currentSegment = adapter.get();
if (currentSegment == null && newSegment == null) {
return;
}
if (currentSegment != null && newSegment != null &&
!newSegment.getIdentifier().equals(currentSegment.getIdentifier())) {
// Sanity check: identifier should not change // Sanity check: identifier should not change
throw new ISE( throw new ISE(
"WTF?! Cannot swap identifier[%s] -> [%s]!", "WTF?! Cannot swap identifier[%s] -> [%s]!",
adapter.getIdentifier(), currentSegment.getIdentifier(),
newAdapter.getIdentifier() newSegment.getIdentifier()
); );
} }
if (this.adapter != null) { if (currentSegment == newSegment) {
try { throw new ISE("Cannot swap to the same segment");
this.adapter.close();
} }
catch (IOException e) { ReferenceCountingSegment newReferenceCountingSegment =
throw Throwables.propagate(e); newSegment != null ? new ReferenceCountingSegment(newSegment) : null;
if (adapter.compareAndSet(currentSegment, newReferenceCountingSegment)) {
if (currentSegment != null) {
currentSegment.close();
} }
index = null;
return;
} }
this.adapter = new ReferenceCountingSegment(newAdapter);
this.index = null;
} }
} }
public Pair<Segment, Closeable> getAndIncrementSegment() public Pair<Segment, Closeable> getAndIncrementSegment()
{ {
// Prevent swapping of index before increment is called ReferenceCountingSegment segment = getIncrementedSegment();
synchronized (swapLock) { return new Pair<>(segment, segment.decrementOnceCloseable());
Closeable closeable = adapter.increment();
return new Pair<Segment, Closeable>(adapter, closeable);
}
} }
@Override @Override
@ -120,7 +143,7 @@ public class FireHydrant
{ {
return "FireHydrant{" + return "FireHydrant{" +
"index=" + index + "index=" + index +
", queryable=" + adapter.getIdentifier() + ", queryable=" + adapter.get().getIdentifier() +
", count=" + count + ", count=" + count +
'}'; '}';
} }

View File

@ -51,6 +51,7 @@ import io.druid.java.util.common.ISE;
import io.druid.java.util.common.Pair; import io.druid.java.util.common.Pair;
import io.druid.java.util.common.RetryUtils; import io.druid.java.util.common.RetryUtils;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.io.Closer;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryRunnerFactoryConglomerate;
@ -75,6 +76,7 @@ import org.apache.commons.io.FileUtils;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.File; import java.io.File;
import java.io.FilenameFilter; import java.io.FilenameFilter;
import java.io.IOException; import java.io.IOException;
@ -576,15 +578,18 @@ public class AppenderatorImpl implements Appenderator
throw new ISE("Merged target[%s] exists after removing?!", mergedTarget); throw new ISE("Merged target[%s] exists after removing?!", mergedTarget);
} }
final File mergedFile;
List<QueryableIndex> indexes = Lists.newArrayList(); List<QueryableIndex> indexes = Lists.newArrayList();
Closer closer = Closer.create();
try {
for (FireHydrant fireHydrant : sink) { for (FireHydrant fireHydrant : sink) {
Segment segment = fireHydrant.getSegment(); Pair<Segment, Closeable> segmentAndCloseable = fireHydrant.getAndIncrementSegment();
final QueryableIndex queryableIndex = segment.asQueryableIndex(); final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex();
log.info("Adding hydrant[%s]", fireHydrant); log.info("Adding hydrant[%s]", fireHydrant);
indexes.add(queryableIndex); indexes.add(queryableIndex);
closer.register(segmentAndCloseable.rhs);
} }
final File mergedFile;
mergedFile = indexMerger.mergeQueryableIndex( mergedFile = indexMerger.mergeQueryableIndex(
indexes, indexes,
schema.getGranularitySpec().isRollup(), schema.getGranularitySpec().isRollup(),
@ -592,6 +597,13 @@ public class AppenderatorImpl implements Appenderator
mergedTarget, mergedTarget,
tuningConfig.getIndexSpec() tuningConfig.getIndexSpec()
); );
}
catch (Throwable t) {
throw closer.rethrow(t);
}
finally {
closer.close();
}
// Retry pushing segments because uploading to deep storage might fail especially for cloud storage types // Retry pushing segments because uploading to deep storage might fail especially for cloud storage types
final DataSegment segment = RetryUtils.retry( final DataSegment segment = RetryUtils.retry(
@ -947,14 +959,7 @@ public class AppenderatorImpl implements Appenderator
if (cache != null) { if (cache != null) {
cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant)); cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant));
} }
try { hydrant.swapSegment(null);
hydrant.getSegment().close();
}
catch (IOException e) {
log.makeAlert(e, "Failed to explicitly close segment[%s]", schema.getDataSource())
.addData("identifier", hydrant.getSegment().getIdentifier())
.emit();
}
} }
if (removeOnDiskData) { if (removeOnDiskData) {
@ -1040,7 +1045,7 @@ public class AppenderatorImpl implements Appenderator
indexToPersist.swapSegment( indexToPersist.swapSegment(
new QueryableIndexSegment( new QueryableIndexSegment(
indexToPersist.getSegment().getIdentifier(), indexToPersist.getSegmentIdentifier(),
indexIO.loadIndex(persistedFile) indexIO.loadIndex(persistedFile)
) )
); );

View File

@ -309,6 +309,6 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker
public static String makeHydrantCacheIdentifier(FireHydrant input) public static String makeHydrantCacheIdentifier(FireHydrant input)
{ {
return input.getSegment().getIdentifier() + "_" + input.getCount(); return input.getSegmentIdentifier() + "_" + input.getCount();
} }
} }

View File

@ -49,6 +49,7 @@ import io.druid.java.util.common.Pair;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.concurrent.ScheduledExecutors; import io.druid.java.util.common.concurrent.ScheduledExecutors;
import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.io.Closer;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryRunnerFactoryConglomerate;
@ -79,6 +80,7 @@ import org.joda.time.Duration;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.joda.time.Period; import org.joda.time.Period;
import java.io.Closeable;
import java.io.File; import java.io.File;
import java.io.FilenameFilter; import java.io.FilenameFilter;
import java.io.IOException; import java.io.IOException;
@ -408,21 +410,33 @@ public class RealtimePlumber implements Plumber
final long mergeThreadCpuTime = VMUtils.safeGetThreadCpuTime(); final long mergeThreadCpuTime = VMUtils.safeGetThreadCpuTime();
mergeStopwatch = Stopwatch.createStarted(); mergeStopwatch = Stopwatch.createStarted();
final File mergedFile;
List<QueryableIndex> indexes = Lists.newArrayList(); List<QueryableIndex> indexes = Lists.newArrayList();
Closer closer = Closer.create();
try {
for (FireHydrant fireHydrant : sink) { for (FireHydrant fireHydrant : sink) {
Segment segment = fireHydrant.getSegment(); Pair<Segment, Closeable> segmentAndCloseable = fireHydrant.getAndIncrementSegment();
final QueryableIndex queryableIndex = segment.asQueryableIndex(); final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex();
log.info("Adding hydrant[%s]", fireHydrant); log.info("Adding hydrant[%s]", fireHydrant);
indexes.add(queryableIndex); indexes.add(queryableIndex);
closer.register(segmentAndCloseable.rhs);
} }
final File mergedFile = indexMerger.mergeQueryableIndex(
mergedFile = indexMerger.mergeQueryableIndex(
indexes, indexes,
schema.getGranularitySpec().isRollup(), schema.getGranularitySpec().isRollup(),
schema.getAggregators(), schema.getAggregators(),
mergedTarget, mergedTarget,
config.getIndexSpec() config.getIndexSpec()
); );
}
catch (Throwable t) {
throw closer.rethrow(t);
}
finally {
closer.close();
}
// emit merge metrics before publishing segment // emit merge metrics before publishing segment
metrics.incrementMergeCpuTime(VMUtils.safeGetThreadCpuTime() - mergeThreadCpuTime); metrics.incrementMergeCpuTime(VMUtils.safeGetThreadCpuTime() - mergeThreadCpuTime);
@ -857,7 +871,7 @@ public class RealtimePlumber implements Plumber
); );
for (FireHydrant hydrant : sink) { for (FireHydrant hydrant : sink) {
cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant)); cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant));
hydrant.getSegment().close(); hydrant.swapSegment(null);
} }
synchronized (handoffCondition) { synchronized (handoffCondition) {
handoffCondition.notifyAll(); handoffCondition.notifyAll();
@ -936,7 +950,7 @@ public class RealtimePlumber implements Plumber
indexToPersist.swapSegment( indexToPersist.swapSegment(
new QueryableIndexSegment( new QueryableIndexSegment(
indexToPersist.getSegment().getIdentifier(), indexToPersist.getSegmentIdentifier(),
indexIO.loadIndex(persistedFile) indexIO.loadIndex(persistedFile)
) )
); );

View File

@ -32,6 +32,7 @@ import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndex;
import io.druid.segment.ReferenceCountingSegment;
import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ColumnCapabilitiesImpl;
import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IncrementalIndexSchema;
@ -111,7 +112,13 @@ public class Sink implements Iterable<FireHydrant>
throw new ISE("hydrant[%s] not the right count[%s]", hydrant, i); throw new ISE("hydrant[%s] not the right count[%s]", hydrant, i);
} }
maxCount = hydrant.getCount(); maxCount = hydrant.getCount();
numRowsExcludingCurrIndex.addAndGet(hydrant.getSegment().asQueryableIndex().getNumRows()); ReferenceCountingSegment segment = hydrant.getIncrementedSegment();
try {
numRowsExcludingCurrIndex.addAndGet(segment.asQueryableIndex().getNumRows());
}
finally {
segment.decrement();
}
} }
this.hydrants.addAll(hydrants); this.hydrants.addAll(hydrants);
@ -272,11 +279,17 @@ public class Sink implements Iterable<FireHydrant>
Map<String, ColumnCapabilitiesImpl> oldCapabilities; Map<String, ColumnCapabilitiesImpl> oldCapabilities;
if (lastHydrant.hasSwapped()) { if (lastHydrant.hasSwapped()) {
oldCapabilities = Maps.newHashMap(); oldCapabilities = Maps.newHashMap();
QueryableIndex oldIndex = lastHydrant.getSegment().asQueryableIndex(); ReferenceCountingSegment segment = lastHydrant.getIncrementedSegment();
try {
QueryableIndex oldIndex = segment.asQueryableIndex();
for (String dim : oldIndex.getAvailableDimensions()) { for (String dim : oldIndex.getAvailableDimensions()) {
dimOrder.add(dim); dimOrder.add(dim);
oldCapabilities.put(dim, (ColumnCapabilitiesImpl) oldIndex.getColumn(dim).getCapabilities()); oldCapabilities.put(dim, (ColumnCapabilitiesImpl) oldIndex.getColumn(dim).getCapabilities());
} }
}
finally {
segment.decrement();
}
} else { } else {
IncrementalIndex oldIndex = lastHydrant.getIndex(); IncrementalIndex oldIndex = lastHydrant.getIndex();
dimOrder.addAll(oldIndex.getDimensionOrder()); dimOrder.addAll(oldIndex.getDimensionOrder());

View File

@ -34,7 +34,6 @@ import io.druid.timeline.partition.PartitionChunk;
import io.druid.timeline.partition.PartitionHolder; import io.druid.timeline.partition.PartitionHolder;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
@ -236,16 +235,8 @@ public class SegmentManager
if (oldQueryable != null) { if (oldQueryable != null) {
dataSourceState.removeSegment(segment); dataSourceState.removeSegment(segment);
try {
log.info("Attempting to close segment %s", segment.getIdentifier()); log.info("Attempting to close segment %s", segment.getIdentifier());
oldQueryable.close(); oldQueryable.close();
}
catch (IOException e) {
log.makeAlert(e, "Exception closing segment")
.addData("dataSource", dataSourceName)
.addData("segmentId", segment.getIdentifier())
.emit();
}
} else { } else {
log.info( log.info(
"Told to delete a queryable on dataSource[%s] for interval[%s] and version [%s] that I don't have.", "Told to delete a queryable on dataSource[%s] for interval[%s] and version [%s] that I don't have.",

View File

@ -46,6 +46,7 @@ import io.druid.query.SegmentDescriptor;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndex;
import io.druid.segment.ReferenceCountingSegment;
import io.druid.segment.TestHelper; import io.druid.segment.TestHelper;
import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.RealtimeTuningConfig;
@ -425,17 +426,17 @@ public class RealtimePlumberSchoolTest
Assert.assertEquals(0, hydrants.get(0).getCount()); Assert.assertEquals(0, hydrants.get(0).getCount());
Assert.assertEquals( Assert.assertEquals(
expectedInterval, expectedInterval,
hydrants.get(0).getSegment().getDataInterval() hydrants.get(0).getSegmentDataInterval()
); );
Assert.assertEquals(2, hydrants.get(1).getCount()); Assert.assertEquals(2, hydrants.get(1).getCount());
Assert.assertEquals( Assert.assertEquals(
expectedInterval, expectedInterval,
hydrants.get(1).getSegment().getDataInterval() hydrants.get(1).getSegmentDataInterval()
); );
Assert.assertEquals(4, hydrants.get(2).getCount()); Assert.assertEquals(4, hydrants.get(2).getCount());
Assert.assertEquals( Assert.assertEquals(
expectedInterval, expectedInterval,
hydrants.get(2).getSegment().getDataInterval() hydrants.get(2).getSegmentDataInterval()
); );
/* Delete all the hydrants and reload, no sink should be created */ /* Delete all the hydrants and reload, no sink should be created */
@ -563,10 +564,16 @@ public class RealtimePlumberSchoolTest
for (int i = 0; i < hydrants.size(); i++) { for (int i = 0; i < hydrants.size(); i++) {
hydrant = hydrants.get(i); hydrant = hydrants.get(i);
qindex = hydrant.getSegment().asQueryableIndex(); ReferenceCountingSegment segment = hydrant.getIncrementedSegment();
try {
qindex = segment.asQueryableIndex();
Assert.assertEquals(i, hydrant.getCount()); Assert.assertEquals(i, hydrant.getCount());
Assert.assertEquals(expectedDims.get(i), ImmutableList.copyOf(qindex.getAvailableDimensions())); Assert.assertEquals(expectedDims.get(i), ImmutableList.copyOf(qindex.getAvailableDimensions()));
} }
finally {
segment.decrement();
}
}
} }
private InputRow getTestInputRow(final String timeStr) private InputRow getTestInputRow(final String timeStr)