mirror of https://github.com/apache/druid.git
Add unit tests
This commit is contained in:
parent
64e3851160
commit
1c05c2f5f5
|
@ -225,7 +225,7 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
|
||||||
throw DruidException
|
throw DruidException
|
||||||
.forPersona(DruidException.Persona.DEVELOPER)
|
.forPersona(DruidException.Persona.DEVELOPER)
|
||||||
.ofCategory(DruidException.Category.SERVICE_UNAVAILABLE)
|
.ofCategory(DruidException.Category.SERVICE_UNAVAILABLE)
|
||||||
.build("Cannot append segments to [%s] right now." +
|
.build("Cannot append segments to [%s] right now. " +
|
||||||
"There might be another task waiting to publish its segments. Check the overlord logs for details.",
|
"There might be another task waiting to publish its segments. Check the overlord logs for details.",
|
||||||
dataSourceToInsert
|
dataSourceToInsert
|
||||||
);
|
);
|
||||||
|
|
|
@ -22,25 +22,45 @@ package org.apache.druid.indexing.common.actions;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
import org.apache.druid.data.input.impl.ByteEntity;
|
||||||
|
import org.apache.druid.error.DruidException;
|
||||||
import org.apache.druid.error.InvalidInput;
|
import org.apache.druid.error.InvalidInput;
|
||||||
import org.apache.druid.indexing.common.TaskLockType;
|
import org.apache.druid.indexing.common.TaskLockType;
|
||||||
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.task.NoopTask;
|
import org.apache.druid.indexing.common.task.NoopTask;
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.indexing.overlord.LockResult;
|
import org.apache.druid.indexing.overlord.LockResult;
|
||||||
import org.apache.druid.indexing.overlord.ObjectMetadata;
|
import org.apache.druid.indexing.overlord.ObjectMetadata;
|
||||||
import org.apache.druid.indexing.overlord.SegmentPublishResult;
|
import org.apache.druid.indexing.overlord.SegmentPublishResult;
|
||||||
import org.apache.druid.indexing.overlord.Segments;
|
import org.apache.druid.indexing.overlord.Segments;
|
||||||
import org.apache.druid.indexing.overlord.TimeChunkLockRequest;
|
import org.apache.druid.indexing.overlord.TimeChunkLockRequest;
|
||||||
|
import org.apache.druid.indexing.overlord.supervisor.SupervisorManager;
|
||||||
|
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask;
|
||||||
|
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
|
||||||
|
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
|
||||||
|
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig;
|
||||||
|
import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||||
import org.assertj.core.api.Assertions;
|
import org.assertj.core.api.Assertions;
|
||||||
|
import org.easymock.EasyMock;
|
||||||
|
import org.easymock.EasyMockRunner;
|
||||||
|
import org.easymock.EasyMockSupport;
|
||||||
|
import org.easymock.Mock;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
|
||||||
public class SegmentTransactionalInsertActionTest
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
@RunWith(EasyMockRunner.class)
|
||||||
|
public class SegmentTransactionalInsertActionTest extends EasyMockSupport
|
||||||
{
|
{
|
||||||
@Rule
|
@Rule
|
||||||
public TaskActionTestKit actionTestKit = new TaskActionTestKit();
|
public TaskActionTestKit actionTestKit = new TaskActionTestKit();
|
||||||
|
@ -86,6 +106,12 @@ public class SegmentTransactionalInsertActionTest
|
||||||
1024
|
1024
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private TaskActionToolbox taskActionToolbox;
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
SupervisorManager supervisorManager;
|
||||||
|
|
||||||
private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interval interval, long timeoutMs)
|
private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interval interval, long timeoutMs)
|
||||||
throws InterruptedException
|
throws InterruptedException
|
||||||
{
|
{
|
||||||
|
@ -175,4 +201,87 @@ public class SegmentTransactionalInsertActionTest
|
||||||
);
|
);
|
||||||
Assert.assertTrue(exception.getMessage().contains("are not covered by locks"));
|
Assert.assertTrue(exception.getMessage().contains("are not covered by locks"));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testStreamingTaskNotPublishable() throws Exception
|
||||||
|
{
|
||||||
|
// Mocking the config classes because they have a lot of logic in their constructors that we don't really want here.
|
||||||
|
SeekableStreamIndexTaskTuningConfig taskTuningConfig = EasyMock.createMock(SeekableStreamIndexTaskTuningConfig.class);
|
||||||
|
SeekableStreamIndexTaskIOConfig taskIOConfig = EasyMock.createMock(SeekableStreamIndexTaskIOConfig.class);
|
||||||
|
|
||||||
|
final SeekableStreamIndexTask streamingTask = new TestSeekableStreamIndexTask(
|
||||||
|
"id1",
|
||||||
|
null,
|
||||||
|
DataSchema.builder().withDataSource(DATA_SOURCE).build(),
|
||||||
|
taskTuningConfig,
|
||||||
|
taskIOConfig,
|
||||||
|
ImmutableMap.of(),
|
||||||
|
"0"
|
||||||
|
);
|
||||||
|
|
||||||
|
EasyMock.expect(taskActionToolbox.getSupervisorManager()).andReturn(supervisorManager);
|
||||||
|
EasyMock.expect(taskActionToolbox.getTaskLockbox()).andReturn(actionTestKit.getTaskLockbox());
|
||||||
|
EasyMock.expect(supervisorManager.canPublishSegments(EasyMock.anyString(), EasyMock.anyInt(), EasyMock.anyString()))
|
||||||
|
.andReturn(false);
|
||||||
|
|
||||||
|
actionTestKit.getTaskLockbox().add(streamingTask);
|
||||||
|
acquireTimeChunkLock(TaskLockType.EXCLUSIVE, streamingTask, INTERVAL, 5000);
|
||||||
|
replayAll();
|
||||||
|
|
||||||
|
DruidException druidException = Assert.assertThrows(DruidException.class, () -> SegmentTransactionalInsertAction.appendAction(
|
||||||
|
ImmutableSet.of(SEGMENT1),
|
||||||
|
new ObjectMetadata(null),
|
||||||
|
new ObjectMetadata(ImmutableList.of(1)),
|
||||||
|
null
|
||||||
|
).perform(
|
||||||
|
streamingTask,
|
||||||
|
taskActionToolbox
|
||||||
|
));
|
||||||
|
verifyAll();
|
||||||
|
|
||||||
|
Assert.assertEquals(503, druidException.getStatusCode());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class TestSeekableStreamIndexTask extends SeekableStreamIndexTask<String, String, ByteEntity>
|
||||||
|
{
|
||||||
|
public TestSeekableStreamIndexTask(
|
||||||
|
String id,
|
||||||
|
@Nullable TaskResource taskResource,
|
||||||
|
DataSchema dataSchema,
|
||||||
|
SeekableStreamIndexTaskTuningConfig tuningConfig,
|
||||||
|
SeekableStreamIndexTaskIOConfig<String, String> ioConfig,
|
||||||
|
@Nullable Map<String, Object> context,
|
||||||
|
@Nullable String groupId
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(
|
||||||
|
id,
|
||||||
|
taskResource,
|
||||||
|
dataSchema,
|
||||||
|
tuningConfig,
|
||||||
|
ioConfig,
|
||||||
|
context,
|
||||||
|
groupId
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected SeekableStreamIndexTaskRunner<String, String, ByteEntity> createTaskRunner()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected RecordSupplier<String, String, ByteEntity> newTaskRecordSupplier(final TaskToolbox toolbox)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getType()
|
||||||
|
{
|
||||||
|
return "test";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -291,6 +291,52 @@ public class SupervisorManagerTest extends EasyMockSupport
|
||||||
verifyAll();
|
verifyAll();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCanPublishSegments_returnsFalse()
|
||||||
|
{
|
||||||
|
String taskId = "id1";
|
||||||
|
String supervisorId = "supervisor-id";
|
||||||
|
Integer groupId = 1;
|
||||||
|
Map<String, SupervisorSpec> existingSpecs = ImmutableMap.of(
|
||||||
|
supervisorId, new TestSupervisorSpec(supervisorId, supervisor1)
|
||||||
|
);
|
||||||
|
|
||||||
|
EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs);
|
||||||
|
supervisor1.start();
|
||||||
|
EasyMock.expect(supervisor1.canPublishSegments(groupId, taskId)).andReturn(false);
|
||||||
|
|
||||||
|
replayAll();
|
||||||
|
|
||||||
|
manager.start();
|
||||||
|
|
||||||
|
Assert.assertFalse(manager.canPublishSegments(supervisorId, groupId, taskId));
|
||||||
|
|
||||||
|
verifyAll();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCanPublishSegments_throwsException_returnsTrue()
|
||||||
|
{
|
||||||
|
String taskId = "id1";
|
||||||
|
String supervisorId = "supervisor-id";
|
||||||
|
Integer groupId = 1;
|
||||||
|
Map<String, SupervisorSpec> existingSpecs = ImmutableMap.of(
|
||||||
|
supervisorId, new TestSupervisorSpec(supervisorId, supervisor1)
|
||||||
|
);
|
||||||
|
|
||||||
|
EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs);
|
||||||
|
supervisor1.start();
|
||||||
|
EasyMock.expect(supervisor1.canPublishSegments(groupId, taskId)).andThrow(new RuntimeException());
|
||||||
|
|
||||||
|
replayAll();
|
||||||
|
|
||||||
|
manager.start();
|
||||||
|
|
||||||
|
Assert.assertTrue(manager.canPublishSegments(supervisorId, groupId, taskId));
|
||||||
|
|
||||||
|
verifyAll();
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testStartAlreadyStarted()
|
public void testStartAlreadyStarted()
|
||||||
{
|
{
|
||||||
|
|
|
@ -100,4 +100,69 @@ public class StreamSupervisorTest
|
||||||
ex.getMessage()
|
ex.getMessage()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDefaultCanPublishSegments()
|
||||||
|
{
|
||||||
|
// Create an instance of stream supervisor without overriding handoffTaskGroupsEarly().
|
||||||
|
final StreamSupervisor streamSupervisor = new StreamSupervisor()
|
||||||
|
{
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void start()
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void stop(boolean stopGracefully)
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SupervisorReport getStatus()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SupervisorStateManager.State getState()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void reset(@Nullable DataSourceMetadata dataSourceMetadata)
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void resetOffsets(DataSourceMetadata resetDataSourceMetadata)
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata)
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public LagStats computeLagStats()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getActiveTaskGroupsCount()
|
||||||
|
{
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
Assert.assertTrue(streamSupervisor.canPublishSegments(1, "taskId"));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue