mirror of https://github.com/apache/druid.git
Merge pull request #45 from metamx/fjy
bug fixes for exceptions in rules
This commit is contained in:
commit
7670c8483b
|
@ -44,6 +44,7 @@ import com.metamx.druid.client.ServerInventoryManager;
|
|||
import com.metamx.druid.coordination.DruidClusterInfo;
|
||||
import com.metamx.druid.db.DatabaseRuleManager;
|
||||
import com.metamx.druid.db.DatabaseSegmentManager;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import com.metamx.phonebook.PhoneBook;
|
||||
|
@ -69,7 +70,7 @@ public class DruidMaster
|
|||
{
|
||||
public static final String MASTER_OWNER_NODE = "_MASTER";
|
||||
|
||||
private static final Logger log = new Logger(DruidMaster.class);
|
||||
private static final EmittingLogger log = new EmittingLogger(DruidMaster.class);
|
||||
|
||||
private final Object lock = new Object();
|
||||
|
||||
|
@ -575,7 +576,7 @@ public class DruidMaster
|
|||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(e, "Caught exception, ignoring so that schedule keeps going.");
|
||||
log.makeAlert(e, "Caught exception, ignoring so that schedule keeps going.").emit();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.metamx.druid.client.DataSegment;
|
|||
import com.metamx.druid.db.DatabaseRuleManager;
|
||||
import com.metamx.druid.master.rules.Rule;
|
||||
import com.metamx.druid.master.rules.RuleMap;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
|
@ -32,7 +33,7 @@ import java.util.List;
|
|||
*/
|
||||
public class DruidMasterRuleRunner implements DruidMasterHelper
|
||||
{
|
||||
private static final Logger log = new Logger(DruidMasterRuleRunner.class);
|
||||
private static final EmittingLogger log = new EmittingLogger(DruidMasterRuleRunner.class);
|
||||
|
||||
private final DruidMaster master;
|
||||
|
||||
|
@ -67,7 +68,12 @@ public class DruidMasterRuleRunner implements DruidMasterHelper
|
|||
}
|
||||
|
||||
if (!foundMatchingRule) {
|
||||
throw new ISE("Unable to find a matching rule for segment[%s]", segment.getIdentifier());
|
||||
log.makeAlert(
|
||||
"Unable to find a matching rule for dataSource[%s]",
|
||||
segment.getDataSource()
|
||||
)
|
||||
.addData("segment", segment.getIdentifier())
|
||||
.emit();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -52,7 +52,7 @@ public abstract class LoadRule implements Rule
|
|||
MinMaxPriorityQueue<ServerHolder> serverQueue = params.getDruidCluster().getServersByTier(getTier());
|
||||
if (serverQueue == null) {
|
||||
log.makeAlert("Tier[%s] has no servers! Check your cluster configuration!", getTier()).emit();
|
||||
throw new ISE("Tier[%s] has no servers! Check your cluster configuration!", getTier());
|
||||
return stats;
|
||||
}
|
||||
|
||||
stats.accumulate(assign(expectedReplicants, actualReplicants, serverQueue, segment));
|
||||
|
|
|
@ -357,7 +357,7 @@ public class DruidMasterRuleRunnerTest
|
|||
public void testRunTwoTiersTierDoesNotExist() throws Exception
|
||||
{
|
||||
emitter.emit(EasyMock.<ServiceEventBuilder>anyObject());
|
||||
EasyMock.expectLastCall().atLeastOnce();
|
||||
EasyMock.expectLastCall().times(12);
|
||||
EasyMock.replay(emitter);
|
||||
|
||||
EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.<String>anyObject())).andReturn(
|
||||
|
@ -397,20 +397,61 @@ public class DruidMasterRuleRunnerTest
|
|||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
|
||||
.build();
|
||||
|
||||
boolean exceptionOccurred = false;
|
||||
try {
|
||||
ruleRunner.run(params);
|
||||
}
|
||||
catch (Exception e) {
|
||||
exceptionOccurred = true;
|
||||
}
|
||||
|
||||
Assert.assertTrue(exceptionOccurred);
|
||||
ruleRunner.run(params);
|
||||
|
||||
EasyMock.verify(emitter);
|
||||
EasyMock.verify(mockPeon);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunRuleDoesNotExist() throws Exception
|
||||
{
|
||||
emitter.emit(EasyMock.<ServiceEventBuilder>anyObject());
|
||||
EasyMock.expectLastCall().times(availableSegments.size());
|
||||
EasyMock.replay(emitter);
|
||||
|
||||
EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.<String>anyObject())).andReturn(
|
||||
Lists.<Rule>newArrayList(
|
||||
new IntervalLoadRule(new Interval("2012-01-02T00:00:00.000Z/2012-01-03T00:00:00.000Z"), 1, "normal")
|
||||
)
|
||||
).atLeastOnce();
|
||||
EasyMock.replay(databaseRuleManager);
|
||||
|
||||
DruidCluster druidCluster = new DruidCluster(
|
||||
ImmutableMap.of(
|
||||
"normal",
|
||||
MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create(
|
||||
Arrays.asList(
|
||||
new ServerHolder(
|
||||
new DruidServer(
|
||||
"serverNorm",
|
||||
"hostNorm",
|
||||
1000,
|
||||
"historical",
|
||||
"normal"
|
||||
),
|
||||
mockPeon
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
DruidMasterRuntimeParams params =
|
||||
new DruidMasterRuntimeParams.Builder()
|
||||
.withEmitter(emitter)
|
||||
.withDruidCluster(druidCluster)
|
||||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
|
||||
.build();
|
||||
|
||||
ruleRunner.run(params);
|
||||
|
||||
EasyMock.verify(emitter);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDropRemove() throws Exception
|
||||
{
|
||||
|
|
Loading…
Reference in New Issue