Merge branch 'master' of github.com:metamx/druid

This commit is contained in:
Fangjin Yang 2013-04-15 15:42:19 -07:00
commit 3cfa081c6c
74 changed files with 2888 additions and 688 deletions

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -24,11 +24,11 @@
<artifactId>druid-services</artifactId>
<name>druid-services</name>
<description>druid-services</description>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
</parent>
<modules>

View File

@ -9,7 +9,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid-examples</artifactId>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -9,7 +9,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid-examples</artifactId>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -1,25 +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;
public interface TaskCallback
{
public void notify(TaskStatus status);
}

View File

@ -143,7 +143,7 @@ public class TaskToolbox
@Override
public File getCacheDirectory()
{
return new File(getTaskDir(), "fetched_segments");
return new File(getTaskWorkDir(), "fetched_segments");
}
}
);
@ -156,8 +156,7 @@ public class TaskToolbox
return retVal;
}
public File getTaskDir() {
return new File(config.getBaseTaskDir(), task.getId());
public File getTaskWorkDir() {
return new File(new File(config.getBaseTaskDir(), task.getId()), "work");
}
}

View File

@ -28,15 +28,17 @@ public class LocalTaskActionClient implements TaskActionClient
final RetType ret = taskAction.perform(task, toolbox);
// Add audit log
try {
storage.addAuditLog(task, taskAction);
}
catch (Exception e) {
log.makeAlert(e, "Failed to record action in audit log")
.addData("task", task.getId())
.addData("actionClass", taskAction.getClass().getName())
.emit();
if (taskAction.isAudited()) {
// Add audit log
try {
storage.addAuditLog(task, taskAction);
}
catch (Exception e) {
log.makeAlert(e, "Failed to record action in audit log")
.addData("task", task.getId())
.addData("actionClass", taskAction.getClass().getName())
.emit();
}
}
return ret;

View File

@ -1,6 +1,7 @@
package com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Throwables;
@ -10,6 +11,7 @@ import org.joda.time.Interval;
public class LockAcquireAction implements TaskAction<TaskLock>
{
@JsonIgnore
private final Interval interval;
@JsonCreator
@ -44,6 +46,12 @@ public class LockAcquireAction implements TaskAction<TaskLock>
}
}
@Override
public boolean isAudited()
{
return true;
}
@Override
public String toString()
{

View File

@ -1,8 +1,8 @@
package com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.core.type.TypeReference;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.core.type.TypeReference;
import java.util.List;
@ -19,6 +19,12 @@ public class LockListAction implements TaskAction<List<TaskLock>>
return toolbox.getTaskLockbox().findLocksForTask(task);
}
@Override
public boolean isAudited()
{
return false;
}
@Override
public String toString()
{

View File

@ -1,6 +1,7 @@
package com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.metamx.druid.merger.common.task.Task;
@ -8,6 +9,7 @@ import org.joda.time.Interval;
public class LockReleaseAction implements TaskAction<Void>
{
@JsonIgnore
private final Interval interval;
@JsonCreator
@ -36,6 +38,12 @@ public class LockReleaseAction implements TaskAction<Void>
return null;
}
@Override
public boolean isAudited()
{
return true;
}
@Override
public String toString()
{

View File

@ -80,6 +80,12 @@ public class SegmentInsertAction implements TaskAction<Set<DataSegment>>
return retVal;
}
@Override
public boolean isAudited()
{
return true;
}
@Override
public String toString()
{

View File

@ -1,6 +1,7 @@
package com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.metamx.druid.client.DataSegment;
@ -12,7 +13,10 @@ import java.util.List;
public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
{
@JsonIgnore
private final String dataSource;
@JsonIgnore
private final Interval interval;
@JsonCreator
@ -48,6 +52,12 @@ public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval);
}
@Override
public boolean isAudited()
{
return false;
}
@Override
public String toString()
{

View File

@ -1,6 +1,7 @@
package com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.metamx.druid.client.DataSegment;
@ -12,7 +13,10 @@ import java.util.List;
public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
{
@JsonIgnore
private final String dataSource;
@JsonIgnore
private final Interval interval;
@JsonCreator
@ -48,6 +52,12 @@ public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval);
}
@Override
public boolean isAudited()
{
return false;
}
@Override
public String toString()
{

View File

@ -1,6 +1,7 @@
package com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
@ -14,6 +15,7 @@ import java.util.Set;
public class SegmentNukeAction implements TaskAction<Void>
{
@JsonIgnore
private final Set<DataSegment> segments;
@JsonCreator
@ -57,6 +59,12 @@ public class SegmentNukeAction implements TaskAction<Void>
return null;
}
@Override
public boolean isAudited()
{
return true;
}
@Override
public String toString()
{

View File

@ -1,6 +1,7 @@
package com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableList;
@ -10,6 +11,7 @@ import java.util.List;
public class SpawnTasksAction implements TaskAction<Void>
{
@JsonIgnore
private final List<Task> newTasks;
@JsonCreator
@ -41,6 +43,12 @@ public class SpawnTasksAction implements TaskAction<Void>
return null;
}
@Override
public boolean isAudited()
{
return true;
}
@Override
public String toString()
{

View File

@ -22,4 +22,5 @@ public interface TaskAction<RetType>
{
public TypeReference<RetType> getReturnTypeReference(); // T_T
public RetType perform(Task task, TaskActionToolbox toolbox) throws IOException;
public boolean isAudited();
}

View File

@ -0,0 +1,15 @@
package com.metamx.druid.merger.common.config;
import org.skife.config.Config;
import org.skife.config.DefaultNull;
public abstract class TaskLogConfig
{
@Config("druid.merger.logs.s3bucket")
@DefaultNull
public abstract String getLogStorageBucket();
@Config("druid.merger.logs.s3prefix")
@DefaultNull
public abstract String getLogStoragePrefix();
}

View File

@ -27,8 +27,8 @@ import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.metamx.druid.Query;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.SegmentListUsedAction;
import com.metamx.druid.merger.common.actions.TaskActionClient;
import com.metamx.druid.query.QueryRunner;
import org.joda.time.Interval;
@ -96,11 +96,17 @@ public abstract class AbstractTask implements Task
}
@Override
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
public TaskStatus preflight(TaskActionClient taskActionClient) throws Exception
{
return TaskStatus.running(id);
}
@Override
public void shutdown()
{
// Do nothing.
}
@Override
public String toString()
{

View File

@ -92,7 +92,7 @@ public class DeleteTask extends AbstractTask
.shardSpec(new NoneShardSpec())
.build();
final File outDir = new File(toolbox.getTaskDir(), segment.getIdentifier());
final File outDir = new File(toolbox.getTaskWorkDir(), segment.getIdentifier());
final File fileToUpload = IndexMerger.merge(Lists.newArrayList(emptyAdapter), new AggregatorFactory[0], outDir);
// Upload the segment

View File

@ -109,7 +109,7 @@ public class IndexGeneratorTask extends AbstractTask
// Set up temporary directory for indexing
final File tmpDir = new File(
toolbox.getTaskDir(),
toolbox.getTaskWorkDir(),
String.format(
"%s_%s_%s_%s_%s",
this.getDataSource(),
@ -219,7 +219,7 @@ public class IndexGeneratorTask extends AbstractTask
return schema.getShardSpec().isInChunk(eventDimensions);
}
@JsonProperty
@JsonProperty("firehose")
public FirehoseFactory getFirehoseFactory()
{
return firehoseFactory;

View File

@ -31,6 +31,7 @@ import com.metamx.druid.indexer.granularity.GranularitySpec;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
import com.metamx.druid.merger.common.actions.TaskActionClient;
import com.metamx.druid.realtime.FirehoseFactory;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.shard.NoneShardSpec;
@ -144,9 +145,9 @@ public class IndexTask extends AbstractTask
}
@Override
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
public TaskStatus preflight(TaskActionClient taskActionClient) throws Exception
{
toolbox.getTaskActionClient().submit(new SpawnTasksAction(toSubtasks()));
taskActionClient.submit(new SpawnTasksAction(toSubtasks()));
return TaskStatus.success(getId());
}

View File

@ -42,6 +42,7 @@ import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.LockListAction;
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
import com.metamx.druid.merger.common.actions.TaskActionClient;
import com.metamx.druid.shard.NoneShardSpec;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
@ -121,7 +122,7 @@ public abstract class MergeTaskBase extends AbstractTask
final ServiceEmitter emitter = toolbox.getEmitter();
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments);
final File taskDir = toolbox.getTaskDir();
final File taskDir = toolbox.getTaskWorkDir();
try {
@ -185,7 +186,7 @@ public abstract class MergeTaskBase extends AbstractTask
* we are operating on every segment that overlaps the chosen interval.
*/
@Override
public TaskStatus preflight(TaskToolbox toolbox)
public TaskStatus preflight(TaskActionClient taskActionClient)
{
try {
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
@ -198,7 +199,7 @@ public abstract class MergeTaskBase extends AbstractTask
};
final Set<String> current = ImmutableSet.copyOf(
Iterables.transform(toolbox.getTaskActionClient().submit(defaultListUsedAction()), toIdentifier)
Iterables.transform(taskActionClient.submit(defaultListUsedAction()), toIdentifier)
);
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));

View File

@ -40,8 +40,8 @@ import com.metamx.druid.merger.common.actions.SegmentInsertAction;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.realtime.FireDepartmentConfig;
import com.metamx.druid.realtime.FireDepartmentMetrics;
import com.metamx.druid.realtime.Firehose;
import com.metamx.druid.realtime.FirehoseFactory;
import com.metamx.druid.realtime.GracefulShutdownFirehose;
import com.metamx.druid.realtime.plumber.Plumber;
import com.metamx.druid.realtime.plumber.RealtimePlumberSchool;
import com.metamx.druid.realtime.Schema;
@ -59,6 +59,8 @@ import java.io.IOException;
public class RealtimeIndexTask extends AbstractTask
{
private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class);
@JsonIgnore
final Schema schema;
@ -77,7 +79,14 @@ public class RealtimeIndexTask extends AbstractTask
@JsonIgnore
private volatile Plumber plumber = null;
private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class);
@JsonIgnore
private volatile GracefulShutdownFirehose firehose = null;
@JsonIgnore
private final Object lock = new Object();
@JsonIgnore
private volatile boolean shutdown = false;
@JsonCreator
public RealtimeIndexTask(
@ -142,13 +151,19 @@ public class RealtimeIndexTask extends AbstractTask
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
final Period intermediatePersistPeriod = fireDepartmentConfig.getIntermediatePersistPeriod();
final Firehose firehose = firehoseFactory.connect();
synchronized (lock) {
if (shutdown) {
return TaskStatus.success(getId());
}
firehose = new GracefulShutdownFirehose(firehoseFactory.connect(), segmentGranularity, windowPeriod);
}
// TODO -- Take PlumberSchool in constructor (although that will need jackson injectables for stuff like
// TODO -- the ServerView, which seems kind of odd?)
final RealtimePlumberSchool realtimePlumberSchool = new RealtimePlumberSchool(
windowPeriod,
new File(toolbox.getTaskDir(), "persist"),
new File(toolbox.getTaskWorkDir(), "persist"),
segmentGranularity
);
@ -175,7 +190,8 @@ public class RealtimeIndexTask extends AbstractTask
{
try {
toolbox.getSegmentAnnouncer().unannounceSegment(segment);
} finally {
}
finally {
toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval()));
}
}
@ -198,7 +214,8 @@ public class RealtimeIndexTask extends AbstractTask
.submit(new LockAcquireAction(interval));
return myLock.getVersion();
} catch (IOException e) {
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
@ -226,6 +243,9 @@ public class RealtimeIndexTask extends AbstractTask
final InputRow inputRow;
try {
inputRow = firehose.nextRow();
if (inputRow == null) {
continue;
}
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
if (sink == null) {
@ -264,21 +284,39 @@ public class RealtimeIndexTask extends AbstractTask
throw Throwables.propagate(e);
}
finally {
Closeables.closeQuietly(firehose);
if (normalExit) {
try {
plumber.persist(firehose.commit());
plumber.finishJob();
} catch(Exception e) {
}
catch (Exception e) {
log.makeAlert(e, "Failed to finish realtime task").emit();
}
finally {
Closeables.closeQuietly(firehose);
}
}
}
return TaskStatus.success(getId());
}
@Override
public void shutdown()
{
try {
synchronized (lock) {
shutdown = true;
if (firehose != null) {
firehose.shutdown();
}
}
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
@JsonProperty
public Schema getSchema()
{

View File

@ -25,6 +25,7 @@ import com.google.common.base.Optional;
import com.metamx.druid.Query;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.TaskActionClient;
import com.metamx.druid.query.QueryRunner;
import org.joda.time.Interval;
@ -37,8 +38,8 @@ import org.joda.time.Interval;
* <li>Tasks are each part of a "task group", which is a set of tasks that can share interval locks. These are
* useful for producing sharded segments.</li>
* <li>Tasks can optionally have an "implicit lock interval". Tasks with this property are guaranteed to have
* a lock on that interval during their {@link #preflight(com.metamx.druid.merger.common.TaskToolbox)} and
* {@link #run(com.metamx.druid.merger.common.TaskToolbox)} methods.</li>
* a lock on that interval during their {@link #preflight(com.metamx.druid.merger.common.actions.TaskActionClient)}
* and {@link #run(com.metamx.druid.merger.common.TaskToolbox)} methods.</li>
* <li>Tasks do not need to explicitly release locks; they are released upon task completion. Tasks may choose
* to release locks early if they desire.</li>
* </ul>
@ -97,14 +98,14 @@ public interface Task
* holding a lock on our dataSource and implicit lock interval (if any). If this method throws an exception, the
* task should be considered a failure.
*
* @param toolbox Toolbox for this task
* @param taskActionClient action client for this task (not the full toolbox)
*
* @return Some kind of status (runnable means continue on to a worker, non-runnable means we completed without
* using a worker).
*
* @throws Exception
*/
public TaskStatus preflight(TaskToolbox toolbox) throws Exception;
public TaskStatus preflight(TaskActionClient taskActionClient) throws Exception;
/**
* Execute a task. This typically runs on a worker as determined by a TaskRunner, and will be run while
@ -118,4 +119,10 @@ public interface Task
* @throws Exception
*/
public TaskStatus run(TaskToolbox toolbox) throws Exception;
/**
* Best-effort task cancellation. May or may not do anything. Calling this multiple times may have
* a stronger effect.
*/
public void shutdown();
}

View File

@ -135,15 +135,13 @@ public class VersionConverterTask extends AbstractTask
}
@Override
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
public TaskStatus preflight(TaskActionClient taskActionClient) throws Exception
{
if (segment != null) {
return super.preflight(toolbox);
return super.preflight(taskActionClient);
}
final TaskActionClient taskClient = toolbox.getTaskActionClient();
List<DataSegment> segments = taskClient.submit(defaultListUsedAction());
List<DataSegment> segments = taskActionClient.submit(defaultListUsedAction());
final FunctionalIterable<Task> tasks = FunctionalIterable
.create(segments)
@ -164,7 +162,7 @@ public class VersionConverterTask extends AbstractTask
}
);
taskClient.submit(new SpawnTasksAction(Lists.newArrayList(tasks)));
taskActionClient.submit(new SpawnTasksAction(Lists.newArrayList(tasks)));
return TaskStatus.success(getId());
}

View File

@ -0,0 +1,26 @@
package com.metamx.druid.merger.common.tasklogs;
import com.google.common.base.Optional;
import com.google.common.io.InputSupplier;
import com.metamx.common.logger.Logger;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
public class NoopTaskLogs implements TaskLogs
{
private final Logger log = new Logger(TaskLogs.class);
@Override
public Optional<InputSupplier<InputStream>> streamTaskLog(String taskid, long offset) throws IOException
{
return Optional.absent();
}
@Override
public void pushTaskLog(String taskid, File logFile) throws IOException
{
log.info("Not pushing logs for task: %s", taskid);
}
}

View File

@ -0,0 +1,110 @@
package com.metamx.druid.merger.common.tasklogs;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.io.InputSupplier;
import com.metamx.common.logger.Logger;
import org.jets3t.service.ServiceException;
import org.jets3t.service.StorageService;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.model.StorageObject;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
/**
* Provides task logs archived on S3.
*/
public class S3TaskLogs implements TaskLogs
{
private static final Logger log = new Logger(S3TaskLogs.class);
private final String bucket;
private final String prefix;
private final StorageService service;
public S3TaskLogs(String bucket, String prefix, RestS3Service service)
{
this.bucket = Preconditions.checkNotNull(bucket, "bucket");
this.prefix = Preconditions.checkNotNull(prefix, "prefix");
this.service = Preconditions.checkNotNull(service, "service");
}
@Override
public Optional<InputSupplier<InputStream>> streamTaskLog(final String taskid, final long offset) throws IOException
{
final String taskKey = getTaskLogKey(taskid);
try {
final StorageObject objectDetails = service.getObjectDetails(bucket, taskKey, null, null, null, null);
return Optional.<InputSupplier<InputStream>>of(
new InputSupplier<InputStream>()
{
@Override
public InputStream getInput() throws IOException
{
try {
final long start;
final long end = objectDetails.getContentLength() - 1;
if (offset > 0 && offset < objectDetails.getContentLength()) {
start = offset;
} else if (offset < 0 && (-1 * offset) < objectDetails.getContentLength()) {
start = objectDetails.getContentLength() + offset;
} else {
start = 0;
}
return service.getObject(
bucket,
taskKey,
null,
null,
new String[]{objectDetails.getETag()},
null,
start,
end
).getDataInputStream();
}
catch (ServiceException e) {
throw new IOException(e);
}
}
}
);
}
catch (ServiceException e) {
if (e.getErrorCode() != null && (e.getErrorCode().equals("NoSuchKey") || e.getErrorCode()
.equals("NoSuchBucket"))) {
return Optional.absent();
} else {
throw new IOException(String.format("Failed to stream logs from: %s", taskKey), e);
}
}
}
public void pushTaskLog(String taskid, File logFile) throws IOException
{
final String taskKey = getTaskLogKey(taskid);
try {
log.info("Pushing task log %s to: %s", logFile, taskKey);
final StorageObject object = new StorageObject(logFile);
object.setKey(taskKey);
service.putObject(bucket, object);
}
catch (Exception e) {
Throwables.propagateIfInstanceOf(e, IOException.class);
throw Throwables.propagate(e);
}
}
private String getTaskLogKey(String taskid)
{
return String.format("%s/%s/log", prefix, taskid);
}
}

View File

@ -0,0 +1,35 @@
package com.metamx.druid.merger.common.tasklogs;
import com.google.common.base.Optional;
import com.google.common.collect.ImmutableList;
import com.google.common.io.InputSupplier;
import java.io.IOException;
import java.io.InputStream;
import java.util.List;
/**
* Provides task logs based on a series of underlying task log providers.
*/
public class SwitchingTaskLogProvider implements TaskLogProvider
{
private final List<TaskLogProvider> providers;
public SwitchingTaskLogProvider(List<TaskLogProvider> providers)
{
this.providers = ImmutableList.copyOf(providers);
}
@Override
public Optional<InputSupplier<InputStream>> streamTaskLog(String taskid, long offset) throws IOException
{
for (TaskLogProvider provider : providers) {
final Optional<InputSupplier<InputStream>> stream = provider.streamTaskLog(taskid, offset);
if (stream.isPresent()) {
return stream;
}
}
return Optional.absent();
}
}

View File

@ -0,0 +1,23 @@
package com.metamx.druid.merger.common.tasklogs;
import com.google.common.base.Optional;
import com.google.common.io.InputSupplier;
import java.io.IOException;
import java.io.InputStream;
/**
* Something that knows how to stream logs for tasks.
*/
public interface TaskLogProvider
{
/**
* Stream log for a task.
*
* @param offset If zero, stream the entire log. If positive, attempt to read from this position onwards. If
* negative, attempt to read this many bytes from the end of the file (like <tt>tail -n</tt>).
*
* @return input supplier for this log, if available from this provider
*/
public Optional<InputSupplier<InputStream>> streamTaskLog(String taskid, long offset) throws IOException;
}

View File

@ -0,0 +1,12 @@
package com.metamx.druid.merger.common.tasklogs;
import java.io.File;
import java.io.IOException;
/**
* Something that knows how to persist local task logs to some form of long-term storage.
*/
public interface TaskLogPusher
{
public void pushTaskLog(String taskid, File logFile) throws IOException;
}

View File

@ -0,0 +1,5 @@
package com.metamx.druid.merger.common.tasklogs;
public interface TaskLogs extends TaskLogProvider, TaskLogPusher
{
}

View File

@ -21,21 +21,32 @@ package com.metamx.druid.merger.coordinator;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.Query;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.query.NoopQueryRunner;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.druid.query.segment.SegmentDescriptor;
import com.metamx.emitter.EmittingLogger;
import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.io.File;
import java.util.Collection;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ThreadPoolExecutor;
@ -43,22 +54,21 @@ import java.util.concurrent.ThreadPoolExecutor;
/**
* Runs tasks in a JVM thread using an ExecutorService.
*/
public class LocalTaskRunner implements TaskRunner
public class ExecutorServiceTaskRunner implements TaskRunner, QuerySegmentWalker
{
private final TaskToolboxFactory toolboxFactory;
private final ExecutorService exec;
private final ListeningExecutorService exec;
private final Set<TaskRunnerWorkItem> runningItems = new ConcurrentSkipListSet<TaskRunnerWorkItem>();
private static final Logger log = new Logger(LocalTaskRunner.class);
private static final EmittingLogger log = new EmittingLogger(ExecutorServiceTaskRunner.class);
public LocalTaskRunner(
public ExecutorServiceTaskRunner(
TaskToolboxFactory toolboxFactory,
ExecutorService exec
)
{
this.toolboxFactory = toolboxFactory;
this.exec = exec;
this.exec = MoreExecutors.listeningDecorator(exec);
}
@LifecycleStop
@ -68,11 +78,20 @@ public class LocalTaskRunner implements TaskRunner
}
@Override
public void run(final Task task, final TaskCallback callback)
public ListenableFuture<TaskStatus> run(final Task task)
{
final TaskToolbox toolbox = toolboxFactory.build(task);
return exec.submit(new ExecutorServiceTaskRunnerCallable(task, toolbox));
}
exec.submit(new LocalTaskRunnerRunnable(task, toolbox, callback));
@Override
public void shutdown(final String taskid)
{
for (final TaskRunnerWorkItem runningItem : runningItems) {
if (runningItem.getTask().getId().equals(taskid)) {
runningItem.getTask().shutdown();
}
}
}
@Override
@ -95,8 +114,8 @@ public class LocalTaskRunner implements TaskRunner
@Override
public TaskRunnerWorkItem apply(Runnable input)
{
if (input instanceof LocalTaskRunnerRunnable) {
return ((LocalTaskRunnerRunnable) input).getTaskRunnerWorkItem();
if (input instanceof ExecutorServiceTaskRunnerCallable) {
return ((ExecutorServiceTaskRunnerCallable) input).getTaskRunnerWorkItem();
}
return null;
}
@ -114,27 +133,72 @@ public class LocalTaskRunner implements TaskRunner
return Lists.newArrayList();
}
private static class LocalTaskRunnerRunnable implements Runnable
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<Interval> intervals)
{
return getQueryRunnerImpl(query);
}
@Override
public <T> QueryRunner<T> getQueryRunnerForSegments(Query<T> query, Iterable<SegmentDescriptor> specs)
{
return getQueryRunnerImpl(query);
}
private <T> QueryRunner<T> getQueryRunnerImpl(Query<T> query)
{
QueryRunner<T> queryRunner = null;
final List<Task> runningTasks = Lists.transform(
ImmutableList.copyOf(getRunningTasks()), new Function<TaskRunnerWorkItem, Task>()
{
@Override
public Task apply(TaskRunnerWorkItem o)
{
return o.getTask();
}
}
);
for (final Task task : runningTasks) {
if (task.getDataSource().equals(query.getDataSource())) {
final QueryRunner<T> taskQueryRunner = task.getQueryRunner(query);
if (taskQueryRunner != null) {
if (queryRunner == null) {
queryRunner = taskQueryRunner;
} else {
log.makeAlert("Found too many query runners for datasource")
.addData("dataSource", query.getDataSource())
.emit();
}
}
}
}
return queryRunner == null ? new NoopQueryRunner<T>() : queryRunner;
}
private static class ExecutorServiceTaskRunnerCallable implements Callable<TaskStatus>
{
private final Task task;
private final TaskToolbox toolbox;
private final TaskCallback callback;
private final DateTime createdTime;
public LocalTaskRunnerRunnable(Task task, TaskToolbox toolbox, TaskCallback callback)
public ExecutorServiceTaskRunnerCallable(Task task, TaskToolbox toolbox)
{
this.task = task;
this.toolbox = toolbox;
this.callback = callback;
this.createdTime = new DateTime();
}
@Override
public void run()
public TaskStatus call()
{
final long startTime = System.currentTimeMillis();
final File taskDir = toolbox.getTaskWorkDir();
TaskStatus status;
@ -156,20 +220,22 @@ public class LocalTaskRunner implements TaskRunner
}
try {
final File taskDir = toolbox.getTaskDir();
if (taskDir.exists()) {
log.info("Removing task directory: %s", taskDir);
FileUtils.deleteDirectory(taskDir);
}
}
catch (Exception e) {
log.error(e, "Failed to delete task directory: %s", task.getId());
log.makeAlert(e, "Failed to delete task directory")
.addData("taskDir", taskDir.toString())
.addData("task", task.getId())
.emit();
}
try {
callback.notify(status.withDuration(System.currentTimeMillis() - startTime));
} catch(Exception e) {
return status.withDuration(System.currentTimeMillis() - startTime);
}
catch (Exception e) {
log.error(e, "Uncaught Exception during callback for task[%s]", task);
throw Throwables.propagate(e);
}
@ -179,7 +245,7 @@ public class LocalTaskRunner implements TaskRunner
{
return new TaskRunnerWorkItem(
task,
callback,
null,
null,
createdTime
);

View File

@ -0,0 +1,422 @@
/*
* 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.databind.ObjectMapper;
import com.google.common.base.CharMatcher;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Splitter;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.Files;
import com.google.common.io.InputSupplier;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.metamx.common.ISE;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.common.tasklogs.TaskLogProvider;
import com.metamx.druid.merger.common.tasklogs.TaskLogPusher;
import com.metamx.druid.merger.coordinator.config.ForkingTaskRunnerConfig;
import com.metamx.druid.merger.worker.executor.ExecutorMain;
import com.metamx.emitter.EmittingLogger;
import org.apache.commons.io.FileUtils;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.io.RandomAccessFile;
import java.nio.channels.Channels;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.UUID;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Runs tasks in separate processes using {@link ExecutorMain}.
*/
public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
{
private static final EmittingLogger log = new EmittingLogger(ForkingTaskRunner.class);
private static final String CHILD_PROPERTY_PREFIX = "druid.indexer.fork.property.";
private final ForkingTaskRunnerConfig config;
private final Properties props;
private final TaskLogPusher taskLogPusher;
private final ListeningExecutorService exec;
private final ObjectMapper jsonMapper;
private final Map<String, TaskInfo> tasks = Maps.newHashMap();
public ForkingTaskRunner(
ForkingTaskRunnerConfig config,
Properties props,
TaskLogPusher taskLogPusher,
ExecutorService exec,
ObjectMapper jsonMapper
)
{
this.config = config;
this.props = props;
this.taskLogPusher = taskLogPusher;
this.exec = MoreExecutors.listeningDecorator(exec);
this.jsonMapper = jsonMapper;
}
@Override
public ListenableFuture<TaskStatus> run(final Task task)
{
synchronized (tasks) {
if (!tasks.containsKey(task.getId())) {
tasks.put(
task.getId(),
new TaskInfo(
exec.submit(
new Callable<TaskStatus>()
{
@Override
public TaskStatus call()
{
final String attemptUUID = UUID.randomUUID().toString();
final File taskDir = new File(config.getBaseTaskDir(), task.getId());
final File attemptDir = new File(taskDir, attemptUUID);
final ProcessHolder processHolder;
try {
if (!attemptDir.mkdirs()) {
throw new IOException(String.format("Could not create directories: %s", attemptDir));
}
final File taskFile = new File(attemptDir, "task.json");
final File statusFile = new File(attemptDir, "status.json");
final File logFile = new File(attemptDir, "log");
// time to adjust process holders
synchronized (tasks) {
if (Thread.interrupted()) {
throw new InterruptedException();
}
final TaskInfo taskInfo = tasks.get(task.getId());
if (taskInfo == null) {
throw new ISE("WTF?! TaskInfo disappeared for task: %s", task.getId());
}
if (taskInfo.processHolder != null) {
throw new ISE("WTF?! TaskInfo already has a process holder for task: %s", task.getId());
}
final List<String> command = Lists.newArrayList();
final int childPort = findUnusedPort();
final String childHost = String.format(config.getHostPattern(), childPort);
command.add(config.getJavaCommand());
command.add("-cp");
command.add(config.getJavaClasspath());
Iterables.addAll(
command,
Splitter.on(CharMatcher.WHITESPACE)
.omitEmptyStrings()
.split(config.getJavaOptions())
);
for (String propName : props.stringPropertyNames()) {
if (propName.startsWith(CHILD_PROPERTY_PREFIX)) {
command.add(
String.format(
"-D%s=%s",
propName.substring(CHILD_PROPERTY_PREFIX.length()),
System.getProperty(propName)
)
);
}
}
command.add(String.format("-Ddruid.host=%s", childHost));
command.add(String.format("-Ddruid.port=%d", childPort));
command.add(config.getMainClass());
command.add(taskFile.toString());
command.add(statusFile.toString());
jsonMapper.writeValue(taskFile, task);
log.info("Running command: %s", Joiner.on(" ").join(command));
taskInfo.processHolder = new ProcessHolder(
new ProcessBuilder(ImmutableList.copyOf(command)).redirectErrorStream(true).start(),
logFile,
childPort
);
processHolder = taskInfo.processHolder;
}
log.info("Logging task %s output to: %s", task.getId(), logFile);
final OutputStream toProc = processHolder.process.getOutputStream();
final InputStream fromProc = processHolder.process.getInputStream();
final OutputStream toLogfile = Files.newOutputStreamSupplier(logFile).getOutput();
boolean runFailed = false;
try {
ByteStreams.copy(fromProc, toLogfile);
final int statusCode = processHolder.process.waitFor();
log.info("Process exited with status[%d] for task: %s", statusCode, task.getId());
if (statusCode != 0) {
runFailed = true;
}
}
catch (Exception e) {
log.warn(e, "Failed to read from process for task: %s", task.getId());
runFailed = true;
}
finally {
Closeables.closeQuietly(fromProc);
Closeables.closeQuietly(toLogfile);
Closeables.closeQuietly(toProc);
}
// Upload task logs
// XXX: Consider uploading periodically for very long-lived tasks to prevent
// XXX: bottlenecks at the end or the possibility of losing a lot of logs all
// XXX: at once.
taskLogPusher.pushTaskLog(task.getId(), logFile);
if (!runFailed) {
// Process exited successfully
return jsonMapper.readValue(statusFile, TaskStatus.class);
} else {
// Process exited unsuccessfully
return TaskStatus.failure(task.getId());
}
}
catch (InterruptedException e) {
log.info(e, "Interrupted during execution");
return TaskStatus.failure(task.getId());
}
catch (IOException e) {
throw Throwables.propagate(e);
}
finally {
try {
synchronized (tasks) {
final TaskInfo taskInfo = tasks.remove(task.getId());
if (taskInfo != null && taskInfo.processHolder != null) {
taskInfo.processHolder.process.destroy();
}
}
log.info("Removing temporary directory: %s", attemptDir);
FileUtils.deleteDirectory(attemptDir);
}
catch (Exception e) {
log.error(e, "Suppressing exception caught while cleaning up task");
}
}
}
}
)
)
);
}
return tasks.get(task.getId()).statusFuture;
}
}
@LifecycleStop
public void stop()
{
synchronized (tasks) {
exec.shutdown();
for (TaskInfo taskInfo : tasks.values()) {
if (taskInfo.processHolder != null) {
log.info("Destroying process: %s", taskInfo.processHolder.process);
taskInfo.processHolder.process.destroy();
}
}
}
}
@Override
public void shutdown(final String taskid)
{
final TaskInfo taskInfo;
synchronized (tasks) {
taskInfo = tasks.get(taskid);
if (taskInfo == null) {
log.info("Ignoring request to cancel unknown task: %s", taskid);
return;
}
}
taskInfo.statusFuture.cancel(true);
if (taskInfo.processHolder != null) {
final int shutdowns = taskInfo.processHolder.shutdowns.getAndIncrement();
if (shutdowns == 0) {
log.info("Attempting to gracefully shutdown task: %s", taskid);
try {
// This is gross, but it may still be nicer than talking to the forked JVM via HTTP.
final OutputStream out = taskInfo.processHolder.process.getOutputStream();
out.write(
jsonMapper.writeValueAsBytes(
ImmutableMap.of(
"shutdown",
"now"
)
)
);
out.write('\n');
out.flush();
}
catch (IOException e) {
throw Throwables.propagate(e);
}
} else {
// Will trigger normal failure mechanisms due to process exit
log.info("Killing process for task: %s", taskid);
taskInfo.processHolder.process.destroy();
}
}
}
@Override
public Collection<TaskRunnerWorkItem> getRunningTasks()
{
return ImmutableList.of();
}
@Override
public Collection<TaskRunnerWorkItem> getPendingTasks()
{
return ImmutableList.of();
}
@Override
public Collection<ZkWorker> getWorkers()
{
return ImmutableList.of();
}
@Override
public Optional<InputSupplier<InputStream>> streamTaskLog(final String taskid, final long offset)
{
final ProcessHolder processHolder;
synchronized (tasks) {
final TaskInfo taskInfo = tasks.get(taskid);
if (taskInfo != null && taskInfo.processHolder != null) {
processHolder = taskInfo.processHolder;
} else {
return Optional.absent();
}
}
return Optional.<InputSupplier<InputStream>>of(
new InputSupplier<InputStream>()
{
@Override
public InputStream getInput() throws IOException
{
final RandomAccessFile raf = new RandomAccessFile(processHolder.logFile, "r");
final long rafLength = raf.length();
if (offset > 0) {
raf.seek(offset);
} else if (offset < 0 && offset < rafLength) {
raf.seek(rafLength + offset);
}
return Channels.newInputStream(raf.getChannel());
}
}
);
}
private int findUnusedPort()
{
synchronized (tasks) {
int port = config.getStartPort();
int maxPortSoFar = -1;
for (TaskInfo taskInfo : tasks.values()) {
if (taskInfo.processHolder != null) {
if (taskInfo.processHolder.port > maxPortSoFar) {
maxPortSoFar = taskInfo.processHolder.port;
}
if (taskInfo.processHolder.port == port) {
port = maxPortSoFar + 1;
}
}
}
return port;
}
}
private static class TaskInfo
{
private final ListenableFuture<TaskStatus> statusFuture;
private volatile ProcessHolder processHolder = null;
private TaskInfo(ListenableFuture<TaskStatus> statusFuture)
{
this.statusFuture = statusFuture;
}
}
private static class ProcessHolder
{
private final Process process;
private final File logFile;
private final int port;
private final AtomicInteger shutdowns = new AtomicInteger(0);
private ProcessHolder(Process process, File logFile, int port)
{
this.process = process;
this.logFile = logFile;
this.port = port;
}
}
}

View File

@ -20,25 +20,35 @@
package com.metamx.druid.merger.coordinator;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.google.common.collect.MinMaxPriorityQueue;
import com.google.common.collect.Sets;
import com.google.common.io.InputSupplier;
import com.google.common.primitives.Ints;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import com.metamx.common.ISE;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.merger.common.RetryPolicy;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.tasklogs.TaskLogProvider;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.worker.Worker;
import com.metamx.emitter.EmittingLogger;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.InputStreamResponseHandler;
import com.metamx.http.client.response.ToStringResponseHandler;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent;
@ -46,7 +56,12 @@ import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
import com.netflix.curator.utils.ZKPaths;
import org.apache.zookeeper.CreateMode;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import java.io.IOException;
import java.io.InputStream;
import java.net.MalformedURLException;
import java.net.URL;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
@ -54,6 +69,7 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
@ -72,10 +88,13 @@ import java.util.concurrent.atomic.AtomicReference;
* <p/>
* If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will automatically retry any tasks
* that were associated with the node.
* <p/>
* The RemoteTaskRunner uses ZK for job management and assignment and http for IPC messages.
*/
public class RemoteTaskRunner implements TaskRunner
public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
{
private static final EmittingLogger log = new EmittingLogger(RemoteTaskRunner.class);
private static final ToStringResponseHandler STRING_RESPONSE_HANDLER = new ToStringResponseHandler(Charsets.UTF_8);
private static final Joiner JOINER = Joiner.on("/");
private final ObjectMapper jsonMapper;
@ -85,6 +104,7 @@ public class RemoteTaskRunner implements TaskRunner
private final ScheduledExecutorService scheduledExec;
private final RetryPolicyFactory retryPolicyFactory;
private final AtomicReference<WorkerSetupData> workerSetupData;
private final HttpClient httpClient;
// all workers that exist in ZK
private final Map<String, ZkWorker> zkWorkers = new ConcurrentHashMap<String, ZkWorker>();
@ -106,7 +126,8 @@ public class RemoteTaskRunner implements TaskRunner
PathChildrenCache workerPathCache,
ScheduledExecutorService scheduledExec,
RetryPolicyFactory retryPolicyFactory,
AtomicReference<WorkerSetupData> workerSetupData
AtomicReference<WorkerSetupData> workerSetupData,
HttpClient httpClient
)
{
this.jsonMapper = jsonMapper;
@ -116,6 +137,7 @@ public class RemoteTaskRunner implements TaskRunner
this.scheduledExec = scheduledExec;
this.retryPolicyFactory = retryPolicyFactory;
this.workerSetupData = workerSetupData;
this.httpClient = httpClient;
}
@LifecycleStart
@ -211,21 +233,125 @@ public class RemoteTaskRunner implements TaskRunner
/**
* A task will be run only if there is no current knowledge in the RemoteTaskRunner of the task.
*
* @param task task to run
* @param callback callback to be called exactly once
* @param task task to run
*/
@Override
public void run(Task task, TaskCallback callback)
public ListenableFuture<TaskStatus> run(final Task task)
{
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());
}
TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem(
task, callback, retryPolicyFactory.makeRetryPolicy(), new DateTime()
task, SettableFuture.<TaskStatus>create(), retryPolicyFactory.makeRetryPolicy(), new DateTime()
);
addPendingTask(taskRunnerWorkItem);
return taskRunnerWorkItem.getResult();
}
/**
* Finds the worker running the task and forwards the shutdown signal to the worker.
*
* @param taskId
*/
@Override
public void shutdown(String taskId)
{
if (pendingTasks.containsKey(taskId)) {
pendingTasks.remove(taskId);
return;
}
final ZkWorker zkWorker = findWorkerRunningTask(taskId);
if (zkWorker == null) {
// Would be nice to have an ability to shut down pending tasks
log.info("Can't shutdown! No worker running task %s", taskId);
return;
}
final RetryPolicy shutdownRetryPolicy = retryPolicyFactory.makeRetryPolicy();
final URL url = workerURL(zkWorker.getWorker(), String.format("/task/%s/shutdown", taskId));
while (!shutdownRetryPolicy.hasExceededRetryThreshold()) {
try {
final String response = httpClient.post(url)
.go(STRING_RESPONSE_HANDLER)
.get();
log.info("Sent shutdown message to worker: %s, response: %s", zkWorker.getWorker().getHost(), response);
return;
}
catch (Exception e) {
log.error(e, "Exception shutting down taskId: %s", taskId);
if (shutdownRetryPolicy.hasExceededRetryThreshold()) {
throw Throwables.propagate(e);
} else {
try {
final long sleepTime = shutdownRetryPolicy.getAndIncrementRetryDelay().getMillis();
log.info("Will try again in %s.", new Duration(sleepTime).toString());
Thread.sleep(sleepTime);
}
catch (InterruptedException e2) {
throw Throwables.propagate(e2);
}
}
}
}
}
@Override
public Optional<InputSupplier<InputStream>> streamTaskLog(final String taskId, final long offset)
{
final ZkWorker zkWorker = findWorkerRunningTask(taskId);
if (zkWorker == null) {
// Worker is not running this task, it might be available in deep storage
return Optional.absent();
} else {
// Worker is still running this task
final URL url = workerURL(zkWorker.getWorker(), String.format("/task/%s/log?offset=%d", taskId, offset));
return Optional.<InputSupplier<InputStream>>of(
new InputSupplier<InputStream>()
{
@Override
public InputStream getInput() throws IOException
{
try {
return httpClient.get(url)
.go(new InputStreamResponseHandler())
.get();
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
}
catch (ExecutionException e) {
// Unwrap if possible
Throwables.propagateIfPossible(e.getCause(), IOException.class);
throw Throwables.propagate(e);
}
}
}
);
}
}
private URL workerURL(Worker worker, String path)
{
Preconditions.checkArgument(path.startsWith("/"), "path must start with '/': %s", path);
try {
return new URL(String.format("http://%s/mmx/worker/v1%s", worker.getHost(), path));
}
catch (MalformedURLException e) {
throw Throwables.propagate(e);
}
}
/**
* Adds a task to the pending queue
* @param taskRunnerWorkItem
*/
private void addPendingTask(final TaskRunnerWorkItem taskRunnerWorkItem)
{
log.info("Added pending task %s", taskRunnerWorkItem.getTask().getId());
@ -447,9 +573,9 @@ public class RemoteTaskRunner implements TaskRunner
if (taskStatus.isComplete()) {
if (taskRunnerWorkItem != null) {
final TaskCallback callback = taskRunnerWorkItem.getCallback();
if (callback != null) {
callback.notify(taskStatus);
final SettableFuture<TaskStatus> result = taskRunnerWorkItem.getResult();
if (result != null) {
result.set(taskStatus);
}
}
@ -571,9 +697,4 @@ public class RemoteTaskRunner implements TaskRunner
throw Throwables.propagate(e);
}
}
public static void main(String[] args)
{
System.out.println("2013-03-11".compareTo("0"));
}
}

View File

@ -19,14 +19,15 @@
package com.metamx.druid.merger.coordinator;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.TaskActionClient;
import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.exec.TaskConsumer;
@ -51,7 +52,7 @@ public class TaskMasterLifecycle
private final ReentrantLock giant = new ReentrantLock();
private final Condition mayBeStopped = giant.newCondition();
private final TaskQueue taskQueue;
private final TaskToolboxFactory taskToolboxFactory;
private final TaskActionClientFactory taskActionClientFactory;
private volatile boolean leading = false;
private volatile TaskRunner taskRunner;
@ -61,7 +62,7 @@ public class TaskMasterLifecycle
public TaskMasterLifecycle(
final TaskQueue taskQueue,
final TaskToolboxFactory taskToolboxFactory,
final TaskActionClientFactory taskActionClientFactory,
final IndexerCoordinatorConfig indexerCoordinatorConfig,
final ServiceDiscoveryConfig serviceDiscoveryConfig,
final TaskRunnerFactory runnerFactory,
@ -71,7 +72,7 @@ public class TaskMasterLifecycle
)
{
this.taskQueue = taskQueue;
this.taskToolboxFactory = taskToolboxFactory;
this.taskActionClientFactory = taskActionClientFactory;
this.leaderSelector = new LeaderSelector(
curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener()
@ -89,7 +90,7 @@ public class TaskMasterLifecycle
final TaskConsumer taskConsumer = new TaskConsumer(
taskQueue,
taskRunner,
taskToolboxFactory,
taskActionClientFactory,
emitter
);
@ -104,17 +105,20 @@ public class TaskMasterLifecycle
leaderLifecycle.addManagedInstance(taskConsumer);
leaderLifecycle.addManagedInstance(resourceManagementScheduler);
leading = true;
try {
leaderLifecycle.start();
leading = true;
while (leading) {
while (leading && !Thread.currentThread().isInterrupted()) {
mayBeStopped.await();
}
}
catch (InterruptedException e) {
// Suppress so we can bow out gracefully
}
finally {
log.info("Bowing out!");
stopLeading();
leaderLifecycle.stop();
}
}
@ -209,23 +213,39 @@ public class TaskMasterLifecycle
}
}
public TaskRunner getTaskRunner()
public Optional<TaskRunner> getTaskRunner()
{
return taskRunner;
if (leading) {
return Optional.of(taskRunner);
} else {
return Optional.absent();
}
}
public TaskQueue getTaskQueue()
public Optional<TaskQueue> getTaskQueue()
{
return taskQueue;
if (leading) {
return Optional.of(taskQueue);
} else {
return Optional.absent();
}
}
public TaskToolbox getTaskToolbox(Task task)
public Optional<TaskActionClient> getTaskActionClient(Task task)
{
return taskToolboxFactory.build(task);
if (leading) {
return Optional.of(taskActionClientFactory.create(task));
} else {
return Optional.absent();
}
}
public ResourceManagementScheduler getResourceManagementScheduler()
public Optional<ResourceManagementScheduler> getResourceManagementScheduler()
{
return resourceManagementScheduler;
if (leading) {
return Optional.of(resourceManagementScheduler);
} else {
return Optional.absent();
}
}
}

View File

@ -80,6 +80,11 @@ public class TaskQueue
*/
public void bootstrap()
{
// NOTE: Bootstraps can resurrect bogus stuff caused by leader races or whatevs.
// We may want to periodically fixup the database to refer to what we think is happening, to prevent
// this from occurring and also so that bogus stuff is detected by clients in a timely manner.
giant.lock();
try {

View File

@ -19,7 +19,8 @@
package com.metamx.druid.merger.coordinator;
import com.metamx.druid.merger.common.TaskCallback;
import com.google.common.util.concurrent.ListenableFuture;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.task.Task;
import java.util.Collection;
@ -31,13 +32,18 @@ import java.util.Collection;
public interface TaskRunner
{
/**
* Run a task with a particular context and call a callback. The callback may be called multiple times with RUNNING
* status, but should be called exactly once with a non-RUNNING status (e.g. SUCCESS, FAILED, CONTINUED...).
* Run a task. The returned status should be some kind of completed status.
*
* @param task task to run
* @param callback callback to be called exactly once
* @return task status, eventually
*/
public void run(Task task, TaskCallback callback);
public ListenableFuture<TaskStatus> run(Task task);
/**
* Best-effort task cancellation. May or may not do anything. Calling this multiple times may have
* a stronger effect.
*/
public void shutdown(String taskid);
public Collection<TaskRunnerWorkItem> getRunningTasks();

View File

@ -20,8 +20,9 @@
package com.metamx.druid.merger.coordinator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.util.concurrent.SettableFuture;
import com.metamx.druid.merger.common.RetryPolicy;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.task.Task;
import org.joda.time.DateTime;
import org.joda.time.DateTimeComparator;
@ -32,7 +33,7 @@ import org.joda.time.DateTimeComparator;
public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
{
private final Task task;
private final TaskCallback callback;
private final SettableFuture<TaskStatus> result;
private final RetryPolicy retryPolicy;
private final DateTime createdTime;
@ -40,13 +41,13 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
public TaskRunnerWorkItem(
Task task,
TaskCallback callback,
SettableFuture<TaskStatus> result,
RetryPolicy retryPolicy,
DateTime createdTime
)
{
this.task = task;
this.callback = callback;
this.result = result;
this.retryPolicy = retryPolicy;
this.createdTime = createdTime;
}
@ -57,9 +58,9 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
return task;
}
public TaskCallback getCallback()
public SettableFuture<TaskStatus> getResult()
{
return callback;
return result;
}
public RetryPolicy getRetryPolicy()
@ -91,46 +92,12 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
return DateTimeComparator.getInstance().compare(createdTime, taskRunnerWorkItem.getCreatedTime());
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TaskRunnerWorkItem that = (TaskRunnerWorkItem) o;
if (callback != null ? !callback.equals(that.callback) : that.callback != null) {
return false;
}
if (retryPolicy != null ? !retryPolicy.equals(that.retryPolicy) : that.retryPolicy != null) {
return false;
}
if (task != null ? !task.equals(that.task) : that.task != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = task != null ? task.hashCode() : 0;
result = 31 * result + (callback != null ? callback.hashCode() : 0);
result = 31 * result + (retryPolicy != null ? retryPolicy.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "TaskRunnerWorkItem{" +
"task=" + task +
", callback=" + callback +
", result=" + result +
", retryPolicy=" + retryPolicy +
", createdTime=" + createdTime +
'}';

View File

@ -0,0 +1,38 @@
package com.metamx.druid.merger.coordinator.config;
import com.metamx.druid.merger.worker.executor.ExecutorMain;
import org.skife.config.Config;
import org.skife.config.Default;
import java.io.File;
public abstract class ForkingTaskRunnerConfig
{
@Config("druid.merger.taskDir")
public abstract File getBaseTaskDir();
@Config("druid.indexer.fork.java")
@Default("java")
public abstract String getJavaCommand();
@Config("druid.indexer.fork.opts")
@Default("")
public abstract String getJavaOptions();
@Config("druid.indexer.fork.classpath")
public String getJavaClasspath() {
return System.getProperty("java.class.path");
}
@Config("druid.indexer.fork.main")
public String getMainClass()
{
return ExecutorMain.class.getName();
}
@Config("druid.indexer.fork.hostpattern")
public abstract String getHostPattern();
@Config("druid.indexer.fork.startport")
public abstract int getStartPort();
}

View File

@ -23,6 +23,7 @@ import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableSet;
import org.skife.config.Config;
import org.skife.config.Default;
import org.skife.config.DefaultNull;
import java.util.Set;

View File

@ -20,11 +20,13 @@
package com.metamx.druid.merger.coordinator.exec;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskQueue;
import com.metamx.druid.merger.coordinator.TaskRunner;
@ -36,7 +38,7 @@ public class TaskConsumer implements Runnable
{
private final TaskQueue queue;
private final TaskRunner runner;
private final TaskToolboxFactory toolboxFactory;
private final TaskActionClientFactory taskActionClientFactory;
private final ServiceEmitter emitter;
private final Thread thready;
@ -47,13 +49,13 @@ public class TaskConsumer implements Runnable
public TaskConsumer(
TaskQueue queue,
TaskRunner runner,
TaskToolboxFactory toolboxFactory,
TaskActionClientFactory taskActionClientFactory,
ServiceEmitter emitter
)
{
this.queue = queue;
this.runner = runner;
this.toolboxFactory = toolboxFactory;
this.taskActionClientFactory = taskActionClientFactory;
this.emitter = emitter;
this.thready = new Thread(this);
}
@ -84,8 +86,9 @@ public class TaskConsumer implements Runnable
task = queue.take();
}
catch (InterruptedException e) {
log.info(e, "Interrupted while waiting for new work");
throw e;
log.info("Interrupted while waiting for new work");
Thread.currentThread().interrupt();
break;
}
try {
@ -123,7 +126,7 @@ public class TaskConsumer implements Runnable
// Run preflight checks
TaskStatus preflightStatus;
try {
preflightStatus = task.preflight(toolboxFactory.build(task));
preflightStatus = task.preflight(taskActionClientFactory.create(task));
log.info("Preflight done for task: %s", task.getId());
}
catch (Exception e) {
@ -138,15 +141,34 @@ public class TaskConsumer implements Runnable
}
// Hand off work to TaskRunner, with a callback
runner.run(
task, new TaskCallback()
final ListenableFuture<TaskStatus> status = runner.run(task);
Futures.addCallback(
status, new FutureCallback<TaskStatus>()
{
@Override
public void notify(final TaskStatus statusFromRunner)
public void onSuccess(final TaskStatus status)
{
log.info("Received %s status for task: %s", status.getStatusCode(), task);
handleStatus(status);
}
@Override
public void onFailure(Throwable t)
{
log.makeAlert(t, "Failed to run task")
.addData("task", task.getId())
.addData("type", task.getType())
.addData("dataSource", task.getDataSource())
.addData("interval", task.getImplicitLockInterval())
.emit();
handleStatus(TaskStatus.failure(task.getId()));
}
private void handleStatus(TaskStatus status)
{
try {
log.info("Received %s status for task: %s", statusFromRunner.getStatusCode(), task);
// If we're not supposed to be running anymore, don't do anything. Somewhat racey if the flag gets set after
// we check and before we commit the database transaction, but better than nothing.
if (shutdown) {
@ -154,34 +176,25 @@ public class TaskConsumer implements Runnable
return;
}
queue.notify(task, statusFromRunner);
queue.notify(task, status);
// Emit event and log, if the task is done
if (statusFromRunner.isComplete()) {
metricBuilder.setUser3(statusFromRunner.getStatusCode().toString());
emitter.emit(metricBuilder.build("indexer/time/run/millis", statusFromRunner.getDuration()));
if (statusFromRunner.isFailure()) {
log.makeAlert("Failed to index")
.addData("task", task.getId())
.addData("type", task.getType())
.addData("dataSource", task.getDataSource())
.addData("interval", task.getImplicitLockInterval())
.emit();
}
if (status.isComplete()) {
metricBuilder.setUser3(status.getStatusCode().toString());
emitter.emit(metricBuilder.build("indexer/time/run/millis", status.getDuration()));
log.info(
"Task %s: %s (%d run duration)",
statusFromRunner.getStatusCode(),
status.getStatusCode(),
task,
statusFromRunner.getDuration()
status.getDuration()
);
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to handle task callback")
log.makeAlert(e, "Failed to handle task status")
.addData("task", task.getId())
.addData("statusCode", statusFromRunner.getStatusCode())
.addData("statusCode", status.getStatusCode())
.emit();
}
}

View File

@ -21,13 +21,14 @@ package com.metamx.druid.merger.coordinator.http;
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.services.ec2.AmazonEC2Client;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.base.Charsets;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.io.InputSupplier;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.inject.Guice;
import com.google.inject.Injector;
@ -40,11 +41,7 @@ import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.BaseServerNode;
import com.metamx.druid.client.ClientConfig;
import com.metamx.druid.client.ClientInventoryManager;
import com.metamx.druid.client.MutableServerView;
import com.metamx.druid.client.OnlyNewSegmentWatcherServerView;
import com.metamx.druid.RegisteringNode;
import com.metamx.druid.config.ConfigManager;
import com.metamx.druid.config.ConfigManagerConfig;
import com.metamx.druid.config.JacksonConfigManager;
@ -56,23 +53,24 @@ import com.metamx.druid.http.RedirectInfo;
import com.metamx.druid.http.StatusServlet;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServerInit;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentKiller;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory;
import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
import com.metamx.druid.merger.common.actions.TaskActionToolbox;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.config.TaskLogConfig;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
import com.metamx.druid.merger.common.tasklogs.NoopTaskLogs;
import com.metamx.druid.merger.common.tasklogs.S3TaskLogs;
import com.metamx.druid.merger.common.tasklogs.SwitchingTaskLogProvider;
import com.metamx.druid.merger.common.tasklogs.TaskLogProvider;
import com.metamx.druid.merger.common.tasklogs.TaskLogs;
import com.metamx.druid.merger.coordinator.DbTaskStorage;
import com.metamx.druid.merger.coordinator.ForkingTaskRunner;
import com.metamx.druid.merger.coordinator.HeapMemoryTaskStorage;
import com.metamx.druid.merger.coordinator.LocalTaskRunner;
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
import com.metamx.druid.merger.coordinator.RemoteTaskRunner;
import com.metamx.druid.merger.coordinator.TaskLockbox;
@ -83,6 +81,7 @@ import com.metamx.druid.merger.coordinator.TaskRunnerFactory;
import com.metamx.druid.merger.coordinator.TaskStorage;
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig;
import com.metamx.druid.merger.coordinator.config.ForkingTaskRunnerConfig;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
@ -95,9 +94,6 @@ import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFa
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy;
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncerConfig;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.Emitters;
@ -126,6 +122,8 @@ import org.mortbay.resource.ResourceCollection;
import org.skife.config.ConfigurationObjectFactory;
import org.skife.jdbi.v2.DBI;
import java.io.IOException;
import java.io.InputStream;
import java.net.URL;
import java.util.List;
import java.util.Properties;
@ -136,7 +134,7 @@ import java.util.concurrent.atomic.AtomicReference;
/**
*/
public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNode>
public class IndexerCoordinatorNode extends RegisteringNode
{
private static final Logger log = new Logger(IndexerCoordinatorNode.class);
@ -146,29 +144,29 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
}
private final Lifecycle lifecycle;
private final ObjectMapper jsonMapper;
private final Properties props;
private final ConfigurationObjectFactory configFactory;
private RestS3Service s3Service = null;
private List<Monitor> monitors = null;
private ServiceEmitter emitter = null;
private DbConnectorConfig dbConnectorConfig = null;
private DBI dbi = null;
private RestS3Service s3Service = null;
private IndexerCoordinatorConfig config = null;
private TaskConfig taskConfig = null;
private DataSegmentPusher segmentPusher = null;
private TaskToolboxFactory taskToolboxFactory = null;
private MergerDBCoordinator mergerDBCoordinator = null;
private TaskStorage taskStorage = null;
private TaskQueue taskQueue = null;
private TaskLockbox taskLockbox = null;
private CuratorFramework curatorFramework = null;
private ScheduledExecutorFactory scheduledExecutorFactory = null;
private IndexerZkConfig indexerZkConfig;
private TaskRunnerFactory taskRunnerFactory = null;
private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null;
private HttpClient httpClient = null;
private TaskActionClientFactory taskActionClientFactory = null;
private TaskMasterLifecycle taskMasterLifecycle = null;
private MutableServerView newSegmentServerView = null;
private TaskLogs persistentTaskLogs = null;
private TaskLogProvider taskLogProvider = null;
private Server server = null;
private boolean initialized = false;
@ -177,14 +175,14 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
Properties props,
Lifecycle lifecycle,
ObjectMapper jsonMapper,
ObjectMapper smileMapper,
ConfigurationObjectFactory configFactory
)
{
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
super(ImmutableList.of(jsonMapper));
this.lifecycle = lifecycle;
this.props = props;
this.jsonMapper = jsonMapper;
this.configFactory = configFactory;
}
@ -200,33 +198,21 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
return this;
}
public IndexerCoordinatorNode setTaskQueue(TaskQueue taskQueue)
{
this.taskQueue = taskQueue;
return this;
}
public IndexerCoordinatorNode setNewSegmentServerView(MutableServerView newSegmentServerView)
{
this.newSegmentServerView = newSegmentServerView;
return this;
}
public IndexerCoordinatorNode setS3Service(RestS3Service s3Service)
{
this.s3Service = s3Service;
return this;
}
public IndexerCoordinatorNode setTaskLockbox(TaskLockbox taskLockbox)
public IndexerCoordinatorNode setTaskQueue(TaskQueue taskQueue)
{
this.taskLockbox = taskLockbox;
this.taskQueue = taskQueue;
return this;
}
public IndexerCoordinatorNode setSegmentPusher(DataSegmentPusher segmentPusher)
public IndexerCoordinatorNode setTaskLockbox(TaskLockbox taskLockbox)
{
this.segmentPusher = segmentPusher;
this.taskLockbox = taskLockbox;
return this;
}
@ -254,9 +240,15 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
return this;
}
public IndexerCoordinatorNode setHttpClient(HttpClient httpClient)
{
this.httpClient = httpClient;
return this;
}
public void doInit() throws Exception
{
scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
initializeDB();
final ConfigManagerConfig managerConfig = configFactory.build(ConfigManagerConfig.class);
@ -271,25 +263,23 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
), getJsonMapper()
);
initializeHttpClient();
initializeEmitter();
initializeMonitors();
initializeIndexerCoordinatorConfig();
initializeTaskConfig();
initializeS3Service();
initializeMergeDBCoordinator();
initializeNewSegmentServerView();
initializeTaskStorage();
initializeTaskLockbox();
initializeTaskQueue();
initializeDataSegmentPusher();
initializeTaskToolbox();
initializeJacksonInjections();
initializeJacksonSubtypes();
initializeCurator();
initializeIndexerZkConfig();
initializeTaskActionClientFactory();
initializeTaskRunnerFactory(configManager);
initializeResourceManagement(configManager);
initializeTaskMasterLifecycle();
initializePersistentTaskLogs();
initializeTaskLogProvider();
initializeServer();
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
@ -308,6 +298,7 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
emitter,
taskMasterLifecycle,
new TaskStorageQueryAdapter(taskStorage),
taskLogProvider,
configManager
)
);
@ -367,13 +358,28 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
initialized = true;
}
private ObjectMapper getJsonMapper()
{
return jsonMapper;
}
private void initializeTaskActionClientFactory()
{
if (taskActionClientFactory == null) {
taskActionClientFactory = new LocalTaskActionClientFactory(
taskStorage,
new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter)
);
}
}
private void initializeTaskMasterLifecycle()
{
if (taskMasterLifecycle == null) {
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
taskMasterLifecycle = new TaskMasterLifecycle(
taskQueue,
taskToolboxFactory,
taskActionClientFactory,
config,
serviceDiscoveryConfig,
taskRunnerFactory,
@ -385,11 +391,57 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
}
}
private void initializePersistentTaskLogs() throws S3ServiceException
{
if (persistentTaskLogs == null) {
final TaskLogConfig taskLogConfig = configFactory.build(TaskLogConfig.class);
if (taskLogConfig.getLogStorageBucket() != null) {
initializeS3Service();
persistentTaskLogs = new S3TaskLogs(
taskLogConfig.getLogStorageBucket(),
taskLogConfig.getLogStoragePrefix(),
s3Service
);
} else {
persistentTaskLogs = new NoopTaskLogs();
}
}
}
private void initializeTaskLogProvider()
{
if (taskLogProvider == null) {
final List<TaskLogProvider> providers = Lists.newArrayList();
// Use our TaskRunner if it is also a TaskLogProvider
providers.add(
new TaskLogProvider()
{
@Override
public Optional<InputSupplier<InputStream>> streamTaskLog(String taskid, long offset) throws IOException
{
final TaskRunner runner = taskMasterLifecycle.getTaskRunner().orNull();
if (runner instanceof TaskLogProvider) {
return ((TaskLogProvider) runner).streamTaskLog(taskid, offset);
} else {
return Optional.absent();
}
}
}
);
// Use our persistent log storage
providers.add(persistentTaskLogs);
taskLogProvider = new SwitchingTaskLogProvider(providers);
}
}
@LifecycleStart
public synchronized void start() throws Exception
{
if (!initialized) {
init();
doInit();
}
lifecycle.start();
@ -432,25 +484,15 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
}
}
private void initializeJacksonInjections()
{
InjectableValues.Std injectables = new InjectableValues.Std();
injectables.addValue("s3Client", s3Service)
.addValue("segmentPusher", segmentPusher);
getJsonMapper().setInjectableValues(injectables);
}
private void initializeJacksonSubtypes()
{
getJsonMapper().registerSubtypes(StaticS3FirehoseFactory.class);
}
private void initializeEmitter()
private void initializeHttpClient()
{
if (emitter == null) {
final HttpClient httpClient = HttpClientInit.createClient(
if (httpClient == null) {
httpClient = HttpClientInit.createClient(
HttpClientConfig.builder().withNumConnections(1).withReadTimeout(
new Duration(
PropUtils.getProperty(
@ -460,7 +502,12 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
)
).build(), lifecycle
);
}
}
private void initializeEmitter()
{
if (emitter == null) {
emitter = new ServiceEmitter(
PropUtils.getProperty(props, "druid.service"),
PropUtils.getProperty(props, "druid.host"),
@ -470,6 +517,18 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
EmittingLogger.registerEmitter(emitter);
}
private void initializeS3Service() throws S3ServiceException
{
if(s3Service == null) {
s3Service = new RestS3Service(
new AWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
)
);
}
}
private void initializeMonitors()
{
if (monitors == null) {
@ -496,71 +555,6 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
}
}
private void initializeTaskConfig()
{
if (taskConfig == null) {
taskConfig = configFactory.build(TaskConfig.class);
}
}
private void initializeNewSegmentServerView()
{
if (newSegmentServerView == null) {
final MutableServerView view = new OnlyNewSegmentWatcherServerView();
final ClientInventoryManager clientInventoryManager = new ClientInventoryManager(
getConfigFactory().build(ClientConfig.class),
getPhoneBook(),
view
);
lifecycle.addManagedInstance(clientInventoryManager);
this.newSegmentServerView = view;
}
}
public void initializeS3Service() throws S3ServiceException
{
this.s3Service = new RestS3Service(
new AWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
)
);
}
public void initializeDataSegmentPusher()
{
if (segmentPusher == null) {
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, getJsonMapper());
}
}
public void initializeTaskToolbox()
{
if (taskToolboxFactory == null) {
final SegmentAnnouncer segmentAnnouncer = new ZkSegmentAnnouncer(
configFactory.build(ZkSegmentAnnouncerConfig.class),
getPhoneBook()
);
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
taskToolboxFactory = new TaskToolboxFactory(
taskConfig,
new LocalTaskActionClientFactory(
taskStorage,
new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter)
),
emitter,
s3Service,
segmentPusher,
dataSegmentKiller,
segmentAnnouncer,
newSegmentServerView,
getConglomerate(),
getJsonMapper()
);
}
}
public void initializeMergeDBCoordinator()
{
if (mergerDBCoordinator == null) {
@ -642,7 +636,7 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
.build()
);
RemoteTaskRunner remoteTaskRunner = new RemoteTaskRunner(
return new RemoteTaskRunner(
getJsonMapper(),
configFactory.build(RemoteTaskRunnerConfig.class),
curatorFramework,
@ -654,10 +648,9 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
ImmutableMap.of("base_path", "druid.indexing")
)
),
configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class)
configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class),
httpClient
);
return remoteTaskRunner;
}
};
@ -668,7 +661,13 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
public TaskRunner build()
{
final ExecutorService runnerExec = Executors.newFixedThreadPool(config.getNumLocalThreads());
return new LocalTaskRunner(taskToolboxFactory, runnerExec);
return new ForkingTaskRunner(
configFactory.build(ForkingTaskRunnerConfig.class),
props,
persistentTaskLogs,
runnerExec,
getJsonMapper()
);
}
};
} else {
@ -764,16 +763,8 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
public IndexerCoordinatorNode build()
{
if (jsonMapper == null && smileMapper == null) {
if (jsonMapper == null) {
jsonMapper = new DefaultObjectMapper();
smileMapper = new DefaultObjectMapper(new SmileFactory());
smileMapper.getJsonFactory().setCodec(smileMapper);
} else if (jsonMapper == null || smileMapper == null) {
throw new ISE(
"Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.",
jsonMapper,
smileMapper
);
}
if (lifecycle == null) {
@ -788,7 +779,7 @@ public class IndexerCoordinatorNode extends BaseServerNode<IndexerCoordinatorNod
configFactory = Config.createFactory(props);
}
return new IndexerCoordinatorNode(props, lifecycle, jsonMapper, smileMapper, configFactory);
return new IndexerCoordinatorNode(props, lifecycle, jsonMapper, configFactory);
}
}
}

View File

@ -21,30 +21,40 @@ package com.metamx.druid.merger.coordinator.http;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.common.io.InputSupplier;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.merger.common.tasklogs.TaskLogProvider;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.actions.TaskActionClient;
import com.metamx.druid.merger.common.actions.TaskActionHolder;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
import com.metamx.druid.merger.coordinator.TaskQueue;
import com.metamx.druid.merger.coordinator.TaskRunner;
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.emitter.service.ServiceEmitter;
import javax.ws.rs.Consumes;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Response;
import java.io.IOException;
import java.io.InputStream;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
@ -60,6 +70,7 @@ public class IndexerCoordinatorResource
private final ServiceEmitter emitter;
private final TaskMasterLifecycle taskMasterLifecycle;
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
private final TaskLogProvider taskLogProvider;
private final JacksonConfigManager configManager;
private final ObjectMapper jsonMapper;
@ -71,6 +82,7 @@ public class IndexerCoordinatorResource
ServiceEmitter emitter,
TaskMasterLifecycle taskMasterLifecycle,
TaskStorageQueryAdapter taskStorageQueryAdapter,
TaskLogProvider taskLogProvider,
JacksonConfigManager configManager,
ObjectMapper jsonMapper
) throws Exception
@ -79,6 +91,7 @@ public class IndexerCoordinatorResource
this.emitter = emitter;
this.taskMasterLifecycle = taskMasterLifecycle;
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
this.taskLogProvider = taskLogProvider;
this.configManager = configManager;
this.jsonMapper = jsonMapper;
}
@ -108,8 +121,18 @@ public class IndexerCoordinatorResource
@Produces("application/json")
public Response taskPost(final Task task)
{
taskMasterLifecycle.getTaskQueue().add(task);
return Response.ok(ImmutableMap.of("task", task.getId())).build();
return asLeaderWith(
taskMasterLifecycle.getTaskQueue(),
new Function<TaskQueue, Response>()
{
@Override
public Response apply(TaskQueue taskQueue)
{
taskQueue.add(task);
return Response.ok(ImmutableMap.of("task", task.getId())).build();
}
}
);
}
@GET
@ -134,6 +157,25 @@ public class IndexerCoordinatorResource
return Response.ok().entity(segments).build();
}
@POST
@Path("/task/{taskid}/shutdown")
@Produces("application/json")
public Response doShutdown(@PathParam("taskid") final String taskid)
{
return asLeaderWith(
taskMasterLifecycle.getTaskRunner(),
new Function<TaskRunner, Response>()
{
@Override
public Response apply(TaskRunner taskRunner)
{
taskRunner.shutdown(taskid);
return Response.ok(ImmutableMap.of("task", taskid)).build();
}
}
);
}
// Legacy endpoint
// TODO Remove
@Deprecated
@ -188,19 +230,30 @@ public class IndexerCoordinatorResource
@Produces("application/json")
public <T> Response doAction(final TaskActionHolder<T> holder)
{
final Map<String, Object> retMap;
return asLeaderWith(
taskMasterLifecycle.getTaskActionClient(holder.getTask()),
new Function<TaskActionClient, Response>()
{
@Override
public Response apply(TaskActionClient taskActionClient)
{
final Map<String, Object> retMap;
try {
final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask())
.getTaskActionClient()
.submit(holder.getAction());
retMap = Maps.newHashMap();
retMap.put("result", ret);
} catch(IOException e) {
return Response.serverError().build();
}
// TODO make sure this worker is supposed to be running this task (attempt id? token?)
return Response.ok().entity(retMap).build();
try {
final T ret = taskActionClient.submit(holder.getAction());
retMap = Maps.newHashMap();
retMap.put("result", ret);
}
catch (IOException e) {
return Response.serverError().build();
}
return Response.ok().entity(retMap).build();
}
}
);
}
@GET
@ -208,10 +261,17 @@ public class IndexerCoordinatorResource
@Produces("application/json")
public Response getPendingTasks()
{
if (taskMasterLifecycle.getTaskRunner() == null) {
return Response.noContent().build();
}
return Response.ok(taskMasterLifecycle.getTaskRunner().getPendingTasks()).build();
return asLeaderWith(
taskMasterLifecycle.getTaskRunner(),
new Function<TaskRunner, Response>()
{
@Override
public Response apply(TaskRunner taskRunner)
{
return Response.ok(taskRunner.getPendingTasks()).build();
}
}
);
}
@GET
@ -219,10 +279,17 @@ public class IndexerCoordinatorResource
@Produces("application/json")
public Response getRunningTasks()
{
if (taskMasterLifecycle.getTaskRunner() == null) {
return Response.noContent().build();
}
return Response.ok(taskMasterLifecycle.getTaskRunner().getRunningTasks()).build();
return asLeaderWith(
taskMasterLifecycle.getTaskRunner(),
new Function<TaskRunner, Response>()
{
@Override
public Response apply(TaskRunner taskRunner)
{
return Response.ok(taskRunner.getRunningTasks()).build();
}
}
);
}
@GET
@ -230,10 +297,17 @@ public class IndexerCoordinatorResource
@Produces("application/json")
public Response getWorkers()
{
if (taskMasterLifecycle.getTaskRunner() == null) {
return Response.noContent().build();
}
return Response.ok(taskMasterLifecycle.getTaskRunner().getWorkers()).build();
return asLeaderWith(
taskMasterLifecycle.getTaskRunner(),
new Function<TaskRunner, Response>()
{
@Override
public Response apply(TaskRunner taskRunner)
{
return Response.ok(taskRunner.getWorkers()).build();
}
}
);
}
@GET
@ -241,9 +315,47 @@ public class IndexerCoordinatorResource
@Produces("application/json")
public Response getScalingState()
{
if (taskMasterLifecycle.getResourceManagementScheduler() == null) {
return Response.noContent().build();
return asLeaderWith(
taskMasterLifecycle.getResourceManagementScheduler(),
new Function<ResourceManagementScheduler, Response>()
{
@Override
public Response apply(ResourceManagementScheduler resourceManagementScheduler)
{
return Response.ok(resourceManagementScheduler.getStats()).build();
}
}
);
}
@GET
@Path("/task/{taskid}/log")
@Produces("text/plain")
public Response doGetLog(
@PathParam("taskid") final String taskid,
@QueryParam("offset") @DefaultValue("0") final long offset
)
{
try {
final Optional<InputSupplier<InputStream>> stream = taskLogProvider.streamTaskLog(taskid, offset);
if (stream.isPresent()) {
return Response.ok(stream.get().getInput()).build();
} else {
return Response.status(Response.Status.NOT_FOUND).build();
}
} catch (Exception e) {
log.warn(e, "Failed to stream log for task %s", taskid);
return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
}
}
public <T> Response asLeaderWith(Optional<T> x, Function<T, Response> f)
{
if (x.isPresent()) {
return f.apply(x.get());
} else {
// Encourage client to try again soon, when we'll likely have a redirect set up
return Response.status(Response.Status.SERVICE_UNAVAILABLE).build();
}
return Response.ok(taskMasterLifecycle.getResourceManagementScheduler().getStats()).build();
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
import com.google.inject.Provides;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.merger.common.tasklogs.TaskLogProvider;
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
@ -41,6 +42,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
private final ServiceEmitter emitter;
private final TaskMasterLifecycle taskMasterLifecycle;
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
private final TaskLogProvider taskLogProvider;
private final JacksonConfigManager configManager;
public IndexerCoordinatorServletModule(
@ -49,6 +51,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
ServiceEmitter emitter,
TaskMasterLifecycle taskMasterLifecycle,
TaskStorageQueryAdapter taskStorageQueryAdapter,
TaskLogProvider taskLogProvider,
JacksonConfigManager configManager
)
{
@ -57,6 +60,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
this.emitter = emitter;
this.taskMasterLifecycle = taskMasterLifecycle;
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
this.taskLogProvider = taskLogProvider;
this.configManager = configManager;
}
@ -69,6 +73,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
bind(ServiceEmitter.class).toInstance(emitter);
bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle);
bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter);
bind(TaskLogProvider.class).toInstance(taskLogProvider);
bind(JacksonConfigManager.class).toInstance(configManager);
serve("/*").with(GuiceContainer.class);

