mirror of https://github.com/apache/druid.git
bug fixes for dropping segments and logging
This commit is contained in:
parent
e37e24cc25
commit
6a11e99681
|
@ -42,18 +42,7 @@ public class SegmentReplicantLookup
|
||||||
for (ServerHolder serverHolder : serversByType) {
|
for (ServerHolder serverHolder : serversByType) {
|
||||||
DruidServer server = serverHolder.getServer();
|
DruidServer server = serverHolder.getServer();
|
||||||
|
|
||||||
for (DruidDataSource dataSource : server.getDataSources()) {
|
for (DataSegment segment : server.getSegments().values()) {
|
||||||
for (DataSegment segment : dataSource.getSegments()) {
|
|
||||||
Integer numReplicants = segmentsInCluster.get(segment.getIdentifier(), server.getTier());
|
|
||||||
if (numReplicants == null) {
|
|
||||||
numReplicants = 0;
|
|
||||||
}
|
|
||||||
segmentsInCluster.put(segment.getIdentifier(), server.getTier(), ++numReplicants);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Also account for queued segments
|
|
||||||
for (DataSegment segment : serverHolder.getPeon().getSegmentsToLoad()) {
|
|
||||||
Integer numReplicants = segmentsInCluster.get(segment.getIdentifier(), server.getTier());
|
Integer numReplicants = segmentsInCluster.get(segment.getIdentifier(), server.getTier());
|
||||||
if (numReplicants == null) {
|
if (numReplicants == null) {
|
||||||
numReplicants = 0;
|
numReplicants = 0;
|
||||||
|
|
|
@ -84,6 +84,7 @@ public abstract class LoadRule implements Rule
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
if (holder.containsSegment(segment)) {
|
if (holder.containsSegment(segment)) {
|
||||||
|
serverQueue.add(holder);
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -536,7 +536,6 @@ public class DruidMasterRuleRunnerTest
|
||||||
{
|
{
|
||||||
mockPeon.dropSegment(EasyMock.<DataSegment>anyObject(), EasyMock.<LoadPeonCallback>anyObject());
|
mockPeon.dropSegment(EasyMock.<DataSegment>anyObject(), EasyMock.<LoadPeonCallback>anyObject());
|
||||||
EasyMock.expectLastCall().atLeastOnce();
|
EasyMock.expectLastCall().atLeastOnce();
|
||||||
EasyMock.expect(mockPeon.getSegmentsToLoad()).andReturn(Sets.<DataSegment>newHashSet()).atLeastOnce();
|
|
||||||
EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).atLeastOnce();
|
EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).atLeastOnce();
|
||||||
EasyMock.replay(mockPeon);
|
EasyMock.replay(mockPeon);
|
||||||
|
|
||||||
|
@ -690,8 +689,6 @@ public class DruidMasterRuleRunnerTest
|
||||||
{
|
{
|
||||||
mockPeon.loadSegment(EasyMock.<DataSegment>anyObject(), EasyMock.<LoadPeonCallback>anyObject());
|
mockPeon.loadSegment(EasyMock.<DataSegment>anyObject(), EasyMock.<LoadPeonCallback>anyObject());
|
||||||
EasyMock.expectLastCall().atLeastOnce();
|
EasyMock.expectLastCall().atLeastOnce();
|
||||||
mockPeon.dropSegment(EasyMock.<DataSegment>anyObject(), EasyMock.<LoadPeonCallback>anyObject());
|
|
||||||
EasyMock.expectLastCall().atLeastOnce();
|
|
||||||
EasyMock.expect(mockPeon.getSegmentsToLoad()).andReturn(Sets.<DataSegment>newHashSet()).atLeastOnce();
|
EasyMock.expect(mockPeon.getSegmentsToLoad()).andReturn(Sets.<DataSegment>newHashSet()).atLeastOnce();
|
||||||
EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).atLeastOnce();
|
EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).atLeastOnce();
|
||||||
EasyMock.replay(mockPeon);
|
EasyMock.replay(mockPeon);
|
||||||
|
@ -759,6 +756,8 @@ public class DruidMasterRuleRunnerTest
|
||||||
|
|
||||||
Assert.assertTrue(stats.getPerTierStats().get("droppedCount") == null);
|
Assert.assertTrue(stats.getPerTierStats().get("droppedCount") == null);
|
||||||
Assert.assertTrue(stats.getGlobalStats().get("deletedCount").get() == 12);
|
Assert.assertTrue(stats.getGlobalStats().get("deletedCount").get() == 12);
|
||||||
|
|
||||||
|
EasyMock.verify(mockPeon);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -799,12 +798,10 @@ public class DruidMasterRuleRunnerTest
|
||||||
|
|
||||||
mockPeon.dropSegment(EasyMock.<DataSegment>anyObject(), EasyMock.<LoadPeonCallback>anyObject());
|
mockPeon.dropSegment(EasyMock.<DataSegment>anyObject(), EasyMock.<LoadPeonCallback>anyObject());
|
||||||
EasyMock.expectLastCall().atLeastOnce();
|
EasyMock.expectLastCall().atLeastOnce();
|
||||||
EasyMock.expect(mockPeon.getSegmentsToLoad()).andReturn(Sets.<DataSegment>newHashSet()).atLeastOnce();
|
|
||||||
EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).atLeastOnce();
|
EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).atLeastOnce();
|
||||||
EasyMock.replay(mockPeon);
|
EasyMock.replay(mockPeon);
|
||||||
|
|
||||||
LoadQueuePeon anotherMockPeon = EasyMock.createMock(LoadQueuePeon.class);
|
LoadQueuePeon anotherMockPeon = EasyMock.createMock(LoadQueuePeon.class);
|
||||||
EasyMock.expect(anotherMockPeon.getSegmentsToLoad()).andReturn(Sets.<DataSegment>newHashSet()).atLeastOnce();
|
|
||||||
EasyMock.expect(anotherMockPeon.getLoadQueueSize()).andReturn(10L).atLeastOnce();
|
EasyMock.expect(anotherMockPeon.getLoadQueueSize()).andReturn(10L).atLeastOnce();
|
||||||
EasyMock.replay(anotherMockPeon);
|
EasyMock.replay(anotherMockPeon);
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue