mirror of
https://github.com/apache/druid.git
synced 2025-02-09 03:24:55 +00:00
Merge branch 'task-stuff' into indexing_refactor
Conflicts: merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java merger/src/main/java/com/metamx/druid/merger/common/task/Task.java merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkQueue.java merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java
This commit is contained in:
commit
bf0d9c601d
@ -19,8 +19,6 @@
|
|||||||
|
|
||||||
package com.metamx.druid.merger.common;
|
package com.metamx.druid.merger.common;
|
||||||
|
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
|
||||||
|
|
||||||
public interface TaskCallback
|
public interface TaskCallback
|
||||||
{
|
{
|
||||||
public void notify(TaskStatus status);
|
public void notify(TaskStatus status);
|
||||||
|
@ -1,57 +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.fasterxml.jackson.annotation.JsonCreator;
|
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
|
||||||
import com.metamx.druid.merger.common.task.Task;
|
|
||||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
*/
|
|
||||||
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.
|
* 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.Objects;
|
||||||
import com.google.common.base.Preconditions;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.google.common.collect.Maps;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
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.joda.time.Interval;
|
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 groupId;
|
||||||
private final String dataSource;
|
private final String dataSource;
|
||||||
private final Interval interval;
|
private final Interval interval;
|
||||||
private final String version;
|
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.groupId = groupId;
|
||||||
this.dataSource = dataSource;
|
this.dataSource = dataSource;
|
||||||
@ -48,75 +48,48 @@ public class TaskGroup
|
|||||||
this.version = version;
|
this.version = version;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
public String getGroupId()
|
public String getGroupId()
|
||||||
{
|
{
|
||||||
return groupId;
|
return groupId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
public String getDataSource()
|
public String getDataSource()
|
||||||
{
|
{
|
||||||
return dataSource;
|
return dataSource;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
public Interval getInterval()
|
public Interval getInterval()
|
||||||
{
|
{
|
||||||
return interval;
|
return interval;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
public String getVersion()
|
public String getVersion()
|
||||||
{
|
{
|
||||||
return version;
|
return version;
|
||||||
}
|
}
|
||||||
|
|
||||||
public CommitStyle getCommitStyle()
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
{
|
{
|
||||||
// TODO -- should be configurable
|
if (!(o instanceof TaskLock)) {
|
||||||
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())) {
|
|
||||||
return false;
|
return false;
|
||||||
} else {
|
} else {
|
||||||
taskMap.put(task.getId(), task);
|
final TaskLock x = (TaskLock) o;
|
||||||
return true;
|
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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
@Override
|
||||||
* Returns true if this group contains a particular task.
|
public int hashCode()
|
||||||
*/
|
|
||||||
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)
|
|
||||||
{
|
{
|
||||||
return taskMap.remove(taskId);
|
return Objects.hashCode(groupId, dataSource, interval, version);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -127,7 +100,6 @@ public class TaskGroup
|
|||||||
.add("dataSource", dataSource)
|
.add("dataSource", dataSource)
|
||||||
.add("interval", interval)
|
.add("interval", interval)
|
||||||
.add("version", version)
|
.add("version", version)
|
||||||
.add("tasks", taskMap.keySet())
|
|
||||||
.toString();
|
.toString();
|
||||||
}
|
}
|
||||||
}
|
}
|
@ -24,18 +24,11 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
|
|||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Objects;
|
import com.google.common.base.Objects;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.ImmutableList;
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
|
||||||
import com.metamx.druid.client.DataSegment;
|
|
||||||
import com.metamx.druid.merger.common.task.Task;
|
|
||||||
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Set;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Represents the status of a task. The task may be ongoing ({@link #isComplete()} false) or it may be
|
* Represents the status of a task. The task may be ongoing ({@link #isComplete()} false) or it may be
|
||||||
* complete ({@link #isComplete()} true).
|
* complete ({@link #isComplete()} true).
|
||||||
*
|
* <p/>
|
||||||
* TaskStatus objects are immutable.
|
* TaskStatus objects are immutable.
|
||||||
*/
|
*/
|
||||||
public class TaskStatus
|
public class TaskStatus
|
||||||
@ -49,49 +42,20 @@ public class TaskStatus
|
|||||||
|
|
||||||
public static TaskStatus running(String taskId)
|
public static TaskStatus running(String taskId)
|
||||||
{
|
{
|
||||||
return new TaskStatus(
|
return new TaskStatus(taskId, Status.RUNNING, -1);
|
||||||
taskId,
|
|
||||||
Status.RUNNING,
|
|
||||||
ImmutableSet.<DataSegment>of(),
|
|
||||||
ImmutableSet.<DataSegment>of(),
|
|
||||||
ImmutableList.<Task>of(),
|
|
||||||
-1
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static TaskStatus success(String taskId)
|
public static TaskStatus success(String taskId)
|
||||||
{
|
{
|
||||||
return success(taskId, ImmutableSet.<DataSegment>of());
|
return new TaskStatus(taskId, Status.SUCCESS, -1);
|
||||||
}
|
|
||||||
|
|
||||||
public static TaskStatus success(String taskId, Set<DataSegment> segments)
|
|
||||||
{
|
|
||||||
return new TaskStatus(
|
|
||||||
taskId,
|
|
||||||
Status.SUCCESS,
|
|
||||||
segments,
|
|
||||||
ImmutableSet.<DataSegment>of(),
|
|
||||||
ImmutableList.<Task>of(),
|
|
||||||
-1
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static TaskStatus failure(String taskId)
|
public static TaskStatus failure(String taskId)
|
||||||
{
|
{
|
||||||
return new TaskStatus(
|
return new TaskStatus(taskId, Status.FAILED, -1);
|
||||||
taskId,
|
|
||||||
Status.FAILED,
|
|
||||||
ImmutableSet.<DataSegment>of(),
|
|
||||||
ImmutableSet.<DataSegment>of(),
|
|
||||||
ImmutableList.<Task>of(),
|
|
||||||
-1
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private final String id;
|
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 Status status;
|
||||||
private final long duration;
|
private final long duration;
|
||||||
|
|
||||||
@ -99,42 +63,16 @@ public class TaskStatus
|
|||||||
private TaskStatus(
|
private TaskStatus(
|
||||||
@JsonProperty("id") String id,
|
@JsonProperty("id") String id,
|
||||||
@JsonProperty("status") Status status,
|
@JsonProperty("status") Status status,
|
||||||
@JsonProperty("segments") Set<DataSegment> segments,
|
|
||||||
@JsonProperty("segmentsNuked") Set<DataSegment> segmentsNuked,
|
|
||||||
@JsonProperty("nextTasks") List<Task> nextTasks,
|
|
||||||
@JsonProperty("duration") long duration
|
@JsonProperty("duration") long duration
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.id = id;
|
this.id = id;
|
||||||
this.segments = ImmutableSet.copyOf(segments);
|
|
||||||
this.segmentsNuked = ImmutableSet.copyOf(segmentsNuked);
|
|
||||||
this.nextTasks = ImmutableList.copyOf(nextTasks);
|
|
||||||
this.status = status;
|
this.status = status;
|
||||||
this.duration = duration;
|
this.duration = duration;
|
||||||
|
|
||||||
// Check class invariants.
|
// Check class invariants.
|
||||||
Preconditions.checkNotNull(id, "id");
|
Preconditions.checkNotNull(id, "id");
|
||||||
Preconditions.checkNotNull(status, "status");
|
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")
|
@JsonProperty("id")
|
||||||
@ -149,24 +87,6 @@ public class TaskStatus
|
|||||||
return status;
|
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")
|
@JsonProperty("duration")
|
||||||
public long getDuration()
|
public long getDuration()
|
||||||
{
|
{
|
||||||
@ -212,25 +132,9 @@ public class TaskStatus
|
|||||||
return status == Status.FAILED;
|
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)
|
public TaskStatus withDuration(long _duration)
|
||||||
{
|
{
|
||||||
return new TaskStatus(id, status, segments, segmentsNuked, nextTasks, _duration);
|
return new TaskStatus(id, status, _duration);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -238,8 +142,6 @@ public class TaskStatus
|
|||||||
{
|
{
|
||||||
return Objects.toStringHelper(this)
|
return Objects.toStringHelper(this)
|
||||||
.add("id", id)
|
.add("id", id)
|
||||||
.add("segments", segments)
|
|
||||||
.add("nextTasks", nextTasks)
|
|
||||||
.add("status", status)
|
.add("status", status)
|
||||||
.add("duration", duration)
|
.add("duration", duration)
|
||||||
.toString();
|
.toString();
|
||||||
|
@ -27,6 +27,7 @@ import com.metamx.druid.loading.S3ZippedSegmentPuller;
|
|||||||
import com.metamx.druid.loading.SegmentKiller;
|
import com.metamx.druid.loading.SegmentKiller;
|
||||||
import com.metamx.druid.loading.SegmentPuller;
|
import com.metamx.druid.loading.SegmentPuller;
|
||||||
import com.metamx.druid.loading.SegmentPusher;
|
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.config.TaskConfig;
|
||||||
import com.metamx.druid.merger.common.task.Task;
|
import com.metamx.druid.merger.common.task.Task;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
@ -41,6 +42,7 @@ import java.util.Map;
|
|||||||
public class TaskToolbox
|
public class TaskToolbox
|
||||||
{
|
{
|
||||||
private final TaskConfig config;
|
private final TaskConfig config;
|
||||||
|
private final TaskActionClient taskActionClient;
|
||||||
private final ServiceEmitter emitter;
|
private final ServiceEmitter emitter;
|
||||||
private final RestS3Service s3Client;
|
private final RestS3Service s3Client;
|
||||||
private final SegmentPusher segmentPusher;
|
private final SegmentPusher segmentPusher;
|
||||||
@ -49,6 +51,7 @@ public class TaskToolbox
|
|||||||
|
|
||||||
public TaskToolbox(
|
public TaskToolbox(
|
||||||
TaskConfig config,
|
TaskConfig config,
|
||||||
|
TaskActionClient taskActionClient,
|
||||||
ServiceEmitter emitter,
|
ServiceEmitter emitter,
|
||||||
RestS3Service s3Client,
|
RestS3Service s3Client,
|
||||||
SegmentPusher segmentPusher,
|
SegmentPusher segmentPusher,
|
||||||
@ -57,6 +60,7 @@ public class TaskToolbox
|
|||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.config = config;
|
this.config = config;
|
||||||
|
this.taskActionClient = taskActionClient;
|
||||||
this.emitter = emitter;
|
this.emitter = emitter;
|
||||||
this.s3Client = s3Client;
|
this.s3Client = s3Client;
|
||||||
this.segmentPusher = segmentPusher;
|
this.segmentPusher = segmentPusher;
|
||||||
@ -69,16 +73,16 @@ public class TaskToolbox
|
|||||||
return config;
|
return config;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public TaskActionClient getTaskActionClient()
|
||||||
|
{
|
||||||
|
return taskActionClient;
|
||||||
|
}
|
||||||
|
|
||||||
public ServiceEmitter getEmitter()
|
public ServiceEmitter getEmitter()
|
||||||
{
|
{
|
||||||
return emitter;
|
return emitter;
|
||||||
}
|
}
|
||||||
|
|
||||||
public RestS3Service getS3Client()
|
|
||||||
{
|
|
||||||
return s3Client;
|
|
||||||
}
|
|
||||||
|
|
||||||
public SegmentPusher getSegmentPusher()
|
public SegmentPusher getSegmentPusher()
|
||||||
{
|
{
|
||||||
return segmentPusher;
|
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 com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.fasterxml.jackson.core.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 com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.fasterxml.jackson.core.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 com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.fasterxml.jackson.core.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 com.fasterxml.jackson.databind.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 com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.fasterxml.jackson.core.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 com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.fasterxml.jackson.core.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 com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.fasterxml.jackson.core.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 com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.fasterxml.jackson.core.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 com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.fasterxml.jackson.core.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 com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
|
import com.fasterxml.jackson.core.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;
|
||||||
|
}
|
||||||
|
}
|
@ -22,11 +22,10 @@ package com.metamx.druid.merger.common.task;
|
|||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Objects;
|
import com.google.common.base.Objects;
|
||||||
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
import com.metamx.druid.merger.common.TaskStatus;
|
||||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
import com.metamx.druid.merger.common.TaskToolbox;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
public abstract class AbstractTask implements Task
|
public abstract class AbstractTask implements Task
|
||||||
@ -34,30 +33,19 @@ public abstract class AbstractTask implements Task
|
|||||||
private final String id;
|
private final String id;
|
||||||
private final String groupId;
|
private final String groupId;
|
||||||
private final String dataSource;
|
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);
|
this(id, id, dataSource, interval);
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonCreator
|
protected AbstractTask(String id, String groupId, String dataSource, Interval interval)
|
||||||
public AbstractTask(
|
|
||||||
@JsonProperty("id") String id,
|
|
||||||
@JsonProperty("groupId") String groupId,
|
|
||||||
@JsonProperty("dataSource") String dataSource,
|
|
||||||
@JsonProperty("interval") Interval interval
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(id, "id");
|
this.id = Preconditions.checkNotNull(id, "id");
|
||||||
Preconditions.checkNotNull(groupId, "groupId");
|
this.groupId = Preconditions.checkNotNull(groupId, "groupId");
|
||||||
Preconditions.checkNotNull(dataSource, "dataSource");
|
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
|
||||||
Preconditions.checkNotNull(interval, "interval");
|
this.interval = Optional.fromNullable(interval);
|
||||||
|
|
||||||
this.id = id;
|
|
||||||
this.groupId = groupId;
|
|
||||||
this.dataSource = dataSource;
|
|
||||||
this.interval = interval;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
@ -81,15 +69,22 @@ public abstract class AbstractTask implements Task
|
|||||||
return dataSource;
|
return dataSource;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
|
||||||
@Override
|
@Override
|
||||||
public Interval getInterval()
|
public Optional<Interval> getFixedInterval()
|
||||||
{
|
{
|
||||||
return interval;
|
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
|
@Override
|
||||||
public TaskStatus preflight(TaskContext context) throws Exception
|
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
|
||||||
{
|
{
|
||||||
return TaskStatus.running(id);
|
return TaskStatus.running(id);
|
||||||
}
|
}
|
||||||
@ -101,7 +96,7 @@ public abstract class AbstractTask implements Task
|
|||||||
.add("id", id)
|
.add("id", id)
|
||||||
.add("type", getType())
|
.add("type", getType())
|
||||||
.add("dataSource", dataSource)
|
.add("dataSource", dataSource)
|
||||||
.add("interval", getInterval())
|
.add("interval", getFixedInterval())
|
||||||
.toString();
|
.toString();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -62,7 +62,7 @@ public class AppendTask extends MergeTask
|
|||||||
throws Exception
|
throws Exception
|
||||||
{
|
{
|
||||||
VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<String, DataSegment>(
|
VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<String, DataSegment>(
|
||||||
Ordering.natural().nullsFirst()
|
Ordering.<String>natural().nullsFirst()
|
||||||
);
|
);
|
||||||
|
|
||||||
for (DataSegment segment : segments.keySet()) {
|
for (DataSegment segment : segments.keySet()) {
|
||||||
@ -111,9 +111,9 @@ public class AppendTask extends MergeTask
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return Task.Type.APPEND;
|
return "append";
|
||||||
}
|
}
|
||||||
|
|
||||||
private class SegmentToMergeHolder
|
private class SegmentToMergeHolder
|
||||||
|
@ -82,8 +82,8 @@ public class DefaultMergeTask extends MergeTask
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return Task.Type.MERGE;
|
return "merge";
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -19,9 +19,11 @@
|
|||||||
|
|
||||||
package com.metamx.druid.merger.common.task;
|
package com.metamx.druid.merger.common.task;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
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.google.common.collect.Lists;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.QueryGranularity;
|
import com.metamx.druid.QueryGranularity;
|
||||||
@ -31,10 +33,11 @@ import com.metamx.druid.index.v1.IncrementalIndex;
|
|||||||
import com.metamx.druid.index.v1.IncrementalIndexAdapter;
|
import com.metamx.druid.index.v1.IncrementalIndexAdapter;
|
||||||
import com.metamx.druid.index.v1.IndexMerger;
|
import com.metamx.druid.index.v1.IndexMerger;
|
||||||
import com.metamx.druid.index.v1.IndexableAdapter;
|
import com.metamx.druid.index.v1.IndexableAdapter;
|
||||||
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.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
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.shard.NoneShardSpec;
|
import com.metamx.druid.shard.NoneShardSpec;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
@ -60,29 +63,31 @@ public class DeleteTask extends AbstractTask
|
|||||||
new DateTime().toString()
|
new DateTime().toString()
|
||||||
),
|
),
|
||||||
dataSource,
|
dataSource,
|
||||||
interval
|
Preconditions.checkNotNull(interval, "interval")
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return Task.Type.DELETE;
|
return "delete";
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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
|
// 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 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
|
// Create DataSegment
|
||||||
final DataSegment segment =
|
final DataSegment segment =
|
||||||
DataSegment.builder()
|
DataSegment.builder()
|
||||||
.dataSource(this.getDataSource())
|
.dataSource(this.getDataSource())
|
||||||
.interval(this.getInterval())
|
.interval(interval)
|
||||||
.version(context.getVersion())
|
.version(myLock.getVersion())
|
||||||
.shardSpec(new NoneShardSpec())
|
.shardSpec(new NoneShardSpec())
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
@ -99,6 +104,8 @@ public class DeleteTask extends AbstractTask
|
|||||||
segment.getVersion()
|
segment.getVersion()
|
||||||
);
|
);
|
||||||
|
|
||||||
return TaskStatus.success(getId(), ImmutableSet.of(uploadedSegment));
|
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment)));
|
||||||
|
|
||||||
|
return TaskStatus.success(getId());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -22,6 +22,7 @@ package com.metamx.druid.merger.common.task;
|
|||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
@ -30,18 +31,15 @@ import com.google.common.collect.TreeMultiset;
|
|||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.input.InputRow;
|
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.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
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.Firehose;
|
||||||
import com.metamx.druid.realtime.FirehoseFactory;
|
import com.metamx.druid.realtime.FirehoseFactory;
|
||||||
import com.metamx.druid.realtime.Schema;
|
import com.metamx.druid.realtime.Schema;
|
||||||
import com.metamx.druid.shard.NoneShardSpec;
|
import com.metamx.druid.shard.NoneShardSpec;
|
||||||
import com.metamx.druid.shard.ShardSpec;
|
import com.metamx.druid.shard.ShardSpec;
|
||||||
import com.metamx.druid.shard.SingleDimensionShardSpec;
|
import com.metamx.druid.shard.SingleDimensionShardSpec;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@ -52,8 +50,10 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||||||
{
|
{
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
private final FirehoseFactory firehoseFactory;
|
private final FirehoseFactory firehoseFactory;
|
||||||
@JsonProperty private final Schema schema;
|
@JsonProperty
|
||||||
@JsonProperty private final long targetPartitionSize;
|
private final Schema schema;
|
||||||
|
@JsonProperty
|
||||||
|
private final long targetPartitionSize;
|
||||||
|
|
||||||
private static final Logger log = new Logger(IndexTask.class);
|
private static final Logger log = new Logger(IndexTask.class);
|
||||||
|
|
||||||
@ -75,7 +75,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||||||
),
|
),
|
||||||
groupId,
|
groupId,
|
||||||
schema.getDataSource(),
|
schema.getDataSource(),
|
||||||
interval
|
Preconditions.checkNotNull(interval, "interval")
|
||||||
);
|
);
|
||||||
|
|
||||||
this.firehoseFactory = firehoseFactory;
|
this.firehoseFactory = firehoseFactory;
|
||||||
@ -84,21 +84,20 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return Type.INDEX;
|
return "index_partitions";
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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);
|
log.info("Running with targetPartitionSize[%d]", targetPartitionSize);
|
||||||
|
|
||||||
// This is similar to what DeterminePartitionsJob does in the hadoop indexer, but we don't require
|
// TODO: Replace/merge/whatever with hadoop determine-partitions code
|
||||||
// a preconfigured partition dimension (we'll just pick the one with highest cardinality).
|
|
||||||
|
|
||||||
// NOTE: Space-efficiency (stores all unique dimension values, although at least not all combinations)
|
// We know this exists
|
||||||
// NOTE: Time-efficiency (runs all this on one single node instead of through map/reduce)
|
final Interval interval = getFixedInterval().get();
|
||||||
|
|
||||||
// Blacklist dimensions that have multiple values per row
|
// Blacklist dimensions that have multiple values per row
|
||||||
final Set<String> unusableDimensions = Sets.newHashSet();
|
final Set<String> unusableDimensions = Sets.newHashSet();
|
||||||
@ -114,7 +113,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||||||
|
|
||||||
final InputRow inputRow = firehose.nextRow();
|
final InputRow inputRow = firehose.nextRow();
|
||||||
|
|
||||||
if (getInterval().contains(inputRow.getTimestampFromEpoch())) {
|
if (interval.contains(inputRow.getTimestampFromEpoch())) {
|
||||||
|
|
||||||
// Extract dimensions from event
|
// Extract dimensions from event
|
||||||
for (final String dim : inputRow.getDimensions()) {
|
for (final String dim : inputRow.getDimensions()) {
|
||||||
@ -229,28 +228,30 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return TaskStatus.success(getId()).withNextTasks(
|
List<Task> nextTasks = Lists.transform(
|
||||||
Lists.transform(
|
shardSpecs,
|
||||||
shardSpecs,
|
new Function<ShardSpec, Task>()
|
||||||
new Function<ShardSpec, Task>()
|
{
|
||||||
{
|
@Override
|
||||||
@Override
|
public Task apply(ShardSpec shardSpec)
|
||||||
public Task apply(ShardSpec shardSpec)
|
{
|
||||||
{
|
return new IndexGeneratorTask(
|
||||||
return new IndexGeneratorTask(
|
getGroupId(),
|
||||||
getGroupId(),
|
getFixedInterval().get(),
|
||||||
getInterval(),
|
firehoseFactory,
|
||||||
firehoseFactory,
|
new Schema(
|
||||||
new Schema(
|
schema.getDataSource(),
|
||||||
schema.getDataSource(),
|
schema.getAggregators(),
|
||||||
schema.getAggregators(),
|
schema.getIndexGranularity(),
|
||||||
schema.getIndexGranularity(),
|
shardSpec
|
||||||
shardSpec
|
)
|
||||||
)
|
);
|
||||||
);
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
)
|
|
||||||
);
|
);
|
||||||
|
|
||||||
|
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks));
|
||||||
|
|
||||||
|
return TaskStatus.success(getId());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -21,27 +21,26 @@ package com.metamx.druid.merger.common.task;
|
|||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
import com.metamx.druid.input.InputRow;
|
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.TaskLock;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
import com.metamx.druid.merger.common.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
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.common.index.YeOldePlumberSchool;
|
||||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
|
||||||
import com.metamx.druid.realtime.FireDepartmentMetrics;
|
import com.metamx.druid.realtime.FireDepartmentMetrics;
|
||||||
import com.metamx.druid.realtime.Firehose;
|
import com.metamx.druid.realtime.Firehose;
|
||||||
import com.metamx.druid.realtime.FirehoseFactory;
|
import com.metamx.druid.realtime.FirehoseFactory;
|
||||||
import com.metamx.druid.realtime.Plumber;
|
import com.metamx.druid.realtime.Plumber;
|
||||||
import com.metamx.druid.realtime.Schema;
|
import com.metamx.druid.realtime.Schema;
|
||||||
import com.metamx.druid.loading.SegmentPusher;
|
|
||||||
import com.metamx.druid.realtime.Sink;
|
import com.metamx.druid.realtime.Sink;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
@ -79,7 +78,7 @@ public class IndexGeneratorTask extends AbstractTask
|
|||||||
),
|
),
|
||||||
groupId,
|
groupId,
|
||||||
schema.getDataSource(),
|
schema.getDataSource(),
|
||||||
interval
|
Preconditions.checkNotNull(interval, "interval")
|
||||||
);
|
);
|
||||||
|
|
||||||
this.firehoseFactory = firehoseFactory;
|
this.firehoseFactory = firehoseFactory;
|
||||||
@ -87,14 +86,20 @@ public class IndexGeneratorTask extends AbstractTask
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return Type.INDEX;
|
return "index_generator";
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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
|
// Set up temporary directory for indexing
|
||||||
final File tmpDir = new File(
|
final File tmpDir = new File(
|
||||||
String.format(
|
String.format(
|
||||||
@ -103,9 +108,9 @@ public class IndexGeneratorTask extends AbstractTask
|
|||||||
String.format(
|
String.format(
|
||||||
"%s_%s_%s_%s_%s",
|
"%s_%s_%s_%s_%s",
|
||||||
this.getDataSource(),
|
this.getDataSource(),
|
||||||
this.getInterval().getStart(),
|
interval.getStart(),
|
||||||
this.getInterval().getEnd(),
|
interval.getEnd(),
|
||||||
context.getVersion(),
|
myLock.getVersion(),
|
||||||
schema.getShardSpec().getPartitionNum()
|
schema.getShardSpec().getPartitionNum()
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
@ -128,8 +133,8 @@ public class IndexGeneratorTask extends AbstractTask
|
|||||||
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
|
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
|
||||||
final Firehose firehose = firehoseFactory.connect();
|
final Firehose firehose = firehoseFactory.connect();
|
||||||
final Plumber plumber = new YeOldePlumberSchool(
|
final Plumber plumber = new YeOldePlumberSchool(
|
||||||
getInterval(),
|
interval,
|
||||||
context.getVersion(),
|
myLock.getVersion(),
|
||||||
wrappedSegmentPusher,
|
wrappedSegmentPusher,
|
||||||
tmpDir
|
tmpDir
|
||||||
).findPlumber(schema, metrics);
|
).findPlumber(schema, metrics);
|
||||||
@ -177,8 +182,11 @@ public class IndexGeneratorTask extends AbstractTask
|
|||||||
metrics.rowOutput()
|
metrics.rowOutput()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
// Request segment pushes
|
||||||
|
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(pushedSegments)));
|
||||||
|
|
||||||
// Done
|
// Done
|
||||||
return TaskStatus.success(getId(), ImmutableSet.copyOf(pushedSegments));
|
return TaskStatus.success(getId());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -187,7 +195,7 @@ public class IndexGeneratorTask extends AbstractTask
|
|||||||
* @return true or false
|
* @return true or false
|
||||||
*/
|
*/
|
||||||
private boolean shouldIndex(InputRow inputRow) {
|
private boolean shouldIndex(InputRow inputRow) {
|
||||||
if(!getInterval().contains(inputRow.getTimestampFromEpoch())) {
|
if(!getFixedInterval().get().contains(inputRow.getTimestampFromEpoch())) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -27,10 +27,9 @@ import com.metamx.common.logger.Logger;
|
|||||||
import com.metamx.druid.QueryGranularity;
|
import com.metamx.druid.QueryGranularity;
|
||||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
import com.metamx.druid.indexer.granularity.GranularitySpec;
|
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.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
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.FirehoseFactory;
|
||||||
import com.metamx.druid.realtime.Schema;
|
import com.metamx.druid.realtime.Schema;
|
||||||
import com.metamx.druid.shard.NoneShardSpec;
|
import com.metamx.druid.shard.NoneShardSpec;
|
||||||
@ -121,19 +120,20 @@ public class IndexTask extends AbstractTask
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return Type.INDEX;
|
return "index";
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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
|
@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!");
|
throw new IllegalStateException("IndexTasks should not be run!");
|
||||||
}
|
}
|
||||||
|
@ -1,15 +1,23 @@
|
|||||||
package com.metamx.druid.merger.common.task;
|
package com.metamx.druid.merger.common.task;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.google.common.collect.Iterables;
|
||||||
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.merger.common.TaskCallback;
|
import com.metamx.druid.client.DataSegment;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
import com.metamx.druid.merger.common.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
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 com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class KillTask extends AbstractTask
|
public class KillTask extends AbstractTask
|
||||||
@ -36,18 +44,55 @@ public class KillTask extends AbstractTask
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return Task.Type.KILL;
|
return "kill";
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception
|
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||||
{
|
{
|
||||||
// Kill segments
|
// Confirm we have a lock (will throw if there isn't exactly one element)
|
||||||
toolbox.getSegmentKiller()
|
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
|
||||||
.kill(context.getUnusedSegments());
|
|
||||||
|
|
||||||
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());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -38,10 +38,12 @@ import com.metamx.common.ISE;
|
|||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
import com.metamx.druid.loading.SegmentPuller;
|
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.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
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.druid.shard.NoneShardSpec;
|
||||||
import com.metamx.emitter.service.AlertEvent;
|
import com.metamx.emitter.service.AlertEvent;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
@ -119,11 +121,12 @@ public abstract class MergeTask extends AbstractTask
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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 ServiceEmitter emitter = toolbox.getEmitter();
|
||||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
|
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);
|
final File taskDir = toolbox.getConfig().getTaskDir(this);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
@ -177,7 +180,9 @@ public abstract class MergeTask extends AbstractTask
|
|||||||
emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart));
|
emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart));
|
||||||
emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize()));
|
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) {
|
catch (Exception e) {
|
||||||
log.error(
|
log.error(
|
||||||
@ -206,7 +211,7 @@ public abstract class MergeTask extends AbstractTask
|
|||||||
* we are operating on every segment that overlaps the chosen interval.
|
* we are operating on every segment that overlaps the chosen interval.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public TaskStatus preflight(TaskContext context)
|
public TaskStatus preflight(TaskToolbox toolbox)
|
||||||
{
|
{
|
||||||
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
|
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
|
||||||
{
|
{
|
||||||
@ -217,7 +222,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> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
|
||||||
|
|
||||||
final Set<String> missingFromRequested = Sets.difference(current, requested);
|
final Set<String> missingFromRequested = Sets.difference(current, requested);
|
||||||
@ -255,7 +266,7 @@ public abstract class MergeTask extends AbstractTask
|
|||||||
return Objects.toStringHelper(this)
|
return Objects.toStringHelper(this)
|
||||||
.add("id", getId())
|
.add("id", getId())
|
||||||
.add("dataSource", getDataSource())
|
.add("dataSource", getDataSource())
|
||||||
.add("interval", getInterval())
|
.add("interval", getFixedInterval())
|
||||||
.add("segments", segments)
|
.add("segments", segments)
|
||||||
.toString();
|
.toString();
|
||||||
}
|
}
|
||||||
@ -281,7 +292,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)
|
private static Interval computeMergedInterval(final List<DataSegment> segments)
|
||||||
|
@ -21,10 +21,9 @@ package com.metamx.druid.merger.common.task;
|
|||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
import com.metamx.druid.merger.common.TaskCallback;
|
import com.google.common.base.Optional;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
import com.metamx.druid.merger.common.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
import com.metamx.druid.merger.common.TaskToolbox;
|
||||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -33,6 +32,7 @@ import org.joda.time.Interval;
|
|||||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class)
|
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class)
|
||||||
@JsonSubTypes(value = {
|
@JsonSubTypes(value = {
|
||||||
@JsonSubTypes.Type(name = "append", value = AppendTask.class),
|
@JsonSubTypes.Type(name = "append", value = AppendTask.class),
|
||||||
|
@JsonSubTypes.Type(name = "merge", value = DefaultMergeTask.class),
|
||||||
@JsonSubTypes.Type(name = "delete", value = DeleteTask.class),
|
@JsonSubTypes.Type(name = "delete", value = DeleteTask.class),
|
||||||
@JsonSubTypes.Type(name = "kill", value = KillTask.class),
|
@JsonSubTypes.Type(name = "kill", value = KillTask.class),
|
||||||
@JsonSubTypes.Type(name = "index", value = IndexTask.class),
|
@JsonSubTypes.Type(name = "index", value = IndexTask.class),
|
||||||
@ -41,49 +41,57 @@ import org.joda.time.Interval;
|
|||||||
})
|
})
|
||||||
public interface Task
|
public interface Task
|
||||||
{
|
{
|
||||||
enum Type
|
/**
|
||||||
{
|
* Returns ID of this task. Must be unique across all tasks ever created.
|
||||||
INDEX,
|
*/
|
||||||
MERGE,
|
|
||||||
APPEND,
|
|
||||||
DELETE,
|
|
||||||
TEST,
|
|
||||||
KILL
|
|
||||||
}
|
|
||||||
|
|
||||||
public String getId();
|
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 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 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
|
* Execute preflight checks for a task. This typically runs on the coordinator, and will be run while
|
||||||
* holding a lock on our dataSource and interval. If this method throws an exception, the task should be
|
* holding a lock on our dataSource and interval. If this method throws an exception, the task should be
|
||||||
* considered a failure.
|
* 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
|
* @return Some kind of status (runnable means continue on to a worker, non-runnable means we completed without
|
||||||
* using a worker).
|
* using a worker).
|
||||||
|
*
|
||||||
* @throws Exception
|
* @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
|
* 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
|
* holding a lock on our dataSource and interval. If this method throws an exception, the task should be
|
||||||
* considered a failure.
|
* considered a failure.
|
||||||
*
|
*
|
||||||
* @param context Context for this task, gathered under indexer lock
|
|
||||||
* @param toolbox Toolbox for this task
|
* @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).
|
* @return Some kind of finished status (isRunnable must be false).
|
||||||
|
*
|
||||||
* @throws Exception
|
* @throws Exception
|
||||||
*/
|
*/
|
||||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception;
|
public TaskStatus run(TaskToolbox toolbox) throws Exception;
|
||||||
}
|
}
|
||||||
|
@ -1,11 +1,8 @@
|
|||||||
package com.metamx.druid.merger.common.task;
|
package com.metamx.druid.merger.common.task;
|
||||||
|
|
||||||
import com.metamx.druid.merger.common.TaskCallback;
|
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
import com.metamx.druid.merger.common.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
import com.metamx.druid.merger.common.TaskToolbox;
|
||||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
@ -26,15 +23,13 @@ public class V8toV9UpgradeTask extends AbstractTask
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException("Do we really need to return a Type?");
|
return "8to9";
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TaskStatus run(
|
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||||
TaskContext context, TaskToolbox toolbox, TaskCallback callback
|
|
||||||
) throws Exception
|
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
@ -24,10 +24,15 @@ import com.google.common.base.Function;
|
|||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
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.common.logger.Logger;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
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.common.task.Task;
|
||||||
import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig;
|
import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig;
|
||||||
|
|
||||||
@ -37,6 +42,7 @@ import org.skife.jdbi.v2.Handle;
|
|||||||
import org.skife.jdbi.v2.exceptions.StatementException;
|
import org.skife.jdbi.v2.exceptions.StatementException;
|
||||||
import org.skife.jdbi.v2.tweak.HandleCallback;
|
import org.skife.jdbi.v2.tweak.HandleCallback;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
@ -104,18 +110,11 @@ public class DbTaskStorage implements TaskStorage
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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.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(
|
int updated = dbi.withHandle(
|
||||||
new HandleCallback<Integer>()
|
new HandleCallback<Integer>()
|
||||||
@ -125,12 +124,13 @@ public class DbTaskStorage implements TaskStorage
|
|||||||
{
|
{
|
||||||
return handle.createStatement(
|
return handle.createStatement(
|
||||||
String.format(
|
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()
|
dbConnectorConfig.getTaskTable()
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.bind("id", taskid)
|
.bind("id", status.getId())
|
||||||
.bind("status_code", status.getStatusCode().toString())
|
.bind("status_code", status.getStatusCode().toString())
|
||||||
|
.bind("old_status_code", TaskStatus.Status.RUNNING.toString())
|
||||||
.bind("status_payload", jsonMapper.writeValueAsString(status))
|
.bind("status_payload", jsonMapper.writeValueAsString(status))
|
||||||
.execute();
|
.execute();
|
||||||
}
|
}
|
||||||
@ -138,40 +138,10 @@ public class DbTaskStorage implements TaskStorage
|
|||||||
);
|
);
|
||||||
|
|
||||||
if(updated != 1) {
|
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
|
@Override
|
||||||
public Optional<Task> getTask(final String taskid)
|
public Optional<Task> getTask(final String taskid)
|
||||||
{
|
{
|
||||||
@ -232,36 +202,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
|
@Override
|
||||||
public List<Task> getRunningTasks()
|
public List<Task> getRunningTasks()
|
||||||
{
|
{
|
||||||
@ -299,4 +239,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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -59,7 +59,7 @@ public class LocalTaskRunner implements TaskRunner
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void run(final Task task, final TaskContext context, final TaskCallback callback)
|
public void run(final Task task, final TaskCallback callback)
|
||||||
{
|
{
|
||||||
exec.submit(
|
exec.submit(
|
||||||
new Runnable()
|
new Runnable()
|
||||||
@ -73,7 +73,7 @@ public class LocalTaskRunner implements TaskRunner
|
|||||||
|
|
||||||
try {
|
try {
|
||||||
log.info("Running task: %s", task.getId());
|
log.info("Running task: %s", task.getId());
|
||||||
status = task.run(context, toolbox, callback);
|
status = task.run(toolbox);
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
log.error(e, "Interrupted while running task[%s]", task);
|
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.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
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.ImmutableList;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
|
import com.google.common.collect.Multimap;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
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.common.task.Task;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
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
|
* 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
|
public class LocalTaskStorage implements TaskStorage
|
||||||
{
|
{
|
||||||
|
private final ReentrantLock giant = new ReentrantLock();
|
||||||
private final Map<String, TaskStuff> tasks = Maps.newHashMap();
|
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);
|
private static final Logger log = new Logger(LocalTaskStorage.class);
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void insert(Task task, TaskStatus status)
|
public void insert(Task task, TaskStatus status)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(task, "task");
|
giant.lock();
|
||||||
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())) {
|
try {
|
||||||
throw new TaskExistsException(task.getId());
|
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
|
@Override
|
||||||
public Optional<Task> getTask(String taskid)
|
public Optional<Task> getTask(String taskid)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(taskid, "taskid");
|
giant.lock();
|
||||||
if(tasks.containsKey(taskid)) {
|
|
||||||
return Optional.of(tasks.get(taskid).getTask());
|
try {
|
||||||
} else {
|
Preconditions.checkNotNull(taskid, "taskid");
|
||||||
return Optional.absent();
|
if(tasks.containsKey(taskid)) {
|
||||||
|
return Optional.of(tasks.get(taskid).getTask());
|
||||||
|
} else {
|
||||||
|
return Optional.absent();
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
giant.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setStatus(String taskid, TaskStatus status)
|
public void setStatus(TaskStatus status)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(taskid, "taskid");
|
giant.lock();
|
||||||
Preconditions.checkNotNull(status, "status");
|
|
||||||
Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid);
|
try {
|
||||||
log.info("Updating task %s to status: %s", taskid, status);
|
Preconditions.checkNotNull(status, "status");
|
||||||
tasks.put(taskid, tasks.get(taskid).withStatus(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
|
@Override
|
||||||
public Optional<TaskStatus> getStatus(String taskid)
|
public Optional<TaskStatus> getStatus(String taskid)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(taskid, "taskid");
|
giant.lock();
|
||||||
if(tasks.containsKey(taskid)) {
|
|
||||||
return Optional.of(tasks.get(taskid).getStatus());
|
|
||||||
} else {
|
|
||||||
return Optional.absent();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
try {
|
||||||
public void setVersion(String taskid, String version)
|
Preconditions.checkNotNull(taskid, "taskid");
|
||||||
{
|
if(tasks.containsKey(taskid)) {
|
||||||
Preconditions.checkNotNull(taskid, "taskid");
|
return Optional.of(tasks.get(taskid).getStatus());
|
||||||
Preconditions.checkNotNull(version, "status");
|
} else {
|
||||||
Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid);
|
return Optional.absent();
|
||||||
log.info("Updating task %s to version: %s", taskid, version);
|
}
|
||||||
tasks.put(taskid, tasks.get(taskid).withVersion(version));
|
} finally {
|
||||||
}
|
giant.unlock();
|
||||||
|
|
||||||
@Override
|
|
||||||
public Optional<String> getVersion(String taskid)
|
|
||||||
{
|
|
||||||
Preconditions.checkNotNull(taskid, "taskid");
|
|
||||||
if(tasks.containsKey(taskid)) {
|
|
||||||
return tasks.get(taskid).getVersion();
|
|
||||||
} else {
|
|
||||||
return Optional.absent();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<Task> getRunningTasks()
|
public List<Task> getRunningTasks()
|
||||||
{
|
{
|
||||||
final ImmutableList.Builder<Task> listBuilder = ImmutableList.builder();
|
giant.lock();
|
||||||
for(final TaskStuff taskStuff : tasks.values()) {
|
|
||||||
if(taskStuff.getStatus().isRunnable()) {
|
|
||||||
listBuilder.add(taskStuff.getTask());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
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
|
private static class TaskStuff
|
||||||
{
|
{
|
||||||
final Task task;
|
final Task task;
|
||||||
final TaskStatus status;
|
final TaskStatus status;
|
||||||
final Optional<String> version;
|
|
||||||
|
|
||||||
private TaskStuff(Task task, TaskStatus status)
|
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(task);
|
||||||
Preconditions.checkNotNull(status);
|
Preconditions.checkNotNull(status);
|
||||||
@ -145,7 +221,6 @@ public class LocalTaskStorage implements TaskStorage
|
|||||||
|
|
||||||
this.task = task;
|
this.task = task;
|
||||||
this.status = status;
|
this.status = status;
|
||||||
this.version = version;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public Task getTask()
|
public Task getTask()
|
||||||
@ -158,19 +233,9 @@ public class LocalTaskStorage implements TaskStorage
|
|||||||
return status;
|
return status;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Optional<String> getVersion()
|
|
||||||
{
|
|
||||||
return version;
|
|
||||||
}
|
|
||||||
|
|
||||||
private TaskStuff withStatus(TaskStatus _status)
|
private TaskStuff withStatus(TaskStatus _status)
|
||||||
{
|
{
|
||||||
return new TaskStuff(task, _status, version);
|
return new TaskStuff(task, _status);
|
||||||
}
|
|
||||||
|
|
||||||
private TaskStuff withVersion(String _version)
|
|
||||||
{
|
|
||||||
return new TaskStuff(task, status, Optional.of(_version));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -29,7 +29,6 @@ import com.metamx.druid.TimelineObjectHolder;
|
|||||||
import com.metamx.druid.VersionedIntervalTimeline;
|
import com.metamx.druid.VersionedIntervalTimeline;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
import com.metamx.druid.db.DbConnectorConfig;
|
import com.metamx.druid.db.DbConnectorConfig;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.skife.jdbi.v2.DBI;
|
import org.skife.jdbi.v2.DBI;
|
||||||
@ -46,6 +45,7 @@ import java.io.IOException;
|
|||||||
import java.sql.SQLException;
|
import java.sql.SQLException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
@ -129,52 +129,25 @@ public class MergerDBCoordinator
|
|||||||
return segments;
|
return segments;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void commitTaskStatus(final TaskStatus taskStatus)
|
public void announceHistoricalSegments(final Set<DataSegment> segments) throws Exception
|
||||||
{
|
{
|
||||||
try {
|
dbi.inTransaction(
|
||||||
dbi.inTransaction(
|
new TransactionCallback<Void>()
|
||||||
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>()
|
|
||||||
{
|
{
|
||||||
@Override
|
@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;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private void announceHistoricalSegment(final Handle handle, final DataSegment segment) throws Exception
|
private void announceHistoricalSegment(final Handle handle, final DataSegment segment) throws Exception
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
@ -218,15 +191,18 @@ public class MergerDBCoordinator
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void deleteSegment(final DataSegment segment)
|
public void deleteSegments(final Set<DataSegment> segments) throws Exception
|
||||||
{
|
{
|
||||||
dbi.withHandle(
|
dbi.inTransaction(
|
||||||
new HandleCallback<Void>()
|
new TransactionCallback<Void>()
|
||||||
{
|
{
|
||||||
@Override
|
@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;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -32,7 +32,6 @@ import com.metamx.common.guava.FunctionalIterable;
|
|||||||
import com.metamx.common.lifecycle.LifecycleStart;
|
import com.metamx.common.lifecycle.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.druid.merger.common.TaskCallback;
|
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.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.task.Task;
|
import com.metamx.druid.merger.common.task.Task;
|
||||||
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
|
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
|
||||||
@ -213,17 +212,16 @@ public class RemoteTaskRunner implements TaskRunner
|
|||||||
* A task will be run only if there is no current knowledge in the RemoteTaskRunner of the task.
|
* A task will be run only if there is no current knowledge in the RemoteTaskRunner of the task.
|
||||||
*
|
*
|
||||||
* @param task task to run
|
* @param task task to run
|
||||||
* @param context task context to run under
|
|
||||||
* @param callback callback to be called exactly once
|
* @param callback callback to be called exactly once
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void run(Task task, TaskContext context, TaskCallback callback)
|
public void run(Task task, TaskCallback callback)
|
||||||
{
|
{
|
||||||
if (runningTasks.containsKey(task.getId()) || pendingTasks.containsKey(task.getId())) {
|
if (runningTasks.containsKey(task.getId()) || pendingTasks.containsKey(task.getId())) {
|
||||||
throw new ISE("Assigned a task[%s] that is already running or pending, WTF is happening?!", task.getId());
|
throw new ISE("Assigned a task[%s] that is already running or pending, WTF is happening?!", task.getId());
|
||||||
}
|
}
|
||||||
TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem(
|
TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem(
|
||||||
task, context, callback, retryPolicyFactory.makeRetryPolicy(), new DateTime()
|
task, callback, retryPolicyFactory.makeRetryPolicy(), new DateTime()
|
||||||
);
|
);
|
||||||
addPendingTask(taskRunnerWorkItem);
|
addPendingTask(taskRunnerWorkItem);
|
||||||
}
|
}
|
||||||
@ -360,11 +358,10 @@ public class RemoteTaskRunner implements TaskRunner
|
|||||||
private void announceTask(Worker theWorker, TaskRunnerWorkItem taskRunnerWorkItem) throws Exception
|
private void announceTask(Worker theWorker, TaskRunnerWorkItem taskRunnerWorkItem) throws Exception
|
||||||
{
|
{
|
||||||
final Task task = taskRunnerWorkItem.getTask();
|
final Task task = taskRunnerWorkItem.getTask();
|
||||||
final TaskContext taskContext = taskRunnerWorkItem.getTaskContext();
|
|
||||||
|
|
||||||
log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId());
|
log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId());
|
||||||
|
|
||||||
byte[] rawBytes = jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext));
|
byte[] rawBytes = jsonMapper.writeValueAsBytes(task);
|
||||||
if (rawBytes.length > config.getMaxNumBytes()) {
|
if (rawBytes.length > config.getMaxNumBytes()) {
|
||||||
throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes());
|
throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes());
|
||||||
}
|
}
|
||||||
@ -377,7 +374,7 @@ public class RemoteTaskRunner implements TaskRunner
|
|||||||
theWorker.getHost(),
|
theWorker.getHost(),
|
||||||
task.getId()
|
task.getId()
|
||||||
),
|
),
|
||||||
jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext))
|
rawBytes
|
||||||
);
|
);
|
||||||
|
|
||||||
runningTasks.put(task.getId(), pendingTasks.remove(task.getId()));
|
runningTasks.put(task.getId(), pendingTasks.remove(task.getId()));
|
||||||
|
@ -1,68 +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.fasterxml.jackson.annotation.JsonCreator;
|
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
|
||||||
import com.metamx.druid.client.DataSegment;
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
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.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.druid.initialization.Initialization;
|
import com.metamx.druid.initialization.Initialization;
|
||||||
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
|
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.config.IndexerCoordinatorConfig;
|
||||||
import com.metamx.druid.merger.coordinator.exec.TaskConsumer;
|
import com.metamx.druid.merger.coordinator.exec.TaskConsumer;
|
||||||
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler;
|
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler;
|
||||||
@ -42,28 +43,33 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||||||
/**
|
/**
|
||||||
* Encapsulates the indexer leadership lifecycle.
|
* Encapsulates the indexer leadership lifecycle.
|
||||||
*/
|
*/
|
||||||
public class TaskMaster
|
public class TaskMasterLifecycle
|
||||||
{
|
{
|
||||||
private final LeaderSelector leaderSelector;
|
private final LeaderSelector leaderSelector;
|
||||||
private final ReentrantLock giant = new ReentrantLock();
|
private final ReentrantLock giant = new ReentrantLock();
|
||||||
private final Condition mayBeStopped = giant.newCondition();
|
private final Condition mayBeStopped = giant.newCondition();
|
||||||
|
private final TaskQueue taskQueue;
|
||||||
|
private final TaskToolbox taskToolbox;
|
||||||
|
|
||||||
private volatile boolean leading = false;
|
private volatile boolean leading = false;
|
||||||
private volatile TaskRunner taskRunner;
|
private volatile TaskRunner theRunner;
|
||||||
|
|
||||||
private static final EmittingLogger log = new EmittingLogger(TaskMaster.class);
|
private static final EmittingLogger log = new EmittingLogger(TaskMasterLifecycle.class);
|
||||||
|
|
||||||
public TaskMaster(
|
public TaskMasterLifecycle(
|
||||||
final TaskQueue queue,
|
final TaskQueue taskQueue,
|
||||||
|
final TaskToolbox taskToolbox,
|
||||||
final IndexerCoordinatorConfig indexerCoordinatorConfig,
|
final IndexerCoordinatorConfig indexerCoordinatorConfig,
|
||||||
final ServiceDiscoveryConfig serviceDiscoveryConfig,
|
final ServiceDiscoveryConfig serviceDiscoveryConfig,
|
||||||
final MergerDBCoordinator mergerDBCoordinator,
|
|
||||||
final TaskRunnerFactory runnerFactory,
|
final TaskRunnerFactory runnerFactory,
|
||||||
final ResourceManagementSchedulerFactory managementSchedulerFactory,
|
final ResourceManagementSchedulerFactory managementSchedulerFactory,
|
||||||
final CuratorFramework curator,
|
final CuratorFramework curator,
|
||||||
final ServiceEmitter emitter
|
final ServiceEmitter emitter
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
this.taskQueue = taskQueue;
|
||||||
|
this.taskToolbox = taskToolbox;
|
||||||
|
|
||||||
this.leaderSelector = new LeaderSelector(
|
this.leaderSelector = new LeaderSelector(
|
||||||
curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener()
|
curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener()
|
||||||
{
|
{
|
||||||
@ -75,34 +81,45 @@ public class TaskMaster
|
|||||||
try {
|
try {
|
||||||
log.info("By the power of Grayskull, I have the power!");
|
log.info("By the power of Grayskull, I have the power!");
|
||||||
|
|
||||||
final TaskRunner runner = runnerFactory.build();
|
final TaskRunner taskRunner = runnerFactory.build();
|
||||||
taskRunner = runner;
|
theRunner = taskRunner;
|
||||||
final ResourceManagementScheduler scheduler = managementSchedulerFactory.build(runner);
|
final ResourceManagementScheduler scheduler = managementSchedulerFactory.build(theRunner);
|
||||||
final TaskConsumer consumer = new TaskConsumer(queue, runner, mergerDBCoordinator, emitter);
|
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:
|
// Sensible order to start stuff:
|
||||||
final Lifecycle leaderLifecycle = new Lifecycle();
|
final Lifecycle leaderLifecycle = new Lifecycle();
|
||||||
leaderLifecycle.addManagedInstance(queue);
|
leaderLifecycle.addManagedInstance(taskQueue);
|
||||||
leaderLifecycle.addManagedInstance(scheduler);
|
leaderLifecycle.addManagedInstance(taskRunner);
|
||||||
leaderLifecycle.addManagedInstance(runner);
|
|
||||||
Initialization.makeServiceDiscoveryClient(curator, serviceDiscoveryConfig, leaderLifecycle);
|
Initialization.makeServiceDiscoveryClient(curator, serviceDiscoveryConfig, leaderLifecycle);
|
||||||
leaderLifecycle.addManagedInstance(consumer);
|
leaderLifecycle.addManagedInstance(taskConsumer);
|
||||||
leaderLifecycle.start();
|
|
||||||
|
|
||||||
leading = true;
|
leading = true;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
leaderLifecycle.start();
|
||||||
|
|
||||||
while (leading) {
|
while (leading) {
|
||||||
mayBeStopped.await();
|
mayBeStopped.await();
|
||||||
}
|
}
|
||||||
} finally {
|
}
|
||||||
|
finally {
|
||||||
log.info("Bowing out!");
|
log.info("Bowing out!");
|
||||||
leaderLifecycle.stop();
|
leaderLifecycle.stop();
|
||||||
}
|
}
|
||||||
} catch(Exception e) {
|
}
|
||||||
|
catch (Exception e) {
|
||||||
log.makeAlert(e, "Failed to lead").emit();
|
log.makeAlert(e, "Failed to lead").emit();
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
} finally {
|
}
|
||||||
|
finally {
|
||||||
giant.unlock();
|
giant.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -191,6 +208,16 @@ public class TaskMaster
|
|||||||
|
|
||||||
public TaskRunner getTaskRunner()
|
public TaskRunner getTaskRunner()
|
||||||
{
|
{
|
||||||
return taskRunner;
|
return theRunner;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TaskQueue getTaskQueue()
|
||||||
|
{
|
||||||
|
return taskQueue;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TaskToolbox getTaskToolbox()
|
||||||
|
{
|
||||||
|
return taskToolbox;
|
||||||
}
|
}
|
||||||
}
|
}
|
@ -19,37 +19,22 @@
|
|||||||
|
|
||||||
package com.metamx.druid.merger.coordinator;
|
package com.metamx.druid.merger.coordinator;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Predicate;
|
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.HashMultimap;
|
import com.google.common.collect.ArrayListMultimap;
|
||||||
import com.google.common.collect.Iterables;
|
|
||||||
import com.google.common.collect.Iterators;
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
|
||||||
import com.google.common.collect.Multimap;
|
import com.google.common.collect.Multimap;
|
||||||
import com.google.common.collect.Ordering;
|
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.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
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.druid.merger.common.task.Task;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import org.joda.time.DateTime;
|
|
||||||
import org.joda.time.Interval;
|
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
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.Condition;
|
||||||
import java.util.concurrent.locks.ReentrantLock;
|
import java.util.concurrent.locks.ReentrantLock;
|
||||||
|
|
||||||
@ -60,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
|
* {@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.
|
* with a currently-running task. In that case, tasks are skipped until a runnable one is found.
|
||||||
* <p/>
|
* <p/>
|
||||||
* To manage locking, the queue keeps track of currently-running tasks as {@link TaskGroup} objects. The idea is that
|
* 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 TaskGroup can be running on a particular dataSource + interval, and that TaskGroup has a single version
|
* 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 TaskGroup may run concurrently.
|
* string that all tasks in the group must use to publish segments. Tasks in the same TaskLock may run concurrently.
|
||||||
* <p/>
|
* <p/>
|
||||||
* For persistence, the queue saves new tasks from {@link #add} and task status updates from {@link #notify} using a
|
* 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/>
|
* <p/>
|
||||||
* To support leader election of our containing system, the queue can be stopped (in which case it will not accept
|
* 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).
|
* any new tasks, or hand out any more tasks, until started again).
|
||||||
@ -73,10 +58,8 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||||||
public class TaskQueue
|
public class TaskQueue
|
||||||
{
|
{
|
||||||
private final List<Task> queue = Lists.newLinkedList();
|
private final List<Task> queue = Lists.newLinkedList();
|
||||||
private final Map<String, NavigableMap<Interval, TaskGroup>> running = Maps.newHashMap();
|
|
||||||
|
|
||||||
private final TaskStorage taskStorage;
|
private final TaskStorage taskStorage;
|
||||||
|
private final TaskLockbox taskLockbox;
|
||||||
private final ReentrantLock giant = new ReentrantLock();
|
private final ReentrantLock giant = new ReentrantLock();
|
||||||
private final Condition workMayBeAvailable = giant.newCondition();
|
private final Condition workMayBeAvailable = giant.newCondition();
|
||||||
|
|
||||||
@ -84,14 +67,98 @@ public class TaskQueue
|
|||||||
|
|
||||||
private static final EmittingLogger log = new EmittingLogger(TaskQueue.class);
|
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.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
|
* Bootstraps this task queue and associated task lockbox. Clears the lockbox before running. Should be called
|
||||||
* new tasks. This should not be called on an already-started queue.
|
* 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
|
@LifecycleStart
|
||||||
public void start()
|
public void start()
|
||||||
@ -99,50 +166,9 @@ public class TaskQueue
|
|||||||
giant.lock();
|
giant.lock();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
Preconditions.checkState(!active, "queue must be stopped");
|
||||||
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());
|
|
||||||
|
|
||||||
active = true;
|
active = true;
|
||||||
|
|
||||||
workMayBeAvailable.signalAll();
|
workMayBeAvailable.signalAll();
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
@ -160,13 +186,10 @@ public class TaskQueue
|
|||||||
giant.lock();
|
giant.lock();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
|
||||||
log.info("Naptime! Shutting down until we are started again.");
|
log.info("Naptime! Shutting down until we are started again.");
|
||||||
|
|
||||||
queue.clear();
|
queue.clear();
|
||||||
running.clear();
|
taskLockbox.clear();
|
||||||
active = false;
|
active = false;
|
||||||
|
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
giant.unlock();
|
giant.unlock();
|
||||||
@ -202,7 +225,9 @@ public class TaskQueue
|
|||||||
// Attempt to add this task to a running task group. Silently continue if this is not possible.
|
// 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
|
// 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.
|
// as their parent whenever possible.
|
||||||
tryLock(task);
|
if(task.getFixedInterval().isPresent()) {
|
||||||
|
taskLockbox.tryLock(task, task.getFixedInterval().get());
|
||||||
|
}
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -216,19 +241,22 @@ public class TaskQueue
|
|||||||
*
|
*
|
||||||
* @return runnable task
|
* @return runnable task
|
||||||
*/
|
*/
|
||||||
public VersionedTaskWrapper take() throws InterruptedException
|
public Task take() throws InterruptedException
|
||||||
{
|
{
|
||||||
giant.lock();
|
giant.lock();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
VersionedTaskWrapper taskWrapper;
|
Task task;
|
||||||
|
|
||||||
while ((taskWrapper = poll()) == null) {
|
log.info("Waiting for work...");
|
||||||
log.info("Waiting for work...");
|
|
||||||
workMayBeAvailable.await();
|
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 {
|
finally {
|
||||||
giant.unlock();
|
giant.unlock();
|
||||||
@ -240,24 +268,28 @@ public class TaskQueue
|
|||||||
*
|
*
|
||||||
* @return runnable task or null
|
* @return runnable task or null
|
||||||
*/
|
*/
|
||||||
public VersionedTaskWrapper poll()
|
public Task poll()
|
||||||
{
|
{
|
||||||
giant.lock();
|
giant.lock();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
log.info("Checking for doable work");
|
|
||||||
for (final Task task : queue) {
|
for (final Task task : queue) {
|
||||||
final Optional<String> maybeVersion = tryLock(task);
|
if(task.getFixedInterval().isPresent()) {
|
||||||
if (maybeVersion.isPresent()) {
|
// If this task has a fixed interval, attempt to lock it right now.
|
||||||
Preconditions.checkState(active, "wtf? Found task when inactive");
|
final Optional<TaskLock> maybeLock = taskLockbox.tryLock(task, task.getFixedInterval().get());
|
||||||
taskStorage.setVersion(task.getId(), maybeVersion.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);
|
queue.remove(task);
|
||||||
log.info("Task claimed: %s", task);
|
return task;
|
||||||
return new VersionedTaskWrapper(task, maybeVersion.get());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
log.info("No doable work found.");
|
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
@ -265,354 +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
|
* 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
|
* the task storage facility. If the status is a completed status, the task will be unlocked and no further
|
||||||
* status, the task will be unlocked and no further updates will be accepted. If this task has failed, the task group
|
* updates will be accepted.
|
||||||
* it is part of will be terminated.
|
|
||||||
* <p/>
|
|
||||||
* Finally, if this task is not supposed to be running, this method will simply do nothing.
|
|
||||||
*
|
*
|
||||||
* @param task task to update
|
* @param task task to update
|
||||||
* @param originalStatus new task status
|
* @param taskStatus new task status
|
||||||
* @param commitRunnable operation to perform if this task is ready to commit
|
|
||||||
*
|
*
|
||||||
* @throws NullPointerException if task or status is null
|
* @throws NullPointerException if task or status is null
|
||||||
* @throws IllegalArgumentException if the task ID does not match the status ID
|
* @throws IllegalArgumentException if the task ID does not match the status ID
|
||||||
* @throws IllegalStateException if this queue is currently shut down
|
* @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();
|
giant.lock();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
Preconditions.checkNotNull(task, "task");
|
Preconditions.checkNotNull(task, "task");
|
||||||
Preconditions.checkNotNull(originalStatus, "status");
|
Preconditions.checkNotNull(taskStatus, "status");
|
||||||
Preconditions.checkState(active, "Queue is not active!");
|
Preconditions.checkState(active, "Queue is not active!");
|
||||||
Preconditions.checkArgument(
|
Preconditions.checkArgument(
|
||||||
task.getId().equals(originalStatus.getId()),
|
task.getId().equals(taskStatus.getId()),
|
||||||
"Mismatching task ids[%s/%s]",
|
"Mismatching task ids[%s/%s]",
|
||||||
task.getId(),
|
task.getId(),
|
||||||
originalStatus.getId()
|
taskStatus.getId()
|
||||||
);
|
);
|
||||||
|
|
||||||
final TaskGroup taskGroup;
|
// Save status to DB
|
||||||
|
boolean didPersistStatus = false;
|
||||||
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;
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
taskStorage.setStatus(task.getId(), statusToSave);
|
final Optional<TaskStatus> previousStatus = taskStorage.getStatus(task.getId());
|
||||||
didSetStatus = true;
|
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) {
|
} catch(Exception e) {
|
||||||
// TODO: This could be a task-status-submission retry queue instead of retrying the entire task,
|
log.makeAlert(e, "Failed to persist status for 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")
|
|
||||||
.addData("task", task.getId())
|
.addData("task", task.getId())
|
||||||
.addData("statusCode", statusToSave.getStatusCode())
|
.addData("statusCode", taskStatus.getStatusCode())
|
||||||
.emit();
|
.emit();
|
||||||
|
|
||||||
queue.add(task);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if(didSetStatus && statusToSave.isComplete()) {
|
if(taskStatus.isComplete()) {
|
||||||
unlock(task);
|
if(didPersistStatus) {
|
||||||
log.info("Task done: %s", task);
|
log.info("Task done: %s", task);
|
||||||
}
|
taskLockbox.unlock(task);
|
||||||
}
|
workMayBeAvailable.signalAll();
|
||||||
finally {
|
|
||||||
giant.unlock();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public Collection<Task> getAvailableTasks()
|
|
||||||
{
|
|
||||||
// TODO: actually implement this
|
|
||||||
return Lists.newArrayList();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* 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;
|
|
||||||
} else {
|
} 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 {
|
finally {
|
||||||
|
@ -25,8 +25,8 @@ import com.metamx.druid.merger.common.task.Task;
|
|||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Interface for handing off tasks. Used by a {@link com.metamx.druid.merger.coordinator.exec.TaskConsumer} to run tasks that
|
* Interface for handing off tasks. Used by a {@link com.metamx.druid.merger.coordinator.exec.TaskConsumer} to
|
||||||
* have been locked.
|
* run tasks that have been locked.
|
||||||
*/
|
*/
|
||||||
public interface TaskRunner
|
public interface TaskRunner
|
||||||
{
|
{
|
||||||
@ -35,10 +35,9 @@ public interface TaskRunner
|
|||||||
* status, but should be called exactly once with a non-RUNNING status (e.g. SUCCESS, FAILED, CONTINUED...).
|
* status, but should be called exactly once with a non-RUNNING status (e.g. SUCCESS, FAILED, CONTINUED...).
|
||||||
*
|
*
|
||||||
* @param task task to run
|
* @param task task to run
|
||||||
* @param context task context to run under
|
|
||||||
* @param callback callback to be called exactly once
|
* @param callback callback to be called exactly once
|
||||||
*/
|
*/
|
||||||
public void run(Task task, TaskContext context, TaskCallback callback);
|
public void run(Task task, TaskCallback callback);
|
||||||
|
|
||||||
public Collection<TaskRunnerWorkItem> getRunningTasks();
|
public Collection<TaskRunnerWorkItem> getRunningTasks();
|
||||||
|
|
||||||
|
@ -30,7 +30,6 @@ import org.joda.time.DateTimeComparator;
|
|||||||
public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
||||||
{
|
{
|
||||||
private final Task task;
|
private final Task task;
|
||||||
private final TaskContext taskContext;
|
|
||||||
private final TaskCallback callback;
|
private final TaskCallback callback;
|
||||||
private final RetryPolicy retryPolicy;
|
private final RetryPolicy retryPolicy;
|
||||||
private final DateTime createdTime;
|
private final DateTime createdTime;
|
||||||
@ -39,14 +38,12 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
|||||||
|
|
||||||
public TaskRunnerWorkItem(
|
public TaskRunnerWorkItem(
|
||||||
Task task,
|
Task task,
|
||||||
TaskContext taskContext,
|
|
||||||
TaskCallback callback,
|
TaskCallback callback,
|
||||||
RetryPolicy retryPolicy,
|
RetryPolicy retryPolicy,
|
||||||
DateTime createdTime
|
DateTime createdTime
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.task = task;
|
this.task = task;
|
||||||
this.taskContext = taskContext;
|
|
||||||
this.callback = callback;
|
this.callback = callback;
|
||||||
this.retryPolicy = retryPolicy;
|
this.retryPolicy = retryPolicy;
|
||||||
this.createdTime = createdTime;
|
this.createdTime = createdTime;
|
||||||
@ -57,11 +54,6 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
|||||||
return task;
|
return task;
|
||||||
}
|
}
|
||||||
|
|
||||||
public TaskContext getTaskContext()
|
|
||||||
{
|
|
||||||
return taskContext;
|
|
||||||
}
|
|
||||||
|
|
||||||
public TaskCallback getCallback()
|
public TaskCallback getCallback()
|
||||||
{
|
{
|
||||||
return callback;
|
return callback;
|
||||||
@ -115,9 +107,6 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
|||||||
if (task != null ? !task.equals(that.task) : that.task != null) {
|
if (task != null ? !task.equals(that.task) : that.task != null) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
if (taskContext != null ? !taskContext.equals(that.taskContext) : that.taskContext != null) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -126,7 +115,6 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
|||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
int result = task != null ? task.hashCode() : 0;
|
int result = task != null ? task.hashCode() : 0;
|
||||||
result = 31 * result + (taskContext != null ? taskContext.hashCode() : 0);
|
|
||||||
result = 31 * result + (callback != null ? callback.hashCode() : 0);
|
result = 31 * result + (callback != null ? callback.hashCode() : 0);
|
||||||
result = 31 * result + (retryPolicy != null ? retryPolicy.hashCode() : 0);
|
result = 31 * result + (retryPolicy != null ? retryPolicy.hashCode() : 0);
|
||||||
return result;
|
return result;
|
||||||
@ -137,7 +125,6 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
|||||||
{
|
{
|
||||||
return "TaskRunnerWorkItem{" +
|
return "TaskRunnerWorkItem{" +
|
||||||
"task=" + task +
|
"task=" + task +
|
||||||
", taskContext=" + taskContext +
|
|
||||||
", callback=" + callback +
|
", callback=" + callback +
|
||||||
", retryPolicy=" + retryPolicy +
|
", retryPolicy=" + retryPolicy +
|
||||||
", createdTime=" + createdTime +
|
", createdTime=" + createdTime +
|
||||||
|
@ -21,6 +21,8 @@ package com.metamx.druid.merger.coordinator;
|
|||||||
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
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.common.task.Task;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@ -34,15 +36,21 @@ public interface TaskStorage
|
|||||||
public void insert(Task task, TaskStatus status);
|
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
|
* Persists lock state in the storage facility.
|
||||||
* an exception.
|
|
||||||
*/
|
*/
|
||||||
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
|
* 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);
|
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
|
* Add an action taken by a task to the audit log.
|
||||||
* but was not yet assigned a version, this will return an absentee Optional.
|
|
||||||
*/
|
*/
|
||||||
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.
|
* Returns a list of currently-running tasks as stored in the storage facility, in no particular order.
|
||||||
*/
|
*/
|
||||||
public List<Task> getRunningTasks();
|
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.Function;
|
||||||
import com.google.common.base.Optional;
|
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.ImmutableMap;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
|
import com.metamx.common.guava.FunctionalIterable;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
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 com.metamx.druid.merger.common.task.Task;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@ -62,8 +68,29 @@ public class TaskStorageQueryAdapter
|
|||||||
|
|
||||||
resultBuilder.put(taskid, statusOptional);
|
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()) {
|
if(taskOptional.isPresent() && statusOptional.isPresent()) {
|
||||||
for(final Task nextTask : statusOptional.get().getNextTasks()) {
|
for(final Task nextTask : nextTasks) {
|
||||||
if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) {
|
if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) {
|
||||||
resultBuilder.putAll(getSameGroupChildStatuses(nextTask.getId()));
|
resultBuilder.putAll(getSameGroupChildStatuses(nextTask.getId()));
|
||||||
}
|
}
|
||||||
@ -84,20 +111,12 @@ public class TaskStorageQueryAdapter
|
|||||||
int nFailures = 0;
|
int nFailures = 0;
|
||||||
int nTotal = 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()) {
|
for(final Optional<TaskStatus> statusOption : statuses.values()) {
|
||||||
nTotal ++;
|
nTotal ++;
|
||||||
|
|
||||||
if(statusOption.isPresent()) {
|
if(statusOption.isPresent()) {
|
||||||
final TaskStatus status = statusOption.get();
|
final TaskStatus status = statusOption.get();
|
||||||
|
|
||||||
segments.addAll(status.getSegments());
|
|
||||||
segmentsNuked.addAll(status.getSegmentsNuked());
|
|
||||||
nextTasks.addAll(status.getNextTasks());
|
|
||||||
|
|
||||||
if(status.isSuccess()) {
|
if(status.isSuccess()) {
|
||||||
nSuccesses ++;
|
nSuccesses ++;
|
||||||
} else if(status.isFailure()) {
|
} else if(status.isFailure()) {
|
||||||
@ -111,10 +130,7 @@ public class TaskStorageQueryAdapter
|
|||||||
if(nTotal == 0) {
|
if(nTotal == 0) {
|
||||||
status = Optional.absent();
|
status = Optional.absent();
|
||||||
} else if(nSuccesses == nTotal) {
|
} else if(nSuccesses == nTotal) {
|
||||||
status = Optional.of(TaskStatus.success(taskid)
|
status = Optional.of(TaskStatus.success(taskid));
|
||||||
.withSegments(segments)
|
|
||||||
.withSegmentsNuked(segmentsNuked)
|
|
||||||
.withNextTasks(nextTasks));
|
|
||||||
} else if(nFailures > 0) {
|
} else if(nFailures > 0) {
|
||||||
status = Optional.of(TaskStatus.failure(taskid));
|
status = Optional.of(TaskStatus.failure(taskid));
|
||||||
} else {
|
} 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
|
* Returns all segments created by descendants for a particular task that stayed within the same task group. Includes
|
||||||
* version field of the returned {@link VersionedTaskWrapper} will be null.
|
* 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(
|
// TODO: This is useful for regular index tasks (so we know what was published), but
|
||||||
storage.getRunningTasks(),
|
// TODO: for long-lived index tasks the list can get out of hand. We may want a limit.
|
||||||
new Function<Task, VersionedTaskWrapper>()
|
|
||||||
{
|
final Optional<Task> taskOptional = storage.getTask(taskid);
|
||||||
@Override
|
final Set<DataSegment> segments = Sets.newHashSet();
|
||||||
public VersionedTaskWrapper apply(Task task)
|
final List<Task> nextTasks = Lists.newArrayList();
|
||||||
{
|
|
||||||
return new VersionedTaskWrapper(task, storage.getVersion(task.getId()).orNull());
|
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;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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();
|
|
||||||
}
|
|
||||||
}
|
|
@ -29,4 +29,12 @@ public abstract class IndexerDbConnectorConfig extends DbConnectorConfig
|
|||||||
@JsonProperty("taskTable")
|
@JsonProperty("taskTable")
|
||||||
@Config("druid.database.taskTable")
|
@Config("druid.database.taskTable")
|
||||||
public abstract String getTaskTable();
|
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;
|
package com.metamx.druid.merger.coordinator.exec;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.ImmutableSet;
|
|
||||||
import com.metamx.common.lifecycle.LifecycleStart;
|
import com.metamx.common.lifecycle.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
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.TaskCallback;
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
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.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.TaskQueue;
|
||||||
import com.metamx.druid.merger.coordinator.TaskRunner;
|
import com.metamx.druid.merger.coordinator.TaskRunner;
|
||||||
import com.metamx.druid.merger.coordinator.VersionedTaskWrapper;
|
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||||
|
|
||||||
import java.util.Set;
|
|
||||||
|
|
||||||
public class TaskConsumer implements Runnable
|
public class TaskConsumer implements Runnable
|
||||||
{
|
{
|
||||||
private final TaskQueue queue;
|
private final TaskQueue queue;
|
||||||
private final TaskRunner runner;
|
private final TaskRunner runner;
|
||||||
private final MergerDBCoordinator mergerDBCoordinator;
|
private final TaskToolbox toolbox;
|
||||||
private final ServiceEmitter emitter;
|
private final ServiceEmitter emitter;
|
||||||
private final Thread thready;
|
private final Thread thready;
|
||||||
|
|
||||||
@ -53,13 +47,13 @@ public class TaskConsumer implements Runnable
|
|||||||
public TaskConsumer(
|
public TaskConsumer(
|
||||||
TaskQueue queue,
|
TaskQueue queue,
|
||||||
TaskRunner runner,
|
TaskRunner runner,
|
||||||
MergerDBCoordinator mergerDBCoordinator,
|
TaskToolbox toolbox,
|
||||||
ServiceEmitter emitter
|
ServiceEmitter emitter
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.queue = queue;
|
this.queue = queue;
|
||||||
this.runner = runner;
|
this.runner = runner;
|
||||||
this.mergerDBCoordinator = mergerDBCoordinator;
|
this.toolbox = toolbox;
|
||||||
this.emitter = emitter;
|
this.emitter = emitter;
|
||||||
this.thready = new Thread(this);
|
this.thready = new Thread(this);
|
||||||
}
|
}
|
||||||
@ -85,12 +79,9 @@ public class TaskConsumer implements Runnable
|
|||||||
while (!Thread.currentThread().isInterrupted()) {
|
while (!Thread.currentThread().isInterrupted()) {
|
||||||
|
|
||||||
final Task task;
|
final Task task;
|
||||||
final String version;
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
final VersionedTaskWrapper taskWrapper = queue.take();
|
task = queue.take();
|
||||||
task = taskWrapper.getTask();
|
|
||||||
version = taskWrapper.getVersion();
|
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
log.info(e, "Interrupted while waiting for new work");
|
log.info(e, "Interrupted while waiting for new work");
|
||||||
@ -98,17 +89,17 @@ public class TaskConsumer implements Runnable
|
|||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
handoff(task, version);
|
handoff(task);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.makeAlert(e, "Failed to hand off task")
|
log.makeAlert(e, "Failed to hand off task")
|
||||||
.addData("task", task.getId())
|
.addData("task", task.getId())
|
||||||
.addData("type", task.getType().toString())
|
.addData("type", task.getType())
|
||||||
.addData("dataSource", task.getDataSource())
|
.addData("dataSource", task.getDataSource())
|
||||||
.addData("interval", task.getInterval())
|
.addData("interval", task.getFixedInterval())
|
||||||
.emit();
|
.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) {
|
if (!shutdown) {
|
||||||
queue.notify(task, TaskStatus.failure(task.getId()));
|
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(
|
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
|
||||||
version,
|
|
||||||
ImmutableSet.copyOf(
|
|
||||||
mergerDBCoordinator.getUsedSegmentsForInterval(
|
|
||||||
task.getDataSource(),
|
|
||||||
task.getInterval()
|
|
||||||
)
|
|
||||||
),
|
|
||||||
ImmutableSet.copyOf(
|
|
||||||
mergerDBCoordinator.getUnusedSegmentsForInterval(
|
|
||||||
task.getDataSource(),
|
|
||||||
task.getInterval()
|
|
||||||
)
|
|
||||||
)
|
|
||||||
);
|
|
||||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder()
|
|
||||||
.setUser2(task.getDataSource())
|
.setUser2(task.getDataSource())
|
||||||
.setUser4(task.getType().toString())
|
.setUser4(task.getType())
|
||||||
.setUser5(task.getInterval().toString());
|
.setUser5(task.getFixedInterval().toString());
|
||||||
|
|
||||||
// Run preflight checks
|
// Run preflight checks
|
||||||
TaskStatus preflightStatus;
|
TaskStatus preflightStatus;
|
||||||
try {
|
try {
|
||||||
preflightStatus = task.preflight(context);
|
preflightStatus = task.preflight(toolbox);
|
||||||
log.info("Preflight done for task: %s", task.getId());
|
log.info("Preflight done for task: %s", task.getId());
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
@ -161,11 +137,9 @@ public class TaskConsumer implements Runnable
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Hand off work to TaskRunner
|
// Hand off work to TaskRunner, with a callback
|
||||||
// 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.
|
|
||||||
runner.run(
|
runner.run(
|
||||||
task, context, new TaskCallback()
|
task, new TaskCallback()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public void notify(final TaskStatus statusFromRunner)
|
public void notify(final TaskStatus statusFromRunner)
|
||||||
@ -180,82 +154,26 @@ public class TaskConsumer implements Runnable
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
queue.notify(
|
queue.notify(task, statusFromRunner);
|
||||||
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);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
|
|
||||||
// Emit event and log, if the task is done
|
// Emit event and log, if the task is done
|
||||||
if (statusFromRunner.isComplete()) {
|
if (statusFromRunner.isComplete()) {
|
||||||
builder.setUser3(statusFromRunner.getStatusCode().toString());
|
metricBuilder.setUser3(statusFromRunner.getStatusCode().toString());
|
||||||
|
emitter.emit(metricBuilder.build("indexer/time/run/millis", statusFromRunner.getDuration()));
|
||||||
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()));
|
|
||||||
|
|
||||||
if (statusFromRunner.isFailure()) {
|
if (statusFromRunner.isFailure()) {
|
||||||
log.makeAlert("Failed to index")
|
log.makeAlert("Failed to index")
|
||||||
.addData("task", task.getId())
|
.addData("task", task.getId())
|
||||||
.addData("type", task.getType().toString())
|
.addData("type", task.getType())
|
||||||
.addData("dataSource", task.getDataSource())
|
.addData("dataSource", task.getDataSource())
|
||||||
.addData("interval", task.getInterval())
|
.addData("interval", task.getFixedInterval())
|
||||||
.emit();
|
.emit();
|
||||||
}
|
}
|
||||||
|
|
||||||
log.info(
|
log.info(
|
||||||
"Task %s: %s (%d segments) (%d run duration)",
|
"Task %s: %s (%d run duration)",
|
||||||
statusFromRunner.getStatusCode(),
|
statusFromRunner.getStatusCode(),
|
||||||
task,
|
task,
|
||||||
statusFromRunner.getSegments().size(),
|
|
||||||
statusFromRunner.getDuration()
|
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()
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -21,7 +21,7 @@ package com.metamx.druid.merger.coordinator.http;
|
|||||||
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.merger.coordinator.TaskMaster;
|
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
|
||||||
|
|
||||||
import javax.ws.rs.GET;
|
import javax.ws.rs.GET;
|
||||||
import javax.ws.rs.Path;
|
import javax.ws.rs.Path;
|
||||||
@ -35,12 +35,12 @@ public class IndexerCoordinatorInfoResource
|
|||||||
{
|
{
|
||||||
private static final Logger log = new Logger(IndexerCoordinatorInfoResource.class);
|
private static final Logger log = new Logger(IndexerCoordinatorInfoResource.class);
|
||||||
|
|
||||||
private final TaskMaster taskMaster;
|
private final TaskMasterLifecycle taskMasterLifecycle;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public IndexerCoordinatorInfoResource(TaskMaster taskMaster)
|
public IndexerCoordinatorInfoResource(TaskMasterLifecycle taskMasterLifecycle)
|
||||||
{
|
{
|
||||||
this.taskMaster = taskMaster;
|
this.taskMasterLifecycle = taskMasterLifecycle;
|
||||||
}
|
}
|
||||||
|
|
||||||
@GET
|
@GET
|
||||||
@ -48,10 +48,10 @@ public class IndexerCoordinatorInfoResource
|
|||||||
@Produces("application/json")
|
@Produces("application/json")
|
||||||
public Response getPendingTasks()
|
public Response getPendingTasks()
|
||||||
{
|
{
|
||||||
if (taskMaster.getTaskRunner() == null) {
|
if (taskMasterLifecycle.getTaskRunner() == null) {
|
||||||
return Response.noContent().build();
|
return Response.noContent().build();
|
||||||
}
|
}
|
||||||
return Response.ok(taskMaster.getTaskRunner().getPendingTasks()).build();
|
return Response.ok(taskMasterLifecycle.getTaskRunner().getPendingTasks()).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@GET
|
@GET
|
||||||
@ -59,9 +59,9 @@ public class IndexerCoordinatorInfoResource
|
|||||||
@Produces("application/json")
|
@Produces("application/json")
|
||||||
public Response getRunningTasks()
|
public Response getRunningTasks()
|
||||||
{
|
{
|
||||||
if (taskMaster.getTaskRunner() == null) {
|
if (taskMasterLifecycle.getTaskRunner() == null) {
|
||||||
return Response.noContent().build();
|
return Response.noContent().build();
|
||||||
}
|
}
|
||||||
return Response.ok(taskMaster.getTaskRunner().getRunningTasks()).build();
|
return Response.ok(taskMasterLifecycle.getTaskRunner().getRunningTasks()).build();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -55,6 +55,8 @@ import com.metamx.druid.loading.S3SegmentPusherConfig;
|
|||||||
import com.metamx.druid.loading.SegmentKiller;
|
import com.metamx.druid.loading.SegmentKiller;
|
||||||
import com.metamx.druid.loading.SegmentPusher;
|
import com.metamx.druid.loading.SegmentPusher;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
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.IndexerZkConfig;
|
||||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||||
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
|
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
|
||||||
@ -64,7 +66,8 @@ import com.metamx.druid.merger.coordinator.LocalTaskStorage;
|
|||||||
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
|
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
|
||||||
import com.metamx.druid.merger.coordinator.RemoteTaskRunner;
|
import com.metamx.druid.merger.coordinator.RemoteTaskRunner;
|
||||||
import com.metamx.druid.merger.coordinator.RetryPolicyFactory;
|
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.TaskQueue;
|
||||||
import com.metamx.druid.merger.coordinator.TaskRunner;
|
import com.metamx.druid.merger.coordinator.TaskRunner;
|
||||||
import com.metamx.druid.merger.coordinator.TaskRunnerFactory;
|
import com.metamx.druid.merger.coordinator.TaskRunnerFactory;
|
||||||
@ -141,19 +144,21 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||||||
private ServiceEmitter emitter = null;
|
private ServiceEmitter emitter = null;
|
||||||
private DbConnectorConfig dbConnectorConfig = null;
|
private DbConnectorConfig dbConnectorConfig = null;
|
||||||
private DBI dbi = null;
|
private DBI dbi = null;
|
||||||
|
private RestS3Service s3Service = null;
|
||||||
private IndexerCoordinatorConfig config = null;
|
private IndexerCoordinatorConfig config = null;
|
||||||
private TaskConfig taskConfig = null;
|
private TaskConfig taskConfig = null;
|
||||||
private TaskToolbox taskToolbox = null;
|
private TaskToolbox taskToolbox = null;
|
||||||
private MergerDBCoordinator mergerDBCoordinator = null;
|
private MergerDBCoordinator mergerDBCoordinator = null;
|
||||||
private TaskStorage taskStorage = null;
|
private TaskStorage taskStorage = null;
|
||||||
private TaskQueue taskQueue = null;
|
private TaskQueue taskQueue = null;
|
||||||
|
private TaskLockbox taskLockbox = null;
|
||||||
private CuratorFramework curatorFramework = null;
|
private CuratorFramework curatorFramework = null;
|
||||||
private ScheduledExecutorFactory scheduledExecutorFactory = null;
|
private ScheduledExecutorFactory scheduledExecutorFactory = null;
|
||||||
private IndexerZkConfig indexerZkConfig;
|
private IndexerZkConfig indexerZkConfig;
|
||||||
private WorkerSetupManager workerSetupManager = null;
|
private WorkerSetupManager workerSetupManager = null;
|
||||||
private TaskRunnerFactory taskRunnerFactory = null;
|
private TaskRunnerFactory taskRunnerFactory = null;
|
||||||
private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null;
|
private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null;
|
||||||
private TaskMaster taskMaster = null;
|
private TaskMasterLifecycle taskMasterLifecycle = null;
|
||||||
private Server server = null;
|
private Server server = null;
|
||||||
|
|
||||||
private boolean initialized = false;
|
private boolean initialized = false;
|
||||||
@ -191,6 +196,18 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||||||
return this;
|
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)
|
public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator)
|
||||||
{
|
{
|
||||||
this.mergerDBCoordinator = mergeDbCoordinator;
|
this.mergerDBCoordinator = mergeDbCoordinator;
|
||||||
@ -230,10 +247,12 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||||||
initializeDB();
|
initializeDB();
|
||||||
initializeIndexerCoordinatorConfig();
|
initializeIndexerCoordinatorConfig();
|
||||||
initializeTaskConfig();
|
initializeTaskConfig();
|
||||||
|
initializeS3Service();
|
||||||
initializeMergeDBCoordinator();
|
initializeMergeDBCoordinator();
|
||||||
initializeTaskToolbox();
|
|
||||||
initializeTaskStorage();
|
initializeTaskStorage();
|
||||||
|
initializeTaskLockbox();
|
||||||
initializeTaskQueue();
|
initializeTaskQueue();
|
||||||
|
initializeTaskToolbox();
|
||||||
initializeJacksonInjections();
|
initializeJacksonInjections();
|
||||||
initializeJacksonSubtypes();
|
initializeJacksonSubtypes();
|
||||||
initializeCurator();
|
initializeCurator();
|
||||||
@ -241,7 +260,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||||||
initializeWorkerSetupManager();
|
initializeWorkerSetupManager();
|
||||||
initializeTaskRunnerFactory();
|
initializeTaskRunnerFactory();
|
||||||
initializeResourceManagement();
|
initializeResourceManagement();
|
||||||
initializeTaskMaster();
|
initializeTaskMasterLifecycle();
|
||||||
initializeServer();
|
initializeServer();
|
||||||
|
|
||||||
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
|
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
|
||||||
@ -258,10 +277,9 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||||||
jsonMapper,
|
jsonMapper,
|
||||||
config,
|
config,
|
||||||
emitter,
|
emitter,
|
||||||
taskQueue,
|
taskMasterLifecycle,
|
||||||
new TaskStorageQueryAdapter(taskStorage),
|
new TaskStorageQueryAdapter(taskStorage),
|
||||||
workerSetupManager,
|
workerSetupManager
|
||||||
taskMaster
|
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -279,7 +297,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||||||
@Override
|
@Override
|
||||||
public boolean doLocal()
|
public boolean doLocal()
|
||||||
{
|
{
|
||||||
return taskMaster.isLeading();
|
return taskMasterLifecycle.isLeading();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -289,7 +307,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||||||
return new URL(
|
return new URL(
|
||||||
String.format(
|
String.format(
|
||||||
"http://%s%s",
|
"http://%s%s",
|
||||||
taskMaster.getLeader(),
|
taskMasterLifecycle.getLeader(),
|
||||||
requestURI
|
requestURI
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
@ -307,21 +325,21 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||||||
initialized = true;
|
initialized = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void initializeTaskMaster()
|
private void initializeTaskMasterLifecycle()
|
||||||
{
|
{
|
||||||
if (taskMaster == null) {
|
if (taskMasterLifecycle == null) {
|
||||||
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
|
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
|
||||||
taskMaster = new TaskMaster(
|
taskMasterLifecycle = new TaskMasterLifecycle(
|
||||||
taskQueue,
|
taskQueue,
|
||||||
|
taskToolbox,
|
||||||
config,
|
config,
|
||||||
serviceDiscoveryConfig,
|
serviceDiscoveryConfig,
|
||||||
mergerDBCoordinator,
|
|
||||||
taskRunnerFactory,
|
taskRunnerFactory,
|
||||||
resourceManagementSchedulerFactory,
|
resourceManagementSchedulerFactory,
|
||||||
curatorFramework,
|
curatorFramework,
|
||||||
emitter
|
emitter
|
||||||
);
|
);
|
||||||
lifecycle.addManagedInstance(taskMaster);
|
lifecycle.addManagedInstance(taskMasterLifecycle);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -376,7 +394,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||||||
{
|
{
|
||||||
InjectableValues.Std injectables = new InjectableValues.Std();
|
InjectableValues.Std injectables = new InjectableValues.Std();
|
||||||
|
|
||||||
injectables.addValue("s3Client", taskToolbox.getS3Client())
|
injectables.addValue("s3Client", s3Service)
|
||||||
.addValue("segmentPusher", taskToolbox.getSegmentPusher());
|
.addValue("segmentPusher", taskToolbox.getSegmentPusher());
|
||||||
|
|
||||||
jsonMapper.setInjectableValues(injectables);
|
jsonMapper.setInjectableValues(injectables);
|
||||||
@ -436,24 +454,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) {
|
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(
|
final SegmentPusher segmentPusher = new S3SegmentPusher(
|
||||||
s3Client,
|
s3Service,
|
||||||
configFactory.build(S3SegmentPusherConfig.class),
|
configFactory.build(S3SegmentPusherConfig.class),
|
||||||
jsonMapper
|
jsonMapper
|
||||||
);
|
);
|
||||||
final SegmentKiller segmentKiller = new S3SegmentKiller(
|
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);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -471,8 +504,15 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||||||
public void initializeTaskQueue()
|
public void initializeTaskQueue()
|
||||||
{
|
{
|
||||||
if (taskQueue == null) {
|
if (taskQueue == null) {
|
||||||
// Don't start it here. The TaskMaster will handle that when it feels like it.
|
// Don't start it here. The TaskMasterLifecycle will handle that when it feels like it.
|
||||||
taskQueue = new TaskQueue(taskStorage);
|
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.common.collect.ImmutableMap;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.metamx.common.logger.Logger;
|
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.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.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.TaskStorageQueryAdapter;
|
||||||
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
||||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
|
|
||||||
import javax.ws.rs.Consumes;
|
import javax.ws.rs.Consumes;
|
||||||
import javax.ws.rs.GET;
|
import javax.ws.rs.GET;
|
||||||
@ -40,6 +43,8 @@ import javax.ws.rs.Path;
|
|||||||
import javax.ws.rs.PathParam;
|
import javax.ws.rs.PathParam;
|
||||||
import javax.ws.rs.Produces;
|
import javax.ws.rs.Produces;
|
||||||
import javax.ws.rs.core.Response;
|
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 IndexerCoordinatorConfig config;
|
||||||
private final ServiceEmitter emitter;
|
private final ServiceEmitter emitter;
|
||||||
private final TaskQueue tasks;
|
private final TaskMasterLifecycle taskMasterLifecycle;
|
||||||
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
|
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
|
||||||
private final WorkerSetupManager workerSetupManager;
|
private final WorkerSetupManager workerSetupManager;
|
||||||
|
private final ObjectMapper jsonMapper;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public IndexerCoordinatorResource(
|
public IndexerCoordinatorResource(
|
||||||
IndexerCoordinatorConfig config,
|
IndexerCoordinatorConfig config,
|
||||||
ServiceEmitter emitter,
|
ServiceEmitter emitter,
|
||||||
TaskQueue tasks,
|
TaskMasterLifecycle taskMasterLifecycle,
|
||||||
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
||||||
WorkerSetupManager workerSetupManager
|
WorkerSetupManager workerSetupManager,
|
||||||
|
ObjectMapper jsonMapper
|
||||||
) throws Exception
|
) throws Exception
|
||||||
{
|
{
|
||||||
this.config = config;
|
this.config = config;
|
||||||
this.emitter = emitter;
|
this.emitter = emitter;
|
||||||
this.tasks = tasks;
|
this.taskMasterLifecycle = taskMasterLifecycle;
|
||||||
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
|
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
|
||||||
this.workerSetupManager = workerSetupManager;
|
this.workerSetupManager = workerSetupManager;
|
||||||
|
this.jsonMapper = jsonMapper;
|
||||||
}
|
}
|
||||||
|
|
||||||
@POST
|
@POST
|
||||||
@Path("/merge")
|
@Path("/merge")
|
||||||
@Consumes("application/json")
|
@Consumes("application/json")
|
||||||
@Produces("application/json")
|
@Produces("application/json")
|
||||||
public Response doMerge(
|
public Response doMerge(final Task task)
|
||||||
final MergeTask task
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
// legacy endpoint
|
// legacy endpoint
|
||||||
return doIndex(task);
|
return doIndex(task);
|
||||||
@ -87,9 +92,7 @@ public class IndexerCoordinatorResource
|
|||||||
@Path("/index")
|
@Path("/index")
|
||||||
@Consumes("application/json")
|
@Consumes("application/json")
|
||||||
@Produces("application/json")
|
@Produces("application/json")
|
||||||
public Response doIndex(
|
public Response doIndex(final Task task)
|
||||||
final Task task
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
// verify against whitelist
|
// verify against whitelist
|
||||||
if (config.isWhitelistEnabled() && !config.getWhitelistDatasources().contains(task.getDataSource())) {
|
if (config.isWhitelistEnabled() && !config.getWhitelistDatasources().contains(task.getDataSource())) {
|
||||||
@ -103,14 +106,14 @@ public class IndexerCoordinatorResource
|
|||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
tasks.add(task);
|
taskMasterLifecycle.getTaskQueue().add(task);
|
||||||
return okResponse(task.getId());
|
return Response.ok(ImmutableMap.of("task", task.getId())).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@GET
|
@GET
|
||||||
@Path("/status/{taskid}")
|
@Path("/task/{taskid}/status")
|
||||||
@Produces("application/json")
|
@Produces("application/json")
|
||||||
public Response doStatus(@PathParam("taskid") String taskid)
|
public Response getTaskStatus(@PathParam("taskid") String taskid)
|
||||||
{
|
{
|
||||||
final Optional<TaskStatus> status = taskStorageQueryAdapter.getSameGroupMergedStatus(taskid);
|
final Optional<TaskStatus> status = taskStorageQueryAdapter.getSameGroupMergedStatus(taskid);
|
||||||
if (!status.isPresent()) {
|
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
|
@GET
|
||||||
@ -145,4 +169,13 @@ public class IndexerCoordinatorResource
|
|||||||
}
|
}
|
||||||
return Response.ok().build();
|
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();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -22,8 +22,7 @@ package com.metamx.druid.merger.coordinator.http;
|
|||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
|
import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
|
||||||
import com.google.inject.Provides;
|
import com.google.inject.Provides;
|
||||||
import com.metamx.druid.merger.coordinator.TaskMaster;
|
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.TaskStorageQueryAdapter;
|
||||||
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
||||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||||
@ -31,8 +30,6 @@ import com.metamx.emitter.service.ServiceEmitter;
|
|||||||
import com.sun.jersey.guice.JerseyServletModule;
|
import com.sun.jersey.guice.JerseyServletModule;
|
||||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import javax.inject.Singleton;
|
import javax.inject.Singleton;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -42,28 +39,25 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
|
|||||||
private final ObjectMapper jsonMapper;
|
private final ObjectMapper jsonMapper;
|
||||||
private final IndexerCoordinatorConfig indexerCoordinatorConfig;
|
private final IndexerCoordinatorConfig indexerCoordinatorConfig;
|
||||||
private final ServiceEmitter emitter;
|
private final ServiceEmitter emitter;
|
||||||
private final TaskQueue tasks;
|
private final TaskMasterLifecycle taskMasterLifecycle;
|
||||||
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
|
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
|
||||||
private final WorkerSetupManager workerSetupManager;
|
private final WorkerSetupManager workerSetupManager;
|
||||||
private final TaskMaster taskMaster;
|
|
||||||
|
|
||||||
public IndexerCoordinatorServletModule(
|
public IndexerCoordinatorServletModule(
|
||||||
ObjectMapper jsonMapper,
|
ObjectMapper jsonMapper,
|
||||||
IndexerCoordinatorConfig indexerCoordinatorConfig,
|
IndexerCoordinatorConfig indexerCoordinatorConfig,
|
||||||
ServiceEmitter emitter,
|
ServiceEmitter emitter,
|
||||||
TaskQueue tasks,
|
TaskMasterLifecycle taskMasterLifecycle,
|
||||||
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
||||||
WorkerSetupManager workerSetupManager,
|
WorkerSetupManager workerSetupManager
|
||||||
TaskMaster taskMaster
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.jsonMapper = jsonMapper;
|
this.jsonMapper = jsonMapper;
|
||||||
this.indexerCoordinatorConfig = indexerCoordinatorConfig;
|
this.indexerCoordinatorConfig = indexerCoordinatorConfig;
|
||||||
this.emitter = emitter;
|
this.emitter = emitter;
|
||||||
this.tasks = tasks;
|
this.taskMasterLifecycle = taskMasterLifecycle;
|
||||||
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
|
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
|
||||||
this.workerSetupManager = workerSetupManager;
|
this.workerSetupManager = workerSetupManager;
|
||||||
this.taskMaster = taskMaster;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -74,10 +68,9 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
|
|||||||
bind(ObjectMapper.class).toInstance(jsonMapper);
|
bind(ObjectMapper.class).toInstance(jsonMapper);
|
||||||
bind(IndexerCoordinatorConfig.class).toInstance(indexerCoordinatorConfig);
|
bind(IndexerCoordinatorConfig.class).toInstance(indexerCoordinatorConfig);
|
||||||
bind(ServiceEmitter.class).toInstance(emitter);
|
bind(ServiceEmitter.class).toInstance(emitter);
|
||||||
bind(TaskQueue.class).toInstance(tasks);
|
bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle);
|
||||||
bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter);
|
bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter);
|
||||||
bind(WorkerSetupManager.class).toInstance(workerSetupManager);
|
bind(WorkerSetupManager.class).toInstance(workerSetupManager);
|
||||||
bind(TaskMaster.class).toInstance(taskMaster);
|
|
||||||
|
|
||||||
serve("/*").with(GuiceContainer.class);
|
serve("/*").with(GuiceContainer.class);
|
||||||
}
|
}
|
||||||
|
@ -21,12 +21,9 @@ package com.metamx.druid.merger.worker;
|
|||||||
|
|
||||||
import com.metamx.common.lifecycle.LifecycleStart;
|
import com.metamx.common.lifecycle.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
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.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
import com.metamx.druid.merger.common.TaskToolbox;
|
||||||
import com.metamx.druid.merger.common.task.Task;
|
import com.metamx.druid.merger.common.task.Task;
|
||||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import com.netflix.curator.framework.CuratorFramework;
|
import com.netflix.curator.framework.CuratorFramework;
|
||||||
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
|
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
|
||||||
@ -34,7 +31,6 @@ import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent;
|
|||||||
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
|
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
|
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
@ -85,12 +81,10 @@ public class TaskMonitor
|
|||||||
throws Exception
|
throws Exception
|
||||||
{
|
{
|
||||||
if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) {
|
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()),
|
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())) {
|
if (workerCuratorCoordinator.statusExists(task.getId())) {
|
||||||
log.warn("Got task %s that I am already running...", task.getId());
|
log.warn("Got task %s that I am already running...", task.getId());
|
||||||
@ -113,14 +107,7 @@ public class TaskMonitor
|
|||||||
try {
|
try {
|
||||||
workerCuratorCoordinator.unannounceTask(task.getId());
|
workerCuratorCoordinator.unannounceTask(task.getId());
|
||||||
workerCuratorCoordinator.announceStatus(TaskStatus.running(task.getId()));
|
workerCuratorCoordinator.announceStatus(TaskStatus.running(task.getId()));
|
||||||
taskStatus = task.run(taskContext, toolbox, new TaskCallback()
|
taskStatus = task.run(toolbox);
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void notify(TaskStatus status)
|
|
||||||
{
|
|
||||||
workerCuratorCoordinator.updateStatus(status);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.makeAlert(e, "Failed to run task")
|
log.makeAlert(e, "Failed to run task")
|
||||||
|
@ -42,10 +42,10 @@ import com.metamx.druid.loading.S3SegmentPusherConfig;
|
|||||||
import com.metamx.druid.loading.SegmentKiller;
|
import com.metamx.druid.loading.SegmentKiller;
|
||||||
import com.metamx.druid.loading.SegmentPusher;
|
import com.metamx.druid.loading.SegmentPusher;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
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.IndexerZkConfig;
|
||||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||||
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
|
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.TaskMonitor;
|
||||||
import com.metamx.druid.merger.worker.Worker;
|
import com.metamx.druid.merger.worker.Worker;
|
||||||
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
|
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
|
||||||
@ -64,8 +64,6 @@ import com.metamx.metrics.MonitorSchedulerConfig;
|
|||||||
import com.metamx.metrics.SysMonitor;
|
import com.metamx.metrics.SysMonitor;
|
||||||
import com.netflix.curator.framework.CuratorFramework;
|
import com.netflix.curator.framework.CuratorFramework;
|
||||||
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
|
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
|
||||||
|
|
||||||
|
|
||||||
import org.jets3t.service.S3ServiceException;
|
import org.jets3t.service.S3ServiceException;
|
||||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||||
import org.jets3t.service.security.AWSCredentials;
|
import org.jets3t.service.security.AWSCredentials;
|
||||||
@ -99,7 +97,9 @@ public class WorkerNode extends RegisteringNode
|
|||||||
private final Properties props;
|
private final Properties props;
|
||||||
private final ConfigurationObjectFactory configFactory;
|
private final ConfigurationObjectFactory configFactory;
|
||||||
|
|
||||||
|
private RestS3Service s3Service = null;
|
||||||
private List<Monitor> monitors = null;
|
private List<Monitor> monitors = null;
|
||||||
|
private HttpClient httpClient = null;
|
||||||
private ServiceEmitter emitter = null;
|
private ServiceEmitter emitter = null;
|
||||||
private TaskConfig taskConfig = null;
|
private TaskConfig taskConfig = null;
|
||||||
private WorkerConfig workerConfig = null;
|
private WorkerConfig workerConfig = null;
|
||||||
@ -126,12 +126,24 @@ public class WorkerNode extends RegisteringNode
|
|||||||
this.configFactory = configFactory;
|
this.configFactory = configFactory;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public WorkerNode setHttpClient(HttpClient httpClient)
|
||||||
|
{
|
||||||
|
this.httpClient = httpClient;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public WorkerNode setEmitter(ServiceEmitter emitter)
|
public WorkerNode setEmitter(ServiceEmitter emitter)
|
||||||
{
|
{
|
||||||
this.emitter = emitter;
|
this.emitter = emitter;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public WorkerNode setS3Service(RestS3Service s3Service)
|
||||||
|
{
|
||||||
|
this.s3Service = s3Service;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public WorkerNode setTaskToolbox(TaskToolbox taskToolbox)
|
public WorkerNode setTaskToolbox(TaskToolbox taskToolbox)
|
||||||
{
|
{
|
||||||
this.taskToolbox = taskToolbox;
|
this.taskToolbox = taskToolbox;
|
||||||
@ -158,7 +170,9 @@ public class WorkerNode extends RegisteringNode
|
|||||||
|
|
||||||
public void init() throws Exception
|
public void init() throws Exception
|
||||||
{
|
{
|
||||||
|
initializeHttpClient();
|
||||||
initializeEmitter();
|
initializeEmitter();
|
||||||
|
initializeS3Service();
|
||||||
initializeMonitors();
|
initializeMonitors();
|
||||||
initializeMergerConfig();
|
initializeMergerConfig();
|
||||||
initializeTaskToolbox();
|
initializeTaskToolbox();
|
||||||
@ -237,7 +251,7 @@ public class WorkerNode extends RegisteringNode
|
|||||||
{
|
{
|
||||||
InjectableValues.Std injectables = new InjectableValues.Std();
|
InjectableValues.Std injectables = new InjectableValues.Std();
|
||||||
|
|
||||||
injectables.addValue("s3Client", taskToolbox.getS3Client())
|
injectables.addValue("s3Client", s3Service)
|
||||||
.addValue("segmentPusher", taskToolbox.getSegmentPusher());
|
.addValue("segmentPusher", taskToolbox.getSegmentPusher());
|
||||||
|
|
||||||
jsonMapper.setInjectableValues(injectables);
|
jsonMapper.setInjectableValues(injectables);
|
||||||
@ -248,13 +262,18 @@ public class WorkerNode extends RegisteringNode
|
|||||||
jsonMapper.registerSubtypes(StaticS3FirehoseFactory.class);
|
jsonMapper.registerSubtypes(StaticS3FirehoseFactory.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void initializeHttpClient()
|
||||||
|
{
|
||||||
|
if (httpClient == null) {
|
||||||
|
httpClient = HttpClientInit.createClient(
|
||||||
|
HttpClientConfig.builder().withNumConnections(1).build(), lifecycle
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private void initializeEmitter()
|
private void initializeEmitter()
|
||||||
{
|
{
|
||||||
if (emitter == null) {
|
if (emitter == null) {
|
||||||
final HttpClient httpClient = HttpClientInit.createClient(
|
|
||||||
HttpClientConfig.builder().withNumConnections(1).build(), lifecycle
|
|
||||||
);
|
|
||||||
|
|
||||||
emitter = new ServiceEmitter(
|
emitter = new ServiceEmitter(
|
||||||
PropUtils.getProperty(props, "druid.service"),
|
PropUtils.getProperty(props, "druid.service"),
|
||||||
PropUtils.getProperty(props, "druid.host"),
|
PropUtils.getProperty(props, "druid.host"),
|
||||||
@ -264,6 +283,18 @@ public class WorkerNode extends RegisteringNode
|
|||||||
EmittingLogger.registerEmitter(emitter);
|
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()
|
private void initializeMonitors()
|
||||||
{
|
{
|
||||||
if (monitors == null) {
|
if (monitors == null) {
|
||||||
@ -287,21 +318,23 @@ public class WorkerNode extends RegisteringNode
|
|||||||
public void initializeTaskToolbox() throws S3ServiceException
|
public void initializeTaskToolbox() throws S3ServiceException
|
||||||
{
|
{
|
||||||
if (taskToolbox == null) {
|
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(
|
final SegmentPusher segmentPusher = new S3SegmentPusher(
|
||||||
s3Client,
|
s3Service,
|
||||||
configFactory.build(S3SegmentPusherConfig.class),
|
configFactory.build(S3SegmentPusherConfig.class),
|
||||||
jsonMapper
|
jsonMapper
|
||||||
);
|
);
|
||||||
final SegmentKiller segmentKiller = new S3SegmentKiller(
|
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);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -24,11 +24,9 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
import com.metamx.druid.client.DataSegment;
|
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.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
import com.metamx.druid.merger.common.TaskToolbox;
|
||||||
import com.metamx.druid.merger.common.task.DefaultMergeTask;
|
import com.metamx.druid.merger.common.task.DefaultMergeTask;
|
||||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
@ -63,9 +61,10 @@ public class TestTask extends DefaultMergeTask
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
@JsonProperty
|
||||||
|
public String getType()
|
||||||
{
|
{
|
||||||
return Type.TEST;
|
return "test";
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
@ -75,7 +74,7 @@ public class TestTask extends DefaultMergeTask
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception
|
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||||
{
|
{
|
||||||
return status;
|
return status;
|
||||||
}
|
}
|
||||||
|
@ -51,9 +51,9 @@ public class MergeTaskTest
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return Type.TEST;
|
return "test";
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -66,13 +66,13 @@ public class MergeTaskTest
|
|||||||
@Test
|
@Test
|
||||||
public void testInterval()
|
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
|
@Test
|
||||||
public void testID()
|
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-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-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"
|
+ "_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 com.fasterxml.jackson.databind.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());
|
||||||
|
}
|
||||||
|
}
|
@ -4,7 +4,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Sets;
|
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
@ -131,7 +130,6 @@ public class RemoteTaskRunnerTest
|
|||||||
{
|
{
|
||||||
remoteTaskRunner.run(
|
remoteTaskRunner.run(
|
||||||
task1,
|
task1,
|
||||||
new TaskContext(new DateTime().toString(), Sets.<DataSegment>newHashSet(), Sets.<DataSegment>newHashSet()),
|
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
@ -147,19 +145,10 @@ public class RemoteTaskRunnerTest
|
|||||||
Lists.<AggregatorFactory>newArrayList(),
|
Lists.<AggregatorFactory>newArrayList(),
|
||||||
TaskStatus.running(task1.getId())
|
TaskStatus.running(task1.getId())
|
||||||
),
|
),
|
||||||
new TaskContext(new DateTime().toString(), Sets.<DataSegment>newHashSet(), Sets.<DataSegment>newHashSet()),
|
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
try {
|
try {
|
||||||
remoteTaskRunner.run(
|
remoteTaskRunner.run(task1, null);
|
||||||
task1,
|
|
||||||
new TaskContext(
|
|
||||||
new DateTime().toString(),
|
|
||||||
Sets.<DataSegment>newHashSet(),
|
|
||||||
Sets.<DataSegment>newHashSet()
|
|
||||||
),
|
|
||||||
null
|
|
||||||
);
|
|
||||||
fail("ISE expected");
|
fail("ISE expected");
|
||||||
}
|
}
|
||||||
catch (ISE expected) {
|
catch (ISE expected) {
|
||||||
@ -191,7 +180,6 @@ public class RemoteTaskRunnerTest
|
|||||||
Lists.<AggregatorFactory>newArrayList(),
|
Lists.<AggregatorFactory>newArrayList(),
|
||||||
TaskStatus.success("foo")
|
TaskStatus.success("foo")
|
||||||
),
|
),
|
||||||
new TaskContext(new DateTime().toString(), Sets.<DataSegment>newHashSet(), Sets.<DataSegment>newHashSet()),
|
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
@ -213,7 +201,6 @@ public class RemoteTaskRunnerTest
|
|||||||
Lists.<AggregatorFactory>newArrayList(),
|
Lists.<AggregatorFactory>newArrayList(),
|
||||||
TaskStatus.running(task1.getId())
|
TaskStatus.running(task1.getId())
|
||||||
),
|
),
|
||||||
new TaskContext(new DateTime().toString(), Sets.<DataSegment>newHashSet(), Sets.<DataSegment>newHashSet()),
|
|
||||||
new TaskCallback()
|
new TaskCallback()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
@ -313,7 +300,7 @@ public class RemoteTaskRunnerTest
|
|||||||
{
|
{
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
}, null, null, null, null, jsonMapper
|
}, null, null, null, null, null, jsonMapper
|
||||||
),
|
),
|
||||||
Executors.newSingleThreadExecutor()
|
Executors.newSingleThreadExecutor()
|
||||||
);
|
);
|
||||||
|
@ -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;
|
package com.metamx.druid.merger.coordinator;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableList;
|
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.Lists;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.merger.common.TaskLock;
|
||||||
import com.metamx.druid.merger.common.TaskCallback;
|
|
||||||
import com.metamx.druid.merger.common.TaskStatus;
|
import com.metamx.druid.merger.common.TaskStatus;
|
||||||
import com.metamx.druid.merger.common.TaskToolbox;
|
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.AbstractTask;
|
||||||
import com.metamx.druid.merger.common.task.Task;
|
import com.metamx.druid.merger.common.task.Task;
|
||||||
import org.joda.time.DateTime;
|
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
@ -43,7 +44,8 @@ public class TaskQueueTest
|
|||||||
public void testEmptyQueue() throws Exception
|
public void testEmptyQueue() throws Exception
|
||||||
{
|
{
|
||||||
final TaskStorage ts = new LocalTaskStorage();
|
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
|
// get task status for nonexistent task
|
||||||
Assert.assertFalse("getStatus", ts.getStatus("foo").isPresent());
|
Assert.assertFalse("getStatus", ts.getStatus("foo").isPresent());
|
||||||
@ -52,9 +54,10 @@ public class TaskQueueTest
|
|||||||
Assert.assertNull("poll", tq.poll());
|
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();
|
tq.start();
|
||||||
return tq;
|
return tq;
|
||||||
}
|
}
|
||||||
@ -63,7 +66,8 @@ public class TaskQueueTest
|
|||||||
public void testAddRemove() throws Exception
|
public void testAddRemove() throws Exception
|
||||||
{
|
{
|
||||||
final TaskStorage ts = new LocalTaskStorage();
|
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 = {
|
final Task[] tasks = {
|
||||||
newTask("T0", "G0", "bar", new Interval("2011/P1Y")),
|
newTask("T0", "G0", "bar", new Interval("2011/P1Y")),
|
||||||
@ -96,9 +100,9 @@ public class TaskQueueTest
|
|||||||
// take max number of tasks
|
// take max number of tasks
|
||||||
final List<Task> taken = Lists.newArrayList();
|
final List<Task> taken = Lists.newArrayList();
|
||||||
while (true) {
|
while (true) {
|
||||||
final VersionedTaskWrapper taskWrapper = tq.poll();
|
final Task task = tq.poll();
|
||||||
if(taskWrapper != null) {
|
if(task != null) {
|
||||||
taken.add(taskWrapper.getTask());
|
taken.add(task);
|
||||||
} else {
|
} else {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
@ -114,8 +118,7 @@ public class TaskQueueTest
|
|||||||
);
|
);
|
||||||
|
|
||||||
// mark one done
|
// mark one done
|
||||||
final TestCommitRunnable commit1 = newCommitRunnable();
|
tq.notify(tasks[2], tasks[2].run(null));
|
||||||
tq.notify(tasks[2], tasks[2].run(null, null, null), commit1);
|
|
||||||
|
|
||||||
// get its status back
|
// get its status back
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
@ -124,20 +127,12 @@ public class TaskQueueTest
|
|||||||
ts.getStatus(tasks[2].getId()).get().getStatusCode()
|
ts.getStatus(tasks[2].getId()).get().getStatusCode()
|
||||||
);
|
);
|
||||||
|
|
||||||
Assert.assertEquals("Commit #1 wasRun", commit1.wasRun(), true);
|
// We should be able to get one more task now
|
||||||
|
|
||||||
// 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
|
|
||||||
taken.clear();
|
taken.clear();
|
||||||
while (true) {
|
while (true) {
|
||||||
final VersionedTaskWrapper taskWrapper = tq.poll();
|
final Task task = tq.poll();
|
||||||
if(taskWrapper != null) {
|
if(task != null) {
|
||||||
taken.add(taskWrapper.getTask());
|
taken.add(task);
|
||||||
} else {
|
} else {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
@ -160,7 +155,17 @@ public class TaskQueueTest
|
|||||||
public void testContinues() throws Exception
|
public void testContinues() throws Exception
|
||||||
{
|
{
|
||||||
final TaskStorage ts = new LocalTaskStorage();
|
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 t0 = newTask("T0", "G0", "bar", new Interval("2011/P1Y"));
|
||||||
final Task t1 = newContinuedTask("T1", "G1", "bar", new Interval("2013/P1Y"), Lists.newArrayList(t0));
|
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("T0 isPresent (#1)", !ts.getStatus("T0").isPresent());
|
||||||
Assert.assertTrue("T1 isPresent (#1)", ts.getStatus("T1").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());
|
Assert.assertTrue("T1 isComplete (#1)", !ts.getStatus("T1").get().isComplete());
|
||||||
|
|
||||||
// should be able to get t1 out
|
// 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());
|
Assert.assertNull("poll #2", tq.poll());
|
||||||
|
|
||||||
// report T1 done. Should cause T0 to be created
|
// 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 isRunnable (#2)", ts.getStatus("T0").get().isRunnable());
|
||||||
Assert.assertTrue("T0 isComplete (#2)", !ts.getStatus("T0").get().isComplete());
|
Assert.assertTrue("T0 isComplete (#2)", !ts.getStatus("T0").get().isComplete());
|
||||||
Assert.assertTrue("T1 isPresent (#2)", ts.getStatus("T1").isPresent());
|
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());
|
Assert.assertTrue("T1 isComplete (#2)", ts.getStatus("T1").get().isComplete());
|
||||||
|
|
||||||
// should be able to get t0 out
|
// 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());
|
Assert.assertNull("poll #4", tq.poll());
|
||||||
|
|
||||||
// report T0 done. Should cause T0, T1 to be marked complete
|
// 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 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable());
|
||||||
Assert.assertTrue("T0 isComplete (#3)", ts.getStatus("T0").get().isComplete());
|
Assert.assertTrue("T0 isComplete (#3)", ts.getStatus("T0").get().isComplete());
|
||||||
Assert.assertTrue("T1 isPresent (#3)", ts.getStatus("T1").isPresent());
|
Assert.assertTrue("T1 isPresent (#3)", ts.getStatus("T1").isPresent());
|
||||||
@ -207,7 +212,17 @@ public class TaskQueueTest
|
|||||||
public void testConcurrency() throws Exception
|
public void testConcurrency() throws Exception
|
||||||
{
|
{
|
||||||
final TaskStorage ts = new LocalTaskStorage();
|
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
|
// Imagine a larger task that splits itself up into pieces
|
||||||
final Task t1 = newTask("T1", "G0", "bar", new Interval("2011-01-01/P1D"));
|
final Task t1 = newTask("T1", "G0", "bar", new Interval("2011-01-01/P1D"));
|
||||||
@ -224,15 +239,16 @@ public class TaskQueueTest
|
|||||||
|
|
||||||
tq.add(t0);
|
tq.add(t0);
|
||||||
|
|
||||||
final VersionedTaskWrapper wt0 = tq.poll();
|
final Task wt0 = tq.poll();
|
||||||
Assert.assertEquals("wt0 task id", "T0", wt0.getTask().getId());
|
final TaskLock wt0Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt0));
|
||||||
|
Assert.assertEquals("wt0 task id", "T0", wt0.getId());
|
||||||
Assert.assertNull("null poll #1", tq.poll());
|
Assert.assertNull("null poll #1", tq.poll());
|
||||||
|
|
||||||
// Sleep a bit to avoid false test passes
|
// Sleep a bit to avoid false test passes
|
||||||
Thread.sleep(5);
|
Thread.sleep(5);
|
||||||
|
|
||||||
// Finish t0
|
// Finish t0
|
||||||
tq.notify(t0, t0.run(null, null, null));
|
tq.notify(t0, t0.run(tb));
|
||||||
|
|
||||||
// take max number of tasks
|
// take max number of tasks
|
||||||
final Set<String> taken = Sets.newHashSet();
|
final Set<String> taken = Sets.newHashSet();
|
||||||
@ -241,15 +257,16 @@ public class TaskQueueTest
|
|||||||
// Sleep a bit to avoid false test passes
|
// Sleep a bit to avoid false test passes
|
||||||
Thread.sleep(5);
|
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(
|
Assert.assertEquals(
|
||||||
String.format("%s version", taskWrapper.getTask().getId()),
|
String.format("%s version", task.getId()),
|
||||||
wt0.getVersion(),
|
wt0Lock.getVersion(),
|
||||||
taskWrapper.getVersion()
|
taskLock.getVersion()
|
||||||
);
|
);
|
||||||
taken.add(taskWrapper.getTask().getId());
|
taken.add(task.getId());
|
||||||
} else {
|
} else {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
@ -259,34 +276,36 @@ public class TaskQueueTest
|
|||||||
Assert.assertEquals("taken", Sets.newHashSet("T1", "T3"), taken);
|
Assert.assertEquals("taken", Sets.newHashSet("T1", "T3"), taken);
|
||||||
|
|
||||||
// Finish t1
|
// Finish t1
|
||||||
tq.notify(t1, t1.run(null, null, null));
|
tq.notify(t1, t1.run(null));
|
||||||
Assert.assertNull("null poll #2", tq.poll());
|
Assert.assertNull("null poll #2", tq.poll());
|
||||||
|
|
||||||
// Finish t3
|
// Finish t3
|
||||||
tq.notify(t3, t3.run(null, null, null));
|
tq.notify(t3, t3.run(tb));
|
||||||
|
|
||||||
// We should be able to get t2 now
|
// We should be able to get t2 now
|
||||||
final VersionedTaskWrapper wt2 = tq.poll();
|
final Task wt2 = tq.poll();
|
||||||
Assert.assertEquals("wt2 task id", "T2", wt2.getTask().getId());
|
final TaskLock wt2Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt2));
|
||||||
Assert.assertEquals("wt2 group id", "G1", wt2.getTask().getGroupId());
|
Assert.assertEquals("wt2 task id", "T2", wt2.getId());
|
||||||
Assert.assertNotSame("wt2 version", wt0.getVersion(), wt2.getVersion());
|
Assert.assertEquals("wt2 group id", "G1", wt2.getGroupId());
|
||||||
|
Assert.assertNotSame("wt2 version", wt0Lock.getVersion(), wt2Lock.getVersion());
|
||||||
Assert.assertNull("null poll #3", tq.poll());
|
Assert.assertNull("null poll #3", tq.poll());
|
||||||
|
|
||||||
// Finish t2
|
// Finish t2
|
||||||
tq.notify(t2, t2.run(null, null, null));
|
tq.notify(t2, t2.run(tb));
|
||||||
|
|
||||||
// We should be able to get t4
|
// 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
|
// 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
|
// (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)
|
// one version if they end and are re-started)
|
||||||
final VersionedTaskWrapper wt4 = tq.poll();
|
final Task wt4 = tq.poll();
|
||||||
Assert.assertEquals("wt4 task id", "T4", wt4.getTask().getId());
|
final TaskLock wt4Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt4));
|
||||||
Assert.assertEquals("wt4 group id", "G0", wt4.getTask().getGroupId());
|
Assert.assertEquals("wt4 task id", "T4", wt4.getId());
|
||||||
Assert.assertNotSame("wt4 version", wt0.getVersion(), wt4.getVersion());
|
Assert.assertEquals("wt4 group id", "G0", wt4.getGroupId());
|
||||||
Assert.assertNotSame("wt4 version", wt2.getVersion(), wt4.getVersion());
|
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
|
// 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());
|
Assert.assertNull("null poll #4", tq.poll());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -294,162 +313,44 @@ public class TaskQueueTest
|
|||||||
public void testBootstrap() throws Exception
|
public void testBootstrap() throws Exception
|
||||||
{
|
{
|
||||||
final TaskStorage storage = new LocalTaskStorage();
|
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("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.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();
|
final Task vt1 = tq.poll();
|
||||||
Assert.assertEquals("vt1 id", "T1", vt1.getTask().getId());
|
final TaskLock vt1Lock = Iterables.getOnlyElement(lockbox.findLocksForTask(vt1));
|
||||||
Assert.assertEquals("vt1 version", "1234", vt1.getVersion());
|
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
|
// re-bootstrap
|
||||||
tq.stop();
|
tq.stop();
|
||||||
storage.setStatus("T2", TaskStatus.failure("T2"));
|
storage.setStatus(TaskStatus.failure("T2"));
|
||||||
|
tq.bootstrap();
|
||||||
tq.start();
|
tq.start();
|
||||||
|
|
||||||
Assert.assertNull("null poll", tq.poll());
|
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)
|
private static Task newTask(final String id, final String groupId, final String dataSource, final Interval interval)
|
||||||
{
|
{
|
||||||
return new AbstractTask(id, groupId, dataSource, interval)
|
return new AbstractTask(id, groupId, dataSource, interval)
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public TaskStatus run(TaskContext context, TaskToolbox toolbox, TaskCallback callback) throws Exception
|
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||||
{
|
{
|
||||||
return TaskStatus.success(
|
return TaskStatus.success(id);
|
||||||
id,
|
|
||||||
ImmutableSet.of(
|
|
||||||
new DataSegment(
|
|
||||||
dataSource,
|
|
||||||
interval,
|
|
||||||
new DateTime("2012-01-02").toString(),
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
-1
|
|
||||||
)
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return Type.TEST;
|
return "null";
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
@ -465,37 +366,17 @@ public class TaskQueueTest
|
|||||||
return new AbstractTask(id, groupId, dataSource, interval)
|
return new AbstractTask(id, groupId, dataSource, interval)
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Type getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return Type.TEST;
|
return "null";
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -121,7 +121,7 @@ public class SimpleResourceManagementStrategyTest
|
|||||||
|
|
||||||
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<TaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime())
|
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(testTask)
|
new TestZkWorker(testTask)
|
||||||
@ -149,7 +149,7 @@ public class SimpleResourceManagementStrategyTest
|
|||||||
|
|
||||||
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<TaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime())
|
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(testTask)
|
new TestZkWorker(testTask)
|
||||||
@ -165,7 +165,7 @@ public class SimpleResourceManagementStrategyTest
|
|||||||
|
|
||||||
provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<TaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime())
|
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(testTask)
|
new TestZkWorker(testTask)
|
||||||
@ -199,7 +199,7 @@ public class SimpleResourceManagementStrategyTest
|
|||||||
|
|
||||||
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<TaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime())
|
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(null)
|
new TestZkWorker(null)
|
||||||
@ -231,7 +231,7 @@ public class SimpleResourceManagementStrategyTest
|
|||||||
|
|
||||||
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<TaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime())
|
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(null)
|
new TestZkWorker(null)
|
||||||
@ -246,7 +246,7 @@ public class SimpleResourceManagementStrategyTest
|
|||||||
|
|
||||||
terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<TaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null, null).withQueueInsertionTime(new DateTime())
|
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(null)
|
new TestZkWorker(null)
|
||||||
|
@ -54,7 +54,6 @@ import com.metamx.phonebook.PhoneBook;
|
|||||||
import com.metamx.phonebook.PhoneBookPeon;
|
import com.metamx.phonebook.PhoneBookPeon;
|
||||||
import com.netflix.curator.x.discovery.ServiceProvider;
|
import com.netflix.curator.x.discovery.ServiceProvider;
|
||||||
import org.I0Itec.zkclient.exception.ZkNodeExistsException;
|
import org.I0Itec.zkclient.exception.ZkNodeExistsException;
|
||||||
|
|
||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
Loading…
x
Reference in New Issue
Block a user