View File

@ -28,12 +28,19 @@ public interface AutoScalingStrategy<T>
{
public AutoScalingData<T> provision();
public AutoScalingData<T> terminate(List<String> ids);
public AutoScalingData<T> terminate(List<String> ips);
/**
* Provides a lookup of ip addresses to node ids
* @param ips - nodes ips
* @param ips - nodes IPs
* @return node ids
*/
public List<String> ipToIdLookup(List<String> ips);
/**
* Provides a lookup of node ids to ip addresses
* @param nodeIds - nodes ids
* @return IPs associated with the node
*/
public List<String> idToIpLookup(List<String> nodeIds);
}

View File

@ -127,16 +127,16 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy<Instance>
}
@Override
public AutoScalingData<Instance> terminate(List<String> ids)
public AutoScalingData<Instance> terminate(List<String> ips)
{
if (ids.isEmpty()) {
if (ips.isEmpty()) {
return new AutoScalingData<Instance>(Lists.<String>newArrayList(), Lists.<Instance>newArrayList());
}
DescribeInstancesResult result = amazonEC2Client.describeInstances(
new DescribeInstancesRequest()
.withFilters(
new Filter("private-ip-address", ids)
new Filter("private-ip-address", ips)
)
);
@ -165,7 +165,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy<Instance>
return new AutoScalingData<Instance>(
Lists.transform(
ids,
ips,
new Function<String, String>()
{
@Override
@ -216,4 +216,36 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy<Instance>
return retVal;
}
@Override
public List<String> idToIpLookup(List<String> nodeIds)
{
DescribeInstancesResult result = amazonEC2Client.describeInstances(
new DescribeInstancesRequest()
.withFilters(
new Filter("instance-id", nodeIds)
)
);
List<Instance> instances = Lists.newArrayList();
for (Reservation reservation : result.getReservations()) {
instances.addAll(reservation.getInstances());
}
List<String> retVal = Lists.transform(
instances,
new Function<Instance, String>()
{
@Override
public String apply(Instance input)
{
return input.getPrivateIpAddress();
}
}
);
log.info("Performing lookup: %s --> %s", nodeIds, retVal);
return retVal;
}
}

View File

@ -38,9 +38,9 @@ public class NoopAutoScalingStrategy implements AutoScalingStrategy<String>
}
@Override
public AutoScalingData<String> terminate(List<String> nodeIds)
public AutoScalingData<String> terminate(List<String> ips)
{
log.info("If I were a real strategy I'd terminate %s now", nodeIds);
log.info("If I were a real strategy I'd terminate %s now", ips);
return null;
}
@ -50,4 +50,11 @@ public class NoopAutoScalingStrategy implements AutoScalingStrategy<String>
log.info("I'm not a real strategy so I'm returning what I got %s", ips);
return ips;
}
@Override
public List<String> idToIpLookup(List<String> nodeIds)
{
log.info("I'm not a real strategy so I'm returning what I got %s", nodeIds);
return nodeIds;
}
}

