mirror of https://github.com/apache/druid.git
Merge branch 'master' of github.com:metamx/druid
This commit is contained in:
commit
3cfa081c6c
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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");
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -80,6 +80,12 @@ public class SegmentInsertAction implements TaskAction<Set<DataSegment>>
|
|||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAudited()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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));
|
||||
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -0,0 +1,5 @@
|
|||
package com.metamx.druid.merger.common.tasklogs;
|
||||
|
||||
public interface TaskLogs extends TaskLogProvider, TaskLogPusher
|
||||
{
|
||||
}
|
|
@ -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
|
||||
);
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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"))
|
||||
);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
2
pom.xml
2
pom.xml
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -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())
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
Loading…
Reference in New Issue