From d7379db6fe4bbb9d52768752a0cbdfb8b7116dbd Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Mon, 4 Mar 2013 13:09:42 -0600 Subject: [PATCH 01/17] 1) Initial commit: Converter Task --- .../com/metamx/druid/index/v1/IndexIO.java | 15 +++- .../merger/common/task/AbstractTask.java | 20 +++++ .../druid/merger/common/task/KillTask.java | 30 +++++-- .../metamx/druid/merger/common/task/Task.java | 4 +- .../merger/common/task/V8toV9UpgradeTask.java | 36 --------- .../common/task/VersionConverterSubTask.java | 79 +++++++++++++++++++ .../common/task/VersionConverterTask.java | 74 +++++++++++++++++ 7 files changed, 212 insertions(+), 46 deletions(-) delete mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index d26e73f5b3c..621989b0d08 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -199,6 +199,19 @@ public class IndexIO } } + public static boolean convertSegment(File toConvert, File converted) throws IOException + { + final int version = getVersionFromDir(toConvert); + + switch (version) { + case 8: + DefaultIndexIOHandler.convertV8toV9(toConvert, converted); + return true; + default: + return false; + } + } + public static interface IndexIOHandler { /** @@ -229,7 +242,7 @@ public class IndexIO public void storeLatest(Index index, File file) throws IOException; } - static class DefaultIndexIOHandler implements IndexIOHandler + public static class DefaultIndexIOHandler implements IndexIOHandler { private static final Logger log = new Logger(DefaultIndexIOHandler.class); @Override diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index 899e2066fb0..119eab3ec87 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -20,15 +20,19 @@ package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Joiner; import com.google.common.base.Objects; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.SegmentListUsedAction; import org.joda.time.Interval; public abstract class AbstractTask implements Task { + private static final Joiner ID_JOINER = Joiner.on("_"); + private final String id; private final String groupId; private final String dataSource; @@ -91,4 +95,20 @@ public abstract class AbstractTask implements Task .add("interval", getImplicitLockInterval()) .toString(); } + + /** Start helper methods **/ + public static String joinId(Object... objects) + { + return ID_JOINER.join(objects); + } + + public SegmentListUsedAction makeListUsedAction() + { + return new SegmentListUsedAction(this, getDataSource(), getFixedInterval().get()); + } + + public TaskStatus success() + { + return TaskStatus.success(getId()); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index 0c1602c759d..35a3c2edbaf 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -1,3 +1,22 @@ +/* + * 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.task; import com.google.common.collect.ImmutableSet; @@ -64,14 +83,9 @@ public class KillTask extends AbstractTask } // List unused segments - final List unusedSegments = toolbox.getTaskActionClient() - .submit( - new SegmentListUnusedAction( - this, - myLock.getDataSource(), - myLock.getInterval() - ) - ); + final List unusedSegments = toolbox + .getTaskActionClient() + .submit(new SegmentListUnusedAction(this, myLock.getDataSource(), myLock.getInterval())); // Verify none of these segments have versions > lock version for(final DataSegment unusedSegment : unusedSegments) { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 8d93486ff6b..b1a4598a2d2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -50,7 +50,9 @@ import org.joda.time.Interval; @JsonSubTypes.Type(name = "index", value = IndexTask.class), @JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class), @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class), - @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class) + @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), + @JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class), + @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterSubTask.class), }) public interface Task { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java deleted file mode 100644 index 28d3ab1fec1..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java +++ /dev/null @@ -1,36 +0,0 @@ -package com.metamx.druid.merger.common.task; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -/** - */ -public class V8toV9UpgradeTask extends AbstractTask -{ - public V8toV9UpgradeTask( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval - ) - { - super( - String.format("v8tov9_%s_%s_%s", dataSource, interval.toString().replace("/", "_"), new DateTime()), - dataSource, - interval - ); - } - - @Override - public String getType() - { - return "8to9"; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception - { - throw new UnsupportedOperationException(); - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java new file mode 100644 index 00000000000..4b5c3d251f0 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java @@ -0,0 +1,79 @@ +/* + * 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.task; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Sets; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; + +import java.io.File; +import java.util.Arrays; +import java.util.Map; + +/** + */ +public class VersionConverterSubTask extends AbstractTask +{ + private final DataSegment segment; + + protected VersionConverterSubTask( + @JsonProperty("groupId") String groupId, + @JsonProperty("segment") DataSegment segment + ) + { + super( + joinId( + groupId, + "sub", + segment.getInterval().getStart(), + segment.getInterval().getEnd(), + segment.getShardSpec().getPartitionNum() + ), + segment.getDataSource(), + segment.getInterval() + ); + this.segment = segment; + } + + @Override + public String getType() + { + return "version_converter_sub"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final Map localSegments = toolbox.getSegments(this, Arrays.asList(segment)); + + final File location = localSegments.get(segment); + final File outLocation = new File(location, "v9_out"); + if (IndexIO.convertSegment(location, outLocation)) { + final DataSegment updatedSegment = toolbox.getSegmentPusher().push(outLocation, segment); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, Sets.newHashSet(updatedSegment))); + } + + return success(); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java new file mode 100644 index 00000000000..062f3751767 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java @@ -0,0 +1,74 @@ +package com.metamx.druid.merger.common.task; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.metamx.common.ISE; +import com.metamx.druid.client.DataSegment; +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 org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; + +/** + */ +public class VersionConverterTask extends AbstractTask +{ + private static final String TYPE = "version_converter"; + + public VersionConverterTask( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval + ) + { + super( + joinId(TYPE, dataSource, interval.getStart(), interval.getEnd(), new DateTime()), + dataSource, + interval + ); + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + throw new ISE("Should never run, %s just exists to create subtasks", this.getClass().getSimpleName()); + } + + @Override + public TaskStatus preflight(TaskToolbox toolbox) throws Exception + { + final TaskActionClient taskClient = toolbox.getTaskActionClient(); + + List segments = taskClient.submit(makeListUsedAction()); + + taskClient.submit( + new SpawnTasksAction( + this, + Lists.transform( + segments, + new Function() + { + @Override + public Task apply(@Nullable DataSegment input) + { + return new VersionConverterSubTask(getGroupId(), input); + } + } + ) + ) + ); + + return TaskStatus.success(getId()); + } +} From a9f7094ef3ceb4ea4ccc3df7e3dd064dbb1094b7 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Mon, 4 Mar 2013 18:10:32 -0600 Subject: [PATCH 02/17] 1) Remove the need for TaskActions to require a Task as a constructor parameter --- .../merger/common/TaskToolboxFactory.java | 92 +++++++++++++++++++ .../common/actions/LocalTaskActionClient.java | 11 ++- .../actions/LocalTaskActionClientFactory.java | 43 +++++++++ .../common/actions/LockAcquireAction.java | 11 +-- .../merger/common/actions/LockListAction.java | 18 +--- .../common/actions/LockReleaseAction.java | 11 +-- .../actions/RemoteTaskActionClient.java | 7 +- .../RemoteTaskActionClientFactory.java | 47 ++++++++++ .../common/actions/SegmentInsertAction.java | 11 +-- .../actions/SegmentListUnusedAction.java | 11 +-- .../common/actions/SegmentListUsedAction.java | 11 +-- .../common/actions/SegmentNukeAction.java | 11 +-- .../common/actions/SpawnTasksAction.java | 11 +-- .../merger/common/actions/TaskAction.java | 3 +- .../actions/TaskActionClientFactory.java | 29 ++++++ .../common/actions/TaskActionHolder.java | 54 +++++++++++ .../merger/common/task/AbstractTask.java | 4 +- .../druid/merger/common/task/DeleteTask.java | 4 +- .../merger/common/task/HadoopIndexTask.java | 23 ++++- .../task/IndexDeterminePartitionsTask.java | 2 +- .../common/task/IndexGeneratorTask.java | 4 +- .../druid/merger/common/task/IndexTask.java | 2 +- .../druid/merger/common/task/KillTask.java | 6 +- .../druid/merger/common/task/MergeTask.java | 6 +- .../metamx/druid/merger/common/task/Task.java | 2 +- .../common/task/VersionConverterSubTask.java | 2 +- .../common/task/VersionConverterTask.java | 3 +- .../merger/coordinator/DbTaskStorage.java | 6 +- ...torage.java => HeapMemoryTaskStorage.java} | 8 +- .../merger/coordinator/LocalTaskRunner.java | 9 +- .../druid/merger/coordinator/TaskLockbox.java | 19 ++++ .../coordinator/TaskMasterLifecycle.java | 14 +-- .../druid/merger/coordinator/TaskStorage.java | 2 +- .../merger/coordinator/exec/TaskConsumer.java | 10 +- .../http/IndexerCoordinatorNode.java | 51 +++++----- .../http/IndexerCoordinatorResource.java | 8 +- .../druid/merger/worker/TaskMonitor.java | 10 +- .../druid/merger/worker/http/WorkerNode.java | 45 +++++---- .../coordinator/RemoteTaskRunnerTest.java | 3 +- .../merger/coordinator/TaskLifecycleTest.java | 86 ++++++++--------- .../merger/coordinator/TaskQueueTest.java | 35 +++---- pom.xml | 10 +- .../metamx/druid/realtime/RealtimeNode.java | 38 +------- .../druid/initialization/ServerInit.java | 39 ++++++++ 44 files changed, 549 insertions(+), 283 deletions(-) create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java rename merger/src/main/java/com/metamx/druid/merger/coordinator/{LocalTaskStorage.java => HeapMemoryTaskStorage.java} (95%) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java new file mode 100644 index 00000000000..9a804bef79e --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java @@ -0,0 +1,92 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.merger.common; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Maps; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.MMappedQueryableIndexFactory; +import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.SegmentLoaderConfig; +import com.metamx.druid.loading.SegmentLoadingException; +import com.metamx.druid.loading.SingleSegmentLoader; +import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.merger.common.actions.TaskActionClientFactory; +import com.metamx.druid.merger.common.config.TaskConfig; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.emitter.service.ServiceEmitter; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; + +import java.io.File; +import java.util.List; +import java.util.Map; + +/** + * Stuff that may be needed by a Task in order to conduct its business. + */ +public class TaskToolboxFactory +{ + private final TaskConfig config; + private final TaskActionClientFactory taskActionClient; + private final ServiceEmitter emitter; + private final RestS3Service s3Client; + private final DataSegmentPusher segmentPusher; + private final SegmentKiller segmentKiller; + private final ObjectMapper objectMapper; + + public TaskToolboxFactory( + TaskConfig config, + TaskActionClientFactory taskActionClient, + ServiceEmitter emitter, + RestS3Service s3Client, + DataSegmentPusher segmentPusher, + SegmentKiller segmentKiller, + ObjectMapper objectMapper + ) + { + this.config = config; + this.taskActionClient = taskActionClient; + this.emitter = emitter; + this.s3Client = s3Client; + this.segmentPusher = segmentPusher; + this.segmentKiller = segmentKiller; + this.objectMapper = objectMapper; + } + + public ObjectMapper getObjectMapper() + { + return objectMapper; + } + + public TaskToolbox build(Task task) + { + return new TaskToolbox( + config, + taskActionClient == null ? null : taskActionClient.create(task), + emitter, + s3Client, + segmentPusher, + segmentKiller, + objectMapper + ); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java index f6740064f52..e36dbf65a6c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java @@ -1,17 +1,20 @@ package com.metamx.druid.merger.common.actions; +import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskStorage; import com.metamx.emitter.EmittingLogger; public class LocalTaskActionClient implements TaskActionClient { + private final Task task; private final TaskStorage storage; private final TaskActionToolbox toolbox; private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class); - public LocalTaskActionClient(TaskStorage storage, TaskActionToolbox toolbox) + public LocalTaskActionClient(Task task, TaskStorage storage, TaskActionToolbox toolbox) { + this.task = task; this.storage = storage; this.toolbox = toolbox; } @@ -19,15 +22,15 @@ public class LocalTaskActionClient implements TaskActionClient @Override public RetType submit(TaskAction taskAction) { - final RetType ret = taskAction.perform(toolbox); + final RetType ret = taskAction.perform(task, toolbox); // Add audit log try { - storage.addAuditLog(taskAction); + storage.addAuditLog(task, taskAction); } catch (Exception e) { log.makeAlert(e, "Failed to record action in audit log") - .addData("task", taskAction.getTask().getId()) + .addData("task", task.getId()) .addData("actionClass", taskAction.getClass().getName()) .emit(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java new file mode 100644 index 00000000000..89e37ceafab --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java @@ -0,0 +1,43 @@ +/* + * 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.actions; + +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.TaskStorage; + +/** + */ +public class LocalTaskActionClientFactory implements TaskActionClientFactory +{ + private final TaskStorage storage; + private final TaskActionToolbox toolbox; + + public LocalTaskActionClientFactory(TaskStorage storage, TaskActionToolbox toolbox) + { + this.storage = storage; + this.toolbox = toolbox; + } + + @Override + public TaskActionClient create(Task task) + { + return new LocalTaskActionClient(task, storage, toolbox); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java index f669af33625..de325ba274f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java @@ -11,25 +11,16 @@ import org.joda.time.Interval; public class LockAcquireAction implements TaskAction> { - private final Task task; private final Interval interval; @JsonCreator public LockAcquireAction( - @JsonProperty("task") Task task, @JsonProperty("interval") Interval interval ) { - this.task = task; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Interval getInterval() { @@ -42,7 +33,7 @@ public class LockAcquireAction implements TaskAction> } @Override - public Optional perform(TaskActionToolbox toolbox) + public Optional perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getTaskLockbox().tryLock(task, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java index e0e3eddb71f..06a2879ec47 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java @@ -12,29 +12,13 @@ import java.util.List; public class LockListAction implements TaskAction> { - private final Task task; - - @JsonCreator - public LockListAction( - @JsonProperty("task") Task task - ) - { - this.task = task; - } - - @JsonProperty - public Task getTask() - { - return task; - } - public TypeReference> getReturnTypeReference() { return new TypeReference>() {}; } @Override - public List perform(TaskActionToolbox toolbox) + public List perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getTaskLockbox().findLocksForTask(task); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java index 5c84d024a50..b932e748ed1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java @@ -12,25 +12,16 @@ import java.util.List; public class LockReleaseAction implements TaskAction { - private final Task task; private final Interval interval; @JsonCreator public LockReleaseAction( - @JsonProperty("task") Task task, @JsonProperty("interval") Interval interval ) { - this.task = task; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Interval getInterval() { @@ -43,7 +34,7 @@ public class LockReleaseAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { try { toolbox.getTaskLockbox().unlock(task, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java index 26900e29942..5cebc6ee1ec 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java @@ -4,6 +4,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; +import com.metamx.druid.merger.common.task.Task; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.ToStringResponseHandler; import com.fasterxml.jackson.databind.ObjectMapper; @@ -16,14 +17,16 @@ import java.util.Map; public class RemoteTaskActionClient implements TaskActionClient { + private final Task task; private final HttpClient httpClient; private final ServiceProvider serviceProvider; private final ObjectMapper jsonMapper; private static final Logger log = new Logger(RemoteTaskActionClient.class); - public RemoteTaskActionClient(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) + public RemoteTaskActionClient(Task task, HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) { + this.task = task; this.httpClient = httpClient; this.serviceProvider = serviceProvider; this.jsonMapper = jsonMapper; @@ -33,7 +36,7 @@ public class RemoteTaskActionClient implements TaskActionClient public RetType submit(TaskAction taskAction) { try { - byte[] dataToSend = jsonMapper.writeValueAsBytes(taskAction); + byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction)); final String response = httpClient.post(getServiceUri().toURL()) .setContent("application/json", dataToSend) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java new file mode 100644 index 00000000000..659042bb592 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java @@ -0,0 +1,47 @@ +/* + * 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.actions; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.http.client.HttpClient; +import com.netflix.curator.x.discovery.ServiceProvider; + +/** + */ +public class RemoteTaskActionClientFactory implements TaskActionClientFactory +{ + private final HttpClient httpClient; + private final ServiceProvider serviceProvider; + private final ObjectMapper jsonMapper; + + public RemoteTaskActionClientFactory(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) + { + this.httpClient = httpClient; + this.serviceProvider = serviceProvider; + this.jsonMapper = jsonMapper; + } + + @Override + public TaskActionClient create(Task task) + { + return new RemoteTaskActionClient(task, httpClient, serviceProvider, jsonMapper); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java index 75ad4a9161f..5354e14878c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java @@ -18,25 +18,16 @@ import java.util.Set; public class SegmentInsertAction implements TaskAction { - private final Task task; private final Set segments; @JsonCreator public SegmentInsertAction( - @JsonProperty("task") Task task, @JsonProperty("segments") Set segments ) { - this.task = task; this.segments = ImmutableSet.copyOf(segments); } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Set getSegments() { @@ -49,7 +40,7 @@ public class SegmentInsertAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { if(!toolbox.taskLockCoversSegments(task, segments, false)) { throw new ISE("Segments not covered by locks for task: %s", task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java index b20d130064e..56304533a68 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java @@ -12,28 +12,19 @@ import java.util.List; public class SegmentListUnusedAction implements TaskAction> { - private final Task task; private final String dataSource; private final Interval interval; @JsonCreator public SegmentListUnusedAction( - @JsonProperty("task") Task task, @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval ) { - this.task = task; this.dataSource = dataSource; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public String getDataSource() { @@ -52,7 +43,7 @@ public class SegmentListUnusedAction implements TaskAction> } @Override - public List perform(TaskActionToolbox toolbox) + public List perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java index 0395057fe83..a776ed641cc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java @@ -12,28 +12,19 @@ import java.util.List; public class SegmentListUsedAction implements TaskAction> { - private final Task task; private final String dataSource; private final Interval interval; @JsonCreator public SegmentListUsedAction( - @JsonProperty("task") Task task, @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval ) { - this.task = task; this.dataSource = dataSource; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public String getDataSource() { @@ -52,7 +43,7 @@ public class SegmentListUsedAction implements TaskAction> } @Override - public List perform(TaskActionToolbox toolbox) + public List perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java index f1b61c58d9f..2ebedec0daf 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java @@ -18,25 +18,16 @@ import java.util.Set; public class SegmentNukeAction implements TaskAction { - private final Task task; private final Set segments; @JsonCreator public SegmentNukeAction( - @JsonProperty("task") Task task, @JsonProperty("segments") Set segments ) { - this.task = task; this.segments = ImmutableSet.copyOf(segments); } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Set getSegments() { @@ -49,7 +40,7 @@ public class SegmentNukeAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { if(!toolbox.taskLockCoversSegments(task, segments, true)) { throw new ISE("Segments not covered by locks for task: %s", task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java index a7a73d8eac7..ec48430c49a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java @@ -11,25 +11,16 @@ import java.util.List; public class SpawnTasksAction implements TaskAction { - private final Task task; private final List newTasks; @JsonCreator public SpawnTasksAction( - @JsonProperty("task") Task task, @JsonProperty("newTasks") List newTasks ) { - this.task = task; this.newTasks = ImmutableList.copyOf(newTasks); } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public List getNewTasks() { @@ -42,7 +33,7 @@ public class SpawnTasksAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { try { for(final Task newTask : newTasks) { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java index 7dedf50aad9..019b14a3b62 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java @@ -18,7 +18,6 @@ import com.fasterxml.jackson.core.type.TypeReference; }) public interface TaskAction { - public Task getTask(); // TODO Look into replacing this with task ID so stuff serializes smaller public TypeReference getReturnTypeReference(); // T_T - public RetType perform(TaskActionToolbox toolbox); + public RetType perform(Task task, TaskActionToolbox toolbox); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java new file mode 100644 index 00000000000..2784a442f31 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java @@ -0,0 +1,29 @@ +/* + * 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.actions; + +import com.metamx.druid.merger.common.task.Task; + +/** + */ +public interface TaskActionClientFactory +{ + public TaskActionClient create(Task task); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java new file mode 100644 index 00000000000..a440447a226 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java @@ -0,0 +1,54 @@ +/* + * 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.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.druid.merger.common.task.Task; + +/** + */ +public class TaskActionHolder +{ + private final Task task; + private final TaskAction action; + + @JsonCreator + public TaskActionHolder( + @JsonProperty("task") Task task, + @JsonProperty("action") TaskAction action + ) + { + this.task = task; + this.action = action; + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public TaskAction getAction() + { + return action; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index 119eab3ec87..c2f9a792bb0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -102,9 +102,9 @@ public abstract class AbstractTask implements Task return ID_JOINER.join(objects); } - public SegmentListUsedAction makeListUsedAction() + public SegmentListUsedAction makeImplicitListUsedAction() { - return new SegmentListUsedAction(this, getDataSource(), getFixedInterval().get()); + return new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get()); } public TaskStatus success() diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 991d76228dd..2f10861ef5f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -77,7 +77,7 @@ public class DeleteTask extends AbstractTask public TaskStatus run(TaskToolbox toolbox) throws Exception { // Strategy: Create an empty segment covering the interval to be deleted - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); final Interval interval = this.getImplicitLockInterval().get(); final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty); @@ -104,7 +104,7 @@ public class DeleteTask extends AbstractTask segment.getVersion() ); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java index 29c0c517f17..6e284557529 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java @@ -1,3 +1,22 @@ +/* + * 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.task; import com.fasterxml.jackson.annotation.JsonCreator; @@ -74,7 +93,7 @@ public class HadoopIndexTask extends AbstractTask ); // We should have a lock from before we started running - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); log.info("Setting version to: %s", myLock.getVersion()); configCopy.setVersion(myLock.getVersion()); @@ -105,7 +124,7 @@ public class HadoopIndexTask extends AbstractTask List publishedSegments = job.getPublishedSegments(); // Request segment pushes - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(publishedSegments))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments))); // Done return TaskStatus.success(getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 2886c8cc77a..47f72b12501 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -258,7 +258,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask } ); - toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks)); + toolbox.getTaskActionClient().submit(new SpawnTasksAction(nextTasks)); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index 7dc65797ea8..2a7b4683b8d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -100,7 +100,7 @@ public class IndexGeneratorTask extends AbstractTask public TaskStatus run(final TaskToolbox toolbox) throws Exception { // We should have a lock from before we started running - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); // We know this exists final Interval interval = getImplicitLockInterval().get(); @@ -193,7 +193,7 @@ public class IndexGeneratorTask extends AbstractTask ); // Request segment pushes - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(pushedSegments))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(pushedSegments))); // Done return TaskStatus.success(getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 246389470ef..6ef3481df03 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -144,7 +144,7 @@ public class IndexTask extends AbstractTask @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { - toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, toSubtasks())); + toolbox.getTaskActionClient().submit(new SpawnTasksAction(toSubtasks())); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index 35a3c2edbaf..d8a2d4cba0e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -72,7 +72,7 @@ public class KillTask extends AbstractTask public TaskStatus run(TaskToolbox toolbox) throws Exception { // Confirm we have a lock (will throw if there isn't exactly one element) - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); if(!myLock.getDataSource().equals(getDataSource())) { throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); @@ -85,7 +85,7 @@ public class KillTask extends AbstractTask // List unused segments final List unusedSegments = toolbox .getTaskActionClient() - .submit(new SegmentListUnusedAction(this, myLock.getDataSource(), myLock.getInterval())); + .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval())); // Verify none of these segments have versions > lock version for(final DataSegment unusedSegment : unusedSegments) { @@ -105,7 +105,7 @@ public class KillTask extends AbstractTask toolbox.getSegmentKiller().kill(unusedSegments); // Remove metadata for these segments - toolbox.getTaskActionClient().submit(new SegmentNukeAction(this, ImmutableSet.copyOf(unusedSegments))); + toolbox.getTaskActionClient().submit(new SegmentNukeAction(ImmutableSet.copyOf(unusedSegments))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 1791c2a097d..757a91c2598 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -119,7 +119,7 @@ public abstract class MergeTask extends AbstractTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); final ServiceEmitter emitter = toolbox.getEmitter(); final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments); @@ -170,7 +170,7 @@ public abstract class MergeTask extends AbstractTask emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); return TaskStatus.success(getId()); } @@ -215,7 +215,7 @@ public abstract class MergeTask extends AbstractTask final Set current = ImmutableSet.copyOf( Iterables.transform( toolbox.getTaskActionClient() - .submit(new SegmentListUsedAction(this, getDataSource(), getImplicitLockInterval().get())), + .submit(new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get())), toIdentifier ) ); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index b1a4598a2d2..8418ecf40a8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -52,7 +52,7 @@ import org.joda.time.Interval; @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class), - @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterSubTask.class), + @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterSubTask.class) }) public interface Task { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java index 4b5c3d251f0..db08e788e4d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java @@ -71,7 +71,7 @@ public class VersionConverterSubTask extends AbstractTask final File outLocation = new File(location, "v9_out"); if (IndexIO.convertSegment(location, outLocation)) { final DataSegment updatedSegment = toolbox.getSegmentPusher().push(outLocation, segment); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, Sets.newHashSet(updatedSegment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); } return success(); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java index 062f3751767..4106e210547 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java @@ -50,11 +50,10 @@ public class VersionConverterTask extends AbstractTask { final TaskActionClient taskClient = toolbox.getTaskActionClient(); - List segments = taskClient.submit(makeListUsedAction()); + List segments = taskClient.submit(makeImplicitListUsedAction()); taskClient.submit( new SpawnTasksAction( - this, Lists.transform( segments, new Function() diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java index 5f39efae0bb..b878885dd4a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java @@ -327,11 +327,11 @@ public class DbTaskStorage implements TaskStorage } @Override - public void addAuditLog(final TaskAction taskAction) + public void addAuditLog(final Task task, final TaskAction taskAction) { Preconditions.checkNotNull(taskAction, "taskAction"); - log.info("Logging action for task[%s]: %s", taskAction.getTask().getId(), taskAction); + log.info("Logging action for task[%s]: %s", task.getId(), taskAction); dbi.withHandle( new HandleCallback() @@ -345,7 +345,7 @@ public class DbTaskStorage implements TaskStorage dbConnectorConfig.getTaskLogTable() ) ) - .bind("task_id", taskAction.getTask().getId()) + .bind("task_id", task.getId()) .bind("log_payload", jsonMapper.writeValueAsString(taskAction)) .execute(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java similarity index 95% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java index 042f0b8196f..895804bc7fd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java @@ -40,14 +40,14 @@ import java.util.concurrent.locks.ReentrantLock; * Implements an in-heap TaskStorage facility, with no persistence across restarts. This class is not * thread safe. */ -public class LocalTaskStorage implements TaskStorage +public class HeapMemoryTaskStorage implements TaskStorage { private final ReentrantLock giant = new ReentrantLock(); private final Map tasks = Maps.newHashMap(); private final Multimap taskLocks = HashMultimap.create(); private final Multimap taskActions = ArrayListMultimap.create(); - private static final Logger log = new Logger(LocalTaskStorage.class); + private static final Logger log = new Logger(HeapMemoryTaskStorage.class); @Override public void insert(Task task, TaskStatus status) @@ -185,12 +185,12 @@ public class LocalTaskStorage implements TaskStorage } @Override - public void addAuditLog(TaskAction taskAction) + public void addAuditLog(Task task, TaskAction taskAction) { giant.lock(); try { - taskActions.put(taskAction.getTask().getId(), taskAction); + taskActions.put(task.getId(), taskAction); } finally { giant.unlock(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index 573152fce09..a4f58a361f8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -26,6 +26,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.task.Task; import org.apache.commons.io.FileUtils; @@ -38,17 +39,17 @@ import java.util.concurrent.ExecutorService; */ public class LocalTaskRunner implements TaskRunner { - private final TaskToolbox toolbox; + private final TaskToolboxFactory toolboxFactory; private final ExecutorService exec; private static final Logger log = new Logger(LocalTaskRunner.class); public LocalTaskRunner( - TaskToolbox toolbox, + TaskToolboxFactory toolboxFactory, ExecutorService exec ) { - this.toolbox = toolbox; + this.toolboxFactory = toolboxFactory; this.exec = exec; } @@ -61,6 +62,8 @@ public class LocalTaskRunner implements TaskRunner @Override public void run(final Task task, final TaskCallback callback) { + final TaskToolbox toolbox = toolboxFactory.build(task); + exec.submit( new Runnable() { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java index 07abd0594ac..0a4bd925d4d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java @@ -1,3 +1,22 @@ +/* + * 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.google.common.base.Function; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java index 0eb3a335bc4..7f95e7142c5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java @@ -26,6 +26,8 @@ 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.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.exec.TaskConsumer; import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler; @@ -49,7 +51,7 @@ public class TaskMasterLifecycle private final ReentrantLock giant = new ReentrantLock(); private final Condition mayBeStopped = giant.newCondition(); private final TaskQueue taskQueue; - private final TaskToolbox taskToolbox; + private final TaskToolboxFactory taskToolboxFactory; private volatile boolean leading = false; private volatile TaskRunner theRunner; @@ -58,7 +60,7 @@ public class TaskMasterLifecycle public TaskMasterLifecycle( final TaskQueue taskQueue, - final TaskToolbox taskToolbox, + final TaskToolboxFactory taskToolboxFactory, final IndexerCoordinatorConfig indexerCoordinatorConfig, final ServiceDiscoveryConfig serviceDiscoveryConfig, final TaskRunnerFactory runnerFactory, @@ -68,7 +70,7 @@ public class TaskMasterLifecycle ) { this.taskQueue = taskQueue; - this.taskToolbox = taskToolbox; + this.taskToolboxFactory = taskToolboxFactory; this.leaderSelector = new LeaderSelector( curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener() @@ -87,7 +89,7 @@ public class TaskMasterLifecycle final TaskConsumer taskConsumer = new TaskConsumer( taskQueue, taskRunner, - taskToolbox, + taskToolboxFactory, emitter ); @@ -217,8 +219,8 @@ public class TaskMasterLifecycle return taskQueue; } - public TaskToolbox getTaskToolbox() + public TaskToolbox getTaskToolbox(Task task) { - return taskToolbox; + return taskToolboxFactory.build(task); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java index 2e21f52876b..d6bfbfd889e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java @@ -69,7 +69,7 @@ public interface TaskStorage /** * Add an action taken by a task to the audit log. */ - public void addAuditLog(TaskAction taskAction); + public void addAuditLog(Task task, TaskAction taskAction); /** * Returns all actions taken by a task. diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index f1e31a9e5c1..48d3ecdc471 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -24,7 +24,7 @@ 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.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskRunner; @@ -36,7 +36,7 @@ public class TaskConsumer implements Runnable { private final TaskQueue queue; private final TaskRunner runner; - private final TaskToolbox toolbox; + private final TaskToolboxFactory toolboxFactory; private final ServiceEmitter emitter; private final Thread thready; @@ -47,13 +47,13 @@ public class TaskConsumer implements Runnable public TaskConsumer( TaskQueue queue, TaskRunner runner, - TaskToolbox toolbox, + TaskToolboxFactory toolboxFactory, ServiceEmitter emitter ) { this.queue = queue; this.runner = runner; - this.toolbox = toolbox; + this.toolboxFactory = toolboxFactory; this.emitter = emitter; this.thready = new Thread(this); } @@ -123,7 +123,7 @@ public class TaskConsumer implements Runnable // Run preflight checks TaskStatus preflightStatus; try { - preflightStatus = task.preflight(toolbox); + preflightStatus = task.preflight(toolboxFactory.build(task)); log.info("Preflight done for task: %s", task.getId()); } catch (Exception e) { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index f684981cf2d..901348908df 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -47,22 +47,21 @@ 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.DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.loading.S3SegmentKiller; import com.metamx.druid.loading.SegmentKiller; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.TaskActionToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; import com.metamx.druid.merger.coordinator.DbTaskStorage; +import com.metamx.druid.merger.coordinator.HeapMemoryTaskStorage; import com.metamx.druid.merger.coordinator.LocalTaskRunner; -import com.metamx.druid.merger.coordinator.LocalTaskStorage; import com.metamx.druid.merger.coordinator.MergerDBCoordinator; import com.metamx.druid.merger.coordinator.RemoteTaskRunner; import com.metamx.druid.merger.coordinator.RetryPolicyFactory; @@ -145,7 +144,8 @@ public class IndexerCoordinatorNode extends RegisteringNode private RestS3Service s3Service = null; private IndexerCoordinatorConfig config = null; private TaskConfig taskConfig = null; - private TaskToolbox taskToolbox = null; + private DataSegmentPusher segmentPusher = null; + private TaskToolboxFactory taskToolboxFactory = null; private MergerDBCoordinator mergerDBCoordinator = null; private TaskStorage taskStorage = null; private TaskQueue taskQueue = null; @@ -206,6 +206,12 @@ public class IndexerCoordinatorNode extends RegisteringNode return this; } + public IndexerCoordinatorNode setSegmentPusher(DataSegmentPusher segmentPusher) + { + this.segmentPusher = segmentPusher; + return this; + } + public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator) { this.mergerDBCoordinator = mergeDbCoordinator; @@ -250,6 +256,7 @@ public class IndexerCoordinatorNode extends RegisteringNode initializeTaskStorage(); initializeTaskLockbox(); initializeTaskQueue(); + initializeDataSegmentPusher(); initializeTaskToolbox(); initializeJacksonInjections(); initializeJacksonSubtypes(); @@ -329,7 +336,7 @@ public class IndexerCoordinatorNode extends RegisteringNode final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); taskMasterLifecycle = new TaskMasterLifecycle( taskQueue, - taskToolbox, + taskToolboxFactory, config, serviceDiscoveryConfig, taskRunnerFactory, @@ -393,7 +400,7 @@ public class IndexerCoordinatorNode extends RegisteringNode InjectableValues.Std injectables = new InjectableValues.Std(); injectables.addValue("s3Client", s3Service) - .addValue("segmentPusher", taskToolbox.getSegmentPusher()); + .addValue("segmentPusher", segmentPusher); jsonMapper.setInjectableValues(injectables); } @@ -462,26 +469,26 @@ public class IndexerCoordinatorNode extends RegisteringNode ); } + public void initializeDataSegmentPusher() + { + if (segmentPusher == null) { + segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper); + } + } + public void initializeTaskToolbox() { - if (taskToolbox == null) { - final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( - s3Service, - configFactory.build(S3DataSegmentPusherConfig.class), - jsonMapper - ); - final SegmentKiller segmentKiller = new S3SegmentKiller( - s3Service - ); - taskToolbox = new TaskToolbox( + if (taskToolboxFactory == null) { + final SegmentKiller segmentKiller = new S3SegmentKiller(s3Service); + taskToolboxFactory = new TaskToolboxFactory( taskConfig, - new LocalTaskActionClient( + new LocalTaskActionClientFactory( taskStorage, new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter) ), emitter, s3Service, - dataSegmentPusher, + segmentPusher, segmentKiller, jsonMapper ); @@ -536,7 +543,7 @@ public class IndexerCoordinatorNode extends RegisteringNode { if (taskStorage == null) { if (config.getStorageImpl().equals("local")) { - taskStorage = new LocalTaskStorage(); + taskStorage = new HeapMemoryTaskStorage(); } else if (config.getStorageImpl().equals("db")) { final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class); taskStorage = new DbTaskStorage(jsonMapper, dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI()); @@ -605,7 +612,7 @@ public class IndexerCoordinatorNode extends RegisteringNode public TaskRunner build() { final ExecutorService runnerExec = Executors.newFixedThreadPool(config.getNumLocalThreads()); - return new LocalTaskRunner(taskToolbox, runnerExec); + return new LocalTaskRunner(taskToolboxFactory, runnerExec); } }; } else { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index fd3e02bcad3..71db9113517 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -27,6 +27,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.actions.TaskAction; +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.TaskStorageQueryAdapter; @@ -174,9 +175,12 @@ public class IndexerCoordinatorResource @POST @Path("/action") @Produces("application/json") - public Response doAction(final TaskAction action) + public Response doAction(final TaskActionHolder holder) { - final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action); + final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask()) + .getTaskActionClient() + .submit(holder.getAction()); + final Map retMap = Maps.newHashMap(); retMap.put("result", ret); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index d7836d46faa..ec3a8d992e0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -23,6 +23,7 @@ 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.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.task.Task; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; @@ -45,21 +46,21 @@ public class TaskMonitor private final PathChildrenCache pathChildrenCache; private final CuratorFramework cf; private final WorkerCuratorCoordinator workerCuratorCoordinator; - private final TaskToolbox toolbox; + private final TaskToolboxFactory toolboxFactory; private final ExecutorService exec; public TaskMonitor( PathChildrenCache pathChildrenCache, CuratorFramework cf, WorkerCuratorCoordinator workerCuratorCoordinator, - TaskToolbox toolbox, + TaskToolboxFactory toolboxFactory, ExecutorService exec ) { this.pathChildrenCache = pathChildrenCache; this.cf = cf; this.workerCuratorCoordinator = workerCuratorCoordinator; - this.toolbox = toolbox; + this.toolboxFactory = toolboxFactory; this.exec = exec; } @@ -81,10 +82,11 @@ public class TaskMonitor throws Exception { if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final Task task = toolbox.getObjectMapper().readValue( + final Task task = toolboxFactory.getObjectMapper().readValue( cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()), Task.class ); + final TaskToolbox toolbox = toolboxFactory.build(task); if (workerCuratorCoordinator.statusExists(task.getId())) { log.warn("Got task %s that I am already running...", task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index d02ffa5d9e3..bad04040e73 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -35,6 +35,7 @@ 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.DataSegmentPusher; @@ -43,7 +44,9 @@ import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.loading.S3SegmentKiller; import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.actions.RemoteTaskActionClient; +import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -106,7 +109,8 @@ public class WorkerNode extends RegisteringNode private ServiceEmitter emitter = null; private TaskConfig taskConfig = null; private WorkerConfig workerConfig = null; - private TaskToolbox taskToolbox = null; + private DataSegmentPusher segmentPusher = null; + private TaskToolboxFactory taskToolboxFactory = null; private CuratorFramework curatorFramework = null; private ServiceDiscovery serviceDiscovery = null; private ServiceProvider coordinatorServiceProvider = null; @@ -149,9 +153,15 @@ public class WorkerNode extends RegisteringNode return this; } - public WorkerNode setTaskToolbox(TaskToolbox taskToolbox) + public WorkerNode setSegmentPusher(DataSegmentPusher segmentPusher) { - this.taskToolbox = taskToolbox; + this.segmentPusher = segmentPusher; + return this; + } + + public WorkerNode setTaskToolboxFactory(TaskToolboxFactory taskToolboxFactory) + { + this.taskToolboxFactory = taskToolboxFactory; return this; } @@ -195,6 +205,7 @@ public class WorkerNode extends RegisteringNode initializeCuratorFramework(); initializeServiceDiscovery(); initializeCoordinatorServiceProvider(); + initializeDataSegmentPusher(); initializeTaskToolbox(); initializeJacksonInjections(); initializeJacksonSubtypes(); @@ -271,7 +282,7 @@ public class WorkerNode extends RegisteringNode InjectableValues.Std injectables = new InjectableValues.Std(); injectables.addValue("s3Client", s3Service) - .addValue("segmentPusher", taskToolbox.getSegmentPusher()); + .addValue("segmentPusher", segmentPusher); jsonMapper.setInjectableValues(injectables); } @@ -334,23 +345,23 @@ public class WorkerNode extends RegisteringNode } } + public void initializeDataSegmentPusher() + { + if (segmentPusher == null) { + segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper); + } + } + public void initializeTaskToolbox() throws S3ServiceException { - if (taskToolbox == null) { - final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( - s3Service, - configFactory.build(S3DataSegmentPusherConfig.class), - jsonMapper - ); - final SegmentKiller segmentKiller = new S3SegmentKiller( - s3Service - ); - taskToolbox = new TaskToolbox( + if (taskToolboxFactory == null) { + final SegmentKiller segmentKiller = new S3SegmentKiller(s3Service); + taskToolboxFactory = new TaskToolboxFactory( taskConfig, - new RemoteTaskActionClient(httpClient, coordinatorServiceProvider, jsonMapper), + new RemoteTaskActionClientFactory(httpClient, coordinatorServiceProvider, jsonMapper), emitter, s3Service, - dataSegmentPusher, + segmentPusher, segmentKiller, jsonMapper ); @@ -417,7 +428,7 @@ public class WorkerNode extends RegisteringNode pathChildrenCache, curatorFramework, workerCuratorCoordinator, - taskToolbox, + taskToolboxFactory, workerExec ); lifecycle.addManagedInstance(taskMonitor); diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index 9242e19f355..18ce14a4555 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -12,6 +12,7 @@ import com.metamx.druid.merger.TestTask; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; @@ -280,7 +281,7 @@ public class RemoteTaskRunnerTest new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true), cf, workerCuratorCoordinator, - new TaskToolbox( + new TaskToolboxFactory( new TaskConfig() { @Override diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java index ae5b46fdfc1..3dc889b96f0 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java @@ -1,3 +1,22 @@ +/* + * 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.google.common.base.Optional; @@ -22,11 +41,11 @@ import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.SegmentKiller; -import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.LockAcquireAction; import com.metamx.druid.merger.common.actions.LockListAction; import com.metamx.druid.merger.common.actions.LockReleaseAction; @@ -59,7 +78,6 @@ import java.io.IOException; import java.util.Collection; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.concurrent.Executors; @@ -71,7 +89,7 @@ public class TaskLifecycleTest private TaskQueue tq = null; private TaskRunner tr = null; private MockMergerDBCoordinator mdc = null; - private TaskToolbox tb = null; + private TaskToolboxFactory tb = null; private TaskConsumer tc = null; TaskStorageQueryAdapter tsqa = null; @@ -91,12 +109,12 @@ public class TaskLifecycleTest tmp = Files.createTempDir(); - ts = new LocalTaskStorage(); + ts = new HeapMemoryTaskStorage(); tl = new TaskLockbox(ts); tq = new TaskQueue(ts, tl); mdc = newMockMDC(); - tb = new TaskToolbox( + tb = new TaskToolboxFactory( new TaskConfig() { @Override @@ -117,7 +135,7 @@ public class TaskLifecycleTest return null; } }, - new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())), + new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())), newMockEmitter(), null, // s3 client new DataSegmentPusher() @@ -137,16 +155,7 @@ public class TaskLifecycleTest } }, new DefaultObjectMapper() - ) - { - @Override - public Map getSegments( - Task task, List segments - ) throws SegmentLoadingException - { - return ImmutableMap.of(); - } - }; + ); tr = new LocalTaskRunner( tb, @@ -239,11 +248,12 @@ public class TaskLifecycleTest @Test public void testKillTask() throws Exception { - // TODO: Worst test ever + // This test doesn't actually do anything right now. We should actually put things into the Mocked coordinator + // Such that this test can test things... final Task killTask = new KillTask("foo", new Interval("2010-01-02/P2D")); - final TaskStatus mergedStatus = runTask(killTask); - Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode()); + final TaskStatus status = runTask(killTask); + Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); } @@ -273,8 +283,8 @@ public class TaskLifecycleTest // Sort of similar to what realtime tasks do: // Acquire lock for first interval - final Optional lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval1)); - final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + final Optional lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval1)); + final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertTrue("lock1 present", lock1.isPresent()); @@ -282,8 +292,8 @@ public class TaskLifecycleTest Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1); // Acquire lock for second interval - final Optional lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval2)); - final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + final Optional lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval2)); + final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertTrue("lock2 present", lock2.isPresent()); @@ -294,7 +304,6 @@ public class TaskLifecycleTest toolbox.getTaskActionClient() .submit( new SegmentInsertAction( - this, ImmutableSet.of( DataSegment.builder() .dataSource("foo") @@ -306,8 +315,8 @@ public class TaskLifecycleTest ); // Release first lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval1)); - final List locks3 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + toolbox.getTaskActionClient().submit(new LockReleaseAction(interval1)); + final List locks3 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3); @@ -316,7 +325,6 @@ public class TaskLifecycleTest toolbox.getTaskActionClient() .submit( new SegmentInsertAction( - this, ImmutableSet.of( DataSegment.builder() .dataSource("foo") @@ -328,8 +336,8 @@ public class TaskLifecycleTest ); // Release second lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval2)); - final List locks4 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + toolbox.getTaskActionClient().submit(new LockReleaseAction(interval2)); + final List locks4 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertEquals("locks4", ImmutableList.of(), locks4); @@ -363,7 +371,7 @@ public class TaskLifecycleTest { final TaskLock myLock = Iterables.getOnlyElement( toolbox.getTaskActionClient() - .submit(new LockListAction(this)) + .submit(new LockListAction()) ); final DataSegment segment = DataSegment.builder() @@ -372,7 +380,7 @@ public class TaskLifecycleTest .version(myLock.getVersion()) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; @@ -398,10 +406,7 @@ public class TaskLifecycleTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement( - toolbox.getTaskActionClient() - .submit(new LockListAction(this)) - ); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); final DataSegment segment = DataSegment.builder() .dataSource("ds") @@ -409,7 +414,7 @@ public class TaskLifecycleTest .version(myLock.getVersion()) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; @@ -435,10 +440,7 @@ public class TaskLifecycleTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement( - toolbox.getTaskActionClient() - .submit(new LockListAction(this)) - ); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); final DataSegment segment = DataSegment.builder() .dataSource("ds") @@ -446,7 +448,7 @@ public class TaskLifecycleTest .version(myLock.getVersion() + "1!!!1!!") .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 0d003551ea9..939dc9b6b21 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -26,7 +26,8 @@ import com.google.common.collect.Sets; import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.SpawnTasksAction; import com.metamx.druid.merger.common.actions.TaskActionToolbox; import com.metamx.druid.merger.common.task.AbstractTask; @@ -43,7 +44,7 @@ public class TaskQueueTest @Test public void testEmptyQueue() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); @@ -65,7 +66,7 @@ public class TaskQueueTest @Test public void testAddRemove() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); @@ -154,12 +155,12 @@ public class TaskQueueTest @Test public void testContinues() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); - final TaskToolbox tb = new TaskToolbox( + final TaskToolboxFactory tb = new TaskToolboxFactory( null, - new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), + new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)), null, null, null, @@ -181,7 +182,7 @@ public class TaskQueueTest Assert.assertNull("poll #2", tq.poll()); // report T1 done. Should cause T0 to be created - tq.notify(t1, t1.run(tb)); + tq.notify(t1, t1.run(tb.build(t1))); Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent()); Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable()); @@ -195,7 +196,7 @@ public class TaskQueueTest Assert.assertNull("poll #4", tq.poll()); // report T0 done. Should cause T0, T1 to be marked complete - tq.notify(t0, t0.run(tb)); + tq.notify(t0, t0.run(tb.build(t0))); Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent()); Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable()); @@ -211,12 +212,12 @@ public class TaskQueueTest @Test public void testConcurrency() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); - final TaskToolbox tb = new TaskToolbox( + final TaskToolboxFactory tb = new TaskToolboxFactory( null, - new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), + new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)), null, null, null, @@ -248,7 +249,7 @@ public class TaskQueueTest Thread.sleep(5); // Finish t0 - tq.notify(t0, t0.run(tb)); + tq.notify(t0, t0.run(tb.build(t0))); // take max number of tasks final Set taken = Sets.newHashSet(); @@ -280,7 +281,7 @@ public class TaskQueueTest Assert.assertNull("null poll #2", tq.poll()); // Finish t3 - tq.notify(t3, t3.run(tb)); + tq.notify(t3, t3.run(tb.build(t3))); // We should be able to get t2 now final Task wt2 = tq.poll(); @@ -291,7 +292,7 @@ public class TaskQueueTest Assert.assertNull("null poll #3", tq.poll()); // Finish t2 - tq.notify(t2, t2.run(tb)); + tq.notify(t2, t2.run(tb.build(t2))); // We should be able to get t4 // And it should be in group G0, but that group should have a different version than last time @@ -305,14 +306,14 @@ public class TaskQueueTest Assert.assertNotSame("wt4 version", wt2Lock.getVersion(), wt4Lock.getVersion()); // Kind of done testing at this point, but let's finish t4 anyway - tq.notify(t4, t4.run(tb)); + tq.notify(t4, t4.run(tb.build(t4))); Assert.assertNull("null poll #4", tq.poll()); } @Test public void testBootstrap() throws Exception { - final TaskStorage storage = new LocalTaskStorage(); + final TaskStorage storage = new HeapMemoryTaskStorage(); final TaskLockbox lockbox = new TaskLockbox(storage); storage.insert(newTask("T1", "G1", "bar", new Interval("2011-01-01/P1D")), TaskStatus.running("T1")); @@ -374,7 +375,7 @@ public class TaskQueueTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks)); + toolbox.getTaskActionClient().submit(new SpawnTasksAction(nextTasks)); return TaskStatus.success(id); } }; diff --git a/pom.xml b/pom.xml index 8e97764e169..bb8d325dc48 100644 --- a/pom.xml +++ b/pom.xml @@ -165,17 +165,17 @@ com.fasterxml.jackson.core jackson-annotations - 2.1.2 + 2.1.4 com.fasterxml.jackson.core jackson-core - 2.1.3 + 2.1.4 com.fasterxml.jackson.core jackson-databind - 2.1.4-mmx-2 + 2.1.4 com.fasterxml.jackson.datatype @@ -190,12 +190,12 @@ com.fasterxml.jackson.dataformat jackson-dataformat-smile - 2.1.3 + 2.1.4 com.fasterxml.jackson.jaxrs jackson-jaxrs-json-provider - 2.1.3 + 2.1.4 org.codehaus.jackson diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index 087e87d107f..fb47abab945 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -45,25 +45,15 @@ import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.LocalDataSegmentPusher; -import com.metamx.druid.loading.LocalDataSegmentPusherConfig; -import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.Monitor; - - - - -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.security.AWSCredentials; import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; @@ -259,31 +249,7 @@ public class RealtimeNode extends BaseServerNode private void initializeSegmentPusher() { if (dataSegmentPusher == null) { - final Properties props = getProps(); - if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) { - dataSegmentPusher = new LocalDataSegmentPusher( - getConfigFactory().build(LocalDataSegmentPusherConfig.class), getJsonMapper() - ); - } - else { - - final RestS3Service s3Client; - try { - s3Client = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) - ); - } - catch (S3ServiceException e) { - throw Throwables.propagate(e); - } - - dataSegmentPusher = new S3DataSegmentPusher( - s3Client, getConfigFactory().build(S3DataSegmentPusherConfig.class), getJsonMapper() - ); - } + dataSegmentPusher = ServerInit.getSegmentPusher(getProps(), getConfigFactory(), getJsonMapper()); } } diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 7cd6caf3c1b..cca4f910b77 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -19,17 +19,24 @@ package com.metamx.druid.initialization; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.DruidProcessingConfig; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.DelegatingSegmentLoader; import com.metamx.druid.loading.LocalDataSegmentPuller; +import com.metamx.druid.loading.LocalDataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusherConfig; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.S3DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.query.group.GroupByQueryEngine; @@ -48,12 +55,16 @@ import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; import com.metamx.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory; import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import com.metamx.druid.utils.PropUtils; +import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.security.AWSCredentials; import org.skife.config.ConfigurationObjectFactory; import java.lang.reflect.InvocationTargetException; import java.nio.ByteBuffer; import java.util.Map; +import java.util.Properties; import java.util.concurrent.atomic.AtomicLong; /** @@ -145,6 +156,34 @@ public class ServerInit return queryRunners; } + public static DataSegmentPusher getSegmentPusher( + final Properties props, + final ConfigurationObjectFactory configFactory, + final ObjectMapper jsonMapper + ) + { + if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) { + return new LocalDataSegmentPusher(configFactory.build(LocalDataSegmentPusherConfig.class), jsonMapper); + } + else { + + final RestS3Service s3Client; + try { + s3Client = new RestS3Service( + new AWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ); + } + catch (S3ServiceException e) { + throw Throwables.propagate(e); + } + + return new S3DataSegmentPusher(s3Client, configFactory.build(S3DataSegmentPusherConfig.class), jsonMapper); + } + } + private static class ComputeScratchPool extends StupidPool { private static final Logger log = new Logger(ComputeScratchPool.class); From 848d49ca87b72935f7c4f72923bec52e9d8aea60 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 4 Mar 2013 13:03:15 -0800 Subject: [PATCH 03/17] fix bugs in balancing logic and removing auto-scaled node --- .../merger/coordinator/RemoteTaskRunner.java | 2 - .../SimpleResourceManagementStrategy.java | 2 +- .../druid/master/BalancerCostAnalyzer.java | 74 +++++++++---------- .../druid/master/DruidMasterBalancer.java | 38 ++++------ .../metamx/druid/master/rules/LoadRule.java | 2 +- 5 files changed, 54 insertions(+), 64 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 739a910473f..e4a13136152 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -457,8 +457,6 @@ public class RemoteTaskRunner implements TaskRunner if (runningTasks.containsKey(taskId)) { log.info("Task %s just disappeared!", taskId); retryTask(runningTasks.get(taskId), worker.getHost()); - } else { - log.info("A task disappeared I didn't know about: %s", taskId); } } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java index 1cd3f8e46e6..74d34d718d2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -167,7 +167,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat ) ); - if (thoseLazyWorkers.isEmpty()) { + if (thoseLazyWorkers.size() <= minNumWorkers) { return false; } diff --git a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java index 799bccca581..6519785e76b 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerCostAnalyzer.java @@ -52,11 +52,12 @@ public class BalancerCostAnalyzer /** * Calculates the cost normalization. This is such that the normalized cost is lower bounded * by 1 (e.g. when each segment gets its own compute node). - * @param serverHolders - * A list of ServerHolders for a particular tier. - * @return The normalization value (the sum of the diagonal entries in the - * pairwise cost matrix). This is the cost of a cluster if each - * segment were to get its own compute node. + * + * @param serverHolders A list of ServerHolders for a particular tier. + * + * @return The normalization value (the sum of the diagonal entries in the + * pairwise cost matrix). This is the cost of a cluster if each + * segment were to get its own compute node. */ public double calculateNormalization(final List serverHolders) { @@ -71,9 +72,10 @@ public class BalancerCostAnalyzer /** * Calculates the initial cost of the Druid segment configuration. - * @param serverHolders - * A list of ServerHolders for a particular tier. - * @return The initial cost of the Druid tier. + * + * @param serverHolders A list of ServerHolders for a particular tier. + * + * @return The initial cost of the Druid tier. */ public double calculateInitialTotalCost(final List serverHolders) { @@ -96,11 +98,11 @@ public class BalancerCostAnalyzer * dataSourcePenalty: if two segments belong to the same data source, they are more likely to be involved * in the same queries * gapPenalty: it is more likely that segments close together in time will be queried together - * @param segment1 - * The first DataSegment. - * @param segment2 - * The second DataSegment. - * @return The joint cost of placing the two DataSegments together on one node. + * + * @param segment1 The first DataSegment. + * @param segment2 The second DataSegment. + * + * @return The joint cost of placing the two DataSegments together on one node. */ public double computeJointSegmentCosts(final DataSegment segment1, final DataSegment segment2) { @@ -140,12 +142,12 @@ public class BalancerCostAnalyzer /** * Sample from each server with probability proportional to the number of segments on that server. - * @param serverHolders - * A list of ServerHolders for a particular tier. - * @param numSegments - - * @return A ServerHolder sampled with probability proportional to the - * number of segments on that server + * + * @param serverHolders A list of ServerHolders for a particular tier. + * @param numSegments + * + * @return A ServerHolder sampled with probability proportional to the + * number of segments on that server */ private ServerHolder sampleServer(final List serverHolders, final int numSegments) { @@ -163,11 +165,11 @@ public class BalancerCostAnalyzer /** * The balancing application requires us to pick a proposal segment. - * @param serverHolders - * A list of ServerHolders for a particular tier. - * @param numSegments - * The total number of segments on a particular tier. - * @return A BalancerSegmentHolder sampled uniformly at random. + * + * @param serverHolders A list of ServerHolders for a particular tier. + * @param numSegments The total number of segments on a particular tier. + * + * @return A BalancerSegmentHolder sampled uniformly at random. */ public BalancerSegmentHolder pickSegmentToMove(final List serverHolders, final int numSegments) { @@ -175,7 +177,7 @@ public class BalancerCostAnalyzer ServerHolder fromServerHolder = sampleServer(serverHolders, numSegments); /** and actually pick that segment uniformly at random w.p. 1 / numSegmentsOnServer - so that the probability of picking a segment is 1 / totalSegments. */ + so that the probability of picking a segment is 1 / totalSegments. */ List segments = Lists.newArrayList(fromServerHolder.getServer().getSegments().values()); DataSegment proposalSegment = segments.get(rand.nextInt(segments.size())); @@ -184,18 +186,15 @@ public class BalancerCostAnalyzer /** * The assignment application requires us to supply a proposal segment. - * @param proposalSegment - * A DataSegment that we are proposing to move. - * @param serverHolders - * An iterable of ServerHolders for a particular tier. - * @param assign - * A boolean that is true if used in assignment else false in balancing. - * @return A ServerHolder with the new home for a segment. + * + * @param proposalSegment A DataSegment that we are proposing to move. + * @param serverHolders An iterable of ServerHolders for a particular tier. + * + * @return A ServerHolder with the new home for a segment. */ public ServerHolder findNewSegmentHome( final DataSegment proposalSegment, - final Iterable serverHolders, - final boolean assign + final Iterable serverHolders ) { final long proposalSegmentSize = proposalSegment.getSize(); @@ -204,10 +203,9 @@ public class BalancerCostAnalyzer for (ServerHolder server : serverHolders) { /** Don't calculate cost if the server doesn't have enough space or is loading the segment */ - if (proposalSegmentSize > server.getAvailableSize() - || server.isLoadingSegment(proposalSegment) - /** or if the ask is assignment and the server is serving the segment. */ - || (assign && server.isServingSegment(proposalSegment)) ) { + if (proposalSegmentSize > server.getAvailableSize() || + server.isLoadingSegment(proposalSegment) || + server.isServingSegment(proposalSegment)) { continue; } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 3d66a21f9d7..7678e85624d 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -19,6 +19,7 @@ package com.metamx.druid.master; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.metamx.common.guava.Comparators; @@ -66,9 +67,9 @@ public class DruidMasterBalancer implements DruidMasterHelper holder.reduceLifetime(); if (holder.getLifetime() <= 0) { log.makeAlert("[%s]: Balancer move segments queue has a segment stuck", tier) - .addData("segment", holder.getSegment().getIdentifier()) - .addData("server", holder.getFromServer().getStringProps()) - .emit(); + .addData("segment", holder.getSegment().getIdentifier()) + .addData("server", holder.getFromServer().getStringProps()) + .emit(); } } } @@ -95,7 +96,13 @@ public class DruidMasterBalancer implements DruidMasterHelper continue; } - final List serverHolderList = new ArrayList(entry.getValue()); + final List serverHolderList = Lists.newArrayList(entry.getValue()); + + if (serverHolderList.size() <= 1) { + log.info("[%s]: One or fewer servers found. Cannot balance.", tier); + continue; + } + int numSegments = 0; for (ServerHolder server : serverHolderList) { numSegments += server.getServer().getSegments().size(); @@ -107,19 +114,14 @@ public class DruidMasterBalancer implements DruidMasterHelper } int iter = 0; - while (iter < maxSegmentsToMove) { iter++; final BalancerSegmentHolder segmentToMove = analyzer.pickSegmentToMove(serverHolderList, numSegments); - final ServerHolder holder = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList, false); + final ServerHolder holder = analyzer.findNewSegmentHome(segmentToMove.getSegment(), serverHolderList); if (holder == null) { continue; } - final DruidServer toServer = holder.getServer(); - - if (!currentlyMovingSegments.get(tier).containsKey(segmentToMove.getSegment())) { - moveSegment(segmentToMove, toServer, params); - } + moveSegment(segmentToMove, holder.getServer(), params); } final double initialTotalCost = analyzer.calculateInitialTotalCost(serverHolderList); @@ -132,21 +134,13 @@ public class DruidMasterBalancer implements DruidMasterHelper stats.addToTieredStat("movedCount", tier, currentlyMovingSegments.get(tier).size()); log.info( - "Initial Total Cost: [%f], Normalization: [%f], Initial Normalized Cost: [%f], Segments Moved: [%d]", + "[%s]: Initial Total Cost: [%f], Normalization: [%f], Initial Normalized Cost: [%f], Segments Moved: [%d]", + tier, initialTotalCost, normalization, normalizedInitialCost, currentlyMovingSegments.get(tier).size() ); - - if (serverHolderList.size() <= 1) { - log.info( - "[%s]: One or fewer servers found. Cannot balance.", - tier - ); - continue; - } - } return params.buildFromExisting() @@ -194,7 +188,7 @@ public class DruidMasterBalancer implements DruidMasterHelper log.makeAlert(e, String.format("[%s] : Moving exception", segmentName)).emit(); } } else { - currentlyMovingSegments.get(toServer.getTier()).remove(segment); + currentlyMovingSegments.get(toServer.getTier()).remove(segmentName); } } diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index a59d9c795b5..4cd6acae640 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -91,7 +91,7 @@ public abstract class LoadRule implements Rule final MasterStats stats = new MasterStats(); while (totalReplicants < expectedReplicants) { - final ServerHolder holder = analyzer.findNewSegmentHome(segment, serverHolderList, true); + final ServerHolder holder = analyzer.findNewSegmentHome(segment, serverHolderList); if (holder == null) { log.warn( From 90637ab7b7865d91f576b86fe221a50355d979c2 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 4 Mar 2013 13:08:09 -0800 Subject: [PATCH 04/17] [maven-release-plugin] prepare release druid-0.3.16 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 7735497764a..7fcc0fff2ae 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/common/pom.xml b/common/pom.xml index 212e8f4c035..b7846385378 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index ae429a40002..098d00dd7da 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.16-SNAPSHOT + 0.3.16 com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/examples/pom.xml b/examples/pom.xml index ce809546e0b..ceafebe0f40 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 2f14eb10ceb..348270455eb 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 0f3d21ae1a0..04d1424cab1 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/index-common/pom.xml b/index-common/pom.xml index 2553823ffab..53be8c60664 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/indexer/pom.xml b/indexer/pom.xml index 4338b90e3ce..1a07d1896d2 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/merger/pom.xml b/merger/pom.xml index 80e5667882c..979ccb33315 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/pom.xml b/pom.xml index bb8d325dc48..892b025ae62 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.16-SNAPSHOT + 0.3.16 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 5e012a4bdf5..4cffee16f43 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 diff --git a/server/pom.xml b/server/pom.xml index 82d8754928a..4feb70e6646 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16-SNAPSHOT + 0.3.16 From 75cefa05b5a1c4b5b3bf0ea0a37b42dc6c4c5b5c Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 4 Mar 2013 13:08:26 -0800 Subject: [PATCH 05/17] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 7fcc0fff2ae..8ad2bea5ee2 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index b7846385378..c4d83b8f817 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 098d00dd7da..b71317d2352 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.16 + 0.3.17-SNAPSHOT com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index ceafebe0f40..8dfe0dca4d1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 348270455eb..4b1f8ab2a1d 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 04d1424cab1..60241675115 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 53be8c60664..d06dd7d561e 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 1a07d1896d2..006dea455d5 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 979ccb33315..3d7d4a05f30 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/pom.xml b/pom.xml index 892b025ae62..daa3e806318 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.16 + 0.3.17-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 4cffee16f43..37bc5d12c58 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 4feb70e6646..45584f572ba 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.16 + 0.3.17-SNAPSHOT From 665f1909c6e2c49da63f54d0f3e4e95abb08512e Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 4 Mar 2013 14:40:02 -0800 Subject: [PATCH 06/17] fixing a bug with indexing service not correctly killing worker nodes --- .../SimpleResourceManagementStrategy.java | 51 ++++++++++--------- 1 file changed, 27 insertions(+), 24 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java index 74d34d718d2..3f522908960 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; import com.metamx.druid.merger.coordinator.ZkWorker; @@ -32,7 +33,9 @@ import org.joda.time.DateTime; import org.joda.time.Duration; import java.util.Collection; +import java.util.Iterator; import java.util.List; +import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; /** @@ -83,10 +86,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat ) ); - for (String workerNodeId : workerNodeIds) { - currentlyProvisioning.remove(workerNodeId); - } - + currentlyProvisioning.removeAll(workerNodeIds); boolean nothingProvisioning = currentlyProvisioning.isEmpty(); if (nothingProvisioning) { @@ -122,31 +122,38 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat @Override public boolean doTerminate(Collection pendingTasks, Collection zkWorkers) { - List workerNodeIds = autoScalingStrategy.ipToIdLookup( - Lists.newArrayList( - Iterables.transform( - zkWorkers, - new Function() - { - @Override - public String apply(ZkWorker input) - { - return input.getWorker().getIp(); - } - } + Set workerNodeIds = Sets.newHashSet( + autoScalingStrategy.ipToIdLookup( + Lists.newArrayList( + Iterables.transform( + zkWorkers, + new Function() + { + @Override + public String apply(ZkWorker input) + { + return input.getWorker().getIp(); + } + } + ) ) ) ); - for (String workerNodeId : workerNodeIds) { - currentlyTerminating.remove(workerNodeId); + Set stillExisting = Sets.newHashSet(); + for (String s : currentlyTerminating) { + if (workerNodeIds.contains(s)) { + stillExisting.add(s); + } } - + currentlyTerminating.clear(); + currentlyTerminating.addAll(stillExisting); boolean nothingTerminating = currentlyTerminating.isEmpty(); if (nothingTerminating) { final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers(); if (zkWorkers.size() <= minNumWorkers) { + log.info("Only [%d <= %d] nodes in the cluster, not terminating anything.", zkWorkers.size(), minNumWorkers); return false; } @@ -167,13 +174,9 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat ) ); - if (thoseLazyWorkers.size() <= minNumWorkers) { - return false; - } - AutoScalingData terminated = autoScalingStrategy.terminate( Lists.transform( - thoseLazyWorkers.subList(minNumWorkers, thoseLazyWorkers.size() - 1), + thoseLazyWorkers.subList(minNumWorkers, thoseLazyWorkers.size()), new Function() { @Override From e51a01f85fbb91339ea3d3052eb9f0e627f2438c Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 4 Mar 2013 15:09:55 -0800 Subject: [PATCH 07/17] fix broken UT --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- .../scaling/SimpleResourceManagementStrategy.java | 9 ++++++++- .../scaling/SimpleResourceManagementStrategyTest.java | 4 ++-- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 14 files changed, 23 insertions(+), 16 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 8ad2bea5ee2..9846028de82 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index c4d83b8f817..d857b60efaa 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index b71317d2352..c3f1d4f072f 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT com.metamx druid - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 8dfe0dca4d1..371c796871c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 4b1f8ab2a1d..9c47a184929 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 60241675115..64f0c1c9709 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index d06dd7d561e..73a532064ab 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 006dea455d5..5e04d491bee 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 3d7d4a05f30..47fbaa0c4af 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java index 3f522908960..05c2f2f0b26 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -174,9 +174,16 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat ) ); + int maxPossibleNodesTerminated = zkWorkers.size() - minNumWorkers; + int numNodesToTerminate = Math.min(maxPossibleNodesTerminated, thoseLazyWorkers.size()); + if (numNodesToTerminate <= 0) { + log.info("Found no nodes to terminate."); + return false; + } + AutoScalingData terminated = autoScalingStrategy.terminate( Lists.transform( - thoseLazyWorkers.subList(minNumWorkers, thoseLazyWorkers.size()), + thoseLazyWorkers.subList(0, numNodesToTerminate), new Function() { @Override diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java index 36b6bd41632..742525d38a7 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -223,9 +223,9 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(workerSetupManager); EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) - .andReturn(Lists.newArrayList()).times(2); + .andReturn(Lists.newArrayList("ip")).times(2); EasyMock.expect(autoScalingStrategy.terminate(EasyMock.>anyObject())).andReturn( - new AutoScalingData(Lists.newArrayList("foobar"), Lists.newArrayList("foobrick")) + new AutoScalingData(Lists.newArrayList("ip"), Lists.newArrayList("ip")) ); EasyMock.replay(autoScalingStrategy); diff --git a/pom.xml b/pom.xml index daa3e806318..35247c84dab 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 37bc5d12c58..c17ae1d5d5d 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 45584f572ba..67dd4619867 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.17-SNAPSHOT + 0.3.18-SNAPSHOT From c0ab1a503d2bde6161ad3fb37b9538208d2aadcb Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 4 Mar 2013 15:11:28 -0800 Subject: [PATCH 08/17] [maven-release-plugin] prepare release druid-0.3.18 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 9846028de82..adf95dd155f 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18-SNAPSHOT + 0.3.18 diff --git a/common/pom.xml b/common/pom.xml index d857b60efaa..226a4f3a94f 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18-SNAPSHOT + 0.3.18 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index c3f1d4f072f..62b876377dc 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.18-SNAPSHOT + 0.3.18 com.metamx druid - 0.3.18-SNAPSHOT + 0.3.18 diff --git a/examples/pom.xml b/examples/pom.xml index 371c796871c..8281cbba06d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18-SNAPSHOT + 0.3.18 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 9c47a184929..13168d55641 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.18-SNAPSHOT + 0.3.18 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 64f0c1c9709..ddb2e6fdd76 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.18-SNAPSHOT + 0.3.18 diff --git a/index-common/pom.xml b/index-common/pom.xml index 73a532064ab..81212bc26bf 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18-SNAPSHOT + 0.3.18 diff --git a/indexer/pom.xml b/indexer/pom.xml index 5e04d491bee..213cfd51183 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18-SNAPSHOT + 0.3.18 diff --git a/merger/pom.xml b/merger/pom.xml index 47fbaa0c4af..b914098ebf0 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18-SNAPSHOT + 0.3.18 diff --git a/pom.xml b/pom.xml index 35247c84dab..3329583960d 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.18-SNAPSHOT + 0.3.18 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index c17ae1d5d5d..a58feff6cac 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18-SNAPSHOT + 0.3.18 diff --git a/server/pom.xml b/server/pom.xml index 67dd4619867..37750c517f3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18-SNAPSHOT + 0.3.18 From b410f5e80550f4fa4182fe5468f4fef935b9e2d0 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 4 Mar 2013 15:11:37 -0800 Subject: [PATCH 09/17] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index adf95dd155f..78aa0e60b3d 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18 + 0.3.19-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 226a4f3a94f..2d18f74e4e0 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18 + 0.3.19-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 62b876377dc..d1aab696a14 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.18 + 0.3.19-SNAPSHOT com.metamx druid - 0.3.18 + 0.3.19-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 8281cbba06d..88151a158f5 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18 + 0.3.19-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 13168d55641..54d30ddd5a8 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.18 + 0.3.19-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index ddb2e6fdd76..3b7c2d3582d 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.18 + 0.3.19-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 81212bc26bf..d5db4e3be63 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18 + 0.3.19-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 213cfd51183..b1d065b9977 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18 + 0.3.19-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index b914098ebf0..fab94d85089 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18 + 0.3.19-SNAPSHOT diff --git a/pom.xml b/pom.xml index 3329583960d..29724d5ba3e 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.18 + 0.3.19-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index a58feff6cac..24b19ccf813 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18 + 0.3.19-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 37750c517f3..edbc04a6806 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.18 + 0.3.19-SNAPSHOT From 6e95dd49b368f2c686d1519f5e5f22aa8e2afc43 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Tue, 5 Mar 2013 13:27:25 -0600 Subject: [PATCH 10/17] 1) Remove Task parameter from other methods off of TaskToolbox that required it 2) Move getTaskDir() off of TaskConfig and onto TaskToolbox 3) Rename SegmentKiller interface to DataSegmentKiller 4) Change signature of "DataSegmentKiller.kill(Collection) throws ServiceException" to just kill(DataSegment) throws SegmentLoadingException 5) Add various log messages 6) Update the version of the segment that has been converted --- .../com/metamx/druid/client/DataSegment.java | 1 + .../druid/merger/common/TaskToolbox.java | 35 ++++++---- .../merger/common/TaskToolboxFactory.java | 31 +++----- .../merger/common/config/TaskConfig.java | 23 ++++-- .../merger/common/task/AbstractTask.java | 2 +- .../druid/merger/common/task/DeleteTask.java | 6 +- .../merger/common/task/HadoopIndexTask.java | 4 +- .../task/IndexDeterminePartitionsTask.java | 2 +- .../common/task/IndexGeneratorTask.java | 21 +++--- .../druid/merger/common/task/IndexTask.java | 2 +- .../druid/merger/common/task/KillTask.java | 10 +-- .../druid/merger/common/task/MergeTask.java | 37 +++------- .../common/task/VersionConverterSubTask.java | 21 +++++- .../common/task/VersionConverterTask.java | 63 ++++++++++++----- .../merger/coordinator/LocalTaskRunner.java | 2 +- .../http/IndexerCoordinatorNode.java | 8 +-- .../http/IndexerCoordinatorResource.java | 3 +- .../druid/merger/worker/TaskMonitor.java | 2 +- .../druid/merger/worker/http/WorkerNode.java | 12 ++-- .../merger/coordinator/TaskLifecycleTest.java | 40 +++++------ .../merger/coordinator/TaskQueueTest.java | 2 +- .../druid/loading/DataSegmentKiller.java | 29 ++++++++ .../druid/loading/S3DataSegmentKiller.java | 70 +++++++++++++++++++ .../druid/loading/S3DataSegmentPusher.java | 6 +- .../metamx/druid/loading/S3SegmentKiller.java | 49 ------------- .../metamx/druid/loading/SegmentKiller.java | 14 ---- 26 files changed, 281 insertions(+), 214 deletions(-) create mode 100644 server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java create mode 100644 server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java delete mode 100644 server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java delete mode 100644 server/src/main/java/com/metamx/druid/loading/SegmentKiller.java diff --git a/client/src/main/java/com/metamx/druid/client/DataSegment.java b/client/src/main/java/com/metamx/druid/client/DataSegment.java index b915f7680cd..e8b7e55a495 100644 --- a/client/src/main/java/com/metamx/druid/client/DataSegment.java +++ b/client/src/main/java/com/metamx/druid/client/DataSegment.java @@ -261,6 +261,7 @@ public class DataSegment implements Comparable ", loadSpec=" + loadSpec + ", interval=" + interval + ", dataSource='" + dataSource + '\'' + + ", binaryVersion='" + binaryVersion + '\'' + '}'; } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index 58ad3c5cc43..32cb188273a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -25,11 +25,12 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; -import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.merger.common.actions.TaskActionClientFactory; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.task.Task; import com.metamx.emitter.service.ServiceEmitter; @@ -45,29 +46,32 @@ import java.util.Map; public class TaskToolbox { private final TaskConfig config; - private final TaskActionClient taskActionClient; + private final Task task; + private final TaskActionClientFactory taskActionClientFactory; private final ServiceEmitter emitter; private final RestS3Service s3Client; private final DataSegmentPusher segmentPusher; - private final SegmentKiller segmentKiller; + private final DataSegmentKiller dataSegmentKiller; private final ObjectMapper objectMapper; public TaskToolbox( TaskConfig config, - TaskActionClient taskActionClient, + Task task, + TaskActionClientFactory taskActionClientFactory, ServiceEmitter emitter, RestS3Service s3Client, DataSegmentPusher segmentPusher, - SegmentKiller segmentKiller, + DataSegmentKiller dataSegmentKiller, ObjectMapper objectMapper ) { this.config = config; - this.taskActionClient = taskActionClient; + this.task = task; + this.taskActionClientFactory = taskActionClientFactory; this.emitter = emitter; this.s3Client = s3Client; this.segmentPusher = segmentPusher; - this.segmentKiller = segmentKiller; + this.dataSegmentKiller = dataSegmentKiller; this.objectMapper = objectMapper; } @@ -76,9 +80,9 @@ public class TaskToolbox return config; } - public TaskActionClient getTaskActionClient() + public TaskActionClient getTaskActionClientFactory() { - return taskActionClient; + return taskActionClientFactory.create(task); } public ServiceEmitter getEmitter() @@ -91,9 +95,9 @@ public class TaskToolbox return segmentPusher; } - public SegmentKiller getSegmentKiller() + public DataSegmentKiller getDataSegmentKiller() { - return segmentKiller; + return dataSegmentKiller; } public ObjectMapper getObjectMapper() @@ -101,7 +105,7 @@ public class TaskToolbox return objectMapper; } - public Map getSegments(final Task task, List segments) + public Map getSegments(List segments) throws SegmentLoadingException { final SingleSegmentLoader loader = new SingleSegmentLoader( @@ -112,7 +116,7 @@ public class TaskToolbox @Override public File getCacheDirectory() { - return new File(config.getTaskDir(task), "fetched_segments"); + return new File(getTaskDir(), "fetched_segments"); } } ); @@ -124,4 +128,9 @@ public class TaskToolbox return retVal; } + + public File getTaskDir() { + return new File(config.getBaseTaskDir(), task.getId()); + } + } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java index 9a804bef79e..2266860ea86 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java @@ -20,55 +20,43 @@ package com.metamx.druid.merger.common; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Maps; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.MMappedQueryableIndexFactory; -import com.metamx.druid.loading.S3DataSegmentPuller; -import com.metamx.druid.loading.SegmentKiller; -import com.metamx.druid.loading.SegmentLoaderConfig; -import com.metamx.druid.loading.SegmentLoadingException; -import com.metamx.druid.loading.SingleSegmentLoader; -import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.merger.common.actions.TaskActionClientFactory; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.task.Task; import com.metamx.emitter.service.ServiceEmitter; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import java.io.File; -import java.util.List; -import java.util.Map; - /** * Stuff that may be needed by a Task in order to conduct its business. */ public class TaskToolboxFactory { private final TaskConfig config; - private final TaskActionClientFactory taskActionClient; + private final TaskActionClientFactory taskActionClientFactory; private final ServiceEmitter emitter; private final RestS3Service s3Client; private final DataSegmentPusher segmentPusher; - private final SegmentKiller segmentKiller; + private final DataSegmentKiller dataSegmentKiller; private final ObjectMapper objectMapper; public TaskToolboxFactory( TaskConfig config, - TaskActionClientFactory taskActionClient, + TaskActionClientFactory taskActionClientFactory, ServiceEmitter emitter, RestS3Service s3Client, DataSegmentPusher segmentPusher, - SegmentKiller segmentKiller, + DataSegmentKiller dataSegmentKiller, ObjectMapper objectMapper ) { this.config = config; - this.taskActionClient = taskActionClient; + this.taskActionClientFactory = taskActionClientFactory; this.emitter = emitter; this.s3Client = s3Client; this.segmentPusher = segmentPusher; - this.segmentKiller = segmentKiller; + this.dataSegmentKiller = dataSegmentKiller; this.objectMapper = objectMapper; } @@ -81,11 +69,12 @@ public class TaskToolboxFactory { return new TaskToolbox( config, - taskActionClient == null ? null : taskActionClient.create(task), + task, + taskActionClientFactory, emitter, s3Client, segmentPusher, - segmentKiller, + dataSegmentKiller, objectMapper ); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java index 5918f0627c6..2bd27667514 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java @@ -1,3 +1,22 @@ +/* + * 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.config; import com.metamx.druid.merger.common.task.Task; @@ -17,8 +36,4 @@ public abstract class TaskConfig @Config("druid.merger.hadoopWorkingPath") public abstract String getHadoopWorkingPath(); - - public File getTaskDir(final Task task) { - return new File(getBaseTaskDir(), task.getId()); - } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index c2f9a792bb0..518fb04ab37 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -102,7 +102,7 @@ public abstract class AbstractTask implements Task return ID_JOINER.join(objects); } - public SegmentListUsedAction makeImplicitListUsedAction() + public SegmentListUsedAction defaultListUsedAction() { return new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 2f10861ef5f..67754ee00d7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -77,7 +77,7 @@ public class DeleteTask extends AbstractTask public TaskStatus run(TaskToolbox toolbox) throws Exception { // Strategy: Create an empty segment covering the interval to be deleted - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); final Interval interval = this.getImplicitLockInterval().get(); final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty); @@ -91,7 +91,7 @@ public class DeleteTask extends AbstractTask .shardSpec(new NoneShardSpec()) .build(); - final File outDir = new File(toolbox.getConfig().getTaskDir(this), segment.getIdentifier()); + final File outDir = new File(toolbox.getTaskDir(), segment.getIdentifier()); final File fileToUpload = IndexMerger.merge(Lists.newArrayList(emptyAdapter), new AggregatorFactory[0], outDir); // Upload the segment @@ -104,7 +104,7 @@ public class DeleteTask extends AbstractTask segment.getVersion() ); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java index 6e284557529..f0d09d5137d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java @@ -93,7 +93,7 @@ public class HadoopIndexTask extends AbstractTask ); // We should have a lock from before we started running - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); log.info("Setting version to: %s", myLock.getVersion()); configCopy.setVersion(myLock.getVersion()); @@ -124,7 +124,7 @@ public class HadoopIndexTask extends AbstractTask List publishedSegments = job.getPublishedSegments(); // Request segment pushes - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments))); // Done return TaskStatus.success(getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 47f72b12501..3dfe99a68f1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -258,7 +258,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask } ); - toolbox.getTaskActionClient().submit(new SpawnTasksAction(nextTasks)); + toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(nextTasks)); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index 2a7b4683b8d..1790ddc6aa5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -100,24 +100,21 @@ public class IndexGeneratorTask extends AbstractTask public TaskStatus run(final TaskToolbox toolbox) throws Exception { // We should have a lock from before we started running - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); // We know this exists final Interval interval = getImplicitLockInterval().get(); // Set up temporary directory for indexing final File tmpDir = new File( + toolbox.getTaskDir(), String.format( - "%s/%s", - toolbox.getConfig().getTaskDir(this).toString(), - String.format( - "%s_%s_%s_%s_%s", - this.getDataSource(), - interval.getStart(), - interval.getEnd(), - myLock.getVersion(), - schema.getShardSpec().getPartitionNum() - ) + "%s_%s_%s_%s_%s", + this.getDataSource(), + interval.getStart(), + interval.getEnd(), + myLock.getVersion(), + schema.getShardSpec().getPartitionNum() ) ); @@ -193,7 +190,7 @@ public class IndexGeneratorTask extends AbstractTask ); // Request segment pushes - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(pushedSegments))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.copyOf(pushedSegments))); // Done return TaskStatus.success(getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 6ef3481df03..4f1138e6e0c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -144,7 +144,7 @@ public class IndexTask extends AbstractTask @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { - toolbox.getTaskActionClient().submit(new SpawnTasksAction(toSubtasks())); + toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(toSubtasks())); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index d8a2d4cba0e..b001e108bd2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -72,7 +72,7 @@ public class KillTask extends AbstractTask public TaskStatus run(TaskToolbox toolbox) throws Exception { // Confirm we have a lock (will throw if there isn't exactly one element) - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); if(!myLock.getDataSource().equals(getDataSource())) { throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); @@ -84,7 +84,7 @@ public class KillTask extends AbstractTask // List unused segments final List unusedSegments = toolbox - .getTaskActionClient() + .getTaskActionClientFactory() .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval())); // Verify none of these segments have versions > lock version @@ -102,10 +102,12 @@ public class KillTask extends AbstractTask } // Kill segments - toolbox.getSegmentKiller().kill(unusedSegments); + for (DataSegment segment : unusedSegments) { + toolbox.getDataSegmentKiller().kill(segment); + } // Remove metadata for these segments - toolbox.getTaskActionClient().submit(new SegmentNukeAction(ImmutableSet.copyOf(unusedSegments))); + toolbox.getTaskActionClientFactory().submit(new SegmentNukeAction(ImmutableSet.copyOf(unusedSegments))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 757a91c2598..561cb940639 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -43,8 +43,8 @@ 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.SegmentListUsedAction; import com.metamx.druid.shard.NoneShardSpec; +import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; @@ -67,7 +67,7 @@ public abstract class MergeTask extends AbstractTask { private final List segments; - private static final Logger log = new Logger(MergeTask.class); + private static final EmittingLogger log = new EmittingLogger(MergeTask.class); protected MergeTask(final String dataSource, final List segments) { @@ -119,11 +119,11 @@ public abstract class MergeTask extends AbstractTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); final ServiceEmitter emitter = toolbox.getEmitter(); final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments); - final File taskDir = toolbox.getConfig().getTaskDir(this); + final File taskDir = toolbox.getTaskDir(); try { @@ -147,7 +147,7 @@ public abstract class MergeTask extends AbstractTask // download segments to merge - final Map gettedSegments = toolbox.getSegments(this, segments); + final Map gettedSegments = toolbox.getSegments(segments); // merge files together final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); @@ -170,27 +170,14 @@ public abstract class MergeTask extends AbstractTask emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); return TaskStatus.success(getId()); } catch (Exception e) { - log.error( - e, - String.format( - "Exception merging %s[%s] segments", - mergedSegment.getDataSource(), - mergedSegment.getInterval() - ) - ); - emitter.emit( - new AlertEvent.Builder().build( - "Exception merging", - ImmutableMap.builder() - .put("exception", e.toString()) - .build() - ) - ); + log.makeAlert(e, "Exception merging[%s]", mergedSegment.getDataSource()) + .addData("interval", mergedSegment.getInterval()) + .emit(); return TaskStatus.failure(getId()); } @@ -213,11 +200,7 @@ public abstract class MergeTask extends AbstractTask }; final Set current = ImmutableSet.copyOf( - Iterables.transform( - toolbox.getTaskActionClient() - .submit(new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get())), - toIdentifier - ) + Iterables.transform(toolbox.getTaskActionClientFactory().submit(defaultListUsedAction()), toIdentifier) ); final Set requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier)); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java index db08e788e4d..2099d903d33 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java @@ -21,11 +21,13 @@ package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; +import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import org.joda.time.DateTime; import java.io.File; import java.util.Arrays; @@ -35,6 +37,8 @@ import java.util.Map; */ public class VersionConverterSubTask extends AbstractTask { + private static final Logger log = new Logger(VersionConverterSubTask.class); + private final DataSegment segment; protected VersionConverterSubTask( @@ -50,6 +54,7 @@ public class VersionConverterSubTask extends AbstractTask segment.getInterval().getEnd(), segment.getShardSpec().getPartitionNum() ), + groupId, segment.getDataSource(), segment.getInterval() ); @@ -65,13 +70,23 @@ public class VersionConverterSubTask extends AbstractTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final Map localSegments = toolbox.getSegments(this, Arrays.asList(segment)); + log.info("Converting segment[%s]", segment); + final Map localSegments = toolbox.getSegments(Arrays.asList(segment)); final File location = localSegments.get(segment); final File outLocation = new File(location, "v9_out"); if (IndexIO.convertSegment(location, outLocation)) { - final DataSegment updatedSegment = toolbox.getSegmentPusher().push(outLocation, segment); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); + final int outVersion = IndexIO.getVersionFromDir(outLocation); + + // Appending to the version makes a new version that inherits most comparability parameters of the original + // version, but is "newer" than said original version. + DataSegment updatedSegment = segment.withVersion(String.format("%s_v%s", segment.getVersion(), outVersion)); + updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment); + + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); + } + else { + log.info("Conversion failed."); } return success(); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java index 4106e210547..3859190a1bb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java @@ -1,10 +1,32 @@ +/* + * 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.task; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.metamx.common.ISE; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.actions.SpawnTasksAction; @@ -12,7 +34,6 @@ import com.metamx.druid.merger.common.actions.TaskActionClient; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.util.List; /** @@ -20,6 +41,9 @@ import java.util.List; public class VersionConverterTask extends AbstractTask { private static final String TYPE = "version_converter"; + private static final Integer CURR_VERSION_INTEGER = new Integer(IndexIO.CURRENT_VERSION_ID); + + private static final Logger log = new Logger(VersionConverterTask.class); public VersionConverterTask( @JsonProperty("dataSource") String dataSource, @@ -48,25 +72,30 @@ public class VersionConverterTask extends AbstractTask @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { - final TaskActionClient taskClient = toolbox.getTaskActionClient(); + final TaskActionClient taskClient = toolbox.getTaskActionClientFactory(); - List segments = taskClient.submit(makeImplicitListUsedAction()); + List segments = taskClient.submit(defaultListUsedAction()); - taskClient.submit( - new SpawnTasksAction( - Lists.transform( - segments, - new Function() - { - @Override - public Task apply(@Nullable DataSegment input) - { - return new VersionConverterSubTask(getGroupId(), input); - } + final FunctionalIterable tasks = FunctionalIterable + .create(segments) + .keep( + new Function() + { + @Override + public Task apply(DataSegment segment) + { + final Integer segmentVersion = segment.getBinaryVersion(); + if (!CURR_VERSION_INTEGER.equals(segmentVersion)) { + return new VersionConverterSubTask(getGroupId(), segment); } - ) - ) - ); + + log.info("Skipping[%s], already version[%s]", segment.getIdentifier(), segmentVersion); + return null; + } + } + ); + + taskClient.submit(new SpawnTasksAction(Lists.newArrayList(tasks))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index a4f58a361f8..5dbe8273d6c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -92,7 +92,7 @@ public class LocalTaskRunner implements TaskRunner } try { - final File taskDir = toolbox.getConfig().getTaskDir(task); + final File taskDir = toolbox.getTaskDir(); if (taskDir.exists()) { log.info("Removing task directory: %s", taskDir); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 901348908df..85b28362799 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -51,8 +51,8 @@ import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.S3SegmentKiller; -import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.S3DataSegmentKiller; +import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.TaskActionToolbox; @@ -479,7 +479,7 @@ public class IndexerCoordinatorNode extends RegisteringNode public void initializeTaskToolbox() { if (taskToolboxFactory == null) { - final SegmentKiller segmentKiller = new S3SegmentKiller(s3Service); + final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service); taskToolboxFactory = new TaskToolboxFactory( taskConfig, new LocalTaskActionClientFactory( @@ -489,7 +489,7 @@ public class IndexerCoordinatorNode extends RegisteringNode emitter, s3Service, segmentPusher, - segmentKiller, + dataSegmentKiller, jsonMapper ); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index 71db9113517..1496552bb6e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -26,7 +26,6 @@ import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.actions.TaskAction; import com.metamx.druid.merger.common.actions.TaskActionHolder; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; @@ -178,7 +177,7 @@ public class IndexerCoordinatorResource public Response doAction(final TaskActionHolder holder) { final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask()) - .getTaskActionClient() + .getTaskActionClientFactory() .submit(holder.getAction()); final Map retMap = Maps.newHashMap(); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index ec3a8d992e0..867b8dd9cde 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -101,7 +101,7 @@ public class TaskMonitor public void run() { final long startTime = System.currentTimeMillis(); - final File taskDir = toolbox.getConfig().getTaskDir(task); + final File taskDir = toolbox.getTaskDir(); log.info("Running task [%s]", task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index bad04040e73..caef5bd2935 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -39,13 +39,9 @@ import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusherConfig; -import com.metamx.druid.loading.S3SegmentKiller; -import com.metamx.druid.loading.SegmentKiller; -import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.loading.S3DataSegmentKiller; +import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.merger.common.TaskToolboxFactory; -import com.metamx.druid.merger.common.actions.RemoteTaskActionClient; import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; @@ -355,14 +351,14 @@ public class WorkerNode extends RegisteringNode public void initializeTaskToolbox() throws S3ServiceException { if (taskToolboxFactory == null) { - final SegmentKiller segmentKiller = new S3SegmentKiller(s3Service); + final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service); taskToolboxFactory = new TaskToolboxFactory( taskConfig, new RemoteTaskActionClientFactory(httpClient, coordinatorServiceProvider, jsonMapper), emitter, s3Service, segmentPusher, - segmentKiller, + dataSegmentKiller, jsonMapper ); } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java index 3dc889b96f0..4fac2c5eded 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java @@ -40,7 +40,8 @@ import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.DataSegmentKiller; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; @@ -75,7 +76,6 @@ import org.junit.Test; import java.io.File; import java.io.IOException; -import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Set; @@ -146,10 +146,10 @@ public class TaskLifecycleTest return segment; } }, - new SegmentKiller() + new DataSegmentKiller() { @Override - public void kill(Collection segments) throws ServiceException + public void kill(DataSegment segments) throws SegmentLoadingException { } @@ -283,8 +283,8 @@ public class TaskLifecycleTest // Sort of similar to what realtime tasks do: // Acquire lock for first interval - final Optional lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval1)); - final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction()); + final Optional lock1 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval1)); + final List locks1 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertTrue("lock1 present", lock1.isPresent()); @@ -292,8 +292,8 @@ public class TaskLifecycleTest Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1); // Acquire lock for second interval - final Optional lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval2)); - final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction()); + final Optional lock2 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval2)); + final List locks2 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertTrue("lock2 present", lock2.isPresent()); @@ -301,7 +301,7 @@ public class TaskLifecycleTest Assert.assertEquals("locks2", ImmutableList.of(lock1.get(), lock2.get()), locks2); // Push first segment - toolbox.getTaskActionClient() + toolbox.getTaskActionClientFactory() .submit( new SegmentInsertAction( ImmutableSet.of( @@ -315,14 +315,14 @@ public class TaskLifecycleTest ); // Release first lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(interval1)); - final List locks3 = toolbox.getTaskActionClient().submit(new LockListAction()); + toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval1)); + final List locks3 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3); // Push second segment - toolbox.getTaskActionClient() + toolbox.getTaskActionClientFactory() .submit( new SegmentInsertAction( ImmutableSet.of( @@ -336,8 +336,8 @@ public class TaskLifecycleTest ); // Release second lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(interval2)); - final List locks4 = toolbox.getTaskActionClient().submit(new LockListAction()); + toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval2)); + final List locks4 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertEquals("locks4", ImmutableList.of(), locks4); @@ -370,7 +370,7 @@ public class TaskLifecycleTest public TaskStatus run(TaskToolbox toolbox) throws Exception { final TaskLock myLock = Iterables.getOnlyElement( - toolbox.getTaskActionClient() + toolbox.getTaskActionClientFactory() .submit(new LockListAction()) ); @@ -380,7 +380,7 @@ public class TaskLifecycleTest .version(myLock.getVersion()) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; @@ -406,7 +406,7 @@ public class TaskLifecycleTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); final DataSegment segment = DataSegment.builder() .dataSource("ds") @@ -414,7 +414,7 @@ public class TaskLifecycleTest .version(myLock.getVersion()) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; @@ -440,7 +440,7 @@ public class TaskLifecycleTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); final DataSegment segment = DataSegment.builder() .dataSource("ds") @@ -448,7 +448,7 @@ public class TaskLifecycleTest .version(myLock.getVersion() + "1!!!1!!") .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 939dc9b6b21..dfb0d959d1a 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -375,7 +375,7 @@ public class TaskQueueTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - toolbox.getTaskActionClient().submit(new SpawnTasksAction(nextTasks)); + toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(nextTasks)); return TaskStatus.success(id); } }; diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java new file mode 100644 index 00000000000..85483eaa505 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java @@ -0,0 +1,29 @@ +/* + * 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.loading; + +import com.metamx.druid.client.DataSegment; + +/** + */ +public interface DataSegmentKiller +{ + public void kill(DataSegment segments) throws SegmentLoadingException; +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java new file mode 100644 index 00000000000..47c39c1857e --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java @@ -0,0 +1,70 @@ +/* + * 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.loading; + +import com.google.inject.Inject; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import org.jets3t.service.ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; + +import java.util.Map; + +/** + */ +public class S3DataSegmentKiller implements DataSegmentKiller +{ + private static final Logger log = new Logger(S3DataSegmentKiller.class); + + private final RestS3Service s3Client; + + @Inject + public S3DataSegmentKiller( + RestS3Service s3Client + ) + { + this.s3Client = s3Client; + } + + + @Override + public void kill(DataSegment segment) throws SegmentLoadingException + { + try { + Map loadSpec = segment.getLoadSpec(); + String s3Bucket = MapUtils.getString(loadSpec, "bucket"); + String s3Path = MapUtils.getString(loadSpec, "key"); + String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; + + if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { + log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); + s3Client.deleteObject(s3Bucket, s3Path); + } + if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { + log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); + s3Client.deleteObject(s3Bucket, s3DescriptorPath); + } + } + catch (ServiceException e) { + throw new SegmentLoadingException(e, "Couldn't kill segment[%s]", segment.getIdentifier()); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index dd5ce951695..9dacbe8b546 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -87,11 +87,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher segment = segment.withSize(indexSize) .withLoadSpec( - ImmutableMap.of( - "type", "s3_zip", - "bucket", outputBucket, - "key", toPush.getKey() - ) + ImmutableMap.of("type", "s3_zip", "bucket", outputBucket, "key", toPush.getKey()) ) .withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir)); diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java deleted file mode 100644 index 46f6acfc629..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java +++ /dev/null @@ -1,49 +0,0 @@ -package com.metamx.druid.loading; - -import com.google.inject.Inject; -import com.metamx.common.MapUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import org.jets3t.service.ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; - -import java.util.Collection; -import java.util.Map; - -/** - */ -public class S3SegmentKiller implements SegmentKiller -{ - private static final Logger log = new Logger(S3SegmentKiller.class); - - private final RestS3Service s3Client; - - @Inject - public S3SegmentKiller( - RestS3Service s3Client - ) - { - this.s3Client = s3Client; - } - - - @Override - public void kill(Collection segments) throws ServiceException - { - for (final DataSegment segment : segments) { - Map loadSpec = segment.getLoadSpec(); - String s3Bucket = MapUtils.getString(loadSpec, "bucket"); - String s3Path = MapUtils.getString(loadSpec, "key"); - String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; - - if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { - log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); - s3Client.deleteObject(s3Bucket, s3Path); - } - if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { - log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); - s3Client.deleteObject(s3Bucket, s3DescriptorPath); - } - } - } -} diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java deleted file mode 100644 index 8f8746d5324..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.metamx.druid.loading; - -import com.metamx.druid.client.DataSegment; -import org.jets3t.service.ServiceException; - -import java.util.Collection; -import java.util.List; - -/** - */ -public interface SegmentKiller -{ - public void kill(Collection segments) throws ServiceException; -} From 9ffccb68037cdb058c2bcd1748090b4bacfd4b6e Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 7 Mar 2013 12:59:59 -0600 Subject: [PATCH 11/17] 1) Adjust the Config stuff that WorkerSetupManager was using to be reusable 2) Use new ConfigManager to pull out a whitelist for the Master --- .../indexing}/ClientAppendQuery.java | 12 +- .../indexing/ClientConversionQuery.java | 57 +++ .../indexing/ClientKillQuery.java} | 27 +- .../indexing/ClientMergeQuery.java} | 16 +- .../indexing/IndexingServiceClient.java | 115 ++++++ .../metamx/druid/merge/ClientKillQuery.java | 41 --- .../metamx/druid/merge/ClientMergeQuery.java | 41 --- .../metamx/druid/config/ConfigManager.java | 259 ++++++++++++++ .../druid/config/ConfigManagerConfig.java | 18 + .../com/metamx/druid/config/ConfigSerde.java | 9 + .../druid/config/JacksonConfigManager.java | 134 +++++++ .../java/com/metamx/druid/db/DbConnector.java | 9 +- .../druid/merger/common/task/AppendTask.java | 2 +- .../merger/common/task/DefaultMergeTask.java | 89 ----- .../druid/merger/common/task/MergeTask.java | 326 +++--------------- .../merger/common/task/MergeTaskBase.java | 315 +++++++++++++++++ .../metamx/druid/merger/common/task/Task.java | 6 +- .../common/task/VersionConverterSubTask.java | 94 ----- .../common/task/VersionConverterTask.java | 88 ++++- .../config/WorkerSetupManagerConfig.java | 39 --- .../http/IndexerCoordinatorNode.java | 20 +- .../http/IndexerCoordinatorResource.java | 19 +- .../coordinator/setup/WorkerSetupManager.java | 184 +--------- .../com/metamx/druid/merger/TestTask.java | 4 +- ...geTaskTest.java => MergeTaskBaseTest.java} | 10 +- pom.xml | 2 +- .../com/metamx/druid/http/InfoResource.java | 9 +- .../com/metamx/druid/http/MasterMain.java | 24 +- .../com/metamx/druid/http/MasterResource.java | 1 - .../druid/http/MasterServletModule.java | 7 +- .../com/metamx/druid/master/DruidMaster.java | 110 +++--- .../master/DruidMasterSegmentMerger.java | 57 ++- .../metamx/druid/master/HttpMergerClient.java | 78 ----- ...MergerClient.java => MergerWhitelist.java} | 29 +- .../master/DruidMasterSegmentMergerTest.java | 15 +- .../metamx/druid/master/DruidMasterTest.java | 2 - 36 files changed, 1252 insertions(+), 1016 deletions(-) rename client/src/main/java/com/metamx/druid/{merge => client/indexing}/ClientAppendQuery.java (92%) create mode 100644 client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java rename client/src/main/java/com/metamx/druid/{merge/ClientDeleteQuery.java => client/indexing/ClientKillQuery.java} (83%) rename client/src/main/java/com/metamx/druid/{merge/ClientDefaultMergeQuery.java => client/indexing/ClientMergeQuery.java} (90%) create mode 100644 client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java delete mode 100644 client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java delete mode 100644 client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java create mode 100644 common/src/main/java/com/metamx/druid/config/ConfigManager.java create mode 100644 common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java create mode 100644 common/src/main/java/com/metamx/druid/config/ConfigSerde.java create mode 100644 common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java rename merger/src/test/java/com/metamx/druid/merger/common/task/{MergeTaskTest.java => MergeTaskBaseTest.java} (89%) delete mode 100644 server/src/main/java/com/metamx/druid/master/HttpMergerClient.java rename server/src/main/java/com/metamx/druid/master/{MergerClient.java => MergerWhitelist.java} (57%) diff --git a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java similarity index 92% rename from client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java rename to client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java index 5a8e3bdb50f..5f744918a0a 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merge; +package com.metamx.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -29,7 +29,7 @@ import java.util.List; /** */ -public class ClientAppendQuery implements ClientMergeQuery +public class ClientAppendQuery { private final String dataSource; private final List segments; @@ -45,14 +45,18 @@ public class ClientAppendQuery implements ClientMergeQuery } @JsonProperty - @Override + public String getType() + { + return "append"; + } + + @JsonProperty public String getDataSource() { return dataSource; } @JsonProperty - @Override public List getSegments() { return segments; diff --git a/client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java new file mode 100644 index 00000000000..c0b96bb80b4 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java @@ -0,0 +1,57 @@ +package com.metamx.druid.client.indexing; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.druid.client.DataSegment; +import org.joda.time.Interval; + +/** + */ +public class ClientConversionQuery +{ + private final String dataSource; + private final Interval interval; + private final DataSegment segment; + + public ClientConversionQuery( + DataSegment segment + ) + { + this.dataSource = segment.getDataSource(); + this.interval = segment.getInterval(); + this.segment = segment; + } + + public ClientConversionQuery( + String dataSource, + Interval interval + ) + { + this.dataSource = dataSource; + this.interval = interval; + this.segment = null; + } + + @JsonProperty + public String getType() + { + return "version_converter"; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public DataSegment getSegment() + { + return segment; + } +} diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java similarity index 83% rename from client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java rename to client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java index 3acf20a0a43..3ae8dffb225 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java @@ -17,21 +17,21 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merge; - - +package com.metamx.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; -public class ClientDeleteQuery +/** + */ +public class ClientKillQuery { private final String dataSource; private final Interval interval; @JsonCreator - public ClientDeleteQuery( + public ClientKillQuery( @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval ) @@ -40,22 +40,21 @@ public class ClientDeleteQuery this.interval = interval; } + @JsonProperty + public String getType() + { + return "kill"; + } + + @JsonProperty public String getDataSource() { return dataSource; } + @JsonProperty public Interval getInterval() { return interval; } - - @Override - public String toString() - { - return "ClientDeleteQuery{" + - "dataSource='" + dataSource + '\'' + - ", interval=" + interval + - '}'; - } } diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java similarity index 90% rename from client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java rename to client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java index 4286cd211cb..e000826ff9a 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merge; +package com.metamx.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -30,14 +30,14 @@ import java.util.List; /** */ -public class ClientDefaultMergeQuery implements ClientMergeQuery +public class ClientMergeQuery { private final String dataSource; private final List segments; private final List aggregators; @JsonCreator - public ClientDefaultMergeQuery( + public ClientMergeQuery( @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments, @JsonProperty("aggregations") List aggregators @@ -50,14 +50,18 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery } @JsonProperty - @Override + public String getType() + { + return "append"; + } + + @JsonProperty public String getDataSource() { return dataSource; } @JsonProperty - @Override public List getSegments() { return segments; @@ -72,7 +76,7 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery @Override public String toString() { - return "ClientDefaultMergeQuery{" + + return "ClientMergeQuery{" + "dataSource='" + dataSource + '\'' + ", segments=" + segments + ", aggregators=" + aggregators + diff --git a/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java b/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java new file mode 100644 index 00000000000..b659148d338 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java @@ -0,0 +1,115 @@ +/* + * 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.client.indexing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.metamx.common.IAE; +import com.metamx.druid.client.DataSegment; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.InputStreamResponseHandler; +import com.netflix.curator.x.discovery.ServiceInstance; +import com.netflix.curator.x.discovery.ServiceProvider; +import org.joda.time.Interval; + + +import java.io.InputStream; +import java.net.URL; +import java.util.Iterator; +import java.util.List; + +public class IndexingServiceClient +{ + private static final InputStreamResponseHandler RESPONSE_HANDLER = new InputStreamResponseHandler(); + + private final HttpClient client; + private final ObjectMapper jsonMapper; + private final ServiceProvider serviceProvider; + + public IndexingServiceClient( + HttpClient client, + ObjectMapper jsonMapper, + ServiceProvider serviceProvider + ) + { + this.client = client; + this.jsonMapper = jsonMapper; + this.serviceProvider = serviceProvider; + } + + public void mergeSegments(List segments) + { + final Iterator segmentsIter = segments.iterator(); + if (!segmentsIter.hasNext()) { + return; + } + + final String dataSource = segmentsIter.next().getDataSource(); + while (segmentsIter.hasNext()) { + DataSegment next = segmentsIter.next(); + if (!dataSource.equals(next.getDataSource())) { + throw new IAE("Cannot merge segments of different dataSources[%s] and [%s]", dataSource, next.getDataSource()); + } + } + + runQuery("merge", new ClientAppendQuery(dataSource, segments)); + } + + public void killSegments(String dataSource, Interval interval) + { + runQuery("index", new ClientKillQuery(dataSource, interval)); + } + + public void upgradeSegment(DataSegment dataSegment) + { + runQuery("task", new ClientConversionQuery(dataSegment)); + } + + public void upgradeSegments(String dataSource, Interval interval) + { + runQuery("task", new ClientConversionQuery(dataSource, interval)); + } + + private InputStream runQuery(String endpoint, Object queryObject) + { + try { + return client.post(new URL(String.format("%s/%s", baseUrl(), endpoint))) + .setContent("application/json", jsonMapper.writeValueAsBytes(queryObject)) + .go(RESPONSE_HANDLER) + .get(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + + private String baseUrl() + { + try { + final ServiceInstance instance = serviceProvider.getInstance(); + return String.format("http://%s:%s/mmx/merger/v1", instance.getAddress(), instance.getPort()); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java deleted file mode 100644 index 02e0a7bd141..00000000000 --- a/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java +++ /dev/null @@ -1,41 +0,0 @@ -package com.metamx.druid.merge; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Interval; - -/** - */ -public class ClientKillQuery -{ - private final String dataSource; - private final Interval interval; - - @JsonCreator - public ClientKillQuery( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval - ) - { - this.dataSource = dataSource; - this.interval = interval; - } - - @JsonProperty - public String getType() - { - return "kill"; - } - - @JsonProperty - public String getDataSource() - { - return dataSource; - } - - @JsonProperty - public Interval getInterval() - { - return interval; - } -} diff --git a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java deleted file mode 100644 index 3956991ed6b..00000000000 --- a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java +++ /dev/null @@ -1,41 +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.merge; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.client.DataSegment; - - - -import java.util.List; - -/** - */ -@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class) -@JsonSubTypes(value={ - @JsonSubTypes.Type(name="append", value=ClientAppendQuery.class) -}) -public interface ClientMergeQuery -{ - public String getDataSource(); - - public List getSegments(); -} diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManager.java b/common/src/main/java/com/metamx/druid/config/ConfigManager.java new file mode 100644 index 00000000000..3073c13e9fb --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/ConfigManager.java @@ -0,0 +1,259 @@ +package com.metamx.druid.config; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.common.collect.Maps; +import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import org.joda.time.Duration; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.IDBI; +import org.skife.jdbi.v2.StatementContext; +import org.skife.jdbi.v2.tweak.HandleCallback; +import org.skife.jdbi.v2.tweak.ResultSetMapper; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; + +/** + */ +public class ConfigManager +{ + private static final Logger log = new Logger(ConfigManager.class); + + private final Object lock = new Object(); + private boolean started = false; + + private final IDBI dbi; + private final ConfigManagerConfig config; + + private final ScheduledExecutorService exec; + private final ConcurrentMap watchedConfigs; + private final String selectStatement; + + public ConfigManager(IDBI dbi, ConfigManagerConfig config) + { + this.dbi = dbi; + this.config = config; + + this.exec = ScheduledExecutors.fixed(1, "config-manager-%s"); + this.watchedConfigs = Maps.newConcurrentMap(); + this.selectStatement = String.format("SELECT payload FROM %s WHERE name = :name", config.getConfigTable()); + } + + @LifecycleStart + public void start() + { + synchronized (lock) { + if (started) { + return; + } + + ScheduledExecutors.scheduleWithFixedDelay( + exec, + new Duration(0), + config.getPollDuration(), + new Runnable() + { + @Override + public void run() + { + poll(); + } + } + ); + + started = true; + } + } + + @LifecycleStop + public void stop() + { + synchronized (lock) { + if (!started) { + return; + } + + started = false; + } + } + + private void poll() + { + for (Map.Entry entry : watchedConfigs.entrySet()) { + try { + if (entry.getValue().swapIfNew(lookup(entry.getKey()))) { + log.info("New value for key[%s] seen.", entry.getKey()); + } + } + catch (Exception e) { + log.warn(e, "Exception when checking property[%s]", entry.getKey()); + } + } + } + + @SuppressWarnings("unchecked") + public AtomicReference watchConfig(final String key, final ConfigSerde serde) + { + ConfigHolder holder = watchedConfigs.get(key); + if (holder == null) { + try { + log.info("Creating watch for key[%s]", key); + + holder = exec.submit( + new Callable>() + { + @Override + @SuppressWarnings("unchecked") + public ConfigHolder call() throws Exception + { + if (!started) { + watchedConfigs.put(key, new ConfigHolder(null, serde)); + } + else { + try { + // Multiple of these callables can be submitted at the same time, but the callables themselves + // are executed serially, so double check that it hasn't already been populated. + if (!watchedConfigs.containsKey(key)) { + byte[] value = lookup(key); + ConfigHolder holder = new ConfigHolder(value, serde); + watchedConfigs.put(key, holder); + } + } + catch (Exception e) { + log.warn(e, "Failed loading config for key[%s]", key); + watchedConfigs.put(key, new ConfigHolder(null, serde)); + } + } + + return watchedConfigs.get(key); + } + } + ).get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e); + } + } + + return holder.getReference(); + } + + public byte[] lookup(final String key) + { + return dbi.withHandle( + new HandleCallback() + { + @Override + public byte[] withHandle(Handle handle) throws Exception + { + return handle.createQuery(selectStatement) + .bind("name", key) + .map( + new ResultSetMapper() + { + @Override + public byte[] map(int index, ResultSet r, StatementContext ctx) throws SQLException + { + return r.getBytes("payload"); + } + } + ) + .first(); + } + } + ); + } + + public boolean set(final String key, final ConfigSerde serde, final T obj) + { + if (obj == null) { + return false; + } + + final byte[] newBytes = serde.serialize(obj); + + try { + return exec.submit( + new Callable() + { + @Override + public Boolean call() throws Exception + { + dbi.withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception + { + handle.createStatement("INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload") + .bind("name", key) + .bind("payload", newBytes) + .execute(); + return null; + } + } + ); + + final ConfigHolder configHolder = watchedConfigs.get(key); + if (configHolder != null) { + configHolder.swapIfNew(newBytes); + } + + return true; + } + } + ).get(); + } + catch (Exception e) { + log.warn(e, "Failed to set[%s]", key); + return false; + } + } + + private static class ConfigHolder + { + private final AtomicReference rawBytes; + private final ConfigSerde serde; + private final AtomicReference reference; + + ConfigHolder( + byte[] rawBytes, + ConfigSerde serde + ) + { + this.rawBytes = new AtomicReference(rawBytes); + this.serde = serde; + this.reference = new AtomicReference(serde.deserialize(rawBytes)); + } + + public AtomicReference getReference() + { + return reference; + } + + public boolean swapIfNew(byte[] newBytes) + { + if (!Arrays.equals(newBytes, rawBytes.get())) { + reference.set(serde.deserialize(newBytes)); + rawBytes.set(newBytes); + return true; + } + return false; + } + } +} diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java b/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java new file mode 100644 index 00000000000..24706a83fb7 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java @@ -0,0 +1,18 @@ +package com.metamx.druid.config; + +import org.joda.time.Duration; +import org.skife.config.Config; +import org.skife.config.Default; + +/** + */ +public abstract class ConfigManagerConfig +{ + @Config("druid.indexer.configTable") + public abstract String getConfigTable(); + + @Config("druid.indexer.poll.duration") + @Default("PT1M") + public abstract Duration getPollDuration(); + +} diff --git a/common/src/main/java/com/metamx/druid/config/ConfigSerde.java b/common/src/main/java/com/metamx/druid/config/ConfigSerde.java new file mode 100644 index 00000000000..95f0a1ee7d3 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/ConfigSerde.java @@ -0,0 +1,9 @@ +package com.metamx.druid.config; + +/** +*/ +public interface ConfigSerde +{ + public byte[] serialize(T obj); + public T deserialize(byte[] bytes); +} diff --git a/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java b/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java new file mode 100644 index 00000000000..8e322f3ee80 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java @@ -0,0 +1,134 @@ +/* + * 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.config; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicReference; + +/** + */ +public class JacksonConfigManager +{ + private final ConfigManager configManager; + private final ObjectMapper jsonMapper; + + public JacksonConfigManager( + ConfigManager configManager, + ObjectMapper jsonMapper + ) + { + this.configManager = configManager; + this.jsonMapper = jsonMapper; + } + + public AtomicReference watch(String key, Class clazz) + { + return watch(key, clazz, null); + } + + public AtomicReference watch(String key, Class clazz, T defaultVal) + { + return configManager.watchConfig(key, create(clazz, defaultVal)); + } + + public AtomicReference watch(String key, TypeReference clazz) + { + return watch(key, clazz, null); + } + + public AtomicReference watch(String key, TypeReference clazz, T defaultVal) + { + return configManager.watchConfig(key, create(clazz, defaultVal)); + } + + public boolean set(String key, T val) + { + return configManager.set(key, create(val.getClass(), null), val); + } + + private ConfigSerde create(final Class clazz, final T defaultVal) + { + return new ConfigSerde() + { + @Override + public byte[] serialize(T obj) + { + try { + return jsonMapper.writeValueAsBytes(obj); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + } + + @Override + public T deserialize(byte[] bytes) + { + if (bytes == null) { + return defaultVal; + } + + try { + return jsonMapper.readValue(bytes, clazz); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + }; + } + + private ConfigSerde create(final TypeReference clazz, final T defaultVal) + { + return new ConfigSerde() + { + @Override + public byte[] serialize(T obj) + { + try { + return jsonMapper.writeValueAsBytes(obj); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + } + + @Override + public T deserialize(byte[] bytes) + { + if (bytes == null) { + return defaultVal; + } + + try { + return jsonMapper.readValue(bytes, clazz); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + }; + } +} diff --git a/common/src/main/java/com/metamx/druid/db/DbConnector.java b/common/src/main/java/com/metamx/druid/db/DbConnector.java index 73013ce6aa2..b8ab7a4747e 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -65,7 +65,7 @@ public class DbConnector dbi, configTableName, String.format( - "CREATE table %s (name VARCHAR(255) NOT NULL, payload LONGTEXT NOT NULL, PRIMARY KEY(name))", + "CREATE table %s (name VARCHAR(255) NOT NULL, payload BLOB NOT NULL, PRIMARY KEY(name))", configTableName ) ); @@ -84,12 +84,7 @@ public class DbConnector @Override public Void withHandle(Handle handle) throws Exception { - List> table = handle.select( - String.format( - "SHOW tables LIKE '%s'", - tableName - ) - ); + List> table = handle.select(String.format("SHOW tables LIKE '%s'", tableName)); if (table.isEmpty()) { log.info("Creating table[%s]", tableName); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java index ce88b2481e6..5148b7389bf 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java @@ -46,7 +46,7 @@ import java.util.Map; /** */ -public class AppendTask extends MergeTask +public class AppendTask extends MergeTaskBase { @JsonCreator public AppendTask( diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java deleted file mode 100644 index 4fe6933c2fe..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java +++ /dev/null @@ -1,89 +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.task; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -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.metamx.druid.aggregation.AggregatorFactory; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.QueryableIndex; -import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.index.v1.IndexMerger; - - - -import javax.annotation.Nullable; -import java.io.File; -import java.util.List; -import java.util.Map; - -/** - */ -public class DefaultMergeTask extends MergeTask -{ - private final List aggregators; - - @JsonCreator - public DefaultMergeTask( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("segments") List segments, - @JsonProperty("aggregations") List aggregators - ) - { - super(dataSource, segments); - this.aggregators = aggregators; - } - - @Override - public File merge(final Map segments, final File outDir) - throws Exception - { - return IndexMerger.mergeQueryableIndex( - Lists.transform( - ImmutableList.copyOf(segments.values()), - new Function() - { - @Override - public QueryableIndex apply(@Nullable File input) - { - try { - return IndexIO.loadIndex(input); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ), - aggregators.toArray(new AggregatorFactory[aggregators.size()]), - outDir - ); - } - - @Override - public String getType() - { - return "merge"; - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 561cb940639..4e6102f666b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -19,301 +19,71 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.common.base.Charsets; import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Objects; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; -import com.google.common.collect.Sets; -import com.google.common.hash.Hashing; -import com.metamx.common.ISE; -import com.metamx.common.logger.Logger; +import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LockListAction; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; -import com.metamx.druid.shard.NoneShardSpec; -import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.service.AlertEvent; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; -import org.joda.time.DateTime; -import org.joda.time.Interval; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.index.v1.IndexMerger; + + import javax.annotation.Nullable; import java.io.File; import java.util.List; import java.util.Map; -import java.util.Set; /** */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "append", value = AppendTask.class) -}) -public abstract class MergeTask extends AbstractTask +public class MergeTask extends MergeTaskBase { - private final List segments; + private final List aggregators; - private static final EmittingLogger log = new EmittingLogger(MergeTask.class); - - protected MergeTask(final String dataSource, final List segments) - { - super( - // _not_ the version, just something uniqueish - String.format("merge_%s_%s", computeProcessingID(dataSource, segments), new DateTime().toString()), - dataSource, - computeMergedInterval(segments) - ); - - // Verify segment list is nonempty - Preconditions.checkArgument(segments.size() > 0, "segments nonempty"); - // Verify segments are all in the correct datasource - Preconditions.checkArgument( - Iterables.size( - Iterables.filter( - segments, - new Predicate() - { - @Override - public boolean apply(@Nullable DataSegment segment) - { - return segment == null || !segment.getDataSource().equalsIgnoreCase(dataSource); - } - } - ) - ) == 0, "segments in the wrong datasource" - ); - // Verify segments are all unsharded - Preconditions.checkArgument( - Iterables.size( - Iterables.filter( - segments, - new Predicate() - { - @Override - public boolean apply(@Nullable DataSegment segment) - { - return segment == null || !(segment.getShardSpec() instanceof NoneShardSpec); - } - } - ) - ) == 0, "segments without NoneShardSpec" - ); - - this.segments = segments; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception - { - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); - 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(); - - try { - - final long startTime = System.currentTimeMillis(); - - log.info( - "Starting merge of id[%s], segments: %s", - getId(), - Lists.transform( - segments, - new Function() - { - @Override - public String apply(@Nullable DataSegment input) - { - return input.getIdentifier(); - } - } - ) - ); - - - // download segments to merge - final Map gettedSegments = toolbox.getSegments(segments); - - // merge files together - final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); - - emitter.emit(builder.build("merger/numMerged", segments.size())); - emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime)); - - log.info( - "[%s] : Merged %d segments in %,d millis", - mergedSegment.getDataSource(), - segments.size(), - System.currentTimeMillis() - startTime - ); - - long uploadStart = System.currentTimeMillis(); - - // Upload file - final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, mergedSegment); - - emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); - emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - - toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); - - return TaskStatus.success(getId()); - } - catch (Exception e) { - log.makeAlert(e, "Exception merging[%s]", mergedSegment.getDataSource()) - .addData("interval", mergedSegment.getInterval()) - .emit(); - - return TaskStatus.failure(getId()); - } - } - - /** - * Checks pre-existing segments in "context" to confirm that this merge query is valid. Specifically, confirm that - * we are operating on every segment that overlaps the chosen interval. - */ - @Override - public TaskStatus preflight(TaskToolbox toolbox) - { - final Function toIdentifier = new Function() - { - @Override - public String apply(DataSegment dataSegment) - { - return dataSegment.getIdentifier(); - } - }; - - final Set current = ImmutableSet.copyOf( - Iterables.transform(toolbox.getTaskActionClientFactory().submit(defaultListUsedAction()), toIdentifier) - ); - final Set requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier)); - - final Set missingFromRequested = Sets.difference(current, requested); - if (!missingFromRequested.isEmpty()) { - throw new ISE( - "Merge is invalid: current segment(s) are not in the requested set: %s", - Joiner.on(", ").join(missingFromRequested) - ); - } - - final Set missingFromCurrent = Sets.difference(requested, current); - if (!missingFromCurrent.isEmpty()) { - throw new ISE( - "Merge is invalid: requested segment(s) are not in the current set: %s", - Joiner.on(", ").join(missingFromCurrent) - ); - } - - return TaskStatus.running(getId()); - - } - - protected abstract File merge(Map segments, File outDir) - throws Exception; - - @JsonProperty - public List getSegments() - { - return segments; - } - - @Override - public String toString() - { - return Objects.toStringHelper(this) - .add("id", getId()) - .add("dataSource", getDataSource()) - .add("interval", getImplicitLockInterval()) - .add("segments", segments) - .toString(); - } - - private static String computeProcessingID(final String dataSource, final List segments) - { - final String segmentIDs = Joiner.on("_").join( - Iterables.transform( - Ordering.natural().sortedCopy(segments), new Function() - { - @Override - public String apply(DataSegment x) - { - return String.format( - "%s_%s_%s_%s", - x.getInterval().getStart(), - x.getInterval().getEnd(), - x.getVersion(), - x.getShardSpec().getPartitionNum() - ); - } - } - ) - ); - - return String.format( - "%s_%s", - dataSource, - Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase() - ); - } - - private static Interval computeMergedInterval(final List segments) - { - Preconditions.checkArgument(segments.size() > 0, "segments.size() > 0"); - - DateTime start = null; - DateTime end = null; - - for(final DataSegment segment : segments) { - if(start == null || segment.getInterval().getStart().isBefore(start)) { - start = segment.getInterval().getStart(); - } - - if(end == null || segment.getInterval().getEnd().isAfter(end)) { - end = segment.getInterval().getEnd(); - } - } - - return new Interval(start, end); - } - - private static DataSegment computeMergedSegment( - final String dataSource, - final String version, - final List segments + @JsonCreator + public MergeTask( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segments") List segments, + @JsonProperty("aggregations") List aggregators ) { - final Interval mergedInterval = computeMergedInterval(segments); - final Set mergedDimensions = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); - final Set mergedMetrics = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + super(dataSource, segments); + this.aggregators = aggregators; + } - for (DataSegment segment : segments) { - mergedDimensions.addAll(segment.getDimensions()); - mergedMetrics.addAll(segment.getMetrics()); - } + @Override + public File merge(final Map segments, final File outDir) + throws Exception + { + return IndexMerger.mergeQueryableIndex( + Lists.transform( + ImmutableList.copyOf(segments.values()), + new Function() + { + @Override + public QueryableIndex apply(@Nullable File input) + { + try { + return IndexIO.loadIndex(input); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ), + aggregators.toArray(new AggregatorFactory[aggregators.size()]), + outDir + ); + } - return DataSegment.builder() - .dataSource(dataSource) - .interval(mergedInterval) - .version(version) - .shardSpec(new NoneShardSpec()) - .dimensions(Lists.newArrayList(mergedDimensions)) - .metrics(Lists.newArrayList(mergedMetrics)) - .build(); + @Override + public String getType() + { + return "merge"; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java new file mode 100644 index 00000000000..e0b3dd6ff17 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java @@ -0,0 +1,315 @@ +/* + * 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.task; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.base.Charsets; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; +import com.google.common.hash.Hashing; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskLock; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.LockListAction; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import com.metamx.druid.shard.NoneShardSpec; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.service.AlertEvent; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + */ +public abstract class MergeTaskBase extends AbstractTask +{ + private final List segments; + + private static final EmittingLogger log = new EmittingLogger(MergeTaskBase.class); + + protected MergeTaskBase(final String dataSource, final List segments) + { + super( + // _not_ the version, just something uniqueish + String.format("merge_%s_%s", computeProcessingID(dataSource, segments), new DateTime().toString()), + dataSource, + computeMergedInterval(segments) + ); + + // Verify segment list is nonempty + Preconditions.checkArgument(segments.size() > 0, "segments nonempty"); + // Verify segments are all in the correct datasource + Preconditions.checkArgument( + Iterables.size( + Iterables.filter( + segments, + new Predicate() + { + @Override + public boolean apply(@Nullable DataSegment segment) + { + return segment == null || !segment.getDataSource().equalsIgnoreCase(dataSource); + } + } + ) + ) == 0, "segments in the wrong datasource" + ); + // Verify segments are all unsharded + Preconditions.checkArgument( + Iterables.size( + Iterables.filter( + segments, + new Predicate() + { + @Override + public boolean apply(@Nullable DataSegment segment) + { + return segment == null || !(segment.getShardSpec() instanceof NoneShardSpec); + } + } + ) + ) == 0, "segments without NoneShardSpec" + ); + + this.segments = segments; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); + 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(); + + try { + + final long startTime = System.currentTimeMillis(); + + log.info( + "Starting merge of id[%s], segments: %s", + getId(), + Lists.transform( + segments, + new Function() + { + @Override + public String apply(@Nullable DataSegment input) + { + return input.getIdentifier(); + } + } + ) + ); + + + // download segments to merge + final Map gettedSegments = toolbox.getSegments(segments); + + // merge files together + final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); + + emitter.emit(builder.build("merger/numMerged", segments.size())); + emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime)); + + log.info( + "[%s] : Merged %d segments in %,d millis", + mergedSegment.getDataSource(), + segments.size(), + System.currentTimeMillis() - startTime + ); + + long uploadStart = System.currentTimeMillis(); + + // Upload file + final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, mergedSegment); + + emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); + emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); + + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); + + return TaskStatus.success(getId()); + } + catch (Exception e) { + log.makeAlert(e, "Exception merging[%s]", mergedSegment.getDataSource()) + .addData("interval", mergedSegment.getInterval()) + .emit(); + + return TaskStatus.failure(getId()); + } + } + + /** + * Checks pre-existing segments in "context" to confirm that this merge query is valid. Specifically, confirm that + * we are operating on every segment that overlaps the chosen interval. + */ + @Override + public TaskStatus preflight(TaskToolbox toolbox) + { + final Function toIdentifier = new Function() + { + @Override + public String apply(DataSegment dataSegment) + { + return dataSegment.getIdentifier(); + } + }; + + final Set current = ImmutableSet.copyOf( + Iterables.transform(toolbox.getTaskActionClientFactory().submit(defaultListUsedAction()), toIdentifier) + ); + final Set requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier)); + + final Set missingFromRequested = Sets.difference(current, requested); + if (!missingFromRequested.isEmpty()) { + throw new ISE( + "Merge is invalid: current segment(s) are not in the requested set: %s", + Joiner.on(", ").join(missingFromRequested) + ); + } + + final Set missingFromCurrent = Sets.difference(requested, current); + if (!missingFromCurrent.isEmpty()) { + throw new ISE( + "Merge is invalid: requested segment(s) are not in the current set: %s", + Joiner.on(", ").join(missingFromCurrent) + ); + } + + return TaskStatus.running(getId()); + + } + + protected abstract File merge(Map segments, File outDir) + throws Exception; + + @JsonProperty + public List getSegments() + { + return segments; + } + + @Override + public String toString() + { + return Objects.toStringHelper(this) + .add("id", getId()) + .add("dataSource", getDataSource()) + .add("interval", getImplicitLockInterval()) + .add("segments", segments) + .toString(); + } + + private static String computeProcessingID(final String dataSource, final List segments) + { + final String segmentIDs = Joiner.on("_").join( + Iterables.transform( + Ordering.natural().sortedCopy(segments), new Function() + { + @Override + public String apply(DataSegment x) + { + return String.format( + "%s_%s_%s_%s", + x.getInterval().getStart(), + x.getInterval().getEnd(), + x.getVersion(), + x.getShardSpec().getPartitionNum() + ); + } + } + ) + ); + + return String.format( + "%s_%s", + dataSource, + Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase() + ); + } + + private static Interval computeMergedInterval(final List segments) + { + Preconditions.checkArgument(segments.size() > 0, "segments.size() > 0"); + + DateTime start = null; + DateTime end = null; + + for(final DataSegment segment : segments) { + if(start == null || segment.getInterval().getStart().isBefore(start)) { + start = segment.getInterval().getStart(); + } + + if(end == null || segment.getInterval().getEnd().isAfter(end)) { + end = segment.getInterval().getEnd(); + } + } + + return new Interval(start, end); + } + + private static DataSegment computeMergedSegment( + final String dataSource, + final String version, + final List segments + ) + { + final Interval mergedInterval = computeMergedInterval(segments); + final Set mergedDimensions = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + final Set mergedMetrics = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + + for (DataSegment segment : segments) { + mergedDimensions.addAll(segment.getDimensions()); + mergedMetrics.addAll(segment.getMetrics()); + } + + return DataSegment.builder() + .dataSource(dataSource) + .interval(mergedInterval) + .version(version) + .shardSpec(new NoneShardSpec()) + .dimensions(Lists.newArrayList(mergedDimensions)) + .metrics(Lists.newArrayList(mergedMetrics)) + .build(); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 8418ecf40a8..5f288be99dc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -41,10 +41,10 @@ import org.joda.time.Interval; * to release locks early if they desire. * */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "append", value = AppendTask.class), - @JsonSubTypes.Type(name = "merge", value = DefaultMergeTask.class), + @JsonSubTypes.Type(name = "merge", value = MergeTask.class), @JsonSubTypes.Type(name = "delete", value = DeleteTask.class), @JsonSubTypes.Type(name = "kill", value = KillTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), @@ -52,7 +52,7 @@ import org.joda.time.Interval; @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class), - @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterSubTask.class) + @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class) }) public interface Task { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java deleted file mode 100644 index 2099d903d33..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java +++ /dev/null @@ -1,94 +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.task; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.Sets; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; -import org.joda.time.DateTime; - -import java.io.File; -import java.util.Arrays; -import java.util.Map; - -/** - */ -public class VersionConverterSubTask extends AbstractTask -{ - private static final Logger log = new Logger(VersionConverterSubTask.class); - - private final DataSegment segment; - - protected VersionConverterSubTask( - @JsonProperty("groupId") String groupId, - @JsonProperty("segment") DataSegment segment - ) - { - super( - joinId( - groupId, - "sub", - segment.getInterval().getStart(), - segment.getInterval().getEnd(), - segment.getShardSpec().getPartitionNum() - ), - groupId, - segment.getDataSource(), - segment.getInterval() - ); - this.segment = segment; - } - - @Override - public String getType() - { - return "version_converter_sub"; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception - { - log.info("Converting segment[%s]", segment); - final Map localSegments = toolbox.getSegments(Arrays.asList(segment)); - - final File location = localSegments.get(segment); - final File outLocation = new File(location, "v9_out"); - if (IndexIO.convertSegment(location, outLocation)) { - final int outVersion = IndexIO.getVersionFromDir(outLocation); - - // Appending to the version makes a new version that inherits most comparability parameters of the original - // version, but is "newer" than said original version. - DataSegment updatedSegment = segment.withVersion(String.format("%s_v%s", segment.getVersion(), outVersion)); - updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment); - - toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); - } - else { - log.info("Conversion failed."); - } - - return success(); - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java index 3859190a1bb..c8c0e2cbf42 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java @@ -22,19 +22,26 @@ package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; import com.metamx.druid.merger.common.actions.SpawnTasksAction; import com.metamx.druid.merger.common.actions.TaskActionClient; import org.joda.time.DateTime; import org.joda.time.Interval; +import java.io.File; +import java.io.IOException; +import java.util.Arrays; import java.util.List; +import java.util.Map; /** */ @@ -44,10 +51,12 @@ public class VersionConverterTask extends AbstractTask private static final Integer CURR_VERSION_INTEGER = new Integer(IndexIO.CURRENT_VERSION_ID); private static final Logger log = new Logger(VersionConverterTask.class); + private final DataSegment segment; public VersionConverterTask( @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval + @JsonProperty("interval") Interval interval, + @JsonProperty("segment") DataSegment segment ) { super( @@ -55,6 +64,8 @@ public class VersionConverterTask extends AbstractTask dataSource, interval ); + + this.segment = segment; } @Override @@ -66,12 +77,22 @@ public class VersionConverterTask extends AbstractTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - throw new ISE("Should never run, %s just exists to create subtasks", this.getClass().getSimpleName()); + if (segment == null) { + throw new ISE("Segment was null, this should never run.", this.getClass().getSimpleName()); + } + + log.info("I'm in a subless mood."); + convertSegment(toolbox, segment); + return success(); } @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { + if (segment != null) { + return super.preflight(toolbox); + } + final TaskActionClient taskClient = toolbox.getTaskActionClientFactory(); List segments = taskClient.submit(defaultListUsedAction()); @@ -86,7 +107,7 @@ public class VersionConverterTask extends AbstractTask { final Integer segmentVersion = segment.getBinaryVersion(); if (!CURR_VERSION_INTEGER.equals(segmentVersion)) { - return new VersionConverterSubTask(getGroupId(), segment); + return new SubTask(getGroupId(), segment); } log.info("Skipping[%s], already version[%s]", segment.getIdentifier(), segmentVersion); @@ -99,4 +120,65 @@ public class VersionConverterTask extends AbstractTask return TaskStatus.success(getId()); } + + public static class SubTask extends AbstractTask + { + private final DataSegment segment; + + protected SubTask( + @JsonProperty("groupId") String groupId, + @JsonProperty("segment") DataSegment segment + ) + { + super( + joinId( + groupId, + "sub", + segment.getInterval().getStart(), + segment.getInterval().getEnd(), + segment.getShardSpec().getPartitionNum() + ), + groupId, + segment.getDataSource(), + segment.getInterval() + ); + this.segment = segment; + } + + @Override + public String getType() + { + return "version_converter_sub"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + log.info("Subs are good! Italian BMT and Meatball are probably my favorite."); + convertSegment(toolbox, segment); + return success(); + } + } + + private static void convertSegment(TaskToolbox toolbox, final DataSegment segment) + throws SegmentLoadingException, IOException + { + log.info("Converting segment[%s]", segment); + final Map localSegments = toolbox.getSegments(Arrays.asList(segment)); + + final File location = localSegments.get(segment); + final File outLocation = new File(location, "v9_out"); + if (IndexIO.convertSegment(location, outLocation)) { + final int outVersion = IndexIO.getVersionFromDir(outLocation); + + // Appending to the version makes a new version that inherits most comparability parameters of the original + // version, but is "newer" than said original version. + DataSegment updatedSegment = segment.withVersion(String.format("%s_v%s", segment.getVersion(), outVersion)); + updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment); + + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); + } else { + log.info("Conversion failed."); + } + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java deleted file mode 100644 index 16eeb1c3439..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.merger.coordinator.config; - -import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; - -/** - */ -public abstract class WorkerSetupManagerConfig -{ - @Config("druid.indexer.configTable") - public abstract String getConfigTable(); - - @Config("druid.indexer.workerSetupConfigName") - public abstract String getWorkerSetupConfigName(); - - @Config("druid.indexer.poll.duration") - @Default("PT1M") - public abstract Duration getPollDuration(); -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 85b28362799..4c6d47dbad2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -39,6 +39,9 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.RegisteringNode; +import com.metamx.druid.config.ConfigManager; +import com.metamx.druid.config.ConfigManagerConfig; +import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.http.GuiceServletConfig; @@ -77,7 +80,6 @@ import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; -import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.NoopAutoScalingStrategy; @@ -556,18 +558,12 @@ public class IndexerCoordinatorNode extends RegisteringNode public void initializeWorkerSetupManager() { if (workerSetupManager == null) { - final WorkerSetupManagerConfig workerSetupManagerConfig = configFactory.build(WorkerSetupManagerConfig.class); + final ConfigManagerConfig configManagerConfig = configFactory.build(ConfigManagerConfig.class); + final ConfigManager configManager = new ConfigManager(dbi, configManagerConfig); + lifecycle.addManagedInstance(configManager); - DbConnector.createConfigTable(dbi, workerSetupManagerConfig.getConfigTable()); - workerSetupManager = new WorkerSetupManager( - dbi, Executors.newScheduledThreadPool( - 1, - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("WorkerSetupManagerExec--%d") - .build() - ), jsonMapper, workerSetupManagerConfig - ); + DbConnector.createConfigTable(dbi, configManagerConfig.getConfigTable()); + workerSetupManager = new WorkerSetupManager(new JacksonConfigManager(configManager, jsonMapper)); } lifecycle.addManagedInstance(workerSetupManager); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index 1496552bb6e..762d9ad28ee 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -95,18 +95,15 @@ public class IndexerCoordinatorResource @Produces("application/json") public Response doIndex(final Task task) { - // verify against whitelist - if (config.isWhitelistEnabled() && !config.getWhitelistDatasources().contains(task.getDataSource())) { - return Response.status(Response.Status.BAD_REQUEST) - .entity( - ImmutableMap.of( - "error", - String.format("dataSource[%s] is not whitelisted", task.getDataSource()) - ) - ) - .build(); - } + return taskPost(task); + } + @POST + @Path("/task") + @Consumes("application/json") + @Produces("application/json") + public Response taskPost(final Task task) + { taskMasterLifecycle.getTaskQueue().add(task); return Response.ok(ImmutableMap.of("task", task.getId())).build(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java index 89a0dd2d5c1..b9ce066c327 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java @@ -19,209 +19,41 @@ package com.metamx.druid.merger.coordinator.setup; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.common.logger.Logger; -import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; -import org.apache.commons.collections.MapUtils; +import com.metamx.druid.config.JacksonConfigManager; -import org.joda.time.Duration; -import org.skife.jdbi.v2.DBI; -import org.skife.jdbi.v2.FoldController; -import org.skife.jdbi.v2.Folder3; -import org.skife.jdbi.v2.Handle; -import org.skife.jdbi.v2.StatementContext; -import org.skife.jdbi.v2.tweak.HandleCallback; - -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; /** */ public class WorkerSetupManager { - private static final Logger log = new Logger(WorkerSetupManager.class); + private static final String WORKER_SETUP_KEY = "worker.setup"; - private final DBI dbi; - private final ObjectMapper jsonMapper; - private final ScheduledExecutorService exec; - private final WorkerSetupManagerConfig config; - - private final Object lock = new Object(); + private final JacksonConfigManager configManager; private volatile AtomicReference workerSetupData = new AtomicReference(null); - private volatile boolean started = false; public WorkerSetupManager( - DBI dbi, - ScheduledExecutorService exec, - ObjectMapper jsonMapper, - WorkerSetupManagerConfig config + JacksonConfigManager configManager ) { - this.dbi = dbi; - this.exec = exec; - this.jsonMapper = jsonMapper; - this.config = config; + this.configManager = configManager; } @LifecycleStart public void start() { - synchronized (lock) { - if (started) { - return; - } - - ScheduledExecutors.scheduleWithFixedDelay( - exec, - new Duration(0), - config.getPollDuration(), - new Runnable() - { - @Override - public void run() - { - poll(); - } - } - ); - - started = true; - } + workerSetupData = configManager.watch(WORKER_SETUP_KEY, WorkerSetupData.class); } - @LifecycleStop - public void stop() - { - synchronized (lock) { - if (!started) { - return; - } - - started = false; - } - } - - public void poll() - { - try { - List setupDataList = dbi.withHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) throws Exception - { - return handle.createQuery( - String.format( - "SELECT payload FROM %s WHERE name = :name", - config.getConfigTable() - ) - ) - .bind("name", config.getWorkerSetupConfigName()) - .fold( - Lists.newArrayList(), - new Folder3, Map>() - { - @Override - public ArrayList fold( - ArrayList workerNodeConfigurations, - Map stringObjectMap, - FoldController foldController, - StatementContext statementContext - ) throws SQLException - { - try { - // stringObjectMap lowercases and jackson may fail serde - workerNodeConfigurations.add( - jsonMapper.readValue( - MapUtils.getString(stringObjectMap, "payload"), - WorkerSetupData.class - ) - ); - return workerNodeConfigurations; - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ); - } - } - ); - - if (setupDataList.isEmpty()) { - throw new ISE("WTF?! No configuration found for worker nodes!"); - } else if (setupDataList.size() != 1) { - throw new ISE("WTF?! Found more than one configuration for worker nodes"); - } - - workerSetupData.set(setupDataList.get(0)); - } - catch (Exception e) { - log.error(e, "Exception while polling for worker setup data!"); - } - } - - @SuppressWarnings("unchecked") public WorkerSetupData getWorkerSetupData() { - synchronized (lock) { - if (!started) { - throw new ISE("Must start WorkerSetupManager first!"); - } - - return workerSetupData.get(); - } + return workerSetupData.get(); } public boolean setWorkerSetupData(final WorkerSetupData value) { - synchronized (lock) { - try { - if (!started) { - throw new ISE("Must start WorkerSetupManager first!"); - } - - dbi.withHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) throws Exception - { - handle.createStatement( - String.format( - "INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload", - config.getConfigTable() - ) - ) - .bind("name", config.getWorkerSetupConfigName()) - .bind("payload", jsonMapper.writeValueAsString(value)) - .execute(); - - return null; - } - } - ); - - workerSetupData.set(value); - } - catch (Exception e) { - log.error(e, "Exception updating worker config"); - return false; - } - } - - return true; + return configManager.set(WORKER_SETUP_KEY, value); } } diff --git a/merger/src/test/java/com/metamx/druid/merger/TestTask.java b/merger/src/test/java/com/metamx/druid/merger/TestTask.java index c23b498f739..d0a77cff447 100644 --- a/merger/src/test/java/com/metamx/druid/merger/TestTask.java +++ b/merger/src/test/java/com/metamx/druid/merger/TestTask.java @@ -26,14 +26,14 @@ import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.task.DefaultMergeTask; +import com.metamx.druid.merger.common.task.MergeTask; import java.util.List; /** */ @JsonTypeName("test") -public class TestTask extends DefaultMergeTask +public class TestTask extends MergeTask { private final String id; private final TaskStatus status; diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java similarity index 89% rename from merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java rename to merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java index 7c779f6a74c..a2f6e8175fb 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java @@ -31,7 +31,7 @@ import java.io.File; import java.util.List; import java.util.Map; -public class MergeTaskTest +public class MergeTaskBaseTest { private final DataSegment.Builder segmentBuilder = DataSegment.builder() .dataSource("foo") @@ -43,7 +43,7 @@ public class MergeTaskTest .add(segmentBuilder.interval(new Interval("2012-01-03/2012-01-05")).build()) .build(); - final MergeTask testMergeTask = new MergeTask("foo", segments) + final MergeTaskBase testMergeTaskBase = new MergeTaskBase("foo", segments) { @Override protected File merge(Map segments, File outDir) throws Exception @@ -61,13 +61,13 @@ public class MergeTaskTest @Test public void testDataSource() { - Assert.assertEquals("foo", testMergeTask.getDataSource()); + Assert.assertEquals("foo", testMergeTaskBase.getDataSource()); } @Test public void testInterval() { - Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getImplicitLockInterval().get()); + Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTaskBase.getImplicitLockInterval().get()); } @Test @@ -81,7 +81,7 @@ public class MergeTaskTest ).toString().toLowerCase() + "_"; Assert.assertEquals( desiredPrefix, - testMergeTask.getId().substring(0, desiredPrefix.length()) + testMergeTaskBase.getId().substring(0, desiredPrefix.length()) ); } } diff --git a/pom.xml b/pom.xml index 29724d5ba3e..f61f80c1ff4 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ UTF-8 - 0.20.0 + 0.20.1-SNAPSHOT diff --git a/server/src/main/java/com/metamx/druid/http/InfoResource.java b/server/src/main/java/com/metamx/druid/http/InfoResource.java index e434bff995d..acd94438d84 100644 --- a/server/src/main/java/com/metamx/druid/http/InfoResource.java +++ b/server/src/main/java/com/metamx/druid/http/InfoResource.java @@ -34,8 +34,8 @@ import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.master.DruidMaster; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.master.rules.Rule; -import com.metamx.druid.merge.ClientKillQuery; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -66,6 +66,7 @@ public class InfoResource private final DatabaseSegmentManager databaseSegmentManager; private final DatabaseRuleManager databaseRuleManager; private final DruidClusterInfo druidClusterInfo; + private final IndexingServiceClient indexingServiceClient; @Inject public InfoResource( @@ -73,7 +74,8 @@ public class InfoResource ServerInventoryManager serverInventoryManager, DatabaseSegmentManager databaseSegmentManager, DatabaseRuleManager databaseRuleManager, - DruidClusterInfo druidClusterInfo + DruidClusterInfo druidClusterInfo, + IndexingServiceClient indexingServiceClient ) { this.master = master; @@ -81,6 +83,7 @@ public class InfoResource this.databaseSegmentManager = databaseSegmentManager; this.databaseRuleManager = databaseRuleManager; this.druidClusterInfo = druidClusterInfo; + this.indexingServiceClient = indexingServiceClient; } @GET @@ -374,7 +377,7 @@ public class InfoResource ) { if (kill != null && Boolean.valueOf(kill)) { - master.killSegments(new ClientKillQuery(dataSourceName, new Interval(interval))); + indexingServiceClient.killSegments(dataSourceName, new Interval(interval)); } else { if (!databaseSegmentManager.removeDatasource(dataSourceName)) { return Response.status(Response.Status.NOT_FOUND).build(); diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index 64e63176d0c..32e03e7ba63 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -33,6 +33,9 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.client.ServerInventoryManagerConfig; +import com.metamx.druid.config.ConfigManager; +import com.metamx.druid.config.ConfigManagerConfig; +import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.coordination.DruidClusterInfoConfig; import com.metamx.druid.db.DatabaseRuleManager; @@ -49,6 +52,7 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterConfig; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.master.LoadQueuePeon; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; @@ -86,7 +90,7 @@ import java.util.concurrent.ScheduledExecutorService; */ public class MasterMain { - private static final Logger log = new Logger(ServerMain.class); + private static final Logger log = new Logger(MasterMain.class); public static void main(String[] args) throws Exception { @@ -166,13 +170,14 @@ public class MasterMain lifecycle ); - ServiceProvider serviceProvider = null; + IndexingServiceClient indexingServiceClient = null; if (druidMasterConfig.getMergerServiceName() != null) { - serviceProvider = Initialization.makeServiceProvider( + ServiceProvider serviceProvider = Initialization.makeServiceProvider( druidMasterConfig.getMergerServiceName(), serviceDiscovery, lifecycle ); + indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider); } final DruidClusterInfo druidClusterInfo = new DruidClusterInfo( @@ -180,10 +185,14 @@ public class MasterMain masterYp ); + JacksonConfigManager configManager = new JacksonConfigManager( + new ConfigManager(dbi, configFactory.build(ConfigManagerConfig.class)), jsonMapper + ); + final DruidMaster master = new DruidMaster( druidMasterConfig, druidClusterInfo, - jsonMapper, + configManager, databaseSegmentManager, serverInventoryManager, databaseRuleManager, @@ -191,9 +200,7 @@ public class MasterMain emitter, scheduledExecutorFactory, new ConcurrentHashMap(), - serviceProvider, - httpClient, - new ToStringResponseHandler(Charsets.UTF_8) + indexingServiceClient ); lifecycle.addManagedInstance(master); @@ -226,7 +233,8 @@ public class MasterMain databaseRuleManager, druidClusterInfo, master, - jsonMapper + jsonMapper, + indexingServiceClient ) ); diff --git a/server/src/main/java/com/metamx/druid/http/MasterResource.java b/server/src/main/java/com/metamx/druid/http/MasterResource.java index b725ed7f358..9bb59d79d43 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterResource.java +++ b/server/src/main/java/com/metamx/druid/http/MasterResource.java @@ -21,7 +21,6 @@ package com.metamx.druid.http; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.LoadPeonCallback; -import com.metamx.druid.merge.ClientKillQuery; import javax.inject.Inject; import javax.ws.rs.Consumes; diff --git a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java index 47395f73eeb..64d0c98afa3 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java +++ b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java @@ -27,6 +27,7 @@ import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.master.DruidMaster; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; @@ -44,6 +45,7 @@ public class MasterServletModule extends JerseyServletModule private final DruidClusterInfo druidClusterInfo; private final DruidMaster master; private final ObjectMapper jsonMapper; + private final IndexingServiceClient indexingServiceClient; public MasterServletModule( ServerInventoryManager serverInventoryManager, @@ -51,7 +53,8 @@ public class MasterServletModule extends JerseyServletModule DatabaseRuleManager databaseRuleManager, DruidClusterInfo druidClusterInfo, DruidMaster master, - ObjectMapper jsonMapper + ObjectMapper jsonMapper, + IndexingServiceClient indexingServiceClient ) { this.serverInventoryManager = serverInventoryManager; @@ -60,6 +63,7 @@ public class MasterServletModule extends JerseyServletModule this.druidClusterInfo = druidClusterInfo; this.master = master; this.jsonMapper = jsonMapper; + this.indexingServiceClient = indexingServiceClient; } @Override @@ -72,6 +76,7 @@ public class MasterServletModule extends JerseyServletModule bind(DatabaseRuleManager.class).toInstance(databaseRuleManager); bind(DruidMaster.class).toInstance(master); bind(DruidClusterInfo.class).toInstance(druidClusterInfo); + bind(IndexingServiceClient.class).toInstance(indexingServiceClient); serve("/*").with(GuiceContainer.class); } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index a21408144b3..74b9d17d57b 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -19,10 +19,8 @@ package com.metamx.druid.master; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Predicate; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -41,32 +39,30 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.ServerInventoryManager; +import com.metamx.druid.client.indexing.IndexingServiceClient; +import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; -import com.metamx.druid.merge.ClientKillQuery; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.response.HttpResponseHandler; import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; -import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.exception.ZkNodeExistsException; import org.joda.time.DateTime; import org.joda.time.Duration; import javax.annotation.Nullable; -import java.net.URL; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.Callable; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -83,26 +79,22 @@ public class DruidMaster private final DruidMasterConfig config; private final DruidClusterInfo clusterInfo; + private final JacksonConfigManager configManager; private final DatabaseSegmentManager databaseSegmentManager; private final ServerInventoryManager serverInventoryManager; private final DatabaseRuleManager databaseRuleManager; private final PhoneBook yp; private final ServiceEmitter emitter; + private final IndexingServiceClient indexingServiceClient; private final ScheduledExecutorService exec; private final ScheduledExecutorService peonExec; private final PhoneBookPeon masterPeon; private final Map loadManagementPeons; - private final ServiceProvider serviceProvider; - - private final HttpClient httpClient; - private final HttpResponseHandler responseHandler; - - private final ObjectMapper jsonMapper; public DruidMaster( DruidMasterConfig config, DruidClusterInfo clusterInfo, - ObjectMapper jsonMapper, + JacksonConfigManager configManager, DatabaseSegmentManager databaseSegmentManager, ServerInventoryManager serverInventoryManager, DatabaseRuleManager databaseRuleManager, @@ -110,31 +102,25 @@ public class DruidMaster ServiceEmitter emitter, ScheduledExecutorFactory scheduledExecutorFactory, Map loadManagementPeons, - ServiceProvider serviceProvider, - HttpClient httpClient, - HttpResponseHandler responseHandler + IndexingServiceClient indexingServiceClient ) { this.config = config; this.clusterInfo = clusterInfo; - - this.jsonMapper = jsonMapper; + this.configManager = configManager; this.databaseSegmentManager = databaseSegmentManager; this.serverInventoryManager = serverInventoryManager; this.databaseRuleManager = databaseRuleManager; this.yp = zkPhoneBook; this.emitter = emitter; + this.indexingServiceClient = indexingServiceClient; this.masterPeon = new MasterListeningPeon(); this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d"); this.peonExec = scheduledExecutorFactory.create(1, "Master-PeonExec--%d"); this.loadManagementPeons = loadManagementPeons; - - this.serviceProvider = serviceProvider; - this.httpClient = httpClient; - this.responseHandler = responseHandler; } public boolean isClusterMaster() @@ -349,27 +335,6 @@ public class DruidMaster } } - public void killSegments(ClientKillQuery killQuery) - { - try { - httpClient.post( - new URL( - String.format( - "http://%s:%s/mmx/merger/v1/index", - serviceProvider.getInstance().getAddress(), - serviceProvider.getInstance().getPort() - ) - ) - ) - .setContent("application/json", jsonMapper.writeValueAsBytes(killQuery)) - .go(responseHandler) - .get(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - public Set getAvailableDataSegments() { Set availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator())); @@ -390,7 +355,9 @@ public class DruidMaster for (DataSegment dataSegment : dataSegments) { if (dataSegment.getSize() < 0) { - log.warn("No size on Segment[%s], wtf?", dataSegment); + log.makeAlert("No size on Segment, wtf?") + .addData("segment", dataSegment) + .emit(); } availableSegments.add(dataSegment); } @@ -466,8 +433,14 @@ public class DruidMaster final List> masterRunnables = Lists.newArrayList(); masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod())); - if (config.isMergeSegments() && serviceProvider != null) { - masterRunnables.add(Pair.of(new MasterSegmentMergerRunnable(), config.getMasterSegmentMergerPeriod())); + if (config.isMergeSegments() && indexingServiceClient != null) { + + masterRunnables.add( + Pair.of( + new MasterSegmentMergerRunnable(configManager.watch(MergerWhitelist.CONFIG_KEY, MergerWhitelist.class)), + config.getMasterSegmentMergerPeriod() + ) + ); } for (final Pair masterRunnable : masterRunnables) { @@ -529,6 +502,39 @@ public class DruidMaster } } + public static class DruidMasterVersionConverter implements DruidMasterHelper + { + private final IndexingServiceClient indexingServiceClient; + private final AtomicReference whitelistRef; + + public DruidMasterVersionConverter( + IndexingServiceClient indexingServiceClient, + AtomicReference whitelistRef + ) + { + this.indexingServiceClient = indexingServiceClient; + this.whitelistRef = whitelistRef; + } + + @Override + public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + { + MergerWhitelist whitelist = whitelistRef.get(); + + for (DataSegment dataSegment : params.getAvailableSegments()) { + if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) { + final Integer binaryVersion = dataSegment.getBinaryVersion(); + + if (binaryVersion == null || binaryVersion < IndexIO.CURRENT_VERSION_ID) { + indexingServiceClient.upgradeSegment(dataSegment); + } + } + } + + return params; + } + } + private class MasterListeningPeon implements PhoneBookPeon { @Override @@ -723,12 +729,13 @@ public class DruidMaster private class MasterSegmentMergerRunnable extends MasterRunnable { - private MasterSegmentMergerRunnable() + private MasterSegmentMergerRunnable(final AtomicReference whitelistRef) { super( ImmutableList.of( new DruidMasterSegmentInfoLoader(DruidMaster.this), - new DruidMasterSegmentMerger(jsonMapper, serviceProvider), + new DruidMasterVersionConverter(indexingServiceClient, whitelistRef), + new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef), new DruidMasterHelper() { @Override @@ -739,8 +746,7 @@ public class DruidMaster params.getEmitter().emit( new ServiceMetricEvent.Builder().build( - "master/merge/count", - stats.getGlobalStats().get("mergedCount") + "master/merge/count", stats.getGlobalStats().get("mergedCount") ) ); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java index f3d93d0e6fd..ef4f9d1e6d3 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java @@ -19,8 +19,6 @@ package com.metamx.druid.master; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.HashMultiset; @@ -32,22 +30,19 @@ import com.google.common.collect.Multiset; import com.google.common.collect.Ordering; import com.metamx.common.Pair; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.partition.PartitionChunk; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; -import com.metamx.http.client.response.ToStringResponseHandler; -import com.netflix.curator.x.discovery.ServiceProvider; import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -55,44 +50,40 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper { private static final Logger log = new Logger(DruidMasterSegmentMerger.class); - private final MergerClient mergerClient; + private final IndexingServiceClient indexingServiceClient; + private final AtomicReference whiteListRef; - public DruidMasterSegmentMerger(MergerClient mergerClient) + public DruidMasterSegmentMerger( + IndexingServiceClient indexingServiceClient, + AtomicReference whitelistRef + ) { - this.mergerClient = mergerClient; - } - - public DruidMasterSegmentMerger(ObjectMapper jsonMapper, ServiceProvider serviceProvider) - { - this.mergerClient = new HttpMergerClient( - HttpClientInit.createClient( - HttpClientConfig.builder().withNumConnections(1).build(), - new Lifecycle() - ), - new ToStringResponseHandler(Charsets.UTF_8), - jsonMapper, - serviceProvider - ); + this.indexingServiceClient = indexingServiceClient; + this.whiteListRef = whitelistRef; } @Override public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) { + MergerWhitelist whitelist = whiteListRef.get(); + MasterStats stats = new MasterStats(); Map> dataSources = Maps.newHashMap(); // Find serviced segments by using a timeline for (DataSegment dataSegment : params.getAvailableSegments()) { - VersionedIntervalTimeline timeline = dataSources.get(dataSegment.getDataSource()); - if (timeline == null) { - timeline = new VersionedIntervalTimeline(Ordering.natural()); - dataSources.put(dataSegment.getDataSource(), timeline); + if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) { + VersionedIntervalTimeline timeline = dataSources.get(dataSegment.getDataSource()); + if (timeline == null) { + timeline = new VersionedIntervalTimeline(Ordering.natural()); + dataSources.put(dataSegment.getDataSource(), timeline); + } + timeline.add( + dataSegment.getInterval(), + dataSegment.getVersion(), + dataSegment.getShardSpec().createChunk(dataSegment) + ); } - timeline.add( - dataSegment.getInterval(), - dataSegment.getVersion(), - dataSegment.getShardSpec().createChunk(dataSegment) - ); } // Find segments to merge @@ -161,7 +152,7 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper log.info("[%s] Found %d segments to merge %s", dataSource, segments.size(), segmentNames); try { - mergerClient.runRequest(dataSource, segments); + indexingServiceClient.mergeSegments(segments); } catch (Exception e) { log.error( diff --git a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java b/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java deleted file mode 100644 index 07a0f8dba6e..00000000000 --- a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java +++ /dev/null @@ -1,78 +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.master; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merge.ClientAppendQuery; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.response.HttpResponseHandler; -import com.netflix.curator.x.discovery.ServiceProvider; - - -import java.net.URL; -import java.util.List; - -public class HttpMergerClient implements MergerClient -{ - private final HttpClient client; - private final HttpResponseHandler responseHandler; - private final ObjectMapper jsonMapper; - private final ServiceProvider serviceProvider; - - public HttpMergerClient( - HttpClient client, - HttpResponseHandler responseHandler, - ObjectMapper jsonMapper, - ServiceProvider serviceProvider - ) - { - this.client = client; - this.responseHandler = responseHandler; - this.jsonMapper = jsonMapper; - this.serviceProvider = serviceProvider; - } - - public void runRequest(String dataSource, List segments) - { - try { - byte[] dataToSend = jsonMapper.writeValueAsBytes( - new ClientAppendQuery(dataSource, segments) - ); - - client.post( - new URL( - String.format( - "http://%s:%s/mmx/merger/v1/merge", - serviceProvider.getInstance().getAddress(), - serviceProvider.getInstance().getPort() - ) - ) - ) - .setContent("application/json", dataToSend) - .go(responseHandler) - .get(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } -} diff --git a/server/src/main/java/com/metamx/druid/master/MergerClient.java b/server/src/main/java/com/metamx/druid/master/MergerWhitelist.java similarity index 57% rename from server/src/main/java/com/metamx/druid/master/MergerClient.java rename to server/src/main/java/com/metamx/druid/master/MergerWhitelist.java index c2556ccaea0..bd55a0cf057 100644 --- a/server/src/main/java/com/metamx/druid/master/MergerClient.java +++ b/server/src/main/java/com/metamx/druid/master/MergerWhitelist.java @@ -19,13 +19,36 @@ package com.metamx.druid.master; -import com.metamx.druid.client.DataSegment; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.collect.Sets; import java.util.List; +import java.util.Set; /** */ -public interface MergerClient +public class MergerWhitelist { - public void runRequest(String dataSource, List segments); + public static final String CONFIG_KEY = "merger.whitelist"; + + private final Set dataSources; + + @JsonCreator + public MergerWhitelist(Set dataSources) + { + this.dataSources = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + this.dataSources.addAll(dataSources); + } + + @JsonValue + public Set getDataSources() + { + return dataSources; + } + + public boolean contains(String val) + { + return dataSources.contains(val); + } } diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java index 8e19ed5d330..1ad1f96d163 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java @@ -23,12 +23,14 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.indexing.IndexingServiceClient; import junit.framework.Assert; import org.joda.time.Interval; import org.junit.Test; import java.util.Collection; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; public class DruidMasterSegmentMergerTest { @@ -367,11 +369,7 @@ public class DruidMasterSegmentMergerTest DataSegment.builder().dataSource("foo").interval(new Interval("2012-01-06/P1D")).version("2").size(80).build() ); - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of(segments.get(4), segments.get(5)) - ), merge(segments) - ); + Assert.assertEquals(ImmutableList.of(ImmutableList.of(segments.get(4), segments.get(5))), merge(segments)); } /** @@ -380,16 +378,17 @@ public class DruidMasterSegmentMergerTest private static List> merge(final Collection segments) { final List> retVal = Lists.newArrayList(); - final MergerClient mergerClient = new MergerClient() + final IndexingServiceClient indexingServiceClient = new IndexingServiceClient(null, null, null) { @Override - public void runRequest(String dataSource, List segmentsToMerge) + public void mergeSegments(List segmentsToMerge) { retVal.add(segmentsToMerge); } }; - final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(mergerClient); + final AtomicReference whitelistRef = new AtomicReference(null); + final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef); final DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder() .withAvailableSegments(ImmutableSet.copyOf(segments)) .withMergeBytesLimit(mergeBytesLimit) diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java index 895659586ec..584ae31de47 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -152,8 +152,6 @@ public class DruidMasterTest new NoopServiceEmitter(), scheduledExecutorFactory, loadManagementPeons, - null, - null, null ); } From ef4ccce30c2abea08c471c62adbf069e10137296 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 7 Mar 2013 17:43:37 -0600 Subject: [PATCH 12/17] 1) Stop polling in ConfigManager when stop is called 2) Remove WorkSetupManager in favor of just using ConfigManager --- .../metamx/druid/config/ConfigManager.java | 55 ++++++++++------- .../merger/coordinator/RemoteTaskRunner.java | 11 ++-- .../http/IndexerCoordinatorNode.java | 49 ++++++--------- .../http/IndexerCoordinatorResource.java | 23 +++++--- .../http/IndexerCoordinatorServletModule.java | 10 ++-- .../scaling/EC2AutoScalingStrategy.java | 10 ++-- .../SimpleResourceManagementStrategy.java | 12 ++-- .../coordinator/setup/WorkerSetupData.java | 2 + .../coordinator/setup/WorkerSetupManager.java | 59 ------------------- .../coordinator/RemoteTaskRunnerTest.java | 17 +----- .../scaling/EC2AutoScalingStrategyTest.java | 12 ++-- .../SimpleResourceManagementStrategyTest.java | 16 ++--- pom.xml | 2 +- 13 files changed, 106 insertions(+), 172 deletions(-) delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManager.java b/common/src/main/java/com/metamx/druid/config/ConfigManager.java index 3073c13e9fb..1ecfd24482c 100644 --- a/common/src/main/java/com/metamx/druid/config/ConfigManager.java +++ b/common/src/main/java/com/metamx/druid/config/ConfigManager.java @@ -1,6 +1,5 @@ package com.metamx.druid.config; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.metamx.common.concurrent.ScheduledExecutors; @@ -40,6 +39,8 @@ public class ConfigManager private final ConcurrentMap watchedConfigs; private final String selectStatement; + private volatile ConfigManager.PollingCallable poller; + public ConfigManager(IDBI dbi, ConfigManagerConfig config) { this.dbi = dbi; @@ -58,19 +59,8 @@ public class ConfigManager return; } - ScheduledExecutors.scheduleWithFixedDelay( - exec, - new Duration(0), - config.getPollDuration(), - new Runnable() - { - @Override - public void run() - { - poll(); - } - } - ); + poller = new PollingCallable(); + ScheduledExecutors.scheduleWithFixedDelay(exec, new Duration(0), config.getPollDuration(), poller); started = true; } @@ -84,6 +74,9 @@ public class ConfigManager return; } + poller.stop(); + poller = null; + started = false; } } @@ -119,8 +112,7 @@ public class ConfigManager { if (!started) { watchedConfigs.put(key, new ConfigHolder(null, serde)); - } - else { + } else { try { // Multiple of these callables can be submitted at the same time, but the callables themselves // are executed serially, so double check that it hasn't already been populated. @@ -200,10 +192,12 @@ public class ConfigManager @Override public Void withHandle(Handle handle) throws Exception { - handle.createStatement("INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload") - .bind("name", key) - .bind("payload", newBytes) - .execute(); + handle.createStatement( + "INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload" + ) + .bind("name", key) + .bind("payload", newBytes) + .execute(); return null; } } @@ -256,4 +250,25 @@ public class ConfigManager return false; } } + + private class PollingCallable implements Callable + { + private volatile boolean stop = false; + + void stop() + { + stop = true; + } + + @Override + public ScheduledExecutors.Signal call() throws Exception + { + if (stop) { + return ScheduledExecutors.Signal.STOP; + } + + poll(); + return ScheduledExecutors.Signal.REPEAT; + } + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index e4a13136152..b1ed92087bc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -35,7 +35,7 @@ import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; @@ -57,6 +57,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; /** * The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes and manage retries in failure @@ -82,7 +83,7 @@ public class RemoteTaskRunner implements TaskRunner private final PathChildrenCache workerPathCache; private final ScheduledExecutorService scheduledExec; private final RetryPolicyFactory retryPolicyFactory; - private final WorkerSetupManager workerSetupManager; + private final AtomicReference workerSetupData; // all workers that exist in ZK private final Map zkWorkers = new ConcurrentHashMap(); @@ -104,7 +105,7 @@ public class RemoteTaskRunner implements TaskRunner PathChildrenCache workerPathCache, ScheduledExecutorService scheduledExec, RetryPolicyFactory retryPolicyFactory, - WorkerSetupManager workerSetupManager + AtomicReference workerSetupData ) { this.jsonMapper = jsonMapper; @@ -113,7 +114,7 @@ public class RemoteTaskRunner implements TaskRunner this.workerPathCache = workerPathCache; this.scheduledExec = scheduledExec; this.retryPolicyFactory = retryPolicyFactory; - this.workerSetupManager = workerSetupManager; + this.workerSetupData = workerSetupData; } @LifecycleStart @@ -548,7 +549,7 @@ public class RemoteTaskRunner implements TaskRunner return (!input.isAtCapacity() && input.getWorker() .getVersion() - .compareTo(workerSetupManager.getWorkerSetupData().getMinVersion()) >= 0); + .compareTo(workerSetupData.get().getMinVersion()) >= 0); } } ) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 4c6d47dbad2..e717e3f9267 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -53,9 +53,9 @@ 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.loading.DataSegmentKiller; import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.TaskActionToolbox; @@ -88,7 +88,7 @@ import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerCo import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory; import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy; import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -122,6 +122,7 @@ import java.util.Properties; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -155,7 +156,6 @@ public class IndexerCoordinatorNode extends RegisteringNode private CuratorFramework curatorFramework = null; private ScheduledExecutorFactory scheduledExecutorFactory = null; private IndexerZkConfig indexerZkConfig; - private WorkerSetupManager workerSetupManager = null; private TaskRunnerFactory taskRunnerFactory = null; private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null; private TaskMasterLifecycle taskMasterLifecycle = null; @@ -226,12 +226,6 @@ public class IndexerCoordinatorNode extends RegisteringNode return this; } - public IndexerCoordinatorNode setWorkerSetupManager(WorkerSetupManager workerSetupManager) - { - this.workerSetupManager = workerSetupManager; - return this; - } - public IndexerCoordinatorNode setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory) { this.taskRunnerFactory = taskRunnerFactory; @@ -248,6 +242,10 @@ public class IndexerCoordinatorNode extends RegisteringNode { scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); + final ConfigManagerConfig managerConfig = configFactory.build(ConfigManagerConfig.class); + DbConnector.createConfigTable(dbi, managerConfig.getConfigTable()); + JacksonConfigManager configManager = new JacksonConfigManager(new ConfigManager(dbi, managerConfig), jsonMapper); + initializeEmitter(); initializeMonitors(); initializeDB(); @@ -264,9 +262,8 @@ public class IndexerCoordinatorNode extends RegisteringNode initializeJacksonSubtypes(); initializeCurator(); initializeIndexerZkConfig(); - initializeWorkerSetupManager(); - initializeTaskRunnerFactory(); - initializeResourceManagement(); + initializeTaskRunnerFactory(configManager); + initializeResourceManagement(configManager); initializeTaskMasterLifecycle(); initializeServer(); @@ -286,7 +283,7 @@ public class IndexerCoordinatorNode extends RegisteringNode emitter, taskMasterLifecycle, new TaskStorageQueryAdapter(taskStorage), - workerSetupManager + configManager ) ); @@ -555,20 +552,7 @@ public class IndexerCoordinatorNode extends RegisteringNode } } - public void initializeWorkerSetupManager() - { - if (workerSetupManager == null) { - final ConfigManagerConfig configManagerConfig = configFactory.build(ConfigManagerConfig.class); - final ConfigManager configManager = new ConfigManager(dbi, configManagerConfig); - lifecycle.addManagedInstance(configManager); - - DbConnector.createConfigTable(dbi, configManagerConfig.getConfigTable()); - workerSetupManager = new WorkerSetupManager(new JacksonConfigManager(configManager, jsonMapper)); - } - lifecycle.addManagedInstance(workerSetupManager); - } - - public void initializeTaskRunnerFactory() + private void initializeTaskRunnerFactory(final JacksonConfigManager configManager) { if (taskRunnerFactory == null) { if (config.getRunnerImpl().equals("remote")) { @@ -594,7 +578,7 @@ public class IndexerCoordinatorNode extends RegisteringNode new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true), retryScheduledExec, new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)), - workerSetupManager + configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class) ); return remoteTaskRunner; @@ -617,7 +601,7 @@ public class IndexerCoordinatorNode extends RegisteringNode } } - private void initializeResourceManagement() + private void initializeResourceManagement(final JacksonConfigManager configManager) { if (resourceManagementSchedulerFactory == null) { resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory() @@ -632,6 +616,9 @@ public class IndexerCoordinatorNode extends RegisteringNode .setNameFormat("ScalingExec--%d") .build() ); + final AtomicReference workerSetupData = configManager.watch( + WorkerSetupData.CONFIG_KEY, WorkerSetupData.class + ); AutoScalingStrategy strategy; if (config.getStrategyImpl().equalsIgnoreCase("ec2")) { @@ -644,7 +631,7 @@ public class IndexerCoordinatorNode extends RegisteringNode ) ), configFactory.build(EC2AutoScalingStrategyConfig.class), - workerSetupManager + workerSetupData ); } else if (config.getStrategyImpl().equalsIgnoreCase("noop")) { strategy = new NoopAutoScalingStrategy(); @@ -657,7 +644,7 @@ public class IndexerCoordinatorNode extends RegisteringNode new SimpleResourceManagementStrategy( strategy, configFactory.build(SimpleResourceManagmentConfig.class), - workerSetupManager + workerSetupData ), configFactory.build(ResourceManagementSchedulerConfig.class), scalingScheduledExec diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index 762d9ad28ee..fe5432cc713 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -19,12 +19,15 @@ 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.Optional; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; 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.TaskStatus; import com.metamx.druid.merger.common.actions.TaskActionHolder; import com.metamx.druid.merger.common.task.Task; @@ -32,10 +35,7 @@ import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.core.type.TypeReference; import javax.ws.rs.Consumes; import javax.ws.rs.GET; @@ -46,6 +46,7 @@ import javax.ws.rs.Produces; import javax.ws.rs.core.Response; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -58,16 +59,18 @@ public class IndexerCoordinatorResource private final ServiceEmitter emitter; private final TaskMasterLifecycle taskMasterLifecycle; private final TaskStorageQueryAdapter taskStorageQueryAdapter; - private final WorkerSetupManager workerSetupManager; + private final JacksonConfigManager configManager; private final ObjectMapper jsonMapper; + private AtomicReference workerSetupDataRef = null; + @Inject public IndexerCoordinatorResource( IndexerCoordinatorConfig config, ServiceEmitter emitter, TaskMasterLifecycle taskMasterLifecycle, TaskStorageQueryAdapter taskStorageQueryAdapter, - WorkerSetupManager workerSetupManager, + JacksonConfigManager configManager, ObjectMapper jsonMapper ) throws Exception { @@ -75,7 +78,7 @@ public class IndexerCoordinatorResource this.emitter = emitter; this.taskMasterLifecycle = taskMasterLifecycle; this.taskStorageQueryAdapter = taskStorageQueryAdapter; - this.workerSetupManager = workerSetupManager; + this.configManager = configManager; this.jsonMapper = jsonMapper; } @@ -152,7 +155,11 @@ public class IndexerCoordinatorResource @Produces("application/json") public Response getWorkerSetupData() { - return Response.ok(workerSetupManager.getWorkerSetupData()).build(); + if (workerSetupDataRef == null) { + workerSetupDataRef = configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class); + } + + return Response.ok(workerSetupDataRef.get()).build(); } @POST @@ -162,7 +169,7 @@ public class IndexerCoordinatorResource final WorkerSetupData workerSetupData ) { - if (!workerSetupManager.setWorkerSetupData(workerSetupData)) { + if (!configManager.set(WorkerSetupData.CONFIG_KEY, workerSetupData)) { return Response.status(Response.Status.BAD_REQUEST).build(); } return Response.ok().build(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java index f5bdfd6cbb5..1024d921668 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java @@ -22,10 +22,10 @@ package com.metamx.druid.merger.coordinator.http; 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.coordinator.TaskMasterLifecycle; import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; @@ -41,7 +41,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule private final ServiceEmitter emitter; private final TaskMasterLifecycle taskMasterLifecycle; private final TaskStorageQueryAdapter taskStorageQueryAdapter; - private final WorkerSetupManager workerSetupManager; + private final JacksonConfigManager configManager; public IndexerCoordinatorServletModule( ObjectMapper jsonMapper, @@ -49,7 +49,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule ServiceEmitter emitter, TaskMasterLifecycle taskMasterLifecycle, TaskStorageQueryAdapter taskStorageQueryAdapter, - WorkerSetupManager workerSetupManager + JacksonConfigManager configManager ) { this.jsonMapper = jsonMapper; @@ -57,7 +57,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule this.emitter = emitter; this.taskMasterLifecycle = taskMasterLifecycle; this.taskStorageQueryAdapter = taskStorageQueryAdapter; - this.workerSetupManager = workerSetupManager; + this.configManager = configManager; } @Override @@ -70,7 +70,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule bind(ServiceEmitter.class).toInstance(emitter); bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle); bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter); - bind(WorkerSetupManager.class).toInstance(workerSetupManager); + bind(JacksonConfigManager.class).toInstance(configManager); serve("/*").with(GuiceContainer.class); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index 47ef22152dd..f9ecbb06f70 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -34,13 +34,13 @@ import com.google.common.collect.Lists; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.setup.EC2NodeData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.EmittingLogger; import org.apache.commons.codec.binary.Base64; import javax.annotation.Nullable; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -51,26 +51,26 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy private final ObjectMapper jsonMapper; private final AmazonEC2Client amazonEC2Client; private final EC2AutoScalingStrategyConfig config; - private final WorkerSetupManager workerSetupManager; + private final AtomicReference workerSetupDataRef; public EC2AutoScalingStrategy( ObjectMapper jsonMapper, AmazonEC2Client amazonEC2Client, EC2AutoScalingStrategyConfig config, - WorkerSetupManager workerSetupManager + AtomicReference workerSetupDataRef ) { this.jsonMapper = jsonMapper; this.amazonEC2Client = amazonEC2Client; this.config = config; - this.workerSetupManager = workerSetupManager; + this.workerSetupDataRef = workerSetupDataRef; } @Override public AutoScalingData provision() { try { - WorkerSetupData setupData = workerSetupManager.getWorkerSetupData(); + WorkerSetupData setupData = workerSetupDataRef.get(); EC2NodeData workerConfig = setupData.getNodeData(); RunInstancesResult result = amazonEC2Client.runInstances( diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java index 05c2f2f0b26..082870c83c8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -27,16 +27,16 @@ import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; import com.metamx.druid.merger.coordinator.ZkWorker; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; import org.joda.time.Duration; import java.util.Collection; -import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -46,7 +46,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat private final AutoScalingStrategy autoScalingStrategy; private final SimpleResourceManagmentConfig config; - private final WorkerSetupManager workerSetupManager; + private final AtomicReference workerSetupdDataRef; private final ScalingStats scalingStats; private final ConcurrentSkipListSet currentlyProvisioning = new ConcurrentSkipListSet(); @@ -58,12 +58,12 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat public SimpleResourceManagementStrategy( AutoScalingStrategy autoScalingStrategy, SimpleResourceManagmentConfig config, - WorkerSetupManager workerSetupManager + AtomicReference workerSetupdDataRef ) { this.autoScalingStrategy = autoScalingStrategy; this.config = config; - this.workerSetupManager = workerSetupManager; + this.workerSetupdDataRef = workerSetupdDataRef; this.scalingStats = new ScalingStats(config.getNumEventsToTrack()); } @@ -151,7 +151,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat boolean nothingTerminating = currentlyTerminating.isEmpty(); if (nothingTerminating) { - final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers(); + final int minNumWorkers = workerSetupdDataRef.get().getMinNumWorkers(); if (zkWorkers.size() <= minNumWorkers) { log.info("Only [%d <= %d] nodes in the cluster, not terminating anything.", zkWorkers.size(), minNumWorkers); return false; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java index fada73cb40e..18cd85e6962 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java @@ -31,6 +31,8 @@ import java.util.List; */ public class WorkerSetupData { + public static final String CONFIG_KEY = "worker.setup"; + private final String minVersion; private final int minNumWorkers; private final EC2NodeData nodeData; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java deleted file mode 100644 index b9ce066c327..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.merger.coordinator.setup; - -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.druid.config.JacksonConfigManager; - -import java.util.concurrent.atomic.AtomicReference; - -/** - */ -public class WorkerSetupManager -{ - private static final String WORKER_SETUP_KEY = "worker.setup"; - - private final JacksonConfigManager configManager; - - private volatile AtomicReference workerSetupData = new AtomicReference(null); - - public WorkerSetupManager( - JacksonConfigManager configManager - ) - { - this.configManager = configManager; - } - - @LifecycleStart - public void start() - { - workerSetupData = configManager.watch(WORKER_SETUP_KEY, WorkerSetupData.class); - } - - public WorkerSetupData getWorkerSetupData() - { - return workerSetupData.get(); - } - - public boolean setWorkerSetupData(final WorkerSetupData value) - { - return configManager.set(WORKER_SETUP_KEY, value); - } -} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index 18ce14a4555..d88ac044aed 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -11,14 +11,12 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.merger.TestTask; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; @@ -43,6 +41,7 @@ import org.junit.Test; import java.io.File; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; import static junit.framework.Assert.fail; @@ -61,7 +60,6 @@ public class RemoteTaskRunnerTest private PathChildrenCache pathChildrenCache; private RemoteTaskRunner remoteTaskRunner; private TaskMonitor taskMonitor; - private WorkerSetupManager workerSetupManager; private ScheduledExecutorService scheduledExec; @@ -317,17 +315,6 @@ public class RemoteTaskRunnerTest private void makeRemoteTaskRunner() throws Exception { scheduledExec = EasyMock.createMock(ScheduledExecutorService.class); - workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); - - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn( - new WorkerSetupData( - "0", - 0, - null, - null - ) - ).atLeastOnce(); - EasyMock.replay(workerSetupManager); remoteTaskRunner = new RemoteTaskRunner( jsonMapper, @@ -336,7 +323,7 @@ public class RemoteTaskRunnerTest pathChildrenCache, scheduledExec, new RetryPolicyFactory(new TestRetryPolicyConfig()), - workerSetupManager + new AtomicReference(new WorkerSetupData("0", 0, null, null)) ); // Create a single worker and wait for things for be ready diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java index c3aa8378b07..cd569cb77e8 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -33,7 +33,6 @@ import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.setup.EC2NodeData; import com.metamx.druid.merger.coordinator.setup.GalaxyUserData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -42,6 +41,7 @@ import org.junit.Test; import java.util.Arrays; import java.util.Date; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -57,7 +57,7 @@ public class EC2AutoScalingStrategyTest private Reservation reservation; private Instance instance; private EC2AutoScalingStrategy strategy; - private WorkerSetupManager workerSetupManager; + private AtomicReference workerSetupData; @Before public void setUp() throws Exception @@ -66,7 +66,7 @@ public class EC2AutoScalingStrategyTest runInstancesResult = EasyMock.createMock(RunInstancesResult.class); describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class); reservation = EasyMock.createMock(Reservation.class); - workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); + workerSetupData = new AtomicReference(null); instance = new Instance() .withInstanceId(INSTANCE_ID) @@ -85,7 +85,7 @@ public class EC2AutoScalingStrategyTest return "8080"; } }, - workerSetupManager + workerSetupData ); } @@ -96,13 +96,12 @@ public class EC2AutoScalingStrategyTest EasyMock.verify(runInstancesResult); EasyMock.verify(describeInstancesResult); EasyMock.verify(reservation); - EasyMock.verify(workerSetupManager); } @Test public void testScale() { - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn( + workerSetupData.set( new WorkerSetupData( "0", 0, @@ -110,7 +109,6 @@ public class EC2AutoScalingStrategyTest new GalaxyUserData("env", "version", "type") ) ); - EasyMock.replay(workerSetupManager); EasyMock.expect(amazonEC2Client.runInstances(EasyMock.anyObject(RunInstancesRequest.class))).andReturn( runInstancesResult diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java index 742525d38a7..2052ae014bb 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -29,7 +29,6 @@ import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; import com.metamx.druid.merger.coordinator.ZkWorker; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.Worker; import junit.framework.Assert; import org.easymock.EasyMock; @@ -42,21 +41,22 @@ import org.junit.Test; import java.util.Arrays; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; /** */ public class SimpleResourceManagementStrategyTest { private AutoScalingStrategy autoScalingStrategy; - private WorkerSetupManager workerSetupManager; private Task testTask; private SimpleResourceManagementStrategy simpleResourceManagementStrategy; + private AtomicReference workerSetupData; @Before public void setUp() throws Exception { - workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); autoScalingStrategy = EasyMock.createMock(AutoScalingStrategy.class); + workerSetupData = new AtomicReference(null); testTask = new TestTask( "task1", @@ -105,7 +105,7 @@ public class SimpleResourceManagementStrategyTest return new Duration(0); } }, - workerSetupManager + workerSetupData ); } @@ -187,8 +187,7 @@ public class SimpleResourceManagementStrategyTest @Test public void testDoSuccessfulTerminate() throws Exception { - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); - EasyMock.replay(workerSetupManager); + workerSetupData.set(new WorkerSetupData("0", 0, null, null)); EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList()); @@ -212,15 +211,13 @@ public class SimpleResourceManagementStrategyTest simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE ); - EasyMock.verify(workerSetupManager); EasyMock.verify(autoScalingStrategy); } @Test public void testSomethingTerminating() throws Exception { - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); - EasyMock.replay(workerSetupManager); + workerSetupData.set(new WorkerSetupData("0", 0, null, null)); EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList("ip")).times(2); @@ -259,7 +256,6 @@ public class SimpleResourceManagementStrategyTest simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE ); - EasyMock.verify(workerSetupManager); EasyMock.verify(autoScalingStrategy); } diff --git a/pom.xml b/pom.xml index f61f80c1ff4..d430a91ae5d 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ UTF-8 - 0.20.1-SNAPSHOT + 0.21.0 From a11a34f87ee80cec05cc32be407aa38156040651 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Mon, 4 Mar 2013 13:09:42 -0600 Subject: [PATCH 13/17] 1) Initial commit: Converter Task --- .../com/metamx/druid/index/v1/IndexIO.java | 15 +++- .../merger/common/task/AbstractTask.java | 20 +++++ .../druid/merger/common/task/KillTask.java | 30 +++++-- .../metamx/druid/merger/common/task/Task.java | 4 +- .../merger/common/task/V8toV9UpgradeTask.java | 36 --------- .../common/task/VersionConverterSubTask.java | 79 +++++++++++++++++++ .../common/task/VersionConverterTask.java | 74 +++++++++++++++++ 7 files changed, 212 insertions(+), 46 deletions(-) delete mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index d26e73f5b3c..621989b0d08 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -199,6 +199,19 @@ public class IndexIO } } + public static boolean convertSegment(File toConvert, File converted) throws IOException + { + final int version = getVersionFromDir(toConvert); + + switch (version) { + case 8: + DefaultIndexIOHandler.convertV8toV9(toConvert, converted); + return true; + default: + return false; + } + } + public static interface IndexIOHandler { /** @@ -229,7 +242,7 @@ public class IndexIO public void storeLatest(Index index, File file) throws IOException; } - static class DefaultIndexIOHandler implements IndexIOHandler + public static class DefaultIndexIOHandler implements IndexIOHandler { private static final Logger log = new Logger(DefaultIndexIOHandler.class); @Override diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index 899e2066fb0..119eab3ec87 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -20,15 +20,19 @@ package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Joiner; import com.google.common.base.Objects; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.SegmentListUsedAction; import org.joda.time.Interval; public abstract class AbstractTask implements Task { + private static final Joiner ID_JOINER = Joiner.on("_"); + private final String id; private final String groupId; private final String dataSource; @@ -91,4 +95,20 @@ public abstract class AbstractTask implements Task .add("interval", getImplicitLockInterval()) .toString(); } + + /** Start helper methods **/ + public static String joinId(Object... objects) + { + return ID_JOINER.join(objects); + } + + public SegmentListUsedAction makeListUsedAction() + { + return new SegmentListUsedAction(this, getDataSource(), getFixedInterval().get()); + } + + public TaskStatus success() + { + return TaskStatus.success(getId()); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index 8c2b5af8e6e..bf1bbbabd90 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -1,3 +1,22 @@ +/* + * 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.task; import com.fasterxml.jackson.annotation.JsonCreator; @@ -64,14 +83,9 @@ public class KillTask extends AbstractTask } // List unused segments - final List unusedSegments = toolbox.getTaskActionClient() - .submit( - new SegmentListUnusedAction( - this, - myLock.getDataSource(), - myLock.getInterval() - ) - ); + final List unusedSegments = toolbox + .getTaskActionClient() + .submit(new SegmentListUnusedAction(this, myLock.getDataSource(), myLock.getInterval())); // Verify none of these segments have versions > lock version for(final DataSegment unusedSegment : unusedSegments) { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 8d93486ff6b..b1a4598a2d2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -50,7 +50,9 @@ import org.joda.time.Interval; @JsonSubTypes.Type(name = "index", value = IndexTask.class), @JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class), @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class), - @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class) + @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), + @JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class), + @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterSubTask.class), }) public interface Task { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java deleted file mode 100644 index 28d3ab1fec1..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java +++ /dev/null @@ -1,36 +0,0 @@ -package com.metamx.druid.merger.common.task; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -/** - */ -public class V8toV9UpgradeTask extends AbstractTask -{ - public V8toV9UpgradeTask( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval - ) - { - super( - String.format("v8tov9_%s_%s_%s", dataSource, interval.toString().replace("/", "_"), new DateTime()), - dataSource, - interval - ); - } - - @Override - public String getType() - { - return "8to9"; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception - { - throw new UnsupportedOperationException(); - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java new file mode 100644 index 00000000000..4b5c3d251f0 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java @@ -0,0 +1,79 @@ +/* + * 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.task; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Sets; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; + +import java.io.File; +import java.util.Arrays; +import java.util.Map; + +/** + */ +public class VersionConverterSubTask extends AbstractTask +{ + private final DataSegment segment; + + protected VersionConverterSubTask( + @JsonProperty("groupId") String groupId, + @JsonProperty("segment") DataSegment segment + ) + { + super( + joinId( + groupId, + "sub", + segment.getInterval().getStart(), + segment.getInterval().getEnd(), + segment.getShardSpec().getPartitionNum() + ), + segment.getDataSource(), + segment.getInterval() + ); + this.segment = segment; + } + + @Override + public String getType() + { + return "version_converter_sub"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final Map localSegments = toolbox.getSegments(this, Arrays.asList(segment)); + + final File location = localSegments.get(segment); + final File outLocation = new File(location, "v9_out"); + if (IndexIO.convertSegment(location, outLocation)) { + final DataSegment updatedSegment = toolbox.getSegmentPusher().push(outLocation, segment); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, Sets.newHashSet(updatedSegment))); + } + + return success(); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java new file mode 100644 index 00000000000..062f3751767 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java @@ -0,0 +1,74 @@ +package com.metamx.druid.merger.common.task; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.metamx.common.ISE; +import com.metamx.druid.client.DataSegment; +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 org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; + +/** + */ +public class VersionConverterTask extends AbstractTask +{ + private static final String TYPE = "version_converter"; + + public VersionConverterTask( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval + ) + { + super( + joinId(TYPE, dataSource, interval.getStart(), interval.getEnd(), new DateTime()), + dataSource, + interval + ); + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + throw new ISE("Should never run, %s just exists to create subtasks", this.getClass().getSimpleName()); + } + + @Override + public TaskStatus preflight(TaskToolbox toolbox) throws Exception + { + final TaskActionClient taskClient = toolbox.getTaskActionClient(); + + List segments = taskClient.submit(makeListUsedAction()); + + taskClient.submit( + new SpawnTasksAction( + this, + Lists.transform( + segments, + new Function() + { + @Override + public Task apply(@Nullable DataSegment input) + { + return new VersionConverterSubTask(getGroupId(), input); + } + } + ) + ) + ); + + return TaskStatus.success(getId()); + } +} From c9b411c0ca32dd1e53f1d27ea503e40b1037fa9d Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Mon, 4 Mar 2013 18:10:32 -0600 Subject: [PATCH 14/17] 1) Remove the need for TaskActions to require a Task as a constructor parameter --- .../merger/common/TaskToolboxFactory.java | 92 +++++++++++++++++++ .../common/actions/LocalTaskActionClient.java | 11 ++- .../actions/LocalTaskActionClientFactory.java | 43 +++++++++ .../common/actions/LockAcquireAction.java | 11 +-- .../merger/common/actions/LockListAction.java | 18 +--- .../common/actions/LockReleaseAction.java | 11 +-- .../actions/RemoteTaskActionClient.java | 7 +- .../RemoteTaskActionClientFactory.java | 47 ++++++++++ .../common/actions/SegmentInsertAction.java | 11 +-- .../actions/SegmentListUnusedAction.java | 11 +-- .../common/actions/SegmentListUsedAction.java | 11 +-- .../common/actions/SegmentNukeAction.java | 11 +-- .../common/actions/SpawnTasksAction.java | 11 +-- .../merger/common/actions/TaskAction.java | 3 +- .../actions/TaskActionClientFactory.java | 29 ++++++ .../common/actions/TaskActionHolder.java | 54 +++++++++++ .../merger/common/task/AbstractTask.java | 4 +- .../druid/merger/common/task/DeleteTask.java | 4 +- .../merger/common/task/HadoopIndexTask.java | 23 ++++- .../task/IndexDeterminePartitionsTask.java | 2 +- .../common/task/IndexGeneratorTask.java | 4 +- .../druid/merger/common/task/IndexTask.java | 2 +- .../druid/merger/common/task/KillTask.java | 6 +- .../druid/merger/common/task/MergeTask.java | 6 +- .../metamx/druid/merger/common/task/Task.java | 2 +- .../common/task/VersionConverterSubTask.java | 2 +- .../common/task/VersionConverterTask.java | 3 +- .../merger/coordinator/DbTaskStorage.java | 6 +- ...torage.java => HeapMemoryTaskStorage.java} | 8 +- .../merger/coordinator/LocalTaskRunner.java | 9 +- .../druid/merger/coordinator/TaskLockbox.java | 19 ++++ .../coordinator/TaskMasterLifecycle.java | 14 +-- .../druid/merger/coordinator/TaskStorage.java | 2 +- .../merger/coordinator/exec/TaskConsumer.java | 10 +- .../http/IndexerCoordinatorNode.java | 51 +++++----- .../http/IndexerCoordinatorResource.java | 8 +- .../druid/merger/worker/TaskMonitor.java | 10 +- .../druid/merger/worker/http/WorkerNode.java | 45 +++++---- .../coordinator/RemoteTaskRunnerTest.java | 3 +- .../merger/coordinator/TaskLifecycleTest.java | 86 ++++++++--------- .../merger/coordinator/TaskQueueTest.java | 35 +++---- pom.xml | 10 +- .../metamx/druid/realtime/RealtimeNode.java | 38 +------- .../druid/initialization/ServerInit.java | 39 ++++++++ 44 files changed, 549 insertions(+), 283 deletions(-) create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java rename merger/src/main/java/com/metamx/druid/merger/coordinator/{LocalTaskStorage.java => HeapMemoryTaskStorage.java} (95%) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java new file mode 100644 index 00000000000..9a804bef79e --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java @@ -0,0 +1,92 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.merger.common; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Maps; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.MMappedQueryableIndexFactory; +import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.SegmentLoaderConfig; +import com.metamx.druid.loading.SegmentLoadingException; +import com.metamx.druid.loading.SingleSegmentLoader; +import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.merger.common.actions.TaskActionClientFactory; +import com.metamx.druid.merger.common.config.TaskConfig; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.emitter.service.ServiceEmitter; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; + +import java.io.File; +import java.util.List; +import java.util.Map; + +/** + * Stuff that may be needed by a Task in order to conduct its business. + */ +public class TaskToolboxFactory +{ + private final TaskConfig config; + private final TaskActionClientFactory taskActionClient; + private final ServiceEmitter emitter; + private final RestS3Service s3Client; + private final DataSegmentPusher segmentPusher; + private final SegmentKiller segmentKiller; + private final ObjectMapper objectMapper; + + public TaskToolboxFactory( + TaskConfig config, + TaskActionClientFactory taskActionClient, + ServiceEmitter emitter, + RestS3Service s3Client, + DataSegmentPusher segmentPusher, + SegmentKiller segmentKiller, + ObjectMapper objectMapper + ) + { + this.config = config; + this.taskActionClient = taskActionClient; + this.emitter = emitter; + this.s3Client = s3Client; + this.segmentPusher = segmentPusher; + this.segmentKiller = segmentKiller; + this.objectMapper = objectMapper; + } + + public ObjectMapper getObjectMapper() + { + return objectMapper; + } + + public TaskToolbox build(Task task) + { + return new TaskToolbox( + config, + taskActionClient == null ? null : taskActionClient.create(task), + emitter, + s3Client, + segmentPusher, + segmentKiller, + objectMapper + ); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java index f6740064f52..e36dbf65a6c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java @@ -1,17 +1,20 @@ package com.metamx.druid.merger.common.actions; +import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskStorage; import com.metamx.emitter.EmittingLogger; public class LocalTaskActionClient implements TaskActionClient { + private final Task task; private final TaskStorage storage; private final TaskActionToolbox toolbox; private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class); - public LocalTaskActionClient(TaskStorage storage, TaskActionToolbox toolbox) + public LocalTaskActionClient(Task task, TaskStorage storage, TaskActionToolbox toolbox) { + this.task = task; this.storage = storage; this.toolbox = toolbox; } @@ -19,15 +22,15 @@ public class LocalTaskActionClient implements TaskActionClient @Override public RetType submit(TaskAction taskAction) { - final RetType ret = taskAction.perform(toolbox); + final RetType ret = taskAction.perform(task, toolbox); // Add audit log try { - storage.addAuditLog(taskAction); + storage.addAuditLog(task, taskAction); } catch (Exception e) { log.makeAlert(e, "Failed to record action in audit log") - .addData("task", taskAction.getTask().getId()) + .addData("task", task.getId()) .addData("actionClass", taskAction.getClass().getName()) .emit(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java new file mode 100644 index 00000000000..89e37ceafab --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java @@ -0,0 +1,43 @@ +/* + * 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.actions; + +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.TaskStorage; + +/** + */ +public class LocalTaskActionClientFactory implements TaskActionClientFactory +{ + private final TaskStorage storage; + private final TaskActionToolbox toolbox; + + public LocalTaskActionClientFactory(TaskStorage storage, TaskActionToolbox toolbox) + { + this.storage = storage; + this.toolbox = toolbox; + } + + @Override + public TaskActionClient create(Task task) + { + return new LocalTaskActionClient(task, storage, toolbox); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java index f669af33625..de325ba274f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java @@ -11,25 +11,16 @@ import org.joda.time.Interval; public class LockAcquireAction implements TaskAction> { - private final Task task; private final Interval interval; @JsonCreator public LockAcquireAction( - @JsonProperty("task") Task task, @JsonProperty("interval") Interval interval ) { - this.task = task; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Interval getInterval() { @@ -42,7 +33,7 @@ public class LockAcquireAction implements TaskAction> } @Override - public Optional perform(TaskActionToolbox toolbox) + public Optional perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getTaskLockbox().tryLock(task, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java index e0e3eddb71f..06a2879ec47 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java @@ -12,29 +12,13 @@ import java.util.List; public class LockListAction implements TaskAction> { - private final Task task; - - @JsonCreator - public LockListAction( - @JsonProperty("task") Task task - ) - { - this.task = task; - } - - @JsonProperty - public Task getTask() - { - return task; - } - public TypeReference> getReturnTypeReference() { return new TypeReference>() {}; } @Override - public List perform(TaskActionToolbox toolbox) + public List perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getTaskLockbox().findLocksForTask(task); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java index 5c84d024a50..b932e748ed1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java @@ -12,25 +12,16 @@ import java.util.List; public class LockReleaseAction implements TaskAction { - private final Task task; private final Interval interval; @JsonCreator public LockReleaseAction( - @JsonProperty("task") Task task, @JsonProperty("interval") Interval interval ) { - this.task = task; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Interval getInterval() { @@ -43,7 +34,7 @@ public class LockReleaseAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { try { toolbox.getTaskLockbox().unlock(task, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java index 26900e29942..5cebc6ee1ec 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java @@ -4,6 +4,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; +import com.metamx.druid.merger.common.task.Task; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.ToStringResponseHandler; import com.fasterxml.jackson.databind.ObjectMapper; @@ -16,14 +17,16 @@ import java.util.Map; public class RemoteTaskActionClient implements TaskActionClient { + private final Task task; private final HttpClient httpClient; private final ServiceProvider serviceProvider; private final ObjectMapper jsonMapper; private static final Logger log = new Logger(RemoteTaskActionClient.class); - public RemoteTaskActionClient(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) + public RemoteTaskActionClient(Task task, HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) { + this.task = task; this.httpClient = httpClient; this.serviceProvider = serviceProvider; this.jsonMapper = jsonMapper; @@ -33,7 +36,7 @@ public class RemoteTaskActionClient implements TaskActionClient public RetType submit(TaskAction taskAction) { try { - byte[] dataToSend = jsonMapper.writeValueAsBytes(taskAction); + byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction)); final String response = httpClient.post(getServiceUri().toURL()) .setContent("application/json", dataToSend) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java new file mode 100644 index 00000000000..659042bb592 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java @@ -0,0 +1,47 @@ +/* + * 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.actions; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.http.client.HttpClient; +import com.netflix.curator.x.discovery.ServiceProvider; + +/** + */ +public class RemoteTaskActionClientFactory implements TaskActionClientFactory +{ + private final HttpClient httpClient; + private final ServiceProvider serviceProvider; + private final ObjectMapper jsonMapper; + + public RemoteTaskActionClientFactory(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) + { + this.httpClient = httpClient; + this.serviceProvider = serviceProvider; + this.jsonMapper = jsonMapper; + } + + @Override + public TaskActionClient create(Task task) + { + return new RemoteTaskActionClient(task, httpClient, serviceProvider, jsonMapper); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java index 75ad4a9161f..5354e14878c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java @@ -18,25 +18,16 @@ import java.util.Set; public class SegmentInsertAction implements TaskAction { - private final Task task; private final Set segments; @JsonCreator public SegmentInsertAction( - @JsonProperty("task") Task task, @JsonProperty("segments") Set segments ) { - this.task = task; this.segments = ImmutableSet.copyOf(segments); } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Set getSegments() { @@ -49,7 +40,7 @@ public class SegmentInsertAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { if(!toolbox.taskLockCoversSegments(task, segments, false)) { throw new ISE("Segments not covered by locks for task: %s", task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java index b20d130064e..56304533a68 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java @@ -12,28 +12,19 @@ import java.util.List; public class SegmentListUnusedAction implements TaskAction> { - private final Task task; private final String dataSource; private final Interval interval; @JsonCreator public SegmentListUnusedAction( - @JsonProperty("task") Task task, @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval ) { - this.task = task; this.dataSource = dataSource; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public String getDataSource() { @@ -52,7 +43,7 @@ public class SegmentListUnusedAction implements TaskAction> } @Override - public List perform(TaskActionToolbox toolbox) + public List perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java index 0395057fe83..a776ed641cc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java @@ -12,28 +12,19 @@ import java.util.List; public class SegmentListUsedAction implements TaskAction> { - private final Task task; private final String dataSource; private final Interval interval; @JsonCreator public SegmentListUsedAction( - @JsonProperty("task") Task task, @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval ) { - this.task = task; this.dataSource = dataSource; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public String getDataSource() { @@ -52,7 +43,7 @@ public class SegmentListUsedAction implements TaskAction> } @Override - public List perform(TaskActionToolbox toolbox) + public List perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java index f1b61c58d9f..2ebedec0daf 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java @@ -18,25 +18,16 @@ import java.util.Set; public class SegmentNukeAction implements TaskAction { - private final Task task; private final Set segments; @JsonCreator public SegmentNukeAction( - @JsonProperty("task") Task task, @JsonProperty("segments") Set segments ) { - this.task = task; this.segments = ImmutableSet.copyOf(segments); } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Set getSegments() { @@ -49,7 +40,7 @@ public class SegmentNukeAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { if(!toolbox.taskLockCoversSegments(task, segments, true)) { throw new ISE("Segments not covered by locks for task: %s", task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java index a7a73d8eac7..ec48430c49a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java @@ -11,25 +11,16 @@ import java.util.List; public class SpawnTasksAction implements TaskAction { - private final Task task; private final List newTasks; @JsonCreator public SpawnTasksAction( - @JsonProperty("task") Task task, @JsonProperty("newTasks") List newTasks ) { - this.task = task; this.newTasks = ImmutableList.copyOf(newTasks); } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public List getNewTasks() { @@ -42,7 +33,7 @@ public class SpawnTasksAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { try { for(final Task newTask : newTasks) { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java index 7dedf50aad9..019b14a3b62 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java @@ -18,7 +18,6 @@ import com.fasterxml.jackson.core.type.TypeReference; }) public interface TaskAction { - public Task getTask(); // TODO Look into replacing this with task ID so stuff serializes smaller public TypeReference getReturnTypeReference(); // T_T - public RetType perform(TaskActionToolbox toolbox); + public RetType perform(Task task, TaskActionToolbox toolbox); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java new file mode 100644 index 00000000000..2784a442f31 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java @@ -0,0 +1,29 @@ +/* + * 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.actions; + +import com.metamx.druid.merger.common.task.Task; + +/** + */ +public interface TaskActionClientFactory +{ + public TaskActionClient create(Task task); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java new file mode 100644 index 00000000000..a440447a226 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java @@ -0,0 +1,54 @@ +/* + * 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.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.druid.merger.common.task.Task; + +/** + */ +public class TaskActionHolder +{ + private final Task task; + private final TaskAction action; + + @JsonCreator + public TaskActionHolder( + @JsonProperty("task") Task task, + @JsonProperty("action") TaskAction action + ) + { + this.task = task; + this.action = action; + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public TaskAction getAction() + { + return action; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index 119eab3ec87..c2f9a792bb0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -102,9 +102,9 @@ public abstract class AbstractTask implements Task return ID_JOINER.join(objects); } - public SegmentListUsedAction makeListUsedAction() + public SegmentListUsedAction makeImplicitListUsedAction() { - return new SegmentListUsedAction(this, getDataSource(), getFixedInterval().get()); + return new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get()); } public TaskStatus success() diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 991d76228dd..2f10861ef5f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -77,7 +77,7 @@ public class DeleteTask extends AbstractTask public TaskStatus run(TaskToolbox toolbox) throws Exception { // Strategy: Create an empty segment covering the interval to be deleted - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); final Interval interval = this.getImplicitLockInterval().get(); final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty); @@ -104,7 +104,7 @@ public class DeleteTask extends AbstractTask segment.getVersion() ); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java index 29c0c517f17..6e284557529 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java @@ -1,3 +1,22 @@ +/* + * 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.task; import com.fasterxml.jackson.annotation.JsonCreator; @@ -74,7 +93,7 @@ public class HadoopIndexTask extends AbstractTask ); // We should have a lock from before we started running - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); log.info("Setting version to: %s", myLock.getVersion()); configCopy.setVersion(myLock.getVersion()); @@ -105,7 +124,7 @@ public class HadoopIndexTask extends AbstractTask List publishedSegments = job.getPublishedSegments(); // Request segment pushes - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(publishedSegments))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments))); // Done return TaskStatus.success(getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 2886c8cc77a..47f72b12501 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -258,7 +258,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask } ); - toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks)); + toolbox.getTaskActionClient().submit(new SpawnTasksAction(nextTasks)); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index 7dc65797ea8..2a7b4683b8d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -100,7 +100,7 @@ public class IndexGeneratorTask extends AbstractTask public TaskStatus run(final TaskToolbox toolbox) throws Exception { // We should have a lock from before we started running - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); // We know this exists final Interval interval = getImplicitLockInterval().get(); @@ -193,7 +193,7 @@ public class IndexGeneratorTask extends AbstractTask ); // Request segment pushes - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(pushedSegments))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(pushedSegments))); // Done return TaskStatus.success(getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 4e2f34a5171..35babcd6a22 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -142,7 +142,7 @@ public class IndexTask extends AbstractTask @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { - toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, toSubtasks())); + toolbox.getTaskActionClient().submit(new SpawnTasksAction(toSubtasks())); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index bf1bbbabd90..52b8b48b8b4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -72,7 +72,7 @@ public class KillTask extends AbstractTask public TaskStatus run(TaskToolbox toolbox) throws Exception { // Confirm we have a lock (will throw if there isn't exactly one element) - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); if(!myLock.getDataSource().equals(getDataSource())) { throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); @@ -85,7 +85,7 @@ public class KillTask extends AbstractTask // List unused segments final List unusedSegments = toolbox .getTaskActionClient() - .submit(new SegmentListUnusedAction(this, myLock.getDataSource(), myLock.getInterval())); + .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval())); // Verify none of these segments have versions > lock version for(final DataSegment unusedSegment : unusedSegments) { @@ -105,7 +105,7 @@ public class KillTask extends AbstractTask toolbox.getSegmentKiller().kill(unusedSegments); // Remove metadata for these segments - toolbox.getTaskActionClient().submit(new SegmentNukeAction(this, ImmutableSet.copyOf(unusedSegments))); + toolbox.getTaskActionClient().submit(new SegmentNukeAction(ImmutableSet.copyOf(unusedSegments))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 1791c2a097d..757a91c2598 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -119,7 +119,7 @@ public abstract class MergeTask extends AbstractTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); final ServiceEmitter emitter = toolbox.getEmitter(); final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments); @@ -170,7 +170,7 @@ public abstract class MergeTask extends AbstractTask emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); return TaskStatus.success(getId()); } @@ -215,7 +215,7 @@ public abstract class MergeTask extends AbstractTask final Set current = ImmutableSet.copyOf( Iterables.transform( toolbox.getTaskActionClient() - .submit(new SegmentListUsedAction(this, getDataSource(), getImplicitLockInterval().get())), + .submit(new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get())), toIdentifier ) ); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index b1a4598a2d2..8418ecf40a8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -52,7 +52,7 @@ import org.joda.time.Interval; @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class), - @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterSubTask.class), + @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterSubTask.class) }) public interface Task { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java index 4b5c3d251f0..db08e788e4d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java @@ -71,7 +71,7 @@ public class VersionConverterSubTask extends AbstractTask final File outLocation = new File(location, "v9_out"); if (IndexIO.convertSegment(location, outLocation)) { final DataSegment updatedSegment = toolbox.getSegmentPusher().push(outLocation, segment); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, Sets.newHashSet(updatedSegment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); } return success(); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java index 062f3751767..4106e210547 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java @@ -50,11 +50,10 @@ public class VersionConverterTask extends AbstractTask { final TaskActionClient taskClient = toolbox.getTaskActionClient(); - List segments = taskClient.submit(makeListUsedAction()); + List segments = taskClient.submit(makeImplicitListUsedAction()); taskClient.submit( new SpawnTasksAction( - this, Lists.transform( segments, new Function() diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java index 5f39efae0bb..b878885dd4a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java @@ -327,11 +327,11 @@ public class DbTaskStorage implements TaskStorage } @Override - public void addAuditLog(final TaskAction taskAction) + public void addAuditLog(final Task task, final TaskAction taskAction) { Preconditions.checkNotNull(taskAction, "taskAction"); - log.info("Logging action for task[%s]: %s", taskAction.getTask().getId(), taskAction); + log.info("Logging action for task[%s]: %s", task.getId(), taskAction); dbi.withHandle( new HandleCallback() @@ -345,7 +345,7 @@ public class DbTaskStorage implements TaskStorage dbConnectorConfig.getTaskLogTable() ) ) - .bind("task_id", taskAction.getTask().getId()) + .bind("task_id", task.getId()) .bind("log_payload", jsonMapper.writeValueAsString(taskAction)) .execute(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java similarity index 95% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java index 042f0b8196f..895804bc7fd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java @@ -40,14 +40,14 @@ import java.util.concurrent.locks.ReentrantLock; * Implements an in-heap TaskStorage facility, with no persistence across restarts. This class is not * thread safe. */ -public class LocalTaskStorage implements TaskStorage +public class HeapMemoryTaskStorage implements TaskStorage { private final ReentrantLock giant = new ReentrantLock(); private final Map tasks = Maps.newHashMap(); private final Multimap taskLocks = HashMultimap.create(); private final Multimap taskActions = ArrayListMultimap.create(); - private static final Logger log = new Logger(LocalTaskStorage.class); + private static final Logger log = new Logger(HeapMemoryTaskStorage.class); @Override public void insert(Task task, TaskStatus status) @@ -185,12 +185,12 @@ public class LocalTaskStorage implements TaskStorage } @Override - public void addAuditLog(TaskAction taskAction) + public void addAuditLog(Task task, TaskAction taskAction) { giant.lock(); try { - taskActions.put(taskAction.getTask().getId(), taskAction); + taskActions.put(task.getId(), taskAction); } finally { giant.unlock(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index 573152fce09..a4f58a361f8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -26,6 +26,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.task.Task; import org.apache.commons.io.FileUtils; @@ -38,17 +39,17 @@ import java.util.concurrent.ExecutorService; */ public class LocalTaskRunner implements TaskRunner { - private final TaskToolbox toolbox; + private final TaskToolboxFactory toolboxFactory; private final ExecutorService exec; private static final Logger log = new Logger(LocalTaskRunner.class); public LocalTaskRunner( - TaskToolbox toolbox, + TaskToolboxFactory toolboxFactory, ExecutorService exec ) { - this.toolbox = toolbox; + this.toolboxFactory = toolboxFactory; this.exec = exec; } @@ -61,6 +62,8 @@ public class LocalTaskRunner implements TaskRunner @Override public void run(final Task task, final TaskCallback callback) { + final TaskToolbox toolbox = toolboxFactory.build(task); + exec.submit( new Runnable() { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java index 07abd0594ac..0a4bd925d4d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java @@ -1,3 +1,22 @@ +/* + * 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.google.common.base.Function; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java index 0a38e5efa37..da73ede5923 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java @@ -26,6 +26,8 @@ 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.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.exec.TaskConsumer; import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler; @@ -49,7 +51,7 @@ public class TaskMasterLifecycle private final ReentrantLock giant = new ReentrantLock(); private final Condition mayBeStopped = giant.newCondition(); private final TaskQueue taskQueue; - private final TaskToolbox taskToolbox; + private final TaskToolboxFactory taskToolboxFactory; private volatile boolean leading = false; private volatile TaskRunner taskRunner; @@ -59,7 +61,7 @@ public class TaskMasterLifecycle public TaskMasterLifecycle( final TaskQueue taskQueue, - final TaskToolbox taskToolbox, + final TaskToolboxFactory taskToolboxFactory, final IndexerCoordinatorConfig indexerCoordinatorConfig, final ServiceDiscoveryConfig serviceDiscoveryConfig, final TaskRunnerFactory runnerFactory, @@ -69,7 +71,7 @@ public class TaskMasterLifecycle ) { this.taskQueue = taskQueue; - this.taskToolbox = taskToolbox; + this.taskToolboxFactory = taskToolboxFactory; this.leaderSelector = new LeaderSelector( curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener() @@ -87,7 +89,7 @@ public class TaskMasterLifecycle final TaskConsumer taskConsumer = new TaskConsumer( taskQueue, taskRunner, - taskToolbox, + taskToolboxFactory, emitter ); @@ -217,9 +219,9 @@ public class TaskMasterLifecycle return taskQueue; } - public TaskToolbox getTaskToolbox() + public TaskToolbox getTaskToolbox(Task task) { - return taskToolbox; + return taskToolboxFactory.build(task); } public ResourceManagementScheduler getResourceManagementScheduler() diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java index 2e21f52876b..d6bfbfd889e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java @@ -69,7 +69,7 @@ public interface TaskStorage /** * Add an action taken by a task to the audit log. */ - public void addAuditLog(TaskAction taskAction); + public void addAuditLog(Task task, TaskAction taskAction); /** * Returns all actions taken by a task. diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index f1e31a9e5c1..48d3ecdc471 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -24,7 +24,7 @@ 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.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskRunner; @@ -36,7 +36,7 @@ public class TaskConsumer implements Runnable { private final TaskQueue queue; private final TaskRunner runner; - private final TaskToolbox toolbox; + private final TaskToolboxFactory toolboxFactory; private final ServiceEmitter emitter; private final Thread thready; @@ -47,13 +47,13 @@ public class TaskConsumer implements Runnable public TaskConsumer( TaskQueue queue, TaskRunner runner, - TaskToolbox toolbox, + TaskToolboxFactory toolboxFactory, ServiceEmitter emitter ) { this.queue = queue; this.runner = runner; - this.toolbox = toolbox; + this.toolboxFactory = toolboxFactory; this.emitter = emitter; this.thready = new Thread(this); } @@ -123,7 +123,7 @@ public class TaskConsumer implements Runnable // Run preflight checks TaskStatus preflightStatus; try { - preflightStatus = task.preflight(toolbox); + preflightStatus = task.preflight(toolboxFactory.build(task)); log.info("Preflight done for task: %s", task.getId()); } catch (Exception e) { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 5da9c557936..833c3976ee8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -48,22 +48,21 @@ 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.DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.loading.S3SegmentKiller; import com.metamx.druid.loading.SegmentKiller; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.TaskActionToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; import com.metamx.druid.merger.coordinator.DbTaskStorage; +import com.metamx.druid.merger.coordinator.HeapMemoryTaskStorage; import com.metamx.druid.merger.coordinator.LocalTaskRunner; -import com.metamx.druid.merger.coordinator.LocalTaskStorage; import com.metamx.druid.merger.coordinator.MergerDBCoordinator; import com.metamx.druid.merger.coordinator.RemoteTaskRunner; import com.metamx.druid.merger.coordinator.RetryPolicyFactory; @@ -147,7 +146,8 @@ public class IndexerCoordinatorNode extends RegisteringNode private RestS3Service s3Service = null; private IndexerCoordinatorConfig config = null; private TaskConfig taskConfig = null; - private TaskToolbox taskToolbox = null; + private DataSegmentPusher segmentPusher = null; + private TaskToolboxFactory taskToolboxFactory = null; private MergerDBCoordinator mergerDBCoordinator = null; private TaskStorage taskStorage = null; private TaskQueue taskQueue = null; @@ -208,6 +208,12 @@ public class IndexerCoordinatorNode extends RegisteringNode return this; } + public IndexerCoordinatorNode setSegmentPusher(DataSegmentPusher segmentPusher) + { + this.segmentPusher = segmentPusher; + return this; + } + public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator) { this.mergerDBCoordinator = mergeDbCoordinator; @@ -252,6 +258,7 @@ public class IndexerCoordinatorNode extends RegisteringNode initializeTaskStorage(); initializeTaskLockbox(); initializeTaskQueue(); + initializeDataSegmentPusher(); initializeTaskToolbox(); initializeJacksonInjections(); initializeJacksonSubtypes(); @@ -339,7 +346,7 @@ public class IndexerCoordinatorNode extends RegisteringNode final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); taskMasterLifecycle = new TaskMasterLifecycle( taskQueue, - taskToolbox, + taskToolboxFactory, config, serviceDiscoveryConfig, taskRunnerFactory, @@ -403,7 +410,7 @@ public class IndexerCoordinatorNode extends RegisteringNode InjectableValues.Std injectables = new InjectableValues.Std(); injectables.addValue("s3Client", s3Service) - .addValue("segmentPusher", taskToolbox.getSegmentPusher()); + .addValue("segmentPusher", segmentPusher); jsonMapper.setInjectableValues(injectables); } @@ -472,26 +479,26 @@ public class IndexerCoordinatorNode extends RegisteringNode ); } + public void initializeDataSegmentPusher() + { + if (segmentPusher == null) { + segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper); + } + } + public void initializeTaskToolbox() { - if (taskToolbox == null) { - final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( - s3Service, - configFactory.build(S3DataSegmentPusherConfig.class), - jsonMapper - ); - final SegmentKiller segmentKiller = new S3SegmentKiller( - s3Service - ); - taskToolbox = new TaskToolbox( + if (taskToolboxFactory == null) { + final SegmentKiller segmentKiller = new S3SegmentKiller(s3Service); + taskToolboxFactory = new TaskToolboxFactory( taskConfig, - new LocalTaskActionClient( + new LocalTaskActionClientFactory( taskStorage, new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter) ), emitter, s3Service, - dataSegmentPusher, + segmentPusher, segmentKiller, jsonMapper ); @@ -546,7 +553,7 @@ public class IndexerCoordinatorNode extends RegisteringNode { if (taskStorage == null) { if (config.getStorageImpl().equals("local")) { - taskStorage = new LocalTaskStorage(); + taskStorage = new HeapMemoryTaskStorage(); } else if (config.getStorageImpl().equals("db")) { final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class); taskStorage = new DbTaskStorage(jsonMapper, dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI()); @@ -615,7 +622,7 @@ public class IndexerCoordinatorNode extends RegisteringNode public TaskRunner build() { final ExecutorService runnerExec = Executors.newFixedThreadPool(config.getNumLocalThreads()); - return new LocalTaskRunner(taskToolbox, runnerExec); + return new LocalTaskRunner(taskToolboxFactory, runnerExec); } }; } else { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index bde7bd6a2fd..3ac16741028 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -28,6 +28,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.actions.TaskAction; +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.TaskStorageQueryAdapter; @@ -181,9 +182,12 @@ public class IndexerCoordinatorResource @POST @Path("/action") @Produces("application/json") - public Response doAction(final TaskAction action) + public Response doAction(final TaskActionHolder holder) { - final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action); + final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask()) + .getTaskActionClient() + .submit(holder.getAction()); + final Map retMap = Maps.newHashMap(); retMap.put("result", ret); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index d7836d46faa..ec3a8d992e0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -23,6 +23,7 @@ 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.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.task.Task; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; @@ -45,21 +46,21 @@ public class TaskMonitor private final PathChildrenCache pathChildrenCache; private final CuratorFramework cf; private final WorkerCuratorCoordinator workerCuratorCoordinator; - private final TaskToolbox toolbox; + private final TaskToolboxFactory toolboxFactory; private final ExecutorService exec; public TaskMonitor( PathChildrenCache pathChildrenCache, CuratorFramework cf, WorkerCuratorCoordinator workerCuratorCoordinator, - TaskToolbox toolbox, + TaskToolboxFactory toolboxFactory, ExecutorService exec ) { this.pathChildrenCache = pathChildrenCache; this.cf = cf; this.workerCuratorCoordinator = workerCuratorCoordinator; - this.toolbox = toolbox; + this.toolboxFactory = toolboxFactory; this.exec = exec; } @@ -81,10 +82,11 @@ public class TaskMonitor throws Exception { if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final Task task = toolbox.getObjectMapper().readValue( + final Task task = toolboxFactory.getObjectMapper().readValue( cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()), Task.class ); + final TaskToolbox toolbox = toolboxFactory.build(task); if (workerCuratorCoordinator.statusExists(task.getId())) { log.warn("Got task %s that I am already running...", task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index d02ffa5d9e3..bad04040e73 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -35,6 +35,7 @@ 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.DataSegmentPusher; @@ -43,7 +44,9 @@ import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.loading.S3SegmentKiller; import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.actions.RemoteTaskActionClient; +import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -106,7 +109,8 @@ public class WorkerNode extends RegisteringNode private ServiceEmitter emitter = null; private TaskConfig taskConfig = null; private WorkerConfig workerConfig = null; - private TaskToolbox taskToolbox = null; + private DataSegmentPusher segmentPusher = null; + private TaskToolboxFactory taskToolboxFactory = null; private CuratorFramework curatorFramework = null; private ServiceDiscovery serviceDiscovery = null; private ServiceProvider coordinatorServiceProvider = null; @@ -149,9 +153,15 @@ public class WorkerNode extends RegisteringNode return this; } - public WorkerNode setTaskToolbox(TaskToolbox taskToolbox) + public WorkerNode setSegmentPusher(DataSegmentPusher segmentPusher) { - this.taskToolbox = taskToolbox; + this.segmentPusher = segmentPusher; + return this; + } + + public WorkerNode setTaskToolboxFactory(TaskToolboxFactory taskToolboxFactory) + { + this.taskToolboxFactory = taskToolboxFactory; return this; } @@ -195,6 +205,7 @@ public class WorkerNode extends RegisteringNode initializeCuratorFramework(); initializeServiceDiscovery(); initializeCoordinatorServiceProvider(); + initializeDataSegmentPusher(); initializeTaskToolbox(); initializeJacksonInjections(); initializeJacksonSubtypes(); @@ -271,7 +282,7 @@ public class WorkerNode extends RegisteringNode InjectableValues.Std injectables = new InjectableValues.Std(); injectables.addValue("s3Client", s3Service) - .addValue("segmentPusher", taskToolbox.getSegmentPusher()); + .addValue("segmentPusher", segmentPusher); jsonMapper.setInjectableValues(injectables); } @@ -334,23 +345,23 @@ public class WorkerNode extends RegisteringNode } } + public void initializeDataSegmentPusher() + { + if (segmentPusher == null) { + segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper); + } + } + public void initializeTaskToolbox() throws S3ServiceException { - if (taskToolbox == null) { - final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( - s3Service, - configFactory.build(S3DataSegmentPusherConfig.class), - jsonMapper - ); - final SegmentKiller segmentKiller = new S3SegmentKiller( - s3Service - ); - taskToolbox = new TaskToolbox( + if (taskToolboxFactory == null) { + final SegmentKiller segmentKiller = new S3SegmentKiller(s3Service); + taskToolboxFactory = new TaskToolboxFactory( taskConfig, - new RemoteTaskActionClient(httpClient, coordinatorServiceProvider, jsonMapper), + new RemoteTaskActionClientFactory(httpClient, coordinatorServiceProvider, jsonMapper), emitter, s3Service, - dataSegmentPusher, + segmentPusher, segmentKiller, jsonMapper ); @@ -417,7 +428,7 @@ public class WorkerNode extends RegisteringNode pathChildrenCache, curatorFramework, workerCuratorCoordinator, - taskToolbox, + taskToolboxFactory, workerExec ); lifecycle.addManagedInstance(taskMonitor); diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index 9242e19f355..18ce14a4555 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -12,6 +12,7 @@ import com.metamx.druid.merger.TestTask; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; @@ -280,7 +281,7 @@ public class RemoteTaskRunnerTest new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true), cf, workerCuratorCoordinator, - new TaskToolbox( + new TaskToolboxFactory( new TaskConfig() { @Override diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java index ae5b46fdfc1..3dc889b96f0 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java @@ -1,3 +1,22 @@ +/* + * 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.google.common.base.Optional; @@ -22,11 +41,11 @@ import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.SegmentKiller; -import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.LockAcquireAction; import com.metamx.druid.merger.common.actions.LockListAction; import com.metamx.druid.merger.common.actions.LockReleaseAction; @@ -59,7 +78,6 @@ import java.io.IOException; import java.util.Collection; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.concurrent.Executors; @@ -71,7 +89,7 @@ public class TaskLifecycleTest private TaskQueue tq = null; private TaskRunner tr = null; private MockMergerDBCoordinator mdc = null; - private TaskToolbox tb = null; + private TaskToolboxFactory tb = null; private TaskConsumer tc = null; TaskStorageQueryAdapter tsqa = null; @@ -91,12 +109,12 @@ public class TaskLifecycleTest tmp = Files.createTempDir(); - ts = new LocalTaskStorage(); + ts = new HeapMemoryTaskStorage(); tl = new TaskLockbox(ts); tq = new TaskQueue(ts, tl); mdc = newMockMDC(); - tb = new TaskToolbox( + tb = new TaskToolboxFactory( new TaskConfig() { @Override @@ -117,7 +135,7 @@ public class TaskLifecycleTest return null; } }, - new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())), + new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())), newMockEmitter(), null, // s3 client new DataSegmentPusher() @@ -137,16 +155,7 @@ public class TaskLifecycleTest } }, new DefaultObjectMapper() - ) - { - @Override - public Map getSegments( - Task task, List segments - ) throws SegmentLoadingException - { - return ImmutableMap.of(); - } - }; + ); tr = new LocalTaskRunner( tb, @@ -239,11 +248,12 @@ public class TaskLifecycleTest @Test public void testKillTask() throws Exception { - // TODO: Worst test ever + // This test doesn't actually do anything right now. We should actually put things into the Mocked coordinator + // Such that this test can test things... final Task killTask = new KillTask("foo", new Interval("2010-01-02/P2D")); - final TaskStatus mergedStatus = runTask(killTask); - Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode()); + final TaskStatus status = runTask(killTask); + Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); } @@ -273,8 +283,8 @@ public class TaskLifecycleTest // Sort of similar to what realtime tasks do: // Acquire lock for first interval - final Optional lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval1)); - final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + final Optional lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval1)); + final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertTrue("lock1 present", lock1.isPresent()); @@ -282,8 +292,8 @@ public class TaskLifecycleTest Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1); // Acquire lock for second interval - final Optional lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval2)); - final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + final Optional lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval2)); + final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertTrue("lock2 present", lock2.isPresent()); @@ -294,7 +304,6 @@ public class TaskLifecycleTest toolbox.getTaskActionClient() .submit( new SegmentInsertAction( - this, ImmutableSet.of( DataSegment.builder() .dataSource("foo") @@ -306,8 +315,8 @@ public class TaskLifecycleTest ); // Release first lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval1)); - final List locks3 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + toolbox.getTaskActionClient().submit(new LockReleaseAction(interval1)); + final List locks3 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3); @@ -316,7 +325,6 @@ public class TaskLifecycleTest toolbox.getTaskActionClient() .submit( new SegmentInsertAction( - this, ImmutableSet.of( DataSegment.builder() .dataSource("foo") @@ -328,8 +336,8 @@ public class TaskLifecycleTest ); // Release second lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval2)); - final List locks4 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + toolbox.getTaskActionClient().submit(new LockReleaseAction(interval2)); + final List locks4 = toolbox.getTaskActionClient().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertEquals("locks4", ImmutableList.of(), locks4); @@ -363,7 +371,7 @@ public class TaskLifecycleTest { final TaskLock myLock = Iterables.getOnlyElement( toolbox.getTaskActionClient() - .submit(new LockListAction(this)) + .submit(new LockListAction()) ); final DataSegment segment = DataSegment.builder() @@ -372,7 +380,7 @@ public class TaskLifecycleTest .version(myLock.getVersion()) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; @@ -398,10 +406,7 @@ public class TaskLifecycleTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement( - toolbox.getTaskActionClient() - .submit(new LockListAction(this)) - ); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); final DataSegment segment = DataSegment.builder() .dataSource("ds") @@ -409,7 +414,7 @@ public class TaskLifecycleTest .version(myLock.getVersion()) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; @@ -435,10 +440,7 @@ public class TaskLifecycleTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement( - toolbox.getTaskActionClient() - .submit(new LockListAction(this)) - ); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); final DataSegment segment = DataSegment.builder() .dataSource("ds") @@ -446,7 +448,7 @@ public class TaskLifecycleTest .version(myLock.getVersion() + "1!!!1!!") .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 0d003551ea9..939dc9b6b21 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -26,7 +26,8 @@ import com.google.common.collect.Sets; import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.SpawnTasksAction; import com.metamx.druid.merger.common.actions.TaskActionToolbox; import com.metamx.druid.merger.common.task.AbstractTask; @@ -43,7 +44,7 @@ public class TaskQueueTest @Test public void testEmptyQueue() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); @@ -65,7 +66,7 @@ public class TaskQueueTest @Test public void testAddRemove() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); @@ -154,12 +155,12 @@ public class TaskQueueTest @Test public void testContinues() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); - final TaskToolbox tb = new TaskToolbox( + final TaskToolboxFactory tb = new TaskToolboxFactory( null, - new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), + new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)), null, null, null, @@ -181,7 +182,7 @@ public class TaskQueueTest Assert.assertNull("poll #2", tq.poll()); // report T1 done. Should cause T0 to be created - tq.notify(t1, t1.run(tb)); + tq.notify(t1, t1.run(tb.build(t1))); Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent()); Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable()); @@ -195,7 +196,7 @@ public class TaskQueueTest Assert.assertNull("poll #4", tq.poll()); // report T0 done. Should cause T0, T1 to be marked complete - tq.notify(t0, t0.run(tb)); + tq.notify(t0, t0.run(tb.build(t0))); Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent()); Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable()); @@ -211,12 +212,12 @@ public class TaskQueueTest @Test public void testConcurrency() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); - final TaskToolbox tb = new TaskToolbox( + final TaskToolboxFactory tb = new TaskToolboxFactory( null, - new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), + new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)), null, null, null, @@ -248,7 +249,7 @@ public class TaskQueueTest Thread.sleep(5); // Finish t0 - tq.notify(t0, t0.run(tb)); + tq.notify(t0, t0.run(tb.build(t0))); // take max number of tasks final Set taken = Sets.newHashSet(); @@ -280,7 +281,7 @@ public class TaskQueueTest Assert.assertNull("null poll #2", tq.poll()); // Finish t3 - tq.notify(t3, t3.run(tb)); + tq.notify(t3, t3.run(tb.build(t3))); // We should be able to get t2 now final Task wt2 = tq.poll(); @@ -291,7 +292,7 @@ public class TaskQueueTest Assert.assertNull("null poll #3", tq.poll()); // Finish t2 - tq.notify(t2, t2.run(tb)); + tq.notify(t2, t2.run(tb.build(t2))); // We should be able to get t4 // And it should be in group G0, but that group should have a different version than last time @@ -305,14 +306,14 @@ public class TaskQueueTest Assert.assertNotSame("wt4 version", wt2Lock.getVersion(), wt4Lock.getVersion()); // Kind of done testing at this point, but let's finish t4 anyway - tq.notify(t4, t4.run(tb)); + tq.notify(t4, t4.run(tb.build(t4))); Assert.assertNull("null poll #4", tq.poll()); } @Test public void testBootstrap() throws Exception { - final TaskStorage storage = new LocalTaskStorage(); + final TaskStorage storage = new HeapMemoryTaskStorage(); final TaskLockbox lockbox = new TaskLockbox(storage); storage.insert(newTask("T1", "G1", "bar", new Interval("2011-01-01/P1D")), TaskStatus.running("T1")); @@ -374,7 +375,7 @@ public class TaskQueueTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks)); + toolbox.getTaskActionClient().submit(new SpawnTasksAction(nextTasks)); return TaskStatus.success(id); } }; diff --git a/pom.xml b/pom.xml index 3c8f6e76b18..59a91567adb 100644 --- a/pom.xml +++ b/pom.xml @@ -165,17 +165,17 @@ com.fasterxml.jackson.core jackson-annotations - 2.1.2 + 2.1.4 com.fasterxml.jackson.core jackson-core - 2.1.3 + 2.1.4 com.fasterxml.jackson.core jackson-databind - 2.1.4-mmx-2 + 2.1.4 com.fasterxml.jackson.datatype @@ -190,12 +190,12 @@ com.fasterxml.jackson.dataformat jackson-dataformat-smile - 2.1.3 + 2.1.4 com.fasterxml.jackson.jaxrs jackson-jaxrs-json-provider - 2.1.3 + 2.1.4 org.codehaus.jackson diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index 087e87d107f..fb47abab945 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -45,25 +45,15 @@ import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.LocalDataSegmentPusher; -import com.metamx.druid.loading.LocalDataSegmentPusherConfig; -import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.Monitor; - - - - -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.security.AWSCredentials; import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; @@ -259,31 +249,7 @@ public class RealtimeNode extends BaseServerNode private void initializeSegmentPusher() { if (dataSegmentPusher == null) { - final Properties props = getProps(); - if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) { - dataSegmentPusher = new LocalDataSegmentPusher( - getConfigFactory().build(LocalDataSegmentPusherConfig.class), getJsonMapper() - ); - } - else { - - final RestS3Service s3Client; - try { - s3Client = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) - ); - } - catch (S3ServiceException e) { - throw Throwables.propagate(e); - } - - dataSegmentPusher = new S3DataSegmentPusher( - s3Client, getConfigFactory().build(S3DataSegmentPusherConfig.class), getJsonMapper() - ); - } + dataSegmentPusher = ServerInit.getSegmentPusher(getProps(), getConfigFactory(), getJsonMapper()); } } diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 7cd6caf3c1b..cca4f910b77 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -19,17 +19,24 @@ package com.metamx.druid.initialization; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.DruidProcessingConfig; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.DelegatingSegmentLoader; import com.metamx.druid.loading.LocalDataSegmentPuller; +import com.metamx.druid.loading.LocalDataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusherConfig; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.S3DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.query.group.GroupByQueryEngine; @@ -48,12 +55,16 @@ import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; import com.metamx.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory; import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import com.metamx.druid.utils.PropUtils; +import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.security.AWSCredentials; import org.skife.config.ConfigurationObjectFactory; import java.lang.reflect.InvocationTargetException; import java.nio.ByteBuffer; import java.util.Map; +import java.util.Properties; import java.util.concurrent.atomic.AtomicLong; /** @@ -145,6 +156,34 @@ public class ServerInit return queryRunners; } + public static DataSegmentPusher getSegmentPusher( + final Properties props, + final ConfigurationObjectFactory configFactory, + final ObjectMapper jsonMapper + ) + { + if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) { + return new LocalDataSegmentPusher(configFactory.build(LocalDataSegmentPusherConfig.class), jsonMapper); + } + else { + + final RestS3Service s3Client; + try { + s3Client = new RestS3Service( + new AWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ); + } + catch (S3ServiceException e) { + throw Throwables.propagate(e); + } + + return new S3DataSegmentPusher(s3Client, configFactory.build(S3DataSegmentPusherConfig.class), jsonMapper); + } + } + private static class ComputeScratchPool extends StupidPool { private static final Logger log = new Logger(ComputeScratchPool.class); From 6864007c0567335890e1807afc1f5629a075f3f9 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Tue, 5 Mar 2013 13:27:25 -0600 Subject: [PATCH 15/17] 1) Remove Task parameter from other methods off of TaskToolbox that required it 2) Move getTaskDir() off of TaskConfig and onto TaskToolbox 3) Rename SegmentKiller interface to DataSegmentKiller 4) Change signature of "DataSegmentKiller.kill(Collection) throws ServiceException" to just kill(DataSegment) throws SegmentLoadingException 5) Add various log messages 6) Update the version of the segment that has been converted --- .../com/metamx/druid/client/DataSegment.java | 1 + .../druid/merger/common/TaskToolbox.java | 35 ++++++---- .../merger/common/TaskToolboxFactory.java | 31 +++----- .../merger/common/config/TaskConfig.java | 23 ++++-- .../merger/common/task/AbstractTask.java | 2 +- .../druid/merger/common/task/DeleteTask.java | 6 +- .../merger/common/task/HadoopIndexTask.java | 4 +- .../task/IndexDeterminePartitionsTask.java | 2 +- .../common/task/IndexGeneratorTask.java | 21 +++--- .../druid/merger/common/task/IndexTask.java | 2 +- .../druid/merger/common/task/KillTask.java | 10 +-- .../druid/merger/common/task/MergeTask.java | 37 +++------- .../common/task/VersionConverterSubTask.java | 21 +++++- .../common/task/VersionConverterTask.java | 63 ++++++++++++----- .../merger/coordinator/LocalTaskRunner.java | 2 +- .../http/IndexerCoordinatorNode.java | 8 +-- .../http/IndexerCoordinatorResource.java | 3 +- .../druid/merger/worker/TaskMonitor.java | 2 +- .../druid/merger/worker/http/WorkerNode.java | 12 ++-- .../merger/coordinator/TaskLifecycleTest.java | 40 +++++------ .../merger/coordinator/TaskQueueTest.java | 2 +- .../druid/loading/DataSegmentKiller.java | 29 ++++++++ .../druid/loading/S3DataSegmentKiller.java | 70 +++++++++++++++++++ .../druid/loading/S3DataSegmentPusher.java | 6 +- .../metamx/druid/loading/S3SegmentKiller.java | 49 ------------- .../metamx/druid/loading/SegmentKiller.java | 14 ---- 26 files changed, 281 insertions(+), 214 deletions(-) create mode 100644 server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java create mode 100644 server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java delete mode 100644 server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java delete mode 100644 server/src/main/java/com/metamx/druid/loading/SegmentKiller.java diff --git a/client/src/main/java/com/metamx/druid/client/DataSegment.java b/client/src/main/java/com/metamx/druid/client/DataSegment.java index b915f7680cd..e8b7e55a495 100644 --- a/client/src/main/java/com/metamx/druid/client/DataSegment.java +++ b/client/src/main/java/com/metamx/druid/client/DataSegment.java @@ -261,6 +261,7 @@ public class DataSegment implements Comparable ", loadSpec=" + loadSpec + ", interval=" + interval + ", dataSource='" + dataSource + '\'' + + ", binaryVersion='" + binaryVersion + '\'' + '}'; } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index 58ad3c5cc43..32cb188273a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -25,11 +25,12 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; -import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.merger.common.actions.TaskActionClientFactory; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.task.Task; import com.metamx.emitter.service.ServiceEmitter; @@ -45,29 +46,32 @@ import java.util.Map; public class TaskToolbox { private final TaskConfig config; - private final TaskActionClient taskActionClient; + private final Task task; + private final TaskActionClientFactory taskActionClientFactory; private final ServiceEmitter emitter; private final RestS3Service s3Client; private final DataSegmentPusher segmentPusher; - private final SegmentKiller segmentKiller; + private final DataSegmentKiller dataSegmentKiller; private final ObjectMapper objectMapper; public TaskToolbox( TaskConfig config, - TaskActionClient taskActionClient, + Task task, + TaskActionClientFactory taskActionClientFactory, ServiceEmitter emitter, RestS3Service s3Client, DataSegmentPusher segmentPusher, - SegmentKiller segmentKiller, + DataSegmentKiller dataSegmentKiller, ObjectMapper objectMapper ) { this.config = config; - this.taskActionClient = taskActionClient; + this.task = task; + this.taskActionClientFactory = taskActionClientFactory; this.emitter = emitter; this.s3Client = s3Client; this.segmentPusher = segmentPusher; - this.segmentKiller = segmentKiller; + this.dataSegmentKiller = dataSegmentKiller; this.objectMapper = objectMapper; } @@ -76,9 +80,9 @@ public class TaskToolbox return config; } - public TaskActionClient getTaskActionClient() + public TaskActionClient getTaskActionClientFactory() { - return taskActionClient; + return taskActionClientFactory.create(task); } public ServiceEmitter getEmitter() @@ -91,9 +95,9 @@ public class TaskToolbox return segmentPusher; } - public SegmentKiller getSegmentKiller() + public DataSegmentKiller getDataSegmentKiller() { - return segmentKiller; + return dataSegmentKiller; } public ObjectMapper getObjectMapper() @@ -101,7 +105,7 @@ public class TaskToolbox return objectMapper; } - public Map getSegments(final Task task, List segments) + public Map getSegments(List segments) throws SegmentLoadingException { final SingleSegmentLoader loader = new SingleSegmentLoader( @@ -112,7 +116,7 @@ public class TaskToolbox @Override public File getCacheDirectory() { - return new File(config.getTaskDir(task), "fetched_segments"); + return new File(getTaskDir(), "fetched_segments"); } } ); @@ -124,4 +128,9 @@ public class TaskToolbox return retVal; } + + public File getTaskDir() { + return new File(config.getBaseTaskDir(), task.getId()); + } + } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java index 9a804bef79e..2266860ea86 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java @@ -20,55 +20,43 @@ package com.metamx.druid.merger.common; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Maps; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.MMappedQueryableIndexFactory; -import com.metamx.druid.loading.S3DataSegmentPuller; -import com.metamx.druid.loading.SegmentKiller; -import com.metamx.druid.loading.SegmentLoaderConfig; -import com.metamx.druid.loading.SegmentLoadingException; -import com.metamx.druid.loading.SingleSegmentLoader; -import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.merger.common.actions.TaskActionClientFactory; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.task.Task; import com.metamx.emitter.service.ServiceEmitter; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import java.io.File; -import java.util.List; -import java.util.Map; - /** * Stuff that may be needed by a Task in order to conduct its business. */ public class TaskToolboxFactory { private final TaskConfig config; - private final TaskActionClientFactory taskActionClient; + private final TaskActionClientFactory taskActionClientFactory; private final ServiceEmitter emitter; private final RestS3Service s3Client; private final DataSegmentPusher segmentPusher; - private final SegmentKiller segmentKiller; + private final DataSegmentKiller dataSegmentKiller; private final ObjectMapper objectMapper; public TaskToolboxFactory( TaskConfig config, - TaskActionClientFactory taskActionClient, + TaskActionClientFactory taskActionClientFactory, ServiceEmitter emitter, RestS3Service s3Client, DataSegmentPusher segmentPusher, - SegmentKiller segmentKiller, + DataSegmentKiller dataSegmentKiller, ObjectMapper objectMapper ) { this.config = config; - this.taskActionClient = taskActionClient; + this.taskActionClientFactory = taskActionClientFactory; this.emitter = emitter; this.s3Client = s3Client; this.segmentPusher = segmentPusher; - this.segmentKiller = segmentKiller; + this.dataSegmentKiller = dataSegmentKiller; this.objectMapper = objectMapper; } @@ -81,11 +69,12 @@ public class TaskToolboxFactory { return new TaskToolbox( config, - taskActionClient == null ? null : taskActionClient.create(task), + task, + taskActionClientFactory, emitter, s3Client, segmentPusher, - segmentKiller, + dataSegmentKiller, objectMapper ); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java index 5918f0627c6..2bd27667514 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java @@ -1,3 +1,22 @@ +/* + * 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.config; import com.metamx.druid.merger.common.task.Task; @@ -17,8 +36,4 @@ public abstract class TaskConfig @Config("druid.merger.hadoopWorkingPath") public abstract String getHadoopWorkingPath(); - - public File getTaskDir(final Task task) { - return new File(getBaseTaskDir(), task.getId()); - } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index c2f9a792bb0..518fb04ab37 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -102,7 +102,7 @@ public abstract class AbstractTask implements Task return ID_JOINER.join(objects); } - public SegmentListUsedAction makeImplicitListUsedAction() + public SegmentListUsedAction defaultListUsedAction() { return new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 2f10861ef5f..67754ee00d7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -77,7 +77,7 @@ public class DeleteTask extends AbstractTask public TaskStatus run(TaskToolbox toolbox) throws Exception { // Strategy: Create an empty segment covering the interval to be deleted - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); final Interval interval = this.getImplicitLockInterval().get(); final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty); @@ -91,7 +91,7 @@ public class DeleteTask extends AbstractTask .shardSpec(new NoneShardSpec()) .build(); - final File outDir = new File(toolbox.getConfig().getTaskDir(this), segment.getIdentifier()); + final File outDir = new File(toolbox.getTaskDir(), segment.getIdentifier()); final File fileToUpload = IndexMerger.merge(Lists.newArrayList(emptyAdapter), new AggregatorFactory[0], outDir); // Upload the segment @@ -104,7 +104,7 @@ public class DeleteTask extends AbstractTask segment.getVersion() ); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java index 6e284557529..f0d09d5137d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java @@ -93,7 +93,7 @@ public class HadoopIndexTask extends AbstractTask ); // We should have a lock from before we started running - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); log.info("Setting version to: %s", myLock.getVersion()); configCopy.setVersion(myLock.getVersion()); @@ -124,7 +124,7 @@ public class HadoopIndexTask extends AbstractTask List publishedSegments = job.getPublishedSegments(); // Request segment pushes - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments))); // Done return TaskStatus.success(getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 47f72b12501..3dfe99a68f1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -258,7 +258,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask } ); - toolbox.getTaskActionClient().submit(new SpawnTasksAction(nextTasks)); + toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(nextTasks)); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index 2a7b4683b8d..1790ddc6aa5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -100,24 +100,21 @@ public class IndexGeneratorTask extends AbstractTask public TaskStatus run(final TaskToolbox toolbox) throws Exception { // We should have a lock from before we started running - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); // We know this exists final Interval interval = getImplicitLockInterval().get(); // Set up temporary directory for indexing final File tmpDir = new File( + toolbox.getTaskDir(), String.format( - "%s/%s", - toolbox.getConfig().getTaskDir(this).toString(), - String.format( - "%s_%s_%s_%s_%s", - this.getDataSource(), - interval.getStart(), - interval.getEnd(), - myLock.getVersion(), - schema.getShardSpec().getPartitionNum() - ) + "%s_%s_%s_%s_%s", + this.getDataSource(), + interval.getStart(), + interval.getEnd(), + myLock.getVersion(), + schema.getShardSpec().getPartitionNum() ) ); @@ -193,7 +190,7 @@ public class IndexGeneratorTask extends AbstractTask ); // Request segment pushes - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(pushedSegments))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.copyOf(pushedSegments))); // Done return TaskStatus.success(getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 35babcd6a22..d1bfc6d77fc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -142,7 +142,7 @@ public class IndexTask extends AbstractTask @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { - toolbox.getTaskActionClient().submit(new SpawnTasksAction(toSubtasks())); + toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(toSubtasks())); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index 52b8b48b8b4..e652ab69151 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -72,7 +72,7 @@ public class KillTask extends AbstractTask public TaskStatus run(TaskToolbox toolbox) throws Exception { // Confirm we have a lock (will throw if there isn't exactly one element) - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); if(!myLock.getDataSource().equals(getDataSource())) { throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); @@ -84,7 +84,7 @@ public class KillTask extends AbstractTask // List unused segments final List unusedSegments = toolbox - .getTaskActionClient() + .getTaskActionClientFactory() .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval())); // Verify none of these segments have versions > lock version @@ -102,10 +102,12 @@ public class KillTask extends AbstractTask } // Kill segments - toolbox.getSegmentKiller().kill(unusedSegments); + for (DataSegment segment : unusedSegments) { + toolbox.getDataSegmentKiller().kill(segment); + } // Remove metadata for these segments - toolbox.getTaskActionClient().submit(new SegmentNukeAction(ImmutableSet.copyOf(unusedSegments))); + toolbox.getTaskActionClientFactory().submit(new SegmentNukeAction(ImmutableSet.copyOf(unusedSegments))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 757a91c2598..561cb940639 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -43,8 +43,8 @@ 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.SegmentListUsedAction; import com.metamx.druid.shard.NoneShardSpec; +import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; @@ -67,7 +67,7 @@ public abstract class MergeTask extends AbstractTask { private final List segments; - private static final Logger log = new Logger(MergeTask.class); + private static final EmittingLogger log = new EmittingLogger(MergeTask.class); protected MergeTask(final String dataSource, final List segments) { @@ -119,11 +119,11 @@ public abstract class MergeTask extends AbstractTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); final ServiceEmitter emitter = toolbox.getEmitter(); final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments); - final File taskDir = toolbox.getConfig().getTaskDir(this); + final File taskDir = toolbox.getTaskDir(); try { @@ -147,7 +147,7 @@ public abstract class MergeTask extends AbstractTask // download segments to merge - final Map gettedSegments = toolbox.getSegments(this, segments); + final Map gettedSegments = toolbox.getSegments(segments); // merge files together final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); @@ -170,27 +170,14 @@ public abstract class MergeTask extends AbstractTask emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); return TaskStatus.success(getId()); } catch (Exception e) { - log.error( - e, - String.format( - "Exception merging %s[%s] segments", - mergedSegment.getDataSource(), - mergedSegment.getInterval() - ) - ); - emitter.emit( - new AlertEvent.Builder().build( - "Exception merging", - ImmutableMap.builder() - .put("exception", e.toString()) - .build() - ) - ); + log.makeAlert(e, "Exception merging[%s]", mergedSegment.getDataSource()) + .addData("interval", mergedSegment.getInterval()) + .emit(); return TaskStatus.failure(getId()); } @@ -213,11 +200,7 @@ public abstract class MergeTask extends AbstractTask }; final Set current = ImmutableSet.copyOf( - Iterables.transform( - toolbox.getTaskActionClient() - .submit(new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get())), - toIdentifier - ) + Iterables.transform(toolbox.getTaskActionClientFactory().submit(defaultListUsedAction()), toIdentifier) ); final Set requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier)); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java index db08e788e4d..2099d903d33 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java @@ -21,11 +21,13 @@ package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; +import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import org.joda.time.DateTime; import java.io.File; import java.util.Arrays; @@ -35,6 +37,8 @@ import java.util.Map; */ public class VersionConverterSubTask extends AbstractTask { + private static final Logger log = new Logger(VersionConverterSubTask.class); + private final DataSegment segment; protected VersionConverterSubTask( @@ -50,6 +54,7 @@ public class VersionConverterSubTask extends AbstractTask segment.getInterval().getEnd(), segment.getShardSpec().getPartitionNum() ), + groupId, segment.getDataSource(), segment.getInterval() ); @@ -65,13 +70,23 @@ public class VersionConverterSubTask extends AbstractTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final Map localSegments = toolbox.getSegments(this, Arrays.asList(segment)); + log.info("Converting segment[%s]", segment); + final Map localSegments = toolbox.getSegments(Arrays.asList(segment)); final File location = localSegments.get(segment); final File outLocation = new File(location, "v9_out"); if (IndexIO.convertSegment(location, outLocation)) { - final DataSegment updatedSegment = toolbox.getSegmentPusher().push(outLocation, segment); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); + final int outVersion = IndexIO.getVersionFromDir(outLocation); + + // Appending to the version makes a new version that inherits most comparability parameters of the original + // version, but is "newer" than said original version. + DataSegment updatedSegment = segment.withVersion(String.format("%s_v%s", segment.getVersion(), outVersion)); + updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment); + + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); + } + else { + log.info("Conversion failed."); } return success(); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java index 4106e210547..3859190a1bb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java @@ -1,10 +1,32 @@ +/* + * 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.task; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.metamx.common.ISE; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.actions.SpawnTasksAction; @@ -12,7 +34,6 @@ import com.metamx.druid.merger.common.actions.TaskActionClient; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.util.List; /** @@ -20,6 +41,9 @@ import java.util.List; public class VersionConverterTask extends AbstractTask { private static final String TYPE = "version_converter"; + private static final Integer CURR_VERSION_INTEGER = new Integer(IndexIO.CURRENT_VERSION_ID); + + private static final Logger log = new Logger(VersionConverterTask.class); public VersionConverterTask( @JsonProperty("dataSource") String dataSource, @@ -48,25 +72,30 @@ public class VersionConverterTask extends AbstractTask @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { - final TaskActionClient taskClient = toolbox.getTaskActionClient(); + final TaskActionClient taskClient = toolbox.getTaskActionClientFactory(); - List segments = taskClient.submit(makeImplicitListUsedAction()); + List segments = taskClient.submit(defaultListUsedAction()); - taskClient.submit( - new SpawnTasksAction( - Lists.transform( - segments, - new Function() - { - @Override - public Task apply(@Nullable DataSegment input) - { - return new VersionConverterSubTask(getGroupId(), input); - } + final FunctionalIterable tasks = FunctionalIterable + .create(segments) + .keep( + new Function() + { + @Override + public Task apply(DataSegment segment) + { + final Integer segmentVersion = segment.getBinaryVersion(); + if (!CURR_VERSION_INTEGER.equals(segmentVersion)) { + return new VersionConverterSubTask(getGroupId(), segment); } - ) - ) - ); + + log.info("Skipping[%s], already version[%s]", segment.getIdentifier(), segmentVersion); + return null; + } + } + ); + + taskClient.submit(new SpawnTasksAction(Lists.newArrayList(tasks))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index a4f58a361f8..5dbe8273d6c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -92,7 +92,7 @@ public class LocalTaskRunner implements TaskRunner } try { - final File taskDir = toolbox.getConfig().getTaskDir(task); + final File taskDir = toolbox.getTaskDir(); if (taskDir.exists()) { log.info("Removing task directory: %s", taskDir); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 833c3976ee8..286d2e1649a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -52,8 +52,8 @@ import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.S3SegmentKiller; -import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.S3DataSegmentKiller; +import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.TaskActionToolbox; @@ -489,7 +489,7 @@ public class IndexerCoordinatorNode extends RegisteringNode public void initializeTaskToolbox() { if (taskToolboxFactory == null) { - final SegmentKiller segmentKiller = new S3SegmentKiller(s3Service); + final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service); taskToolboxFactory = new TaskToolboxFactory( taskConfig, new LocalTaskActionClientFactory( @@ -499,7 +499,7 @@ public class IndexerCoordinatorNode extends RegisteringNode emitter, s3Service, segmentPusher, - segmentKiller, + dataSegmentKiller, jsonMapper ); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index 3ac16741028..5f0714a8d61 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -27,7 +27,6 @@ import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.actions.TaskAction; import com.metamx.druid.merger.common.actions.TaskActionHolder; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; @@ -185,7 +184,7 @@ public class IndexerCoordinatorResource public Response doAction(final TaskActionHolder holder) { final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask()) - .getTaskActionClient() + .getTaskActionClientFactory() .submit(holder.getAction()); final Map retMap = Maps.newHashMap(); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index ec3a8d992e0..867b8dd9cde 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -101,7 +101,7 @@ public class TaskMonitor public void run() { final long startTime = System.currentTimeMillis(); - final File taskDir = toolbox.getConfig().getTaskDir(task); + final File taskDir = toolbox.getTaskDir(); log.info("Running task [%s]", task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index bad04040e73..caef5bd2935 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -39,13 +39,9 @@ import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusherConfig; -import com.metamx.druid.loading.S3SegmentKiller; -import com.metamx.druid.loading.SegmentKiller; -import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.loading.S3DataSegmentKiller; +import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.merger.common.TaskToolboxFactory; -import com.metamx.druid.merger.common.actions.RemoteTaskActionClient; import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; @@ -355,14 +351,14 @@ public class WorkerNode extends RegisteringNode public void initializeTaskToolbox() throws S3ServiceException { if (taskToolboxFactory == null) { - final SegmentKiller segmentKiller = new S3SegmentKiller(s3Service); + final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service); taskToolboxFactory = new TaskToolboxFactory( taskConfig, new RemoteTaskActionClientFactory(httpClient, coordinatorServiceProvider, jsonMapper), emitter, s3Service, segmentPusher, - segmentKiller, + dataSegmentKiller, jsonMapper ); } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java index 3dc889b96f0..4fac2c5eded 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java @@ -40,7 +40,8 @@ import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.DataSegmentKiller; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; @@ -75,7 +76,6 @@ import org.junit.Test; import java.io.File; import java.io.IOException; -import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Set; @@ -146,10 +146,10 @@ public class TaskLifecycleTest return segment; } }, - new SegmentKiller() + new DataSegmentKiller() { @Override - public void kill(Collection segments) throws ServiceException + public void kill(DataSegment segments) throws SegmentLoadingException { } @@ -283,8 +283,8 @@ public class TaskLifecycleTest // Sort of similar to what realtime tasks do: // Acquire lock for first interval - final Optional lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval1)); - final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction()); + final Optional lock1 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval1)); + final List locks1 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertTrue("lock1 present", lock1.isPresent()); @@ -292,8 +292,8 @@ public class TaskLifecycleTest Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1); // Acquire lock for second interval - final Optional lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval2)); - final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction()); + final Optional lock2 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval2)); + final List locks2 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertTrue("lock2 present", lock2.isPresent()); @@ -301,7 +301,7 @@ public class TaskLifecycleTest Assert.assertEquals("locks2", ImmutableList.of(lock1.get(), lock2.get()), locks2); // Push first segment - toolbox.getTaskActionClient() + toolbox.getTaskActionClientFactory() .submit( new SegmentInsertAction( ImmutableSet.of( @@ -315,14 +315,14 @@ public class TaskLifecycleTest ); // Release first lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(interval1)); - final List locks3 = toolbox.getTaskActionClient().submit(new LockListAction()); + toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval1)); + final List locks3 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3); // Push second segment - toolbox.getTaskActionClient() + toolbox.getTaskActionClientFactory() .submit( new SegmentInsertAction( ImmutableSet.of( @@ -336,8 +336,8 @@ public class TaskLifecycleTest ); // Release second lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(interval2)); - final List locks4 = toolbox.getTaskActionClient().submit(new LockListAction()); + toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval2)); + final List locks4 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertEquals("locks4", ImmutableList.of(), locks4); @@ -370,7 +370,7 @@ public class TaskLifecycleTest public TaskStatus run(TaskToolbox toolbox) throws Exception { final TaskLock myLock = Iterables.getOnlyElement( - toolbox.getTaskActionClient() + toolbox.getTaskActionClientFactory() .submit(new LockListAction()) ); @@ -380,7 +380,7 @@ public class TaskLifecycleTest .version(myLock.getVersion()) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; @@ -406,7 +406,7 @@ public class TaskLifecycleTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); final DataSegment segment = DataSegment.builder() .dataSource("ds") @@ -414,7 +414,7 @@ public class TaskLifecycleTest .version(myLock.getVersion()) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; @@ -440,7 +440,7 @@ public class TaskLifecycleTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); final DataSegment segment = DataSegment.builder() .dataSource("ds") @@ -448,7 +448,7 @@ public class TaskLifecycleTest .version(myLock.getVersion() + "1!!!1!!") .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 939dc9b6b21..dfb0d959d1a 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -375,7 +375,7 @@ public class TaskQueueTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - toolbox.getTaskActionClient().submit(new SpawnTasksAction(nextTasks)); + toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(nextTasks)); return TaskStatus.success(id); } }; diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java new file mode 100644 index 00000000000..85483eaa505 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java @@ -0,0 +1,29 @@ +/* + * 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.loading; + +import com.metamx.druid.client.DataSegment; + +/** + */ +public interface DataSegmentKiller +{ + public void kill(DataSegment segments) throws SegmentLoadingException; +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java new file mode 100644 index 00000000000..47c39c1857e --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java @@ -0,0 +1,70 @@ +/* + * 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.loading; + +import com.google.inject.Inject; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import org.jets3t.service.ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; + +import java.util.Map; + +/** + */ +public class S3DataSegmentKiller implements DataSegmentKiller +{ + private static final Logger log = new Logger(S3DataSegmentKiller.class); + + private final RestS3Service s3Client; + + @Inject + public S3DataSegmentKiller( + RestS3Service s3Client + ) + { + this.s3Client = s3Client; + } + + + @Override + public void kill(DataSegment segment) throws SegmentLoadingException + { + try { + Map loadSpec = segment.getLoadSpec(); + String s3Bucket = MapUtils.getString(loadSpec, "bucket"); + String s3Path = MapUtils.getString(loadSpec, "key"); + String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; + + if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { + log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); + s3Client.deleteObject(s3Bucket, s3Path); + } + if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { + log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); + s3Client.deleteObject(s3Bucket, s3DescriptorPath); + } + } + catch (ServiceException e) { + throw new SegmentLoadingException(e, "Couldn't kill segment[%s]", segment.getIdentifier()); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index dd5ce951695..9dacbe8b546 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -87,11 +87,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher segment = segment.withSize(indexSize) .withLoadSpec( - ImmutableMap.of( - "type", "s3_zip", - "bucket", outputBucket, - "key", toPush.getKey() - ) + ImmutableMap.of("type", "s3_zip", "bucket", outputBucket, "key", toPush.getKey()) ) .withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir)); diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java deleted file mode 100644 index 46f6acfc629..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java +++ /dev/null @@ -1,49 +0,0 @@ -package com.metamx.druid.loading; - -import com.google.inject.Inject; -import com.metamx.common.MapUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import org.jets3t.service.ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; - -import java.util.Collection; -import java.util.Map; - -/** - */ -public class S3SegmentKiller implements SegmentKiller -{ - private static final Logger log = new Logger(S3SegmentKiller.class); - - private final RestS3Service s3Client; - - @Inject - public S3SegmentKiller( - RestS3Service s3Client - ) - { - this.s3Client = s3Client; - } - - - @Override - public void kill(Collection segments) throws ServiceException - { - for (final DataSegment segment : segments) { - Map loadSpec = segment.getLoadSpec(); - String s3Bucket = MapUtils.getString(loadSpec, "bucket"); - String s3Path = MapUtils.getString(loadSpec, "key"); - String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; - - if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { - log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); - s3Client.deleteObject(s3Bucket, s3Path); - } - if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { - log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); - s3Client.deleteObject(s3Bucket, s3DescriptorPath); - } - } - } -} diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java deleted file mode 100644 index 8f8746d5324..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.metamx.druid.loading; - -import com.metamx.druid.client.DataSegment; -import org.jets3t.service.ServiceException; - -import java.util.Collection; -import java.util.List; - -/** - */ -public interface SegmentKiller -{ - public void kill(Collection segments) throws ServiceException; -} From 1c3ef48f34b7454d6bdd8bba55e079b92e7fc352 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 7 Mar 2013 12:59:59 -0600 Subject: [PATCH 16/17] 1) Adjust the Config stuff that WorkerSetupManager was using to be reusable 2) Use new ConfigManager to pull out a whitelist for the Master --- .../indexing}/ClientAppendQuery.java | 12 +- .../indexing/ClientConversionQuery.java | 57 +++ .../indexing/ClientKillQuery.java} | 27 +- .../indexing/ClientMergeQuery.java} | 16 +- .../indexing/IndexingServiceClient.java | 115 ++++++ .../metamx/druid/merge/ClientKillQuery.java | 41 --- .../metamx/druid/merge/ClientMergeQuery.java | 41 --- .../metamx/druid/config/ConfigManager.java | 259 ++++++++++++++ .../druid/config/ConfigManagerConfig.java | 18 + .../com/metamx/druid/config/ConfigSerde.java | 9 + .../druid/config/JacksonConfigManager.java | 134 +++++++ .../java/com/metamx/druid/db/DbConnector.java | 9 +- .../druid/merger/common/task/AppendTask.java | 2 +- .../merger/common/task/DefaultMergeTask.java | 87 ----- .../druid/merger/common/task/MergeTask.java | 326 +++--------------- .../merger/common/task/MergeTaskBase.java | 315 +++++++++++++++++ .../metamx/druid/merger/common/task/Task.java | 6 +- .../common/task/VersionConverterSubTask.java | 94 ----- .../common/task/VersionConverterTask.java | 88 ++++- .../config/WorkerSetupManagerConfig.java | 39 --- .../http/IndexerCoordinatorNode.java | 20 +- .../http/IndexerCoordinatorResource.java | 19 +- .../coordinator/scaling/ScalingStats.java | 19 + .../coordinator/setup/WorkerSetupManager.java | 184 +--------- .../com/metamx/druid/merger/TestTask.java | 4 +- ...geTaskTest.java => MergeTaskBaseTest.java} | 10 +- pom.xml | 2 +- .../com/metamx/druid/http/InfoResource.java | 9 +- .../com/metamx/druid/http/MasterMain.java | 24 +- .../com/metamx/druid/http/MasterResource.java | 1 - .../druid/http/MasterServletModule.java | 7 +- .../com/metamx/druid/master/DruidMaster.java | 110 +++--- .../master/DruidMasterSegmentMerger.java | 57 ++- .../metamx/druid/master/HttpMergerClient.java | 78 ----- ...MergerClient.java => MergerWhitelist.java} | 29 +- .../master/DruidMasterSegmentMergerTest.java | 15 +- .../metamx/druid/master/DruidMasterTest.java | 2 - 37 files changed, 1271 insertions(+), 1014 deletions(-) rename client/src/main/java/com/metamx/druid/{merge => client/indexing}/ClientAppendQuery.java (92%) create mode 100644 client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java rename client/src/main/java/com/metamx/druid/{merge/ClientDeleteQuery.java => client/indexing/ClientKillQuery.java} (83%) rename client/src/main/java/com/metamx/druid/{merge/ClientDefaultMergeQuery.java => client/indexing/ClientMergeQuery.java} (90%) create mode 100644 client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java delete mode 100644 client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java delete mode 100644 client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java create mode 100644 common/src/main/java/com/metamx/druid/config/ConfigManager.java create mode 100644 common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java create mode 100644 common/src/main/java/com/metamx/druid/config/ConfigSerde.java create mode 100644 common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java rename merger/src/test/java/com/metamx/druid/merger/common/task/{MergeTaskTest.java => MergeTaskBaseTest.java} (89%) delete mode 100644 server/src/main/java/com/metamx/druid/master/HttpMergerClient.java rename server/src/main/java/com/metamx/druid/master/{MergerClient.java => MergerWhitelist.java} (57%) diff --git a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java similarity index 92% rename from client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java rename to client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java index 5a8e3bdb50f..5f744918a0a 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merge; +package com.metamx.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -29,7 +29,7 @@ import java.util.List; /** */ -public class ClientAppendQuery implements ClientMergeQuery +public class ClientAppendQuery { private final String dataSource; private final List segments; @@ -45,14 +45,18 @@ public class ClientAppendQuery implements ClientMergeQuery } @JsonProperty - @Override + public String getType() + { + return "append"; + } + + @JsonProperty public String getDataSource() { return dataSource; } @JsonProperty - @Override public List getSegments() { return segments; diff --git a/client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java new file mode 100644 index 00000000000..c0b96bb80b4 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java @@ -0,0 +1,57 @@ +package com.metamx.druid.client.indexing; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.druid.client.DataSegment; +import org.joda.time.Interval; + +/** + */ +public class ClientConversionQuery +{ + private final String dataSource; + private final Interval interval; + private final DataSegment segment; + + public ClientConversionQuery( + DataSegment segment + ) + { + this.dataSource = segment.getDataSource(); + this.interval = segment.getInterval(); + this.segment = segment; + } + + public ClientConversionQuery( + String dataSource, + Interval interval + ) + { + this.dataSource = dataSource; + this.interval = interval; + this.segment = null; + } + + @JsonProperty + public String getType() + { + return "version_converter"; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public DataSegment getSegment() + { + return segment; + } +} diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java similarity index 83% rename from client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java rename to client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java index 3acf20a0a43..3ae8dffb225 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java @@ -17,21 +17,21 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merge; - - +package com.metamx.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; -public class ClientDeleteQuery +/** + */ +public class ClientKillQuery { private final String dataSource; private final Interval interval; @JsonCreator - public ClientDeleteQuery( + public ClientKillQuery( @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval ) @@ -40,22 +40,21 @@ public class ClientDeleteQuery this.interval = interval; } + @JsonProperty + public String getType() + { + return "kill"; + } + + @JsonProperty public String getDataSource() { return dataSource; } + @JsonProperty public Interval getInterval() { return interval; } - - @Override - public String toString() - { - return "ClientDeleteQuery{" + - "dataSource='" + dataSource + '\'' + - ", interval=" + interval + - '}'; - } } diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java similarity index 90% rename from client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java rename to client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java index 4286cd211cb..e000826ff9a 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merge; +package com.metamx.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -30,14 +30,14 @@ import java.util.List; /** */ -public class ClientDefaultMergeQuery implements ClientMergeQuery +public class ClientMergeQuery { private final String dataSource; private final List segments; private final List aggregators; @JsonCreator - public ClientDefaultMergeQuery( + public ClientMergeQuery( @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments, @JsonProperty("aggregations") List aggregators @@ -50,14 +50,18 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery } @JsonProperty - @Override + public String getType() + { + return "append"; + } + + @JsonProperty public String getDataSource() { return dataSource; } @JsonProperty - @Override public List getSegments() { return segments; @@ -72,7 +76,7 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery @Override public String toString() { - return "ClientDefaultMergeQuery{" + + return "ClientMergeQuery{" + "dataSource='" + dataSource + '\'' + ", segments=" + segments + ", aggregators=" + aggregators + diff --git a/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java b/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java new file mode 100644 index 00000000000..b659148d338 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java @@ -0,0 +1,115 @@ +/* + * 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.client.indexing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.metamx.common.IAE; +import com.metamx.druid.client.DataSegment; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.InputStreamResponseHandler; +import com.netflix.curator.x.discovery.ServiceInstance; +import com.netflix.curator.x.discovery.ServiceProvider; +import org.joda.time.Interval; + + +import java.io.InputStream; +import java.net.URL; +import java.util.Iterator; +import java.util.List; + +public class IndexingServiceClient +{ + private static final InputStreamResponseHandler RESPONSE_HANDLER = new InputStreamResponseHandler(); + + private final HttpClient client; + private final ObjectMapper jsonMapper; + private final ServiceProvider serviceProvider; + + public IndexingServiceClient( + HttpClient client, + ObjectMapper jsonMapper, + ServiceProvider serviceProvider + ) + { + this.client = client; + this.jsonMapper = jsonMapper; + this.serviceProvider = serviceProvider; + } + + public void mergeSegments(List segments) + { + final Iterator segmentsIter = segments.iterator(); + if (!segmentsIter.hasNext()) { + return; + } + + final String dataSource = segmentsIter.next().getDataSource(); + while (segmentsIter.hasNext()) { + DataSegment next = segmentsIter.next(); + if (!dataSource.equals(next.getDataSource())) { + throw new IAE("Cannot merge segments of different dataSources[%s] and [%s]", dataSource, next.getDataSource()); + } + } + + runQuery("merge", new ClientAppendQuery(dataSource, segments)); + } + + public void killSegments(String dataSource, Interval interval) + { + runQuery("index", new ClientKillQuery(dataSource, interval)); + } + + public void upgradeSegment(DataSegment dataSegment) + { + runQuery("task", new ClientConversionQuery(dataSegment)); + } + + public void upgradeSegments(String dataSource, Interval interval) + { + runQuery("task", new ClientConversionQuery(dataSource, interval)); + } + + private InputStream runQuery(String endpoint, Object queryObject) + { + try { + return client.post(new URL(String.format("%s/%s", baseUrl(), endpoint))) + .setContent("application/json", jsonMapper.writeValueAsBytes(queryObject)) + .go(RESPONSE_HANDLER) + .get(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + + private String baseUrl() + { + try { + final ServiceInstance instance = serviceProvider.getInstance(); + return String.format("http://%s:%s/mmx/merger/v1", instance.getAddress(), instance.getPort()); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java deleted file mode 100644 index 02e0a7bd141..00000000000 --- a/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java +++ /dev/null @@ -1,41 +0,0 @@ -package com.metamx.druid.merge; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Interval; - -/** - */ -public class ClientKillQuery -{ - private final String dataSource; - private final Interval interval; - - @JsonCreator - public ClientKillQuery( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval - ) - { - this.dataSource = dataSource; - this.interval = interval; - } - - @JsonProperty - public String getType() - { - return "kill"; - } - - @JsonProperty - public String getDataSource() - { - return dataSource; - } - - @JsonProperty - public Interval getInterval() - { - return interval; - } -} diff --git a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java deleted file mode 100644 index 3956991ed6b..00000000000 --- a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java +++ /dev/null @@ -1,41 +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.merge; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.client.DataSegment; - - - -import java.util.List; - -/** - */ -@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class) -@JsonSubTypes(value={ - @JsonSubTypes.Type(name="append", value=ClientAppendQuery.class) -}) -public interface ClientMergeQuery -{ - public String getDataSource(); - - public List getSegments(); -} diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManager.java b/common/src/main/java/com/metamx/druid/config/ConfigManager.java new file mode 100644 index 00000000000..3073c13e9fb --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/ConfigManager.java @@ -0,0 +1,259 @@ +package com.metamx.druid.config; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.common.collect.Maps; +import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import org.joda.time.Duration; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.IDBI; +import org.skife.jdbi.v2.StatementContext; +import org.skife.jdbi.v2.tweak.HandleCallback; +import org.skife.jdbi.v2.tweak.ResultSetMapper; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; + +/** + */ +public class ConfigManager +{ + private static final Logger log = new Logger(ConfigManager.class); + + private final Object lock = new Object(); + private boolean started = false; + + private final IDBI dbi; + private final ConfigManagerConfig config; + + private final ScheduledExecutorService exec; + private final ConcurrentMap watchedConfigs; + private final String selectStatement; + + public ConfigManager(IDBI dbi, ConfigManagerConfig config) + { + this.dbi = dbi; + this.config = config; + + this.exec = ScheduledExecutors.fixed(1, "config-manager-%s"); + this.watchedConfigs = Maps.newConcurrentMap(); + this.selectStatement = String.format("SELECT payload FROM %s WHERE name = :name", config.getConfigTable()); + } + + @LifecycleStart + public void start() + { + synchronized (lock) { + if (started) { + return; + } + + ScheduledExecutors.scheduleWithFixedDelay( + exec, + new Duration(0), + config.getPollDuration(), + new Runnable() + { + @Override + public void run() + { + poll(); + } + } + ); + + started = true; + } + } + + @LifecycleStop + public void stop() + { + synchronized (lock) { + if (!started) { + return; + } + + started = false; + } + } + + private void poll() + { + for (Map.Entry entry : watchedConfigs.entrySet()) { + try { + if (entry.getValue().swapIfNew(lookup(entry.getKey()))) { + log.info("New value for key[%s] seen.", entry.getKey()); + } + } + catch (Exception e) { + log.warn(e, "Exception when checking property[%s]", entry.getKey()); + } + } + } + + @SuppressWarnings("unchecked") + public AtomicReference watchConfig(final String key, final ConfigSerde serde) + { + ConfigHolder holder = watchedConfigs.get(key); + if (holder == null) { + try { + log.info("Creating watch for key[%s]", key); + + holder = exec.submit( + new Callable>() + { + @Override + @SuppressWarnings("unchecked") + public ConfigHolder call() throws Exception + { + if (!started) { + watchedConfigs.put(key, new ConfigHolder(null, serde)); + } + else { + try { + // Multiple of these callables can be submitted at the same time, but the callables themselves + // are executed serially, so double check that it hasn't already been populated. + if (!watchedConfigs.containsKey(key)) { + byte[] value = lookup(key); + ConfigHolder holder = new ConfigHolder(value, serde); + watchedConfigs.put(key, holder); + } + } + catch (Exception e) { + log.warn(e, "Failed loading config for key[%s]", key); + watchedConfigs.put(key, new ConfigHolder(null, serde)); + } + } + + return watchedConfigs.get(key); + } + } + ).get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e); + } + } + + return holder.getReference(); + } + + public byte[] lookup(final String key) + { + return dbi.withHandle( + new HandleCallback() + { + @Override + public byte[] withHandle(Handle handle) throws Exception + { + return handle.createQuery(selectStatement) + .bind("name", key) + .map( + new ResultSetMapper() + { + @Override + public byte[] map(int index, ResultSet r, StatementContext ctx) throws SQLException + { + return r.getBytes("payload"); + } + } + ) + .first(); + } + } + ); + } + + public boolean set(final String key, final ConfigSerde serde, final T obj) + { + if (obj == null) { + return false; + } + + final byte[] newBytes = serde.serialize(obj); + + try { + return exec.submit( + new Callable() + { + @Override + public Boolean call() throws Exception + { + dbi.withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception + { + handle.createStatement("INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload") + .bind("name", key) + .bind("payload", newBytes) + .execute(); + return null; + } + } + ); + + final ConfigHolder configHolder = watchedConfigs.get(key); + if (configHolder != null) { + configHolder.swapIfNew(newBytes); + } + + return true; + } + } + ).get(); + } + catch (Exception e) { + log.warn(e, "Failed to set[%s]", key); + return false; + } + } + + private static class ConfigHolder + { + private final AtomicReference rawBytes; + private final ConfigSerde serde; + private final AtomicReference reference; + + ConfigHolder( + byte[] rawBytes, + ConfigSerde serde + ) + { + this.rawBytes = new AtomicReference(rawBytes); + this.serde = serde; + this.reference = new AtomicReference(serde.deserialize(rawBytes)); + } + + public AtomicReference getReference() + { + return reference; + } + + public boolean swapIfNew(byte[] newBytes) + { + if (!Arrays.equals(newBytes, rawBytes.get())) { + reference.set(serde.deserialize(newBytes)); + rawBytes.set(newBytes); + return true; + } + return false; + } + } +} diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java b/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java new file mode 100644 index 00000000000..24706a83fb7 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java @@ -0,0 +1,18 @@ +package com.metamx.druid.config; + +import org.joda.time.Duration; +import org.skife.config.Config; +import org.skife.config.Default; + +/** + */ +public abstract class ConfigManagerConfig +{ + @Config("druid.indexer.configTable") + public abstract String getConfigTable(); + + @Config("druid.indexer.poll.duration") + @Default("PT1M") + public abstract Duration getPollDuration(); + +} diff --git a/common/src/main/java/com/metamx/druid/config/ConfigSerde.java b/common/src/main/java/com/metamx/druid/config/ConfigSerde.java new file mode 100644 index 00000000000..95f0a1ee7d3 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/ConfigSerde.java @@ -0,0 +1,9 @@ +package com.metamx.druid.config; + +/** +*/ +public interface ConfigSerde +{ + public byte[] serialize(T obj); + public T deserialize(byte[] bytes); +} diff --git a/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java b/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java new file mode 100644 index 00000000000..8e322f3ee80 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java @@ -0,0 +1,134 @@ +/* + * 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.config; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicReference; + +/** + */ +public class JacksonConfigManager +{ + private final ConfigManager configManager; + private final ObjectMapper jsonMapper; + + public JacksonConfigManager( + ConfigManager configManager, + ObjectMapper jsonMapper + ) + { + this.configManager = configManager; + this.jsonMapper = jsonMapper; + } + + public AtomicReference watch(String key, Class clazz) + { + return watch(key, clazz, null); + } + + public AtomicReference watch(String key, Class clazz, T defaultVal) + { + return configManager.watchConfig(key, create(clazz, defaultVal)); + } + + public AtomicReference watch(String key, TypeReference clazz) + { + return watch(key, clazz, null); + } + + public AtomicReference watch(String key, TypeReference clazz, T defaultVal) + { + return configManager.watchConfig(key, create(clazz, defaultVal)); + } + + public boolean set(String key, T val) + { + return configManager.set(key, create(val.getClass(), null), val); + } + + private ConfigSerde create(final Class clazz, final T defaultVal) + { + return new ConfigSerde() + { + @Override + public byte[] serialize(T obj) + { + try { + return jsonMapper.writeValueAsBytes(obj); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + } + + @Override + public T deserialize(byte[] bytes) + { + if (bytes == null) { + return defaultVal; + } + + try { + return jsonMapper.readValue(bytes, clazz); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + }; + } + + private ConfigSerde create(final TypeReference clazz, final T defaultVal) + { + return new ConfigSerde() + { + @Override + public byte[] serialize(T obj) + { + try { + return jsonMapper.writeValueAsBytes(obj); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + } + + @Override + public T deserialize(byte[] bytes) + { + if (bytes == null) { + return defaultVal; + } + + try { + return jsonMapper.readValue(bytes, clazz); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + }; + } +} diff --git a/common/src/main/java/com/metamx/druid/db/DbConnector.java b/common/src/main/java/com/metamx/druid/db/DbConnector.java index 73013ce6aa2..b8ab7a4747e 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -65,7 +65,7 @@ public class DbConnector dbi, configTableName, String.format( - "CREATE table %s (name VARCHAR(255) NOT NULL, payload LONGTEXT NOT NULL, PRIMARY KEY(name))", + "CREATE table %s (name VARCHAR(255) NOT NULL, payload BLOB NOT NULL, PRIMARY KEY(name))", configTableName ) ); @@ -84,12 +84,7 @@ public class DbConnector @Override public Void withHandle(Handle handle) throws Exception { - List> table = handle.select( - String.format( - "SHOW tables LIKE '%s'", - tableName - ) - ); + List> table = handle.select(String.format("SHOW tables LIKE '%s'", tableName)); if (table.isEmpty()) { log.info("Creating table[%s]", tableName); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java index 48ccdda13de..5d15269677a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java @@ -44,7 +44,7 @@ import java.util.Map; /** */ -public class AppendTask extends MergeTask +public class AppendTask extends MergeTaskBase { @JsonCreator public AppendTask( diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java deleted file mode 100644 index a46c24e91f3..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java +++ /dev/null @@ -1,87 +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.task; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -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.metamx.druid.aggregation.AggregatorFactory; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.QueryableIndex; -import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.index.v1.IndexMerger; - -import javax.annotation.Nullable; -import java.io.File; -import java.util.List; -import java.util.Map; - -/** - */ -public class DefaultMergeTask extends MergeTask -{ - private final List aggregators; - - @JsonCreator - public DefaultMergeTask( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("segments") List segments, - @JsonProperty("aggregations") List aggregators - ) - { - super(dataSource, segments); - this.aggregators = aggregators; - } - - @Override - public File merge(final Map segments, final File outDir) - throws Exception - { - return IndexMerger.mergeQueryableIndex( - Lists.transform( - ImmutableList.copyOf(segments.values()), - new Function() - { - @Override - public QueryableIndex apply(@Nullable File input) - { - try { - return IndexIO.loadIndex(input); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ), - aggregators.toArray(new AggregatorFactory[aggregators.size()]), - outDir - ); - } - - @Override - public String getType() - { - return "merge"; - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 561cb940639..4e6102f666b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -19,301 +19,71 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.common.base.Charsets; import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Objects; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; -import com.google.common.collect.Sets; -import com.google.common.hash.Hashing; -import com.metamx.common.ISE; -import com.metamx.common.logger.Logger; +import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LockListAction; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; -import com.metamx.druid.shard.NoneShardSpec; -import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.service.AlertEvent; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; -import org.joda.time.DateTime; -import org.joda.time.Interval; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.index.v1.IndexMerger; + + import javax.annotation.Nullable; import java.io.File; import java.util.List; import java.util.Map; -import java.util.Set; /** */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "append", value = AppendTask.class) -}) -public abstract class MergeTask extends AbstractTask +public class MergeTask extends MergeTaskBase { - private final List segments; + private final List aggregators; - private static final EmittingLogger log = new EmittingLogger(MergeTask.class); - - protected MergeTask(final String dataSource, final List segments) - { - super( - // _not_ the version, just something uniqueish - String.format("merge_%s_%s", computeProcessingID(dataSource, segments), new DateTime().toString()), - dataSource, - computeMergedInterval(segments) - ); - - // Verify segment list is nonempty - Preconditions.checkArgument(segments.size() > 0, "segments nonempty"); - // Verify segments are all in the correct datasource - Preconditions.checkArgument( - Iterables.size( - Iterables.filter( - segments, - new Predicate() - { - @Override - public boolean apply(@Nullable DataSegment segment) - { - return segment == null || !segment.getDataSource().equalsIgnoreCase(dataSource); - } - } - ) - ) == 0, "segments in the wrong datasource" - ); - // Verify segments are all unsharded - Preconditions.checkArgument( - Iterables.size( - Iterables.filter( - segments, - new Predicate() - { - @Override - public boolean apply(@Nullable DataSegment segment) - { - return segment == null || !(segment.getShardSpec() instanceof NoneShardSpec); - } - } - ) - ) == 0, "segments without NoneShardSpec" - ); - - this.segments = segments; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception - { - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); - 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(); - - try { - - final long startTime = System.currentTimeMillis(); - - log.info( - "Starting merge of id[%s], segments: %s", - getId(), - Lists.transform( - segments, - new Function() - { - @Override - public String apply(@Nullable DataSegment input) - { - return input.getIdentifier(); - } - } - ) - ); - - - // download segments to merge - final Map gettedSegments = toolbox.getSegments(segments); - - // merge files together - final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); - - emitter.emit(builder.build("merger/numMerged", segments.size())); - emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime)); - - log.info( - "[%s] : Merged %d segments in %,d millis", - mergedSegment.getDataSource(), - segments.size(), - System.currentTimeMillis() - startTime - ); - - long uploadStart = System.currentTimeMillis(); - - // Upload file - final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, mergedSegment); - - emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); - emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - - toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); - - return TaskStatus.success(getId()); - } - catch (Exception e) { - log.makeAlert(e, "Exception merging[%s]", mergedSegment.getDataSource()) - .addData("interval", mergedSegment.getInterval()) - .emit(); - - return TaskStatus.failure(getId()); - } - } - - /** - * Checks pre-existing segments in "context" to confirm that this merge query is valid. Specifically, confirm that - * we are operating on every segment that overlaps the chosen interval. - */ - @Override - public TaskStatus preflight(TaskToolbox toolbox) - { - final Function toIdentifier = new Function() - { - @Override - public String apply(DataSegment dataSegment) - { - return dataSegment.getIdentifier(); - } - }; - - final Set current = ImmutableSet.copyOf( - Iterables.transform(toolbox.getTaskActionClientFactory().submit(defaultListUsedAction()), toIdentifier) - ); - final Set requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier)); - - final Set missingFromRequested = Sets.difference(current, requested); - if (!missingFromRequested.isEmpty()) { - throw new ISE( - "Merge is invalid: current segment(s) are not in the requested set: %s", - Joiner.on(", ").join(missingFromRequested) - ); - } - - final Set missingFromCurrent = Sets.difference(requested, current); - if (!missingFromCurrent.isEmpty()) { - throw new ISE( - "Merge is invalid: requested segment(s) are not in the current set: %s", - Joiner.on(", ").join(missingFromCurrent) - ); - } - - return TaskStatus.running(getId()); - - } - - protected abstract File merge(Map segments, File outDir) - throws Exception; - - @JsonProperty - public List getSegments() - { - return segments; - } - - @Override - public String toString() - { - return Objects.toStringHelper(this) - .add("id", getId()) - .add("dataSource", getDataSource()) - .add("interval", getImplicitLockInterval()) - .add("segments", segments) - .toString(); - } - - private static String computeProcessingID(final String dataSource, final List segments) - { - final String segmentIDs = Joiner.on("_").join( - Iterables.transform( - Ordering.natural().sortedCopy(segments), new Function() - { - @Override - public String apply(DataSegment x) - { - return String.format( - "%s_%s_%s_%s", - x.getInterval().getStart(), - x.getInterval().getEnd(), - x.getVersion(), - x.getShardSpec().getPartitionNum() - ); - } - } - ) - ); - - return String.format( - "%s_%s", - dataSource, - Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase() - ); - } - - private static Interval computeMergedInterval(final List segments) - { - Preconditions.checkArgument(segments.size() > 0, "segments.size() > 0"); - - DateTime start = null; - DateTime end = null; - - for(final DataSegment segment : segments) { - if(start == null || segment.getInterval().getStart().isBefore(start)) { - start = segment.getInterval().getStart(); - } - - if(end == null || segment.getInterval().getEnd().isAfter(end)) { - end = segment.getInterval().getEnd(); - } - } - - return new Interval(start, end); - } - - private static DataSegment computeMergedSegment( - final String dataSource, - final String version, - final List segments + @JsonCreator + public MergeTask( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segments") List segments, + @JsonProperty("aggregations") List aggregators ) { - final Interval mergedInterval = computeMergedInterval(segments); - final Set mergedDimensions = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); - final Set mergedMetrics = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + super(dataSource, segments); + this.aggregators = aggregators; + } - for (DataSegment segment : segments) { - mergedDimensions.addAll(segment.getDimensions()); - mergedMetrics.addAll(segment.getMetrics()); - } + @Override + public File merge(final Map segments, final File outDir) + throws Exception + { + return IndexMerger.mergeQueryableIndex( + Lists.transform( + ImmutableList.copyOf(segments.values()), + new Function() + { + @Override + public QueryableIndex apply(@Nullable File input) + { + try { + return IndexIO.loadIndex(input); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ), + aggregators.toArray(new AggregatorFactory[aggregators.size()]), + outDir + ); + } - return DataSegment.builder() - .dataSource(dataSource) - .interval(mergedInterval) - .version(version) - .shardSpec(new NoneShardSpec()) - .dimensions(Lists.newArrayList(mergedDimensions)) - .metrics(Lists.newArrayList(mergedMetrics)) - .build(); + @Override + public String getType() + { + return "merge"; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java new file mode 100644 index 00000000000..e0b3dd6ff17 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java @@ -0,0 +1,315 @@ +/* + * 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.task; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.base.Charsets; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; +import com.google.common.hash.Hashing; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskLock; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.LockListAction; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import com.metamx.druid.shard.NoneShardSpec; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.service.AlertEvent; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + */ +public abstract class MergeTaskBase extends AbstractTask +{ + private final List segments; + + private static final EmittingLogger log = new EmittingLogger(MergeTaskBase.class); + + protected MergeTaskBase(final String dataSource, final List segments) + { + super( + // _not_ the version, just something uniqueish + String.format("merge_%s_%s", computeProcessingID(dataSource, segments), new DateTime().toString()), + dataSource, + computeMergedInterval(segments) + ); + + // Verify segment list is nonempty + Preconditions.checkArgument(segments.size() > 0, "segments nonempty"); + // Verify segments are all in the correct datasource + Preconditions.checkArgument( + Iterables.size( + Iterables.filter( + segments, + new Predicate() + { + @Override + public boolean apply(@Nullable DataSegment segment) + { + return segment == null || !segment.getDataSource().equalsIgnoreCase(dataSource); + } + } + ) + ) == 0, "segments in the wrong datasource" + ); + // Verify segments are all unsharded + Preconditions.checkArgument( + Iterables.size( + Iterables.filter( + segments, + new Predicate() + { + @Override + public boolean apply(@Nullable DataSegment segment) + { + return segment == null || !(segment.getShardSpec() instanceof NoneShardSpec); + } + } + ) + ) == 0, "segments without NoneShardSpec" + ); + + this.segments = segments; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); + 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(); + + try { + + final long startTime = System.currentTimeMillis(); + + log.info( + "Starting merge of id[%s], segments: %s", + getId(), + Lists.transform( + segments, + new Function() + { + @Override + public String apply(@Nullable DataSegment input) + { + return input.getIdentifier(); + } + } + ) + ); + + + // download segments to merge + final Map gettedSegments = toolbox.getSegments(segments); + + // merge files together + final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); + + emitter.emit(builder.build("merger/numMerged", segments.size())); + emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime)); + + log.info( + "[%s] : Merged %d segments in %,d millis", + mergedSegment.getDataSource(), + segments.size(), + System.currentTimeMillis() - startTime + ); + + long uploadStart = System.currentTimeMillis(); + + // Upload file + final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, mergedSegment); + + emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); + emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); + + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); + + return TaskStatus.success(getId()); + } + catch (Exception e) { + log.makeAlert(e, "Exception merging[%s]", mergedSegment.getDataSource()) + .addData("interval", mergedSegment.getInterval()) + .emit(); + + return TaskStatus.failure(getId()); + } + } + + /** + * Checks pre-existing segments in "context" to confirm that this merge query is valid. Specifically, confirm that + * we are operating on every segment that overlaps the chosen interval. + */ + @Override + public TaskStatus preflight(TaskToolbox toolbox) + { + final Function toIdentifier = new Function() + { + @Override + public String apply(DataSegment dataSegment) + { + return dataSegment.getIdentifier(); + } + }; + + final Set current = ImmutableSet.copyOf( + Iterables.transform(toolbox.getTaskActionClientFactory().submit(defaultListUsedAction()), toIdentifier) + ); + final Set requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier)); + + final Set missingFromRequested = Sets.difference(current, requested); + if (!missingFromRequested.isEmpty()) { + throw new ISE( + "Merge is invalid: current segment(s) are not in the requested set: %s", + Joiner.on(", ").join(missingFromRequested) + ); + } + + final Set missingFromCurrent = Sets.difference(requested, current); + if (!missingFromCurrent.isEmpty()) { + throw new ISE( + "Merge is invalid: requested segment(s) are not in the current set: %s", + Joiner.on(", ").join(missingFromCurrent) + ); + } + + return TaskStatus.running(getId()); + + } + + protected abstract File merge(Map segments, File outDir) + throws Exception; + + @JsonProperty + public List getSegments() + { + return segments; + } + + @Override + public String toString() + { + return Objects.toStringHelper(this) + .add("id", getId()) + .add("dataSource", getDataSource()) + .add("interval", getImplicitLockInterval()) + .add("segments", segments) + .toString(); + } + + private static String computeProcessingID(final String dataSource, final List segments) + { + final String segmentIDs = Joiner.on("_").join( + Iterables.transform( + Ordering.natural().sortedCopy(segments), new Function() + { + @Override + public String apply(DataSegment x) + { + return String.format( + "%s_%s_%s_%s", + x.getInterval().getStart(), + x.getInterval().getEnd(), + x.getVersion(), + x.getShardSpec().getPartitionNum() + ); + } + } + ) + ); + + return String.format( + "%s_%s", + dataSource, + Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase() + ); + } + + private static Interval computeMergedInterval(final List segments) + { + Preconditions.checkArgument(segments.size() > 0, "segments.size() > 0"); + + DateTime start = null; + DateTime end = null; + + for(final DataSegment segment : segments) { + if(start == null || segment.getInterval().getStart().isBefore(start)) { + start = segment.getInterval().getStart(); + } + + if(end == null || segment.getInterval().getEnd().isAfter(end)) { + end = segment.getInterval().getEnd(); + } + } + + return new Interval(start, end); + } + + private static DataSegment computeMergedSegment( + final String dataSource, + final String version, + final List segments + ) + { + final Interval mergedInterval = computeMergedInterval(segments); + final Set mergedDimensions = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + final Set mergedMetrics = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + + for (DataSegment segment : segments) { + mergedDimensions.addAll(segment.getDimensions()); + mergedMetrics.addAll(segment.getMetrics()); + } + + return DataSegment.builder() + .dataSource(dataSource) + .interval(mergedInterval) + .version(version) + .shardSpec(new NoneShardSpec()) + .dimensions(Lists.newArrayList(mergedDimensions)) + .metrics(Lists.newArrayList(mergedMetrics)) + .build(); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 8418ecf40a8..5f288be99dc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -41,10 +41,10 @@ import org.joda.time.Interval; * to release locks early if they desire. * */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "append", value = AppendTask.class), - @JsonSubTypes.Type(name = "merge", value = DefaultMergeTask.class), + @JsonSubTypes.Type(name = "merge", value = MergeTask.class), @JsonSubTypes.Type(name = "delete", value = DeleteTask.class), @JsonSubTypes.Type(name = "kill", value = KillTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), @@ -52,7 +52,7 @@ import org.joda.time.Interval; @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class), - @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterSubTask.class) + @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class) }) public interface Task { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java deleted file mode 100644 index 2099d903d33..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterSubTask.java +++ /dev/null @@ -1,94 +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.task; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.Sets; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; -import org.joda.time.DateTime; - -import java.io.File; -import java.util.Arrays; -import java.util.Map; - -/** - */ -public class VersionConverterSubTask extends AbstractTask -{ - private static final Logger log = new Logger(VersionConverterSubTask.class); - - private final DataSegment segment; - - protected VersionConverterSubTask( - @JsonProperty("groupId") String groupId, - @JsonProperty("segment") DataSegment segment - ) - { - super( - joinId( - groupId, - "sub", - segment.getInterval().getStart(), - segment.getInterval().getEnd(), - segment.getShardSpec().getPartitionNum() - ), - groupId, - segment.getDataSource(), - segment.getInterval() - ); - this.segment = segment; - } - - @Override - public String getType() - { - return "version_converter_sub"; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception - { - log.info("Converting segment[%s]", segment); - final Map localSegments = toolbox.getSegments(Arrays.asList(segment)); - - final File location = localSegments.get(segment); - final File outLocation = new File(location, "v9_out"); - if (IndexIO.convertSegment(location, outLocation)) { - final int outVersion = IndexIO.getVersionFromDir(outLocation); - - // Appending to the version makes a new version that inherits most comparability parameters of the original - // version, but is "newer" than said original version. - DataSegment updatedSegment = segment.withVersion(String.format("%s_v%s", segment.getVersion(), outVersion)); - updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment); - - toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); - } - else { - log.info("Conversion failed."); - } - - return success(); - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java index 3859190a1bb..c8c0e2cbf42 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java @@ -22,19 +22,26 @@ package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; import com.metamx.druid.merger.common.actions.SpawnTasksAction; import com.metamx.druid.merger.common.actions.TaskActionClient; import org.joda.time.DateTime; import org.joda.time.Interval; +import java.io.File; +import java.io.IOException; +import java.util.Arrays; import java.util.List; +import java.util.Map; /** */ @@ -44,10 +51,12 @@ public class VersionConverterTask extends AbstractTask private static final Integer CURR_VERSION_INTEGER = new Integer(IndexIO.CURRENT_VERSION_ID); private static final Logger log = new Logger(VersionConverterTask.class); + private final DataSegment segment; public VersionConverterTask( @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval + @JsonProperty("interval") Interval interval, + @JsonProperty("segment") DataSegment segment ) { super( @@ -55,6 +64,8 @@ public class VersionConverterTask extends AbstractTask dataSource, interval ); + + this.segment = segment; } @Override @@ -66,12 +77,22 @@ public class VersionConverterTask extends AbstractTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - throw new ISE("Should never run, %s just exists to create subtasks", this.getClass().getSimpleName()); + if (segment == null) { + throw new ISE("Segment was null, this should never run.", this.getClass().getSimpleName()); + } + + log.info("I'm in a subless mood."); + convertSegment(toolbox, segment); + return success(); } @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { + if (segment != null) { + return super.preflight(toolbox); + } + final TaskActionClient taskClient = toolbox.getTaskActionClientFactory(); List segments = taskClient.submit(defaultListUsedAction()); @@ -86,7 +107,7 @@ public class VersionConverterTask extends AbstractTask { final Integer segmentVersion = segment.getBinaryVersion(); if (!CURR_VERSION_INTEGER.equals(segmentVersion)) { - return new VersionConverterSubTask(getGroupId(), segment); + return new SubTask(getGroupId(), segment); } log.info("Skipping[%s], already version[%s]", segment.getIdentifier(), segmentVersion); @@ -99,4 +120,65 @@ public class VersionConverterTask extends AbstractTask return TaskStatus.success(getId()); } + + public static class SubTask extends AbstractTask + { + private final DataSegment segment; + + protected SubTask( + @JsonProperty("groupId") String groupId, + @JsonProperty("segment") DataSegment segment + ) + { + super( + joinId( + groupId, + "sub", + segment.getInterval().getStart(), + segment.getInterval().getEnd(), + segment.getShardSpec().getPartitionNum() + ), + groupId, + segment.getDataSource(), + segment.getInterval() + ); + this.segment = segment; + } + + @Override + public String getType() + { + return "version_converter_sub"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + log.info("Subs are good! Italian BMT and Meatball are probably my favorite."); + convertSegment(toolbox, segment); + return success(); + } + } + + private static void convertSegment(TaskToolbox toolbox, final DataSegment segment) + throws SegmentLoadingException, IOException + { + log.info("Converting segment[%s]", segment); + final Map localSegments = toolbox.getSegments(Arrays.asList(segment)); + + final File location = localSegments.get(segment); + final File outLocation = new File(location, "v9_out"); + if (IndexIO.convertSegment(location, outLocation)) { + final int outVersion = IndexIO.getVersionFromDir(outLocation); + + // Appending to the version makes a new version that inherits most comparability parameters of the original + // version, but is "newer" than said original version. + DataSegment updatedSegment = segment.withVersion(String.format("%s_v%s", segment.getVersion(), outVersion)); + updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment); + + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); + } else { + log.info("Conversion failed."); + } + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java deleted file mode 100644 index 16eeb1c3439..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.merger.coordinator.config; - -import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; - -/** - */ -public abstract class WorkerSetupManagerConfig -{ - @Config("druid.indexer.configTable") - public abstract String getConfigTable(); - - @Config("druid.indexer.workerSetupConfigName") - public abstract String getWorkerSetupConfigName(); - - @Config("druid.indexer.poll.duration") - @Default("PT1M") - public abstract Duration getPollDuration(); -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 286d2e1649a..2112e3b6e37 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -39,6 +39,9 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.RegisteringNode; +import com.metamx.druid.config.ConfigManager; +import com.metamx.druid.config.ConfigManagerConfig; +import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.http.GuiceServletConfig; @@ -78,7 +81,6 @@ import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; -import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.NoopAutoScalingStrategy; @@ -566,18 +568,12 @@ public class IndexerCoordinatorNode extends RegisteringNode public void initializeWorkerSetupManager() { if (workerSetupManager == null) { - final WorkerSetupManagerConfig workerSetupManagerConfig = configFactory.build(WorkerSetupManagerConfig.class); + final ConfigManagerConfig configManagerConfig = configFactory.build(ConfigManagerConfig.class); + final ConfigManager configManager = new ConfigManager(dbi, configManagerConfig); + lifecycle.addManagedInstance(configManager); - DbConnector.createConfigTable(dbi, workerSetupManagerConfig.getConfigTable()); - workerSetupManager = new WorkerSetupManager( - dbi, Executors.newScheduledThreadPool( - 1, - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("WorkerSetupManagerExec--%d") - .build() - ), jsonMapper, workerSetupManagerConfig - ); + DbConnector.createConfigTable(dbi, configManagerConfig.getConfigTable()); + workerSetupManager = new WorkerSetupManager(new JacksonConfigManager(configManager, jsonMapper)); } lifecycle.addManagedInstance(workerSetupManager); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index 5f0714a8d61..d2d613c3ea3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -98,18 +98,15 @@ public class IndexerCoordinatorResource @Produces("application/json") public Response doIndex(final Task task) { - // verify against whitelist - if (config.isWhitelistEnabled() && !config.getWhitelistDatasources().contains(task.getDataSource())) { - return Response.status(Response.Status.BAD_REQUEST) - .entity( - ImmutableMap.of( - "error", - String.format("dataSource[%s] is not whitelisted", task.getDataSource()) - ) - ) - .build(); - } + return taskPost(task); + } + @POST + @Path("/task") + @Consumes("application/json") + @Produces("application/json") + public Response taskPost(final Task task) + { taskMasterLifecycle.getTaskQueue().add(task); return Response.ok(ImmutableMap.of("task", task.getId())).build(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java index 666bbaa998c..70c15bcada0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java @@ -1,3 +1,22 @@ +/* + * 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.scaling; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java index 89a0dd2d5c1..b9ce066c327 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java @@ -19,209 +19,41 @@ package com.metamx.druid.merger.coordinator.setup; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.common.logger.Logger; -import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; -import org.apache.commons.collections.MapUtils; +import com.metamx.druid.config.JacksonConfigManager; -import org.joda.time.Duration; -import org.skife.jdbi.v2.DBI; -import org.skife.jdbi.v2.FoldController; -import org.skife.jdbi.v2.Folder3; -import org.skife.jdbi.v2.Handle; -import org.skife.jdbi.v2.StatementContext; -import org.skife.jdbi.v2.tweak.HandleCallback; - -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; /** */ public class WorkerSetupManager { - private static final Logger log = new Logger(WorkerSetupManager.class); + private static final String WORKER_SETUP_KEY = "worker.setup"; - private final DBI dbi; - private final ObjectMapper jsonMapper; - private final ScheduledExecutorService exec; - private final WorkerSetupManagerConfig config; - - private final Object lock = new Object(); + private final JacksonConfigManager configManager; private volatile AtomicReference workerSetupData = new AtomicReference(null); - private volatile boolean started = false; public WorkerSetupManager( - DBI dbi, - ScheduledExecutorService exec, - ObjectMapper jsonMapper, - WorkerSetupManagerConfig config + JacksonConfigManager configManager ) { - this.dbi = dbi; - this.exec = exec; - this.jsonMapper = jsonMapper; - this.config = config; + this.configManager = configManager; } @LifecycleStart public void start() { - synchronized (lock) { - if (started) { - return; - } - - ScheduledExecutors.scheduleWithFixedDelay( - exec, - new Duration(0), - config.getPollDuration(), - new Runnable() - { - @Override - public void run() - { - poll(); - } - } - ); - - started = true; - } + workerSetupData = configManager.watch(WORKER_SETUP_KEY, WorkerSetupData.class); } - @LifecycleStop - public void stop() - { - synchronized (lock) { - if (!started) { - return; - } - - started = false; - } - } - - public void poll() - { - try { - List setupDataList = dbi.withHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) throws Exception - { - return handle.createQuery( - String.format( - "SELECT payload FROM %s WHERE name = :name", - config.getConfigTable() - ) - ) - .bind("name", config.getWorkerSetupConfigName()) - .fold( - Lists.newArrayList(), - new Folder3, Map>() - { - @Override - public ArrayList fold( - ArrayList workerNodeConfigurations, - Map stringObjectMap, - FoldController foldController, - StatementContext statementContext - ) throws SQLException - { - try { - // stringObjectMap lowercases and jackson may fail serde - workerNodeConfigurations.add( - jsonMapper.readValue( - MapUtils.getString(stringObjectMap, "payload"), - WorkerSetupData.class - ) - ); - return workerNodeConfigurations; - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ); - } - } - ); - - if (setupDataList.isEmpty()) { - throw new ISE("WTF?! No configuration found for worker nodes!"); - } else if (setupDataList.size() != 1) { - throw new ISE("WTF?! Found more than one configuration for worker nodes"); - } - - workerSetupData.set(setupDataList.get(0)); - } - catch (Exception e) { - log.error(e, "Exception while polling for worker setup data!"); - } - } - - @SuppressWarnings("unchecked") public WorkerSetupData getWorkerSetupData() { - synchronized (lock) { - if (!started) { - throw new ISE("Must start WorkerSetupManager first!"); - } - - return workerSetupData.get(); - } + return workerSetupData.get(); } public boolean setWorkerSetupData(final WorkerSetupData value) { - synchronized (lock) { - try { - if (!started) { - throw new ISE("Must start WorkerSetupManager first!"); - } - - dbi.withHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) throws Exception - { - handle.createStatement( - String.format( - "INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload", - config.getConfigTable() - ) - ) - .bind("name", config.getWorkerSetupConfigName()) - .bind("payload", jsonMapper.writeValueAsString(value)) - .execute(); - - return null; - } - } - ); - - workerSetupData.set(value); - } - catch (Exception e) { - log.error(e, "Exception updating worker config"); - return false; - } - } - - return true; + return configManager.set(WORKER_SETUP_KEY, value); } } diff --git a/merger/src/test/java/com/metamx/druid/merger/TestTask.java b/merger/src/test/java/com/metamx/druid/merger/TestTask.java index c23b498f739..d0a77cff447 100644 --- a/merger/src/test/java/com/metamx/druid/merger/TestTask.java +++ b/merger/src/test/java/com/metamx/druid/merger/TestTask.java @@ -26,14 +26,14 @@ import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.task.DefaultMergeTask; +import com.metamx.druid.merger.common.task.MergeTask; import java.util.List; /** */ @JsonTypeName("test") -public class TestTask extends DefaultMergeTask +public class TestTask extends MergeTask { private final String id; private final TaskStatus status; diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java similarity index 89% rename from merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java rename to merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java index 7c779f6a74c..a2f6e8175fb 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java @@ -31,7 +31,7 @@ import java.io.File; import java.util.List; import java.util.Map; -public class MergeTaskTest +public class MergeTaskBaseTest { private final DataSegment.Builder segmentBuilder = DataSegment.builder() .dataSource("foo") @@ -43,7 +43,7 @@ public class MergeTaskTest .add(segmentBuilder.interval(new Interval("2012-01-03/2012-01-05")).build()) .build(); - final MergeTask testMergeTask = new MergeTask("foo", segments) + final MergeTaskBase testMergeTaskBase = new MergeTaskBase("foo", segments) { @Override protected File merge(Map segments, File outDir) throws Exception @@ -61,13 +61,13 @@ public class MergeTaskTest @Test public void testDataSource() { - Assert.assertEquals("foo", testMergeTask.getDataSource()); + Assert.assertEquals("foo", testMergeTaskBase.getDataSource()); } @Test public void testInterval() { - Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getImplicitLockInterval().get()); + Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTaskBase.getImplicitLockInterval().get()); } @Test @@ -81,7 +81,7 @@ public class MergeTaskTest ).toString().toLowerCase() + "_"; Assert.assertEquals( desiredPrefix, - testMergeTask.getId().substring(0, desiredPrefix.length()) + testMergeTaskBase.getId().substring(0, desiredPrefix.length()) ); } } diff --git a/pom.xml b/pom.xml index 59a91567adb..078927877c6 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ UTF-8 - 0.20.0 + 0.20.1-SNAPSHOT diff --git a/server/src/main/java/com/metamx/druid/http/InfoResource.java b/server/src/main/java/com/metamx/druid/http/InfoResource.java index e434bff995d..acd94438d84 100644 --- a/server/src/main/java/com/metamx/druid/http/InfoResource.java +++ b/server/src/main/java/com/metamx/druid/http/InfoResource.java @@ -34,8 +34,8 @@ import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.master.DruidMaster; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.master.rules.Rule; -import com.metamx.druid.merge.ClientKillQuery; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -66,6 +66,7 @@ public class InfoResource private final DatabaseSegmentManager databaseSegmentManager; private final DatabaseRuleManager databaseRuleManager; private final DruidClusterInfo druidClusterInfo; + private final IndexingServiceClient indexingServiceClient; @Inject public InfoResource( @@ -73,7 +74,8 @@ public class InfoResource ServerInventoryManager serverInventoryManager, DatabaseSegmentManager databaseSegmentManager, DatabaseRuleManager databaseRuleManager, - DruidClusterInfo druidClusterInfo + DruidClusterInfo druidClusterInfo, + IndexingServiceClient indexingServiceClient ) { this.master = master; @@ -81,6 +83,7 @@ public class InfoResource this.databaseSegmentManager = databaseSegmentManager; this.databaseRuleManager = databaseRuleManager; this.druidClusterInfo = druidClusterInfo; + this.indexingServiceClient = indexingServiceClient; } @GET @@ -374,7 +377,7 @@ public class InfoResource ) { if (kill != null && Boolean.valueOf(kill)) { - master.killSegments(new ClientKillQuery(dataSourceName, new Interval(interval))); + indexingServiceClient.killSegments(dataSourceName, new Interval(interval)); } else { if (!databaseSegmentManager.removeDatasource(dataSourceName)) { return Response.status(Response.Status.NOT_FOUND).build(); diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index 64e63176d0c..32e03e7ba63 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -33,6 +33,9 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.client.ServerInventoryManagerConfig; +import com.metamx.druid.config.ConfigManager; +import com.metamx.druid.config.ConfigManagerConfig; +import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.coordination.DruidClusterInfoConfig; import com.metamx.druid.db.DatabaseRuleManager; @@ -49,6 +52,7 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterConfig; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.master.LoadQueuePeon; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; @@ -86,7 +90,7 @@ import java.util.concurrent.ScheduledExecutorService; */ public class MasterMain { - private static final Logger log = new Logger(ServerMain.class); + private static final Logger log = new Logger(MasterMain.class); public static void main(String[] args) throws Exception { @@ -166,13 +170,14 @@ public class MasterMain lifecycle ); - ServiceProvider serviceProvider = null; + IndexingServiceClient indexingServiceClient = null; if (druidMasterConfig.getMergerServiceName() != null) { - serviceProvider = Initialization.makeServiceProvider( + ServiceProvider serviceProvider = Initialization.makeServiceProvider( druidMasterConfig.getMergerServiceName(), serviceDiscovery, lifecycle ); + indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider); } final DruidClusterInfo druidClusterInfo = new DruidClusterInfo( @@ -180,10 +185,14 @@ public class MasterMain masterYp ); + JacksonConfigManager configManager = new JacksonConfigManager( + new ConfigManager(dbi, configFactory.build(ConfigManagerConfig.class)), jsonMapper + ); + final DruidMaster master = new DruidMaster( druidMasterConfig, druidClusterInfo, - jsonMapper, + configManager, databaseSegmentManager, serverInventoryManager, databaseRuleManager, @@ -191,9 +200,7 @@ public class MasterMain emitter, scheduledExecutorFactory, new ConcurrentHashMap(), - serviceProvider, - httpClient, - new ToStringResponseHandler(Charsets.UTF_8) + indexingServiceClient ); lifecycle.addManagedInstance(master); @@ -226,7 +233,8 @@ public class MasterMain databaseRuleManager, druidClusterInfo, master, - jsonMapper + jsonMapper, + indexingServiceClient ) ); diff --git a/server/src/main/java/com/metamx/druid/http/MasterResource.java b/server/src/main/java/com/metamx/druid/http/MasterResource.java index b725ed7f358..9bb59d79d43 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterResource.java +++ b/server/src/main/java/com/metamx/druid/http/MasterResource.java @@ -21,7 +21,6 @@ package com.metamx.druid.http; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.LoadPeonCallback; -import com.metamx.druid.merge.ClientKillQuery; import javax.inject.Inject; import javax.ws.rs.Consumes; diff --git a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java index 47395f73eeb..64d0c98afa3 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java +++ b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java @@ -27,6 +27,7 @@ import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.master.DruidMaster; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; @@ -44,6 +45,7 @@ public class MasterServletModule extends JerseyServletModule private final DruidClusterInfo druidClusterInfo; private final DruidMaster master; private final ObjectMapper jsonMapper; + private final IndexingServiceClient indexingServiceClient; public MasterServletModule( ServerInventoryManager serverInventoryManager, @@ -51,7 +53,8 @@ public class MasterServletModule extends JerseyServletModule DatabaseRuleManager databaseRuleManager, DruidClusterInfo druidClusterInfo, DruidMaster master, - ObjectMapper jsonMapper + ObjectMapper jsonMapper, + IndexingServiceClient indexingServiceClient ) { this.serverInventoryManager = serverInventoryManager; @@ -60,6 +63,7 @@ public class MasterServletModule extends JerseyServletModule this.druidClusterInfo = druidClusterInfo; this.master = master; this.jsonMapper = jsonMapper; + this.indexingServiceClient = indexingServiceClient; } @Override @@ -72,6 +76,7 @@ public class MasterServletModule extends JerseyServletModule bind(DatabaseRuleManager.class).toInstance(databaseRuleManager); bind(DruidMaster.class).toInstance(master); bind(DruidClusterInfo.class).toInstance(druidClusterInfo); + bind(IndexingServiceClient.class).toInstance(indexingServiceClient); serve("/*").with(GuiceContainer.class); } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index a21408144b3..74b9d17d57b 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -19,10 +19,8 @@ package com.metamx.druid.master; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Predicate; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -41,32 +39,30 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.ServerInventoryManager; +import com.metamx.druid.client.indexing.IndexingServiceClient; +import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; -import com.metamx.druid.merge.ClientKillQuery; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.response.HttpResponseHandler; import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; -import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.exception.ZkNodeExistsException; import org.joda.time.DateTime; import org.joda.time.Duration; import javax.annotation.Nullable; -import java.net.URL; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.Callable; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -83,26 +79,22 @@ public class DruidMaster private final DruidMasterConfig config; private final DruidClusterInfo clusterInfo; + private final JacksonConfigManager configManager; private final DatabaseSegmentManager databaseSegmentManager; private final ServerInventoryManager serverInventoryManager; private final DatabaseRuleManager databaseRuleManager; private final PhoneBook yp; private final ServiceEmitter emitter; + private final IndexingServiceClient indexingServiceClient; private final ScheduledExecutorService exec; private final ScheduledExecutorService peonExec; private final PhoneBookPeon masterPeon; private final Map loadManagementPeons; - private final ServiceProvider serviceProvider; - - private final HttpClient httpClient; - private final HttpResponseHandler responseHandler; - - private final ObjectMapper jsonMapper; public DruidMaster( DruidMasterConfig config, DruidClusterInfo clusterInfo, - ObjectMapper jsonMapper, + JacksonConfigManager configManager, DatabaseSegmentManager databaseSegmentManager, ServerInventoryManager serverInventoryManager, DatabaseRuleManager databaseRuleManager, @@ -110,31 +102,25 @@ public class DruidMaster ServiceEmitter emitter, ScheduledExecutorFactory scheduledExecutorFactory, Map loadManagementPeons, - ServiceProvider serviceProvider, - HttpClient httpClient, - HttpResponseHandler responseHandler + IndexingServiceClient indexingServiceClient ) { this.config = config; this.clusterInfo = clusterInfo; - - this.jsonMapper = jsonMapper; + this.configManager = configManager; this.databaseSegmentManager = databaseSegmentManager; this.serverInventoryManager = serverInventoryManager; this.databaseRuleManager = databaseRuleManager; this.yp = zkPhoneBook; this.emitter = emitter; + this.indexingServiceClient = indexingServiceClient; this.masterPeon = new MasterListeningPeon(); this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d"); this.peonExec = scheduledExecutorFactory.create(1, "Master-PeonExec--%d"); this.loadManagementPeons = loadManagementPeons; - - this.serviceProvider = serviceProvider; - this.httpClient = httpClient; - this.responseHandler = responseHandler; } public boolean isClusterMaster() @@ -349,27 +335,6 @@ public class DruidMaster } } - public void killSegments(ClientKillQuery killQuery) - { - try { - httpClient.post( - new URL( - String.format( - "http://%s:%s/mmx/merger/v1/index", - serviceProvider.getInstance().getAddress(), - serviceProvider.getInstance().getPort() - ) - ) - ) - .setContent("application/json", jsonMapper.writeValueAsBytes(killQuery)) - .go(responseHandler) - .get(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - public Set getAvailableDataSegments() { Set availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator())); @@ -390,7 +355,9 @@ public class DruidMaster for (DataSegment dataSegment : dataSegments) { if (dataSegment.getSize() < 0) { - log.warn("No size on Segment[%s], wtf?", dataSegment); + log.makeAlert("No size on Segment, wtf?") + .addData("segment", dataSegment) + .emit(); } availableSegments.add(dataSegment); } @@ -466,8 +433,14 @@ public class DruidMaster final List> masterRunnables = Lists.newArrayList(); masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod())); - if (config.isMergeSegments() && serviceProvider != null) { - masterRunnables.add(Pair.of(new MasterSegmentMergerRunnable(), config.getMasterSegmentMergerPeriod())); + if (config.isMergeSegments() && indexingServiceClient != null) { + + masterRunnables.add( + Pair.of( + new MasterSegmentMergerRunnable(configManager.watch(MergerWhitelist.CONFIG_KEY, MergerWhitelist.class)), + config.getMasterSegmentMergerPeriod() + ) + ); } for (final Pair masterRunnable : masterRunnables) { @@ -529,6 +502,39 @@ public class DruidMaster } } + public static class DruidMasterVersionConverter implements DruidMasterHelper + { + private final IndexingServiceClient indexingServiceClient; + private final AtomicReference whitelistRef; + + public DruidMasterVersionConverter( + IndexingServiceClient indexingServiceClient, + AtomicReference whitelistRef + ) + { + this.indexingServiceClient = indexingServiceClient; + this.whitelistRef = whitelistRef; + } + + @Override + public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + { + MergerWhitelist whitelist = whitelistRef.get(); + + for (DataSegment dataSegment : params.getAvailableSegments()) { + if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) { + final Integer binaryVersion = dataSegment.getBinaryVersion(); + + if (binaryVersion == null || binaryVersion < IndexIO.CURRENT_VERSION_ID) { + indexingServiceClient.upgradeSegment(dataSegment); + } + } + } + + return params; + } + } + private class MasterListeningPeon implements PhoneBookPeon { @Override @@ -723,12 +729,13 @@ public class DruidMaster private class MasterSegmentMergerRunnable extends MasterRunnable { - private MasterSegmentMergerRunnable() + private MasterSegmentMergerRunnable(final AtomicReference whitelistRef) { super( ImmutableList.of( new DruidMasterSegmentInfoLoader(DruidMaster.this), - new DruidMasterSegmentMerger(jsonMapper, serviceProvider), + new DruidMasterVersionConverter(indexingServiceClient, whitelistRef), + new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef), new DruidMasterHelper() { @Override @@ -739,8 +746,7 @@ public class DruidMaster params.getEmitter().emit( new ServiceMetricEvent.Builder().build( - "master/merge/count", - stats.getGlobalStats().get("mergedCount") + "master/merge/count", stats.getGlobalStats().get("mergedCount") ) ); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java index f3d93d0e6fd..ef4f9d1e6d3 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java @@ -19,8 +19,6 @@ package com.metamx.druid.master; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.HashMultiset; @@ -32,22 +30,19 @@ import com.google.common.collect.Multiset; import com.google.common.collect.Ordering; import com.metamx.common.Pair; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.partition.PartitionChunk; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; -import com.metamx.http.client.response.ToStringResponseHandler; -import com.netflix.curator.x.discovery.ServiceProvider; import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -55,44 +50,40 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper { private static final Logger log = new Logger(DruidMasterSegmentMerger.class); - private final MergerClient mergerClient; + private final IndexingServiceClient indexingServiceClient; + private final AtomicReference whiteListRef; - public DruidMasterSegmentMerger(MergerClient mergerClient) + public DruidMasterSegmentMerger( + IndexingServiceClient indexingServiceClient, + AtomicReference whitelistRef + ) { - this.mergerClient = mergerClient; - } - - public DruidMasterSegmentMerger(ObjectMapper jsonMapper, ServiceProvider serviceProvider) - { - this.mergerClient = new HttpMergerClient( - HttpClientInit.createClient( - HttpClientConfig.builder().withNumConnections(1).build(), - new Lifecycle() - ), - new ToStringResponseHandler(Charsets.UTF_8), - jsonMapper, - serviceProvider - ); + this.indexingServiceClient = indexingServiceClient; + this.whiteListRef = whitelistRef; } @Override public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) { + MergerWhitelist whitelist = whiteListRef.get(); + MasterStats stats = new MasterStats(); Map> dataSources = Maps.newHashMap(); // Find serviced segments by using a timeline for (DataSegment dataSegment : params.getAvailableSegments()) { - VersionedIntervalTimeline timeline = dataSources.get(dataSegment.getDataSource()); - if (timeline == null) { - timeline = new VersionedIntervalTimeline(Ordering.natural()); - dataSources.put(dataSegment.getDataSource(), timeline); + if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) { + VersionedIntervalTimeline timeline = dataSources.get(dataSegment.getDataSource()); + if (timeline == null) { + timeline = new VersionedIntervalTimeline(Ordering.natural()); + dataSources.put(dataSegment.getDataSource(), timeline); + } + timeline.add( + dataSegment.getInterval(), + dataSegment.getVersion(), + dataSegment.getShardSpec().createChunk(dataSegment) + ); } - timeline.add( - dataSegment.getInterval(), - dataSegment.getVersion(), - dataSegment.getShardSpec().createChunk(dataSegment) - ); } // Find segments to merge @@ -161,7 +152,7 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper log.info("[%s] Found %d segments to merge %s", dataSource, segments.size(), segmentNames); try { - mergerClient.runRequest(dataSource, segments); + indexingServiceClient.mergeSegments(segments); } catch (Exception e) { log.error( diff --git a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java b/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java deleted file mode 100644 index 07a0f8dba6e..00000000000 --- a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java +++ /dev/null @@ -1,78 +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.master; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merge.ClientAppendQuery; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.response.HttpResponseHandler; -import com.netflix.curator.x.discovery.ServiceProvider; - - -import java.net.URL; -import java.util.List; - -public class HttpMergerClient implements MergerClient -{ - private final HttpClient client; - private final HttpResponseHandler responseHandler; - private final ObjectMapper jsonMapper; - private final ServiceProvider serviceProvider; - - public HttpMergerClient( - HttpClient client, - HttpResponseHandler responseHandler, - ObjectMapper jsonMapper, - ServiceProvider serviceProvider - ) - { - this.client = client; - this.responseHandler = responseHandler; - this.jsonMapper = jsonMapper; - this.serviceProvider = serviceProvider; - } - - public void runRequest(String dataSource, List segments) - { - try { - byte[] dataToSend = jsonMapper.writeValueAsBytes( - new ClientAppendQuery(dataSource, segments) - ); - - client.post( - new URL( - String.format( - "http://%s:%s/mmx/merger/v1/merge", - serviceProvider.getInstance().getAddress(), - serviceProvider.getInstance().getPort() - ) - ) - ) - .setContent("application/json", dataToSend) - .go(responseHandler) - .get(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } -} diff --git a/server/src/main/java/com/metamx/druid/master/MergerClient.java b/server/src/main/java/com/metamx/druid/master/MergerWhitelist.java similarity index 57% rename from server/src/main/java/com/metamx/druid/master/MergerClient.java rename to server/src/main/java/com/metamx/druid/master/MergerWhitelist.java index c2556ccaea0..bd55a0cf057 100644 --- a/server/src/main/java/com/metamx/druid/master/MergerClient.java +++ b/server/src/main/java/com/metamx/druid/master/MergerWhitelist.java @@ -19,13 +19,36 @@ package com.metamx.druid.master; -import com.metamx.druid.client.DataSegment; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.collect.Sets; import java.util.List; +import java.util.Set; /** */ -public interface MergerClient +public class MergerWhitelist { - public void runRequest(String dataSource, List segments); + public static final String CONFIG_KEY = "merger.whitelist"; + + private final Set dataSources; + + @JsonCreator + public MergerWhitelist(Set dataSources) + { + this.dataSources = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + this.dataSources.addAll(dataSources); + } + + @JsonValue + public Set getDataSources() + { + return dataSources; + } + + public boolean contains(String val) + { + return dataSources.contains(val); + } } diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java index 8e19ed5d330..1ad1f96d163 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java @@ -23,12 +23,14 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.indexing.IndexingServiceClient; import junit.framework.Assert; import org.joda.time.Interval; import org.junit.Test; import java.util.Collection; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; public class DruidMasterSegmentMergerTest { @@ -367,11 +369,7 @@ public class DruidMasterSegmentMergerTest DataSegment.builder().dataSource("foo").interval(new Interval("2012-01-06/P1D")).version("2").size(80).build() ); - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of(segments.get(4), segments.get(5)) - ), merge(segments) - ); + Assert.assertEquals(ImmutableList.of(ImmutableList.of(segments.get(4), segments.get(5))), merge(segments)); } /** @@ -380,16 +378,17 @@ public class DruidMasterSegmentMergerTest private static List> merge(final Collection segments) { final List> retVal = Lists.newArrayList(); - final MergerClient mergerClient = new MergerClient() + final IndexingServiceClient indexingServiceClient = new IndexingServiceClient(null, null, null) { @Override - public void runRequest(String dataSource, List segmentsToMerge) + public void mergeSegments(List segmentsToMerge) { retVal.add(segmentsToMerge); } }; - final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(mergerClient); + final AtomicReference whitelistRef = new AtomicReference(null); + final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef); final DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder() .withAvailableSegments(ImmutableSet.copyOf(segments)) .withMergeBytesLimit(mergeBytesLimit) diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java index 895659586ec..584ae31de47 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -152,8 +152,6 @@ public class DruidMasterTest new NoopServiceEmitter(), scheduledExecutorFactory, loadManagementPeons, - null, - null, null ); } From f70f71243d71a85c204714c6bec4f6426ddcfa48 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 7 Mar 2013 17:43:37 -0600 Subject: [PATCH 17/17] 1) Stop polling in ConfigManager when stop is called 2) Remove WorkSetupManager in favor of just using ConfigManager --- .../metamx/druid/config/ConfigManager.java | 55 ++++++++++------- .../merger/coordinator/RemoteTaskRunner.java | 11 ++-- .../http/IndexerCoordinatorNode.java | 49 ++++++--------- .../http/IndexerCoordinatorResource.java | 23 +++++--- .../http/IndexerCoordinatorServletModule.java | 10 ++-- .../scaling/EC2AutoScalingStrategy.java | 10 ++-- .../SimpleResourceManagementStrategy.java | 12 ++-- .../coordinator/setup/WorkerSetupData.java | 2 + .../coordinator/setup/WorkerSetupManager.java | 59 ------------------- .../coordinator/RemoteTaskRunnerTest.java | 17 +----- .../scaling/EC2AutoScalingStrategyTest.java | 12 ++-- .../SimpleResourceManagementStrategyTest.java | 16 ++--- pom.xml | 2 +- 13 files changed, 106 insertions(+), 172 deletions(-) delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManager.java b/common/src/main/java/com/metamx/druid/config/ConfigManager.java index 3073c13e9fb..1ecfd24482c 100644 --- a/common/src/main/java/com/metamx/druid/config/ConfigManager.java +++ b/common/src/main/java/com/metamx/druid/config/ConfigManager.java @@ -1,6 +1,5 @@ package com.metamx.druid.config; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.metamx.common.concurrent.ScheduledExecutors; @@ -40,6 +39,8 @@ public class ConfigManager private final ConcurrentMap watchedConfigs; private final String selectStatement; + private volatile ConfigManager.PollingCallable poller; + public ConfigManager(IDBI dbi, ConfigManagerConfig config) { this.dbi = dbi; @@ -58,19 +59,8 @@ public class ConfigManager return; } - ScheduledExecutors.scheduleWithFixedDelay( - exec, - new Duration(0), - config.getPollDuration(), - new Runnable() - { - @Override - public void run() - { - poll(); - } - } - ); + poller = new PollingCallable(); + ScheduledExecutors.scheduleWithFixedDelay(exec, new Duration(0), config.getPollDuration(), poller); started = true; } @@ -84,6 +74,9 @@ public class ConfigManager return; } + poller.stop(); + poller = null; + started = false; } } @@ -119,8 +112,7 @@ public class ConfigManager { if (!started) { watchedConfigs.put(key, new ConfigHolder(null, serde)); - } - else { + } else { try { // Multiple of these callables can be submitted at the same time, but the callables themselves // are executed serially, so double check that it hasn't already been populated. @@ -200,10 +192,12 @@ public class ConfigManager @Override public Void withHandle(Handle handle) throws Exception { - handle.createStatement("INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload") - .bind("name", key) - .bind("payload", newBytes) - .execute(); + handle.createStatement( + "INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload" + ) + .bind("name", key) + .bind("payload", newBytes) + .execute(); return null; } } @@ -256,4 +250,25 @@ public class ConfigManager return false; } } + + private class PollingCallable implements Callable + { + private volatile boolean stop = false; + + void stop() + { + stop = true; + } + + @Override + public ScheduledExecutors.Signal call() throws Exception + { + if (stop) { + return ScheduledExecutors.Signal.STOP; + } + + poll(); + return ScheduledExecutors.Signal.REPEAT; + } + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index e4a13136152..b1ed92087bc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -35,7 +35,7 @@ import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; @@ -57,6 +57,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; /** * The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes and manage retries in failure @@ -82,7 +83,7 @@ public class RemoteTaskRunner implements TaskRunner private final PathChildrenCache workerPathCache; private final ScheduledExecutorService scheduledExec; private final RetryPolicyFactory retryPolicyFactory; - private final WorkerSetupManager workerSetupManager; + private final AtomicReference workerSetupData; // all workers that exist in ZK private final Map zkWorkers = new ConcurrentHashMap(); @@ -104,7 +105,7 @@ public class RemoteTaskRunner implements TaskRunner PathChildrenCache workerPathCache, ScheduledExecutorService scheduledExec, RetryPolicyFactory retryPolicyFactory, - WorkerSetupManager workerSetupManager + AtomicReference workerSetupData ) { this.jsonMapper = jsonMapper; @@ -113,7 +114,7 @@ public class RemoteTaskRunner implements TaskRunner this.workerPathCache = workerPathCache; this.scheduledExec = scheduledExec; this.retryPolicyFactory = retryPolicyFactory; - this.workerSetupManager = workerSetupManager; + this.workerSetupData = workerSetupData; } @LifecycleStart @@ -548,7 +549,7 @@ public class RemoteTaskRunner implements TaskRunner return (!input.isAtCapacity() && input.getWorker() .getVersion() - .compareTo(workerSetupManager.getWorkerSetupData().getMinVersion()) >= 0); + .compareTo(workerSetupData.get().getMinVersion()) >= 0); } } ) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 2112e3b6e37..5452acac163 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -54,9 +54,9 @@ 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.loading.DataSegmentKiller; import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.TaskActionToolbox; @@ -89,7 +89,7 @@ import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerCo import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory; import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy; import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -124,6 +124,7 @@ import java.util.Properties; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -157,7 +158,6 @@ public class IndexerCoordinatorNode extends RegisteringNode private CuratorFramework curatorFramework = null; private ScheduledExecutorFactory scheduledExecutorFactory = null; private IndexerZkConfig indexerZkConfig; - private WorkerSetupManager workerSetupManager = null; private TaskRunnerFactory taskRunnerFactory = null; private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null; private TaskMasterLifecycle taskMasterLifecycle = null; @@ -228,12 +228,6 @@ public class IndexerCoordinatorNode extends RegisteringNode return this; } - public IndexerCoordinatorNode setWorkerSetupManager(WorkerSetupManager workerSetupManager) - { - this.workerSetupManager = workerSetupManager; - return this; - } - public IndexerCoordinatorNode setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory) { this.taskRunnerFactory = taskRunnerFactory; @@ -250,6 +244,10 @@ public class IndexerCoordinatorNode extends RegisteringNode { scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); + final ConfigManagerConfig managerConfig = configFactory.build(ConfigManagerConfig.class); + DbConnector.createConfigTable(dbi, managerConfig.getConfigTable()); + JacksonConfigManager configManager = new JacksonConfigManager(new ConfigManager(dbi, managerConfig), jsonMapper); + initializeEmitter(); initializeMonitors(); initializeDB(); @@ -266,9 +264,8 @@ public class IndexerCoordinatorNode extends RegisteringNode initializeJacksonSubtypes(); initializeCurator(); initializeIndexerZkConfig(); - initializeWorkerSetupManager(); - initializeTaskRunnerFactory(); - initializeResourceManagement(); + initializeTaskRunnerFactory(configManager); + initializeResourceManagement(configManager); initializeTaskMasterLifecycle(); initializeServer(); @@ -288,7 +285,7 @@ public class IndexerCoordinatorNode extends RegisteringNode emitter, taskMasterLifecycle, new TaskStorageQueryAdapter(taskStorage), - workerSetupManager + configManager ) ); @@ -565,20 +562,7 @@ public class IndexerCoordinatorNode extends RegisteringNode } } - public void initializeWorkerSetupManager() - { - if (workerSetupManager == null) { - final ConfigManagerConfig configManagerConfig = configFactory.build(ConfigManagerConfig.class); - final ConfigManager configManager = new ConfigManager(dbi, configManagerConfig); - lifecycle.addManagedInstance(configManager); - - DbConnector.createConfigTable(dbi, configManagerConfig.getConfigTable()); - workerSetupManager = new WorkerSetupManager(new JacksonConfigManager(configManager, jsonMapper)); - } - lifecycle.addManagedInstance(workerSetupManager); - } - - public void initializeTaskRunnerFactory() + private void initializeTaskRunnerFactory(final JacksonConfigManager configManager) { if (taskRunnerFactory == null) { if (config.getRunnerImpl().equals("remote")) { @@ -604,7 +588,7 @@ public class IndexerCoordinatorNode extends RegisteringNode new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true), retryScheduledExec, new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)), - workerSetupManager + configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class) ); return remoteTaskRunner; @@ -627,7 +611,7 @@ public class IndexerCoordinatorNode extends RegisteringNode } } - private void initializeResourceManagement() + private void initializeResourceManagement(final JacksonConfigManager configManager) { if (resourceManagementSchedulerFactory == null) { resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory() @@ -642,6 +626,9 @@ public class IndexerCoordinatorNode extends RegisteringNode .setNameFormat("ScalingExec--%d") .build() ); + final AtomicReference workerSetupData = configManager.watch( + WorkerSetupData.CONFIG_KEY, WorkerSetupData.class + ); AutoScalingStrategy strategy; if (config.getStrategyImpl().equalsIgnoreCase("ec2")) { @@ -654,7 +641,7 @@ public class IndexerCoordinatorNode extends RegisteringNode ) ), configFactory.build(EC2AutoScalingStrategyConfig.class), - workerSetupManager + workerSetupData ); } else if (config.getStrategyImpl().equalsIgnoreCase("noop")) { strategy = new NoopAutoScalingStrategy(); @@ -667,7 +654,7 @@ public class IndexerCoordinatorNode extends RegisteringNode new SimpleResourceManagementStrategy( strategy, configFactory.build(SimpleResourceManagmentConfig.class), - workerSetupManager + workerSetupData ), configFactory.build(ResourceManagementSchedulerConfig.class), scalingScheduledExec diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index d2d613c3ea3..b5afa1dceef 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -19,6 +19,8 @@ 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.Optional; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -26,6 +28,7 @@ import com.google.common.collect.Maps; 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.TaskStatus; import com.metamx.druid.merger.common.actions.TaskActionHolder; import com.metamx.druid.merger.common.task.Task; @@ -35,10 +38,7 @@ import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; import com.metamx.druid.merger.coordinator.scaling.ScalingStats; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.core.type.TypeReference; import javax.ws.rs.Consumes; import javax.ws.rs.GET; @@ -49,6 +49,7 @@ import javax.ws.rs.Produces; import javax.ws.rs.core.Response; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -61,16 +62,18 @@ public class IndexerCoordinatorResource private final ServiceEmitter emitter; private final TaskMasterLifecycle taskMasterLifecycle; private final TaskStorageQueryAdapter taskStorageQueryAdapter; - private final WorkerSetupManager workerSetupManager; + private final JacksonConfigManager configManager; private final ObjectMapper jsonMapper; + private AtomicReference workerSetupDataRef = null; + @Inject public IndexerCoordinatorResource( IndexerCoordinatorConfig config, ServiceEmitter emitter, TaskMasterLifecycle taskMasterLifecycle, TaskStorageQueryAdapter taskStorageQueryAdapter, - WorkerSetupManager workerSetupManager, + JacksonConfigManager configManager, ObjectMapper jsonMapper ) throws Exception { @@ -78,7 +81,7 @@ public class IndexerCoordinatorResource this.emitter = emitter; this.taskMasterLifecycle = taskMasterLifecycle; this.taskStorageQueryAdapter = taskStorageQueryAdapter; - this.workerSetupManager = workerSetupManager; + this.configManager = configManager; this.jsonMapper = jsonMapper; } @@ -159,7 +162,11 @@ public class IndexerCoordinatorResource @Produces("application/json") public Response getWorkerSetupData() { - return Response.ok(workerSetupManager.getWorkerSetupData()).build(); + if (workerSetupDataRef == null) { + workerSetupDataRef = configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class); + } + + return Response.ok(workerSetupDataRef.get()).build(); } @POST @@ -169,7 +176,7 @@ public class IndexerCoordinatorResource final WorkerSetupData workerSetupData ) { - if (!workerSetupManager.setWorkerSetupData(workerSetupData)) { + if (!configManager.set(WorkerSetupData.CONFIG_KEY, workerSetupData)) { return Response.status(Response.Status.BAD_REQUEST).build(); } return Response.ok().build(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java index 2b26e08de6a..97f04dd6bad 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java @@ -22,10 +22,10 @@ package com.metamx.druid.merger.coordinator.http; 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.coordinator.TaskMasterLifecycle; import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; @@ -41,7 +41,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule private final ServiceEmitter emitter; private final TaskMasterLifecycle taskMasterLifecycle; private final TaskStorageQueryAdapter taskStorageQueryAdapter; - private final WorkerSetupManager workerSetupManager; + private final JacksonConfigManager configManager; public IndexerCoordinatorServletModule( ObjectMapper jsonMapper, @@ -49,7 +49,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule ServiceEmitter emitter, TaskMasterLifecycle taskMasterLifecycle, TaskStorageQueryAdapter taskStorageQueryAdapter, - WorkerSetupManager workerSetupManager + JacksonConfigManager configManager ) { this.jsonMapper = jsonMapper; @@ -57,7 +57,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule this.emitter = emitter; this.taskMasterLifecycle = taskMasterLifecycle; this.taskStorageQueryAdapter = taskStorageQueryAdapter; - this.workerSetupManager = workerSetupManager; + this.configManager = configManager; } @Override @@ -69,7 +69,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule bind(ServiceEmitter.class).toInstance(emitter); bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle); bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter); - bind(WorkerSetupManager.class).toInstance(workerSetupManager); + bind(JacksonConfigManager.class).toInstance(configManager); serve("/*").with(GuiceContainer.class); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index 47ef22152dd..f9ecbb06f70 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -34,13 +34,13 @@ import com.google.common.collect.Lists; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.setup.EC2NodeData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.EmittingLogger; import org.apache.commons.codec.binary.Base64; import javax.annotation.Nullable; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -51,26 +51,26 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy private final ObjectMapper jsonMapper; private final AmazonEC2Client amazonEC2Client; private final EC2AutoScalingStrategyConfig config; - private final WorkerSetupManager workerSetupManager; + private final AtomicReference workerSetupDataRef; public EC2AutoScalingStrategy( ObjectMapper jsonMapper, AmazonEC2Client amazonEC2Client, EC2AutoScalingStrategyConfig config, - WorkerSetupManager workerSetupManager + AtomicReference workerSetupDataRef ) { this.jsonMapper = jsonMapper; this.amazonEC2Client = amazonEC2Client; this.config = config; - this.workerSetupManager = workerSetupManager; + this.workerSetupDataRef = workerSetupDataRef; } @Override public AutoScalingData provision() { try { - WorkerSetupData setupData = workerSetupManager.getWorkerSetupData(); + WorkerSetupData setupData = workerSetupDataRef.get(); EC2NodeData workerConfig = setupData.getNodeData(); RunInstancesResult result = amazonEC2Client.runInstances( diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java index 05c2f2f0b26..082870c83c8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -27,16 +27,16 @@ import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; import com.metamx.druid.merger.coordinator.ZkWorker; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; import org.joda.time.Duration; import java.util.Collection; -import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -46,7 +46,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat private final AutoScalingStrategy autoScalingStrategy; private final SimpleResourceManagmentConfig config; - private final WorkerSetupManager workerSetupManager; + private final AtomicReference workerSetupdDataRef; private final ScalingStats scalingStats; private final ConcurrentSkipListSet currentlyProvisioning = new ConcurrentSkipListSet(); @@ -58,12 +58,12 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat public SimpleResourceManagementStrategy( AutoScalingStrategy autoScalingStrategy, SimpleResourceManagmentConfig config, - WorkerSetupManager workerSetupManager + AtomicReference workerSetupdDataRef ) { this.autoScalingStrategy = autoScalingStrategy; this.config = config; - this.workerSetupManager = workerSetupManager; + this.workerSetupdDataRef = workerSetupdDataRef; this.scalingStats = new ScalingStats(config.getNumEventsToTrack()); } @@ -151,7 +151,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat boolean nothingTerminating = currentlyTerminating.isEmpty(); if (nothingTerminating) { - final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers(); + final int minNumWorkers = workerSetupdDataRef.get().getMinNumWorkers(); if (zkWorkers.size() <= minNumWorkers) { log.info("Only [%d <= %d] nodes in the cluster, not terminating anything.", zkWorkers.size(), minNumWorkers); return false; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java index fada73cb40e..18cd85e6962 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java @@ -31,6 +31,8 @@ import java.util.List; */ public class WorkerSetupData { + public static final String CONFIG_KEY = "worker.setup"; + private final String minVersion; private final int minNumWorkers; private final EC2NodeData nodeData; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java deleted file mode 100644 index b9ce066c327..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.merger.coordinator.setup; - -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.druid.config.JacksonConfigManager; - -import java.util.concurrent.atomic.AtomicReference; - -/** - */ -public class WorkerSetupManager -{ - private static final String WORKER_SETUP_KEY = "worker.setup"; - - private final JacksonConfigManager configManager; - - private volatile AtomicReference workerSetupData = new AtomicReference(null); - - public WorkerSetupManager( - JacksonConfigManager configManager - ) - { - this.configManager = configManager; - } - - @LifecycleStart - public void start() - { - workerSetupData = configManager.watch(WORKER_SETUP_KEY, WorkerSetupData.class); - } - - public WorkerSetupData getWorkerSetupData() - { - return workerSetupData.get(); - } - - public boolean setWorkerSetupData(final WorkerSetupData value) - { - return configManager.set(WORKER_SETUP_KEY, value); - } -} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index 18ce14a4555..d88ac044aed 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -11,14 +11,12 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.merger.TestTask; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; @@ -43,6 +41,7 @@ import org.junit.Test; import java.io.File; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; import static junit.framework.Assert.fail; @@ -61,7 +60,6 @@ public class RemoteTaskRunnerTest private PathChildrenCache pathChildrenCache; private RemoteTaskRunner remoteTaskRunner; private TaskMonitor taskMonitor; - private WorkerSetupManager workerSetupManager; private ScheduledExecutorService scheduledExec; @@ -317,17 +315,6 @@ public class RemoteTaskRunnerTest private void makeRemoteTaskRunner() throws Exception { scheduledExec = EasyMock.createMock(ScheduledExecutorService.class); - workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); - - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn( - new WorkerSetupData( - "0", - 0, - null, - null - ) - ).atLeastOnce(); - EasyMock.replay(workerSetupManager); remoteTaskRunner = new RemoteTaskRunner( jsonMapper, @@ -336,7 +323,7 @@ public class RemoteTaskRunnerTest pathChildrenCache, scheduledExec, new RetryPolicyFactory(new TestRetryPolicyConfig()), - workerSetupManager + new AtomicReference(new WorkerSetupData("0", 0, null, null)) ); // Create a single worker and wait for things for be ready diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java index c3aa8378b07..cd569cb77e8 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -33,7 +33,6 @@ import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.setup.EC2NodeData; import com.metamx.druid.merger.coordinator.setup.GalaxyUserData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -42,6 +41,7 @@ import org.junit.Test; import java.util.Arrays; import java.util.Date; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -57,7 +57,7 @@ public class EC2AutoScalingStrategyTest private Reservation reservation; private Instance instance; private EC2AutoScalingStrategy strategy; - private WorkerSetupManager workerSetupManager; + private AtomicReference workerSetupData; @Before public void setUp() throws Exception @@ -66,7 +66,7 @@ public class EC2AutoScalingStrategyTest runInstancesResult = EasyMock.createMock(RunInstancesResult.class); describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class); reservation = EasyMock.createMock(Reservation.class); - workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); + workerSetupData = new AtomicReference(null); instance = new Instance() .withInstanceId(INSTANCE_ID) @@ -85,7 +85,7 @@ public class EC2AutoScalingStrategyTest return "8080"; } }, - workerSetupManager + workerSetupData ); } @@ -96,13 +96,12 @@ public class EC2AutoScalingStrategyTest EasyMock.verify(runInstancesResult); EasyMock.verify(describeInstancesResult); EasyMock.verify(reservation); - EasyMock.verify(workerSetupManager); } @Test public void testScale() { - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn( + workerSetupData.set( new WorkerSetupData( "0", 0, @@ -110,7 +109,6 @@ public class EC2AutoScalingStrategyTest new GalaxyUserData("env", "version", "type") ) ); - EasyMock.replay(workerSetupManager); EasyMock.expect(amazonEC2Client.runInstances(EasyMock.anyObject(RunInstancesRequest.class))).andReturn( runInstancesResult diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java index 742525d38a7..2052ae014bb 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -29,7 +29,6 @@ import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; import com.metamx.druid.merger.coordinator.ZkWorker; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.Worker; import junit.framework.Assert; import org.easymock.EasyMock; @@ -42,21 +41,22 @@ import org.junit.Test; import java.util.Arrays; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; /** */ public class SimpleResourceManagementStrategyTest { private AutoScalingStrategy autoScalingStrategy; - private WorkerSetupManager workerSetupManager; private Task testTask; private SimpleResourceManagementStrategy simpleResourceManagementStrategy; + private AtomicReference workerSetupData; @Before public void setUp() throws Exception { - workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); autoScalingStrategy = EasyMock.createMock(AutoScalingStrategy.class); + workerSetupData = new AtomicReference(null); testTask = new TestTask( "task1", @@ -105,7 +105,7 @@ public class SimpleResourceManagementStrategyTest return new Duration(0); } }, - workerSetupManager + workerSetupData ); } @@ -187,8 +187,7 @@ public class SimpleResourceManagementStrategyTest @Test public void testDoSuccessfulTerminate() throws Exception { - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); - EasyMock.replay(workerSetupManager); + workerSetupData.set(new WorkerSetupData("0", 0, null, null)); EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList()); @@ -212,15 +211,13 @@ public class SimpleResourceManagementStrategyTest simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE ); - EasyMock.verify(workerSetupManager); EasyMock.verify(autoScalingStrategy); } @Test public void testSomethingTerminating() throws Exception { - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); - EasyMock.replay(workerSetupManager); + workerSetupData.set(new WorkerSetupData("0", 0, null, null)); EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList("ip")).times(2); @@ -259,7 +256,6 @@ public class SimpleResourceManagementStrategyTest simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE ); - EasyMock.verify(workerSetupManager); EasyMock.verify(autoScalingStrategy); } diff --git a/pom.xml b/pom.xml index 078927877c6..27e639e7c18 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ UTF-8 - 0.20.1-SNAPSHOT + 0.21.0