View File

@ -125,6 +125,8 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
.addData("provisioningCount", currentlyProvisioning.size())
.emit();
List<String> nodeIps = autoScalingStrategy.idToIpLookup(Lists.newArrayList(currentlyProvisioning));
autoScalingStrategy.terminate(nodeIps);
currentlyProvisioning.clear();
}
}

View File

@ -19,24 +19,22 @@
package com.metamx.druid.merger.worker;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.Query;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.query.NoopQueryRunner;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.merger.coordinator.TaskRunner;
import com.metamx.druid.merger.coordinator.TaskRunnerFactory;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.druid.query.segment.SegmentDescriptor;
import com.metamx.emitter.EmittingLogger;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent;
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
import org.apache.commons.io.FileUtils;
import org.joda.time.Interval;
import java.io.File;
import java.util.List;
@ -50,29 +48,32 @@ import java.util.concurrent.ExecutorService;
* The monitor implements {@link QuerySegmentWalker} so tasks can offer up queryable data. This is useful for
* realtime index tasks.
*/
public class WorkerTaskMonitor implements QuerySegmentWalker
public class WorkerTaskMonitor
{
private static final EmittingLogger log = new EmittingLogger(WorkerTaskMonitor.class);
private final ObjectMapper jsonMapper;
private final PathChildrenCache pathChildrenCache;
private final CuratorFramework cf;
private final WorkerCuratorCoordinator workerCuratorCoordinator;
private final TaskToolboxFactory toolboxFactory;
private final TaskRunner taskRunner;
private final ExecutorService exec;
private final List<Task> running = new CopyOnWriteArrayList<Task>();
public WorkerTaskMonitor(
ObjectMapper jsonMapper,
PathChildrenCache pathChildrenCache,
CuratorFramework cf,
WorkerCuratorCoordinator workerCuratorCoordinator,
TaskToolboxFactory toolboxFactory,
TaskRunner taskRunner,
ExecutorService exec
)
{
this.jsonMapper = jsonMapper;
this.pathChildrenCache = pathChildrenCache;
this.cf = cf;
this.workerCuratorCoordinator = workerCuratorCoordinator;
this.toolboxFactory = toolboxFactory;
this.taskRunner = taskRunner;
this.exec = exec;
}
@ -94,11 +95,10 @@ public class WorkerTaskMonitor implements QuerySegmentWalker
throws Exception
{
if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) {
final Task task = toolboxFactory.getObjectMapper().readValue(
final Task task = jsonMapper.readValue(
cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()),
Task.class
);
final TaskToolbox toolbox = toolboxFactory.build(task);
if (isTaskRunning(task)) {
log.warn("Got task %s that I am already running...", task.getId());
@ -113,7 +113,6 @@ public class WorkerTaskMonitor implements QuerySegmentWalker
public void run()
{
final long startTime = System.currentTimeMillis();
final File taskDir = toolbox.getTaskDir();
log.info("Running task [%s]", task.getId());
running.add(task);
@ -122,7 +121,7 @@ public class WorkerTaskMonitor implements QuerySegmentWalker
try {
workerCuratorCoordinator.unannounceTask(task.getId());
workerCuratorCoordinator.announceStatus(TaskStatus.running(task.getId()));
taskStatus = task.run(toolbox);
taskStatus = taskRunner.run(task).get();
}
catch (Exception e) {
log.makeAlert(e, "Failed to run task")
@ -144,19 +143,6 @@ public class WorkerTaskMonitor implements QuerySegmentWalker
.addData("task", task.getId())
.emit();
}
try {
if (taskDir.exists()) {
log.info("Removing task directory: %s", taskDir);
FileUtils.deleteDirectory(taskDir);
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to delete task directory")
.addData("taskDir", taskDir.toString())
.addData("task", task.getId())
.emit();
}
}
}
);
@ -196,38 +182,4 @@ public class WorkerTaskMonitor implements QuerySegmentWalker
.emit();
}
}
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<Interval> intervals)
{
return getQueryRunnerImpl(query);
}
@Override
public <T> QueryRunner<T> getQueryRunnerForSegments(Query<T> query, Iterable<SegmentDescriptor> specs)
{
return getQueryRunnerImpl(query);
}
private <T> QueryRunner<T> getQueryRunnerImpl(Query<T> query) {
QueryRunner<T> queryRunner = null;
for (final Task task : running) {
if (task.getDataSource().equals(query.getDataSource())) {
final QueryRunner<T> taskQueryRunner = task.getQueryRunner(query);
if (taskQueryRunner != null) {
if (queryRunner == null) {
queryRunner = taskQueryRunner;
} else {
log.makeAlert("Found too many query runners for datasource")
.addData("dataSource", query.getDataSource())
.emit();
}
}
}
}
return queryRunner == null ? new NoopQueryRunner<T>() : queryRunner;
}
}

