mirror of https://github.com/apache/druid.git
clean up the error reporting code according to code review
This commit is contained in:
parent
0c360c05c2
commit
2ec2957207
|
@ -20,12 +20,9 @@
|
||||||
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;
|
||||||
|
@ -39,7 +36,6 @@ 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;
|
||||||
|
@ -279,49 +275,34 @@ public class ZkCoordinator implements DataSegmentChangeHandler
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
final List<String> segmentFailures = Lists.newArrayList();
|
final List<String> segmentFailures = Lists.newArrayList();
|
||||||
Iterable<DataSegment> validSegments = FunctionalIterable
|
final List<DataSegment> validSegments = Lists.newArrayList();
|
||||||
.create(segments)
|
|
||||||
.transform(
|
|
||||||
new Function<DataSegment, DataSegment>()
|
|
||||||
{
|
|
||||||
@Nullable
|
|
||||||
@Override
|
|
||||||
public DataSegment apply(@Nullable DataSegment segment)
|
|
||||||
{
|
|
||||||
if (segment == null) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
log.info("Loading segment %s", segment.getIdentifier());
|
for (DataSegment segment : segments) {
|
||||||
|
log.info("Loading segment %s", segment.getIdentifier());
|
||||||
|
|
||||||
try {
|
try {
|
||||||
serverManager.loadSegment(segment);
|
serverManager.loadSegment(segment);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.error(e, "Exception loading segment[%s]", segment.getIdentifier());
|
log.error(e, "Exception loading segment[%s]", segment.getIdentifier());
|
||||||
removeSegment(segment);
|
removeSegment(segment);
|
||||||
segmentFailures.add(segment.getIdentifier());
|
segmentFailures.add(segment.getIdentifier());
|
||||||
return null;
|
}
|
||||||
}
|
|
||||||
|
|
||||||
File segmentInfoCacheFile = new File(config.getSegmentInfoCacheDirectory(), segment.getIdentifier());
|
File segmentInfoCacheFile = new File(config.getSegmentInfoCacheDirectory(), segment.getIdentifier());
|
||||||
if (!segmentInfoCacheFile.exists()) {
|
if (!segmentInfoCacheFile.exists()) {
|
||||||
try {
|
try {
|
||||||
jsonMapper.writeValue(segmentInfoCacheFile, segment);
|
jsonMapper.writeValue(segmentInfoCacheFile, segment);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
log.error(e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile);
|
log.error(e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile);
|
||||||
removeSegment(segment);
|
removeSegment(segment);
|
||||||
segmentFailures.add(segment.getIdentifier());
|
segmentFailures.add(segment.getIdentifier());
|
||||||
return null;
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
return segment;
|
validSegments.add(segment);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
)
|
|
||||||
.filter(Predicates.<DataSegment>notNull());
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
announcer.announceSegments(validSegments);
|
announcer.announceSegments(validSegments);
|
||||||
|
@ -331,7 +312,8 @@ public class ZkCoordinator implements DataSegmentChangeHandler
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!segmentFailures.isEmpty()) {
|
if (!segmentFailures.isEmpty()) {
|
||||||
throw new SegmentLoadingException("Error loading segments: %s", segmentFailures);
|
log.error("Exception loading segments: %s", segmentFailures);
|
||||||
|
throw new SegmentLoadingException("%,d errors seen while loading segments", segmentFailures.size());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (SegmentLoadingException e) {
|
catch (SegmentLoadingException e) {
|
||||||
|
|
Loading…
Reference in New Issue