clean up some logging in Zk coordinator

This commit is contained in:
fjy 2014-07-29 11:49:06 -07:00
parent 0be5d27f2a
commit 4a417cbcfc
1 changed files with 12 additions and 6 deletions

View File

@ -103,14 +103,14 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler
switch (event.getType()) {
case CHILD_ADDED:
final String path = child.getPath();
final DataSegmentChangeRequest segment = jsonMapper.readValue(
final DataSegmentChangeRequest request = jsonMapper.readValue(
child.getData(), DataSegmentChangeRequest.class
);
log.info("New node[%s] with segmentClass[%s]", path, segment.getClass());
log.info("New node[%s] with segmentClass[%s]", path, request.getClass());
try {
segment.go(
request.go(
getDataSegmentChangeHandler(),
new DataSegmentChangeCallback()
{
@ -122,11 +122,17 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler
try {
if (!hasRun) {
curator.delete().guaranteed().forPath(path);
log.info("Completed processing for node[%s]", path);
log.info("Completed processing and removing entry for node[%s]", path);
hasRun = true;
}
}
catch (Exception e) {
try {
curator.delete().guaranteed().forPath(path);
}
catch (Exception e1) {
log.info(e1, "Failed to delete node[%s], but ignoring exception.", path);
}
throw Throwables.propagate(e);
}
}
@ -143,13 +149,13 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler
log.makeAlert(e, "Segment load/unload: uncaught exception.")
.addData("node", path)
.addData("nodeProperties", segment)
.addData("nodeProperties", request)
.emit();
}
break;
case CHILD_REMOVED:
log.info("%s was removed", event.getData().getPath());
log.info("[%s] was removed", event.getData().getPath());
break;
default:
log.info("Ignoring event[%s]", event);