View File

@ -0,0 +1,161 @@
package com.metamx.druid.merger.worker.executor;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.ISE;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskRunner;
import com.metamx.emitter.EmittingLogger;
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
/**
* Encapsulates the lifecycle of a task executor. Loads one task, runs it, writes its status, and all the while
* monitors its parent process.
*/
public class ExecutorLifecycle
{
private static final EmittingLogger log = new EmittingLogger(ExecutorLifecycle.class);
private final File taskFile;
private final File statusFile;
private final TaskRunner taskRunner;
private final InputStream parentStream;
private final ObjectMapper jsonMapper;
private final ExecutorService parentMonitorExec = Executors.newFixedThreadPool(
1,
new ThreadFactoryBuilder().setNameFormat("parent-monitor-%d").setDaemon(true).build()
);
private volatile ListenableFuture<TaskStatus> statusFuture = null;
public ExecutorLifecycle(
File taskFile,
File statusFile,
TaskRunner taskRunner,
InputStream parentStream,
ObjectMapper jsonMapper
)
{
this.taskFile = taskFile;
this.statusFile = statusFile;
this.taskRunner = taskRunner;
this.parentStream = parentStream;
this.jsonMapper = jsonMapper;
}
@LifecycleStart
public void start()
{
final Task task;
try {
task = jsonMapper.readValue(taskFile, Task.class);
log.info(
"Running with task: %s",
jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(task)
);
}
catch (IOException e) {
throw Throwables.propagate(e);
}
// Spawn monitor thread to keep a watch on parent's stdin
// If a message comes over stdin, we want to handle it
// If stdin reaches eof, the parent is gone, and we should shut down
parentMonitorExec.submit(
new Runnable()
{
@Override
public void run()
{
try {
final BufferedReader parentReader = new BufferedReader(new InputStreamReader(parentStream));
String messageString;
while ((messageString = parentReader.readLine()) != null) {
final Map<String, Object> message = jsonMapper
.readValue(
messageString,
new TypeReference<Map<String, Object>>()
{
}
);
if (message == null) {
break;
} else if (message.get("shutdown") != null && message.get("shutdown").equals("now")) {
log.info("Shutting down!");
task.shutdown();
} else {
throw new ISE("Unrecognized message from parent: %s", message);
}
}
}
catch (Exception e) {
log.error(e, "Failed to read from stdin");
}
// Kind of gross, but best way to kill the JVM as far as I know
log.info("Triggering JVM shutdown.");
System.exit(2);
}
}
);
statusFuture = Futures.transform(
taskRunner.run(task), new Function<TaskStatus, TaskStatus>()
{
@Override
public TaskStatus apply(TaskStatus taskStatus)
{
try {
log.info(
"Task completed with status: %s",
jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(taskStatus)
);
jsonMapper.writeValue(statusFile, taskStatus);
return taskStatus;
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
);
}
public void join()
{
try {
statusFuture.get();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@LifecycleStop
public void stop()
{
parentMonitorExec.shutdown();
}
}

View File

@ -0,0 +1,26 @@
package com.metamx.druid.merger.worker.executor;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.druid.merger.coordinator.TaskRunner;
import java.io.File;
import java.io.InputStream;
public class ExecutorLifecycleFactory
{
private final File taskFile;
private final File statusFile;
private final InputStream parentStream;
public ExecutorLifecycleFactory(File taskFile, File statusFile, InputStream parentStream)
{
this.taskFile = taskFile;
this.statusFile = statusFile;
this.parentStream = parentStream;
}
public ExecutorLifecycle build(TaskRunner taskRunner, ObjectMapper jsonMapper)
{
return new ExecutorLifecycle(taskFile, statusFile, taskRunner, parentStream, jsonMapper);
}
}

View File

@ -0,0 +1,66 @@
/*
* 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.worker.executor;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import com.metamx.druid.log.LogLevelAdjuster;
import java.io.File;
/**
*/
public class ExecutorMain
{
private static final Logger log = new Logger(ExecutorMain.class);
public static void main(String[] args) throws Exception
{
LogLevelAdjuster.register();
if (args.length != 2) {
log.info("Usage: ExecutorMain <task.json> <status.json>");
System.exit(2);
}
final ExecutorNode node = ExecutorNode.builder()
.build(
new ExecutorLifecycleFactory(
new File(args[0]),
new File(args[1]),
System.in
)
);
final Lifecycle lifecycle = new Lifecycle();
lifecycle.addManagedInstance(node);
try {
lifecycle.start();
node.join();
lifecycle.stop();
}
catch (Throwable t) {
log.info(t, "Throwable caught at startup, committing seppuku");
System.exit(2);
}
}
}

View File

@ -0,0 +1,526 @@
/*
* 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.worker.executor;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutorFactory;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.config.Config;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.BaseServerNode;
import com.metamx.druid.client.ClientConfig;
import com.metamx.druid.client.ClientInventoryManager;
import com.metamx.druid.client.MutableServerView;
import com.metamx.druid.client.OnlyNewSegmentWatcherServerView;
import com.metamx.druid.http.QueryServlet;
import com.metamx.druid.http.StatusServlet;
import com.metamx.druid.initialization.CuratorConfig;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServerInit;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentKiller;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.ExecutorServiceTaskRunner;
import com.metamx.druid.merger.worker.config.WorkerConfig;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncerConfig;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.Emitters;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.HttpClientConfig;
import com.metamx.http.client.HttpClientInit;
import com.metamx.metrics.JvmMonitor;
import com.metamx.metrics.Monitor;
import com.metamx.metrics.MonitorScheduler;
import com.metamx.metrics.MonitorSchedulerConfig;
import com.metamx.metrics.SysMonitor;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.x.discovery.ServiceDiscovery;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.jets3t.service.S3ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.security.AWSCredentials;
import org.mortbay.jetty.Server;
import org.mortbay.jetty.servlet.Context;
import org.mortbay.jetty.servlet.ServletHolder;
import org.skife.config.ConfigurationObjectFactory;
import java.io.IOException;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
/**
*/
public class ExecutorNode extends BaseServerNode<ExecutorNode>
{
private static final EmittingLogger log = new EmittingLogger(ExecutorNode.class);
public static Builder builder()
{
return new Builder();
}
private final Lifecycle lifecycle;
private final Properties props;
private final ConfigurationObjectFactory configFactory;
private final ExecutorLifecycleFactory executorLifecycleFactory;
private RestS3Service s3Service = null;
private List<Monitor> monitors = null;
private HttpClient httpClient = null;
private ServiceEmitter emitter = null;
private TaskConfig taskConfig = null;
private WorkerConfig workerConfig = null;
private DataSegmentPusher segmentPusher = null;
private TaskToolboxFactory taskToolboxFactory = null;
private CuratorFramework curatorFramework = null;
private ServiceDiscovery serviceDiscovery = null;
private ServiceProvider coordinatorServiceProvider = null;
private MutableServerView newSegmentServerView = null;
private Server server = null;
private ExecutorServiceTaskRunner taskRunner = null;
private ExecutorLifecycle executorLifecycle = null;
public ExecutorNode(
Properties props,
Lifecycle lifecycle,
ObjectMapper jsonMapper,
ObjectMapper smileMapper,
ConfigurationObjectFactory configFactory,
ExecutorLifecycleFactory executorLifecycleFactory
)
{
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
this.lifecycle = lifecycle;
this.props = props;
this.configFactory = configFactory;
this.executorLifecycleFactory = executorLifecycleFactory;
}
public ExecutorNode setHttpClient(HttpClient httpClient)
{
this.httpClient = httpClient;
return this;
}
public ExecutorNode setEmitter(ServiceEmitter emitter)
{
this.emitter = emitter;
return this;
}
public ExecutorNode setS3Service(RestS3Service s3Service)
{
this.s3Service = s3Service;
return this;
}
public ExecutorNode setSegmentPusher(DataSegmentPusher segmentPusher)
{
this.segmentPusher = segmentPusher;
return this;
}
public ExecutorNode setTaskToolboxFactory(TaskToolboxFactory taskToolboxFactory)
{
this.taskToolboxFactory = taskToolboxFactory;
return this;
}
public ExecutorNode setCuratorFramework(CuratorFramework curatorFramework)
{
this.curatorFramework = curatorFramework;
return this;
}
public ExecutorNode setCoordinatorServiceProvider(ServiceProvider coordinatorServiceProvider)
{
this.coordinatorServiceProvider = coordinatorServiceProvider;
return this;
}
public ExecutorNode setServiceDiscovery(ServiceDiscovery serviceDiscovery)
{
this.serviceDiscovery = serviceDiscovery;
return this;
}
public ExecutorNode setNewSegmentServerView(MutableServerView newSegmentServerView)
{
this.newSegmentServerView = newSegmentServerView;
return this;
}
@Override
public void doInit() throws Exception
{
initializeHttpClient();
initializeEmitter();
initializeS3Service();
initializeMonitors();
initializeMergerConfig();
initializeCuratorFramework();
initializeServiceDiscovery();
initializeCoordinatorServiceProvider();
initializeNewSegmentServerView();
initializeDataSegmentPusher();
initializeTaskToolbox();
initializeTaskRunner();
initializeJacksonInjections();
initializeJacksonSubtypes();
initializeServer();
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
final MonitorScheduler monitorScheduler = new MonitorScheduler(
configFactory.build(MonitorSchedulerConfig.class),
globalScheduledExec,
emitter,
monitors
);
lifecycle.addManagedInstance(monitorScheduler);
executorLifecycle = executorLifecycleFactory.build(taskRunner, getJsonMapper());
lifecycle.addManagedInstance(executorLifecycle);
final Context root = new Context(server, "/", Context.SESSIONS);
root.addServlet(new ServletHolder(new StatusServlet()), "/status");
root.addServlet(
new ServletHolder(
new QueryServlet(getJsonMapper(), getSmileMapper(), taskRunner, emitter, getRequestLogger())
),
"/druid/v2/*"
);
}
@LifecycleStart
public synchronized void start() throws Exception
{
init();
lifecycle.start();
}
@LifecycleStop
public synchronized void stop()
{
lifecycle.stop();
}
public void join()
{
executorLifecycle.join();
}
public ExecutorServiceTaskRunner getTaskRunner()
{
return taskRunner;
}
private void initializeServer()
{
if (server == null) {
server = Initialization.makeJettyServer(configFactory.build(ServerConfig.class));
lifecycle.addHandler(
new Lifecycle.Handler()
{
@Override
public void start() throws Exception
{
log.info("Starting Jetty");
server.start();
}
@Override
public void stop()
{
log.info("Stopping Jetty");
try {
server.stop();
}
catch (Exception e) {
log.error(e, "Exception thrown while stopping Jetty");
}
}
}
);
}
}
private void initializeJacksonInjections()
{
InjectableValues.Std injectables = new InjectableValues.Std();
injectables.addValue("s3Client", s3Service)
.addValue("segmentPusher", segmentPusher);
getJsonMapper().setInjectableValues(injectables);
}
private void initializeJacksonSubtypes()
{
getJsonMapper().registerSubtypes(StaticS3FirehoseFactory.class);
}
private void initializeHttpClient()
{
if (httpClient == null) {
httpClient = HttpClientInit.createClient(
HttpClientConfig.builder().withNumConnections(1).build(), lifecycle
);
}
}
private void initializeEmitter()
{
if (emitter == null) {
emitter = new ServiceEmitter(
PropUtils.getProperty(props, "druid.service"),
PropUtils.getProperty(props, "druid.host"),
Emitters.create(props, httpClient, getJsonMapper(), lifecycle)
);
}
EmittingLogger.registerEmitter(emitter);
}
private void initializeS3Service() throws S3ServiceException
{
if(s3Service == null) {
s3Service = new RestS3Service(
new AWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
)
);
}
}
private void initializeMonitors()
{
if (monitors == null) {
monitors = Lists.newArrayList();
monitors.add(new JvmMonitor());
monitors.add(new SysMonitor());
}
}
private void initializeMergerConfig()
{
if (taskConfig == null) {
taskConfig = configFactory.build(TaskConfig.class);
}
if (workerConfig == null) {
workerConfig = configFactory.build(WorkerConfig.class);
}
}
public void initializeDataSegmentPusher()
{
if (segmentPusher == null) {
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, getJsonMapper());
}
}
public void initializeTaskToolbox() throws S3ServiceException
{
if (taskToolboxFactory == null) {
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
final SegmentAnnouncer segmentAnnouncer = new ZkSegmentAnnouncer(
configFactory.build(ZkSegmentAnnouncerConfig.class),
getPhoneBook()
);
lifecycle.addManagedInstance(segmentAnnouncer);
taskToolboxFactory = new TaskToolboxFactory(
taskConfig,
new RemoteTaskActionClientFactory(
httpClient,
coordinatorServiceProvider,
new RetryPolicyFactory(
configFactory.buildWithReplacements(
RetryPolicyConfig.class,
ImmutableMap.of("base_path", "druid.worker.taskActionClient")
)
),
getJsonMapper()
),
emitter,
s3Service,
segmentPusher,
dataSegmentKiller,
segmentAnnouncer,
newSegmentServerView,
getConglomerate(),
getJsonMapper()
);
}
}
public void initializeCuratorFramework() throws IOException
{
if (curatorFramework == null) {
final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class);
curatorFramework = Initialization.makeCuratorFrameworkClient(
curatorConfig,
lifecycle
);
}
}
public void initializeServiceDiscovery() throws Exception
{
if (serviceDiscovery == null) {
final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class);
this.serviceDiscovery = Initialization.makeServiceDiscoveryClient(
curatorFramework,
config,
lifecycle
);
}
}
public void initializeCoordinatorServiceProvider()
{
if (coordinatorServiceProvider == null) {
this.coordinatorServiceProvider = Initialization.makeServiceProvider(
workerConfig.getMasterService(),
serviceDiscovery,
lifecycle
);
}
}
private void initializeNewSegmentServerView()
{
if (newSegmentServerView == null) {
final MutableServerView view = new OnlyNewSegmentWatcherServerView();
final ClientInventoryManager clientInventoryManager = new ClientInventoryManager(
getConfigFactory().build(ClientConfig.class),
getPhoneBook(),
view
);
lifecycle.addManagedInstance(clientInventoryManager);
this.newSegmentServerView = view;
}
}
public void initializeTaskRunner()
{
if (taskRunner == null) {
this.taskRunner = lifecycle.addManagedInstance(
new ExecutorServiceTaskRunner(
taskToolboxFactory,
Executors.newSingleThreadExecutor(
new ThreadFactoryBuilder()
.setNameFormat("task-runner-%d")
.build()
)
)
);;
}
}
public static class Builder
{
private ObjectMapper jsonMapper = null;
private ObjectMapper smileMapper = null;
private Lifecycle lifecycle = null;
private Properties props = null;
private ConfigurationObjectFactory configFactory = null;
public Builder withMapper(ObjectMapper jsonMapper)
{
this.jsonMapper = jsonMapper;
return this;
}
public Builder withLifecycle(Lifecycle lifecycle)
{
this.lifecycle = lifecycle;
return this;
}
public Builder withProps(Properties props)
{
this.props = props;
return this;
}
public Builder withConfigFactory(ConfigurationObjectFactory configFactory)
{
this.configFactory = configFactory;
return this;
}
public ExecutorNode build(ExecutorLifecycleFactory executorLifecycleFactory)
{
if (jsonMapper == null && smileMapper == null) {
jsonMapper = new DefaultObjectMapper();
smileMapper = new DefaultObjectMapper(new SmileFactory());
smileMapper.getJsonFactory().setCodec(smileMapper);
}
else if (jsonMapper == null || smileMapper == null) {
throw new ISE("Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", jsonMapper, smileMapper);
}
if (lifecycle == null) {
lifecycle = new Lifecycle();
}
if (props == null) {
props = Initialization.loadProperties();
}
if (configFactory == null) {
configFactory = Config.createFactory(props);
}
return new ExecutorNode(props, lifecycle, jsonMapper, smileMapper, configFactory, executorLifecycleFactory);
}
}
}

View File

@ -19,49 +19,38 @@
package com.metamx.druid.merger.worker.http;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.servlet.GuiceFilter;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutorFactory;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.config.Config;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.BaseServerNode;
import com.metamx.druid.client.ClientConfig;
import com.metamx.druid.client.ClientInventoryManager;
import com.metamx.druid.client.MutableServerView;
import com.metamx.druid.client.OnlyNewSegmentWatcherServerView;
import com.metamx.druid.http.QueryServlet;
import com.metamx.druid.RegisteringNode;
import com.metamx.druid.http.GuiceServletConfig;
import com.metamx.druid.http.StatusServlet;
import com.metamx.druid.initialization.CuratorConfig;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServerInit;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentKiller;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.config.TaskLogConfig;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
import com.metamx.druid.merger.common.tasklogs.NoopTaskLogs;
import com.metamx.druid.merger.common.tasklogs.S3TaskLogs;
import com.metamx.druid.merger.common.tasklogs.TaskLogs;
import com.metamx.druid.merger.coordinator.ForkingTaskRunner;
import com.metamx.druid.merger.coordinator.config.ForkingTaskRunnerConfig;
import com.metamx.druid.merger.worker.Worker;
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
import com.metamx.druid.merger.worker.WorkerTaskMonitor;
import com.metamx.druid.merger.worker.config.WorkerConfig;
import com.metamx.druid.realtime.SegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncer;
import com.metamx.druid.realtime.ZkSegmentAnnouncerConfig;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.Emitters;
@ -97,7 +86,7 @@ import java.util.concurrent.ScheduledExecutorService;
/**
*/
public class WorkerNode extends BaseServerNode<WorkerNode>
public class WorkerNode extends RegisteringNode
{
private static final EmittingLogger log = new EmittingLogger(WorkerNode.class);
@ -108,22 +97,21 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
private final Lifecycle lifecycle;
private final Properties props;
private final ObjectMapper jsonMapper;
private final ConfigurationObjectFactory configFactory;
private RestS3Service s3Service = null;
private List<Monitor> monitors = null;
private HttpClient httpClient = null;
private ServiceEmitter emitter = null;
private TaskConfig taskConfig = null;
private WorkerConfig workerConfig = null;
private DataSegmentPusher segmentPusher = null;
private TaskToolboxFactory taskToolboxFactory = null;
private CuratorFramework curatorFramework = null;
private ServiceDiscovery serviceDiscovery = null;
private ServiceProvider coordinatorServiceProvider = null;
private WorkerCuratorCoordinator workerCuratorCoordinator = null;
private WorkerTaskMonitor workerTaskMonitor = null;
private MutableServerView newSegmentServerView = null;
private TaskLogs persistentTaskLogs = null;
private ForkingTaskRunner forkingTaskRunner = null;
private Server server = null;
private boolean initialized = false;
@ -132,14 +120,14 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
Properties props,
Lifecycle lifecycle,
ObjectMapper jsonMapper,
ObjectMapper smileMapper,
ConfigurationObjectFactory configFactory
)
{
super(log, props, lifecycle, jsonMapper, smileMapper, configFactory);
super(ImmutableList.of(jsonMapper));
this.lifecycle = lifecycle;
this.props = props;
this.jsonMapper = jsonMapper;
this.configFactory = configFactory;
}
@ -155,24 +143,6 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
return this;
}
public WorkerNode setS3Service(RestS3Service s3Service)
{
this.s3Service = s3Service;
return this;
}
public WorkerNode setSegmentPusher(DataSegmentPusher segmentPusher)
{
this.segmentPusher = segmentPusher;
return this;
}
public WorkerNode setTaskToolboxFactory(TaskToolboxFactory taskToolboxFactory)
{
this.taskToolboxFactory = taskToolboxFactory;
return this;
}
public WorkerNode setCuratorFramework(CuratorFramework curatorFramework)
{
this.curatorFramework = curatorFramework;
@ -197,9 +167,9 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
return this;
}
public WorkerNode setNewSegmentServerView(MutableServerView newSegmentServerView)
public WorkerNode setForkingTaskRunner(ForkingTaskRunner forkingTaskRunner)
{
this.newSegmentServerView = newSegmentServerView;
this.forkingTaskRunner = forkingTaskRunner;
return this;
}
@ -209,23 +179,19 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
return this;
}
@Override
public void doInit() throws Exception
{
initializeHttpClient();
initializeEmitter();
initializeS3Service();
initializeMonitors();
initializeMergerConfig();
initializeCuratorFramework();
initializeServiceDiscovery();
initializeCoordinatorServiceProvider();
initializeNewSegmentServerView();
initializeDataSegmentPusher();
initializeTaskToolbox();
initializeJacksonInjections();
initializeJacksonSubtypes();
initializeCuratorCoordinator();
initializePersistentTaskLogs();
initializeTaskRunner();
initializeWorkerTaskMonitor();
initializeServer();
@ -239,24 +205,26 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
);
lifecycle.addManagedInstance(monitorScheduler);
final Injector injector = Guice.createInjector(
new WorkerServletModule(
getJsonMapper(),
emitter,
forkingTaskRunner
)
);
final Context root = new Context(server, "/", Context.SESSIONS);
root.addServlet(new ServletHolder(new StatusServlet()), "/status");
root.addServlet(new ServletHolder(new DefaultServlet()), "/mmx/*");
root.addServlet(
new ServletHolder(
new QueryServlet(getJsonMapper(), getSmileMapper(), workerTaskMonitor, emitter, getRequestLogger())
),
"/druid/v2/*"
);
root.addFilter(GuiceFilter.class, "/mmx/indexer/worker/v1/*", 0);
root.addServlet(new ServletHolder(new DefaultServlet()), "/*");
root.addEventListener(new GuiceServletConfig(injector));
root.addFilter(GuiceFilter.class, "/mmx/worker/v1/*", 0);
}
@LifecycleStart
public synchronized void start() throws Exception
{
if (!initialized) {
init();
doInit();
}
lifecycle.start();
@ -268,6 +236,11 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
lifecycle.stop();
}
private ObjectMapper getJsonMapper()
{
return jsonMapper;
}
private void initializeServer()
{
if (server == null) {
@ -299,16 +272,6 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
}
}
private void initializeJacksonInjections()
{
InjectableValues.Std injectables = new InjectableValues.Std();
injectables.addValue("s3Client", s3Service)
.addValue("segmentPusher", segmentPusher);
getJsonMapper().setInjectableValues(injectables);
}
private void initializeJacksonSubtypes()
{
getJsonMapper().registerSubtypes(StaticS3FirehoseFactory.class);
@ -360,56 +323,11 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
private void initializeMergerConfig()
{
if (taskConfig == null) {
taskConfig = configFactory.build(TaskConfig.class);
}
if (workerConfig == null) {
workerConfig = configFactory.build(WorkerConfig.class);
}
}
public void initializeDataSegmentPusher()
{
if (segmentPusher == null) {
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, getJsonMapper());
}
}
public void initializeTaskToolbox() throws S3ServiceException
{
if (taskToolboxFactory == null) {
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
final SegmentAnnouncer segmentAnnouncer = new ZkSegmentAnnouncer(
configFactory.build(ZkSegmentAnnouncerConfig.class),
getPhoneBook()
);
lifecycle.addManagedInstance(segmentAnnouncer);
taskToolboxFactory = new TaskToolboxFactory(
taskConfig,
new RemoteTaskActionClientFactory(
httpClient,
coordinatorServiceProvider,
new RetryPolicyFactory(
configFactory.buildWithReplacements(
RetryPolicyConfig.class,
ImmutableMap.of("base_path", "druid.worker.taskActionClient")
)
),
getJsonMapper()
),
emitter,
s3Service,
segmentPusher,
dataSegmentKiller,
segmentAnnouncer,
newSegmentServerView,
getConglomerate(),
getJsonMapper()
);
}
}
public void initializeCuratorFramework() throws IOException
{
if (curatorFramework == null) {
@ -457,18 +375,33 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
}
}
private void initializeNewSegmentServerView()
private void initializePersistentTaskLogs() throws S3ServiceException
{
if (newSegmentServerView == null) {
final MutableServerView view = new OnlyNewSegmentWatcherServerView();
final ClientInventoryManager clientInventoryManager = new ClientInventoryManager(
getConfigFactory().build(ClientConfig.class),
getPhoneBook(),
view
);
lifecycle.addManagedInstance(clientInventoryManager);
if (persistentTaskLogs == null) {
final TaskLogConfig taskLogConfig = configFactory.build(TaskLogConfig.class);
if (taskLogConfig.getLogStorageBucket() != null) {
initializeS3Service();
persistentTaskLogs = new S3TaskLogs(
taskLogConfig.getLogStorageBucket(),
taskLogConfig.getLogStoragePrefix(),
s3Service
);
} else {
persistentTaskLogs = new NoopTaskLogs();
}
}
}
this.newSegmentServerView = view;
public void initializeTaskRunner()
{
if (forkingTaskRunner == null) {
forkingTaskRunner = new ForkingTaskRunner(
configFactory.build(ForkingTaskRunnerConfig.class),
props,
persistentTaskLogs,
Executors.newFixedThreadPool(workerConfig.getCapacity()),
getJsonMapper()
);
}
}
@ -482,10 +415,11 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
false
);
workerTaskMonitor = new WorkerTaskMonitor(
getJsonMapper(),
pathChildrenCache,
curatorFramework,
workerCuratorCoordinator,
taskToolboxFactory,
forkingTaskRunner,
workerExec
);
lifecycle.addManagedInstance(workerTaskMonitor);
@ -495,7 +429,6 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
public static class Builder
{
private ObjectMapper jsonMapper = null;
private ObjectMapper smileMapper = null;
private Lifecycle lifecycle = null;
private Properties props = null;
private ConfigurationObjectFactory configFactory = null;
@ -526,16 +459,8 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
public WorkerNode build()
{
if (jsonMapper == null && smileMapper == null) {
if (jsonMapper == null) {
jsonMapper = new DefaultObjectMapper();
smileMapper = new DefaultObjectMapper(new SmileFactory());
smileMapper.getJsonFactory().setCodec(smileMapper);
} else if (jsonMapper == null || smileMapper == null) {
throw new ISE(
"Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.",
jsonMapper,
smileMapper
);
}
if (lifecycle == null) {
@ -550,7 +475,7 @@ public class WorkerNode extends BaseServerNode<WorkerNode>
configFactory = Config.createFactory(props);
}
return new WorkerNode(props, lifecycle, jsonMapper, smileMapper, configFactory);
return new WorkerNode(props, lifecycle, jsonMapper, configFactory);
}
}
}

