mirror of https://github.com/apache/druid.git
RealtimeIndexTask-related stuff.
- New task: RealtimeIndexTask - Add SegmentAnnouncer, ServerView, QueryRunnerFactoryConglomerate to TaskToolbox - Tasks can advertise ability to answer queries (through returning non-null from getQueryRunner) - WorkerTaskMonitor (the thing on a worker that tracks running tasks) is now a QuerySegmentWalker - LockAcquireAction is now blocking Assorted other changes. - TaskAction.perform throws IOException - TaskActions generally have better stringification - Renamed TaskMonitor -> WorkerTaskMonitor
This commit is contained in:
parent
0e4db00d54
commit
6245e38981
|
@ -22,6 +22,7 @@ package com.metamx.druid.merger.common;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.client.MutableServerView;
|
||||
import com.metamx.druid.loading.DataSegmentPusher;
|
||||
import com.metamx.druid.loading.MMappedQueryableIndexFactory;
|
||||
import com.metamx.druid.loading.S3DataSegmentPuller;
|
||||
|
@ -33,6 +34,8 @@ import com.metamx.druid.merger.common.actions.TaskActionClient;
|
|||
import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
|
||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import com.metamx.druid.realtime.SegmentAnnouncer;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||
|
||||
|
@ -52,6 +55,9 @@ public class TaskToolbox
|
|||
private final RestS3Service s3Client;
|
||||
private final DataSegmentPusher segmentPusher;
|
||||
private final DataSegmentKiller dataSegmentKiller;
|
||||
private final SegmentAnnouncer segmentAnnouncer;
|
||||
private final MutableServerView newSegmentServerView;
|
||||
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
|
||||
private final ObjectMapper objectMapper;
|
||||
|
||||
public TaskToolbox(
|
||||
|
@ -62,6 +68,9 @@ public class TaskToolbox
|
|||
RestS3Service s3Client,
|
||||
DataSegmentPusher segmentPusher,
|
||||
DataSegmentKiller dataSegmentKiller,
|
||||
SegmentAnnouncer segmentAnnouncer,
|
||||
MutableServerView newSegmentServerView,
|
||||
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
|
||||
ObjectMapper objectMapper
|
||||
)
|
||||
{
|
||||
|
@ -72,6 +81,9 @@ public class TaskToolbox
|
|||
this.s3Client = s3Client;
|
||||
this.segmentPusher = segmentPusher;
|
||||
this.dataSegmentKiller = dataSegmentKiller;
|
||||
this.segmentAnnouncer = segmentAnnouncer;
|
||||
this.newSegmentServerView = newSegmentServerView;
|
||||
this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
|
||||
this.objectMapper = objectMapper;
|
||||
}
|
||||
|
||||
|
@ -100,6 +112,21 @@ public class TaskToolbox
|
|||
return dataSegmentKiller;
|
||||
}
|
||||
|
||||
public SegmentAnnouncer getSegmentAnnouncer()
|
||||
{
|
||||
return segmentAnnouncer;
|
||||
}
|
||||
|
||||
public MutableServerView getNewSegmentServerView()
|
||||
{
|
||||
return newSegmentServerView;
|
||||
}
|
||||
|
||||
public QueryRunnerFactoryConglomerate getQueryRunnerFactoryConglomerate()
|
||||
{
|
||||
return queryRunnerFactoryConglomerate;
|
||||
}
|
||||
|
||||
public ObjectMapper getObjectMapper()
|
||||
{
|
||||
return objectMapper;
|
||||
|
|
|
@ -20,11 +20,14 @@
|
|||
package com.metamx.druid.merger.common;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.metamx.druid.client.MutableServerView;
|
||||
import com.metamx.druid.loading.DataSegmentPusher;
|
||||
import com.metamx.druid.loading.DataSegmentKiller;
|
||||
import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
|
||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import com.metamx.druid.realtime.SegmentAnnouncer;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||
|
||||
|
@ -39,6 +42,9 @@ public class TaskToolboxFactory
|
|||
private final RestS3Service s3Client;
|
||||
private final DataSegmentPusher segmentPusher;
|
||||
private final DataSegmentKiller dataSegmentKiller;
|
||||
private final SegmentAnnouncer segmentAnnouncer;
|
||||
private final MutableServerView newSegmentServerView;
|
||||
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
|
||||
private final ObjectMapper objectMapper;
|
||||
|
||||
public TaskToolboxFactory(
|
||||
|
@ -48,6 +54,9 @@ public class TaskToolboxFactory
|
|||
RestS3Service s3Client,
|
||||
DataSegmentPusher segmentPusher,
|
||||
DataSegmentKiller dataSegmentKiller,
|
||||
SegmentAnnouncer segmentAnnouncer,
|
||||
MutableServerView newSegmentServerView,
|
||||
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
|
||||
ObjectMapper objectMapper
|
||||
)
|
||||
{
|
||||
|
@ -57,6 +66,9 @@ public class TaskToolboxFactory
|
|||
this.s3Client = s3Client;
|
||||
this.segmentPusher = segmentPusher;
|
||||
this.dataSegmentKiller = dataSegmentKiller;
|
||||
this.segmentAnnouncer = segmentAnnouncer;
|
||||
this.newSegmentServerView = newSegmentServerView;
|
||||
this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
|
||||
this.objectMapper = objectMapper;
|
||||
}
|
||||
|
||||
|
@ -75,6 +87,9 @@ public class TaskToolboxFactory
|
|||
s3Client,
|
||||
segmentPusher,
|
||||
dataSegmentKiller,
|
||||
segmentAnnouncer,
|
||||
newSegmentServerView,
|
||||
queryRunnerFactoryConglomerate,
|
||||
objectMapper
|
||||
);
|
||||
}
|
||||
|
|
|
@ -4,6 +4,8 @@ import com.metamx.druid.merger.common.task.Task;
|
|||
import com.metamx.druid.merger.coordinator.TaskStorage;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class LocalTaskActionClient implements TaskActionClient
|
||||
{
|
||||
private final Task task;
|
||||
|
@ -20,7 +22,7 @@ public class LocalTaskActionClient implements TaskActionClient
|
|||
}
|
||||
|
||||
@Override
|
||||
public <RetType> RetType submit(TaskAction<RetType> taskAction)
|
||||
public <RetType> RetType submit(TaskAction<RetType> taskAction) throws IOException
|
||||
{
|
||||
final RetType ret = taskAction.perform(task, toolbox);
|
||||
|
||||
|
|
|
@ -1,15 +1,14 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
|
||||
public class LockAcquireAction implements TaskAction<TaskLock>
|
||||
{
|
||||
private final Interval interval;
|
||||
|
||||
|
@ -27,18 +26,29 @@ public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
|
|||
return interval;
|
||||
}
|
||||
|
||||
public TypeReference<Optional<TaskLock>> getReturnTypeReference()
|
||||
public TypeReference<TaskLock> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Optional<TaskLock>>() {};
|
||||
return new TypeReference<TaskLock>()
|
||||
{
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<TaskLock> perform(Task task, TaskActionToolbox toolbox)
|
||||
public TaskLock perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
return toolbox.getTaskLockbox().tryLock(task, interval);
|
||||
} catch (Exception e) {
|
||||
return toolbox.getTaskLockbox().lock(task, interval);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "LockAcquireAction{" +
|
||||
"interval=" + interval +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,10 +20,12 @@ public class LockListAction implements TaskAction<List<TaskLock>>
|
|||
@Override
|
||||
public List<TaskLock> perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
return toolbox.getTaskLockbox().findLocksForTask(task);
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
return toolbox.getTaskLockbox().findLocksForTask(task);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "LockListAction{}";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,11 +36,15 @@ public class LockReleaseAction implements TaskAction<Void>
|
|||
@Override
|
||||
public Void perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
toolbox.getTaskLockbox().unlock(task, interval);
|
||||
return null;
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
toolbox.getTaskLockbox().unlock(task, interval);
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "LockReleaseAction{" +
|
||||
"interval=" + interval +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -11,9 +11,10 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.netflix.curator.x.discovery.ServiceInstance;
|
||||
import com.netflix.curator.x.discovery.ServiceProvider;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
public class RemoteTaskActionClient implements TaskActionClient
|
||||
{
|
||||
|
@ -33,26 +34,37 @@ public class RemoteTaskActionClient implements TaskActionClient
|
|||
}
|
||||
|
||||
@Override
|
||||
public <RetType> RetType submit(TaskAction<RetType> taskAction)
|
||||
public <RetType> RetType submit(TaskAction<RetType> taskAction) throws IOException
|
||||
{
|
||||
byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction));
|
||||
|
||||
final URI serviceUri;
|
||||
try {
|
||||
byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction));
|
||||
|
||||
final String response = httpClient.post(getServiceUri().toURL())
|
||||
.setContent("application/json", dataToSend)
|
||||
.go(new ToStringResponseHandler(Charsets.UTF_8))
|
||||
.get();
|
||||
|
||||
final Map<String, Object> responseDict = jsonMapper.readValue(
|
||||
response,
|
||||
new TypeReference<Map<String, Object>>() {}
|
||||
);
|
||||
|
||||
return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference());
|
||||
serviceUri = getServiceUri();
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new IOException("Failed to locate service uri", e);
|
||||
}
|
||||
|
||||
final String response;
|
||||
|
||||
try {
|
||||
response = httpClient.post(serviceUri.toURL())
|
||||
.setContent("application/json", dataToSend)
|
||||
.go(new ToStringResponseHandler(Charsets.UTF_8))
|
||||
.get();
|
||||
}
|
||||
catch (Exception e) {
|
||||
Throwables.propagateIfInstanceOf(e.getCause(), IOException.class);
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
final Map<String, Object> responseDict = jsonMapper.readValue(
|
||||
response,
|
||||
new TypeReference<Map<String, Object>>() {}
|
||||
);
|
||||
|
||||
return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference());
|
||||
}
|
||||
|
||||
private URI getServiceUri() throws Exception
|
||||
|
|
|
@ -1,22 +1,18 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
|
||||
import java.util.List;
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
public class SegmentInsertAction implements TaskAction<Void>
|
||||
public class SegmentInsertAction implements TaskAction<Set<DataSegment>>
|
||||
{
|
||||
private final Set<DataSegment> segments;
|
||||
|
||||
|
@ -34,34 +30,38 @@ public class SegmentInsertAction implements TaskAction<Void>
|
|||
return segments;
|
||||
}
|
||||
|
||||
public TypeReference<Void> getReturnTypeReference()
|
||||
public TypeReference<Set<DataSegment>> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Void>() {};
|
||||
return new TypeReference<Set<DataSegment>>() {};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Void perform(Task task, TaskActionToolbox toolbox)
|
||||
public Set<DataSegment> perform(Task task, TaskActionToolbox toolbox) throws IOException
|
||||
{
|
||||
if(!toolbox.taskLockCoversSegments(task, segments, false)) {
|
||||
throw new ISE("Segments not covered by locks for task: %s", task.getId());
|
||||
throw new ISE("Segments not covered by locks for task[%s]: %s", task.getId(), segments);
|
||||
}
|
||||
|
||||
try {
|
||||
toolbox.getMergerDBCoordinator().announceHistoricalSegments(segments);
|
||||
final Set<DataSegment> retVal = toolbox.getMergerDBCoordinator().announceHistoricalSegments(segments);
|
||||
|
||||
// Emit metrics
|
||||
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
|
||||
.setUser2(task.getDataSource())
|
||||
.setUser4(task.getType());
|
||||
// Emit metrics
|
||||
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
|
||||
.setUser2(task.getDataSource())
|
||||
.setUser4(task.getType());
|
||||
|
||||
for (DataSegment segment : segments) {
|
||||
metricBuilder.setUser5(segment.getInterval().toString());
|
||||
toolbox.getEmitter().emit(metricBuilder.build("indexer/segment/bytes", segment.getSize()));
|
||||
}
|
||||
|
||||
return null;
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
for (DataSegment segment : segments) {
|
||||
metricBuilder.setUser5(segment.getInterval().toString());
|
||||
toolbox.getEmitter().emit(metricBuilder.build("indexer/segment/bytes", segment.getSize()));
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "SegmentInsertAction{" +
|
||||
"segments=" + segments +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,13 +1,13 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
|
||||
|
@ -43,12 +43,17 @@ public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox)
|
||||
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox) throws IOException
|
||||
{
|
||||
try {
|
||||
return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval);
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "SegmentListUnusedAction{" +
|
||||
"dataSource='" + dataSource + '\'' +
|
||||
", interval=" + interval +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,13 +1,13 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
|
||||
|
@ -43,12 +43,17 @@ public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox)
|
||||
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox) throws IOException
|
||||
{
|
||||
try {
|
||||
return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval);
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "SegmentListUsedAction{" +
|
||||
"dataSource='" + dataSource + '\'' +
|
||||
", interval=" + interval +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,19 +1,15 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
|
||||
import java.util.List;
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
public class SegmentNukeAction implements TaskAction<Void>
|
||||
|
@ -40,28 +36,32 @@ public class SegmentNukeAction implements TaskAction<Void>
|
|||
}
|
||||
|
||||
@Override
|
||||
public Void perform(Task task, TaskActionToolbox toolbox)
|
||||
public Void perform(Task task, TaskActionToolbox toolbox) throws IOException
|
||||
{
|
||||
if(!toolbox.taskLockCoversSegments(task, segments, true)) {
|
||||
throw new ISE("Segments not covered by locks for task: %s", task.getId());
|
||||
}
|
||||
|
||||
try {
|
||||
toolbox.getMergerDBCoordinator().deleteSegments(segments);
|
||||
toolbox.getMergerDBCoordinator().deleteSegments(segments);
|
||||
|
||||
// Emit metrics
|
||||
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
|
||||
.setUser2(task.getDataSource())
|
||||
.setUser4(task.getType());
|
||||
// Emit metrics
|
||||
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
|
||||
.setUser2(task.getDataSource())
|
||||
.setUser4(task.getType());
|
||||
|
||||
for (DataSegment segment : segments) {
|
||||
metricBuilder.setUser5(segment.getInterval().toString());
|
||||
toolbox.getEmitter().emit(metricBuilder.build("indexer/segmentNuked/bytes", segment.getSize()));
|
||||
}
|
||||
|
||||
return null;
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
for (DataSegment segment : segments) {
|
||||
metricBuilder.setUser5(segment.getInterval().toString());
|
||||
toolbox.getEmitter().emit(metricBuilder.build("indexer/segmentNuked/bytes", segment.getSize()));
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "SegmentNukeAction{" +
|
||||
"segments=" + segments +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,11 +1,10 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
|
@ -35,14 +34,18 @@ public class SpawnTasksAction implements TaskAction<Void>
|
|||
@Override
|
||||
public Void perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
for(final Task newTask : newTasks) {
|
||||
toolbox.getTaskQueue().add(newTask);
|
||||
}
|
||||
|
||||
return null;
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
for(final Task newTask : newTasks) {
|
||||
toolbox.getTaskQueue().add(newTask);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "SpawnTasksAction{" +
|
||||
"newTasks=" + newTasks +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,6 +5,8 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
|
|||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "lockAcquire", value = LockAcquireAction.class),
|
||||
|
@ -19,5 +21,5 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
|||
public interface TaskAction<RetType>
|
||||
{
|
||||
public TypeReference<RetType> getReturnTypeReference(); // T_T
|
||||
public RetType perform(Task task, TaskActionToolbox toolbox);
|
||||
public RetType perform(Task task, TaskActionToolbox toolbox) throws IOException;
|
||||
}
|
||||
|
|
|
@ -1,6 +1,8 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public interface TaskActionClient
|
||||
{
|
||||
public <RetType> RetType submit(TaskAction<RetType> taskAction);
|
||||
public <RetType> RetType submit(TaskAction<RetType> taskAction) throws IOException;
|
||||
}
|
||||
|
|
|
@ -84,16 +84,22 @@ public class YeOldePlumberSchool implements PlumberSchool
|
|||
public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics)
|
||||
{
|
||||
// There can be only one.
|
||||
final Sink theSink = new Sink(interval, schema);
|
||||
final Sink theSink = new Sink(interval, schema, version);
|
||||
|
||||
// Temporary directory to hold spilled segments.
|
||||
final File persistDir = new File(tmpSegmentDir, theSink.getSegment().withVersion(version).getIdentifier());
|
||||
final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getIdentifier());
|
||||
|
||||
// Set of spilled segments. Will be merged at the end.
|
||||
final Set<File> spilled = Sets.newHashSet();
|
||||
|
||||
return new Plumber()
|
||||
{
|
||||
@Override
|
||||
public void startJob()
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sink getSink(long timestamp)
|
||||
{
|
||||
|
@ -146,7 +152,6 @@ public class YeOldePlumberSchool implements PlumberSchool
|
|||
|
||||
final DataSegment segmentToUpload = theSink.getSegment()
|
||||
.withDimensions(ImmutableList.copyOf(mappedSegment.getAvailableDimensions()))
|
||||
.withVersion(version)
|
||||
.withBinaryVersion(IndexIO.getVersionFromDir(fileToUpload));
|
||||
|
||||
dataSegmentPusher.push(fileToUpload, segmentToUpload);
|
||||
|
|
|
@ -24,9 +24,11 @@ import com.google.common.base.Joiner;
|
|||
import com.google.common.base.Objects;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.actions.SegmentListUsedAction;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
public abstract class AbstractTask implements Task
|
||||
|
@ -79,6 +81,12 @@ public abstract class AbstractTask implements Task
|
|||
return interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunner(Query<T> query)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.base.Joiner;
|
|||
import com.google.common.base.Objects;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -48,6 +49,7 @@ import org.joda.time.Interval;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -181,38 +183,42 @@ public abstract class MergeTaskBase extends AbstractTask
|
|||
@Override
|
||||
public TaskStatus preflight(TaskToolbox toolbox)
|
||||
{
|
||||
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(DataSegment dataSegment)
|
||||
try {
|
||||
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
|
||||
{
|
||||
return dataSegment.getIdentifier();
|
||||
@Override
|
||||
public String apply(DataSegment dataSegment)
|
||||
{
|
||||
return dataSegment.getIdentifier();
|
||||
}
|
||||
};
|
||||
|
||||
final Set<String> current = ImmutableSet.copyOf(
|
||||
Iterables.transform(toolbox.getTaskActionClient().submit(defaultListUsedAction()), toIdentifier)
|
||||
);
|
||||
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
|
||||
|
||||
final Set<String> missingFromRequested = Sets.difference(current, requested);
|
||||
if (!missingFromRequested.isEmpty()) {
|
||||
throw new ISE(
|
||||
"Merge is invalid: current segment(s) are not in the requested set: %s",
|
||||
Joiner.on(", ").join(missingFromRequested)
|
||||
);
|
||||
}
|
||||
};
|
||||
|
||||
final Set<String> current = ImmutableSet.copyOf(
|
||||
Iterables.transform(toolbox.getTaskActionClient().submit(defaultListUsedAction()), toIdentifier)
|
||||
);
|
||||
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
|
||||
final Set<String> missingFromCurrent = Sets.difference(requested, current);
|
||||
if (!missingFromCurrent.isEmpty()) {
|
||||
throw new ISE(
|
||||
"Merge is invalid: requested segment(s) are not in the current set: %s",
|
||||
Joiner.on(", ").join(missingFromCurrent)
|
||||
);
|
||||
}
|
||||
|
||||
final Set<String> missingFromRequested = Sets.difference(current, requested);
|
||||
if (!missingFromRequested.isEmpty()) {
|
||||
throw new ISE(
|
||||
"Merge is invalid: current segment(s) are not in the requested set: %s",
|
||||
Joiner.on(", ").join(missingFromRequested)
|
||||
);
|
||||
return TaskStatus.running(getId());
|
||||
}
|
||||
|
||||
final Set<String> missingFromCurrent = Sets.difference(requested, current);
|
||||
if (!missingFromCurrent.isEmpty()) {
|
||||
throw new ISE(
|
||||
"Merge is invalid: requested segment(s) are not in the current set: %s",
|
||||
Joiner.on(", ").join(missingFromCurrent)
|
||||
);
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
return TaskStatus.running(getId());
|
||||
|
||||
}
|
||||
|
||||
protected abstract File merge(Map<DataSegment, File> segments, File outDir)
|
||||
|
@ -270,12 +276,12 @@ public abstract class MergeTaskBase extends AbstractTask
|
|||
DateTime start = null;
|
||||
DateTime end = null;
|
||||
|
||||
for(final DataSegment segment : segments) {
|
||||
if(start == null || segment.getInterval().getStart().isBefore(start)) {
|
||||
for (final DataSegment segment : segments) {
|
||||
if (start == null || segment.getInterval().getStart().isBefore(start)) {
|
||||
start = segment.getInterval().getStart();
|
||||
}
|
||||
|
||||
if(end == null || segment.getInterval().getEnd().isAfter(end)) {
|
||||
if (end == null || segment.getInterval().getEnd().isAfter(end)) {
|
||||
end = segment.getInterval().getEnd();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,257 @@
|
|||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.index.v1.IndexGranularity;
|
||||
import com.metamx.druid.input.InputRow;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.actions.LockAcquireAction;
|
||||
import com.metamx.druid.merger.common.actions.LockReleaseAction;
|
||||
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.realtime.FireDepartmentConfig;
|
||||
import com.metamx.druid.realtime.FireDepartmentMetrics;
|
||||
import com.metamx.druid.realtime.Firehose;
|
||||
import com.metamx.druid.realtime.FirehoseFactory;
|
||||
import com.metamx.druid.realtime.MetadataUpdater;
|
||||
import com.metamx.druid.realtime.Plumber;
|
||||
import com.metamx.druid.realtime.RealtimePlumberSchool;
|
||||
import com.metamx.druid.realtime.Schema;
|
||||
import com.metamx.druid.realtime.SegmentAnnouncer;
|
||||
import com.metamx.druid.realtime.SegmentPublisher;
|
||||
import com.metamx.druid.realtime.Sink;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Period;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
||||
public class RealtimeIndexTask extends AbstractTask
|
||||
{
|
||||
@JsonProperty
|
||||
final Schema schema;
|
||||
|
||||
@JsonProperty
|
||||
final FirehoseFactory firehoseFactory;
|
||||
|
||||
@JsonProperty
|
||||
final FireDepartmentConfig fireDepartmentConfig;
|
||||
|
||||
@JsonProperty
|
||||
final Period windowPeriod;
|
||||
|
||||
@JsonProperty
|
||||
final IndexGranularity segmentGranularity;
|
||||
|
||||
private volatile Plumber plumber = null;
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class);
|
||||
|
||||
@JsonCreator
|
||||
public RealtimeIndexTask(
|
||||
@JsonProperty("schema") Schema schema,
|
||||
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
|
||||
@JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig, // TODO rename?
|
||||
@JsonProperty("windowPeriod") Period windowPeriod,
|
||||
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity
|
||||
)
|
||||
{
|
||||
super(
|
||||
String.format(
|
||||
"index_realtime_%s_%d_%s",
|
||||
schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime()
|
||||
),
|
||||
String.format(
|
||||
"index_realtime_%s",
|
||||
schema.getDataSource()
|
||||
),
|
||||
schema.getDataSource(),
|
||||
null
|
||||
);
|
||||
|
||||
this.schema = schema;
|
||||
this.firehoseFactory = firehoseFactory;
|
||||
this.fireDepartmentConfig = fireDepartmentConfig;
|
||||
this.windowPeriod = windowPeriod;
|
||||
this.segmentGranularity = segmentGranularity;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "index_realtime";
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunner(Query<T> query)
|
||||
{
|
||||
if (plumber != null) {
|
||||
return plumber.getQueryRunner(query);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(final TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
if (this.plumber != null) {
|
||||
throw new IllegalStateException("WTF?!? run with non-null plumber??!");
|
||||
}
|
||||
|
||||
boolean normalExit = true;
|
||||
|
||||
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
|
||||
final Period intermediatePersistPeriod = fireDepartmentConfig.getIntermediatePersistPeriod();
|
||||
final Firehose firehose = firehoseFactory.connect();
|
||||
|
||||
// TODO Take PlumberSchool in constructor (although that will need jackson injectables for stuff like
|
||||
// TODO the ServerView, which seems kind of odd?)
|
||||
final RealtimePlumberSchool realtimePlumberSchool = new RealtimePlumberSchool(
|
||||
windowPeriod,
|
||||
new File(toolbox.getTaskDir(), "persist"),
|
||||
segmentGranularity
|
||||
);
|
||||
|
||||
final SegmentPublisher segmentPublisher = new TaskActionSegmentPublisher(this, toolbox);
|
||||
|
||||
// Wrap default SegmentAnnouncer such that we unlock intervals as we unannounce segments
|
||||
final SegmentAnnouncer lockingSegmentAnnouncer = new SegmentAnnouncer()
|
||||
{
|
||||
@Override
|
||||
public void announceSegment(final DataSegment segment) throws IOException
|
||||
{
|
||||
toolbox.getSegmentAnnouncer().announceSegment(segment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unannounceSegment(final DataSegment segment) throws IOException
|
||||
{
|
||||
try {
|
||||
toolbox.getSegmentAnnouncer().unannounceSegment(segment);
|
||||
} finally {
|
||||
toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval()));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// TODO -- This can block if there is lock contention, which will block plumber.getSink (and thus the firehose)
|
||||
// TODO -- Shouldn't usually be bad, since we don't expect people to submit tasks that intersect with the
|
||||
// TODO -- realtime window, but if they do it can be problematic
|
||||
final RealtimePlumberSchool.VersioningPolicy versioningPolicy = new RealtimePlumberSchool.VersioningPolicy()
|
||||
{
|
||||
@Override
|
||||
public String getVersion(final Interval interval)
|
||||
{
|
||||
try {
|
||||
// NOTE: Side effect: Calling getVersion causes a lock to be acquired
|
||||
final TaskLock myLock = toolbox.getTaskActionClient()
|
||||
.submit(new LockAcquireAction(interval));
|
||||
|
||||
return myLock.getVersion();
|
||||
} catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// TODO - Might need to have task id in segmentPusher path or some other way of making redundant realtime
|
||||
// TODO - workers not step on each other's toes when pushing segments to S3
|
||||
realtimePlumberSchool.setDataSegmentPusher(toolbox.getSegmentPusher());
|
||||
realtimePlumberSchool.setConglomerate(toolbox.getQueryRunnerFactoryConglomerate());
|
||||
realtimePlumberSchool.setVersioningPolicy(versioningPolicy);
|
||||
realtimePlumberSchool.setMetadataUpdater(new MetadataUpdater(lockingSegmentAnnouncer, segmentPublisher));
|
||||
realtimePlumberSchool.setServerView(toolbox.getNewSegmentServerView());
|
||||
realtimePlumberSchool.setServiceEmitter(toolbox.getEmitter());
|
||||
|
||||
this.plumber = realtimePlumberSchool.findPlumber(schema, metrics);
|
||||
|
||||
try {
|
||||
plumber.startJob();
|
||||
|
||||
long nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
||||
while (firehose.hasMore()) {
|
||||
final InputRow inputRow;
|
||||
try {
|
||||
inputRow = firehose.nextRow();
|
||||
|
||||
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
|
||||
if (sink == null) {
|
||||
metrics.incrementThrownAway();
|
||||
log.debug("Throwing away event[%s]", inputRow);
|
||||
|
||||
if (System.currentTimeMillis() > nextFlush) {
|
||||
plumber.persist(firehose.commit());
|
||||
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
||||
}
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
if (sink.isEmpty()) {
|
||||
log.info("Task %s: New sink: %s", getId(), sink);
|
||||
}
|
||||
|
||||
int currCount = sink.add(inputRow);
|
||||
metrics.incrementProcessed();
|
||||
if (currCount >= fireDepartmentConfig.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) {
|
||||
plumber.persist(firehose.commit());
|
||||
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
||||
}
|
||||
}
|
||||
catch (FormattedException e) {
|
||||
log.warn(e, "unparseable line");
|
||||
metrics.incrementUnparseable();
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert(e, "Exception aborted realtime processing[%s]", schema.getDataSource())
|
||||
.emit();
|
||||
normalExit = false;
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(firehose);
|
||||
|
||||
if (normalExit) {
|
||||
try {
|
||||
plumber.persist(firehose.commit());
|
||||
plumber.finishJob();
|
||||
} catch(Exception e) {
|
||||
log.makeAlert(e, "Failed to finish realtime task").emit();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
|
||||
public static class TaskActionSegmentPublisher implements SegmentPublisher
|
||||
{
|
||||
final Task task;
|
||||
final TaskToolbox taskToolbox;
|
||||
|
||||
public TaskActionSegmentPublisher(Task task, TaskToolbox taskToolbox)
|
||||
{
|
||||
this.task = task;
|
||||
this.taskToolbox = taskToolbox;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void publishSegment(DataSegment segment) throws IOException
|
||||
{
|
||||
taskToolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment)));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -22,8 +22,10 @@ package com.metamx.druid.merger.common.task;
|
|||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.google.common.base.Optional;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
/**
|
||||
|
@ -51,6 +53,7 @@ import org.joda.time.Interval;
|
|||
@JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class),
|
||||
@JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class),
|
||||
@JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class),
|
||||
@JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class),
|
||||
@JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class),
|
||||
@JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class)
|
||||
})
|
||||
|
@ -83,6 +86,12 @@ public interface Task
|
|||
*/
|
||||
public Optional<Interval> getImplicitLockInterval();
|
||||
|
||||
/**
|
||||
* Returns query runners for this task. If this task is not meant to answer queries over its datasource, this method
|
||||
* should return null.
|
||||
*/
|
||||
public <T> QueryRunner<T> getQueryRunner(Query<T> query);
|
||||
|
||||
/**
|
||||
* Execute preflight checks for a task. This typically runs on the coordinator, and will be run while
|
||||
* holding a lock on our dataSource and implicit lock interval (if any). If this method throws an exception, the
|
||||
|
|
|
@ -22,8 +22,10 @@ package com.metamx.druid.merger.coordinator;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.TimelineObjectHolder;
|
||||
import com.metamx.druid.VersionedIntervalTimeline;
|
||||
|
@ -71,13 +73,11 @@ public class MergerDBCoordinator
|
|||
public List<DataSegment> getUsedSegmentsForInterval(final String dataSource, final Interval interval)
|
||||
throws IOException
|
||||
{
|
||||
// XXX Could be reading from a cache if we can assume we're the only one editing the DB
|
||||
|
||||
final VersionedIntervalTimeline<String, DataSegment> timeline = dbi.withHandle(
|
||||
new HandleCallback<VersionedIntervalTimeline<String, DataSegment>>()
|
||||
{
|
||||
@Override
|
||||
public VersionedIntervalTimeline<String, DataSegment> withHandle(Handle handle) throws Exception
|
||||
public VersionedIntervalTimeline<String, DataSegment> withHandle(Handle handle) throws IOException
|
||||
{
|
||||
final VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<String, DataSegment>(
|
||||
Ordering.natural()
|
||||
|
@ -129,31 +129,47 @@ public class MergerDBCoordinator
|
|||
return segments;
|
||||
}
|
||||
|
||||
public void announceHistoricalSegments(final Set<DataSegment> segments) throws Exception
|
||||
/**
|
||||
* Attempts to insert a set of segments to the database. Returns the set of segments actually added (segments
|
||||
* with identifiers already in the database will not be added).
|
||||
*
|
||||
* @param segments set of segments to add
|
||||
* @return set of segments actually added
|
||||
*/
|
||||
public Set<DataSegment> announceHistoricalSegments(final Set<DataSegment> segments) throws IOException
|
||||
{
|
||||
dbi.inTransaction(
|
||||
new TransactionCallback<Void>()
|
||||
return dbi.inTransaction(
|
||||
new TransactionCallback<Set<DataSegment>>()
|
||||
{
|
||||
@Override
|
||||
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception
|
||||
public Set<DataSegment> inTransaction(Handle handle, TransactionStatus transactionStatus) throws IOException
|
||||
{
|
||||
for(final DataSegment segment : segments) {
|
||||
announceHistoricalSegment(handle, segment);
|
||||
final Set<DataSegment> inserted = Sets.newHashSet();
|
||||
|
||||
for (final DataSegment segment : segments) {
|
||||
if (announceHistoricalSegment(handle, segment)) {
|
||||
inserted.add(segment);
|
||||
}
|
||||
}
|
||||
|
||||
return null;
|
||||
return ImmutableSet.copyOf(inserted);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
private void announceHistoricalSegment(final Handle handle, final DataSegment segment) throws Exception
|
||||
/**
|
||||
* Attempts to insert a single segment to the database. If the segment already exists, will do nothing. Meant
|
||||
* to be called from within a transaction.
|
||||
*
|
||||
* @return true if the segment was added, false otherwise
|
||||
*/
|
||||
private boolean announceHistoricalSegment(final Handle handle, final DataSegment segment) throws IOException
|
||||
{
|
||||
try {
|
||||
final List<Map<String, Object>> exists = handle.createQuery(
|
||||
String.format(
|
||||
"SELECT id FROM %s WHERE id = ':identifier'",
|
||||
"SELECT id FROM %s WHERE id = :identifier",
|
||||
dbConnectorConfig.getSegmentTable()
|
||||
)
|
||||
).bind(
|
||||
|
@ -163,7 +179,7 @@ public class MergerDBCoordinator
|
|||
|
||||
if (!exists.isEmpty()) {
|
||||
log.info("Found [%s] in DB, not updating DB", segment.getIdentifier());
|
||||
return;
|
||||
return false;
|
||||
}
|
||||
|
||||
handle.createStatement(
|
||||
|
@ -185,19 +201,21 @@ public class MergerDBCoordinator
|
|||
|
||||
log.info("Published segment [%s] to DB", segment.getIdentifier());
|
||||
}
|
||||
catch (Exception e) {
|
||||
catch (IOException e) {
|
||||
log.error(e, "Exception inserting into DB");
|
||||
throw e;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
public void deleteSegments(final Set<DataSegment> segments) throws Exception
|
||||
public void deleteSegments(final Set<DataSegment> segments) throws IOException
|
||||
{
|
||||
dbi.inTransaction(
|
||||
new TransactionCallback<Void>()
|
||||
{
|
||||
@Override
|
||||
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception
|
||||
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws IOException
|
||||
{
|
||||
for(final DataSegment segment : segments) {
|
||||
deleteSegment(handle, segment);
|
||||
|
@ -223,7 +241,7 @@ public class MergerDBCoordinator
|
|||
new HandleCallback<List<DataSegment>>()
|
||||
{
|
||||
@Override
|
||||
public List<DataSegment> withHandle(Handle handle) throws Exception
|
||||
public List<DataSegment> withHandle(Handle handle) throws IOException
|
||||
{
|
||||
return handle.createQuery(
|
||||
String.format(
|
||||
|
|
|
@ -45,6 +45,7 @@ import java.util.NavigableMap;
|
|||
import java.util.NavigableSet;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
/**
|
||||
|
@ -58,6 +59,7 @@ public class TaskLockbox
|
|||
private final Map<String, NavigableMap<Interval, TaskLockPosse>> running = Maps.newHashMap();
|
||||
private final TaskStorage taskStorage;
|
||||
private final ReentrantLock giant = new ReentrantLock();
|
||||
private final Condition lockReleaseCondition = giant.newCondition();
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(TaskLockbox.class);
|
||||
|
||||
|
@ -66,6 +68,27 @@ public class TaskLockbox
|
|||
this.taskStorage = taskStorage;
|
||||
}
|
||||
|
||||
/**
|
||||
* Locks a task without removing it from the queue. Blocks until the lock is acquired. Throws an exception
|
||||
* if the lock cannot be acquired.
|
||||
*/
|
||||
public TaskLock lock(final Task task, final Interval interval) throws InterruptedException
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
Optional<TaskLock> taskLock;
|
||||
|
||||
while (!(taskLock = tryLock(task, interval)).isPresent()) {
|
||||
lockReleaseCondition.await();
|
||||
}
|
||||
|
||||
return taskLock.get();
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempt to lock a task, without removing it from the queue. Equivalent to the long form of {@code tryLock}
|
||||
* with no preferred version.
|
||||
|
@ -241,6 +264,9 @@ public class TaskLockbox
|
|||
running.remove(dataSource);
|
||||
}
|
||||
|
||||
// Wake up blocking-lock waiters
|
||||
lockReleaseCondition.signalAll();
|
||||
|
||||
// Best effort to remove lock from storage
|
||||
try {
|
||||
taskStorage.removeLock(task.getId(), taskLock);
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.amazonaws.auth.BasicAWSCredentials;
|
|||
import com.amazonaws.services.ec2.AmazonEC2Client;
|
||||
import com.fasterxml.jackson.databind.InjectableValues;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -38,14 +39,18 @@ import com.metamx.common.lifecycle.Lifecycle;
|
|||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.BaseServerNode;
|
||||
import com.metamx.druid.RegisteringNode;
|
||||
import com.metamx.druid.client.ClientConfig;
|
||||
import com.metamx.druid.client.ClientInventoryManager;
|
||||
import com.metamx.druid.client.MutableServerView;
|
||||
import com.metamx.druid.client.OnlyNewSegmentWatcherServerView;
|
||||
import com.metamx.druid.config.ConfigManager;
|
||||
import com.metamx.druid.config.ConfigManagerConfig;
|
||||
import com.metamx.druid.config.JacksonConfigManager;
|
||||
import com.metamx.druid.db.DbConnector;
|
||||
import com.metamx.druid.db.DbConnectorConfig;
|
||||
import com.metamx.druid.http.GuiceServletConfig;
|
||||
import com.metamx.druid.http.MasterMain;
|
||||
import com.metamx.druid.http.RedirectFilter;
|
||||
import com.metamx.druid.http.RedirectInfo;
|
||||
import com.metamx.druid.http.StatusServlet;
|
||||
|
@ -90,6 +95,10 @@ import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFa
|
|||
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy;
|
||||
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.druid.merger.worker.http.WorkerNode;
|
||||
import com.metamx.druid.realtime.MetadataUpdaterConfig;
|
||||
import com.metamx.druid.realtime.SegmentAnnouncer;
|
||||
import com.metamx.druid.realtime.ZkSegmentAnnouncer;
|
||||
import com.metamx.druid.utils.PropUtils;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.core.Emitters;
|
||||
|
@ -128,7 +137,7 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class IndexerCoordinatorNode extends RegisteringNode
|
||||
public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNode>
|
||||
{
|
||||
private static final Logger log = new Logger(IndexerCoordinatorNode.class);
|
||||
|
||||
|
@ -137,7 +146,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
return new Builder();
|
||||
}
|
||||
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final Lifecycle lifecycle;
|
||||
private final Properties props;
|
||||
private final ConfigurationObjectFactory configFactory;
|
||||
|
@ -161,20 +169,21 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
private TaskRunnerFactory taskRunnerFactory = null;
|
||||
private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null;
|
||||
private TaskMasterLifecycle taskMasterLifecycle = null;
|
||||
private MutableServerView newSegmentServerView = null;
|
||||
private Server server = null;
|
||||
|
||||
private boolean initialized = false;
|
||||
|
||||
public IndexerCoordinatorNode(
|
||||
ObjectMapper jsonMapper,
|
||||
Lifecycle lifecycle,
|
||||
Properties props,
|
||||
Lifecycle lifecycle,
|
||||
ObjectMapper jsonMapper,
|
||||
ObjectMapper smileMapper,
|
||||
ConfigurationObjectFactory configFactory
|
||||
)
|
||||
{
|
||||
super(Arrays.asList(jsonMapper));
|
||||
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
|
||||
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.lifecycle = lifecycle;
|
||||
this.props = props;
|
||||
this.configFactory = configFactory;
|
||||
|
@ -198,6 +207,12 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
return this;
|
||||
}
|
||||
|
||||
public IndexerCoordinatorNode setNewSegmentServerView(MutableServerView newSegmentServerView)
|
||||
{
|
||||
this.newSegmentServerView = newSegmentServerView;
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndexerCoordinatorNode setS3Service(RestS3Service s3Service)
|
||||
{
|
||||
this.s3Service = s3Service;
|
||||
|
@ -240,7 +255,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
return this;
|
||||
}
|
||||
|
||||
public void init() throws Exception
|
||||
public void doInit() throws Exception
|
||||
{
|
||||
scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
|
||||
initializeDB();
|
||||
|
@ -254,7 +269,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
dbi,
|
||||
managerConfig
|
||||
)
|
||||
), jsonMapper
|
||||
), getJsonMapper()
|
||||
);
|
||||
|
||||
initializeEmitter();
|
||||
|
@ -263,6 +278,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
initializeTaskConfig();
|
||||
initializeS3Service();
|
||||
initializeMergeDBCoordinator();
|
||||
initializeNewSegmentServerView();
|
||||
initializeTaskStorage();
|
||||
initializeTaskLockbox();
|
||||
initializeTaskQueue();
|
||||
|
@ -288,7 +304,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
|
||||
final Injector injector = Guice.createInjector(
|
||||
new IndexerCoordinatorServletModule(
|
||||
jsonMapper,
|
||||
getJsonMapper(),
|
||||
config,
|
||||
emitter,
|
||||
taskMasterLifecycle,
|
||||
|
@ -306,6 +322,9 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
});
|
||||
staticContext.setBaseResource(resourceCollection);
|
||||
|
||||
// TODO -- Need a QueryServlet and some kind of QuerySegmentWalker if we want to support querying tasks
|
||||
// TODO -- (e.g. for realtime) in local mode
|
||||
|
||||
final Context root = new Context(server, "/", Context.SESSIONS);
|
||||
root.addServlet(new ServletHolder(new StatusServlet()), "/status");
|
||||
root.addServlet(new ServletHolder(new DefaultServlet()), "/mmx/*");
|
||||
|
@ -419,12 +438,12 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
injectables.addValue("s3Client", s3Service)
|
||||
.addValue("segmentPusher", segmentPusher);
|
||||
|
||||
jsonMapper.setInjectableValues(injectables);
|
||||
getJsonMapper().setInjectableValues(injectables);
|
||||
}
|
||||
|
||||
private void initializeJacksonSubtypes()
|
||||
{
|
||||
jsonMapper.registerSubtypes(StaticS3FirehoseFactory.class);
|
||||
getJsonMapper().registerSubtypes(StaticS3FirehoseFactory.class);
|
||||
}
|
||||
|
||||
private void initializeEmitter()
|
||||
|
@ -437,7 +456,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
emitter = new ServiceEmitter(
|
||||
PropUtils.getProperty(props, "druid.service"),
|
||||
PropUtils.getProperty(props, "druid.host"),
|
||||
Emitters.create(props, httpClient, jsonMapper, lifecycle)
|
||||
Emitters.create(props, httpClient, getJsonMapper(), lifecycle)
|
||||
);
|
||||
}
|
||||
EmittingLogger.registerEmitter(emitter);
|
||||
|
@ -476,6 +495,21 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
}
|
||||
}
|
||||
|
||||
private void initializeNewSegmentServerView()
|
||||
{
|
||||
if (newSegmentServerView == null) {
|
||||
final MutableServerView view = new OnlyNewSegmentWatcherServerView();
|
||||
final ClientInventoryManager clientInventoryManager = new ClientInventoryManager(
|
||||
getConfigFactory().build(ClientConfig.class),
|
||||
getPhoneBook(),
|
||||
view
|
||||
);
|
||||
lifecycle.addManagedInstance(clientInventoryManager);
|
||||
|
||||
this.newSegmentServerView = view;
|
||||
}
|
||||
}
|
||||
|
||||
public void initializeS3Service() throws S3ServiceException
|
||||
{
|
||||
this.s3Service = new RestS3Service(
|
||||
|
@ -489,13 +523,17 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
public void initializeDataSegmentPusher()
|
||||
{
|
||||
if (segmentPusher == null) {
|
||||
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper);
|
||||
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, getJsonMapper());
|
||||
}
|
||||
}
|
||||
|
||||
public void initializeTaskToolbox()
|
||||
{
|
||||
if (taskToolboxFactory == null) {
|
||||
final SegmentAnnouncer segmentAnnouncer = new ZkSegmentAnnouncer(
|
||||
configFactory.build(MetadataUpdaterConfig.class),
|
||||
getPhoneBook()
|
||||
);
|
||||
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
|
||||
taskToolboxFactory = new TaskToolboxFactory(
|
||||
taskConfig,
|
||||
|
@ -507,7 +545,10 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
s3Service,
|
||||
segmentPusher,
|
||||
dataSegmentKiller,
|
||||
jsonMapper
|
||||
segmentAnnouncer,
|
||||
newSegmentServerView,
|
||||
getConglomerate(),
|
||||
getJsonMapper()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -516,7 +557,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
{
|
||||
if (mergerDBCoordinator == null) {
|
||||
mergerDBCoordinator = new MergerDBCoordinator(
|
||||
jsonMapper,
|
||||
getJsonMapper(),
|
||||
dbConnectorConfig,
|
||||
dbi
|
||||
);
|
||||
|
@ -563,7 +604,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
taskStorage = new HeapMemoryTaskStorage();
|
||||
} else if (config.getStorageImpl().equals("db")) {
|
||||
final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class);
|
||||
taskStorage = new DbTaskStorage(jsonMapper, dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI());
|
||||
taskStorage = new DbTaskStorage(getJsonMapper(), dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI());
|
||||
} else {
|
||||
throw new ISE("Invalid storage implementation: %s", config.getStorageImpl());
|
||||
}
|
||||
|
@ -590,7 +631,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
);
|
||||
|
||||
RemoteTaskRunner remoteTaskRunner = new RemoteTaskRunner(
|
||||
jsonMapper,
|
||||
getJsonMapper(),
|
||||
configFactory.build(RemoteTaskRunnerConfig.class),
|
||||
curatorFramework,
|
||||
new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true),
|
||||
|
@ -641,7 +682,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
AutoScalingStrategy strategy;
|
||||
if (config.getStrategyImpl().equalsIgnoreCase("ec2")) {
|
||||
strategy = new EC2AutoScalingStrategy(
|
||||
jsonMapper,
|
||||
getJsonMapper(),
|
||||
new AmazonEC2Client(
|
||||
new BasicAWSCredentials(
|
||||
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
|
||||
|
@ -675,6 +716,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
public static class Builder
|
||||
{
|
||||
private ObjectMapper jsonMapper = null;
|
||||
private ObjectMapper smileMapper = null;
|
||||
private Lifecycle lifecycle = null;
|
||||
private Properties props = null;
|
||||
private ConfigurationObjectFactory configFactory = null;
|
||||
|
@ -705,8 +747,13 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
|
||||
public IndexerCoordinatorNode build()
|
||||
{
|
||||
if (jsonMapper == null) {
|
||||
if (jsonMapper == null && smileMapper == null) {
|
||||
jsonMapper = new DefaultObjectMapper();
|
||||
smileMapper = new DefaultObjectMapper(new SmileFactory());
|
||||
smileMapper.getJsonFactory().setCodec(smileMapper);
|
||||
}
|
||||
else if (jsonMapper == null || smileMapper == null) {
|
||||
throw new ISE("Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", jsonMapper, smileMapper);
|
||||
}
|
||||
|
||||
if (lifecycle == null) {
|
||||
|
@ -721,7 +768,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
configFactory = Config.createFactory(props);
|
||||
}
|
||||
|
||||
return new IndexerCoordinatorNode(jsonMapper, lifecycle, props, configFactory);
|
||||
return new IndexerCoordinatorNode(props, lifecycle, jsonMapper, smileMapper, configFactory);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ import javax.ws.rs.Path;
|
|||
import javax.ws.rs.PathParam;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
@ -184,12 +185,17 @@ public class IndexerCoordinatorResource
|
|||
@Produces("application/json")
|
||||
public <T> Response doAction(final TaskActionHolder<T> holder)
|
||||
{
|
||||
final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask())
|
||||
.getTaskActionClient()
|
||||
.submit(holder.getAction());
|
||||
final Map<String, Object> retMap;
|
||||
|
||||
final Map<String, Object> retMap = Maps.newHashMap();
|
||||
retMap.put("result", ret);
|
||||
try {
|
||||
final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask())
|
||||
.getTaskActionClient()
|
||||
.submit(holder.getAction());
|
||||
retMap = Maps.newHashMap();
|
||||
retMap.put("result", ret);
|
||||
} catch(IOException e) {
|
||||
return Response.serverError().build();
|
||||
}
|
||||
|
||||
return Response.ok().entity(retMap).build();
|
||||
}
|
||||
|
|
|
@ -21,35 +21,47 @@ package com.metamx.druid.merger.worker;
|
|||
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.TaskToolboxFactory;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.query.NoopQueryRunner;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.query.segment.QuerySegmentWalker;
|
||||
import com.metamx.druid.query.segment.SegmentDescriptor;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.netflix.curator.framework.CuratorFramework;
|
||||
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
|
||||
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent;
|
||||
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
/**
|
||||
* The monitor watches ZK at a specified path for new tasks to appear. Upon starting the monitor, a listener will be
|
||||
* created that waits for new tasks. Tasks are executed as soon as they are seen.
|
||||
*
|
||||
* The monitor implements {@link QuerySegmentWalker} so tasks can offer up queryable data. This is useful for
|
||||
* realtime index tasks.
|
||||
*/
|
||||
public class TaskMonitor
|
||||
public class WorkerTaskMonitor implements QuerySegmentWalker
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(TaskMonitor.class);
|
||||
private static final EmittingLogger log = new EmittingLogger(WorkerTaskMonitor.class);
|
||||
|
||||
private final PathChildrenCache pathChildrenCache;
|
||||
private final CuratorFramework cf;
|
||||
private final WorkerCuratorCoordinator workerCuratorCoordinator;
|
||||
private final TaskToolboxFactory toolboxFactory;
|
||||
private final ExecutorService exec;
|
||||
private final List<Task> running = new CopyOnWriteArrayList<Task>();
|
||||
|
||||
public TaskMonitor(
|
||||
public WorkerTaskMonitor(
|
||||
PathChildrenCache pathChildrenCache,
|
||||
CuratorFramework cf,
|
||||
WorkerCuratorCoordinator workerCuratorCoordinator,
|
||||
|
@ -88,7 +100,7 @@ public class TaskMonitor
|
|||
);
|
||||
final TaskToolbox toolbox = toolboxFactory.build(task);
|
||||
|
||||
if (workerCuratorCoordinator.statusExists(task.getId())) {
|
||||
if (isTaskRunning(task)) {
|
||||
log.warn("Got task %s that I am already running...", task.getId());
|
||||
workerCuratorCoordinator.unannounceTask(task.getId());
|
||||
return;
|
||||
|
@ -104,6 +116,7 @@ public class TaskMonitor
|
|||
final File taskDir = toolbox.getTaskDir();
|
||||
|
||||
log.info("Running task [%s]", task.getId());
|
||||
running.add(task);
|
||||
|
||||
TaskStatus taskStatus;
|
||||
try {
|
||||
|
@ -116,6 +129,8 @@ public class TaskMonitor
|
|||
.addData("task", task.getId())
|
||||
.emit();
|
||||
taskStatus = TaskStatus.failure(task.getId());
|
||||
} finally {
|
||||
running.remove(task);
|
||||
}
|
||||
|
||||
taskStatus = taskStatus.withDuration(System.currentTimeMillis() - startTime);
|
||||
|
@ -151,12 +166,23 @@ public class TaskMonitor
|
|||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert(e, "Exception starting TaskMonitor")
|
||||
log.makeAlert(e, "Exception starting WorkerTaskMonitor")
|
||||
.addData("exception", e.toString())
|
||||
.emit();
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isTaskRunning(final Task task)
|
||||
{
|
||||
for (final Task runningTask : running) {
|
||||
if (runningTask.equals(task.getId())) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
@LifecycleStop
|
||||
public void stop()
|
||||
{
|
||||
|
@ -165,9 +191,43 @@ public class TaskMonitor
|
|||
exec.shutdown();
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert(e, "Exception stopping TaskMonitor")
|
||||
log.makeAlert(e, "Exception stopping WorkerTaskMonitor")
|
||||
.addData("exception", e.toString())
|
||||
.emit();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<Interval> intervals)
|
||||
{
|
||||
return getQueryRunnerImpl(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunnerForSegments(Query<T> query, Iterable<SegmentDescriptor> specs)
|
||||
{
|
||||
return getQueryRunnerImpl(query);
|
||||
}
|
||||
|
||||
private <T> QueryRunner<T> getQueryRunnerImpl(Query<T> query) {
|
||||
QueryRunner<T> queryRunner = null;
|
||||
|
||||
for (final Task task : running) {
|
||||
if (task.getDataSource().equals(query.getDataSource())) {
|
||||
final QueryRunner<T> taskQueryRunner = task.getQueryRunner(query);
|
||||
|
||||
if (taskQueryRunner != null) {
|
||||
if (queryRunner == null) {
|
||||
queryRunner = taskQueryRunner;
|
||||
} else {
|
||||
log.makeAlert("Found too many query runners for datasource")
|
||||
.addData("dataSource", query.getDataSource())
|
||||
.emit();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return queryRunner == null ? new NoopQueryRunner<T>() : queryRunner;
|
||||
}
|
||||
}
|
|
@ -21,16 +21,23 @@ package com.metamx.druid.merger.worker.http;
|
|||
|
||||
import com.fasterxml.jackson.databind.InjectableValues;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.inject.servlet.GuiceFilter;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.concurrent.ScheduledExecutorFactory;
|
||||
import com.metamx.common.concurrent.ScheduledExecutors;
|
||||
import com.metamx.common.config.Config;
|
||||
import com.metamx.common.lifecycle.Lifecycle;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.RegisteringNode;
|
||||
import com.metamx.druid.BaseServerNode;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.client.ClientConfig;
|
||||
import com.metamx.druid.client.ClientInventoryManager;
|
||||
import com.metamx.druid.client.MutableServerView;
|
||||
import com.metamx.druid.client.OnlyNewSegmentWatcherServerView;
|
||||
import com.metamx.druid.http.QueryServlet;
|
||||
import com.metamx.druid.http.StatusServlet;
|
||||
import com.metamx.druid.initialization.CuratorConfig;
|
||||
import com.metamx.druid.initialization.Initialization;
|
||||
|
@ -46,10 +53,18 @@ import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory;
|
|||
import com.metamx.druid.merger.common.config.IndexerZkConfig;
|
||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
|
||||
import com.metamx.druid.merger.worker.TaskMonitor;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.worker.WorkerTaskMonitor;
|
||||
import com.metamx.druid.merger.worker.Worker;
|
||||
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
|
||||
import com.metamx.druid.merger.worker.config.WorkerConfig;
|
||||
import com.metamx.druid.query.NoopQueryRunner;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.query.segment.QuerySegmentWalker;
|
||||
import com.metamx.druid.query.segment.SegmentDescriptor;
|
||||
import com.metamx.druid.realtime.MetadataUpdaterConfig;
|
||||
import com.metamx.druid.realtime.SegmentAnnouncer;
|
||||
import com.metamx.druid.realtime.ZkSegmentAnnouncer;
|
||||
import com.metamx.druid.utils.PropUtils;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.core.Emitters;
|
||||
|
@ -69,6 +84,7 @@ import com.netflix.curator.x.discovery.ServiceProvider;
|
|||
import org.jets3t.service.S3ServiceException;
|
||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||
import org.jets3t.service.security.AWSCredentials;
|
||||
import org.joda.time.Interval;
|
||||
import org.mortbay.jetty.Server;
|
||||
import org.mortbay.jetty.servlet.Context;
|
||||
import org.mortbay.jetty.servlet.DefaultServlet;
|
||||
|
@ -76,7 +92,6 @@ import org.mortbay.jetty.servlet.ServletHolder;
|
|||
import org.skife.config.ConfigurationObjectFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
@ -85,16 +100,15 @@ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class WorkerNode extends RegisteringNode
|
||||
public class WorkerNode extends BaseServerNode<WorkerNode>
|
||||
{
|
||||
private static final Logger log = new Logger(WorkerNode.class);
|
||||
private static final EmittingLogger log = new EmittingLogger(WorkerNode.class);
|
||||
|
||||
public static Builder builder()
|
||||
{
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final Lifecycle lifecycle;
|
||||
private final Properties props;
|
||||
private final ConfigurationObjectFactory configFactory;
|
||||
|
@ -111,21 +125,22 @@ public class WorkerNode extends RegisteringNode
|
|||
private ServiceDiscovery serviceDiscovery = null;
|
||||
private ServiceProvider coordinatorServiceProvider = null;
|
||||
private WorkerCuratorCoordinator workerCuratorCoordinator = null;
|
||||
private TaskMonitor taskMonitor = null;
|
||||
private WorkerTaskMonitor workerTaskMonitor = null;
|
||||
private MutableServerView newSegmentServerView = null;
|
||||
private Server server = null;
|
||||
|
||||
private boolean initialized = false;
|
||||
|
||||
public WorkerNode(
|
||||
ObjectMapper jsonMapper,
|
||||
Lifecycle lifecycle,
|
||||
Properties props,
|
||||
Lifecycle lifecycle,
|
||||
ObjectMapper jsonMapper,
|
||||
ObjectMapper smileMapper,
|
||||
ConfigurationObjectFactory configFactory
|
||||
)
|
||||
{
|
||||
super(Arrays.asList(jsonMapper));
|
||||
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
|
||||
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.lifecycle = lifecycle;
|
||||
this.props = props;
|
||||
this.configFactory = configFactory;
|
||||
|
@ -185,13 +200,20 @@ public class WorkerNode extends RegisteringNode
|
|||
return this;
|
||||
}
|
||||
|
||||
public WorkerNode setTaskMonitor(TaskMonitor taskMonitor)
|
||||
public WorkerNode setNewSegmentServerView(MutableServerView newSegmentServerView)
|
||||
{
|
||||
this.taskMonitor = taskMonitor;
|
||||
this.newSegmentServerView = newSegmentServerView;
|
||||
return this;
|
||||
}
|
||||
|
||||
public void init() throws Exception
|
||||
public WorkerNode setWorkerTaskMonitor(WorkerTaskMonitor workerTaskMonitor)
|
||||
{
|
||||
this.workerTaskMonitor = workerTaskMonitor;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doInit() throws Exception
|
||||
{
|
||||
initializeHttpClient();
|
||||
initializeEmitter();
|
||||
|
@ -201,12 +223,13 @@ public class WorkerNode extends RegisteringNode
|
|||
initializeCuratorFramework();
|
||||
initializeServiceDiscovery();
|
||||
initializeCoordinatorServiceProvider();
|
||||
initializeNewSegmentServerView();
|
||||
initializeDataSegmentPusher();
|
||||
initializeTaskToolbox();
|
||||
initializeJacksonInjections();
|
||||
initializeJacksonSubtypes();
|
||||
initializeCuratorCoordinator();
|
||||
initializeTaskMonitor();
|
||||
initializeWorkerTaskMonitor();
|
||||
initializeServer();
|
||||
|
||||
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
|
||||
|
@ -223,6 +246,12 @@ public class WorkerNode extends RegisteringNode
|
|||
|
||||
root.addServlet(new ServletHolder(new StatusServlet()), "/status");
|
||||
root.addServlet(new ServletHolder(new DefaultServlet()), "/mmx/*");
|
||||
root.addServlet(
|
||||
new ServletHolder(
|
||||
new QueryServlet(getJsonMapper(), getSmileMapper(), workerTaskMonitor, emitter, getRequestLogger())
|
||||
),
|
||||
"/druid/v2/*"
|
||||
);
|
||||
root.addFilter(GuiceFilter.class, "/mmx/indexer/worker/v1/*", 0);
|
||||
}
|
||||
|
||||
|
@ -280,12 +309,12 @@ public class WorkerNode extends RegisteringNode
|
|||
injectables.addValue("s3Client", s3Service)
|
||||
.addValue("segmentPusher", segmentPusher);
|
||||
|
||||
jsonMapper.setInjectableValues(injectables);
|
||||
getJsonMapper().setInjectableValues(injectables);
|
||||
}
|
||||
|
||||
private void initializeJacksonSubtypes()
|
||||
{
|
||||
jsonMapper.registerSubtypes(StaticS3FirehoseFactory.class);
|
||||
getJsonMapper().registerSubtypes(StaticS3FirehoseFactory.class);
|
||||
}
|
||||
|
||||
private void initializeHttpClient()
|
||||
|
@ -303,7 +332,7 @@ public class WorkerNode extends RegisteringNode
|
|||
emitter = new ServiceEmitter(
|
||||
PropUtils.getProperty(props, "druid.service"),
|
||||
PropUtils.getProperty(props, "druid.host"),
|
||||
Emitters.create(props, httpClient, jsonMapper, lifecycle)
|
||||
Emitters.create(props, httpClient, getJsonMapper(), lifecycle)
|
||||
);
|
||||
}
|
||||
EmittingLogger.registerEmitter(emitter);
|
||||
|
@ -344,7 +373,7 @@ public class WorkerNode extends RegisteringNode
|
|||
public void initializeDataSegmentPusher()
|
||||
{
|
||||
if (segmentPusher == null) {
|
||||
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper);
|
||||
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, getJsonMapper());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -352,14 +381,22 @@ public class WorkerNode extends RegisteringNode
|
|||
{
|
||||
if (taskToolboxFactory == null) {
|
||||
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
|
||||
final SegmentAnnouncer segmentAnnouncer = new ZkSegmentAnnouncer(
|
||||
configFactory.build(MetadataUpdaterConfig.class),
|
||||
getPhoneBook()
|
||||
);
|
||||
lifecycle.addManagedInstance(segmentAnnouncer);
|
||||
taskToolboxFactory = new TaskToolboxFactory(
|
||||
taskConfig,
|
||||
new RemoteTaskActionClientFactory(httpClient, coordinatorServiceProvider, jsonMapper),
|
||||
new RemoteTaskActionClientFactory(httpClient, coordinatorServiceProvider, getJsonMapper()),
|
||||
emitter,
|
||||
s3Service,
|
||||
segmentPusher,
|
||||
dataSegmentKiller,
|
||||
jsonMapper
|
||||
segmentAnnouncer,
|
||||
newSegmentServerView,
|
||||
getConglomerate(),
|
||||
getJsonMapper()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -402,7 +439,7 @@ public class WorkerNode extends RegisteringNode
|
|||
{
|
||||
if (workerCuratorCoordinator == null) {
|
||||
workerCuratorCoordinator = new WorkerCuratorCoordinator(
|
||||
jsonMapper,
|
||||
getJsonMapper(),
|
||||
configFactory.build(IndexerZkConfig.class),
|
||||
curatorFramework,
|
||||
new Worker(workerConfig)
|
||||
|
@ -411,29 +448,45 @@ public class WorkerNode extends RegisteringNode
|
|||
}
|
||||
}
|
||||
|
||||
public void initializeTaskMonitor()
|
||||
private void initializeNewSegmentServerView()
|
||||
{
|
||||
if (taskMonitor == null) {
|
||||
if (newSegmentServerView == null) {
|
||||
final MutableServerView view = new OnlyNewSegmentWatcherServerView();
|
||||
final ClientInventoryManager clientInventoryManager = new ClientInventoryManager(
|
||||
getConfigFactory().build(ClientConfig.class),
|
||||
getPhoneBook(),
|
||||
view
|
||||
);
|
||||
lifecycle.addManagedInstance(clientInventoryManager);
|
||||
|
||||
this.newSegmentServerView = view;
|
||||
}
|
||||
}
|
||||
|
||||
public void initializeWorkerTaskMonitor()
|
||||
{
|
||||
if (workerTaskMonitor == null) {
|
||||
final ExecutorService workerExec = Executors.newFixedThreadPool(workerConfig.getNumThreads());
|
||||
final PathChildrenCache pathChildrenCache = new PathChildrenCache(
|
||||
curatorFramework,
|
||||
workerCuratorCoordinator.getTaskPathForWorker(),
|
||||
false
|
||||
);
|
||||
taskMonitor = new TaskMonitor(
|
||||
workerTaskMonitor = new WorkerTaskMonitor(
|
||||
pathChildrenCache,
|
||||
curatorFramework,
|
||||
workerCuratorCoordinator,
|
||||
taskToolboxFactory,
|
||||
workerExec
|
||||
);
|
||||
lifecycle.addManagedInstance(taskMonitor);
|
||||
lifecycle.addManagedInstance(workerTaskMonitor);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Builder
|
||||
{
|
||||
private ObjectMapper jsonMapper = null;
|
||||
private ObjectMapper smileMapper = null;
|
||||
private Lifecycle lifecycle = null;
|
||||
private Properties props = null;
|
||||
private ConfigurationObjectFactory configFactory = null;
|
||||
|
@ -464,8 +517,13 @@ public class WorkerNode extends RegisteringNode
|
|||
|
||||
public WorkerNode build()
|
||||
{
|
||||
if (jsonMapper == null) {
|
||||
if (jsonMapper == null && smileMapper == null) {
|
||||
jsonMapper = new DefaultObjectMapper();
|
||||
smileMapper = new DefaultObjectMapper(new SmileFactory());
|
||||
smileMapper.getJsonFactory().setCodec(smileMapper);
|
||||
}
|
||||
else if (jsonMapper == null || smileMapper == null) {
|
||||
throw new ISE("Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", jsonMapper, smileMapper);
|
||||
}
|
||||
|
||||
if (lifecycle == null) {
|
||||
|
@ -480,7 +538,7 @@ public class WorkerNode extends RegisteringNode
|
|||
configFactory = Config.createFactory(props);
|
||||
}
|
||||
|
||||
return new WorkerNode(jsonMapper, lifecycle, props, configFactory);
|
||||
return new WorkerNode(props, lifecycle, jsonMapper, smileMapper, configFactory);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,9 +17,9 @@ import com.metamx.druid.merger.common.config.TaskConfig;
|
|||
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
|
||||
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.druid.merger.worker.TaskMonitor;
|
||||
import com.metamx.druid.merger.worker.Worker;
|
||||
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
|
||||
import com.metamx.druid.merger.worker.WorkerTaskMonitor;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.netflix.curator.framework.CuratorFramework;
|
||||
|
@ -59,7 +59,7 @@ public class RemoteTaskRunnerTest
|
|||
private CuratorFramework cf;
|
||||
private PathChildrenCache pathChildrenCache;
|
||||
private RemoteTaskRunner remoteTaskRunner;
|
||||
private TaskMonitor taskMonitor;
|
||||
private WorkerTaskMonitor workerTaskMonitor;
|
||||
|
||||
private ScheduledExecutorService scheduledExec;
|
||||
|
||||
|
@ -123,7 +123,7 @@ public class RemoteTaskRunnerTest
|
|||
{
|
||||
testingCluster.stop();
|
||||
remoteTaskRunner.stop();
|
||||
taskMonitor.stop();
|
||||
workerTaskMonitor.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -275,7 +275,7 @@ public class RemoteTaskRunnerTest
|
|||
);
|
||||
workerCuratorCoordinator.start();
|
||||
|
||||
taskMonitor = new TaskMonitor(
|
||||
workerTaskMonitor = new WorkerTaskMonitor(
|
||||
new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true),
|
||||
cf,
|
||||
workerCuratorCoordinator,
|
||||
|
@ -304,12 +304,12 @@ public class RemoteTaskRunnerTest
|
|||
{
|
||||
return null;
|
||||
}
|
||||
}, null, null, null, null, null, jsonMapper
|
||||
}, null, null, null, null, null, null, null, null, jsonMapper
|
||||
),
|
||||
Executors.newSingleThreadExecutor()
|
||||
);
|
||||
jsonMapper.registerSubtypes(new NamedType(TestTask.class, "test"));
|
||||
taskMonitor.start();
|
||||
workerTaskMonitor.start();
|
||||
}
|
||||
|
||||
private void makeRemoteTaskRunner() throws Exception
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package com.metamx.druid.merger.coordinator;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
@ -153,6 +152,9 @@ public class TaskLifecycleTest
|
|||
|
||||
}
|
||||
},
|
||||
null, // segment announcer
|
||||
null, // new segment server view
|
||||
null, // query runner factory conglomerate corporation unionized collective
|
||||
new DefaultObjectMapper()
|
||||
);
|
||||
|
||||
|
@ -282,22 +284,20 @@ public class TaskLifecycleTest
|
|||
// Sort of similar to what realtime tasks do:
|
||||
|
||||
// Acquire lock for first interval
|
||||
final Optional<TaskLock> lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval1));
|
||||
final TaskLock lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval1));
|
||||
final List<TaskLock> locks1 = toolbox.getTaskActionClient().submit(new LockListAction());
|
||||
|
||||
// (Confirm lock sanity)
|
||||
Assert.assertTrue("lock1 present", lock1.isPresent());
|
||||
Assert.assertEquals("lock1 interval", interval1, lock1.get().getInterval());
|
||||
Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1);
|
||||
Assert.assertEquals("lock1 interval", interval1, lock1.getInterval());
|
||||
Assert.assertEquals("locks1", ImmutableList.of(lock1), locks1);
|
||||
|
||||
// Acquire lock for second interval
|
||||
final Optional<TaskLock> lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval2));
|
||||
final TaskLock lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval2));
|
||||
final List<TaskLock> locks2 = toolbox.getTaskActionClient().submit(new LockListAction());
|
||||
|
||||
// (Confirm lock sanity)
|
||||
Assert.assertTrue("lock2 present", lock2.isPresent());
|
||||
Assert.assertEquals("lock2 interval", interval2, lock2.get().getInterval());
|
||||
Assert.assertEquals("locks2", ImmutableList.of(lock1.get(), lock2.get()), locks2);
|
||||
Assert.assertEquals("lock2 interval", interval2, lock2.getInterval());
|
||||
Assert.assertEquals("locks2", ImmutableList.of(lock1, lock2), locks2);
|
||||
|
||||
// Push first segment
|
||||
toolbox.getTaskActionClient()
|
||||
|
@ -307,7 +307,7 @@ public class TaskLifecycleTest
|
|||
DataSegment.builder()
|
||||
.dataSource("foo")
|
||||
.interval(interval1)
|
||||
.version(lock1.get().getVersion())
|
||||
.version(lock1.getVersion())
|
||||
.build()
|
||||
)
|
||||
)
|
||||
|
@ -318,7 +318,7 @@ public class TaskLifecycleTest
|
|||
final List<TaskLock> locks3 = toolbox.getTaskActionClient().submit(new LockListAction());
|
||||
|
||||
// (Confirm lock sanity)
|
||||
Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3);
|
||||
Assert.assertEquals("locks3", ImmutableList.of(lock2), locks3);
|
||||
|
||||
// Push second segment
|
||||
toolbox.getTaskActionClient()
|
||||
|
@ -328,7 +328,7 @@ public class TaskLifecycleTest
|
|||
DataSegment.builder()
|
||||
.dataSource("foo")
|
||||
.interval(interval2)
|
||||
.version(lock2.get().getVersion())
|
||||
.version(lock2.getVersion())
|
||||
.build()
|
||||
)
|
||||
)
|
||||
|
@ -392,7 +392,7 @@ public class TaskLifecycleTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testBadVersion() throws Exception
|
||||
public void testBadInterval() throws Exception
|
||||
{
|
||||
final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
|
||||
{
|
||||
|
@ -426,7 +426,7 @@ public class TaskLifecycleTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testBadInterval() throws Exception
|
||||
public void testBadVersion() throws Exception
|
||||
{
|
||||
final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
|
||||
{
|
||||
|
@ -506,15 +506,22 @@ public class TaskLifecycleTest
|
|||
}
|
||||
|
||||
@Override
|
||||
public void announceHistoricalSegments(Set<DataSegment> segment)
|
||||
public Set<DataSegment> announceHistoricalSegments(Set<DataSegment> segments)
|
||||
{
|
||||
published.addAll(segment);
|
||||
Set<DataSegment> added = Sets.newHashSet();
|
||||
for(final DataSegment segment : segments) {
|
||||
if(published.add(segment)) {
|
||||
added.add(segment);
|
||||
}
|
||||
}
|
||||
|
||||
return ImmutableSet.copyOf(added);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteSegments(Set<DataSegment> segment)
|
||||
public void deleteSegments(Set<DataSegment> segments)
|
||||
{
|
||||
nuked.addAll(segment);
|
||||
nuked.addAll(segments);
|
||||
}
|
||||
|
||||
public Set<DataSegment> getPublished()
|
||||
|
|
|
@ -165,6 +165,9 @@ public class TaskQueueTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
||||
|
@ -222,6 +225,9 @@ public class TaskQueueTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
||||
|
|
Loading…
Reference in New Issue