better handling of exceptions during startup

This commit is contained in:
fjy 2013-08-12 12:56:59 -07:00
parent 2f728f3478
commit 0c360c05c2
1 changed files with 58 additions and 26 deletions

View File

@ -20,9 +20,12 @@
package com.metamx.druid.coordination; package com.metamx.druid.coordination;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Predicates;
import com.google.common.base.Throwables; import com.google.common.base.Throwables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DataSegment;
@ -36,6 +39,7 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
import org.apache.curator.utils.ZKPaths; import org.apache.curator.utils.ZKPaths;
import javax.annotation.Nullable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
@ -107,7 +111,13 @@ public class ZkCoordinator implements DataSegmentChangeHandler
curator.newNamespaceAwareEnsurePath(liveSegmentsLocation).ensure(curator.getZookeeperClient()); curator.newNamespaceAwareEnsurePath(liveSegmentsLocation).ensure(curator.getZookeeperClient());
if (config.isLoadFromSegmentCacheEnabled()) { if (config.isLoadFromSegmentCacheEnabled()) {
loadCache(); try {
loadCache();
}
catch (Exception e) {
log.makeAlert(e, "Exception loading from cache")
.emit();
}
} }
loadQueueCache.getListenable().addListener( loadQueueCache.getListenable().addListener(
@ -254,7 +264,6 @@ public class ZkCoordinator implements DataSegmentChangeHandler
announcer.announceSegment(segment); announcer.announceSegment(segment);
} }
catch (IOException e) { catch (IOException e) {
removeSegment(segment);
throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getIdentifier()); throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getIdentifier());
} }
@ -269,38 +278,61 @@ public class ZkCoordinator implements DataSegmentChangeHandler
public void addSegments(Iterable<DataSegment> segments) public void addSegments(Iterable<DataSegment> segments)
{ {
try { try {
for (DataSegment segment : segments) { final List<String> segmentFailures = Lists.newArrayList();
log.info("Loading segment %s", segment.getIdentifier()); Iterable<DataSegment> validSegments = FunctionalIterable
.create(segments)
.transform(
new Function<DataSegment, DataSegment>()
{
@Nullable
@Override
public DataSegment apply(@Nullable DataSegment segment)
{
if (segment == null) {
return null;
}
try { log.info("Loading segment %s", segment.getIdentifier());
serverManager.loadSegment(segment);
}
catch (Exception e) {
removeSegment(segment);
throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getIdentifier());
}
File segmentInfoCacheFile = new File(config.getSegmentInfoCacheDirectory(), segment.getIdentifier()); try {
if (!segmentInfoCacheFile.exists()) { serverManager.loadSegment(segment);
try { }
jsonMapper.writeValue(segmentInfoCacheFile, segment); catch (Exception e) {
} log.error(e, "Exception loading segment[%s]", segment.getIdentifier());
catch (IOException e) { removeSegment(segment);
removeSegment(segment); segmentFailures.add(segment.getIdentifier());
throw new SegmentLoadingException( return null;
e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile }
);
} File segmentInfoCacheFile = new File(config.getSegmentInfoCacheDirectory(), segment.getIdentifier());
} if (!segmentInfoCacheFile.exists()) {
} try {
jsonMapper.writeValue(segmentInfoCacheFile, segment);
}
catch (IOException e) {
log.error(e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile);
removeSegment(segment);
segmentFailures.add(segment.getIdentifier());
return null;
}
}
return segment;
}
}
)
.filter(Predicates.<DataSegment>notNull());
try { try {
announcer.announceSegments(segments); announcer.announceSegments(validSegments);
} }
catch (IOException e) { catch (IOException e) {
removeSegments(segments);
throw new SegmentLoadingException(e, "Failed to announce segments[%s]", segments); throw new SegmentLoadingException(e, "Failed to announce segments[%s]", segments);
} }
if (!segmentFailures.isEmpty()) {
throw new SegmentLoadingException("Error loading segments: %s", segmentFailures);
}
} }
catch (SegmentLoadingException e) { catch (SegmentLoadingException e) {
log.makeAlert(e, "Failed to load segments for dataSource") log.makeAlert(e, "Failed to load segments for dataSource")