View File

@ -0,0 +1,99 @@
/*
* 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.worker.http;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.InputSupplier;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import com.metamx.druid.merger.coordinator.ForkingTaskRunner;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Response;
import java.io.IOException;
import java.io.InputStream;
/**
*/
@Path("/mmx/worker/v1")
public class WorkerResource
{
private static final Logger log = new Logger(WorkerResource.class);
private final ObjectMapper jsonMapper;
private final ForkingTaskRunner taskRunner;
@Inject
public WorkerResource(
ObjectMapper jsonMapper,
ForkingTaskRunner taskRunner
) throws Exception
{
this.jsonMapper = jsonMapper;
this.taskRunner = taskRunner;
}
@POST
@Path("/task/{taskid}/shutdown")
@Produces("application/json")
public Response doShutdown(@PathParam("taskid") String taskid)
{
try {
taskRunner.shutdown(taskid);
}
catch (Exception e) {
log.error(e, "Failed to issue shutdown for task: %s", taskid);
return Response.serverError().build();
}
return Response.ok(ImmutableMap.of("task", taskid)).build();
}
@GET
@Path("/task/{taskid}/log")
@Produces("text/plain")
public Response doGetLog(
@PathParam("taskid") String taskid,
@QueryParam("offset") @DefaultValue("0") long offset
)
{
final Optional<InputSupplier<InputStream>> stream = taskRunner.streamTaskLog(taskid, offset);
if (stream.isPresent()) {
try {
return Response.ok(stream.get().getInput()).build();
} catch (Exception e) {
log.warn(e, "Failed to read log for task: %s", taskid);
return Response.serverError().build();
}
} else {
return Response.status(Response.Status.NOT_FOUND).build();
}
}
}

View File

@ -0,0 +1,51 @@
package com.metamx.druid.merger.worker.http;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
import com.google.inject.Provides;
import com.metamx.druid.merger.coordinator.ForkingTaskRunner;
import com.metamx.emitter.service.ServiceEmitter;
import com.sun.jersey.guice.JerseyServletModule;
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
import javax.inject.Singleton;
/**
*/
public class WorkerServletModule extends JerseyServletModule
{
private final ObjectMapper jsonMapper;
private final ServiceEmitter emitter;
private final ForkingTaskRunner forkingTaskRunner;
public WorkerServletModule(
ObjectMapper jsonMapper,
ServiceEmitter emitter,
ForkingTaskRunner forkingTaskRunner
)
{
this.jsonMapper = jsonMapper;
this.emitter = emitter;
this.forkingTaskRunner = forkingTaskRunner;
}
@Override
protected void configureServlets()
{
bind(WorkerResource.class);
bind(ObjectMapper.class).toInstance(jsonMapper);
bind(ServiceEmitter.class).toInstance(emitter);
bind(ForkingTaskRunner.class).toInstance(forkingTaskRunner);
serve("/*").with(GuiceContainer.class);
}
@Provides
@Singleton
public JacksonJsonProvider getJacksonJsonProvider()
{
final JacksonJsonProvider provider = new JacksonJsonProvider();
provider.setMapper(jsonMapper);
return provider;
}
}

View File

@ -4,19 +4,20 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.metamx.common.ISE;
import com.metamx.druid.aggregation.AggregatorFactory;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.merger.TestTask;
import com.metamx.druid.merger.common.RetryPolicyFactory;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
import com.metamx.druid.merger.common.config.RetryPolicyConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.worker.Worker;
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
@ -130,10 +131,7 @@ public class RemoteTaskRunnerTest
@Test
public void testRunNoExistingTask() throws Exception
{
remoteTaskRunner.run(
task1,
null
);
remoteTaskRunner.run(task1);
}
@Test
@ -146,11 +144,10 @@ public class RemoteTaskRunnerTest
task1.getSegments(),
Lists.<AggregatorFactory>newArrayList(),
TaskStatus.running(task1.getId())
),
null
)
);
try {
remoteTaskRunner.run(task1, null);
remoteTaskRunner.run(task1);
fail("ISE expected");
}
catch (ISE expected) {
@ -182,8 +179,7 @@ public class RemoteTaskRunnerTest
),
Lists.<AggregatorFactory>newArrayList(),
TaskStatus.success("foo")
),
null
)
);
EasyMock.verify(emitter);
}
@ -192,22 +188,30 @@ public class RemoteTaskRunnerTest
public void testRunWithCallback() throws Exception
{
final MutableBoolean callbackCalled = new MutableBoolean(false);
remoteTaskRunner.run(
new TestTask(
task1.getId(),
task1.getDataSource(),
task1.getSegments(),
Lists.<AggregatorFactory>newArrayList(),
TaskStatus.running(task1.getId())
),
new TaskCallback()
{
@Override
public void notify(TaskStatus status)
{
callbackCalled.setValue(true);
}
}
Futures.addCallback(
remoteTaskRunner.run(
new TestTask(
task1.getId(),
task1.getDataSource(),
task1.getSegments(),
Lists.<AggregatorFactory>newArrayList(),
TaskStatus.running(task1.getId())
)
), new FutureCallback<TaskStatus>()
{
@Override
public void onSuccess(TaskStatus taskStatus)
{
callbackCalled.setValue(true);
}
@Override
public void onFailure(Throwable throwable)
{
// neg
}
}
);
// Really don't like this way of waiting for the task to appear
@ -277,35 +281,38 @@ public class RemoteTaskRunnerTest
workerCuratorCoordinator.start();
workerTaskMonitor = new WorkerTaskMonitor(
jsonMapper,
new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true),
cf,
workerCuratorCoordinator,
new TaskToolboxFactory(
new TaskConfig()
{
@Override
public File getBaseTaskDir()
{
try {
return File.createTempFile("billy", "yay");
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
new ExecutorServiceTaskRunner(
new TaskToolboxFactory(
new TaskConfig()
{
@Override
public File getBaseTaskDir()
{
try {
return File.createTempFile("billy", "yay");
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Override
public int getDefaultRowFlushBoundary()
{
return 0;
}
@Override
public int getDefaultRowFlushBoundary()
{
return 0;
}
@Override
public String getHadoopWorkingPath()
{
return null;
}
}, null, null, null, null, null, null, null, null, jsonMapper
@Override
public String getHadoopWorkingPath()
{
return null;
}
}, null, null, null, null, null, null, null, null, jsonMapper
), Executors.newSingleThreadExecutor()
),
Executors.newSingleThreadExecutor()
);
@ -324,7 +331,8 @@ public class RemoteTaskRunnerTest
pathChildrenCache,
scheduledExec,
new RetryPolicyFactory(new TestRetryPolicyConfig()),
new AtomicReference<WorkerSetupData>(new WorkerSetupData("0", 0, 1, null, null))
new AtomicReference<WorkerSetupData>(new WorkerSetupData("0", 0, 1, null, null)),
null
);
// Create a single worker and wait for things for be ready

View File

@ -50,6 +50,7 @@ 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.TaskActionClientFactory;
import com.metamx.druid.merger.common.actions.TaskActionToolbox;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.task.AbstractTask;
@ -87,6 +88,7 @@ public class TaskLifecycleTest
private TaskQueue tq = null;
private TaskRunner tr = null;
private MockMergerDBCoordinator mdc = null;
private TaskActionClientFactory tac = null;
private TaskToolboxFactory tb = null;
private TaskConsumer tc = null;
TaskStorageQueryAdapter tsqa = null;
@ -111,6 +113,7 @@ public class TaskLifecycleTest
tl = new TaskLockbox(ts);
tq = new TaskQueue(ts, tl);
mdc = newMockMDC();
tac = new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter()));
tb = new TaskToolboxFactory(
new TaskConfig()
@ -133,7 +136,7 @@ public class TaskLifecycleTest
return null;
}
},
new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())),
tac,
newMockEmitter(),
null, // s3 client
new DataSegmentPusher()
@ -158,12 +161,12 @@ public class TaskLifecycleTest
new DefaultObjectMapper()
);
tr = new LocalTaskRunner(
tr = new ExecutorServiceTaskRunner(
tb,
Executors.newSingleThreadExecutor()
);
tc = new TaskConsumer(tq, tr, tb, newMockEmitter());
tc = new TaskConsumer(tq, tr, tac, newMockEmitter());
tsqa = new TaskStorageQueryAdapter(ts);
tq.start();

View File

@ -203,6 +203,10 @@ public class SimpleResourceManagementStrategyTest
EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject()))
.andReturn(Lists.<String>newArrayList()).times(2);
EasyMock.expect(autoScalingStrategy.idToIpLookup(EasyMock.<List<String>>anyObject()))
.andReturn(Lists.<String>newArrayList());
EasyMock.expect(autoScalingStrategy.terminate(EasyMock.<List<String>>anyObject()))
.andReturn(null);
EasyMock.expect(autoScalingStrategy.provision()).andReturn(
new AutoScalingData(Lists.<String>newArrayList("fake"), Lists.newArrayList("faker"))
);

View File

@ -32,7 +32,7 @@ public class TestAutoScalingStrategy<T> implements AutoScalingStrategy<T>
}
@Override
public AutoScalingData<T> terminate(List<String> ids)
public AutoScalingData<T> terminate(List<String> ips)
{
return null;
}
@ -42,4 +42,10 @@ public class TestAutoScalingStrategy<T> implements AutoScalingStrategy<T>
{
return null;
}
@Override
public List<String> idToIpLookup(List<String> nodeIds)
{
return null;
}
}

View File

@ -23,7 +23,7 @@
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<packaging>pom</packaging>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
<name>druid</name>
<description>druid</description>
<scm>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -0,0 +1,142 @@
/*
* 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.realtime;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.logger.Logger;
import com.metamx.druid.index.v1.IndexGranularity;
import com.metamx.druid.input.InputRow;
import com.metamx.druid.realtime.plumber.IntervalRejectionPolicyFactory;
import com.metamx.druid.realtime.plumber.RejectionPolicy;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Interval;
import org.joda.time.Period;
import java.io.IOException;
import java.util.concurrent.Callable;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
/**
*/
public class GracefulShutdownFirehose implements Firehose
{
private static final Logger log = new Logger(GracefulShutdownFirehose.class);
private final Firehose firehose;
private final IndexGranularity segmentGranularity;
private final long windowMillis;
private final ScheduledExecutorService scheduledExecutor;
private final RejectionPolicy rejectionPolicy;
// when this is set to false, the firehose will have no more rows
private final AtomicBoolean valveOn = new AtomicBoolean(true);
// when this is set to true, the firehose will begin rejecting events
private volatile boolean beginRejectionPolicy = false;
public GracefulShutdownFirehose(
Firehose firehose,
IndexGranularity segmentGranularity,
Period windowPeriod
)
{
this.firehose = firehose;
this.segmentGranularity = segmentGranularity;
this.windowMillis = windowPeriod.toStandardDuration().getMillis() * 2;
this.scheduledExecutor = Executors.newScheduledThreadPool(
1,
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("firehose_scheduled_%d")
.build()
);
final long truncatedNow = segmentGranularity.truncate(new DateTime()).getMillis();
final long end = segmentGranularity.increment(truncatedNow);
this.rejectionPolicy = new IntervalRejectionPolicyFactory(new Interval(truncatedNow, end)).create(windowPeriod);
}
public void shutdown() throws IOException
{
final long truncatedNow = segmentGranularity.truncate(new DateTime()).getMillis();
final long end = segmentGranularity.increment(truncatedNow) + windowMillis;
final Duration timeUntilShutdown = new Duration(System.currentTimeMillis(), end);
log.info("Shutdown at approx. %s (in %s)", new DateTime(end), timeUntilShutdown);
ScheduledExecutors.scheduleWithFixedDelay(
scheduledExecutor,
timeUntilShutdown,
new Callable<ScheduledExecutors.Signal>()
{
@Override
public ScheduledExecutors.Signal call() throws Exception
{
try {
valveOn.set(false);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
return ScheduledExecutors.Signal.STOP;
}
}
);
beginRejectionPolicy = true;
}
@Override
public boolean hasMore()
{
return valveOn.get() && firehose.hasMore();
}
@Override
public InputRow nextRow()
{
InputRow next = firehose.nextRow();
if (!beginRejectionPolicy || rejectionPolicy.accept(next.getTimestampFromEpoch())) {
return next;
}
return null;
}
@Override
public Runnable commit()
{
return firehose.commit();
}
@Override
public void close() throws IOException
{
firehose.close();
}
}

View File

@ -0,0 +1,42 @@
package com.metamx.druid.realtime.plumber;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.Period;
/**
*/
public class IntervalRejectionPolicyFactory implements RejectionPolicyFactory
{
private final Interval interval;
public IntervalRejectionPolicyFactory(Interval interval)
{
this.interval = interval;
}
@Override
public RejectionPolicy create(Period windowPeriod)
{
return new RejectionPolicy()
{
@Override
public DateTime getCurrMaxTime()
{
return new DateTime();
}
@Override
public boolean accept(long timestamp)
{
return interval.contains(timestamp);
}
@Override
public String toString()
{
return String.format("interval-%s", interval);
}
};
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
@ -88,6 +89,9 @@ public class RealtimePlumberSchool implements PlumberSchool
private final Period windowPeriod;
private final File basePersistDirectory;
private final IndexGranularity segmentGranularity;
private final Object handoffCondition = new Object();
private ServiceEmitter emitter;
private volatile VersioningPolicy versioningPolicy = null;
private volatile RejectionPolicyFactory rejectionPolicyFactory = null;
@ -96,7 +100,6 @@ public class RealtimePlumberSchool implements PlumberSchool
private volatile SegmentAnnouncer segmentAnnouncer = null;
private volatile SegmentPublisher segmentPublisher = null;
private volatile ServerView serverView = null;
private ServiceEmitter emitter;
@JsonCreator
public RealtimePlumberSchool(
@ -230,7 +233,7 @@ public class RealtimePlumberSchool implements PlumberSchool
final Function<Query<T>, ServiceMetricEvent.Builder> builderFn =
new Function<Query<T>, ServiceMetricEvent.Builder>()
{
private final QueryToolChest<T,Query<T>> toolchest = factory.getToolchest();
private final QueryToolChest<T, Query<T>> toolchest = factory.getToolchest();
@Override
public ServiceMetricEvent.Builder apply(@Nullable Query<T> input)
@ -303,25 +306,45 @@ public class RealtimePlumberSchool implements PlumberSchool
@Override
public void finishJob()
{
stopped = true;
log.info("Shutting down...");
for (final Sink sink : sinks.values()) {
while (!sinks.isEmpty()) {
try {
segmentAnnouncer.unannounceSegment(sink.getSegment());
log.info(
"Cannot shut down yet! Sinks remaining: %s",
Joiner.on(", ").join(
Iterables.transform(
sinks.values(),
new Function<Sink, String>()
{
@Override
public String apply(Sink input)
{
return input.getSegment().getIdentifier();
}
}
)
)
);
synchronized (handoffCondition) {
while (!sinks.isEmpty()) {
handoffCondition.wait();
}
}
}
catch (Exception e) {
log.makeAlert("Failed to unannounce segment on shutdown")
.addData("segment", sink.getSegment())
.emit();
catch (InterruptedException e) {
throw Throwables.propagate(e);
}
}
// scheduledExecutor is shutdown here, but persistExecutor is shutdown when the
// ServerView sends it a new segment callback
if (scheduledExecutor != null) {
scheduledExecutor.shutdown();
}
stopped = true;
}
private void initializeExecutors()
@ -430,7 +453,12 @@ public class RealtimePlumberSchool implements PlumberSchool
try {
segmentAnnouncer.unannounceSegment(sink.getSegment());
FileUtils.deleteDirectory(computePersistDir(schema, sink.getInterval()));
log.info("Removing sinkKey %d for segment %s", sinkKey, sink.getSegment().getIdentifier());
sinks.remove(sinkKey);
synchronized (handoffCondition) {
handoffCondition.notifyAll();
}
}
catch (IOException e) {
log.makeAlert(e, "Unable to delete old segment for dataSource[%s].", schema.getDataSource())

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.3.35-SNAPSHOT</version>
<version>0.3.36-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -376,6 +376,10 @@ public class InfoResource
@QueryParam("interval") final String interval
)
{
// TODO: will likely be all rewritten once Guice introduced
if (indexingServiceClient == null) {
return Response.status(Response.Status.OK).entity(ImmutableMap.of("error", "no indexing service found")).build();
}
if (kill != null && Boolean.valueOf(kill)) {
indexingServiceClient.killSegments(dataSourceName, new Interval(interval));
} else {

View File

@ -433,11 +433,17 @@ public class DruidMaster
final List<Pair<? extends MasterRunnable, Duration>> masterRunnables = Lists.newArrayList();
masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod()));
if (config.isMergeSegments() && indexingServiceClient != null) {
if (indexingServiceClient != null) {
masterRunnables.add(
Pair.of(
new MasterSegmentMergerRunnable(configManager.watch(MergerWhitelist.CONFIG_KEY, MergerWhitelist.class)),
new MasterIndexingServiceRunnable(
makeIndexingServiceHelpers(
configManager.watch(
MergerWhitelist.CONFIG_KEY,
MergerWhitelist.class
)
)
),
config.getMasterSegmentMergerPeriod()
)
);
@ -502,6 +508,41 @@ public class DruidMaster
}
}
private List<DruidMasterHelper> makeIndexingServiceHelpers(final AtomicReference<MergerWhitelist> whitelistRef)
{
List<DruidMasterHelper> helpers = Lists.newArrayList();
helpers.add(new DruidMasterSegmentInfoLoader(DruidMaster.this));
if (config.isConvertSegments()) {
helpers.add(new DruidMasterVersionConverter(indexingServiceClient, whitelistRef));
}
if (config.isMergeSegments()) {
helpers.add(new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef));
helpers.add(
new DruidMasterHelper()
{
@Override
public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params)
{
MasterStats stats = params.getMasterStats();
log.info("Issued merge requests for %s segments", stats.getGlobalStats().get("mergedCount").get());
params.getEmitter().emit(
new ServiceMetricEvent.Builder().build(
"master/merge/count", stats.getGlobalStats().get("mergedCount")
)
);
return params;
}
}
);
}
return ImmutableList.copyOf(helpers);
}
public static class DruidMasterVersionConverter implements DruidMasterHelper
{
private final IndexingServiceClient indexingServiceClient;
@ -728,34 +769,11 @@ public class DruidMaster
}
}
private class MasterSegmentMergerRunnable extends MasterRunnable
private class MasterIndexingServiceRunnable extends MasterRunnable
{
private MasterSegmentMergerRunnable(final AtomicReference<MergerWhitelist> whitelistRef)
private MasterIndexingServiceRunnable(List<DruidMasterHelper> helpers)
{
super(
ImmutableList.of(
new DruidMasterSegmentInfoLoader(DruidMaster.this),
new DruidMasterVersionConverter(indexingServiceClient, whitelistRef),
new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef),
new DruidMasterHelper()
{
@Override
public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params)
{
MasterStats stats = params.getMasterStats();
log.info("Issued merge requests for %s segments", stats.getGlobalStats().get("mergedCount").get());
params.getEmitter().emit(
new ServiceMetricEvent.Builder().build(
"master/merge/count", stats.getGlobalStats().get("mergedCount")
)
);
return params;
}
}
)
);
super(helpers);
}
}
}

View File

@ -63,6 +63,12 @@ public abstract class DruidMasterConfig
return true;
}
@Config("druid.master.conversion.on")
public boolean isConvertSegments()
{
return true;
}
@Config("druid.master.merger.service")
public String getMergerServiceName()
{