mirror of https://github.com/apache/druid.git
Merger: Changes to support rolling locks and a task-action API
Generally, TaskStatus is no longer used as the sole way for tasks to communicate status and metadata update requests back to the coordinator. It is still used to communicate success or failure, but a TaskActionClient is used for all other communication. In remote mode this uses an http api. TaskContext no longer exists, as its purpose is now served by the TaskActionClient. The TaskQueue and TaskConsumer are no longer in charge of committing stuff for a task. Instead, it happens when an action is sent in. Tasks are expected to ensure that this is OK by making their behavior idempotent (since they may be retried after partially completing). One new kind of request afforded to tasks is the ability to release and acquire arbitrary locks. This will be useful for long-running tasks. Short-lived tasks can still be given a lock automatically upon startup by providing a "fixed interval". New tables when using db storage: - tasklocks, to support persistence of arbitrary locks. - tasklogs, to log actions taken by tasks
This commit is contained in:
parent
ea64eaabce
commit
ff325b1f65
|
@ -19,8 +19,6 @@
|
|||
|
||||
package com.metamx.druid.merger.common;
|
||||
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
|
||||
public interface TaskCallback
|
||||
{
|
||||
public void notify(TaskStatus status);
|
||||
|
|
|
@ -1,55 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.common;
|
||||
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class TaskHolder
|
||||
{
|
||||
private final Task task;
|
||||
private final TaskContext taskContext;
|
||||
|
||||
@JsonCreator
|
||||
public TaskHolder(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("taskContext") TaskContext taskContext
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.taskContext = taskContext;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public TaskContext getTaskContext()
|
||||
{
|
||||
return taskContext;
|
||||
}
|
||||
}
|
|
@ -17,30 +17,30 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.coordinator;
|
||||
package com.metamx.druid.merger.common;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.commit.CommitStyle;
|
||||
import com.metamx.druid.merger.coordinator.commit.ImmediateCommitStyle;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Represents a transaction as well as the lock it holds. Not immutable: the task set can change.
|
||||
* Represents a lock held by some task. Immutable.
|
||||
*/
|
||||
public class TaskGroup
|
||||
public class TaskLock
|
||||
{
|
||||
private final String groupId;
|
||||
private final String dataSource;
|
||||
private final Interval interval;
|
||||
private final String version;
|
||||
private final Map<String, Task> taskMap = Maps.newHashMap();
|
||||
|
||||
public TaskGroup(String groupId, String dataSource, Interval interval, String version)
|
||||
@JsonCreator
|
||||
public TaskLock(
|
||||
@JsonProperty("groupId") String groupId,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval,
|
||||
@JsonProperty("version") String version
|
||||
)
|
||||
{
|
||||
this.groupId = groupId;
|
||||
this.dataSource = dataSource;
|
||||
|
@ -48,75 +48,48 @@ public class TaskGroup
|
|||
this.version = version;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getGroupId()
|
||||
{
|
||||
return groupId;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getVersion()
|
||||
{
|
||||
return version;
|
||||
}
|
||||
|
||||
public CommitStyle getCommitStyle()
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
// TODO -- should be configurable
|
||||
return new ImmediateCommitStyle();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns number of tasks in this group.
|
||||
*/
|
||||
public int size() {
|
||||
return taskMap.size();
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a task to this group.
|
||||
* @param task task to add
|
||||
* @return true iff this group did not already contain the task
|
||||
*/
|
||||
public boolean add(final Task task) {
|
||||
Preconditions.checkArgument(
|
||||
task.getGroupId().equals(groupId),
|
||||
"Task group id[%s] != TaskGroup group id[%s]",
|
||||
task.getGroupId(),
|
||||
groupId
|
||||
);
|
||||
|
||||
if(taskMap.containsKey(task.getId())) {
|
||||
if (!(o instanceof TaskLock)) {
|
||||
return false;
|
||||
} else {
|
||||
taskMap.put(task.getId(), task);
|
||||
return true;
|
||||
final TaskLock x = (TaskLock) o;
|
||||
return Objects.equal(this.groupId, x.groupId) &&
|
||||
Objects.equal(this.dataSource, x.dataSource) &&
|
||||
Objects.equal(this.interval, x.interval) &&
|
||||
Objects.equal(this.version, x.version);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if this group contains a particular task.
|
||||
*/
|
||||
public boolean contains(final String taskId) {
|
||||
return taskMap.containsKey(taskId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes a task from this group.
|
||||
* @param taskId task ID to remove
|
||||
* @return the removed task, or null if the task was not in this group
|
||||
*/
|
||||
public Task remove(final String taskId)
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return taskMap.remove(taskId);
|
||||
return Objects.hashCode(groupId, dataSource, interval, version);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -127,7 +100,6 @@ public class TaskGroup
|
|||
.add("dataSource", dataSource)
|
||||
.add("interval", interval)
|
||||
.add("version", version)
|
||||
.add("tasks", taskMap.keySet())
|
||||
.toString();
|
||||
}
|
||||
}
|
|
@ -35,7 +35,7 @@ import java.util.Set;
|
|||
/**
|
||||
* Represents the status of a task. The task may be ongoing ({@link #isComplete()} false) or it may be
|
||||
* complete ({@link #isComplete()} true).
|
||||
*
|
||||
* <p/>
|
||||
* TaskStatus objects are immutable.
|
||||
*/
|
||||
public class TaskStatus
|
||||
|
@ -49,49 +49,20 @@ public class TaskStatus
|
|||
|
||||
public static TaskStatus running(String taskId)
|
||||
{
|
||||
return new TaskStatus(
|
||||
taskId,
|
||||
Status.RUNNING,
|
||||
ImmutableSet.<DataSegment>of(),
|
||||
ImmutableSet.<DataSegment>of(),
|
||||
ImmutableList.<Task>of(),
|
||||
-1
|
||||
);
|
||||
return new TaskStatus(taskId, Status.RUNNING, -1);
|
||||
}
|
||||
|
||||
public static TaskStatus success(String taskId)
|
||||
{
|
||||
return success(taskId, ImmutableSet.<DataSegment>of());
|
||||
}
|
||||
|
||||
public static TaskStatus success(String taskId, Set<DataSegment> segments)
|
||||
{
|
||||
return new TaskStatus(
|
||||
taskId,
|
||||
Status.SUCCESS,
|
||||
segments,
|
||||
ImmutableSet.<DataSegment>of(),
|
||||
ImmutableList.<Task>of(),
|
||||
-1
|
||||
);
|
||||
return new TaskStatus(taskId, Status.SUCCESS, -1);
|
||||
}
|
||||
|
||||
public static TaskStatus failure(String taskId)
|
||||
{
|
||||
return new TaskStatus(
|
||||
taskId,
|
||||
Status.FAILED,
|
||||
ImmutableSet.<DataSegment>of(),
|
||||
ImmutableSet.<DataSegment>of(),
|
||||
ImmutableList.<Task>of(),
|
||||
-1
|
||||
);
|
||||
return new TaskStatus(taskId, Status.FAILED, -1);
|
||||
}
|
||||
|
||||
private final String id;
|
||||
private final ImmutableSet<DataSegment> segments;
|
||||
private final ImmutableSet<DataSegment> segmentsNuked;
|
||||
private final ImmutableList<Task> nextTasks;
|
||||
private final Status status;
|
||||
private final long duration;
|
||||
|
||||
|
@ -99,42 +70,16 @@ public class TaskStatus
|
|||
private TaskStatus(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("status") Status status,
|
||||
@JsonProperty("segments") Set<DataSegment> segments,
|
||||
@JsonProperty("segmentsNuked") Set<DataSegment> segmentsNuked,
|
||||
@JsonProperty("nextTasks") List<Task> nextTasks,
|
||||
@JsonProperty("duration") long duration
|
||||
)
|
||||
{
|
||||
this.id = id;
|
||||
this.segments = ImmutableSet.copyOf(segments);
|
||||
this.segmentsNuked = ImmutableSet.copyOf(segmentsNuked);
|
||||
this.nextTasks = ImmutableList.copyOf(nextTasks);
|
||||
this.status = status;
|
||||
this.duration = duration;
|
||||
|
||||
// Check class invariants.
|
||||
Preconditions.checkNotNull(id, "id");
|
||||
Preconditions.checkNotNull(status, "status");
|
||||
|
||||
if (this.segments.size() > 0) {
|
||||
Preconditions.checkArgument(
|
||||
status == Status.RUNNING || status == Status.SUCCESS,
|
||||
"segments not allowed for %s tasks",
|
||||
status
|
||||
);
|
||||
}
|
||||
|
||||
if (this.segmentsNuked.size() > 0) {
|
||||
Preconditions.checkArgument(status == Status.SUCCESS, "segmentsNuked not allowed for %s tasks", status);
|
||||
}
|
||||
|
||||
if (this.nextTasks.size() > 0) {
|
||||
Preconditions.checkArgument(
|
||||
status == Status.SUCCESS || status == Status.RUNNING,
|
||||
"nextTasks not allowed for %s tasks",
|
||||
status
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty("id")
|
||||
|
@ -149,24 +94,6 @@ public class TaskStatus
|
|||
return status;
|
||||
}
|
||||
|
||||
@JsonProperty("segments")
|
||||
public Set<DataSegment> getSegments()
|
||||
{
|
||||
return segments;
|
||||
}
|
||||
|
||||
@JsonProperty("segmentsNuked")
|
||||
public Set<DataSegment> getSegmentsNuked()
|
||||
{
|
||||
return segmentsNuked;
|
||||
}
|
||||
|
||||
@JsonProperty("nextTasks")
|
||||
public List<Task> getNextTasks()
|
||||
{
|
||||
return nextTasks;
|
||||
}
|
||||
|
||||
@JsonProperty("duration")
|
||||
public long getDuration()
|
||||
{
|
||||
|
@ -212,25 +139,9 @@ public class TaskStatus
|
|||
return status == Status.FAILED;
|
||||
}
|
||||
|
||||
public TaskStatus withSegments(Set<DataSegment> _segments)
|
||||
{
|
||||
return new TaskStatus(id, status, _segments, segmentsNuked, nextTasks, duration);
|
||||
}
|
||||
|
||||
|
||||
public TaskStatus withSegmentsNuked(Set<DataSegment> _segmentsNuked)
|
||||
{
|
||||
return new TaskStatus(id, status, segments, _segmentsNuked, nextTasks, duration);
|
||||
}
|
||||
|
||||
public TaskStatus withNextTasks(List<Task> _nextTasks)
|
||||
{
|
||||
return new TaskStatus(id, status, segments, segmentsNuked, _nextTasks, duration);
|
||||
}
|
||||
|
||||
public TaskStatus withDuration(long _duration)
|
||||
{
|
||||
return new TaskStatus(id, status, segments, segmentsNuked, nextTasks, _duration);
|
||||
return new TaskStatus(id, status, _duration);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -238,8 +149,6 @@ public class TaskStatus
|
|||
{
|
||||
return Objects.toStringHelper(this)
|
||||
.add("id", id)
|
||||
.add("segments", segments)
|
||||
.add("nextTasks", nextTasks)
|
||||
.add("status", status)
|
||||
.add("duration", duration)
|
||||
.toString();
|
||||
|
|
|
@ -20,15 +20,15 @@
|
|||
package com.metamx.druid.merger.common;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.metamx.druid.loading.S3SegmentPuller;
|
||||
import com.metamx.druid.loading.S3SegmentGetterConfig;
|
||||
import com.metamx.druid.loading.S3SegmentPuller;
|
||||
import com.metamx.druid.loading.S3ZippedSegmentPuller;
|
||||
import com.metamx.druid.loading.SegmentKiller;
|
||||
import com.metamx.druid.loading.SegmentPuller;
|
||||
import com.metamx.druid.loading.SegmentPusher;
|
||||
import com.metamx.druid.merger.common.actions.TaskActionClient;
|
||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
||||
import com.metamx.druid.loading.SegmentPusher;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||
|
@ -42,6 +42,7 @@ import java.util.Map;
|
|||
public class TaskToolbox
|
||||
{
|
||||
private final TaskConfig config;
|
||||
private final TaskActionClient taskActionClient;
|
||||
private final ServiceEmitter emitter;
|
||||
private final RestS3Service s3Client;
|
||||
private final SegmentPusher segmentPusher;
|
||||
|
@ -50,6 +51,7 @@ public class TaskToolbox
|
|||
|
||||
public TaskToolbox(
|
||||
TaskConfig config,
|
||||
TaskActionClient taskActionClient,
|
||||
ServiceEmitter emitter,
|
||||
RestS3Service s3Client,
|
||||
SegmentPusher segmentPusher,
|
||||
|
@ -58,6 +60,7 @@ public class TaskToolbox
|
|||
)
|
||||
{
|
||||
this.config = config;
|
||||
this.taskActionClient = taskActionClient;
|
||||
this.emitter = emitter;
|
||||
this.s3Client = s3Client;
|
||||
this.segmentPusher = segmentPusher;
|
||||
|
@ -70,16 +73,16 @@ public class TaskToolbox
|
|||
return config;
|
||||
}
|
||||
|
||||
public TaskActionClient getTaskActionClient()
|
||||
{
|
||||
return taskActionClient;
|
||||
}
|
||||
|
||||
public ServiceEmitter getEmitter()
|
||||
{
|
||||
return emitter;
|
||||
}
|
||||
|
||||
public RestS3Service getS3Client()
|
||||
{
|
||||
return s3Client;
|
||||
}
|
||||
|
||||
public SegmentPusher getSegmentPusher()
|
||||
{
|
||||
return segmentPusher;
|
||||
|
|
|
@ -0,0 +1,37 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.metamx.druid.merger.coordinator.TaskStorage;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
||||
public class LocalTaskActionClient implements TaskActionClient
|
||||
{
|
||||
private final TaskStorage storage;
|
||||
private final TaskActionToolbox toolbox;
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class);
|
||||
|
||||
public LocalTaskActionClient(TaskStorage storage, TaskActionToolbox toolbox)
|
||||
{
|
||||
this.storage = storage;
|
||||
this.toolbox = toolbox;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <RetType> RetType submit(TaskAction<RetType> taskAction)
|
||||
{
|
||||
final RetType ret = taskAction.perform(toolbox);
|
||||
|
||||
// Add audit log
|
||||
try {
|
||||
storage.addAuditLog(taskAction);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert(e, "Failed to record action in audit log")
|
||||
.addData("task", taskAction.getTask().getId())
|
||||
.addData("actionClass", taskAction.getClass().getName())
|
||||
.emit();
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
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 org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
|
||||
{
|
||||
private final Task task;
|
||||
private final Interval interval;
|
||||
|
||||
@JsonCreator
|
||||
public LockAcquireAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
public TypeReference<Optional<TaskLock>> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Optional<TaskLock>>() {};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<TaskLock> perform(TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
return toolbox.getTaskLockbox().tryLock(task, interval);
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
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 org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class LockListAction implements TaskAction<List<TaskLock>>
|
||||
{
|
||||
private final Task task;
|
||||
|
||||
@JsonCreator
|
||||
public LockListAction(
|
||||
@JsonProperty("task") Task task
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
public TypeReference<List<TaskLock>> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<List<TaskLock>>() {};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TaskLock> perform(TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
return toolbox.getTaskLockbox().findLocksForTask(task);
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,55 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class LockReleaseAction implements TaskAction<Void>
|
||||
{
|
||||
private final Task task;
|
||||
private final Interval interval;
|
||||
|
||||
@JsonCreator
|
||||
public LockReleaseAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
public TypeReference<Void> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Void>() {};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Void perform(TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
toolbox.getTaskLockbox().unlock(task, interval);
|
||||
return null;
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.http.client.HttpClient;
|
||||
import com.metamx.http.client.response.ToStringResponseHandler;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
|
||||
public class RemoteTaskActionClient implements TaskActionClient
|
||||
{
|
||||
private final HttpClient httpClient;
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
||||
private static final Logger log = new Logger(RemoteTaskActionClient.class);
|
||||
|
||||
public RemoteTaskActionClient(HttpClient httpClient, ObjectMapper jsonMapper)
|
||||
{
|
||||
this.httpClient = httpClient;
|
||||
this.jsonMapper = jsonMapper;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <RetType> RetType submit(TaskAction<RetType> taskAction)
|
||||
{
|
||||
try {
|
||||
byte[] dataToSend = jsonMapper.writeValueAsBytes(taskAction);
|
||||
|
||||
final String response = httpClient.post(getServiceUri().toURL())
|
||||
.setContent("application/json", dataToSend)
|
||||
.go(new ToStringResponseHandler(Charsets.UTF_8))
|
||||
.get();
|
||||
|
||||
// TODO Figure out how to check HTTP status code
|
||||
if(response.equals("")) {
|
||||
return null;
|
||||
} else {
|
||||
return jsonMapper.readValue(response, taskAction.getReturnTypeReference());
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
public URI getServiceUri() throws URISyntaxException
|
||||
{
|
||||
return new URI("http://localhost:8087/mmx/merger/v1/action");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,95 @@
|
|||
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 org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
public class SegmentInsertAction implements TaskAction<Void>
|
||||
{
|
||||
private final Task task;
|
||||
private final Set<DataSegment> segments;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentInsertAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("segments") Set<DataSegment> segments
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.segments = ImmutableSet.copyOf(segments);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Set<DataSegment> getSegments()
|
||||
{
|
||||
return segments;
|
||||
}
|
||||
|
||||
public TypeReference<Void> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Void>() {};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Void perform(TaskActionToolbox toolbox)
|
||||
{
|
||||
// Verify that each of these segments-to-insert falls under some lock
|
||||
// TODO: Should this be done while holding the giant lock on TaskLockbox? (To prevent anyone from grabbing
|
||||
// TODO: these locks out from under us while the operation is ongoing.) Probably not necessary.
|
||||
final List<TaskLock> taskLocks = toolbox.getTaskLockbox().findLocksForTask(task);
|
||||
for(final DataSegment segment : segments) {
|
||||
final boolean ok = Iterables.any(
|
||||
taskLocks, new Predicate<TaskLock>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(TaskLock taskLock)
|
||||
{
|
||||
return taskLock.getVersion().equals(segment.getVersion())
|
||||
&& taskLock.getDataSource().equals(segment.getDataSource())
|
||||
&& taskLock.getInterval().contains(segment.getInterval());
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
if(!ok) {
|
||||
throw new ISE("No currently-held lock covers segment: %s", segment);
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
toolbox.getMergerDBCoordinator().announceHistoricalSegments(segments);
|
||||
|
||||
// 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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,63 @@
|
|||
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 org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
|
||||
{
|
||||
private final Task task;
|
||||
private final String dataSource;
|
||||
private final Interval interval;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentListUnusedAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.dataSource = dataSource;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
public TypeReference<List<DataSegment>> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<List<DataSegment>>() {};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<DataSegment> perform(TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval);
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,63 @@
|
|||
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 org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
|
||||
{
|
||||
private final Task task;
|
||||
private final String dataSource;
|
||||
private final Interval interval;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentListUsedAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.dataSource = dataSource;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
public TypeReference<List<DataSegment>> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<List<DataSegment>>() {};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<DataSegment> perform(TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval);
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,95 @@
|
|||
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 org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
public class SegmentNukeAction implements TaskAction<Void>
|
||||
{
|
||||
private final Task task;
|
||||
private final Set<DataSegment> segments;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentNukeAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("segments") Set<DataSegment> segments
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.segments = ImmutableSet.copyOf(segments);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Set<DataSegment> getSegments()
|
||||
{
|
||||
return segments;
|
||||
}
|
||||
|
||||
public TypeReference<Void> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Void>() {};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Void perform(TaskActionToolbox toolbox)
|
||||
{
|
||||
// Verify that each of these segments-to-nuke falls under some lock
|
||||
// TODO: Should this be done while holding the giant lock on TaskLockbox? (To prevent anyone from grabbing
|
||||
// TODO: these locks out from under us while the operation is ongoing.) Probably not necessary.
|
||||
final List<TaskLock> taskLocks = toolbox.getTaskLockbox().findLocksForTask(task);
|
||||
for(final DataSegment segment : segments) {
|
||||
final boolean ok = Iterables.any(
|
||||
taskLocks, new Predicate<TaskLock>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(TaskLock taskLock)
|
||||
{
|
||||
return taskLock.getVersion().compareTo(segment.getVersion()) >= 0
|
||||
&& taskLock.getDataSource().equals(segment.getDataSource())
|
||||
&& taskLock.getInterval().contains(segment.getInterval());
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
if(!ok) {
|
||||
throw new ISE("No currently-held lock covers segment: %s", segment);
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
toolbox.getMergerDBCoordinator().deleteSegments(segments);
|
||||
|
||||
// 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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,57 @@
|
|||
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 org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class SpawnTasksAction implements TaskAction<Void>
|
||||
{
|
||||
private final Task task;
|
||||
private final List<Task> newTasks;
|
||||
|
||||
@JsonCreator
|
||||
public SpawnTasksAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("newTasks") List<Task> newTasks
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.newTasks = ImmutableList.copyOf(newTasks);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<Task> getNewTasks()
|
||||
{
|
||||
return newTasks;
|
||||
}
|
||||
|
||||
public TypeReference<Void> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Void>() {};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Void perform(TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
for(final Task newTask : newTasks) {
|
||||
toolbox.getTaskQueue().add(newTask);
|
||||
}
|
||||
|
||||
return null;
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,24 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import org.codehaus.jackson.annotate.JsonSubTypes;
|
||||
import org.codehaus.jackson.annotate.JsonTypeInfo;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "lockAcquire", value = LockAcquireAction.class),
|
||||
@JsonSubTypes.Type(name = "lockList", value = LockListAction.class),
|
||||
@JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class),
|
||||
@JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class),
|
||||
@JsonSubTypes.Type(name = "segmentListUsed", value = SegmentListUsedAction.class),
|
||||
@JsonSubTypes.Type(name = "segmentListUnused", value = SegmentListUnusedAction.class),
|
||||
@JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class),
|
||||
@JsonSubTypes.Type(name = "spawnTasks", value = SpawnTasksAction.class)
|
||||
})
|
||||
public interface TaskAction<RetType>
|
||||
{
|
||||
public Task getTask(); // TODO Look into replacing this with task ID so stuff serializes smaller
|
||||
public TypeReference<RetType> getReturnTypeReference(); // T_T
|
||||
public RetType perform(TaskActionToolbox toolbox);
|
||||
}
|
|
@ -0,0 +1,6 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
public interface TaskActionClient
|
||||
{
|
||||
public <RetType> RetType submit(TaskAction<RetType> taskAction);
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
|
||||
import com.metamx.druid.merger.coordinator.TaskLockbox;
|
||||
import com.metamx.druid.merger.coordinator.TaskQueue;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
|
||||
public class TaskActionToolbox
|
||||
{
|
||||
private final TaskQueue taskQueue;
|
||||
private final TaskLockbox taskLockbox;
|
||||
private final MergerDBCoordinator mergerDBCoordinator;
|
||||
private final ServiceEmitter emitter;
|
||||
|
||||
public TaskActionToolbox(
|
||||
TaskQueue taskQueue,
|
||||
TaskLockbox taskLockbox,
|
||||
MergerDBCoordinator mergerDBCoordinator,
|
||||
ServiceEmitter emitter
|
||||
)
|
||||
{
|
||||
this.taskQueue = taskQueue;
|
||||
this.taskLockbox = taskLockbox;
|
||||
this.mergerDBCoordinator = mergerDBCoordinator;
|
||||
this.emitter = emitter;
|
||||
}
|
||||
|
||||
public TaskQueue getTaskQueue()
|
||||
{
|
||||
return taskQueue;
|
||||
}
|
||||
|
||||
public TaskLockbox getTaskLockbox()
|
||||
{
|
||||
return taskLockbox;
|
||||
}
|
||||
|
||||
public MergerDBCoordinator getMergerDBCoordinator()
|
||||
{
|
||||
return mergerDBCoordinator;
|
||||
}
|
||||
|
||||
public ServiceEmitter getEmitter()
|
||||
{
|
||||
return emitter;
|
||||
}
|
||||
}
|
|
@ -20,10 +20,10 @@
|
|||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
@ -32,30 +32,19 @@ public abstract class AbstractTask implements Task
|
|||
private final String id;
|
||||
private final String groupId;
|
||||
private final String dataSource;
|
||||
private final Interval interval;
|
||||
private final Optional<Interval> interval;
|
||||
|
||||
public AbstractTask(String id, String dataSource, Interval interval)
|
||||
protected AbstractTask(String id, String dataSource, Interval interval)
|
||||
{
|
||||
this(id, id, dataSource, interval);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public AbstractTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("groupId") String groupId,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
protected AbstractTask(String id, String groupId, String dataSource, Interval interval)
|
||||
{
|
||||
Preconditions.checkNotNull(id, "id");
|
||||
Preconditions.checkNotNull(groupId, "groupId");
|
||||
Preconditions.checkNotNull(dataSource, "dataSource");
|
||||
Preconditions.checkNotNull(interval, "interval");
|
||||
|
||||
this.id = id;
|
||||
this.groupId = groupId;
|
||||
this.dataSource = dataSource;
|
||||
this.interval = interval;
|
||||
this.id = Preconditions.checkNotNull(id, "id");
|
||||
this.groupId = Preconditions.checkNotNull(groupId, "groupId");
|
||||
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
|
||||
this.interval = Optional.fromNullable(interval);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -79,15 +68,22 @@ public abstract class AbstractTask implements Task
|
|||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
public Optional<Interval> getFixedInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
// Awesome hack to get around lack of serde for Optional<T>
|
||||
// TODO Look into jackson-datatype-guava
|
||||
@JsonProperty("interval")
|
||||
private Interval getNullableIntervalForJackson()
|
||||
{
|
||||
return interval.orNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus preflight(TaskContext context) throws Exception
|
||||
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
return TaskStatus.running(id);
|
||||
}
|
||||
|
@ -99,7 +95,7 @@ public abstract class AbstractTask implements Task
|
|||
.add("id", id)
|
||||
.add("type", getType())
|
||||
.add("dataSource", dataSource)
|
||||
.add("interval", getInterval())
|
||||
.add("interval", getFixedInterval())
|
||||
.toString();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -60,7 +60,7 @@ public class AppendTask extends MergeTask
|
|||
throws Exception
|
||||
{
|
||||
VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<String, DataSegment>(
|
||||
Ordering.natural().nullsFirst()
|
||||
Ordering.<String>natural().nullsFirst()
|
||||
);
|
||||
|
||||
for (DataSegment segment : segments.keySet()) {
|
||||
|
@ -109,9 +109,9 @@ public class AppendTask extends MergeTask
|
|||
}
|
||||
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Task.Type.APPEND;
|
||||
return "append";
|
||||
}
|
||||
|
||||
private class SegmentToMergeHolder
|
||||
|
|
|
@ -80,8 +80,8 @@ public class DefaultMergeTask extends MergeTask
|
|||
}
|
||||
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Task.Type.MERGE;
|
||||
return "merge";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,9 @@
|
|||
|
||||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.QueryGranularity;
|
||||
|
@ -29,10 +31,11 @@ import com.metamx.druid.index.v1.IncrementalIndex;
|
|||
import com.metamx.druid.index.v1.IncrementalIndexAdapter;
|
||||
import com.metamx.druid.index.v1.IndexMerger;
|
||||
import com.metamx.druid.index.v1.IndexableAdapter;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
import com.metamx.druid.merger.common.actions.LockListAction;
|
||||
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.shard.NoneShardSpec;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
@ -60,29 +63,31 @@ public class DeleteTask extends AbstractTask
|
|||
new DateTime().toString()
|
||||
),
|
||||
dataSource,
|
||||
interval
|
||||
Preconditions.checkNotNull(interval, "interval")
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Task.Type.DELETE;
|
||||
return "delete";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
// Strategy: Create an empty segment covering the interval to be deleted
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
|
||||
final Interval interval = this.getFixedInterval().get();
|
||||
final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
|
||||
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(this.getInterval(), empty);
|
||||
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty);
|
||||
|
||||
// Create DataSegment
|
||||
final DataSegment segment =
|
||||
DataSegment.builder()
|
||||
.dataSource(this.getDataSource())
|
||||
.interval(this.getInterval())
|
||||
.version(context.getVersion())
|
||||
.interval(interval)
|
||||
.version(myLock.getVersion())
|
||||
.shardSpec(new NoneShardSpec())
|
||||
.build();
|
||||
|
||||
|
@ -99,6 +104,8 @@ public class DeleteTask extends AbstractTask
|
|||
segment.getVersion()
|
||||
);
|
||||
|
||||
return TaskStatus.success(getId(), ImmutableSet.of(uploadedSegment));
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment)));
|
||||
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
|
@ -28,10 +29,9 @@ import com.google.common.collect.TreeMultiset;
|
|||
import com.google.common.primitives.Ints;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.input.InputRow;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
|
||||
import com.metamx.druid.realtime.Firehose;
|
||||
import com.metamx.druid.realtime.FirehoseFactory;
|
||||
import com.metamx.druid.realtime.Schema;
|
||||
|
@ -75,7 +75,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||
),
|
||||
groupId,
|
||||
schema.getDataSource(),
|
||||
interval
|
||||
Preconditions.checkNotNull(interval, "interval")
|
||||
);
|
||||
|
||||
this.firehoseFactory = firehoseFactory;
|
||||
|
@ -84,21 +84,20 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||
}
|
||||
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Type.INDEX;
|
||||
return "index_partitions";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
log.info("Running with targetPartitionSize[%d]", targetPartitionSize);
|
||||
|
||||
// This is similar to what DeterminePartitionsJob does in the hadoop indexer, but we don't require
|
||||
// a preconfigured partition dimension (we'll just pick the one with highest cardinality).
|
||||
// TODO: Replace/merge/whatever with hadoop determine-partitions code
|
||||
|
||||
// NOTE: Space-efficiency (stores all unique dimension values, although at least not all combinations)
|
||||
// NOTE: Time-efficiency (runs all this on one single node instead of through map/reduce)
|
||||
// We know this exists
|
||||
final Interval interval = getFixedInterval().get();
|
||||
|
||||
// Blacklist dimensions that have multiple values per row
|
||||
final Set<String> unusableDimensions = Sets.newHashSet();
|
||||
|
@ -114,7 +113,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||
|
||||
final InputRow inputRow = firehose.nextRow();
|
||||
|
||||
if (getInterval().contains(inputRow.getTimestampFromEpoch())) {
|
||||
if (interval.contains(inputRow.getTimestampFromEpoch())) {
|
||||
|
||||
// Extract dimensions from event
|
||||
for (final String dim : inputRow.getDimensions()) {
|
||||
|
@ -229,28 +228,30 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||
}
|
||||
}
|
||||
|
||||
return TaskStatus.success(getId()).withNextTasks(
|
||||
Lists.transform(
|
||||
shardSpecs,
|
||||
new Function<ShardSpec, Task>()
|
||||
{
|
||||
@Override
|
||||
public Task apply(ShardSpec shardSpec)
|
||||
{
|
||||
return new IndexGeneratorTask(
|
||||
getGroupId(),
|
||||
getInterval(),
|
||||
firehoseFactory,
|
||||
new Schema(
|
||||
schema.getDataSource(),
|
||||
schema.getAggregators(),
|
||||
schema.getIndexGranularity(),
|
||||
shardSpec
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
)
|
||||
List<Task> nextTasks = Lists.transform(
|
||||
shardSpecs,
|
||||
new Function<ShardSpec, Task>()
|
||||
{
|
||||
@Override
|
||||
public Task apply(ShardSpec shardSpec)
|
||||
{
|
||||
return new IndexGeneratorTask(
|
||||
getGroupId(),
|
||||
getFixedInterval().get(),
|
||||
firehoseFactory,
|
||||
new Schema(
|
||||
schema.getDataSource(),
|
||||
schema.getAggregators(),
|
||||
schema.getIndexGranularity(),
|
||||
shardSpec
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks));
|
||||
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,24 +19,25 @@
|
|||
|
||||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.input.InputRow;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.loading.SegmentPusher;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.actions.LockListAction;
|
||||
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
|
||||
import com.metamx.druid.merger.common.index.YeOldePlumberSchool;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.realtime.FireDepartmentMetrics;
|
||||
import com.metamx.druid.realtime.Firehose;
|
||||
import com.metamx.druid.realtime.FirehoseFactory;
|
||||
import com.metamx.druid.realtime.Plumber;
|
||||
import com.metamx.druid.realtime.Schema;
|
||||
import com.metamx.druid.loading.SegmentPusher;
|
||||
import com.metamx.druid.realtime.Sink;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
@ -77,7 +78,7 @@ public class IndexGeneratorTask extends AbstractTask
|
|||
),
|
||||
groupId,
|
||||
schema.getDataSource(),
|
||||
interval
|
||||
Preconditions.checkNotNull(interval, "interval")
|
||||
);
|
||||
|
||||
this.firehoseFactory = firehoseFactory;
|
||||
|
@ -85,14 +86,20 @@ public class IndexGeneratorTask extends AbstractTask
|
|||
}
|
||||
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Type.INDEX;
|
||||
return "index_generator";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(final TaskContext context, final TaskToolbox toolbox, TaskCallback callback) throws Exception
|
||||
public TaskStatus run(final TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
// We should have a lock from before we started running
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
|
||||
|
||||
// We know this exists
|
||||
final Interval interval = getFixedInterval().get();
|
||||
|
||||
// Set up temporary directory for indexing
|
||||
final File tmpDir = new File(
|
||||
String.format(
|
||||
|
@ -101,9 +108,9 @@ public class IndexGeneratorTask extends AbstractTask
|
|||
String.format(
|
||||
"%s_%s_%s_%s_%s",
|
||||
this.getDataSource(),
|
||||
this.getInterval().getStart(),
|
||||
this.getInterval().getEnd(),
|
||||
context.getVersion(),
|
||||
interval.getStart(),
|
||||
interval.getEnd(),
|
||||
myLock.getVersion(),
|
||||
schema.getShardSpec().getPartitionNum()
|
||||
)
|
||||
)
|
||||
|
@ -126,8 +133,8 @@ public class IndexGeneratorTask extends AbstractTask
|
|||
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
|
||||
final Firehose firehose = firehoseFactory.connect();
|
||||
final Plumber plumber = new YeOldePlumberSchool(
|
||||
getInterval(),
|
||||
context.getVersion(),
|
||||
interval,
|
||||
myLock.getVersion(),
|
||||
wrappedSegmentPusher,
|
||||
tmpDir
|
||||
).findPlumber(schema, metrics);
|
||||
|
@ -175,8 +182,11 @@ public class IndexGeneratorTask extends AbstractTask
|
|||
metrics.rowOutput()
|
||||
);
|
||||
|
||||
// Request segment pushes
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(pushedSegments)));
|
||||
|
||||
// Done
|
||||
return TaskStatus.success(getId(), ImmutableSet.copyOf(pushedSegments));
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -185,7 +195,7 @@ public class IndexGeneratorTask extends AbstractTask
|
|||
* @return true or false
|
||||
*/
|
||||
private boolean shouldIndex(InputRow inputRow) {
|
||||
if(!getInterval().contains(inputRow.getTimestampFromEpoch())) {
|
||||
if(!getFixedInterval().get().contains(inputRow.getTimestampFromEpoch())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
|
|
@ -25,10 +25,9 @@ import com.metamx.common.logger.Logger;
|
|||
import com.metamx.druid.QueryGranularity;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.indexer.granularity.GranularitySpec;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
|
||||
import com.metamx.druid.realtime.FirehoseFactory;
|
||||
import com.metamx.druid.realtime.Schema;
|
||||
import com.metamx.druid.shard.NoneShardSpec;
|
||||
|
@ -119,19 +118,20 @@ public class IndexTask extends AbstractTask
|
|||
}
|
||||
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Type.INDEX;
|
||||
return "index";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus preflight(TaskContext context) throws Exception
|
||||
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
return TaskStatus.success(getId()).withNextTasks(toSubtasks());
|
||||
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, toSubtasks()));
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
throw new IllegalStateException("IndexTasks should not be run!");
|
||||
}
|
||||
|
|
|
@ -1,20 +1,22 @@
|
|||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
import com.metamx.druid.merger.common.actions.LockListAction;
|
||||
import com.metamx.druid.merger.common.actions.SegmentListUnusedAction;
|
||||
import com.metamx.druid.merger.common.actions.SegmentNukeAction;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -42,18 +44,55 @@ public class KillTask extends AbstractTask
|
|||
}
|
||||
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Task.Type.KILL;
|
||||
return "kill";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
// Kill segments
|
||||
toolbox.getSegmentKiller()
|
||||
.kill(context.getUnusedSegments());
|
||||
// Confirm we have a lock (will throw if there isn't exactly one element)
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
|
||||
|
||||
return TaskStatus.success(getId()).withSegmentsNuked(context.getUnusedSegments());
|
||||
if(!myLock.getDataSource().equals(getDataSource())) {
|
||||
throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource());
|
||||
}
|
||||
|
||||
if(!myLock.getInterval().equals(getFixedInterval().get())) {
|
||||
throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getFixedInterval().get());
|
||||
}
|
||||
|
||||
// List unused segments
|
||||
final List<DataSegment> unusedSegments = toolbox.getTaskActionClient()
|
||||
.submit(
|
||||
new SegmentListUnusedAction(
|
||||
this,
|
||||
myLock.getDataSource(),
|
||||
myLock.getInterval()
|
||||
)
|
||||
);
|
||||
|
||||
// Verify none of these segments have versions > lock version
|
||||
for(final DataSegment unusedSegment : unusedSegments) {
|
||||
if(unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
|
||||
throw new ISE(
|
||||
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
|
||||
unusedSegment.getIdentifier(),
|
||||
unusedSegment.getVersion(),
|
||||
myLock.getVersion()
|
||||
);
|
||||
}
|
||||
|
||||
log.info("OK to kill segment: %s", unusedSegment.getIdentifier());
|
||||
}
|
||||
|
||||
// Kill segments
|
||||
toolbox.getSegmentKiller().kill(unusedSegments);
|
||||
|
||||
// Remove metadata for these segments
|
||||
toolbox.getTaskActionClient().submit(new SegmentNukeAction(this, ImmutableSet.copyOf(unusedSegments)));
|
||||
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,10 +35,12 @@ import com.metamx.common.ISE;
|
|||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.loading.SegmentPuller;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
import com.metamx.druid.merger.common.actions.LockListAction;
|
||||
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
|
||||
import com.metamx.druid.merger.common.actions.SegmentListUsedAction;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.shard.NoneShardSpec;
|
||||
import com.metamx.emitter.service.AlertEvent;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
|
@ -116,11 +118,12 @@ public abstract class MergeTask extends AbstractTask
|
|||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
|
||||
final ServiceEmitter emitter = toolbox.getEmitter();
|
||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
|
||||
final DataSegment mergedSegment = computeMergedSegment(getDataSource(), context.getVersion(), segments);
|
||||
final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments);
|
||||
final File taskDir = toolbox.getConfig().getTaskDir(this);
|
||||
|
||||
try {
|
||||
|
@ -172,7 +175,9 @@ public abstract class MergeTask extends AbstractTask
|
|||
emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart));
|
||||
emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize()));
|
||||
|
||||
return TaskStatus.success(getId(), ImmutableSet.of(uploadedSegment));
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment)));
|
||||
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(
|
||||
|
@ -201,7 +206,7 @@ public abstract class MergeTask extends AbstractTask
|
|||
* we are operating on every segment that overlaps the chosen interval.
|
||||
*/
|
||||
@Override
|
||||
public TaskStatus preflight(TaskContext context)
|
||||
public TaskStatus preflight(TaskToolbox toolbox)
|
||||
{
|
||||
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
|
||||
{
|
||||
|
@ -212,7 +217,13 @@ public abstract class MergeTask extends AbstractTask
|
|||
}
|
||||
};
|
||||
|
||||
final Set<String> current = ImmutableSet.copyOf(Iterables.transform(context.getCurrentSegments(), toIdentifier));
|
||||
final Set<String> current = ImmutableSet.copyOf(
|
||||
Iterables.transform(
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(new SegmentListUsedAction(this, getDataSource(), getFixedInterval().get())),
|
||||
toIdentifier
|
||||
)
|
||||
);
|
||||
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
|
||||
|
||||
final Set<String> missingFromRequested = Sets.difference(current, requested);
|
||||
|
@ -250,7 +261,7 @@ public abstract class MergeTask extends AbstractTask
|
|||
return Objects.toStringHelper(this)
|
||||
.add("id", getId())
|
||||
.add("dataSource", getDataSource())
|
||||
.add("interval", getInterval())
|
||||
.add("interval", getFixedInterval())
|
||||
.add("segments", segments)
|
||||
.toString();
|
||||
}
|
||||
|
@ -276,7 +287,7 @@ public abstract class MergeTask extends AbstractTask
|
|||
)
|
||||
);
|
||||
|
||||
return String.format("%s_%s", dataSource, DigestUtils.shaHex(segmentIDs).toLowerCase());
|
||||
return String.format("%s_%s", dataSource, DigestUtils.sha1Hex(segmentIDs).toLowerCase());
|
||||
}
|
||||
|
||||
private static Interval computeMergedInterval(final List<DataSegment> segments)
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
import org.codehaus.jackson.annotate.JsonSubTypes;
|
||||
import org.codehaus.jackson.annotate.JsonTypeInfo;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -33,6 +33,7 @@ import org.joda.time.Interval;
|
|||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class)
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "append", value = AppendTask.class),
|
||||
@JsonSubTypes.Type(name = "merge", value = DefaultMergeTask.class),
|
||||
@JsonSubTypes.Type(name = "delete", value = DeleteTask.class),
|
||||
@JsonSubTypes.Type(name = "kill", value = KillTask.class),
|
||||
@JsonSubTypes.Type(name = "index", value = IndexTask.class),
|
||||
|
@ -41,49 +42,53 @@ import org.joda.time.Interval;
|
|||
})
|
||||
public interface Task
|
||||
{
|
||||
enum Type
|
||||
{
|
||||
INDEX,
|
||||
MERGE,
|
||||
APPEND,
|
||||
DELETE,
|
||||
TEST,
|
||||
KILL
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns ID of this task. Must be unique across all tasks ever created.
|
||||
*/
|
||||
public String getId();
|
||||
|
||||
/**
|
||||
* Returns group ID of this task. Tasks with the same group ID can share locks. If tasks do not need to share locks,
|
||||
* a common convention is to set group ID equal to task ID.
|
||||
*/
|
||||
public String getGroupId();
|
||||
|
||||
public Type getType();
|
||||
/**
|
||||
* Returns a descriptive label for this task type. Used for metrics emission and logging.
|
||||
*/
|
||||
public String getType();
|
||||
|
||||
/**
|
||||
* Returns the datasource this task operates on. Each task can operate on only one datasource.
|
||||
*/
|
||||
public String getDataSource();
|
||||
|
||||
public Interval getInterval();
|
||||
/**
|
||||
* Returns fixed interval for this task, if any. Tasks without fixed intervals are not granted locks when started
|
||||
* and must explicitly request them.
|
||||
*/
|
||||
public Optional<Interval> getFixedInterval();
|
||||
|
||||
/**
|
||||
* Execute preflight checks for a task. This typically runs on the coordinator, and will be run while
|
||||
* holding a lock on our dataSouce and interval. If this method throws an exception, the task should be
|
||||
* considered a failure.
|
||||
*
|
||||
* @param context Context for this task, gathered under indexer lock
|
||||
* @param toolbox Toolbox for this task
|
||||
* @return Some kind of status (runnable means continue on to a worker, non-runnable means we completed without
|
||||
* using a worker).
|
||||
* @throws Exception
|
||||
*/
|
||||
public TaskStatus preflight(TaskContext context) throws Exception;
|
||||
public TaskStatus preflight(TaskToolbox toolbox) throws Exception;
|
||||
|
||||
/**
|
||||
* Execute a task. This typically runs on a worker as determined by a TaskRunner, and will be run while
|
||||
* holding a lock on our dataSource and interval. If this method throws an exception, the task should be
|
||||
* considered a failure.
|
||||
*
|
||||
* @param context Context for this task, gathered under indexer lock
|
||||
* @param toolbox Toolbox for this task
|
||||
* @param callback Callback for "early returns". Statuses returned to this callback must not be
|
||||
* complete (isRunnable must be true).
|
||||
* @return Some kind of finished status (isRunnable must be false).
|
||||
* @throws Exception
|
||||
*/
|
||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception;
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception;
|
||||
}
|
||||
|
|
|
@ -23,10 +23,15 @@ import com.google.common.base.Function;
|
|||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.actions.TaskAction;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
@ -36,6 +41,7 @@ import org.skife.jdbi.v2.Handle;
|
|||
import org.skife.jdbi.v2.exceptions.StatementException;
|
||||
import org.skife.jdbi.v2.tweak.HandleCallback;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -103,18 +109,11 @@ public class DbTaskStorage implements TaskStorage
|
|||
}
|
||||
|
||||
@Override
|
||||
public void setStatus(final String taskid, final TaskStatus status)
|
||||
public void setStatus(final TaskStatus status)
|
||||
{
|
||||
Preconditions.checkNotNull(taskid, "task");
|
||||
Preconditions.checkNotNull(status, "status");
|
||||
Preconditions.checkArgument(
|
||||
taskid.equals(status.getId()),
|
||||
"Task/Status ID mismatch[%s/%s]",
|
||||
taskid,
|
||||
status.getId()
|
||||
);
|
||||
|
||||
log.info("Updating task %s to status: %s", taskid, status);
|
||||
log.info("Updating task %s to status: %s", status.getId(), status);
|
||||
|
||||
int updated = dbi.withHandle(
|
||||
new HandleCallback<Integer>()
|
||||
|
@ -124,12 +123,13 @@ public class DbTaskStorage implements TaskStorage
|
|||
{
|
||||
return handle.createStatement(
|
||||
String.format(
|
||||
"UPDATE %s SET status_code = :status_code, status_payload = :status_payload WHERE id = :id",
|
||||
"UPDATE %s SET status_code = :status_code, status_payload = :status_payload WHERE id = :id AND status_code = :old_status_code",
|
||||
dbConnectorConfig.getTaskTable()
|
||||
)
|
||||
)
|
||||
.bind("id", taskid)
|
||||
.bind("id", status.getId())
|
||||
.bind("status_code", status.getStatusCode().toString())
|
||||
.bind("old_status_code", TaskStatus.Status.RUNNING.toString())
|
||||
.bind("status_payload", jsonMapper.writeValueAsString(status))
|
||||
.execute();
|
||||
}
|
||||
|
@ -137,40 +137,10 @@ public class DbTaskStorage implements TaskStorage
|
|||
);
|
||||
|
||||
if(updated != 1) {
|
||||
throw new IllegalStateException(String.format("Task not found: %s", taskid));
|
||||
throw new IllegalStateException(String.format("Running task not found: %s", status.getId()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setVersion(final String taskid, final String version)
|
||||
{
|
||||
Preconditions.checkNotNull(taskid, "task");
|
||||
Preconditions.checkNotNull(version, "version");
|
||||
|
||||
log.info("Updating task %s to version: %s", taskid, version);
|
||||
|
||||
dbi.withHandle(
|
||||
new HandleCallback<Void>()
|
||||
{
|
||||
@Override
|
||||
public Void withHandle(Handle handle) throws Exception
|
||||
{
|
||||
handle.createStatement(
|
||||
String.format(
|
||||
"UPDATE %s SET version = :version WHERE id = :id",
|
||||
dbConnectorConfig.getTaskTable()
|
||||
)
|
||||
)
|
||||
.bind("id", taskid)
|
||||
.bind("version", version)
|
||||
.execute();
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Task> getTask(final String taskid)
|
||||
{
|
||||
|
@ -231,36 +201,6 @@ public class DbTaskStorage implements TaskStorage
|
|||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<String> getVersion(final String taskid)
|
||||
{
|
||||
return dbi.withHandle(
|
||||
new HandleCallback<Optional<String>>()
|
||||
{
|
||||
@Override
|
||||
public Optional<String> withHandle(Handle handle) throws Exception
|
||||
{
|
||||
final List<Map<String, Object>> dbStatuses =
|
||||
handle.createQuery(
|
||||
String.format(
|
||||
"SELECT version FROM %s WHERE id = :id",
|
||||
dbConnectorConfig.getTaskTable()
|
||||
)
|
||||
)
|
||||
.bind("id", taskid)
|
||||
.list();
|
||||
|
||||
if(dbStatuses.size() == 0) {
|
||||
return Optional.absent();
|
||||
} else {
|
||||
final Map<String, Object> dbStatus = Iterables.getOnlyElement(dbStatuses);
|
||||
return Optional.fromNullable((String) dbStatus.get("version"));
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Task> getRunningTasks()
|
||||
{
|
||||
|
@ -298,4 +238,183 @@ public class DbTaskStorage implements TaskStorage
|
|||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addLock(final String taskid, final TaskLock taskLock)
|
||||
{
|
||||
Preconditions.checkNotNull(taskid, "taskid");
|
||||
Preconditions.checkNotNull(taskLock, "taskLock");
|
||||
|
||||
log.info(
|
||||
"Adding lock on interval[%s] version[%s] for task: %s",
|
||||
taskLock.getInterval(),
|
||||
taskLock.getVersion(),
|
||||
taskid
|
||||
);
|
||||
|
||||
dbi.withHandle(
|
||||
new HandleCallback<Integer>()
|
||||
{
|
||||
@Override
|
||||
public Integer withHandle(Handle handle) throws Exception
|
||||
{
|
||||
return handle.createStatement(
|
||||
String.format(
|
||||
"INSERT INTO %s (task_id, lock_payload) VALUES (:task_id, :lock_payload)",
|
||||
dbConnectorConfig.getTaskLockTable()
|
||||
)
|
||||
)
|
||||
.bind("task_id", taskid)
|
||||
.bind("lock_payload", jsonMapper.writeValueAsString(taskLock))
|
||||
.execute();
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeLock(String taskid, TaskLock taskLockToRemove)
|
||||
{
|
||||
Preconditions.checkNotNull(taskid, "taskid");
|
||||
Preconditions.checkNotNull(taskLockToRemove, "taskLockToRemove");
|
||||
|
||||
final Map<Long, TaskLock> taskLocks = getLocksWithIds(taskid);
|
||||
|
||||
for(final Map.Entry<Long, TaskLock> taskLockWithId : taskLocks.entrySet()) {
|
||||
final long id = taskLockWithId.getKey();
|
||||
final TaskLock taskLock = taskLockWithId.getValue();
|
||||
|
||||
if(taskLock.equals(taskLockToRemove)) {
|
||||
log.info("Deleting TaskLock with id[%d]: %s", id, taskLock);
|
||||
|
||||
dbi.withHandle(
|
||||
new HandleCallback<Integer>()
|
||||
{
|
||||
@Override
|
||||
public Integer withHandle(Handle handle) throws Exception
|
||||
{
|
||||
return handle.createStatement(
|
||||
String.format(
|
||||
"DELETE FROM %s WHERE id = :id",
|
||||
dbConnectorConfig.getTaskLockTable()
|
||||
)
|
||||
)
|
||||
.bind("id", id)
|
||||
.execute();
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TaskLock> getLocks(String taskid)
|
||||
{
|
||||
return ImmutableList.copyOf(
|
||||
Iterables.transform(
|
||||
getLocksWithIds(taskid).entrySet(), new Function<Map.Entry<Long, TaskLock>, TaskLock>()
|
||||
{
|
||||
@Override
|
||||
public TaskLock apply(Map.Entry<Long, TaskLock> e)
|
||||
{
|
||||
return e.getValue();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> void addAuditLog(final TaskAction<T> taskAction)
|
||||
{
|
||||
Preconditions.checkNotNull(taskAction, "taskAction");
|
||||
|
||||
log.info("Logging action for task[%s]: %s", taskAction.getTask().getId(), taskAction);
|
||||
|
||||
dbi.withHandle(
|
||||
new HandleCallback<Integer>()
|
||||
{
|
||||
@Override
|
||||
public Integer withHandle(Handle handle) throws Exception
|
||||
{
|
||||
return handle.createStatement(
|
||||
String.format(
|
||||
"INSERT INTO %s (task_id, log_payload) VALUES (:task_id, :log_payload)",
|
||||
dbConnectorConfig.getTaskLogTable()
|
||||
)
|
||||
)
|
||||
.bind("task_id", taskAction.getTask().getId())
|
||||
.bind("log_payload", jsonMapper.writeValueAsString(taskAction))
|
||||
.execute();
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TaskAction> getAuditLogs(final String taskid)
|
||||
{
|
||||
return dbi.withHandle(
|
||||
new HandleCallback<List<TaskAction>>()
|
||||
{
|
||||
@Override
|
||||
public List<TaskAction> withHandle(Handle handle) throws Exception
|
||||
{
|
||||
final List<Map<String, Object>> dbTaskLogs =
|
||||
handle.createQuery(
|
||||
String.format(
|
||||
"SELECT log_payload FROM %s WHERE task_id = :task_id",
|
||||
dbConnectorConfig.getTaskLogTable()
|
||||
)
|
||||
)
|
||||
.bind("task_id", taskid)
|
||||
.list();
|
||||
|
||||
return Lists.transform(
|
||||
dbTaskLogs, new Function<Map<String, Object>, TaskAction>()
|
||||
{
|
||||
@Override
|
||||
public TaskAction apply(Map<String, Object> row)
|
||||
{
|
||||
try {
|
||||
return jsonMapper.readValue(row.get("payload").toString(), TaskAction.class);
|
||||
} catch(Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private Map<Long, TaskLock> getLocksWithIds(final String taskid)
|
||||
{
|
||||
return dbi.withHandle(
|
||||
new HandleCallback<Map<Long, TaskLock>>()
|
||||
{
|
||||
@Override
|
||||
public Map<Long, TaskLock> withHandle(Handle handle) throws Exception
|
||||
{
|
||||
final List<Map<String, Object>> dbTaskLocks =
|
||||
handle.createQuery(
|
||||
String.format(
|
||||
"SELECT id, lock_payload FROM %s WHERE task_id = :task_id",
|
||||
dbConnectorConfig.getTaskLockTable()
|
||||
)
|
||||
)
|
||||
.bind("task_id", taskid)
|
||||
.list();
|
||||
|
||||
final Map<Long, TaskLock> retMap = Maps.newHashMap();
|
||||
for(final Map<String, Object> row : dbTaskLocks) {
|
||||
retMap.put((Long)row.get("id"), jsonMapper.readValue(row.get("lock_payload").toString(), TaskLock.class));
|
||||
}
|
||||
return retMap;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -57,7 +57,7 @@ public class LocalTaskRunner implements TaskRunner
|
|||
}
|
||||
|
||||
@Override
|
||||
public void run(final Task task, final TaskContext context, final TaskCallback callback)
|
||||
public void run(final Task task, final TaskCallback callback)
|
||||
{
|
||||
exec.submit(
|
||||
new Runnable()
|
||||
|
@ -71,7 +71,7 @@ public class LocalTaskRunner implements TaskRunner
|
|||
|
||||
try {
|
||||
log.info("Running task: %s", task.getId());
|
||||
status = task.run(context, toolbox, callback);
|
||||
status = task.run(toolbox);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
log.error(e, "Interrupted while running task[%s]", task);
|
||||
|
|
|
@ -21,14 +21,20 @@ package com.metamx.druid.merger.coordinator;
|
|||
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import com.google.common.collect.HashMultimap;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Multimap;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.actions.TaskAction;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
/**
|
||||
* Implements an in-heap TaskStorage facility, with no persistence across restarts. This class is not
|
||||
|
@ -36,108 +42,178 @@ import java.util.Map;
|
|||
*/
|
||||
public class LocalTaskStorage implements TaskStorage
|
||||
{
|
||||
private final ReentrantLock giant = new ReentrantLock();
|
||||
private final Map<String, TaskStuff> tasks = Maps.newHashMap();
|
||||
private final Multimap<String, TaskLock> taskLocks = HashMultimap.create();
|
||||
private final Multimap<String, TaskAction> taskActions = ArrayListMultimap.create();
|
||||
|
||||
private static final Logger log = new Logger(LocalTaskStorage.class);
|
||||
|
||||
@Override
|
||||
public void insert(Task task, TaskStatus status)
|
||||
{
|
||||
Preconditions.checkNotNull(task, "task");
|
||||
Preconditions.checkNotNull(status, "status");
|
||||
Preconditions.checkArgument(
|
||||
task.getId().equals(status.getId()),
|
||||
"Task/Status ID mismatch[%s/%s]",
|
||||
task.getId(),
|
||||
status.getId()
|
||||
);
|
||||
giant.lock();
|
||||
|
||||
if(tasks.containsKey(task.getId())) {
|
||||
throw new TaskExistsException(task.getId());
|
||||
try {
|
||||
Preconditions.checkNotNull(task, "task");
|
||||
Preconditions.checkNotNull(status, "status");
|
||||
Preconditions.checkArgument(
|
||||
task.getId().equals(status.getId()),
|
||||
"Task/Status ID mismatch[%s/%s]",
|
||||
task.getId(),
|
||||
status.getId()
|
||||
);
|
||||
|
||||
if(tasks.containsKey(task.getId())) {
|
||||
throw new TaskExistsException(task.getId());
|
||||
}
|
||||
|
||||
log.info("Inserting task %s with status: %s", task.getId(), status);
|
||||
tasks.put(task.getId(), new TaskStuff(task, status));
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
|
||||
log.info("Inserting task %s with status: %s", task.getId(), status);
|
||||
tasks.put(task.getId(), new TaskStuff(task, status));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Task> getTask(String taskid)
|
||||
{
|
||||
Preconditions.checkNotNull(taskid, "taskid");
|
||||
if(tasks.containsKey(taskid)) {
|
||||
return Optional.of(tasks.get(taskid).getTask());
|
||||
} else {
|
||||
return Optional.absent();
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
Preconditions.checkNotNull(taskid, "taskid");
|
||||
if(tasks.containsKey(taskid)) {
|
||||
return Optional.of(tasks.get(taskid).getTask());
|
||||
} else {
|
||||
return Optional.absent();
|
||||
}
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setStatus(String taskid, TaskStatus status)
|
||||
public void setStatus(TaskStatus status)
|
||||
{
|
||||
Preconditions.checkNotNull(taskid, "taskid");
|
||||
Preconditions.checkNotNull(status, "status");
|
||||
Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid);
|
||||
log.info("Updating task %s to status: %s", taskid, status);
|
||||
tasks.put(taskid, tasks.get(taskid).withStatus(status));
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
Preconditions.checkNotNull(status, "status");
|
||||
|
||||
final String taskid = status.getId();
|
||||
Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid);
|
||||
Preconditions.checkState(tasks.get(taskid).getStatus().isRunnable(), "Task status must be runnable: %s", taskid);
|
||||
log.info("Updating task %s to status: %s", taskid, status);
|
||||
tasks.put(taskid, tasks.get(taskid).withStatus(status));
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<TaskStatus> getStatus(String taskid)
|
||||
{
|
||||
Preconditions.checkNotNull(taskid, "taskid");
|
||||
if(tasks.containsKey(taskid)) {
|
||||
return Optional.of(tasks.get(taskid).getStatus());
|
||||
} else {
|
||||
return Optional.absent();
|
||||
}
|
||||
}
|
||||
giant.lock();
|
||||
|
||||
@Override
|
||||
public void setVersion(String taskid, String version)
|
||||
{
|
||||
Preconditions.checkNotNull(taskid, "taskid");
|
||||
Preconditions.checkNotNull(version, "status");
|
||||
Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid);
|
||||
log.info("Updating task %s to version: %s", taskid, version);
|
||||
tasks.put(taskid, tasks.get(taskid).withVersion(version));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<String> getVersion(String taskid)
|
||||
{
|
||||
Preconditions.checkNotNull(taskid, "taskid");
|
||||
if(tasks.containsKey(taskid)) {
|
||||
return tasks.get(taskid).getVersion();
|
||||
} else {
|
||||
return Optional.absent();
|
||||
try {
|
||||
Preconditions.checkNotNull(taskid, "taskid");
|
||||
if(tasks.containsKey(taskid)) {
|
||||
return Optional.of(tasks.get(taskid).getStatus());
|
||||
} else {
|
||||
return Optional.absent();
|
||||
}
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Task> getRunningTasks()
|
||||
{
|
||||
final ImmutableList.Builder<Task> listBuilder = ImmutableList.builder();
|
||||
for(final TaskStuff taskStuff : tasks.values()) {
|
||||
if(taskStuff.getStatus().isRunnable()) {
|
||||
listBuilder.add(taskStuff.getTask());
|
||||
}
|
||||
}
|
||||
giant.lock();
|
||||
|
||||
return listBuilder.build();
|
||||
try {
|
||||
final ImmutableList.Builder<Task> listBuilder = ImmutableList.builder();
|
||||
for(final TaskStuff taskStuff : tasks.values()) {
|
||||
if(taskStuff.getStatus().isRunnable()) {
|
||||
listBuilder.add(taskStuff.getTask());
|
||||
}
|
||||
}
|
||||
|
||||
return listBuilder.build();
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addLock(final String taskid, final TaskLock taskLock)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
Preconditions.checkNotNull(taskLock, "taskLock");
|
||||
taskLocks.put(taskid, taskLock);
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeLock(final String taskid, final TaskLock taskLock)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
Preconditions.checkNotNull(taskLock, "taskLock");
|
||||
taskLocks.remove(taskid, taskLock);
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TaskLock> getLocks(final String taskid)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
return ImmutableList.copyOf(taskLocks.get(taskid));
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> void addAuditLog(TaskAction<T> taskAction)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
taskActions.put(taskAction.getTask().getId(), taskAction);
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TaskAction> getAuditLogs(String taskid)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
return ImmutableList.copyOf(taskActions.get(taskid));
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
private static class TaskStuff
|
||||
{
|
||||
final Task task;
|
||||
final TaskStatus status;
|
||||
final Optional<String> version;
|
||||
|
||||
private TaskStuff(Task task, TaskStatus status)
|
||||
{
|
||||
this(task, status, Optional.<String>absent());
|
||||
}
|
||||
|
||||
private TaskStuff(Task task, TaskStatus status, Optional<String> version)
|
||||
{
|
||||
Preconditions.checkNotNull(task);
|
||||
Preconditions.checkNotNull(status);
|
||||
|
@ -145,7 +221,6 @@ public class LocalTaskStorage implements TaskStorage
|
|||
|
||||
this.task = task;
|
||||
this.status = status;
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
public Task getTask()
|
||||
|
@ -158,19 +233,9 @@ public class LocalTaskStorage implements TaskStorage
|
|||
return status;
|
||||
}
|
||||
|
||||
public Optional<String> getVersion()
|
||||
{
|
||||
return version;
|
||||
}
|
||||
|
||||
private TaskStuff withStatus(TaskStatus _status)
|
||||
{
|
||||
return new TaskStuff(task, _status, version);
|
||||
}
|
||||
|
||||
private TaskStuff withVersion(String _version)
|
||||
{
|
||||
return new TaskStuff(task, status, Optional.of(_version));
|
||||
return new TaskStuff(task, _status);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,6 +47,7 @@ import java.io.IOException;
|
|||
import java.sql.SQLException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -130,52 +131,25 @@ public class MergerDBCoordinator
|
|||
return segments;
|
||||
}
|
||||
|
||||
public void commitTaskStatus(final TaskStatus taskStatus)
|
||||
public void announceHistoricalSegments(final Set<DataSegment> segments) throws Exception
|
||||
{
|
||||
try {
|
||||
dbi.inTransaction(
|
||||
new TransactionCallback<Void>()
|
||||
{
|
||||
@Override
|
||||
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception
|
||||
{
|
||||
for(final DataSegment segment : taskStatus.getSegments())
|
||||
{
|
||||
log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), taskStatus.getId());
|
||||
announceHistoricalSegment(handle, segment);
|
||||
}
|
||||
|
||||
for(final DataSegment segment : taskStatus.getSegmentsNuked())
|
||||
{
|
||||
log.info("Deleting segment[%s] for task[%s]", segment.getIdentifier(), taskStatus.getId());
|
||||
deleteSegment(handle, segment);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(String.format("Exception commit task to DB: %s", taskStatus.getId()), e);
|
||||
}
|
||||
}
|
||||
|
||||
public void announceHistoricalSegment(final DataSegment segment) throws Exception
|
||||
{
|
||||
dbi.withHandle(
|
||||
new HandleCallback<Void>()
|
||||
dbi.inTransaction(
|
||||
new TransactionCallback<Void>()
|
||||
{
|
||||
@Override
|
||||
public Void withHandle(Handle handle) throws Exception
|
||||
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception
|
||||
{
|
||||
announceHistoricalSegment(handle, segment);
|
||||
for(final DataSegment segment : segments) {
|
||||
announceHistoricalSegment(handle, segment);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
private void announceHistoricalSegment(final Handle handle, final DataSegment segment) throws Exception
|
||||
{
|
||||
try {
|
||||
|
@ -219,15 +193,18 @@ public class MergerDBCoordinator
|
|||
}
|
||||
}
|
||||
|
||||
public void deleteSegment(final DataSegment segment)
|
||||
public void deleteSegments(final Set<DataSegment> segments) throws Exception
|
||||
{
|
||||
dbi.withHandle(
|
||||
new HandleCallback<Void>()
|
||||
dbi.inTransaction(
|
||||
new TransactionCallback<Void>()
|
||||
{
|
||||
@Override
|
||||
public Void withHandle(Handle handle) throws Exception
|
||||
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception
|
||||
{
|
||||
deleteSegment(handle, segment);
|
||||
for(final DataSegment segment : segments) {
|
||||
deleteSegment(handle, segment);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,7 +34,6 @@ import com.metamx.common.lifecycle.LifecycleStart;
|
|||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.druid.PeriodGranularity;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskHolder;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
|
||||
|
@ -276,14 +275,12 @@ public class RemoteTaskRunner implements TaskRunner
|
|||
}
|
||||
|
||||
@Override
|
||||
public void run(Task task, TaskContext context, TaskCallback callback)
|
||||
public void run(Task task, TaskCallback callback)
|
||||
{
|
||||
if (tasks.containsKey(task.getId())) {
|
||||
throw new ISE("Assigned a task[%s] that already exists, WTF is happening?!", task.getId());
|
||||
}
|
||||
TaskWrapper taskWrapper = new TaskWrapper(
|
||||
task, context, callback, retryPolicyFactory.makeRetryPolicy()
|
||||
);
|
||||
TaskWrapper taskWrapper = new TaskWrapper(task, callback, retryPolicyFactory.makeRetryPolicy());
|
||||
tasks.put(taskWrapper.getTask().getId(), taskWrapper);
|
||||
assignTask(taskWrapper);
|
||||
}
|
||||
|
@ -606,13 +603,12 @@ public class RemoteTaskRunner implements TaskRunner
|
|||
{
|
||||
synchronized (statusLock) {
|
||||
final Task task = taskWrapper.getTask();
|
||||
final TaskContext taskContext = taskWrapper.getTaskContext();
|
||||
try {
|
||||
log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId());
|
||||
|
||||
tasks.put(task.getId(), taskWrapper);
|
||||
|
||||
byte[] rawBytes = jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext));
|
||||
byte[] rawBytes = jsonMapper.writeValueAsBytes(task);
|
||||
|
||||
if (rawBytes.length > config.getMaxNumBytes()) {
|
||||
throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes());
|
||||
|
@ -626,7 +622,7 @@ public class RemoteTaskRunner implements TaskRunner
|
|||
theWorker.getHost(),
|
||||
task.getId()
|
||||
),
|
||||
jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext))
|
||||
jsonMapper.writeValueAsBytes(task)
|
||||
);
|
||||
|
||||
// Syncing state with Zookeeper
|
||||
|
|
|
@ -1,66 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.coordinator;
|
||||
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Information gathered by the coordinator, after acquiring a lock, that may be useful to a task.
|
||||
*/
|
||||
public class TaskContext
|
||||
{
|
||||
final String version;
|
||||
final Set<DataSegment> currentSegments;
|
||||
final Set<DataSegment> unusedSegments;
|
||||
|
||||
@JsonCreator
|
||||
public TaskContext(
|
||||
@JsonProperty("version") String version,
|
||||
@JsonProperty("currentSegments") Set<DataSegment> currentSegments,
|
||||
@JsonProperty("unusedSegments") Set<DataSegment> unusedSegments
|
||||
)
|
||||
{
|
||||
this.version = version;
|
||||
this.currentSegments = currentSegments;
|
||||
this.unusedSegments = unusedSegments;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getVersion()
|
||||
{
|
||||
return version;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Set<DataSegment> getCurrentSegments()
|
||||
{
|
||||
return currentSegments;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Set<DataSegment> getUnusedSegments()
|
||||
{
|
||||
return unusedSegments;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,405 @@
|
|||
package com.metamx.druid.merger.coordinator;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.guava.Comparators;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
/**
|
||||
* Remembers which tasks have locked which intervals. Tasks are permitted to lock an interval if no other task
|
||||
* outside their group has locked an overlapping interval for the same datasource. When a task locks an interval,
|
||||
* it is assigned a version string that it can use to publish segments.
|
||||
*/
|
||||
public class TaskLockbox
|
||||
{
|
||||
// Datasource -> Interval -> Tasks + TaskLock
|
||||
private final Map<String, NavigableMap<Interval, TaskLockPosse>> running = Maps.newHashMap();
|
||||
private final TaskStorage taskStorage;
|
||||
private final ReentrantLock giant = new ReentrantLock();
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(TaskLockbox.class);
|
||||
|
||||
public TaskLockbox(TaskStorage taskStorage)
|
||||
{
|
||||
this.taskStorage = taskStorage;
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempt to lock a task, without removing it from the queue. Equivalent to the long form of {@code tryLock}
|
||||
* with no preferred version.
|
||||
*
|
||||
* @param task task to attempt to lock
|
||||
*
|
||||
* @return lock version if lock was acquired, absent otherwise
|
||||
*/
|
||||
public Optional<TaskLock> tryLock(final Task task, final Interval interval)
|
||||
{
|
||||
return tryLock(task, interval, Optional.<String>absent());
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task.
|
||||
* This method will attempt to assign version strings that obey the invariant that every version string is
|
||||
* lexicographically greater than any other version string previously assigned to the same interval. This invariant
|
||||
* is only mostly guaranteed, however; we assume clock monotonicity and we assume that callers specifying
|
||||
* {@code preferredVersion} are doing the right thing.
|
||||
*
|
||||
* @param task task to attempt to lock
|
||||
* @param preferredVersion use this version string if one has not yet been assigned
|
||||
*
|
||||
* @return lock version if lock was acquired, absent otherwise
|
||||
*/
|
||||
public Optional<TaskLock> tryLock(final Task task, final Interval interval, final Optional<String> preferredVersion)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
|
||||
if(task.getFixedInterval().isPresent() && !task.getFixedInterval().get().equals(interval)) {
|
||||
// Task may only lock its fixed interval, if present
|
||||
throw new IAE("Task must lock its fixed interval: %s", task.getId());
|
||||
}
|
||||
|
||||
final String dataSource = task.getDataSource();
|
||||
final List<TaskLockPosse> foundPosses = findLockPossesForInterval(dataSource, interval);
|
||||
final TaskLockPosse posseToUse;
|
||||
|
||||
if (foundPosses.size() > 1) {
|
||||
|
||||
// Too many existing locks.
|
||||
return Optional.absent();
|
||||
|
||||
} else if (foundPosses.size() == 1) {
|
||||
|
||||
// One existing lock -- check if we can add to it.
|
||||
|
||||
final TaskLockPosse foundPosse = Iterables.getOnlyElement(foundPosses);
|
||||
if (foundPosse.getTaskLock().getInterval().contains(interval) && foundPosse.getTaskLock().getGroupId().equals(task.getGroupId())) {
|
||||
posseToUse = foundPosse;
|
||||
} else {
|
||||
return Optional.absent();
|
||||
}
|
||||
|
||||
} else {
|
||||
|
||||
// No existing locks. We can make a new one.
|
||||
if (!running.containsKey(dataSource)) {
|
||||
running.put(dataSource, new TreeMap<Interval, TaskLockPosse>(Comparators.intervalsByStartThenEnd()));
|
||||
}
|
||||
|
||||
// Create new TaskLock and assign it a version.
|
||||
// Assumption: We'll choose a version that is greater than any previously-chosen version for our interval. (This
|
||||
// may not always be true, unfortunately. See below.)
|
||||
|
||||
final String version;
|
||||
|
||||
if (preferredVersion.isPresent()) {
|
||||
// We have a preferred version. We'll trust our caller to not break our ordering assumptions and just use it.
|
||||
version = preferredVersion.get();
|
||||
} else {
|
||||
// We are running under an interval lock right now, so just using the current time works as long as we can trust
|
||||
// our clock to be monotonic and have enough resolution since the last time we created a TaskLock for the same
|
||||
// interval. This may not always be true; to assure it we would need to use some method of timekeeping other
|
||||
// than the wall clock.
|
||||
version = new DateTime().toString();
|
||||
}
|
||||
|
||||
posseToUse = new TaskLockPosse(new TaskLock(task.getGroupId(), dataSource, interval, version));
|
||||
running.get(dataSource)
|
||||
.put(interval, posseToUse);
|
||||
|
||||
log.info("Created new TaskLockPosse: %s", posseToUse);
|
||||
}
|
||||
|
||||
// Add to existing TaskLockPosse, if necessary
|
||||
if (posseToUse.getTaskIds().add(task.getId())) {
|
||||
log.info("Added task[%s] to TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId());
|
||||
|
||||
// Best effort to update task storage facility
|
||||
try {
|
||||
taskStorage.addLock(task.getId(), posseToUse.getTaskLock());
|
||||
} catch(Exception e) {
|
||||
log.makeAlert("Failed to persist lock in storage")
|
||||
.addData("task", task.getId())
|
||||
.addData("dataSource", posseToUse.getTaskLock().getDataSource())
|
||||
.addData("interval", posseToUse.getTaskLock().getInterval())
|
||||
.addData("version", posseToUse.getTaskLock().getVersion())
|
||||
.emit();
|
||||
}
|
||||
} else {
|
||||
log.info("Task[%s] already present in TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId());
|
||||
}
|
||||
|
||||
return Optional.of(posseToUse.getTaskLock());
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the currently-active locks for some task.
|
||||
*
|
||||
* @param task task for which to locate locks
|
||||
*/
|
||||
public List<TaskLock> findLocksForTask(final Task task)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
return Lists.transform(
|
||||
findLockPossesForTask(task), new Function<TaskLockPosse, TaskLock>()
|
||||
{
|
||||
@Override
|
||||
public TaskLock apply(TaskLockPosse taskLockPosse)
|
||||
{
|
||||
return taskLockPosse.getTaskLock();
|
||||
}
|
||||
}
|
||||
);
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Release lock held for a task on a particular interval. Does nothing if the task does not currently
|
||||
* hold the mentioned lock.
|
||||
*
|
||||
* @param task task to unlock
|
||||
* @param interval interval to unlock
|
||||
*/
|
||||
public void unlock(final Task task, final Interval interval)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
final String dataSource = task.getDataSource();
|
||||
final NavigableMap<Interval, TaskLockPosse> dsRunning = running.get(dataSource);
|
||||
|
||||
// So we can alert if tasks try to release stuff they don't have
|
||||
boolean removed = false;
|
||||
|
||||
if(dsRunning != null) {
|
||||
final TaskLockPosse taskLockPosse = dsRunning.get(interval);
|
||||
if(taskLockPosse != null) {
|
||||
final TaskLock taskLock = taskLockPosse.getTaskLock();
|
||||
|
||||
// Remove task from live list
|
||||
log.info("Removing task[%s] from TaskLock[%s]", task.getId(), taskLock.getGroupId());
|
||||
removed = taskLockPosse.getTaskIds().remove(task.getId());
|
||||
|
||||
if (taskLockPosse.getTaskIds().isEmpty()) {
|
||||
log.info("TaskLock is now empty: %s", taskLock);
|
||||
running.get(dataSource).remove(taskLock.getInterval());
|
||||
}
|
||||
|
||||
if (running.get(dataSource).size() == 0) {
|
||||
running.remove(dataSource);
|
||||
}
|
||||
|
||||
// Best effort to remove lock from storage
|
||||
try {
|
||||
taskStorage.removeLock(task.getId(), taskLock);
|
||||
} catch(Exception e) {
|
||||
log.makeAlert(e, "Failed to clean up lock from storage")
|
||||
.addData("task", task.getId())
|
||||
.addData("dataSource", taskLock.getDataSource())
|
||||
.addData("interval", taskLock.getInterval())
|
||||
.addData("version", taskLock.getVersion())
|
||||
.emit();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if(!removed) {
|
||||
log.makeAlert("Lock release without acquire")
|
||||
.addData("task", task.getId())
|
||||
.addData("interval", interval)
|
||||
.emit();
|
||||
}
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Release all locks for a task. Does nothing if the task is not currently locked.
|
||||
*
|
||||
* @param task task to unlock
|
||||
*/
|
||||
public void unlock(final Task task)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
for(final TaskLockPosse taskLockPosse : findLockPossesForTask(task)) {
|
||||
unlock(task, taskLockPosse.getTaskLock().getInterval());
|
||||
}
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes all locks from this lockbox.
|
||||
*/
|
||||
public void clear()
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
running.clear();
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the currently-active lock posses for some task.
|
||||
*
|
||||
* @param task task for which to locate locks
|
||||
*/
|
||||
private List<TaskLockPosse> findLockPossesForTask(final Task task)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
final Iterable<TaskLockPosse> searchSpace;
|
||||
|
||||
if (task.getFixedInterval().isPresent()) {
|
||||
// Narrow down search using findLockPossesForInterval
|
||||
searchSpace = findLockPossesForInterval(task.getDataSource(), task.getFixedInterval().get());
|
||||
} else {
|
||||
// Scan through all locks for this datasource
|
||||
final NavigableMap<Interval, TaskLockPosse> dsRunning = running.get(task.getDataSource());
|
||||
if(dsRunning == null) {
|
||||
searchSpace = ImmutableList.of();
|
||||
} else {
|
||||
searchSpace = dsRunning.values();
|
||||
}
|
||||
}
|
||||
|
||||
return ImmutableList.copyOf(
|
||||
Iterables.filter(
|
||||
searchSpace, new Predicate<TaskLockPosse>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(TaskLockPosse taskLock)
|
||||
{
|
||||
return taskLock.getTaskIds().contains(task.getId());
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return all locks that overlap some search interval.
|
||||
*/
|
||||
private List<TaskLockPosse> findLockPossesForInterval(final String dataSource, final Interval interval)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
final NavigableMap<Interval, TaskLockPosse> dsRunning = running.get(dataSource);
|
||||
if (dsRunning == null) {
|
||||
// No locks at all
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
// Tasks are indexed by locked interval, which are sorted by interval start. Intervals are non-overlapping, so:
|
||||
final NavigableSet<Interval> dsLockbox = dsRunning.navigableKeySet();
|
||||
final Iterable<Interval> searchIntervals = Iterables.concat(
|
||||
// Single interval that starts at or before ours
|
||||
Collections.singletonList(dsLockbox.floor(new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)))),
|
||||
|
||||
// All intervals that start somewhere between our start instant (exclusive) and end instant (exclusive)
|
||||
dsLockbox.subSet(
|
||||
new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)),
|
||||
false,
|
||||
new Interval(interval.getEnd(), interval.getEnd()),
|
||||
false
|
||||
)
|
||||
);
|
||||
|
||||
return Lists.newArrayList(
|
||||
FunctionalIterable
|
||||
.create(searchIntervals)
|
||||
.filter(
|
||||
new Predicate<Interval>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(@Nullable Interval searchInterval)
|
||||
{
|
||||
return searchInterval != null && searchInterval.overlaps(interval);
|
||||
}
|
||||
}
|
||||
)
|
||||
.transform(
|
||||
new Function<Interval, TaskLockPosse>()
|
||||
{
|
||||
@Override
|
||||
public TaskLockPosse apply(Interval interval)
|
||||
{
|
||||
return dsRunning.get(interval);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
private static class TaskLockPosse
|
||||
{
|
||||
final private TaskLock taskLock;
|
||||
final private Set<String> taskIds;
|
||||
|
||||
public TaskLockPosse(TaskLock taskLock)
|
||||
{
|
||||
this.taskLock = taskLock;
|
||||
taskIds = Sets.newHashSet();
|
||||
}
|
||||
|
||||
public TaskLock getTaskLock()
|
||||
{
|
||||
return taskLock;
|
||||
}
|
||||
|
||||
public Set<String> getTaskIds()
|
||||
{
|
||||
return taskIds;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ import com.metamx.common.lifecycle.LifecycleStart;
|
|||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.druid.initialization.Initialization;
|
||||
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
||||
import com.metamx.druid.merger.coordinator.exec.TaskConsumer;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
@ -40,26 +41,31 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||
/**
|
||||
* Encapsulates the indexer leadership lifecycle.
|
||||
*/
|
||||
public class TaskMaster
|
||||
public class TaskMasterLifecycle
|
||||
{
|
||||
private final LeaderSelector leaderSelector;
|
||||
private final ReentrantLock giant = new ReentrantLock();
|
||||
private final Condition mayBeStopped = giant.newCondition();
|
||||
private final TaskQueue taskQueue;
|
||||
private final TaskToolbox taskToolbox;
|
||||
|
||||
private volatile boolean leading = false;
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(TaskMaster.class);
|
||||
private static final EmittingLogger log = new EmittingLogger(TaskMasterLifecycle.class);
|
||||
|
||||
public TaskMaster(
|
||||
final TaskQueue queue,
|
||||
public TaskMasterLifecycle(
|
||||
final TaskQueue taskQueue,
|
||||
final TaskToolbox taskToolbox,
|
||||
final IndexerCoordinatorConfig indexerCoordinatorConfig,
|
||||
final ServiceDiscoveryConfig serviceDiscoveryConfig,
|
||||
final MergerDBCoordinator mergerDBCoordinator,
|
||||
final TaskRunnerFactory runnerFactory,
|
||||
final CuratorFramework curator,
|
||||
final ServiceEmitter emitter
|
||||
)
|
||||
)
|
||||
{
|
||||
this.taskQueue = taskQueue;
|
||||
this.taskToolbox = taskToolbox;
|
||||
|
||||
this.leaderSelector = new LeaderSelector(
|
||||
curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener()
|
||||
{
|
||||
|
@ -71,15 +77,23 @@ public class TaskMaster
|
|||
try {
|
||||
log.info("By the power of Grayskull, I have the power!");
|
||||
|
||||
final TaskRunner runner = runnerFactory.build();
|
||||
final TaskConsumer consumer = new TaskConsumer(queue, runner, mergerDBCoordinator, emitter);
|
||||
final TaskRunner taskRunner = runnerFactory.build();
|
||||
final TaskConsumer taskConsumer = new TaskConsumer(
|
||||
taskQueue,
|
||||
taskRunner,
|
||||
taskToolbox,
|
||||
emitter
|
||||
);
|
||||
|
||||
// Bootstrap task queue and task lockbox (load state stuff from the database)
|
||||
taskQueue.bootstrap();
|
||||
|
||||
// Sensible order to start stuff:
|
||||
final Lifecycle leaderLifecycle = new Lifecycle();
|
||||
leaderLifecycle.addManagedInstance(queue);
|
||||
leaderLifecycle.addManagedInstance(runner);
|
||||
leaderLifecycle.addManagedInstance(taskQueue);
|
||||
leaderLifecycle.addManagedInstance(taskRunner);
|
||||
Initialization.makeServiceDiscoveryClient(curator, serviceDiscoveryConfig, leaderLifecycle);
|
||||
leaderLifecycle.addManagedInstance(consumer);
|
||||
leaderLifecycle.addManagedInstance(taskConsumer);
|
||||
leaderLifecycle.start();
|
||||
|
||||
leading = true;
|
||||
|
@ -181,4 +195,14 @@ public class TaskMaster
|
|||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
public TaskQueue getTaskQueue()
|
||||
{
|
||||
return taskQueue;
|
||||
}
|
||||
|
||||
public TaskToolbox getTaskToolbox()
|
||||
{
|
||||
return taskToolbox;
|
||||
}
|
||||
}
|
|
@ -19,36 +19,22 @@
|
|||
|
||||
package com.metamx.druid.merger.coordinator;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.HashMultimap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Multimap;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.metamx.common.guava.Comparators;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
|
@ -59,12 +45,12 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||
* {@link #take} or {@link #poll}. Ordering is mostly-FIFO, with deviations when the natural next task would conflict
|
||||
* with a currently-running task. In that case, tasks are skipped until a runnable one is found.
|
||||
* <p/>
|
||||
* To manage locking, the queue keeps track of currently-running tasks as {@link TaskGroup} objects. The idea is that
|
||||
* only one TaskGroup can be running on a particular dataSource + interval, and that TaskGroup has a single version
|
||||
* string that all tasks in the group must use to publish segments. Tasks in the same TaskGroup may run concurrently.
|
||||
* To manage locking, the queue keeps track of currently-running tasks as {@link com.metamx.druid.merger.common.TaskLock} objects. The idea is that
|
||||
* only one TaskLock can be running on a particular dataSource + interval, and that TaskLock has a single version
|
||||
* string that all tasks in the group must use to publish segments. Tasks in the same TaskLock may run concurrently.
|
||||
* <p/>
|
||||
* For persistence, the queue saves new tasks from {@link #add} and task status updates from {@link #notify} using a
|
||||
* {@link TaskStorage} object.
|
||||
* {@link TaskStorage} obj
|
||||
* <p/>
|
||||
* To support leader election of our containing system, the queue can be stopped (in which case it will not accept
|
||||
* any new tasks, or hand out any more tasks, until started again).
|
||||
|
@ -72,10 +58,8 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||
public class TaskQueue
|
||||
{
|
||||
private final List<Task> queue = Lists.newLinkedList();
|
||||
private final Map<String, NavigableMap<Interval, TaskGroup>> running = Maps.newHashMap();
|
||||
|
||||
private final TaskStorage taskStorage;
|
||||
|
||||
private final TaskLockbox taskLockbox;
|
||||
private final ReentrantLock giant = new ReentrantLock();
|
||||
private final Condition workMayBeAvailable = giant.newCondition();
|
||||
|
||||
|
@ -83,14 +67,98 @@ public class TaskQueue
|
|||
|
||||
private static final EmittingLogger log = new EmittingLogger(TaskQueue.class);
|
||||
|
||||
public TaskQueue(TaskStorage taskStorage)
|
||||
public TaskQueue(TaskStorage taskStorage, TaskLockbox taskLockbox)
|
||||
{
|
||||
this.taskStorage = Preconditions.checkNotNull(taskStorage, "taskStorage");
|
||||
this.taskLockbox = Preconditions.checkNotNull(taskLockbox, "taskLockbox");
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts this task queue. Loads tasks from our task storage facility and allows {@link #add(Task)} to accept
|
||||
* new tasks. This should not be called on an already-started queue.
|
||||
* Bootstraps this task queue and associated task lockbox. Clears the lockbox before running. Should be called
|
||||
* while the queue is stopped. It is not a good idea to start the queue if this method fails.
|
||||
*/
|
||||
public void bootstrap()
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
Preconditions.checkState(!active, "queue must be stopped");
|
||||
|
||||
log.info("Bootstrapping queue (and associated lockbox)");
|
||||
|
||||
queue.clear();
|
||||
taskLockbox.clear();
|
||||
|
||||
// Add running tasks to the queue
|
||||
final List<Task> runningTasks = taskStorage.getRunningTasks();
|
||||
|
||||
for(final Task task : runningTasks) {
|
||||
queue.add(task);
|
||||
}
|
||||
|
||||
// Get all locks, along with which tasks they belong to
|
||||
final Multimap<TaskLock, Task> tasksByLock = ArrayListMultimap.create();
|
||||
for(final Task runningTask : runningTasks) {
|
||||
for(final TaskLock taskLock : taskStorage.getLocks(runningTask.getId())) {
|
||||
tasksByLock.put(taskLock, runningTask);
|
||||
}
|
||||
}
|
||||
|
||||
// Sort locks by version
|
||||
final Ordering<TaskLock> byVersionOrdering = new Ordering<TaskLock>()
|
||||
{
|
||||
@Override
|
||||
public int compare(TaskLock left, TaskLock right)
|
||||
{
|
||||
return left.getVersion().compareTo(right.getVersion());
|
||||
}
|
||||
};
|
||||
|
||||
// Acquire as many locks as possible, in version order
|
||||
for(final Map.Entry<TaskLock, Task> taskAndLock : tasksByLock.entries()) {
|
||||
final Task task = taskAndLock.getValue();
|
||||
final TaskLock savedTaskLock = taskAndLock.getKey();
|
||||
|
||||
final Optional<TaskLock> acquiredTaskLock = taskLockbox.tryLock(
|
||||
task,
|
||||
savedTaskLock.getInterval(),
|
||||
Optional.of(savedTaskLock.getVersion())
|
||||
);
|
||||
|
||||
if(acquiredTaskLock.isPresent() && savedTaskLock.getVersion().equals(acquiredTaskLock.get().getVersion())) {
|
||||
log.info(
|
||||
"Reacquired lock on interval[%s] version[%s] for task: %s",
|
||||
savedTaskLock.getInterval(),
|
||||
savedTaskLock.getVersion(),
|
||||
task.getId()
|
||||
);
|
||||
} else if(acquiredTaskLock.isPresent()) {
|
||||
log.info(
|
||||
"Could not reacquire lock on interval[%s] version[%s] (got version[%s] instead) for task: %s",
|
||||
savedTaskLock.getInterval(),
|
||||
savedTaskLock.getVersion(),
|
||||
acquiredTaskLock.get().getVersion(),
|
||||
task.getId()
|
||||
);
|
||||
} else {
|
||||
log.info(
|
||||
"Could not reacquire lock on interval[%s] version[%s] for task: %s",
|
||||
savedTaskLock.getInterval(),
|
||||
savedTaskLock.getVersion(),
|
||||
task.getId()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
log.info("Bootstrapped %,d tasks. Ready to go!", runningTasks.size());
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts this task queue. Allows {@link #add(Task)} to accept new tasks. This should not be called on
|
||||
* an already-started queue.
|
||||
*/
|
||||
@LifecycleStart
|
||||
public void start()
|
||||
|
@ -98,50 +166,9 @@ public class TaskQueue
|
|||
giant.lock();
|
||||
|
||||
try {
|
||||
|
||||
Preconditions.checkState(!active, "queue was already started!");
|
||||
Preconditions.checkState(queue.isEmpty(), "queue must be empty!");
|
||||
Preconditions.checkState(running.isEmpty(), "running list must be empty!");
|
||||
|
||||
final TaskStorageQueryAdapter taskStorageQueryAdapter = new TaskStorageQueryAdapter(taskStorage);
|
||||
final List<VersionedTaskWrapper> runningTasks = taskStorageQueryAdapter.getRunningTaskVersions();
|
||||
|
||||
// Sort by version, with nulls last
|
||||
final Ordering<VersionedTaskWrapper> byVersionOrdering = new Ordering<VersionedTaskWrapper>()
|
||||
{
|
||||
final private Ordering<String> baseOrdering = Ordering.natural().nullsLast();
|
||||
|
||||
@Override
|
||||
public int compare(VersionedTaskWrapper left, VersionedTaskWrapper right)
|
||||
{
|
||||
return baseOrdering.compare(left.getVersion(), right.getVersion());
|
||||
}
|
||||
};
|
||||
|
||||
for (final VersionedTaskWrapper taskAndVersion : byVersionOrdering.sortedCopy(runningTasks)) {
|
||||
final Task task = taskAndVersion.getTask();
|
||||
final String preferredVersion = taskAndVersion.getVersion();
|
||||
|
||||
queue.add(task);
|
||||
|
||||
if (preferredVersion != null) {
|
||||
final Optional<String> version = tryLock(task, Optional.of(preferredVersion));
|
||||
|
||||
log.info(
|
||||
"Bootstrapped task[%s] with preferred version[%s]: %s",
|
||||
task.getId(),
|
||||
preferredVersion,
|
||||
version.isPresent() ? String.format("locked with version[%s]", version.get()) : "not lockable"
|
||||
);
|
||||
} else {
|
||||
log.info("Bootstrapped task[%s] with no preferred version", task.getId());
|
||||
}
|
||||
}
|
||||
|
||||
log.info("Bootstrapped %,d tasks. Ready to go!", runningTasks.size());
|
||||
Preconditions.checkState(!active, "queue must be stopped");
|
||||
|
||||
active = true;
|
||||
|
||||
workMayBeAvailable.signalAll();
|
||||
}
|
||||
finally {
|
||||
|
@ -159,13 +186,10 @@ public class TaskQueue
|
|||
giant.lock();
|
||||
|
||||
try {
|
||||
|
||||
log.info("Naptime! Shutting down until we are started again.");
|
||||
|
||||
queue.clear();
|
||||
running.clear();
|
||||
taskLockbox.clear();
|
||||
active = false;
|
||||
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
|
@ -201,7 +225,9 @@ public class TaskQueue
|
|||
// Attempt to add this task to a running task group. Silently continue if this is not possible.
|
||||
// The main reason this is here is so when subtasks are added, they end up in the same task group
|
||||
// as their parent whenever possible.
|
||||
tryLock(task);
|
||||
if(task.getFixedInterval().isPresent()) {
|
||||
taskLockbox.tryLock(task, task.getFixedInterval().get());
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -215,19 +241,22 @@ public class TaskQueue
|
|||
*
|
||||
* @return runnable task
|
||||
*/
|
||||
public VersionedTaskWrapper take() throws InterruptedException
|
||||
public Task take() throws InterruptedException
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
VersionedTaskWrapper taskWrapper;
|
||||
Task task;
|
||||
|
||||
while ((taskWrapper = poll()) == null) {
|
||||
log.info("Waiting for work...");
|
||||
workMayBeAvailable.await();
|
||||
log.info("Waiting for work...");
|
||||
|
||||
while ((task = poll()) == null) {
|
||||
// awaitNanos because work may become available without this condition signalling,
|
||||
// due to other folks messing with the taskLockbox
|
||||
workMayBeAvailable.awaitNanos(1000000000L /* 1 second */);
|
||||
}
|
||||
|
||||
return taskWrapper;
|
||||
return task;
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
|
@ -239,24 +268,28 @@ public class TaskQueue
|
|||
*
|
||||
* @return runnable task or null
|
||||
*/
|
||||
public VersionedTaskWrapper poll()
|
||||
public Task poll()
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
log.info("Checking for doable work");
|
||||
for (final Task task : queue) {
|
||||
final Optional<String> maybeVersion = tryLock(task);
|
||||
if (maybeVersion.isPresent()) {
|
||||
Preconditions.checkState(active, "wtf? Found task when inactive");
|
||||
taskStorage.setVersion(task.getId(), maybeVersion.get());
|
||||
if(task.getFixedInterval().isPresent()) {
|
||||
// If this task has a fixed interval, attempt to lock it right now.
|
||||
final Optional<TaskLock> maybeLock = taskLockbox.tryLock(task, task.getFixedInterval().get());
|
||||
if(maybeLock.isPresent()) {
|
||||
log.info("Task claimed with fixed interval lock: %s", task.getId());
|
||||
queue.remove(task);
|
||||
return task;
|
||||
}
|
||||
} else {
|
||||
// No fixed interval. Let's just run this and see what happens.
|
||||
log.info("Task claimed with no fixed interval lock: %s", task.getId());
|
||||
queue.remove(task);
|
||||
log.info("Task claimed: %s", task);
|
||||
return new VersionedTaskWrapper(task, maybeVersion.get());
|
||||
return task;
|
||||
}
|
||||
}
|
||||
|
||||
log.info("No doable work found.");
|
||||
return null;
|
||||
}
|
||||
finally {
|
||||
|
@ -264,348 +297,62 @@ public class TaskQueue
|
|||
}
|
||||
}
|
||||
|
||||
public void notify(final Task task, final TaskStatus status)
|
||||
{
|
||||
notify(task, status, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Notify this queue that some task has an updated status. If this update is valid, the status will be persisted in
|
||||
* the task storage facility, and any nextTasks present in the status will be created. If the status is a completed
|
||||
* status, the task will be unlocked and no further updates will be accepted. If this task has failed, the task group
|
||||
* it is part of will be terminated.
|
||||
* <p/>
|
||||
* Finally, if this task is not supposed to be running, this method will simply do nothing.
|
||||
* the task storage facility. If the status is a completed status, the task will be unlocked and no further
|
||||
* updates will be accepted.
|
||||
*
|
||||
* @param task task to update
|
||||
* @param originalStatus new task status
|
||||
* @param commitRunnable operation to perform if this task is ready to commit
|
||||
* @param taskStatus new task status
|
||||
*
|
||||
* @throws NullPointerException if task or status is null
|
||||
* @throws IllegalArgumentException if the task ID does not match the status ID
|
||||
* @throws IllegalStateException if this queue is currently shut down
|
||||
*/
|
||||
public void notify(final Task task, final TaskStatus originalStatus, final Runnable commitRunnable)
|
||||
public void notify(final Task task, final TaskStatus taskStatus)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
Preconditions.checkNotNull(task, "task");
|
||||
Preconditions.checkNotNull(originalStatus, "status");
|
||||
Preconditions.checkNotNull(taskStatus, "status");
|
||||
Preconditions.checkState(active, "Queue is not active!");
|
||||
Preconditions.checkArgument(
|
||||
task.getId().equals(originalStatus.getId()),
|
||||
task.getId().equals(taskStatus.getId()),
|
||||
"Mismatching task ids[%s/%s]",
|
||||
task.getId(),
|
||||
originalStatus.getId()
|
||||
taskStatus.getId()
|
||||
);
|
||||
|
||||
final TaskGroup taskGroup;
|
||||
|
||||
final Optional<TaskGroup> maybeTaskGroup = findTaskGroupForTask(task);
|
||||
if (!maybeTaskGroup.isPresent()) {
|
||||
log.info("Ignoring notification for dead task: %s", task.getId());
|
||||
return;
|
||||
} else {
|
||||
taskGroup = maybeTaskGroup.get();
|
||||
}
|
||||
|
||||
// This is what we want to write to the DB when we're done.
|
||||
// Not final, since we might need to reassign the var later if the commitRunnable fails.
|
||||
TaskStatus statusToSave = originalStatus;
|
||||
|
||||
// Should we commit?
|
||||
if (taskGroup.getCommitStyle().shouldCommit(task, statusToSave)) {
|
||||
log.info("Committing %s status for task: %s", statusToSave.getStatusCode(), task.getId());
|
||||
|
||||
// Add next tasks
|
||||
try {
|
||||
if (commitRunnable != null) {
|
||||
log.info("Running commitRunnable for task: %s", task.getId());
|
||||
commitRunnable.run();
|
||||
}
|
||||
|
||||
// We want to allow tasks to submit RUNNING statuses with the same nextTasks over and over.
|
||||
// So, we need to remember which ones we've already spawned and not do them again.
|
||||
for (final Task nextTask : statusToSave.getNextTasks()) {
|
||||
try {
|
||||
add(nextTask);
|
||||
} catch (TaskExistsException e) {
|
||||
log.info("Already added followup task %s to original task: %s", nextTask.getId(), task.getId());
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert(e, "Failed to commit task")
|
||||
.addData("task", task.getId())
|
||||
.addData("statusCode", statusToSave.getStatusCode())
|
||||
.emit();
|
||||
|
||||
// Rewrite status
|
||||
statusToSave = TaskStatus.failure(task.getId()).withDuration(statusToSave.getDuration());
|
||||
}
|
||||
} else {
|
||||
log.info("Not committing %s status for task: %s", statusToSave.getStatusCode(), task);
|
||||
}
|
||||
|
||||
boolean didSetStatus = false;
|
||||
|
||||
// Save status to DB
|
||||
boolean didPersistStatus = false;
|
||||
try {
|
||||
taskStorage.setStatus(task.getId(), statusToSave);
|
||||
didSetStatus = true;
|
||||
final Optional<TaskStatus> previousStatus = taskStorage.getStatus(task.getId());
|
||||
if (!previousStatus.isPresent() || !previousStatus.get().isRunnable()) {
|
||||
log.makeAlert("Ignoring notification for dead task").addData("task", task.getId()).emit();
|
||||
return;
|
||||
} else {
|
||||
taskStorage.setStatus(taskStatus);
|
||||
didPersistStatus = true;
|
||||
}
|
||||
} catch(Exception e) {
|
||||
// TODO: This could be a task-status-submission retry queue instead of retrying the entire task,
|
||||
// TODO: which is heavy and probably not necessary.
|
||||
log.warn(e, "Status could not be persisted! Reinserting task: %s", task.getId());
|
||||
|
||||
log.makeAlert(e, "Failed to persist task status")
|
||||
log.makeAlert(e, "Failed to persist status for task")
|
||||
.addData("task", task.getId())
|
||||
.addData("statusCode", statusToSave.getStatusCode())
|
||||
.addData("statusCode", taskStatus.getStatusCode())
|
||||
.emit();
|
||||
|
||||
queue.add(task);
|
||||
}
|
||||
|
||||
if(didSetStatus && statusToSave.isComplete()) {
|
||||
unlock(task);
|
||||
log.info("Task done: %s", task);
|
||||
}
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Unlock some work. Does not update the task storage facility. Throws an exception if this work is not currently
|
||||
* running.
|
||||
*
|
||||
* @param task task to unlock
|
||||
*
|
||||
* @throws IllegalStateException if task is not currently locked
|
||||
*/
|
||||
private void unlock(final Task task)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
final String dataSource = task.getDataSource();
|
||||
|
||||
final TaskGroup taskGroup;
|
||||
final Optional<TaskGroup> maybeTaskGroup = findTaskGroupForTask(task);
|
||||
|
||||
if (maybeTaskGroup.isPresent()) {
|
||||
taskGroup = maybeTaskGroup.get();
|
||||
} else {
|
||||
throw new IllegalStateException(String.format("Task must be running: %s", task.getId()));
|
||||
}
|
||||
|
||||
// Remove task from live list
|
||||
log.info("Removing task[%s] from TaskGroup[%s]", task.getId(), taskGroup.getGroupId());
|
||||
taskGroup.remove(task.getId());
|
||||
|
||||
if (taskGroup.size() == 0) {
|
||||
log.info("TaskGroup complete: %s", taskGroup);
|
||||
running.get(dataSource).remove(taskGroup.getInterval());
|
||||
}
|
||||
|
||||
if (running.get(dataSource).size() == 0) {
|
||||
running.remove(dataSource);
|
||||
}
|
||||
|
||||
workMayBeAvailable.signalAll();
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task.
|
||||
*
|
||||
* @param task task to attempt to lock
|
||||
*
|
||||
* @return lock version if lock was acquired, absent otherwise
|
||||
*/
|
||||
private Optional<String> tryLock(final Task task)
|
||||
{
|
||||
return tryLock(task, Optional.<String>absent());
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task.
|
||||
*
|
||||
* @param task task to attempt to lock
|
||||
* @param preferredVersion use this version if possible (no guarantees, though!)
|
||||
*
|
||||
* @return lock version if lock was acquired, absent otherwise
|
||||
*/
|
||||
private Optional<String> tryLock(final Task task, final Optional<String> preferredVersion)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
|
||||
final String dataSource = task.getDataSource();
|
||||
final Interval interval = task.getInterval();
|
||||
|
||||
final List<TaskGroup> foundLocks = findTaskGroupsForInterval(dataSource, interval);
|
||||
final TaskGroup taskGroupToUse;
|
||||
|
||||
if (foundLocks.size() > 1) {
|
||||
|
||||
// Too many existing locks.
|
||||
return Optional.absent();
|
||||
|
||||
} else if (foundLocks.size() == 1) {
|
||||
|
||||
// One existing lock -- check if we can add to it.
|
||||
|
||||
final TaskGroup foundLock = Iterables.getOnlyElement(foundLocks);
|
||||
if (foundLock.getInterval().contains(interval) && foundLock.getGroupId().equals(task.getGroupId())) {
|
||||
taskGroupToUse = foundLock;
|
||||
if(taskStatus.isComplete()) {
|
||||
if(didPersistStatus) {
|
||||
log.info("Task done: %s", task);
|
||||
taskLockbox.unlock(task);
|
||||
workMayBeAvailable.signalAll();
|
||||
} else {
|
||||
return Optional.absent();
|
||||
// TODO: This could be a task-status-submission retry queue instead of retrying the entire task,
|
||||
// TODO: which is heavy and probably not necessary.
|
||||
log.warn("Status could not be persisted! Reinserting task: %s", task.getId());
|
||||
queue.add(task);
|
||||
}
|
||||
|
||||
} else {
|
||||
|
||||
// No existing locks. We can make a new one.
|
||||
if (!running.containsKey(dataSource)) {
|
||||
running.put(dataSource, new TreeMap<Interval, TaskGroup>(Comparators.intervalsByStartThenEnd()));
|
||||
}
|
||||
|
||||
// Create new TaskGroup and assign it a version.
|
||||
// Assumption: We'll choose a version that is greater than any previously-chosen version for our interval. (This
|
||||
// may not always be true, unfortunately. See below.)
|
||||
|
||||
final String version;
|
||||
|
||||
if (preferredVersion.isPresent()) {
|
||||
// We have a preferred version. Since this is a private method, we'll trust our caller to not break our
|
||||
// ordering assumptions and just use it.
|
||||
version = preferredVersion.get();
|
||||
} else {
|
||||
// We are running under an interval lock right now, so just using the current time works as long as we can trust
|
||||
// our clock to be monotonic and have enough resolution since the last time we created a TaskGroup for the same
|
||||
// interval. This may not always be true; to assure it we would need to use some method of timekeeping other
|
||||
// than the wall clock.
|
||||
version = new DateTime().toString();
|
||||
}
|
||||
|
||||
taskGroupToUse = new TaskGroup(task.getGroupId(), dataSource, interval, version);
|
||||
running.get(dataSource)
|
||||
.put(interval, taskGroupToUse);
|
||||
|
||||
log.info("Created new TaskGroup[%s]", taskGroupToUse);
|
||||
|
||||
}
|
||||
|
||||
// Add to existing TaskGroup, if necessary
|
||||
if (taskGroupToUse.add(task)) {
|
||||
log.info("Added task[%s] to TaskGroup[%s]", task.getId(), taskGroupToUse.getGroupId());
|
||||
} else {
|
||||
log.info("Task[%s] already present in TaskGroup[%s]", task.getId(), taskGroupToUse.getGroupId());
|
||||
}
|
||||
|
||||
return Optional.of(taskGroupToUse.getVersion());
|
||||
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the currently-running task group for some task. If the task has no currently-running task group, this will
|
||||
* return an absentee Optional.
|
||||
*
|
||||
* @param task task for which to locate group
|
||||
*/
|
||||
private Optional<TaskGroup> findTaskGroupForTask(final Task task)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
final Iterator<TaskGroup> maybeTaskGroup =
|
||||
FunctionalIterable.create(findTaskGroupsForInterval(task.getDataSource(), task.getInterval()))
|
||||
.filter(
|
||||
new Predicate<TaskGroup>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(TaskGroup taskGroup)
|
||||
{
|
||||
return taskGroup.contains(task.getId());
|
||||
}
|
||||
}
|
||||
)
|
||||
.iterator();
|
||||
|
||||
|
||||
if (!maybeTaskGroup.hasNext()) {
|
||||
return Optional.absent();
|
||||
} else {
|
||||
return Optional.of(Iterators.getOnlyElement(maybeTaskGroup));
|
||||
}
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return all locks that overlap some search interval.
|
||||
*/
|
||||
private List<TaskGroup> findTaskGroupsForInterval(final String dataSource, final Interval interval)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
final NavigableMap<Interval, TaskGroup> dsRunning = running.get(dataSource);
|
||||
if (dsRunning == null) {
|
||||
// No locks at all
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
// Tasks are indexed by locked interval, which are sorted by interval start. Intervals are non-overlapping, so:
|
||||
final NavigableSet<Interval> dsLockbox = dsRunning.navigableKeySet();
|
||||
final Iterable<Interval> searchIntervals = Iterables.concat(
|
||||
// Single interval that starts at or before ours
|
||||
Collections.singletonList(dsLockbox.floor(new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)))),
|
||||
|
||||
// All intervals that start somewhere between our start instant (exclusive) and end instant (exclusive)
|
||||
dsLockbox.subSet(
|
||||
new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)),
|
||||
false,
|
||||
new Interval(interval.getEnd(), interval.getEnd()),
|
||||
false
|
||||
)
|
||||
);
|
||||
|
||||
return Lists.newArrayList(
|
||||
FunctionalIterable
|
||||
.create(searchIntervals)
|
||||
.filter(
|
||||
new Predicate<Interval>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(@Nullable Interval searchInterval)
|
||||
{
|
||||
return searchInterval != null && searchInterval.overlaps(interval);
|
||||
}
|
||||
}
|
||||
)
|
||||
.transform(
|
||||
new Function<Interval, TaskGroup>()
|
||||
{
|
||||
@Override
|
||||
public TaskGroup apply(Interval interval)
|
||||
{
|
||||
return dsRunning.get(interval);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
finally {
|
||||
|
|
|
@ -23,8 +23,8 @@ import com.metamx.druid.merger.common.TaskCallback;
|
|||
import com.metamx.druid.merger.common.task.Task;
|
||||
|
||||
/**
|
||||
* Interface for handing off tasks. Used by a {@link com.metamx.druid.merger.coordinator.exec.TaskConsumer} to run tasks that
|
||||
* have been locked.
|
||||
* Interface for handing off tasks. Used by a {@link com.metamx.druid.merger.coordinator.exec.TaskConsumer} to
|
||||
* run tasks that have been locked.
|
||||
*/
|
||||
public interface TaskRunner
|
||||
{
|
||||
|
@ -33,8 +33,7 @@ public interface TaskRunner
|
|||
* status, but should be called exactly once with a non-RUNNING status (e.g. SUCCESS, FAILED, CONTINUED...).
|
||||
*
|
||||
* @param task task to run
|
||||
* @param context task context to run under
|
||||
* @param callback callback to be called exactly once
|
||||
*/
|
||||
public void run(Task task, TaskContext context, TaskCallback callback);
|
||||
public void run(Task task, TaskCallback callback);
|
||||
}
|
||||
|
|
|
@ -21,6 +21,8 @@ package com.metamx.druid.merger.coordinator;
|
|||
|
||||
import com.google.common.base.Optional;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.actions.TaskAction;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -34,15 +36,21 @@ public interface TaskStorage
|
|||
public void insert(Task task, TaskStatus status);
|
||||
|
||||
/**
|
||||
* Updates task status in the storage facility.
|
||||
* Persists task status in the storage facility. This method should throw an exception if the task status lifecycle
|
||||
* is not respected (absent -> RUNNING -> SUCCESS/FAILURE).
|
||||
*/
|
||||
public void setStatus(String taskid, TaskStatus status);
|
||||
public void setStatus(TaskStatus status);
|
||||
|
||||
/**
|
||||
* Updates task version in the storage facility. If the task already has a version, this method will throw
|
||||
* an exception.
|
||||
* Persists lock state in the storage facility.
|
||||
*/
|
||||
public void setVersion(String taskid, String version);
|
||||
public void addLock(String taskid, TaskLock taskLock);
|
||||
|
||||
/**
|
||||
* Removes lock state from the storage facility. It is harmless to keep old locks in the storage facility, but
|
||||
* this method can help reclaim wasted space.
|
||||
*/
|
||||
public void removeLock(String taskid, TaskLock taskLock);
|
||||
|
||||
/**
|
||||
* Returns task as stored in the storage facility. If the task ID does not exist, this will return an
|
||||
|
@ -59,13 +67,22 @@ public interface TaskStorage
|
|||
public Optional<TaskStatus> getStatus(String taskid);
|
||||
|
||||
/**
|
||||
* Returns task version as stored in the storage facility. If the task ID does not exist, or if the task ID exists
|
||||
* but was not yet assigned a version, this will return an absentee Optional.
|
||||
* Add an action taken by a task to the audit log.
|
||||
*/
|
||||
public Optional<String> getVersion(String taskid);
|
||||
public <T> void addAuditLog(TaskAction<T> taskAction);
|
||||
|
||||
/**
|
||||
* Returns all actions taken by a task.
|
||||
*/
|
||||
public List<TaskAction> getAuditLogs(String taskid);
|
||||
|
||||
/**
|
||||
* Returns a list of currently-running tasks as stored in the storage facility, in no particular order.
|
||||
*/
|
||||
public List<Task> getRunningTasks();
|
||||
|
||||
/**
|
||||
* Returns a list of locks for a particular task.
|
||||
*/
|
||||
public List<TaskLock> getLocks(String taskid);
|
||||
}
|
||||
|
|
|
@ -21,11 +21,17 @@ package com.metamx.druid.merger.coordinator;
|
|||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
|
||||
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
|
||||
import com.metamx.druid.merger.common.actions.TaskAction;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -62,8 +68,29 @@ public class TaskStorageQueryAdapter
|
|||
|
||||
resultBuilder.put(taskid, statusOptional);
|
||||
|
||||
final Iterable<Task> nextTasks = FunctionalIterable
|
||||
.create(storage.getAuditLogs(taskid)).filter(
|
||||
new Predicate<TaskAction>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(TaskAction taskAction)
|
||||
{
|
||||
return taskAction instanceof SpawnTasksAction;
|
||||
}
|
||||
}
|
||||
).transformCat(
|
||||
new Function<TaskAction, Iterable<Task>>()
|
||||
{
|
||||
@Override
|
||||
public Iterable<Task> apply(TaskAction taskAction)
|
||||
{
|
||||
return ((SpawnTasksAction) taskAction).getNewTasks();
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
if(taskOptional.isPresent() && statusOptional.isPresent()) {
|
||||
for(final Task nextTask : statusOptional.get().getNextTasks()) {
|
||||
for(final Task nextTask : nextTasks) {
|
||||
if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) {
|
||||
resultBuilder.putAll(getSameGroupChildStatuses(nextTask.getId()));
|
||||
}
|
||||
|
@ -84,20 +111,12 @@ public class TaskStorageQueryAdapter
|
|||
int nFailures = 0;
|
||||
int nTotal = 0;
|
||||
|
||||
final Set<DataSegment> segments = Sets.newHashSet();
|
||||
final Set<DataSegment> segmentsNuked = Sets.newHashSet();
|
||||
final List<Task> nextTasks = Lists.newArrayList();
|
||||
|
||||
for(final Optional<TaskStatus> statusOption : statuses.values()) {
|
||||
nTotal ++;
|
||||
|
||||
if(statusOption.isPresent()) {
|
||||
final TaskStatus status = statusOption.get();
|
||||
|
||||
segments.addAll(status.getSegments());
|
||||
segmentsNuked.addAll(status.getSegmentsNuked());
|
||||
nextTasks.addAll(status.getNextTasks());
|
||||
|
||||
if(status.isSuccess()) {
|
||||
nSuccesses ++;
|
||||
} else if(status.isFailure()) {
|
||||
|
@ -111,10 +130,7 @@ public class TaskStorageQueryAdapter
|
|||
if(nTotal == 0) {
|
||||
status = Optional.absent();
|
||||
} else if(nSuccesses == nTotal) {
|
||||
status = Optional.of(TaskStatus.success(taskid)
|
||||
.withSegments(segments)
|
||||
.withSegmentsNuked(segmentsNuked)
|
||||
.withNextTasks(nextTasks));
|
||||
status = Optional.of(TaskStatus.success(taskid));
|
||||
} else if(nFailures > 0) {
|
||||
status = Optional.of(TaskStatus.failure(taskid));
|
||||
} else {
|
||||
|
@ -125,21 +141,37 @@ public class TaskStorageQueryAdapter
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns running tasks along with their preferred versions. If no version is present for a task, the
|
||||
* version field of the returned {@link VersionedTaskWrapper} will be null.
|
||||
* Returns all segments created by descendants for a particular task that stayed within the same task group. Includes
|
||||
* that task, plus any tasks it spawned, and so on. Does not include spawned tasks that ended up in a different task
|
||||
* group. Does not include this task's parents or siblings.
|
||||
*/
|
||||
public List<VersionedTaskWrapper> getRunningTaskVersions()
|
||||
public Set<DataSegment> getSameGroupNewSegments(final String taskid)
|
||||
{
|
||||
return Lists.transform(
|
||||
storage.getRunningTasks(),
|
||||
new Function<Task, VersionedTaskWrapper>()
|
||||
{
|
||||
@Override
|
||||
public VersionedTaskWrapper apply(Task task)
|
||||
{
|
||||
return new VersionedTaskWrapper(task, storage.getVersion(task.getId()).orNull());
|
||||
}
|
||||
// TODO: This is useful for regular index tasks (so we know what was published), but
|
||||
// TODO: for long-lived index tasks the list can get out of hand. We may want a limit.
|
||||
|
||||
final Optional<Task> taskOptional = storage.getTask(taskid);
|
||||
final Set<DataSegment> segments = Sets.newHashSet();
|
||||
final List<Task> nextTasks = Lists.newArrayList();
|
||||
|
||||
for(final TaskAction action : storage.getAuditLogs(taskid)) {
|
||||
if(action instanceof SpawnTasksAction) {
|
||||
nextTasks.addAll(((SpawnTasksAction) action).getNewTasks());
|
||||
}
|
||||
|
||||
if(action instanceof SegmentInsertAction) {
|
||||
segments.addAll(((SegmentInsertAction) action).getSegments());
|
||||
}
|
||||
}
|
||||
|
||||
if(taskOptional.isPresent()) {
|
||||
for(final Task nextTask : nextTasks) {
|
||||
if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) {
|
||||
segments.addAll(getSameGroupNewSegments(nextTask.getId()));
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
return segments;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,14 +27,12 @@ import com.metamx.druid.merger.common.task.Task;
|
|||
public class TaskWrapper
|
||||
{
|
||||
private final Task task;
|
||||
private final TaskContext taskContext;
|
||||
private final TaskCallback callback;
|
||||
private final RetryPolicy retryPolicy;
|
||||
|
||||
public TaskWrapper(Task task, TaskContext taskContext, TaskCallback callback, RetryPolicy retryPolicy)
|
||||
public TaskWrapper(Task task, TaskCallback callback, RetryPolicy retryPolicy)
|
||||
{
|
||||
this.task = task;
|
||||
this.taskContext = taskContext;
|
||||
this.callback = callback;
|
||||
this.retryPolicy = retryPolicy;
|
||||
}
|
||||
|
@ -44,11 +42,6 @@ public class TaskWrapper
|
|||
return task;
|
||||
}
|
||||
|
||||
public TaskContext getTaskContext()
|
||||
{
|
||||
return taskContext;
|
||||
}
|
||||
|
||||
public TaskCallback getCallback()
|
||||
{
|
||||
return callback;
|
||||
|
|
|
@ -1,44 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.coordinator;
|
||||
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
|
||||
public class VersionedTaskWrapper
|
||||
{
|
||||
final Task task;
|
||||
final String version;
|
||||
|
||||
public VersionedTaskWrapper(Task task, String version)
|
||||
{
|
||||
this.task = task;
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
public String getVersion()
|
||||
{
|
||||
return version;
|
||||
}
|
||||
}
|
|
@ -1,32 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.coordinator.commit;
|
||||
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.TaskGroup;
|
||||
|
||||
/**
|
||||
* Determines whether or not metadata from a task status update should be committed.
|
||||
*/
|
||||
public interface CommitStyle
|
||||
{
|
||||
public boolean shouldCommit(Task task, TaskStatus taskStatus);
|
||||
}
|
|
@ -1,33 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.coordinator.commit;
|
||||
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.TaskGroup;
|
||||
|
||||
public class ImmediateCommitStyle implements CommitStyle
|
||||
{
|
||||
@Override
|
||||
public boolean shouldCommit(Task task, TaskStatus taskStatus)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -1,32 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.coordinator.commit;
|
||||
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
|
||||
public class TaskCommitStyle implements CommitStyle
|
||||
{
|
||||
@Override
|
||||
public boolean shouldCommit(Task task, TaskStatus taskStatus)
|
||||
{
|
||||
return taskStatus.isSuccess();
|
||||
}
|
||||
}
|
|
@ -28,4 +28,12 @@ public abstract class IndexerDbConnectorConfig extends DbConnectorConfig
|
|||
@JsonProperty("taskTable")
|
||||
@Config("druid.database.taskTable")
|
||||
public abstract String getTaskTable();
|
||||
|
||||
@JsonProperty("taskLockTable")
|
||||
@Config("druid.database.taskLockTable")
|
||||
public abstract String getTaskLockTable();
|
||||
|
||||
@JsonProperty("taskLogTable")
|
||||
@Config("druid.database.taskLogTable")
|
||||
public abstract String getTaskLogTable();
|
||||
}
|
||||
|
|
|
@ -20,29 +20,23 @@
|
|||
package com.metamx.druid.merger.coordinator.exec;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
import com.metamx.druid.merger.coordinator.TaskQueue;
|
||||
import com.metamx.druid.merger.coordinator.TaskRunner;
|
||||
import com.metamx.druid.merger.coordinator.VersionedTaskWrapper;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
public class TaskConsumer implements Runnable
|
||||
{
|
||||
private final TaskQueue queue;
|
||||
private final TaskRunner runner;
|
||||
private final MergerDBCoordinator mergerDBCoordinator;
|
||||
private final TaskToolbox toolbox;
|
||||
private final ServiceEmitter emitter;
|
||||
private final Thread thready;
|
||||
|
||||
|
@ -53,13 +47,13 @@ public class TaskConsumer implements Runnable
|
|||
public TaskConsumer(
|
||||
TaskQueue queue,
|
||||
TaskRunner runner,
|
||||
MergerDBCoordinator mergerDBCoordinator,
|
||||
TaskToolbox toolbox,
|
||||
ServiceEmitter emitter
|
||||
)
|
||||
{
|
||||
this.queue = queue;
|
||||
this.runner = runner;
|
||||
this.mergerDBCoordinator = mergerDBCoordinator;
|
||||
this.toolbox = toolbox;
|
||||
this.emitter = emitter;
|
||||
this.thready = new Thread(this);
|
||||
}
|
||||
|
@ -85,12 +79,9 @@ public class TaskConsumer implements Runnable
|
|||
while (!Thread.currentThread().isInterrupted()) {
|
||||
|
||||
final Task task;
|
||||
final String version;
|
||||
|
||||
try {
|
||||
final VersionedTaskWrapper taskWrapper = queue.take();
|
||||
task = taskWrapper.getTask();
|
||||
version = taskWrapper.getVersion();
|
||||
task = queue.take();
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
log.info(e, "Interrupted while waiting for new work");
|
||||
|
@ -98,17 +89,17 @@ public class TaskConsumer implements Runnable
|
|||
}
|
||||
|
||||
try {
|
||||
handoff(task, version);
|
||||
handoff(task);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert(e, "Failed to hand off task")
|
||||
.addData("task", task.getId())
|
||||
.addData("type", task.getType().toString())
|
||||
.addData("type", task.getType())
|
||||
.addData("dataSource", task.getDataSource())
|
||||
.addData("interval", task.getInterval())
|
||||
.addData("interval", task.getFixedInterval())
|
||||
.emit();
|
||||
|
||||
// Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now.
|
||||
// Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now.
|
||||
if (!shutdown) {
|
||||
queue.notify(task, TaskStatus.failure(task.getId()));
|
||||
}
|
||||
|
@ -122,32 +113,17 @@ public class TaskConsumer implements Runnable
|
|||
}
|
||||
}
|
||||
|
||||
private void handoff(final Task task, final String version) throws Exception
|
||||
private void handoff(final Task task) throws Exception
|
||||
{
|
||||
final TaskContext context = new TaskContext(
|
||||
version,
|
||||
ImmutableSet.copyOf(
|
||||
mergerDBCoordinator.getUsedSegmentsForInterval(
|
||||
task.getDataSource(),
|
||||
task.getInterval()
|
||||
)
|
||||
),
|
||||
ImmutableSet.copyOf(
|
||||
mergerDBCoordinator.getUnusedSegmentsForInterval(
|
||||
task.getDataSource(),
|
||||
task.getInterval()
|
||||
)
|
||||
)
|
||||
);
|
||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder()
|
||||
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
|
||||
.setUser2(task.getDataSource())
|
||||
.setUser4(task.getType().toString())
|
||||
.setUser5(task.getInterval().toString());
|
||||
.setUser4(task.getType())
|
||||
.setUser5(task.getFixedInterval().toString());
|
||||
|
||||
// Run preflight checks
|
||||
TaskStatus preflightStatus;
|
||||
try {
|
||||
preflightStatus = task.preflight(context);
|
||||
preflightStatus = task.preflight(toolbox);
|
||||
log.info("Preflight done for task: %s", task.getId());
|
||||
}
|
||||
catch (Exception e) {
|
||||
|
@ -161,11 +137,9 @@ public class TaskConsumer implements Runnable
|
|||
return;
|
||||
}
|
||||
|
||||
// Hand off work to TaskRunner
|
||||
// TODO -- Should something in the TaskCallback enforce that each returned status is logically after the previous?
|
||||
// TODO -- Probably yes. But make sure it works in the face of RTR retries.
|
||||
// Hand off work to TaskRunner, with a callback
|
||||
runner.run(
|
||||
task, context, new TaskCallback()
|
||||
task, new TaskCallback()
|
||||
{
|
||||
@Override
|
||||
public void notify(final TaskStatus statusFromRunner)
|
||||
|
@ -180,82 +154,26 @@ public class TaskConsumer implements Runnable
|
|||
return;
|
||||
}
|
||||
|
||||
queue.notify(
|
||||
task, statusFromRunner, new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
try {
|
||||
// Validate status
|
||||
for (final DataSegment segment : statusFromRunner.getSegments()) {
|
||||
verifyDataSourceAndInterval(task, context, segment);
|
||||
|
||||
// Verify version (must be equal to our context version)
|
||||
if (!context.getVersion().equals(segment.getVersion())) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Segment for task[%s] has invalid version: %s",
|
||||
task.getId(),
|
||||
segment.getIdentifier()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
for (final DataSegment segment : statusFromRunner.getSegmentsNuked()) {
|
||||
verifyDataSourceAndInterval(task, context, segment);
|
||||
|
||||
// Verify version (must be less than our context version)
|
||||
if (segment.getVersion().compareTo(context.getVersion()) >= 0) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Segment-to-nuke for task[%s] has invalid version: %s",
|
||||
task.getId(),
|
||||
segment.getIdentifier()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
mergerDBCoordinator.commitTaskStatus(statusFromRunner);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(e, "Exception while publishing segments for task: %s", task);
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
queue.notify(task, statusFromRunner);
|
||||
|
||||
// Emit event and log, if the task is done
|
||||
if (statusFromRunner.isComplete()) {
|
||||
builder.setUser3(statusFromRunner.getStatusCode().toString());
|
||||
|
||||
for (DataSegment segment : statusFromRunner.getSegments()) {
|
||||
emitter.emit(builder.build("indexer/segment/bytes", segment.getSize()));
|
||||
}
|
||||
|
||||
for (DataSegment segmentNuked : statusFromRunner.getSegmentsNuked()) {
|
||||
emitter.emit(builder.build("indexer/segmentNuked/bytes", segmentNuked.getSize()));
|
||||
}
|
||||
|
||||
emitter.emit(builder.build("indexer/time/run/millis", statusFromRunner.getDuration()));
|
||||
metricBuilder.setUser3(statusFromRunner.getStatusCode().toString());
|
||||
emitter.emit(metricBuilder.build("indexer/time/run/millis", statusFromRunner.getDuration()));
|
||||
|
||||
if (statusFromRunner.isFailure()) {
|
||||
log.makeAlert("Failed to index")
|
||||
.addData("task", task.getId())
|
||||
.addData("type", task.getType().toString())
|
||||
.addData("type", task.getType())
|
||||
.addData("dataSource", task.getDataSource())
|
||||
.addData("interval", task.getInterval())
|
||||
.addData("interval", task.getFixedInterval())
|
||||
.emit();
|
||||
}
|
||||
|
||||
log.info(
|
||||
"Task %s: %s (%d segments) (%d run duration)",
|
||||
"Task %s: %s (%d run duration)",
|
||||
statusFromRunner.getStatusCode(),
|
||||
task,
|
||||
statusFromRunner.getSegments().size(),
|
||||
statusFromRunner.getDuration()
|
||||
);
|
||||
}
|
||||
|
@ -270,69 +188,4 @@ public class TaskConsumer implements Runnable
|
|||
}
|
||||
);
|
||||
}
|
||||
|
||||
private void deleteSegments(Task task, TaskContext context, Set<DataSegment> segments) throws Exception
|
||||
{
|
||||
for (DataSegment segment : segments) {
|
||||
verifyDataSourceAndInterval(task, context, segment);
|
||||
|
||||
// Verify version (must be less than our context version)
|
||||
if (segment.getVersion().compareTo(context.getVersion()) >= 0) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Segment-to-nuke for task[%s] has invalid version: %s",
|
||||
task.getId(),
|
||||
segment.getIdentifier()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
log.info("Deleting segment[%s] for task[%s]", segment.getIdentifier(), task.getId());
|
||||
mergerDBCoordinator.deleteSegment(segment);
|
||||
}
|
||||
}
|
||||
|
||||
private void publishSegments(Task task, TaskContext context, Set<DataSegment> segments) throws Exception
|
||||
{
|
||||
for (DataSegment segment : segments) {
|
||||
verifyDataSourceAndInterval(task, context, segment);
|
||||
|
||||
// Verify version (must be equal to our context version)
|
||||
if (!context.getVersion().equals(segment.getVersion())) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Segment for task[%s] has invalid version: %s",
|
||||
task.getId(),
|
||||
segment.getIdentifier()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), task.getId());
|
||||
mergerDBCoordinator.announceHistoricalSegment(segment);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyDataSourceAndInterval(Task task, TaskContext context, DataSegment segment)
|
||||
{
|
||||
if (!task.getDataSource().equals(segment.getDataSource())) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Segment for task[%s] has invalid dataSource: %s",
|
||||
task.getId(),
|
||||
segment.getIdentifier()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
if (!task.getInterval().contains(segment.getInterval())) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Segment for task[%s] has invalid interval: %s",
|
||||
task.getId(),
|
||||
segment.getIdentifier()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,11 +48,13 @@ import com.metamx.druid.initialization.ServerConfig;
|
|||
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.druid.loading.S3SegmentKiller;
|
||||
import com.metamx.druid.loading.SegmentKiller;
|
||||
import com.metamx.druid.loading.S3SegmentPusher;
|
||||
import com.metamx.druid.loading.S3SegmentPusherConfig;
|
||||
import com.metamx.druid.loading.SegmentKiller;
|
||||
import com.metamx.druid.loading.SegmentPusher;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.actions.LocalTaskActionClient;
|
||||
import com.metamx.druid.merger.common.actions.TaskActionToolbox;
|
||||
import com.metamx.druid.merger.common.config.IndexerZkConfig;
|
||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
|
||||
|
@ -62,7 +64,8 @@ import com.metamx.druid.merger.coordinator.LocalTaskStorage;
|
|||
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
|
||||
import com.metamx.druid.merger.coordinator.RemoteTaskRunner;
|
||||
import com.metamx.druid.merger.coordinator.RetryPolicyFactory;
|
||||
import com.metamx.druid.merger.coordinator.TaskMaster;
|
||||
import com.metamx.druid.merger.coordinator.TaskLockbox;
|
||||
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
|
||||
import com.metamx.druid.merger.coordinator.TaskQueue;
|
||||
import com.metamx.druid.merger.coordinator.TaskRunner;
|
||||
import com.metamx.druid.merger.coordinator.TaskRunnerFactory;
|
||||
|
@ -134,18 +137,20 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
private ServiceEmitter emitter = null;
|
||||
private DbConnectorConfig dbConnectorConfig = null;
|
||||
private DBI dbi = null;
|
||||
private RestS3Service s3Service = null;
|
||||
private IndexerCoordinatorConfig config = null;
|
||||
private TaskConfig taskConfig = null;
|
||||
private TaskToolbox taskToolbox = null;
|
||||
private MergerDBCoordinator mergerDBCoordinator = null;
|
||||
private TaskStorage taskStorage = null;
|
||||
private TaskQueue taskQueue = null;
|
||||
private TaskLockbox taskLockbox = null;
|
||||
private CuratorFramework curatorFramework = null;
|
||||
private ScheduledExecutorFactory scheduledExecutorFactory = null;
|
||||
private IndexerZkConfig indexerZkConfig;
|
||||
private WorkerSetupManager workerSetupManager = null;
|
||||
private TaskRunnerFactory taskRunnerFactory = null;
|
||||
private TaskMaster taskMaster = null;
|
||||
private TaskMasterLifecycle taskMasterLifecycle = null;
|
||||
private Server server = null;
|
||||
|
||||
private boolean initialized = false;
|
||||
|
@ -183,6 +188,18 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
return this;
|
||||
}
|
||||
|
||||
public IndexerCoordinatorNode setS3Service(RestS3Service s3Service)
|
||||
{
|
||||
this.s3Service = s3Service;
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndexerCoordinatorNode setTaskLockbox(TaskLockbox taskLockbox)
|
||||
{
|
||||
this.taskLockbox = taskLockbox;
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator)
|
||||
{
|
||||
this.mergerDBCoordinator = mergeDbCoordinator;
|
||||
|
@ -216,17 +233,19 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
initializeDB();
|
||||
initializeIndexerCoordinatorConfig();
|
||||
initializeTaskConfig();
|
||||
initializeS3Service();
|
||||
initializeMergeDBCoordinator();
|
||||
initializeTaskToolbox();
|
||||
initializeTaskStorage();
|
||||
initializeTaskLockbox();
|
||||
initializeTaskQueue();
|
||||
initializeTaskToolbox();
|
||||
initializeJacksonInjections();
|
||||
initializeJacksonSubtypes();
|
||||
initializeCurator();
|
||||
initializeIndexerZkConfig();
|
||||
initializeWorkerSetupManager();
|
||||
initializeTaskRunnerFactory();
|
||||
initializeTaskMaster();
|
||||
initializeTaskMasterLifecycle();
|
||||
initializeServer();
|
||||
|
||||
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
|
||||
|
@ -243,7 +262,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
jsonMapper,
|
||||
config,
|
||||
emitter,
|
||||
taskQueue,
|
||||
taskMasterLifecycle,
|
||||
new TaskStorageQueryAdapter(taskStorage),
|
||||
workerSetupManager
|
||||
)
|
||||
|
@ -263,7 +282,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
@Override
|
||||
public boolean doLocal()
|
||||
{
|
||||
return taskMaster.isLeading();
|
||||
return taskMasterLifecycle.isLeading();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -273,7 +292,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
return new URL(
|
||||
String.format(
|
||||
"http://%s%s",
|
||||
taskMaster.getLeader(),
|
||||
taskMasterLifecycle.getLeader(),
|
||||
requestURI
|
||||
)
|
||||
);
|
||||
|
@ -291,20 +310,20 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
initialized = true;
|
||||
}
|
||||
|
||||
private void initializeTaskMaster()
|
||||
private void initializeTaskMasterLifecycle()
|
||||
{
|
||||
if (taskMaster == null) {
|
||||
if (taskMasterLifecycle == null) {
|
||||
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
|
||||
taskMaster = new TaskMaster(
|
||||
taskMasterLifecycle = new TaskMasterLifecycle(
|
||||
taskQueue,
|
||||
taskToolbox,
|
||||
config,
|
||||
serviceDiscoveryConfig,
|
||||
mergerDBCoordinator,
|
||||
taskRunnerFactory,
|
||||
curatorFramework,
|
||||
emitter
|
||||
);
|
||||
lifecycle.addManagedInstance(taskMaster);
|
||||
lifecycle.addManagedInstance(taskMasterLifecycle);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -359,7 +378,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
{
|
||||
InjectableValues.Std injectables = new InjectableValues.Std();
|
||||
|
||||
injectables.addValue("s3Client", taskToolbox.getS3Client())
|
||||
injectables.addValue("s3Client", s3Service)
|
||||
.addValue("segmentPusher", taskToolbox.getSegmentPusher());
|
||||
|
||||
jsonMapper.setInjectableValues(injectables);
|
||||
|
@ -419,24 +438,39 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
}
|
||||
}
|
||||
|
||||
public void initializeTaskToolbox() throws S3ServiceException
|
||||
public void initializeS3Service() throws S3ServiceException
|
||||
{
|
||||
this.s3Service = new RestS3Service(
|
||||
new AWSCredentials(
|
||||
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
|
||||
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public void initializeTaskToolbox()
|
||||
{
|
||||
if (taskToolbox == null) {
|
||||
final RestS3Service s3Client = new RestS3Service(
|
||||
new AWSCredentials(
|
||||
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
|
||||
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
|
||||
)
|
||||
);
|
||||
final SegmentPusher segmentPusher = new S3SegmentPusher(
|
||||
s3Client,
|
||||
s3Service,
|
||||
configFactory.build(S3SegmentPusherConfig.class),
|
||||
jsonMapper
|
||||
);
|
||||
final SegmentKiller segmentKiller = new S3SegmentKiller(
|
||||
s3Client
|
||||
s3Service
|
||||
);
|
||||
taskToolbox = new TaskToolbox(
|
||||
taskConfig,
|
||||
new LocalTaskActionClient(
|
||||
taskStorage,
|
||||
new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter)
|
||||
),
|
||||
emitter,
|
||||
s3Service,
|
||||
segmentPusher,
|
||||
segmentKiller,
|
||||
jsonMapper
|
||||
);
|
||||
taskToolbox = new TaskToolbox(taskConfig, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -454,8 +488,15 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
public void initializeTaskQueue()
|
||||
{
|
||||
if (taskQueue == null) {
|
||||
// Don't start it here. The TaskMaster will handle that when it feels like it.
|
||||
taskQueue = new TaskQueue(taskStorage);
|
||||
// Don't start it here. The TaskMasterLifecycle will handle that when it feels like it.
|
||||
taskQueue = new TaskQueue(taskStorage, taskLockbox);
|
||||
}
|
||||
}
|
||||
|
||||
public void initializeTaskLockbox()
|
||||
{
|
||||
if (taskLockbox == null) {
|
||||
taskLockbox = new TaskLockbox(taskStorage);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -23,15 +23,18 @@ import com.google.common.base.Optional;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.task.MergeTask;
|
||||
import com.metamx.druid.merger.common.actions.TaskAction;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.TaskQueue;
|
||||
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
|
||||
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
|
||||
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.GET;
|
||||
|
@ -40,6 +43,8 @@ import javax.ws.rs.Path;
|
|||
import javax.ws.rs.PathParam;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -50,34 +55,34 @@ public class IndexerCoordinatorResource
|
|||
|
||||
private final IndexerCoordinatorConfig config;
|
||||
private final ServiceEmitter emitter;
|
||||
private final TaskQueue tasks;
|
||||
private final TaskMasterLifecycle taskMasterLifecycle;
|
||||
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
|
||||
private final WorkerSetupManager workerSetupManager;
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
||||
@Inject
|
||||
public IndexerCoordinatorResource(
|
||||
IndexerCoordinatorConfig config,
|
||||
ServiceEmitter emitter,
|
||||
TaskQueue tasks,
|
||||
TaskMasterLifecycle taskMasterLifecycle,
|
||||
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
||||
WorkerSetupManager workerSetupManager
|
||||
|
||||
WorkerSetupManager workerSetupManager,
|
||||
ObjectMapper jsonMapper
|
||||
) throws Exception
|
||||
{
|
||||
this.config = config;
|
||||
this.emitter = emitter;
|
||||
this.tasks = tasks;
|
||||
this.taskMasterLifecycle = taskMasterLifecycle;
|
||||
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
|
||||
this.workerSetupManager = workerSetupManager;
|
||||
this.jsonMapper = jsonMapper;
|
||||
}
|
||||
|
||||
@POST
|
||||
@Path("/merge")
|
||||
@Consumes("application/json")
|
||||
@Produces("application/json")
|
||||
public Response doMerge(
|
||||
final MergeTask task
|
||||
)
|
||||
public Response doMerge(final Task task)
|
||||
{
|
||||
// legacy endpoint
|
||||
return doIndex(task);
|
||||
|
@ -87,9 +92,7 @@ public class IndexerCoordinatorResource
|
|||
@Path("/index")
|
||||
@Consumes("application/json")
|
||||
@Produces("application/json")
|
||||
public Response doIndex(
|
||||
final Task task
|
||||
)
|
||||
public Response doIndex(final Task task)
|
||||
{
|
||||
// verify against whitelist
|
||||
if (config.isWhitelistEnabled() && !config.getWhitelistDatasources().contains(task.getDataSource())) {
|
||||
|
@ -103,14 +106,14 @@ public class IndexerCoordinatorResource
|
|||
.build();
|
||||
}
|
||||
|
||||
tasks.add(task);
|
||||
return okResponse(task.getId());
|
||||
taskMasterLifecycle.getTaskQueue().add(task);
|
||||
return Response.ok(ImmutableMap.of("task", task.getId())).build();
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/status/{taskid}")
|
||||
@Path("/task/{taskid}/status")
|
||||
@Produces("application/json")
|
||||
public Response doStatus(@PathParam("taskid") String taskid)
|
||||
public Response getTaskStatus(@PathParam("taskid") String taskid)
|
||||
{
|
||||
final Optional<TaskStatus> status = taskStorageQueryAdapter.getSameGroupMergedStatus(taskid);
|
||||
if (!status.isPresent()) {
|
||||
|
@ -120,9 +123,30 @@ public class IndexerCoordinatorResource
|
|||
}
|
||||
}
|
||||
|
||||
private Response okResponse(final String taskid)
|
||||
@GET
|
||||
@Path("/task/{taskid}/segments")
|
||||
@Produces("application/json")
|
||||
public Response getTaskSegments(@PathParam("taskid") String taskid)
|
||||
{
|
||||
return Response.ok(ImmutableMap.of("task", taskid)).build();
|
||||
final Set<DataSegment> segments = taskStorageQueryAdapter.getSameGroupNewSegments(taskid);
|
||||
return Response.ok().entity(segments).build();
|
||||
}
|
||||
|
||||
// Legacy endpoint
|
||||
// TODO Remove
|
||||
@Deprecated
|
||||
@GET
|
||||
@Path("/status/{taskid}")
|
||||
@Produces("application/json")
|
||||
public Response getLegacyStatus(@PathParam("taskid") String taskid)
|
||||
{
|
||||
final Optional<TaskStatus> status = taskStorageQueryAdapter.getSameGroupMergedStatus(taskid);
|
||||
final Set<DataSegment> segments = taskStorageQueryAdapter.getSameGroupNewSegments(taskid);
|
||||
|
||||
final Map<String, Object> ret = jsonMapper.convertValue(status, new TypeReference<Map<String, Object>>(){});
|
||||
ret.put("segments", segments);
|
||||
|
||||
return Response.ok().entity(ret).build();
|
||||
}
|
||||
|
||||
@GET
|
||||
|
@ -145,4 +169,13 @@ public class IndexerCoordinatorResource
|
|||
}
|
||||
return Response.ok().build();
|
||||
}
|
||||
|
||||
@POST
|
||||
@Path("/action")
|
||||
@Produces("application/json")
|
||||
public <T> Response doAction(final TaskAction<T> action)
|
||||
{
|
||||
final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action);
|
||||
return Response.ok().entity(ret).build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package com.metamx.druid.merger.coordinator.http;
|
||||
|
||||
import com.google.inject.Provides;
|
||||
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
|
||||
import com.metamx.druid.merger.coordinator.TaskQueue;
|
||||
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
|
||||
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
||||
|
@ -39,7 +40,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
|
|||
private final ObjectMapper jsonMapper;
|
||||
private final IndexerCoordinatorConfig indexerCoordinatorConfig;
|
||||
private final ServiceEmitter emitter;
|
||||
private final TaskQueue tasks;
|
||||
private final TaskMasterLifecycle taskMasterLifecycle;
|
||||
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
|
||||
private final WorkerSetupManager workerSetupManager;
|
||||
|
||||
|
@ -47,7 +48,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
|
|||
ObjectMapper jsonMapper,
|
||||
IndexerCoordinatorConfig indexerCoordinatorConfig,
|
||||
ServiceEmitter emitter,
|
||||
TaskQueue tasks,
|
||||
TaskMasterLifecycle taskMasterLifecycle,
|
||||
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
||||
WorkerSetupManager workerSetupManager
|
||||
)
|
||||
|
@ -55,7 +56,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
|
|||
this.jsonMapper = jsonMapper;
|
||||
this.indexerCoordinatorConfig = indexerCoordinatorConfig;
|
||||
this.emitter = emitter;
|
||||
this.tasks = tasks;
|
||||
this.taskMasterLifecycle = taskMasterLifecycle;
|
||||
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
|
||||
this.workerSetupManager = workerSetupManager;
|
||||
}
|
||||
|
@ -67,7 +68,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
|
|||
bind(ObjectMapper.class).toInstance(jsonMapper);
|
||||
bind(IndexerCoordinatorConfig.class).toInstance(indexerCoordinatorConfig);
|
||||
bind(ServiceEmitter.class).toInstance(emitter);
|
||||
bind(TaskQueue.class).toInstance(tasks);
|
||||
bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle);
|
||||
bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter);
|
||||
bind(WorkerSetupManager.class).toInstance(workerSetupManager);
|
||||
|
||||
|
|
|
@ -21,19 +21,15 @@ package com.metamx.druid.merger.worker;
|
|||
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskHolder;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
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.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
@ -85,12 +81,10 @@ public class TaskMonitor
|
|||
throws Exception
|
||||
{
|
||||
if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) {
|
||||
final TaskHolder taskHolder = toolbox.getObjectMapper().readValue(
|
||||
final Task task = toolbox.getObjectMapper().readValue(
|
||||
cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()),
|
||||
TaskHolder.class
|
||||
Task.class
|
||||
);
|
||||
final Task task = taskHolder.getTask();
|
||||
final TaskContext taskContext = taskHolder.getTaskContext();
|
||||
|
||||
if (workerCuratorCoordinator.statusExists(task.getId())) {
|
||||
return;
|
||||
|
@ -111,14 +105,7 @@ public class TaskMonitor
|
|||
try {
|
||||
workerCuratorCoordinator.unannounceTask(task.getId());
|
||||
workerCuratorCoordinator.announceStatus(TaskStatus.running(task.getId()));
|
||||
taskStatus = task.run(taskContext, toolbox, new TaskCallback()
|
||||
{
|
||||
@Override
|
||||
public void notify(TaskStatus status)
|
||||
{
|
||||
workerCuratorCoordinator.updateStatus(status);
|
||||
}
|
||||
});
|
||||
taskStatus = task.run(toolbox);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert(e, "Failed to run task")
|
||||
|
|
|
@ -40,10 +40,10 @@ import com.metamx.druid.loading.S3SegmentPusherConfig;
|
|||
import com.metamx.druid.loading.SegmentKiller;
|
||||
import com.metamx.druid.loading.SegmentPusher;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.actions.RemoteTaskActionClient;
|
||||
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.coordinator.config.IndexerCoordinatorConfig;
|
||||
import com.metamx.druid.merger.worker.TaskMonitor;
|
||||
import com.metamx.druid.merger.worker.Worker;
|
||||
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
|
||||
|
@ -97,7 +97,9 @@ public class WorkerNode extends RegisteringNode
|
|||
private final Properties props;
|
||||
private final ConfigurationObjectFactory configFactory;
|
||||
|
||||
private RestS3Service s3Service = null;
|
||||
private List<Monitor> monitors = null;
|
||||
private HttpClient httpClient = null;
|
||||
private ServiceEmitter emitter = null;
|
||||
private TaskConfig taskConfig = null;
|
||||
private WorkerConfig workerConfig = null;
|
||||
|
@ -124,12 +126,24 @@ public class WorkerNode extends RegisteringNode
|
|||
this.configFactory = configFactory;
|
||||
}
|
||||
|
||||
public WorkerNode setHttpClient(HttpClient httpClient)
|
||||
{
|
||||
this.httpClient = httpClient;
|
||||
return this;
|
||||
}
|
||||
|
||||
public WorkerNode setEmitter(ServiceEmitter emitter)
|
||||
{
|
||||
this.emitter = emitter;
|
||||
return this;
|
||||
}
|
||||
|
||||
public WorkerNode setS3Service(RestS3Service s3Service)
|
||||
{
|
||||
this.s3Service = s3Service;
|
||||
return this;
|
||||
}
|
||||
|
||||
public WorkerNode setTaskToolbox(TaskToolbox taskToolbox)
|
||||
{
|
||||
this.taskToolbox = taskToolbox;
|
||||
|
@ -156,7 +170,9 @@ public class WorkerNode extends RegisteringNode
|
|||
|
||||
public void init() throws Exception
|
||||
{
|
||||
initializeHttpClient();
|
||||
initializeEmitter();
|
||||
initializeS3Service();
|
||||
initializeMonitors();
|
||||
initializeMergerConfig();
|
||||
initializeTaskToolbox();
|
||||
|
@ -235,7 +251,7 @@ public class WorkerNode extends RegisteringNode
|
|||
{
|
||||
InjectableValues.Std injectables = new InjectableValues.Std();
|
||||
|
||||
injectables.addValue("s3Client", taskToolbox.getS3Client())
|
||||
injectables.addValue("s3Client", s3Service)
|
||||
.addValue("segmentPusher", taskToolbox.getSegmentPusher());
|
||||
|
||||
jsonMapper.setInjectableValues(injectables);
|
||||
|
@ -246,13 +262,18 @@ public class WorkerNode extends RegisteringNode
|
|||
jsonMapper.registerSubtypes(StaticS3FirehoseFactory.class);
|
||||
}
|
||||
|
||||
private void initializeHttpClient()
|
||||
{
|
||||
if (httpClient == null) {
|
||||
httpClient = HttpClientInit.createClient(
|
||||
HttpClientConfig.builder().withNumConnections(1).build(), lifecycle
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private void initializeEmitter()
|
||||
{
|
||||
if (emitter == null) {
|
||||
final HttpClient httpClient = HttpClientInit.createClient(
|
||||
HttpClientConfig.builder().withNumConnections(1).build(), lifecycle
|
||||
);
|
||||
|
||||
emitter = new ServiceEmitter(
|
||||
PropUtils.getProperty(props, "druid.service"),
|
||||
PropUtils.getProperty(props, "druid.host"),
|
||||
|
@ -262,6 +283,18 @@ public class WorkerNode extends RegisteringNode
|
|||
EmittingLogger.registerEmitter(emitter);
|
||||
}
|
||||
|
||||
private void initializeS3Service() throws S3ServiceException
|
||||
{
|
||||
if(s3Service == null) {
|
||||
s3Service = new RestS3Service(
|
||||
new AWSCredentials(
|
||||
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
|
||||
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private void initializeMonitors()
|
||||
{
|
||||
if (monitors == null) {
|
||||
|
@ -285,21 +318,23 @@ public class WorkerNode extends RegisteringNode
|
|||
public void initializeTaskToolbox() throws S3ServiceException
|
||||
{
|
||||
if (taskToolbox == null) {
|
||||
final RestS3Service s3Client = new RestS3Service(
|
||||
new AWSCredentials(
|
||||
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
|
||||
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
|
||||
)
|
||||
);
|
||||
final SegmentPusher segmentPusher = new S3SegmentPusher(
|
||||
s3Client,
|
||||
s3Service,
|
||||
configFactory.build(S3SegmentPusherConfig.class),
|
||||
jsonMapper
|
||||
);
|
||||
final SegmentKiller segmentKiller = new S3SegmentKiller(
|
||||
s3Client
|
||||
s3Service
|
||||
);
|
||||
taskToolbox = new TaskToolbox(
|
||||
taskConfig,
|
||||
new RemoteTaskActionClient(httpClient, jsonMapper),
|
||||
emitter,
|
||||
s3Service,
|
||||
segmentPusher,
|
||||
segmentKiller,
|
||||
jsonMapper
|
||||
);
|
||||
taskToolbox = new TaskToolbox(taskConfig, emitter, s3Client, segmentPusher, segmentKiller, jsonMapper);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -49,9 +49,9 @@ public class MergeTaskTest
|
|||
}
|
||||
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Type.TEST;
|
||||
return "test";
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -64,13 +64,13 @@ public class MergeTaskTest
|
|||
@Test
|
||||
public void testInterval()
|
||||
{
|
||||
Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getInterval());
|
||||
Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getFixedInterval().get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testID()
|
||||
{
|
||||
final String desiredPrefix = "merge_foo_" + DigestUtils.shaHex(
|
||||
final String desiredPrefix = "merge_foo_" + DigestUtils.sha1Hex(
|
||||
"2012-01-03T00:00:00.000Z_2012-01-05T00:00:00.000Z_V1_0"
|
||||
+ "_2012-01-04T00:00:00.000Z_2012-01-06T00:00:00.000Z_V1_0"
|
||||
+ "_2012-01-05T00:00:00.000Z_2012-01-07T00:00:00.000Z_V1_0"
|
||||
|
|
|
@ -0,0 +1,108 @@
|
|||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.metamx.common.Granularity;
|
||||
import com.metamx.druid.QueryGranularity;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.aggregation.DoubleSumAggregatorFactory;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.indexer.granularity.UniformGranularitySpec;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.druid.realtime.Schema;
|
||||
import com.metamx.druid.shard.NoneShardSpec;
|
||||
import junit.framework.Assert;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TaskSerdeTest
|
||||
{
|
||||
@Test
|
||||
public void testIndexTaskSerde() throws Exception
|
||||
{
|
||||
final Task task = new IndexTask(
|
||||
"foo",
|
||||
new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P2D"))),
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
QueryGranularity.NONE,
|
||||
10000,
|
||||
null
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
final String json = jsonMapper.writeValueAsString(task);
|
||||
final Task task2 = jsonMapper.readValue(json, Task.class);
|
||||
|
||||
Assert.assertEquals(task.getId(), task2.getId());
|
||||
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
|
||||
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
|
||||
Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval());
|
||||
Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIndexGeneratorTaskSerde() throws Exception
|
||||
{
|
||||
final Task task = new IndexGeneratorTask(
|
||||
"foo",
|
||||
new Interval("2010-01-01/P1D"),
|
||||
null,
|
||||
new Schema(
|
||||
"foo",
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
QueryGranularity.NONE,
|
||||
new NoneShardSpec()
|
||||
)
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
final String json = jsonMapper.writeValueAsString(task);
|
||||
final Task task2 = jsonMapper.readValue(json, Task.class);
|
||||
|
||||
Assert.assertEquals(task.getId(), task2.getId());
|
||||
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
|
||||
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
|
||||
Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval());
|
||||
Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppendTaskSerde() throws Exception
|
||||
{
|
||||
final Task task = new AppendTask(
|
||||
"foo",
|
||||
ImmutableList.<DataSegment>of(
|
||||
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
|
||||
)
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
final String json = jsonMapper.writeValueAsString(task);
|
||||
final Task task2 = jsonMapper.readValue(json, Task.class);
|
||||
|
||||
Assert.assertEquals(task.getId(), task2.getId());
|
||||
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
|
||||
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
|
||||
Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval());
|
||||
Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeleteTaskSerde() throws Exception
|
||||
{
|
||||
final Task task = new DeleteTask(
|
||||
"foo",
|
||||
new Interval("2010-01-01/P1D")
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
final String json = jsonMapper.writeValueAsString(task);
|
||||
final Task task2 = jsonMapper.readValue(json, Task.class);
|
||||
|
||||
Assert.assertEquals(task.getId(), task2.getId());
|
||||
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
|
||||
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
|
||||
Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval());
|
||||
Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get());
|
||||
}
|
||||
}
|
|
@ -2,7 +2,6 @@ package com.metamx.druid.merger.coordinator;
|
|||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
|
@ -14,7 +13,6 @@ import com.metamx.druid.merger.common.config.IndexerZkConfig;
|
|||
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||
import com.metamx.druid.merger.common.task.DefaultMergeTask;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
||||
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
|
||||
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
|
||||
import com.metamx.druid.merger.coordinator.scaling.AutoScalingData;
|
||||
|
@ -135,7 +133,6 @@ public class RemoteTaskRunnerTest
|
|||
{
|
||||
remoteTaskRunner.run(
|
||||
task1,
|
||||
new TaskContext(new DateTime().toString(), Sets.<DataSegment>newHashSet(), Sets.<DataSegment>newHashSet()),
|
||||
null
|
||||
);
|
||||
}
|
||||
|
@ -145,24 +142,14 @@ public class RemoteTaskRunnerTest
|
|||
{
|
||||
remoteTaskRunner.run(
|
||||
task1,
|
||||
new TaskContext(
|
||||
new DateTime().toString(),
|
||||
Sets.<DataSegment>newHashSet(),
|
||||
Sets.<DataSegment>newHashSet()
|
||||
),
|
||||
null
|
||||
);
|
||||
try {
|
||||
remoteTaskRunner.run(
|
||||
task1,
|
||||
new TaskContext(
|
||||
new DateTime().toString(),
|
||||
Sets.<DataSegment>newHashSet(),
|
||||
Sets.<DataSegment>newHashSet()
|
||||
),
|
||||
null
|
||||
);
|
||||
fail("ISE expected");
|
||||
// fail("ISE expected");
|
||||
}
|
||||
catch (ISE expected) {
|
||||
|
||||
|
@ -191,7 +178,6 @@ public class RemoteTaskRunnerTest
|
|||
)
|
||||
), Lists.<AggregatorFactory>newArrayList()
|
||||
),
|
||||
new TaskContext(new DateTime().toString(), Sets.<DataSegment>newHashSet(), Sets.<DataSegment>newHashSet()),
|
||||
null
|
||||
);
|
||||
}
|
||||
|
@ -224,7 +210,6 @@ public class RemoteTaskRunnerTest
|
|||
final MutableBoolean callbackCalled = new MutableBoolean(false);
|
||||
remoteTaskRunner.run(
|
||||
task1,
|
||||
null,
|
||||
new TaskCallback()
|
||||
{
|
||||
@Override
|
||||
|
@ -296,7 +281,7 @@ public class RemoteTaskRunnerTest
|
|||
{
|
||||
return 0;
|
||||
}
|
||||
}, null, null, null, null, jsonMapper
|
||||
}, null, null, null, null, null, jsonMapper
|
||||
),
|
||||
Executors.newSingleThreadExecutor()
|
||||
);
|
||||
|
@ -470,13 +455,13 @@ public class RemoteTaskRunnerTest
|
|||
}
|
||||
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Type.TEST;
|
||||
return "test";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
return TaskStatus.success("task1");
|
||||
}
|
||||
|
|
|
@ -1,230 +0,0 @@
|
|||
package com.metamx.druid.merger.coordinator;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.task.AbstractTask;
|
||||
import com.metamx.druid.merger.coordinator.exec.TaskConsumer;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.core.Event;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceEventBuilder;
|
||||
import junit.framework.Assert;
|
||||
import org.easymock.EasyMock;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
public class TaskConsumerTest
|
||||
{
|
||||
private TaskStorage ts = null;
|
||||
private TaskQueue tq = null;
|
||||
private TaskRunner tr = null;
|
||||
private MockMergerDBCoordinator mdc = null;
|
||||
private TaskConsumer tc = null;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
EmittingLogger.registerEmitter(EasyMock.createMock(ServiceEmitter.class));
|
||||
|
||||
ts = new LocalTaskStorage();
|
||||
tq = new TaskQueue(ts);
|
||||
tr = new LocalTaskRunner(
|
||||
new TaskToolbox(null, null, null, null, null, null),
|
||||
Executors.newSingleThreadExecutor()
|
||||
);
|
||||
|
||||
mdc = newMockMDC();
|
||||
tc = new TaskConsumer(tq, tr, mdc, newMockEmitter());
|
||||
|
||||
tq.start();
|
||||
tc.start();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown()
|
||||
{
|
||||
tc.stop();
|
||||
tq.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimple() throws Exception
|
||||
{
|
||||
tq.add(
|
||||
new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
|
||||
{
|
||||
@Override
|
||||
public Type getType()
|
||||
{
|
||||
return Type.TEST;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(
|
||||
TaskContext context, TaskToolbox toolbox, TaskCallback callback
|
||||
) throws Exception
|
||||
{
|
||||
return TaskStatus.success(getId()).withSegments(
|
||||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource("ds")
|
||||
.interval(new Interval("2012-01-01/P1D"))
|
||||
.version(context.getVersion())
|
||||
.build()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
while (ts.getStatus("id1").get().isRunnable()) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
final TaskStatus status = ts.getStatus("id1").get();
|
||||
Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
|
||||
Assert.assertEquals("nextTasks.size", 0, status.getNextTasks().size());
|
||||
Assert.assertEquals("segments.size", 1, status.getSegments().size());
|
||||
Assert.assertEquals("segmentsNuked.size", 0, status.getSegmentsNuked().size());
|
||||
Assert.assertEquals("segments published", status.getSegments(), mdc.getPublished());
|
||||
Assert.assertEquals("segments nuked", status.getSegmentsNuked(), mdc.getNuked());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBadVersion() throws Exception
|
||||
{
|
||||
tq.add(
|
||||
new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
|
||||
{
|
||||
@Override
|
||||
public Type getType()
|
||||
{
|
||||
return Type.TEST;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(
|
||||
TaskContext context, TaskToolbox toolbox, TaskCallback callback
|
||||
) throws Exception
|
||||
{
|
||||
return TaskStatus.success(getId()).withSegments(
|
||||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource("ds")
|
||||
.interval(new Interval("2012-01-01/P1D"))
|
||||
.version(context.getVersion() + "1!!!1!!")
|
||||
.build()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
while (ts.getStatus("id1").get().isRunnable()) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
final TaskStatus status = ts.getStatus("id1").get();
|
||||
Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode());
|
||||
Assert.assertEquals("nextTasks.size", 0, status.getNextTasks().size());
|
||||
Assert.assertEquals("segments.size", 0, status.getSegments().size());
|
||||
Assert.assertEquals("segmentsNuked.size", 0, status.getSegmentsNuked().size());
|
||||
Assert.assertEquals("segments published", status.getSegments(), mdc.getPublished());
|
||||
Assert.assertEquals("segments nuked", status.getSegmentsNuked(), mdc.getNuked());
|
||||
}
|
||||
|
||||
private static class MockMergerDBCoordinator extends MergerDBCoordinator
|
||||
{
|
||||
final private Set<DataSegment> published = Sets.newHashSet();
|
||||
final private Set<DataSegment> nuked = Sets.newHashSet();
|
||||
|
||||
private MockMergerDBCoordinator()
|
||||
{
|
||||
super(null, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<DataSegment> getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException
|
||||
{
|
||||
return ImmutableList.of();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<DataSegment> getUnusedSegmentsForInterval(String dataSource, Interval interval)
|
||||
{
|
||||
return ImmutableList.of();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void commitTaskStatus(TaskStatus taskStatus)
|
||||
{
|
||||
for(final DataSegment segment : taskStatus.getSegments())
|
||||
{
|
||||
announceHistoricalSegment(segment);
|
||||
}
|
||||
|
||||
for(final DataSegment segment : taskStatus.getSegmentsNuked())
|
||||
{
|
||||
deleteSegment(segment);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void announceHistoricalSegment(DataSegment segment)
|
||||
{
|
||||
published.add(segment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteSegment(DataSegment segment)
|
||||
{
|
||||
nuked.add(segment);
|
||||
}
|
||||
|
||||
public Set<DataSegment> getPublished()
|
||||
{
|
||||
return ImmutableSet.copyOf(published);
|
||||
}
|
||||
|
||||
public Set<DataSegment> getNuked()
|
||||
{
|
||||
return ImmutableSet.copyOf(nuked);
|
||||
}
|
||||
}
|
||||
|
||||
private MockMergerDBCoordinator newMockMDC()
|
||||
{
|
||||
return new MockMergerDBCoordinator();
|
||||
}
|
||||
|
||||
private ServiceEmitter newMockEmitter()
|
||||
{
|
||||
return new ServiceEmitter(null, null, null)
|
||||
{
|
||||
@Override
|
||||
public void emit(Event event)
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emit(ServiceEventBuilder builder)
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,644 @@
|
|||
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;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.io.Files;
|
||||
import com.metamx.common.Granularity;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.Comparators;
|
||||
import com.metamx.druid.QueryGranularity;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.aggregation.DoubleSumAggregatorFactory;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.indexer.granularity.UniformGranularitySpec;
|
||||
import com.metamx.druid.input.InputRow;
|
||||
import com.metamx.druid.input.MapBasedInputRow;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.druid.loading.SegmentKiller;
|
||||
import com.metamx.druid.loading.SegmentPuller;
|
||||
import com.metamx.druid.loading.SegmentPusher;
|
||||
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.LocalTaskActionClient;
|
||||
import com.metamx.druid.merger.common.actions.LockAcquireAction;
|
||||
import com.metamx.druid.merger.common.actions.LockListAction;
|
||||
import com.metamx.druid.merger.common.actions.LockReleaseAction;
|
||||
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
|
||||
import com.metamx.druid.merger.common.actions.TaskActionToolbox;
|
||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||
import com.metamx.druid.merger.common.task.AbstractTask;
|
||||
import com.metamx.druid.merger.common.task.IndexTask;
|
||||
import com.metamx.druid.merger.common.task.KillTask;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.exec.TaskConsumer;
|
||||
import com.metamx.druid.realtime.Firehose;
|
||||
import com.metamx.druid.realtime.FirehoseFactory;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.core.Event;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceEventBuilder;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.easymock.EasyMock;
|
||||
import org.jets3t.service.ServiceException;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
public class TaskLifecycleTest
|
||||
{
|
||||
private File tmp = null;
|
||||
private TaskStorage ts = null;
|
||||
private TaskLockbox tl = null;
|
||||
private TaskQueue tq = null;
|
||||
private TaskRunner tr = null;
|
||||
private MockMergerDBCoordinator mdc = null;
|
||||
private TaskToolbox tb = null;
|
||||
private TaskConsumer tc = null;
|
||||
TaskStorageQueryAdapter tsqa = null;
|
||||
|
||||
private static final Ordering<DataSegment> byIntervalOrdering = new Ordering<DataSegment>()
|
||||
{
|
||||
@Override
|
||||
public int compare(DataSegment dataSegment, DataSegment dataSegment2)
|
||||
{
|
||||
return Comparators.intervalsByStartThenEnd().compare(dataSegment.getInterval(), dataSegment2.getInterval());
|
||||
}
|
||||
};
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
EmittingLogger.registerEmitter(EasyMock.createMock(ServiceEmitter.class));
|
||||
|
||||
tmp = Files.createTempDir();
|
||||
|
||||
ts = new LocalTaskStorage();
|
||||
tl = new TaskLockbox(ts);
|
||||
tq = new TaskQueue(ts, tl);
|
||||
mdc = newMockMDC();
|
||||
|
||||
tb = new TaskToolbox(
|
||||
new TaskConfig()
|
||||
{
|
||||
@Override
|
||||
public File getBaseTaskDir()
|
||||
{
|
||||
return tmp;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getRowFlushBoundary()
|
||||
{
|
||||
return 50000;
|
||||
}
|
||||
},
|
||||
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())),
|
||||
newMockEmitter(),
|
||||
null, // s3 client
|
||||
new SegmentPusher()
|
||||
{
|
||||
@Override
|
||||
public DataSegment push(File file, DataSegment segment) throws IOException
|
||||
{
|
||||
return segment;
|
||||
}
|
||||
},
|
||||
new SegmentKiller()
|
||||
{
|
||||
@Override
|
||||
public void kill(Collection<DataSegment> segments) throws ServiceException
|
||||
{
|
||||
|
||||
}
|
||||
},
|
||||
new DefaultObjectMapper()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public Map<String, SegmentPuller> getSegmentGetters(Task task)
|
||||
{
|
||||
return ImmutableMap.of();
|
||||
}
|
||||
};
|
||||
|
||||
tr = new LocalTaskRunner(
|
||||
tb,
|
||||
Executors.newSingleThreadExecutor()
|
||||
);
|
||||
|
||||
tc = new TaskConsumer(tq, tr, tb, newMockEmitter());
|
||||
tsqa = new TaskStorageQueryAdapter(ts);
|
||||
|
||||
tq.start();
|
||||
tc.start();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown()
|
||||
{
|
||||
try {
|
||||
FileUtils.deleteDirectory(tmp);
|
||||
} catch(Exception e) {
|
||||
// suppress
|
||||
}
|
||||
tc.stop();
|
||||
tq.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIndexTask() throws Exception
|
||||
{
|
||||
final Task indexTask = new IndexTask(
|
||||
"foo",
|
||||
new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P2D"))),
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
QueryGranularity.NONE,
|
||||
10000,
|
||||
newMockFirehoseFactory(
|
||||
ImmutableList.of(
|
||||
IR("2010-01-01T01", "x", "y", 1),
|
||||
IR("2010-01-01T01", "x", "z", 1),
|
||||
IR("2010-01-02T01", "a", "b", 2),
|
||||
IR("2010-01-02T01", "a", "c", 1)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
final TaskStatus mergedStatus = runTask(indexTask);
|
||||
final TaskStatus status = ts.getStatus(indexTask.getId()).get();
|
||||
final List<DataSegment> publishedSegments = byIntervalOrdering.sortedCopy(mdc.getPublished());
|
||||
final List<DataSegment> loggedSegments = byIntervalOrdering.sortedCopy(tsqa.getSameGroupNewSegments(indexTask.getId()));
|
||||
|
||||
Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
|
||||
Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode());
|
||||
Assert.assertEquals("segments logged vs published", loggedSegments, publishedSegments);
|
||||
Assert.assertEquals("num segments published", 2, mdc.getPublished().size());
|
||||
Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size());
|
||||
|
||||
Assert.assertEquals("segment1 datasource", "foo", publishedSegments.get(0).getDataSource());
|
||||
Assert.assertEquals("segment1 interval", new Interval("2010-01-01/P1D"), publishedSegments.get(0).getInterval());
|
||||
Assert.assertEquals("segment1 dimensions", ImmutableList.of("dim1", "dim2"), publishedSegments.get(0).getDimensions());
|
||||
Assert.assertEquals("segment1 metrics", ImmutableList.of("met"), publishedSegments.get(0).getMetrics());
|
||||
|
||||
Assert.assertEquals("segment2 datasource", "foo", publishedSegments.get(1).getDataSource());
|
||||
Assert.assertEquals("segment2 interval", new Interval("2010-01-02/P1D"), publishedSegments.get(1).getInterval());
|
||||
Assert.assertEquals("segment2 dimensions", ImmutableList.of("dim1", "dim2"), publishedSegments.get(1).getDimensions());
|
||||
Assert.assertEquals("segment2 metrics", ImmutableList.of("met"), publishedSegments.get(1).getMetrics());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIndexTaskFailure() throws Exception
|
||||
{
|
||||
final Task indexTask = new IndexTask(
|
||||
"foo",
|
||||
new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P1D"))),
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
QueryGranularity.NONE,
|
||||
10000,
|
||||
newMockExceptionalFirehoseFactory()
|
||||
);
|
||||
|
||||
final TaskStatus mergedStatus = runTask(indexTask);
|
||||
final TaskStatus status = ts.getStatus(indexTask.getId()).get();
|
||||
|
||||
Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
|
||||
Assert.assertEquals("merged statusCode", TaskStatus.Status.FAILED, mergedStatus.getStatusCode());
|
||||
Assert.assertEquals("num segments published", 0, mdc.getPublished().size());
|
||||
Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKillTask() throws Exception
|
||||
{
|
||||
// TODO: Worst test ever
|
||||
final Task killTask = new KillTask("foo", new Interval("2010-01-02/P2D"));
|
||||
|
||||
final TaskStatus mergedStatus = runTask(killTask);
|
||||
Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode());
|
||||
Assert.assertEquals("num segments published", 0, mdc.getPublished().size());
|
||||
Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRealtimeishTask() throws Exception
|
||||
{
|
||||
class RealtimeishTask extends AbstractTask
|
||||
{
|
||||
RealtimeishTask()
|
||||
{
|
||||
super("rt1", "rt", "foo", null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "realtime_test";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final Interval interval1 = new Interval("2010-01-01T00/PT1H");
|
||||
final Interval interval2 = new Interval("2010-01-01T01/PT1H");
|
||||
|
||||
// Sort of similar to what realtime tasks do:
|
||||
|
||||
// Acquire lock for first interval
|
||||
final Optional<TaskLock> lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval1));
|
||||
final List<TaskLock> locks1 = toolbox.getTaskActionClient().submit(new LockListAction(this));
|
||||
|
||||
// (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);
|
||||
|
||||
// Acquire lock for second interval
|
||||
final Optional<TaskLock> lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval2));
|
||||
final List<TaskLock> locks2 = toolbox.getTaskActionClient().submit(new LockListAction(this));
|
||||
|
||||
// (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);
|
||||
|
||||
// Push first segment
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(
|
||||
new SegmentInsertAction(
|
||||
this,
|
||||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource("foo")
|
||||
.interval(interval1)
|
||||
.version(lock1.get().getVersion())
|
||||
.build()
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
// Release first lock
|
||||
toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval1));
|
||||
final List<TaskLock> locks3 = toolbox.getTaskActionClient().submit(new LockListAction(this));
|
||||
|
||||
// (Confirm lock sanity)
|
||||
Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3);
|
||||
|
||||
// Push second segment
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(
|
||||
new SegmentInsertAction(
|
||||
this,
|
||||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource("foo")
|
||||
.interval(interval2)
|
||||
.version(lock2.get().getVersion())
|
||||
.build()
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
// Release second lock
|
||||
toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval2));
|
||||
final List<TaskLock> locks4 = toolbox.getTaskActionClient().submit(new LockListAction(this));
|
||||
|
||||
// (Confirm lock sanity)
|
||||
Assert.assertEquals("locks4", ImmutableList.<TaskLock>of(), locks4);
|
||||
|
||||
// Exit
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
}
|
||||
|
||||
final Task rtishTask = new RealtimeishTask();
|
||||
final TaskStatus status = runTask(rtishTask);
|
||||
|
||||
Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
|
||||
Assert.assertEquals("num segments published", 2, mdc.getPublished().size());
|
||||
Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimple() throws Exception
|
||||
{
|
||||
final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
|
||||
{
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "test";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final TaskLock myLock = Iterables.getOnlyElement(
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(new LockListAction(this))
|
||||
);
|
||||
|
||||
final DataSegment segment = DataSegment.builder()
|
||||
.dataSource("ds")
|
||||
.interval(new Interval("2012-01-01/P1D"))
|
||||
.version(myLock.getVersion())
|
||||
.build();
|
||||
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment)));
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
};
|
||||
|
||||
final TaskStatus status = runTask(task);
|
||||
|
||||
Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
|
||||
Assert.assertEquals("segments published", 1, mdc.getPublished().size());
|
||||
Assert.assertEquals("segments nuked", 0, mdc.getNuked().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBadVersion() throws Exception
|
||||
{
|
||||
final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
|
||||
{
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "test";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final TaskLock myLock = Iterables.getOnlyElement(
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(new LockListAction(this))
|
||||
);
|
||||
|
||||
final DataSegment segment = DataSegment.builder()
|
||||
.dataSource("ds")
|
||||
.interval(new Interval("2012-01-01/P2D"))
|
||||
.version(myLock.getVersion())
|
||||
.build();
|
||||
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment)));
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
};
|
||||
|
||||
final TaskStatus status = runTask(task);
|
||||
|
||||
Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode());
|
||||
Assert.assertEquals("segments published", 0, mdc.getPublished().size());
|
||||
Assert.assertEquals("segments nuked", 0, mdc.getNuked().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBadInterval() throws Exception
|
||||
{
|
||||
final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
|
||||
{
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "test";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final TaskLock myLock = Iterables.getOnlyElement(
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(new LockListAction(this))
|
||||
);
|
||||
|
||||
final DataSegment segment = DataSegment.builder()
|
||||
.dataSource("ds")
|
||||
.interval(new Interval("2012-01-01/P1D"))
|
||||
.version(myLock.getVersion() + "1!!!1!!")
|
||||
.build();
|
||||
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment)));
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
};
|
||||
|
||||
final TaskStatus status = runTask(task);
|
||||
|
||||
Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode());
|
||||
Assert.assertEquals("segments published", 0, mdc.getPublished().size());
|
||||
Assert.assertEquals("segments nuked", 0, mdc.getNuked().size());
|
||||
}
|
||||
|
||||
private TaskStatus runTask(Task task)
|
||||
{
|
||||
final long startTime = System.currentTimeMillis();
|
||||
|
||||
tq.add(task);
|
||||
|
||||
TaskStatus status;
|
||||
|
||||
try {
|
||||
while ( (status = tsqa.getSameGroupMergedStatus(task.getId()).get()).isRunnable()) {
|
||||
if(System.currentTimeMillis() > startTime + 10 * 1000) {
|
||||
throw new ISE("Where did the task go?!: %s", task.getId());
|
||||
}
|
||||
|
||||
Thread.sleep(100);
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
return status;
|
||||
}
|
||||
|
||||
private static class MockMergerDBCoordinator extends MergerDBCoordinator
|
||||
{
|
||||
final private Set<DataSegment> published = Sets.newHashSet();
|
||||
final private Set<DataSegment> nuked = Sets.newHashSet();
|
||||
|
||||
private MockMergerDBCoordinator()
|
||||
{
|
||||
super(null, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<DataSegment> getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException
|
||||
{
|
||||
return ImmutableList.of();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<DataSegment> getUnusedSegmentsForInterval(String dataSource, Interval interval)
|
||||
{
|
||||
return ImmutableList.of();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void announceHistoricalSegments(Set<DataSegment> segment)
|
||||
{
|
||||
published.addAll(segment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteSegments(Set<DataSegment> segment)
|
||||
{
|
||||
nuked.addAll(segment);
|
||||
}
|
||||
|
||||
public Set<DataSegment> getPublished()
|
||||
{
|
||||
return ImmutableSet.copyOf(published);
|
||||
}
|
||||
|
||||
public Set<DataSegment> getNuked()
|
||||
{
|
||||
return ImmutableSet.copyOf(nuked);
|
||||
}
|
||||
}
|
||||
|
||||
private static MockMergerDBCoordinator newMockMDC()
|
||||
{
|
||||
return new MockMergerDBCoordinator();
|
||||
}
|
||||
|
||||
private static ServiceEmitter newMockEmitter()
|
||||
{
|
||||
return new ServiceEmitter(null, null, null)
|
||||
{
|
||||
@Override
|
||||
public void emit(Event event)
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emit(ServiceEventBuilder builder)
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private static InputRow IR(String dt, String dim1, String dim2, float met)
|
||||
{
|
||||
return new MapBasedInputRow(
|
||||
new DateTime(dt).getMillis(),
|
||||
ImmutableList.of("dim1", "dim2"),
|
||||
ImmutableMap.<String, Object>of(
|
||||
"dim1", dim1,
|
||||
"dim2", dim2,
|
||||
"met", met
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private static FirehoseFactory newMockExceptionalFirehoseFactory()
|
||||
{
|
||||
return new FirehoseFactory()
|
||||
{
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
{
|
||||
return new Firehose()
|
||||
{
|
||||
@Override
|
||||
public boolean hasMore()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRow nextRow()
|
||||
{
|
||||
throw new RuntimeException("HA HA HA");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Runnable commit()
|
||||
{
|
||||
return new Runnable() {
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private static FirehoseFactory newMockFirehoseFactory(final Iterable<InputRow> inputRows)
|
||||
{
|
||||
return new FirehoseFactory()
|
||||
{
|
||||
@Override
|
||||
public Firehose connect() throws IOException
|
||||
{
|
||||
final Iterator<InputRow> inputRowIterator = inputRows.iterator();
|
||||
|
||||
return new Firehose()
|
||||
{
|
||||
@Override
|
||||
public boolean hasMore()
|
||||
{
|
||||
return inputRowIterator.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRow nextRow()
|
||||
{
|
||||
return inputRowIterator.next();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Runnable commit()
|
||||
{
|
||||
return new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -20,16 +20,17 @@
|
|||
package com.metamx.druid.merger.coordinator;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
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.LocalTaskActionClient;
|
||||
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
|
||||
import com.metamx.druid.merger.common.actions.TaskActionToolbox;
|
||||
import com.metamx.druid.merger.common.task.AbstractTask;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -43,7 +44,8 @@ public class TaskQueueTest
|
|||
public void testEmptyQueue() throws Exception
|
||||
{
|
||||
final TaskStorage ts = new LocalTaskStorage();
|
||||
final TaskQueue tq = newTaskQueueWithStorage(ts);
|
||||
final TaskLockbox tl = new TaskLockbox(ts);
|
||||
final TaskQueue tq = newTaskQueue(ts, tl);
|
||||
|
||||
// get task status for nonexistent task
|
||||
Assert.assertFalse("getStatus", ts.getStatus("foo").isPresent());
|
||||
|
@ -52,9 +54,10 @@ public class TaskQueueTest
|
|||
Assert.assertNull("poll", tq.poll());
|
||||
}
|
||||
|
||||
public static TaskQueue newTaskQueueWithStorage(TaskStorage storage)
|
||||
public static TaskQueue newTaskQueue(TaskStorage storage, TaskLockbox lockbox)
|
||||
{
|
||||
final TaskQueue tq = new TaskQueue(storage);
|
||||
final TaskQueue tq = new TaskQueue(storage, lockbox);
|
||||
tq.bootstrap();
|
||||
tq.start();
|
||||
return tq;
|
||||
}
|
||||
|
@ -63,7 +66,8 @@ public class TaskQueueTest
|
|||
public void testAddRemove() throws Exception
|
||||
{
|
||||
final TaskStorage ts = new LocalTaskStorage();
|
||||
final TaskQueue tq = newTaskQueueWithStorage(ts);
|
||||
final TaskLockbox tl = new TaskLockbox(ts);
|
||||
final TaskQueue tq = newTaskQueue(ts, tl);
|
||||
|
||||
final Task[] tasks = {
|
||||
newTask("T0", "G0", "bar", new Interval("2011/P1Y")),
|
||||
|
@ -96,9 +100,9 @@ public class TaskQueueTest
|
|||
// take max number of tasks
|
||||
final List<Task> taken = Lists.newArrayList();
|
||||
while (true) {
|
||||
final VersionedTaskWrapper taskWrapper = tq.poll();
|
||||
if(taskWrapper != null) {
|
||||
taken.add(taskWrapper.getTask());
|
||||
final Task task = tq.poll();
|
||||
if(task != null) {
|
||||
taken.add(task);
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
|
@ -114,8 +118,7 @@ public class TaskQueueTest
|
|||
);
|
||||
|
||||
// mark one done
|
||||
final TestCommitRunnable commit1 = newCommitRunnable();
|
||||
tq.notify(tasks[2], tasks[2].run(null, null, null), commit1);
|
||||
tq.notify(tasks[2], tasks[2].run(null));
|
||||
|
||||
// get its status back
|
||||
Assert.assertEquals(
|
||||
|
@ -124,20 +127,12 @@ public class TaskQueueTest
|
|||
ts.getStatus(tasks[2].getId()).get().getStatusCode()
|
||||
);
|
||||
|
||||
Assert.assertEquals("Commit #1 wasRun", commit1.wasRun(), true);
|
||||
|
||||
// Can't do a task twice
|
||||
final TestCommitRunnable commit2 = newCommitRunnable();
|
||||
tq.notify(tasks[2], tasks[2].run(null, null, null), commit2);
|
||||
|
||||
Assert.assertEquals("Commit #2 wasRun", commit2.wasRun(), false);
|
||||
|
||||
// we should be able to get one more task now
|
||||
// We should be able to get one more task now
|
||||
taken.clear();
|
||||
while (true) {
|
||||
final VersionedTaskWrapper taskWrapper = tq.poll();
|
||||
if(taskWrapper != null) {
|
||||
taken.add(taskWrapper.getTask());
|
||||
final Task task = tq.poll();
|
||||
if(task != null) {
|
||||
taken.add(task);
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
|
@ -160,7 +155,17 @@ public class TaskQueueTest
|
|||
public void testContinues() throws Exception
|
||||
{
|
||||
final TaskStorage ts = new LocalTaskStorage();
|
||||
final TaskQueue tq = newTaskQueueWithStorage(ts);
|
||||
final TaskLockbox tl = new TaskLockbox(ts);
|
||||
final TaskQueue tq = newTaskQueue(ts, tl);
|
||||
final TaskToolbox tb = new TaskToolbox(
|
||||
null,
|
||||
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
||||
final Task t0 = newTask("T0", "G0", "bar", new Interval("2011/P1Y"));
|
||||
final Task t1 = newContinuedTask("T1", "G1", "bar", new Interval("2013/P1Y"), Lists.newArrayList(t0));
|
||||
|
@ -168,17 +173,17 @@ public class TaskQueueTest
|
|||
|
||||
Assert.assertTrue("T0 isPresent (#1)", !ts.getStatus("T0").isPresent());
|
||||
Assert.assertTrue("T1 isPresent (#1)", ts.getStatus("T1").isPresent());
|
||||
Assert.assertTrue("T1 isRunnable (#1)", ts.getStatus("T1").get().isRunnable());
|
||||
Assert.assertTrue("T1 isRunnable (#1)", ts.getStatus("T1").get().isRunnable());
|
||||
Assert.assertTrue("T1 isComplete (#1)", !ts.getStatus("T1").get().isComplete());
|
||||
|
||||
// should be able to get t1 out
|
||||
Assert.assertEquals("poll #1", "T1", tq.poll().getTask().getId());
|
||||
Assert.assertEquals("poll #1", "T1", tq.poll().getId());
|
||||
Assert.assertNull("poll #2", tq.poll());
|
||||
|
||||
// report T1 done. Should cause T0 to be created
|
||||
tq.notify(t1, t1.run(null, null, null));
|
||||
tq.notify(t1, t1.run(tb));
|
||||
|
||||
Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent());
|
||||
Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent());
|
||||
Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable());
|
||||
Assert.assertTrue("T0 isComplete (#2)", !ts.getStatus("T0").get().isComplete());
|
||||
Assert.assertTrue("T1 isPresent (#2)", ts.getStatus("T1").isPresent());
|
||||
|
@ -186,13 +191,13 @@ public class TaskQueueTest
|
|||
Assert.assertTrue("T1 isComplete (#2)", ts.getStatus("T1").get().isComplete());
|
||||
|
||||
// should be able to get t0 out
|
||||
Assert.assertEquals("poll #3", "T0", tq.poll().getTask().getId());
|
||||
Assert.assertEquals("poll #3", "T0", tq.poll().getId());
|
||||
Assert.assertNull("poll #4", tq.poll());
|
||||
|
||||
// report T0 done. Should cause T0, T1 to be marked complete
|
||||
tq.notify(t0, t0.run(null, null, null));
|
||||
tq.notify(t0, t0.run(tb));
|
||||
|
||||
Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent());
|
||||
Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent());
|
||||
Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable());
|
||||
Assert.assertTrue("T0 isComplete (#3)", ts.getStatus("T0").get().isComplete());
|
||||
Assert.assertTrue("T1 isPresent (#3)", ts.getStatus("T1").isPresent());
|
||||
|
@ -207,7 +212,17 @@ public class TaskQueueTest
|
|||
public void testConcurrency() throws Exception
|
||||
{
|
||||
final TaskStorage ts = new LocalTaskStorage();
|
||||
final TaskQueue tq = newTaskQueueWithStorage(ts);
|
||||
final TaskLockbox tl = new TaskLockbox(ts);
|
||||
final TaskQueue tq = newTaskQueue(ts, tl);
|
||||
final TaskToolbox tb = new TaskToolbox(
|
||||
null,
|
||||
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
||||
// Imagine a larger task that splits itself up into pieces
|
||||
final Task t1 = newTask("T1", "G0", "bar", new Interval("2011-01-01/P1D"));
|
||||
|
@ -224,15 +239,16 @@ public class TaskQueueTest
|
|||
|
||||
tq.add(t0);
|
||||
|
||||
final VersionedTaskWrapper wt0 = tq.poll();
|
||||
Assert.assertEquals("wt0 task id", "T0", wt0.getTask().getId());
|
||||
final Task wt0 = tq.poll();
|
||||
final TaskLock wt0Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt0));
|
||||
Assert.assertEquals("wt0 task id", "T0", wt0.getId());
|
||||
Assert.assertNull("null poll #1", tq.poll());
|
||||
|
||||
// Sleep a bit to avoid false test passes
|
||||
Thread.sleep(5);
|
||||
|
||||
// Finish t0
|
||||
tq.notify(t0, t0.run(null, null, null));
|
||||
tq.notify(t0, t0.run(tb));
|
||||
|
||||
// take max number of tasks
|
||||
final Set<String> taken = Sets.newHashSet();
|
||||
|
@ -241,15 +257,16 @@ public class TaskQueueTest
|
|||
// Sleep a bit to avoid false test passes
|
||||
Thread.sleep(5);
|
||||
|
||||
final VersionedTaskWrapper taskWrapper = tq.poll();
|
||||
final Task task = tq.poll();
|
||||
|
||||
if(taskWrapper != null) {
|
||||
if(task != null) {
|
||||
final TaskLock taskLock = Iterables.getOnlyElement(tl.findLocksForTask(task));
|
||||
Assert.assertEquals(
|
||||
String.format("%s version", taskWrapper.getTask().getId()),
|
||||
wt0.getVersion(),
|
||||
taskWrapper.getVersion()
|
||||
String.format("%s version", task.getId()),
|
||||
wt0Lock.getVersion(),
|
||||
taskLock.getVersion()
|
||||
);
|
||||
taken.add(taskWrapper.getTask().getId());
|
||||
taken.add(task.getId());
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
|
@ -259,34 +276,36 @@ public class TaskQueueTest
|
|||
Assert.assertEquals("taken", Sets.newHashSet("T1", "T3"), taken);
|
||||
|
||||
// Finish t1
|
||||
tq.notify(t1, t1.run(null, null, null));
|
||||
tq.notify(t1, t1.run(null));
|
||||
Assert.assertNull("null poll #2", tq.poll());
|
||||
|
||||
// Finish t3
|
||||
tq.notify(t3, t3.run(null, null, null));
|
||||
tq.notify(t3, t3.run(tb));
|
||||
|
||||
// We should be able to get t2 now
|
||||
final VersionedTaskWrapper wt2 = tq.poll();
|
||||
Assert.assertEquals("wt2 task id", "T2", wt2.getTask().getId());
|
||||
Assert.assertEquals("wt2 group id", "G1", wt2.getTask().getGroupId());
|
||||
Assert.assertNotSame("wt2 version", wt0.getVersion(), wt2.getVersion());
|
||||
final Task wt2 = tq.poll();
|
||||
final TaskLock wt2Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt2));
|
||||
Assert.assertEquals("wt2 task id", "T2", wt2.getId());
|
||||
Assert.assertEquals("wt2 group id", "G1", wt2.getGroupId());
|
||||
Assert.assertNotSame("wt2 version", wt0Lock.getVersion(), wt2Lock.getVersion());
|
||||
Assert.assertNull("null poll #3", tq.poll());
|
||||
|
||||
// Finish t2
|
||||
tq.notify(t2, t2.run(null, null, null));
|
||||
tq.notify(t2, t2.run(tb));
|
||||
|
||||
// We should be able to get t4
|
||||
// And it should be in group G0, but that group should have a different version than last time
|
||||
// (Since the previous transaction named "G0" has ended and transaction names are not necessarily tied to
|
||||
// one version if they end and are re-started)
|
||||
final VersionedTaskWrapper wt4 = tq.poll();
|
||||
Assert.assertEquals("wt4 task id", "T4", wt4.getTask().getId());
|
||||
Assert.assertEquals("wt4 group id", "G0", wt4.getTask().getGroupId());
|
||||
Assert.assertNotSame("wt4 version", wt0.getVersion(), wt4.getVersion());
|
||||
Assert.assertNotSame("wt4 version", wt2.getVersion(), wt4.getVersion());
|
||||
final Task wt4 = tq.poll();
|
||||
final TaskLock wt4Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt4));
|
||||
Assert.assertEquals("wt4 task id", "T4", wt4.getId());
|
||||
Assert.assertEquals("wt4 group id", "G0", wt4.getGroupId());
|
||||
Assert.assertNotSame("wt4 version", wt0Lock.getVersion(), wt4Lock.getVersion());
|
||||
Assert.assertNotSame("wt4 version", wt2Lock.getVersion(), wt4Lock.getVersion());
|
||||
|
||||
// Kind of done testing at this point, but let's finish t4 anyway
|
||||
tq.notify(t4, t4.run(null, null, null));
|
||||
tq.notify(t4, t4.run(tb));
|
||||
Assert.assertNull("null poll #4", tq.poll());
|
||||
}
|
||||
|
||||
|
@ -294,161 +313,44 @@ public class TaskQueueTest
|
|||
public void testBootstrap() throws Exception
|
||||
{
|
||||
final TaskStorage storage = new LocalTaskStorage();
|
||||
final TaskLockbox lockbox = new TaskLockbox(storage);
|
||||
|
||||
storage.insert(newTask("T1", "G1", "bar", new Interval("2011-01-01/P1D")), TaskStatus.running("T1"));
|
||||
storage.insert(newTask("T2", "G2", "bar", new Interval("2011-02-01/P1D")), TaskStatus.running("T2"));
|
||||
storage.setVersion("T1", "1234");
|
||||
storage.addLock("T1", new TaskLock("G1", "bar", new Interval("2011-01-01/P1D"), "1234"));
|
||||
|
||||
final TaskQueue tq = newTaskQueueWithStorage(storage);
|
||||
final TaskQueue tq = newTaskQueue(storage, lockbox);
|
||||
|
||||
final VersionedTaskWrapper vt1 = tq.poll();
|
||||
Assert.assertEquals("vt1 id", "T1", vt1.getTask().getId());
|
||||
Assert.assertEquals("vt1 version", "1234", vt1.getVersion());
|
||||
final Task vt1 = tq.poll();
|
||||
final TaskLock vt1Lock = Iterables.getOnlyElement(lockbox.findLocksForTask(vt1));
|
||||
Assert.assertEquals("vt1 id", "T1", vt1.getId());
|
||||
Assert.assertEquals("vt1 version", "1234", vt1Lock.getVersion());
|
||||
|
||||
tq.notify(vt1.getTask(), TaskStatus.success("T1", ImmutableSet.<DataSegment>of()));
|
||||
tq.notify(vt1, TaskStatus.success("T1"));
|
||||
|
||||
// re-bootstrap
|
||||
tq.stop();
|
||||
storage.setStatus("T2", TaskStatus.failure("T2"));
|
||||
storage.setStatus(TaskStatus.failure("T2"));
|
||||
tq.bootstrap();
|
||||
tq.start();
|
||||
|
||||
Assert.assertNull("null poll", tq.poll());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRealtimeish() throws Exception
|
||||
{
|
||||
final TaskStorage ts = new LocalTaskStorage();
|
||||
final TaskQueue tq = newTaskQueueWithStorage(ts);
|
||||
|
||||
class StructThingy
|
||||
{
|
||||
boolean pushed = false;
|
||||
boolean pass1 = false;
|
||||
boolean pass2 = false;
|
||||
}
|
||||
|
||||
final StructThingy structThingy = new StructThingy();
|
||||
|
||||
// Test a task that acts sort of like the realtime task, to make sure this case works.
|
||||
final Task rtTask = new AbstractTask("id1", "ds", new Interval("2010-01-01T00:00:00Z/PT1H"))
|
||||
{
|
||||
@Override
|
||||
public Type getType()
|
||||
{
|
||||
return Type.TEST;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(
|
||||
TaskContext context, TaskToolbox toolbox, TaskCallback callback
|
||||
) throws Exception
|
||||
{
|
||||
final Set<DataSegment> segments = ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource("ds")
|
||||
.interval(new Interval("2010-01-01T00:00:00Z/PT1H"))
|
||||
.version(context.getVersion())
|
||||
.build()
|
||||
);
|
||||
|
||||
final List<Task> nextTasks = ImmutableList.of(
|
||||
newTask(
|
||||
"id2",
|
||||
"id2",
|
||||
"ds",
|
||||
new Interval(
|
||||
"2010-01-01T01:00:00Z/PT1H"
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
final TaskStatus status1 = TaskStatus.running("id1").withNextTasks(nextTasks);
|
||||
final TaskStatus status2 = TaskStatus.running("id1").withNextTasks(nextTasks).withSegments(segments);
|
||||
final TaskStatus status3 = TaskStatus.success("id1").withNextTasks(nextTasks).withSegments(segments);
|
||||
|
||||
// Create a new realtime task!
|
||||
callback.notify(status1);
|
||||
if(ts.getStatus("id2").get().getStatusCode() == TaskStatus.Status.RUNNING) {
|
||||
// test immediate creation of nextTask
|
||||
structThingy.pass1 = true;
|
||||
}
|
||||
|
||||
// Hand off a segment!
|
||||
callback.notify(status2);
|
||||
if(structThingy.pushed) {
|
||||
// test immediate handoff of segment
|
||||
structThingy.pass2 = true;
|
||||
}
|
||||
|
||||
// Return success!
|
||||
return status3;
|
||||
}
|
||||
};
|
||||
|
||||
tq.add(rtTask);
|
||||
|
||||
final VersionedTaskWrapper vt = tq.poll();
|
||||
final TaskCallback callback = new TaskCallback()
|
||||
{
|
||||
@Override
|
||||
public void notify(final TaskStatus status)
|
||||
{
|
||||
final Runnable commitRunnable = new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
if(status.getNextTasks().size() > 0) {
|
||||
structThingy.pushed = true;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
tq.notify(vt.getTask(), status, commitRunnable);
|
||||
}
|
||||
};
|
||||
|
||||
callback.notify(vt.getTask().run(new TaskContext(vt.getVersion(), null, null), null, callback));
|
||||
|
||||
// OK, finally ready to test stuff.
|
||||
Assert.assertTrue("pass1", structThingy.pass1);
|
||||
Assert.assertTrue("pass2", structThingy.pass2);
|
||||
Assert.assertTrue("id1 isSuccess", ts.getStatus("id1").get().isSuccess());
|
||||
Assert.assertTrue(
|
||||
"id1 isSuccess (merged)",
|
||||
new TaskStorageQueryAdapter(ts).getSameGroupMergedStatus("id1").get().isSuccess()
|
||||
);
|
||||
Assert.assertTrue("id2 isRunnable", ts.getStatus("id2").get().isRunnable());
|
||||
}
|
||||
|
||||
private static Task newTask(final String id, final String groupId, final String dataSource, final Interval interval)
|
||||
{
|
||||
return new AbstractTask(id, groupId, dataSource, interval)
|
||||
{
|
||||
@Override
|
||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
return TaskStatus.success(
|
||||
id,
|
||||
ImmutableSet.of(
|
||||
new DataSegment(
|
||||
dataSource,
|
||||
interval,
|
||||
new DateTime("2012-01-02").toString(),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
-1
|
||||
)
|
||||
)
|
||||
);
|
||||
return TaskStatus.success(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Type.TEST;
|
||||
return "null";
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -464,37 +366,17 @@ public class TaskQueueTest
|
|||
return new AbstractTask(id, groupId, dataSource, interval)
|
||||
{
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Type.TEST;
|
||||
return "null";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
return TaskStatus.success(id).withNextTasks(nextTasks);
|
||||
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks));
|
||||
return TaskStatus.success(id);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private static TestCommitRunnable newCommitRunnable()
|
||||
{
|
||||
return new TestCommitRunnable();
|
||||
}
|
||||
|
||||
private static class TestCommitRunnable implements Runnable
|
||||
{
|
||||
private boolean _wasRun = false;
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
_wasRun = true;
|
||||
}
|
||||
|
||||
public boolean wasRun()
|
||||
{
|
||||
return _wasRun;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue